IMPALA-14115: Calcite planner: Added top-n analytic PlanNode optimization.

Impala has an optimization for analytic expressions that have a rank filter on
top of the analytic expression. It can add a top-n plan node to reduce the amount
of rows examined. This is tested in tpcds query 67.

The optimization logic relies on an unassigned rank conjunct within the analyzer
while creating the analytic plan node.

A slight reorganization of the code was needed to implement this optimization.
The SlotRefs for the AnalyticInfo needed to be created a little earlier from
where it was done in the previous commit.

A small fix was made to normalize binary predicates. A non-normalized binary
predicate prevents the optimization from being used.

A call to the checkAndApplyLimitPushdown is needed for some of the optimizations
to kick in.

A new AllProjectInfo internal class was created to hold the relationships
between the Calcite RexNode objects and the Impala Analytic expressions.

Also, IMPALA-14158 is fixed by this commit. The nullsFirst value was
incorrect when the syntax was explicit in the query.

A new Calcite planner test was added in the junit tests to ensure the
optimization kicks in. The new test file is in the
PlannerTest/calcite/limit-pushdown-analytic-calcite.test file. This is a copy
of the limit-pushdown-analytic.test file in its parent directory but with some
modified results. Most of the differences are trivial, but IMPALA-14469 has been
filed to deal with one optimization that did not get fixed, which is when
the order by clause has a constant expression.

Change-Id: Ie6fa6781db56771b13b0cf49bd236f776016bf8d
Reviewed-on: http://gerrit.cloudera.org:8080/23317
Tested-by: Impala Public Jenkins <impala-public-jenkins@cloudera.com>
Reviewed-by: Aman Sinha <amsinha@cloudera.com>
This commit is contained in:
Steve Carlin
2025-06-02 10:28:54 -07:00
parent 98f993da43
commit cde4bc016c
14 changed files with 4695 additions and 899 deletions

View File

@@ -233,6 +233,8 @@ public class SortInfo {
List<Expr> tupleIsNullPreds = new ArrayList<>();
TreeNode.collect(resultExprs, Predicates.instanceOf(TupleIsNullPredicate.class),
tupleIsNullPreds);
TreeNode.collect(sortExprs_, Predicates.instanceOf(TupleIsNullPredicate.class),
tupleIsNullPreds);
Expr.removeDuplicates(tupleIsNullPreds);
addMaterializedExprs(tupleIsNullPreds, analyzer);

View File

@@ -119,7 +119,24 @@ public class AnalyticPlanner {
Expr.listToSql(analyticConjs, ToSqlOptions.SHOW_IMPLICIT_CASTS));
}
List<PartitionLimit> perPartitionLimits = inferPartitionLimits(analyticConjs);
List<PartitionLimit> perPartitionLimits =
inferPartitionLimits(analyzer_, analyticConjs);
PlanNode newRoot = createSingleNodePlan(root, groupingExprs, inputPartitionExprs,
tupleIsNullPreds, perPartitionLimits);
List<Expr> substAnalyticConjs =
Expr.substituteList(analyticConjs, newRoot.getOutputSmap(), analyzer_, false);
overrideSelectivityPushedLimits(analyticConjs, perPartitionLimits,
substAnalyticConjs);
return newRoot.addConjunctsToNode(ctx_, analyzer_, tids, substAnalyticConjs);
}
public PlanNode createSingleNodePlan(PlanNode root,
List<Expr> groupingExprs, List<Expr> inputPartitionExprs,
List<TupleIsNullPredicate> tupleIsNullPreds,
List<PartitionLimit> perPartitionLimits) throws ImpalaException {
List<WindowGroup> windowGroups = collectWindowGroups();
for (int i = 0; i < windowGroups.size(); ++i) {
windowGroups.get(i).init(analyzer_, "wg-" + i);
@@ -151,12 +168,7 @@ public class AnalyticPlanner {
firstSortGroup ? tupleIsNullPreds : emptyPreds);
}
}
List<Expr> substAnalyticConjs =
Expr.substituteList(analyticConjs, root.getOutputSmap(), analyzer_, false);
overrideSelectivityPushedLimits(analyticConjs, perPartitionLimits,
substAnalyticConjs);
return root.addConjunctsToNode(ctx_, analyzer_, tids, substAnalyticConjs);
return root;
}
/**
@@ -187,7 +199,7 @@ public class AnalyticPlanner {
private void overrideSelectivityPushedLimits(List<Expr> analyticConjs,
List<PartitionLimit> perPartitionLimits, List<Expr> substAnalyticConjs) {
for (PartitionLimit limit : perPartitionLimits) {
if (limit.pushed && limit.isLessThan) {
if (limit.shouldOverrideSelectivity()) {
int idx = analyticConjs.indexOf(limit.conjunct);
if (idx >= 0) {
substAnalyticConjs.set(idx,
@@ -882,7 +894,7 @@ public class AnalyticPlanner {
return partitionGroups;
}
private static class PartitionLimit {
public static class PartitionLimit {
public PartitionLimit(Expr conjunct, AnalyticExpr analyticExpr, long limit,
boolean includeTies, boolean isLessThan) {
this.conjunct = conjunct;
@@ -919,14 +931,19 @@ public class AnalyticPlanner {
public void markPushed() {
this.pushed = true;
}
public boolean shouldOverrideSelectivity() {
return pushed && isLessThan && conjunct != null;
}
}
/**
* Extract per-partition limits from 'conjuncts'.
*/
private List<PartitionLimit> inferPartitionLimits(List<Expr> conjuncts) {
public static List<PartitionLimit> inferPartitionLimits(Analyzer analyzer,
List<Expr> conjuncts) {
List<PartitionLimit> result = new ArrayList<>();
if (analyzer_.getQueryOptions().analytic_rank_pushdown_threshold <= 0) return result;
if (analyzer.getQueryOptions().analytic_rank_pushdown_threshold <= 0) return result;
for (Expr conj : conjuncts) {
if (!(Expr.IS_BINARY_PREDICATE.apply(conj))) continue;
BinaryPredicate pred = (BinaryPredicate) conj;

View File

@@ -368,7 +368,7 @@ public class SingleNodePlanner implements SingleNodePlannerIntf {
* For certain qualifying conditions, we can push a limit from the top level
* sort down to the sort associated with an AnalyticEval node.
*/
private static void checkAndApplyLimitPushdown(PlanNode root, SortInfo sortInfo,
public static void checkAndApplyLimitPushdown(PlanNode root, SortInfo sortInfo,
long limit, Analyzer analyzer, PlannerContext planCtx) {
LimitPushdownInfo pushdownLimit = null;
AnalyticEvalNode analyticNode = null;

View File

@@ -950,7 +950,7 @@ public class PlannerTestBase extends FrontendTestBase {
protected void runPlannerTestFile(String testFile, String dbName, TQueryOptions options,
Set<PlannerTestOption> testOptions) {
String fileName = testDir_.resolve(testFile + ".test").toString();
String fileName = getTestDir().resolve(testFile + ".test").toString();
if (options == null) {
options = defaultQueryOptions();
} else {
@@ -1037,4 +1037,8 @@ public class PlannerTestBase extends FrontendTestBase {
statsParser.parseFile();
return statsParser.getDbStatsMap();
}
protected java.nio.file.Path getTestDir() {
return testDir_;
}
}

View File

@@ -20,6 +20,7 @@ package org.apache.impala.calcite.coercenodes;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Lists;
import org.apache.calcite.rel.RelNode;
import org.apache.calcite.rex.RexBuilder;
import org.apache.calcite.rex.RexCall;
@@ -105,6 +106,12 @@ public class CoerceOperandShuttle extends RexShuttle {
// recursively call all embedded RexCalls first
RexCall castedOperandsCall = (RexCall) super.visitCall(call);
// For parquet statistics predicates to be used, the input ref needs to be
// on the left side of a comparison operator and any extraneous casts need
// to be removed.
castedOperandsCall = (RexCall) normalizeCompareOperator(castedOperandsCall);
// need to 'flatten' before putting it back into a filter or else some
// tpcds queries will fail in the junit tests because of an assert statement
// in the Filter constructor.
@@ -232,6 +239,23 @@ public class CoerceOperandShuttle extends RexShuttle {
return retType;
}
private RexNode normalizeCompareOperator(RexCall call) {
if (!SqlKind.BINARY_COMPARISON.contains(call.getKind())) {
return call;
}
RexNode leftOperand = call.getOperands().get(0);
RexNode rightOperand = call.getOperands().get(1);
if (RexUtil.containsInputRef(leftOperand) ||
!RexUtil.containsInputRef(rightOperand)) {
return call;
}
return rexBuilder.makeCall(call.getType(), call.getOperator().reverse(),
Lists.newArrayList(rightOperand, leftOperand));
}
private boolean isCastingNeeded(RexCall rexCall) {
if (NO_CAST_OPERATORS.contains(rexCall.getOperator().getKind())) {
return false;

View File

@@ -18,6 +18,7 @@
package org.apache.impala.calcite.rel.node;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
import org.apache.calcite.plan.RelOptCluster;
import org.apache.calcite.plan.RelTraitSet;
@@ -69,18 +70,24 @@ import org.apache.impala.calcite.functions.RexLiteralConverter;
import org.apache.impala.calcite.rel.util.CreateExprVisitor;
import org.apache.impala.calcite.rel.util.ExprConjunctsConverter;
import org.apache.impala.calcite.type.ImpalaTypeConverter;
import org.apache.impala.calcite.util.SimplifiedAnalyzer;
import org.apache.impala.catalog.Function;
import org.apache.impala.catalog.Type;
import org.apache.impala.common.ImpalaException;
import org.apache.impala.planner.AnalyticPlanner;
import org.apache.impala.planner.AnalyticPlanner.PartitionLimit;
import org.apache.impala.planner.PlannerContext;
import org.apache.impala.planner.PlanNode;
import org.apache.impala.planner.PlanNodeId;
import org.apache.impala.planner.SelectNode;
import java.math.BigDecimal;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedHashMap;
import java.util.LinkedHashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
@@ -91,7 +98,8 @@ import org.slf4j.LoggerFactory;
/**
*
* ImpalaAnalyticRel. Calcite RelNode which maps to an AnalyticEval node and
* which creates all the necessary Plan nodes associated with it.
*/
public class ImpalaAnalyticRel extends Project
implements ImpalaPlanRel {
@@ -115,7 +123,7 @@ public class ImpalaAnalyticRel extends Project
}
/**
* Convert the Project RelNode with analytic exprs into a Impala Plan Nodes.
* Convert the Project RelNode with analytic exprs into Impala Plan Nodes.
*/
@Override
public NodeWithExprs getPlanNode(ParentPlanRelContext context) throws ImpalaException {
@@ -123,27 +131,33 @@ public class ImpalaAnalyticRel extends Project
List<RexNode> projects = getProjects();
NodeWithExprs inputNodeWithExprs = getChildPlanNode(context, projects);
ImpalaPlanRel inputRel = (ImpalaPlanRel) getInput(0);
SimplifiedAnalyzer simplifiedAnalyzer =
(SimplifiedAnalyzer) context.ctx_.getRootAnalyzer();
// Get Info about all the Projects in the Calcite Project RelNode.
AllProjectInfo allProjectInfo = createAllProjectInfo(getProjects(), context.ctx_,
inputRel, inputNodeWithExprs.outputExprs_);
// There should be one element in the projectInfos_ array for each project in
// the original Project RelNode
Preconditions.checkState(getProjects().size() == allProjectInfo.projectInfos_.size());
// retrieve list of all analytic expressions
List<RexOver> overExprs = gatherRexOver(projects);
// get the GroupedAnalyticExpr objects. A GroupedAnalyticExpr object will
// contain a unique analytic expr and all the RexOver objects which are
// equivalent to the unique analytic expr.
List<GroupedAnalyticExpr> groupAnalyticExprs = getGroupedAnalyticExprs(
overExprs, context.ctx_, inputRel, inputNodeWithExprs.outputExprs_);
List<AnalyticExpr> analyticExprs = new ArrayList<>();
for (GroupedAnalyticExpr g : groupAnalyticExprs) {
analyticExprs.add(g.analyticExpr);
}
// Create Impala Analytic planner objects.
AnalyticInfo analyticInfo =
AnalyticInfo.create(analyticExprs, context.ctx_.getRootAnalyzer());
AnalyticInfo.create(allProjectInfo.analyticExprs_, simplifiedAnalyzer);
AnalyticPlanner analyticPlanner =
new AnalyticPlanner(analyticInfo, context.ctx_.getRootAnalyzer(), context.ctx_);
new AnalyticPlanner(analyticInfo, simplifiedAnalyzer, context.ctx_);
// Create a SlotRef for each analytic expression created by the AnalyticInfo.
Map<AnalyticExpr, SlotRef> analyticExprSlotRefMap =
createAnalyticExprSlotRefMap(allProjectInfo.analyticExprs_, analyticInfo);
// Retrieve the "perPartitionLimits" used for a potential top-N optimization
List<PartitionLimit> perPartitionLimits = getPerPartitionLimits(
context.filterCondition_, allProjectInfo, analyticExprSlotRefMap,
simplifiedAnalyzer);
// Pass in all the expressions from the input node wrapped with a
// TupleIsNullPredicate.
// TODO: IMPALA-12961. strip out null exprs which can exist when the tablescan does
// not output all of its columns.
List<Expr> nonNullExprs = inputNodeWithExprs.outputExprs_.stream()
@@ -152,26 +166,33 @@ public class ImpalaAnalyticRel extends Project
TupleIsNullPredicate.getUniqueBoundTupleIsNullPredicates(nonNullExprs,
inputNodeWithExprs.planNode_.getTupleIds());
// Create the plan node
// One difference with the original Impala planner. The original Impala planner
// calls a different outer AnalyticPlanner.createSingleNodePlan which also
// returns the Select node on top of the Analytic Node. The Calcite planner plan
// node here will be of type AnalyticEvalNode. If a SelectNode is needed because
// there is a filter condition, it will be created
PlanNode planNode = analyticPlanner.createSingleNodePlan(
inputNodeWithExprs.planNode_, Collections.emptyList(), new ArrayList<>(),
tupleIsNullPreds);
tupleIsNullPreds, perPartitionLimits);
// Get a mapping of all expressions to its corresponding Impala Expr object. The
// non-analytic expressions will have a RexInputRef type RexNode, while the
// analytic expressions will have the RexOver type RexNode.
// Get a mapping of all project columns to its corresponding Impala Expr object. The
// non-analytic expressions will have a RexInputRef type RexNode which are simply
// pass-through projects. The analytic expressions in the project will be of type
// RexOver.
Map<RexNode, Expr> mapping = createRexNodeExprMapping(inputRel, planNode, projects,
inputNodeWithExprs.outputExprs_, groupAnalyticExprs, context.ctx_, analyticInfo);
inputNodeWithExprs.outputExprs_, allProjectInfo.projectInfos_, context.ctx_,
analyticExprSlotRefMap);
List<Expr> outputExprs =
getOutputExprs(mapping, projects, context.ctx_.getRootAnalyzer());
List<Expr> outputExprs = getOutputExprs(mapping, projects, simplifiedAnalyzer);
NodeWithExprs retNode =
new NodeWithExprs(planNode, outputExprs, getRowType().getFieldNames());
// If there is a filter condition, a SelectNode will get added on top
// of the retNode.
return NodeCreationUtils.wrapInSelectNodeIfNeeded(context, retNode,
getCluster().getRexBuilder());
RexBuilder rexBuilder = getCluster().getRexBuilder();
return context.filterCondition_ != null
? createSelectNode(context, retNode, rexBuilder, perPartitionLimits)
: retNode;
}
private NodeWithExprs getChildPlanNode(ParentPlanRelContext context,
@@ -184,17 +205,30 @@ public class ImpalaAnalyticRel extends Project
return relInput.getPlanNode(builder.build());
}
/**
* Returns true if the given filter condition only contains input expressions
* referencing a rank expression.
*/
private boolean hasOnlyRankSlotRefInputExprs(RexNode filterCondition,
Set<Integer> rankProjects) {
Set<Integer> inputReferences =
getInputReferences(ImmutableList.of(filterCondition));
return inputReferences.size() == 1 &&
rankProjects.containsAll(inputReferences);
}
/**
* Generates the AnalyticExpr object from the RexOver and the input plan node
* expressions.
*/
private AnalyticExpr getAnalyticExpr(RexOver rexOver, PlannerContext ctx,
private AnalyticExpr createAnalyticExpr(RexBuilder rexBuilder,
RexOver rexOver, PlannerContext ctx,
ImpalaPlanRel inputRel, List<Expr> inputExprs) throws ImpalaException {
final RexWindow rexWindow = rexOver.getWindow();
// First parameter is the function call
Function fn = getFunction(rexOver);
Type impalaRetType = ImpalaTypeConverter.createImpalaType(rexOver.getType());
CreateExprVisitor visitor = new CreateExprVisitor(getCluster().getRexBuilder(),
CreateExprVisitor visitor = new CreateExprVisitor(rexBuilder,
inputExprs, ctx.getRootAnalyzer());
List<Expr> operands = CreateExprVisitor.getExprs(visitor, rexOver.operands);
@@ -216,9 +250,15 @@ public class ImpalaAnalyticRel extends Project
if (rexWindow.orderKeys != null) {
for (RexFieldCollation ok : rexWindow.orderKeys) {
Expr orderByExpr = CreateExprVisitor.getExpr(visitor, ok.left);
// Logic here:
// If ascending, nulls first is true only when it explicitly contains nulls first.
//
// If descending, nulls first is true if it explicitly contains nulls first
// or it is blank (does not contain nulls last).
boolean nullsFirst = ok.getDirection() == RelFieldCollation.Direction.ASCENDING
? ok.right.contains(SqlKind.NULLS_FIRST)
: !ok.right.contains(SqlKind.NULLS_FIRST);
: ok.right.contains(SqlKind.NULLS_FIRST)
|| !ok.right.contains(SqlKind.NULLS_LAST);
OrderByElement orderByElement = new OrderByElement(orderByExpr,
ok.getDirection() == RelFieldCollation.Direction.ASCENDING,
nullsFirst);
@@ -281,13 +321,30 @@ public class ImpalaAnalyticRel extends Project
}
}
/**
* Generate a SlotRef for each AnalyticExpr and place in a hash map.
*/
Map<AnalyticExpr, SlotRef> createAnalyticExprSlotRefMap(List<AnalyticExpr> exprs,
AnalyticInfo analyticInfo) {
Preconditions.checkState(
exprs.size() == analyticInfo.getOutputTupleDesc().getSlots().size());
Map<AnalyticExpr, SlotRef> result = new HashMap<>();
for (int i = 0; i < exprs.size(); ++i) {
AnalyticExpr expr = exprs.get(i);
SlotDescriptor slotDesc =
analyticInfo.getOutputTupleDesc().getSlots().get(i);
SlotRef newSlotRef = new SlotRef(slotDesc);
result.put(expr, newSlotRef);
}
return result;
}
private List<Expr> getOutputExprs(Map<RexNode, Expr> mapping,
List<RexNode> projects, Analyzer analyzer) throws ImpalaException {
AnalyticRexVisitor visitor = new AnalyticRexVisitor(mapping,
getCluster().getRexBuilder(), analyzer);
Map<Integer, Expr> projectExprs = new LinkedHashMap<>();
List<Expr> outputExprs = new ArrayList<>();
// Walk through all the projects and grab the already created Expr object that exists
// in the "mapping" variable.
@@ -325,49 +382,17 @@ public class ImpalaAnalyticRel extends Project
}
/**
* Get the analytic Expr objects from the RexOver objects in the Project.
* Impala does not allow duplicate analytic expressions. So if two different
* RexOvers create the same AnalyticExpr object, they get grouped together in
* one GroupedAnalyticExpr object.
* Create a mapping from the Projects for this RelNode to the output Expr
* objects that exist within the newly created Plan Node.
*/
private List<GroupedAnalyticExpr> getGroupedAnalyticExprs(List<RexOver> overExprs,
PlannerContext ctx, ImpalaPlanRel inputRel,
List<Expr> inputExprs) throws ImpalaException {
List<AnalyticExpr> analyticExprs = new ArrayList<>();
List<List<RexOver>> overExprsList = new ArrayList<>();
for (RexOver over : overExprs) {
AnalyticExpr analyticExpr =
getAnalyticExpr(over, ctx, inputRel, inputExprs);
// check if we've seen this analytic expression before.
int index = analyticExprs.indexOf(analyticExpr);
if (index == -1) {
analyticExprs.add(analyticExpr);
overExprsList.add(Lists.newArrayList(over));
} else {
overExprsList.get(index).add(over);
}
}
// The total number of unique analytic expressions should match the number
// of RexOver expression lists created.
Preconditions.checkState(analyticExprs.size() == overExprsList.size());
// Create the GroupedAnalyticExprs from the corresponding lists
List<GroupedAnalyticExpr> groupedAnalyticExprs = new ArrayList<>();
for (int i = 0; i < analyticExprs.size(); ++i) {
groupedAnalyticExprs.add(
new GroupedAnalyticExpr(analyticExprs.get(i), overExprsList.get(i)));
}
return groupedAnalyticExprs;
}
private Map<RexNode, Expr> createRexNodeExprMapping(ImpalaPlanRel inputRel,
PlanNode planNode, List<RexNode> projects, List<Expr> inputExprs,
List<GroupedAnalyticExpr> groupAnalyticExprs,
PlannerContext ctx, AnalyticInfo analyticInfo) {
List<ProjectInfo> projectInfos,
PlannerContext ctx, Map<AnalyticExpr, SlotRef> analyticExprSlotRefMap) {
Map<RexNode, Expr> mapping = new LinkedHashMap<>();
// Gather mappings from nodes created by analytic planner
ExprSubstitutionMap outputExprMap = planNode.getOutputSmap();
// We populate the outputs from the expressions
Map<RexNode, Expr> mapping = new LinkedHashMap<>();
// All the input references are going to get marked as a RexInputRef and
// will be mapped to its given expression's position number
@@ -378,14 +403,11 @@ public class ImpalaAnalyticRel extends Project
mapping.put(RexInputRef.of(pos, getInput(0).getRowType().getFieldList()), e);
}
// Create a new SlotRef for analytic expressions.
for (int i = 0; i < groupAnalyticExprs.size(); i++) {
GroupedAnalyticExpr g = groupAnalyticExprs.get(i);
SlotDescriptor slotDesc =
analyticInfo.getOutputTupleDesc().getSlots().get(i);
SlotRef logicalOutputSlot = new SlotRef(slotDesc);
for (RexOver over : g.overExprsList) {
mapping.put(over, outputExprMap.get(logicalOutputSlot));
// Retrieve the new SlotRef for analytic expressions.
for (ProjectInfo projectInfo : projectInfos) {
for (int i = 0; i < projectInfo.rexOvers_.size(); ++i) {
Expr analyticExpr = analyticExprSlotRefMap.get(projectInfo.analyticExprs_.get(i));
mapping.put(projectInfo.rexOvers_.get(i), outputExprMap.get(analyticExpr));
}
}
@@ -398,13 +420,206 @@ public class ImpalaAnalyticRel extends Project
return shuttle.inputPosReferenced;
}
private static class GroupedAnalyticExpr {
public final AnalyticExpr analyticExpr;
public final List<RexOver> overExprsList;
/**
* Generate the AllProjectInfo structure (see internal class for details)
*/
private AllProjectInfo createAllProjectInfo(List<RexNode> projects, PlannerContext ctx,
ImpalaPlanRel inputRel, List<Expr> inputExprs) throws ImpalaException {
public GroupedAnalyticExpr(AnalyticExpr analyticExpr, List<RexOver> overExprsList) {
this.analyticExpr = analyticExpr;
this.overExprsList = overExprsList;
RexBuilder rexBuilder = getCluster().getRexBuilder();
List<ProjectInfo> projectInfos = new ArrayList<>();
// AnalyticExpr must be unique within Impala's AnalyticPlanner. There can be two
// RexOver projects that generate the same AnalyticExpr.
LinkedHashSet<AnalyticExpr> uniqueAnalyticExprs = new LinkedHashSet<>();
Set<Integer> rankProjects = new HashSet<>();
// For loop generates a new ProjectInfo object for every Project RexNode column.
for (int projectNum = 0; projectNum < projects.size(); ++projectNum) {
RexNode project = projects.get(projectNum);
List<RexOver> overs = gatherRexOver(ImmutableList.of(project));
List<AnalyticExpr> projectAnalyticExprs = new ArrayList<>();
for (RexOver over : overs) {
AnalyticExpr analyticExpr =
createAnalyticExpr(rexBuilder, over, ctx, inputRel, inputExprs);
if (!uniqueAnalyticExprs.contains(analyticExpr)) {
uniqueAnalyticExprs.add(analyticExpr);
}
projectAnalyticExprs.add(analyticExpr);
}
projectInfos.add(new ProjectInfo(project, overs, projectAnalyticExprs));
if (project.getKind() == SqlKind.RANK ||
project.getKind().equals(SqlKind.ROW_NUMBER)) {
rankProjects.add(projectNum);
}
}
return new AllProjectInfo(projectInfos, new ArrayList<>(uniqueAnalyticExprs),
rankProjects);
}
/**
* Retrieve the list of Partition Limits. This is specifically used for the top-n
* optimization feature.
*/
private List<PartitionLimit> getPerPartitionLimits(RexNode filterCondition,
AllProjectInfo allProjectInfo, Map<AnalyticExpr, SlotRef> analyticExprSlotRefMap,
Analyzer analyzer) throws ImpalaException {
// top-n optimization feature only works when there is a filter.
if (filterCondition == null) {
return ImmutableList.of();
}
List<Expr> rankConjuncts = new ArrayList<>();
// Retrieve a list which contains an element for every project but
// only contains a non-null Expr if the Project is of type RANK
// or ROW_NUMBER
List<Expr> rankSlotRefExprs = allProjectInfo.projectInfos_.stream()
.map(e -> e.rankExpr_)
.map(e -> analyticExprSlotRefMap.get(e))
.collect(Collectors.toList());
Preconditions.checkState(getProjects().size() == rankSlotRefExprs.size());
// Break up the filter condition into its individual and elements
List<RexNode> andConjuncts = ExprConjunctsConverter.getAndConjuncts(filterCondition);
for (RexNode andConjunct : andConjuncts) {
// We only want the filter conjuncts that contain a reference to the Rank project
// e.g. 'rnk <= 5' where rnk is an alias to an analytic expr containing rank.
if (hasOnlyRankSlotRefInputExprs(andConjunct, allProjectInfo.rankProjects_)) {
// convert the RexNode filter condition into an Impala Expr that can be
// understood by the Analytic Planner
ExprConjunctsConverter converter = new ExprConjunctsConverter(
andConjunct, rankSlotRefExprs, getCluster().getRexBuilder(),
analyzer);
rankConjuncts.addAll(converter.getImpalaConjuncts());
}
}
return AnalyticPlanner.inferPartitionLimits(analyzer, rankConjuncts);
}
/**
* Create a Select Node when there is a filter on top of this Project.
*/
public NodeWithExprs createSelectNode(ParentPlanRelContext context,
NodeWithExprs nodeWithExprs, RexBuilder rexBuilder,
List<PartitionLimit> perPartitionLimits) throws ImpalaException {
Analyzer analyzer = context.ctx_.getRootAnalyzer();
PlanNodeId nodeId = context.ctx_.getNextNodeId();
ExprConjunctsConverter converter = new ExprConjunctsConverter(
context.filterCondition_, nodeWithExprs.outputExprs_, rexBuilder, analyzer);
List<Expr> filterConjuncts = converter.getImpalaConjuncts();
// For top-n optimization: If criteria is met, the filter conjunct gets its
// selectivity overridden.
filterConjuncts = overrideSelectivity(filterConjuncts, perPartitionLimits,
nodeWithExprs.planNode_, analyzer);
SelectNode selectNode =
SelectNode.createFromCalcite(nodeId, nodeWithExprs.planNode_, filterConjuncts);
selectNode.init(analyzer);
return new NodeWithExprs(selectNode, nodeWithExprs);
}
/**
* Override the selectivity on certain filtered conjuncts if needed.
* The perPartitionLimits contain conjuncts that are used in top-n optimization
* if it has been applied. This method returns the adjusted filtered conjuncts
* with the potentially modified selectivity.
*/
private List<Expr> overrideSelectivity(List<Expr> filterConjuncts,
List<PartitionLimit> perPartitionLimits, PlanNode planNode, Analyzer analyzer) {
ExprSubstitutionMap outputExprMap = planNode.getOutputSmap();
// iterate through the partitions and retrieve the Expr conjuncts that were used
// in the top-n optimization. The ExprSubstitutionMap is used since the
// PartitionLimit contains the input Expr before the Expr node was created
// and gets substituted with the Expr that is output from the AnalyticEval node.
// is output from the AnalyticEval node.
List<Expr> overrideSelectivityInputConjuncts = perPartitionLimits.stream()
.filter(p -> p.shouldOverrideSelectivity())
.map(p -> p.conjunct)
.map(p -> p.substitute(outputExprMap, analyzer, false))
.collect(Collectors.toList());
// If there are no top-n optimizations applied, return the original unmodified
// list.
if (overrideSelectivityInputConjuncts.size() == 0) {
return filterConjuncts;
}
// return the modified conjuncts (which are modified in the
// overrideSelectivityIfNecessary method).
return filterConjuncts.stream()
.map(f -> overrideSelectivityIfNecessary(f, overrideSelectivityInputConjuncts))
.collect(Collectors.toList());
}
/**
* Given a filter conjunct and a list of conjuncts to override: If the filter conjunct
* is contained in the list, return a modified conjunct. Otherwise return the original
* conjunct.
*/
public Expr overrideSelectivityIfNecessary(Expr filterConj,
List<Expr> overrideConjuncts) {
for (Expr overrideConjunct : overrideConjuncts) {
if (overrideConjunct.equals(filterConj)) {
return filterConj.cloneAndOverrideSelectivity(1.0);
}
}
return filterConj;
}
public boolean isOverrideSelectivityConj(PartitionLimit limit) {
return limit.isPushed() && limit.isLessThan && limit.conjunct != null;
}
/**
* AllProjectInfo contains all the information about the Project RelNode, relating
* newly created Impala objects to each Project RexNode.
*/
private static class AllProjectInfo {
// Array of Project RexNodes and its related information. There is one ProjectInfo
// member for each RexNode in the Project RelNode.
private final List<ProjectInfo> projectInfos_;
// List of unique Impala AnalyticExpr objects to be passed into AnalyticInfo.
private final List<AnalyticExpr> analyticExprs_;
// list of all rank/row number expressions needed for TopN optimization.
private final Set<Integer> rankProjects_;
public AllProjectInfo(List<ProjectInfo> projectInfos,
List<AnalyticExpr> analyticExprs, Set<Integer> rankProjects) {
projectInfos_ = projectInfos;
analyticExprs_ = analyticExprs;
rankProjects_ = rankProjects;
}
}
/**
* ProjectInfo contains all Impala information related to a single Project RexNode
* column in the Project RelNode
*/
private static class ProjectInfo {
// The Project RexNode.
public final RexNode project_;
// All the RexOver objects within the RexNode.
public final List<RexOver> rexOvers_;
// All the Impala Analytic expressions, 1:1 with the RexOver objects.
public final List<AnalyticExpr> analyticExprs_;
// Special member variable that is only non-null if the rexOvers_.get(0) is
// specifically a SqlKind.RANK node or SqlKind.ROW_NUMBER.
public final AnalyticExpr rankExpr_;
public ProjectInfo(RexNode project, List<RexOver> rexOvers,
List<AnalyticExpr> analyticExprs) {
project_ = project;
Preconditions.checkState(rexOvers.size() == analyticExprs.size());
rexOvers_ = ImmutableList.copyOf(rexOvers);
analyticExprs_ = ImmutableList.copyOf(analyticExprs);
SqlKind kind = project.getKind();
rankExpr_ = kind == SqlKind.RANK || kind == SqlKind.ROW_NUMBER
? analyticExprs.get(0)
: null;
}
}

View File

@@ -38,6 +38,7 @@ import org.apache.impala.common.AnalysisException;
import org.apache.impala.common.ImpalaException;
import org.apache.impala.planner.PlannerContext;
import org.apache.impala.planner.PlanNode;
import org.apache.impala.planner.SingleNodePlanner;
import org.apache.impala.planner.SortNode;
import java.math.BigDecimal;
@@ -50,10 +51,6 @@ import org.slf4j.LoggerFactory;
/**
* ImpalaSortRel
*
* IMPALA-13172: Optimizations for tryConvertToTopN needed. This should probably
* be made rule based and done in the optimization phase, but keeping track
* here because the Impala code was in the SortNode.
*/
public class ImpalaSortRel extends Sort
implements ImpalaPlanRel {
@@ -105,6 +102,8 @@ public class ImpalaSortRel extends Sort
// a sort or top-n node..just set the limit on the child
if (fieldCollations.size() == 0) {
validateUnorderedLimit(context.filterCondition_, limit_, offset_);
SingleNodePlanner.checkAndApplyLimitPushdown(inputNodeWithExprs.planNode_, null,
limit_, context.ctx_.getRootAnalyzer(), context.ctx_);
// Mutating an existing object here. Either we should pass in a context into
// all PlanNodes containing the limit so the PlanNode constructor can set the
// limit, or leave the code here to mutate.
@@ -137,6 +136,11 @@ public class ImpalaSortRel extends Sort
sortInfo.materializeRequiredSlots(context.ctx_.getRootAnalyzer(),
new ExprSubstitutionMap());
if (limit_ != -1 && offset_ == 0) {
SingleNodePlanner.checkAndApplyLimitPushdown(inputNodeWithExprs.planNode_, sortInfo,
limit_, context.ctx_.getRootAnalyzer(), context.ctx_);
}
// Call a specific implementation of createSortNode(). In the future, we could
// try to leverage Impala's SingleNodePlanner.createSortNode()
SortNode sortNode = createSortNode(context.ctx_,
@@ -218,7 +222,7 @@ public class ImpalaSortRel extends Sort
SortInfo sortInfo, long limit, long offset, boolean hasLimit
) throws ImpalaException {
if (!hasLimit) {
if (!hasLimit || planCtx.getQueryOptions().isDisable_outermost_topn()) {
return SortNode.createTotalSortNode(planCtx.getNextNodeId(), root, sortInfo,
offset);
}

View File

@@ -79,7 +79,7 @@ public class ExprConjunctsConverter {
* will have all 3 clauses on the first level), so we do not
* need to recursively search for clauses.
*/
private static List<RexNode> getAndConjuncts(RexNode conjunct) {
public static List<RexNode> getAndConjuncts(RexNode conjunct) {
if (conjunct == null) {
return ImmutableList.of();
}

View File

@@ -0,0 +1,54 @@
// 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.impala.planner;
import java.nio.file.Paths;
import org.apache.impala.thrift.TQueryOptions;
import org.junit.Test;
public class CalcitePlannerTest extends PlannerTestBase {
private final static java.nio.file.Path calciteTestDir_ =
Paths.get("functional-planner", "queries", "PlannerTest", "calcite");
/**
* Test limit pushdown into analytic sort in isolation.
*/
@Test
public void testLimitPushdownAnalytic() {
// The partitioned top-n optimization interacts with limit pushdown. We run the
// basic limit pushdown tests with it disabled.
TQueryOptions options = defaultQueryOptions();
options.setAnalytic_rank_pushdown_threshold(0);
options.setUse_calcite_planner(true);
runPlannerTestFile("limit-pushdown-analytic-calcite", options);
}
@Test
public void testAnalyticRankPushdown() {
TQueryOptions options = defaultQueryOptions();
options.setUse_calcite_planner(true);
runPlannerTestFile("analytic-rank-pushdown-calcite", options);
}
@Override
protected java.nio.file.Path getTestDir() {
return calciteTestDir_;
}
}

View File

@@ -48,7 +48,7 @@ Max Per-Host Resource Reservation: Memory=263.62MB Threads=1
Per-Host Resource Estimates: Memory=55.19GB
F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
| Per-Instance Resources: mem-estimate=55.19GB mem-reservation=263.62MB thread-reservation=1 runtime-filters-memory=3.00MB
| max-parallelism=1 segment-costs=[176670599328, 73050686125, 54567099909, 45827804421, 1000]
| max-parallelism=1 segment-costs=[176670599328, 73050686125, 233936768446, 1261, 1000]
PLAN-ROOT SINK
| output exprs: CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 9 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 10 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 11 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 12 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 13 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 14 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 15 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 16 THEN NULL END, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.item.i_class WHEN 9 THEN tpcds_partitioned_parquet_snap.item.i_class WHEN 10 THEN tpcds_partitioned_parquet_snap.item.i_class WHEN 11 THEN tpcds_partitioned_parquet_snap.item.i_class WHEN 12 THEN tpcds_partitioned_parquet_snap.item.i_class WHEN 13 THEN tpcds_partitioned_parquet_snap.item.i_class WHEN 14 THEN tpcds_partitioned_parquet_snap.item.i_class WHEN 15 THEN NULL WHEN 16 THEN NULL END, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN 9 THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN 10 THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN 11 THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN 12 THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN 13 THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN 14 THEN NULL WHEN 15 THEN NULL WHEN 16 THEN NULL END, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.item.i_product_name WHEN 9 THEN tpcds_partitioned_parquet_snap.item.i_product_name WHEN 10 THEN tpcds_partitioned_parquet_snap.item.i_product_name WHEN 11 THEN tpcds_partitioned_parquet_snap.item.i_product_name WHEN 12 THEN tpcds_partitioned_parquet_snap.item.i_product_name WHEN 13 THEN NULL WHEN 14 THEN NULL WHEN 15 THEN NULL WHEN 16 THEN NULL END, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.date_dim.d_year WHEN 9 THEN tpcds_partitioned_parquet_snap.date_dim.d_year WHEN 10 THEN tpcds_partitioned_parquet_snap.date_dim.d_year WHEN 11 THEN tpcds_partitioned_parquet_snap.date_dim.d_year WHEN 12 THEN NULL WHEN 13 THEN NULL WHEN 14 THEN NULL WHEN 15 THEN NULL WHEN 16 THEN NULL END, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.date_dim.d_qoy WHEN 9 THEN tpcds_partitioned_parquet_snap.date_dim.d_qoy WHEN 10 THEN tpcds_partitioned_parquet_snap.date_dim.d_qoy WHEN 11 THEN NULL WHEN 12 THEN NULL WHEN 13 THEN NULL WHEN 14 THEN NULL WHEN 15 THEN NULL WHEN 16 THEN NULL END, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.date_dim.d_moy WHEN 9 THEN tpcds_partitioned_parquet_snap.date_dim.d_moy WHEN 10 THEN NULL WHEN 11 THEN NULL WHEN 12 THEN NULL WHEN 13 THEN NULL WHEN 14 THEN NULL WHEN 15 THEN NULL WHEN 16 THEN NULL END, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.store.s_store_id WHEN 9 THEN NULL WHEN 10 THEN NULL WHEN 11 THEN NULL WHEN 12 THEN NULL WHEN 13 THEN NULL WHEN 14 THEN NULL WHEN 15 THEN NULL WHEN 16 THEN NULL END, aggif(valid_tid(16,8,9,10,11,12,13,14,15) IN (8, 9, 10, 11, 12, 13, 14, 15, 16), CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 9 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 10 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 11 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 12 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 13 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 14 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 15 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 16 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) END), rank()
| mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0 cost=1000
@@ -56,13 +56,13 @@ PLAN-ROOT SINK
12:TOP-N [LIMIT=100]
| order by: CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 9 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 10 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 11 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 12 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 13 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 14 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 15 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 16 THEN NULL END ASC NULLS LAST, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.item.i_class WHEN 9 THEN tpcds_partitioned_parquet_snap.item.i_class WHEN 10 THEN tpcds_partitioned_parquet_snap.item.i_class WHEN 11 THEN tpcds_partitioned_parquet_snap.item.i_class WHEN 12 THEN tpcds_partitioned_parquet_snap.item.i_class WHEN 13 THEN tpcds_partitioned_parquet_snap.item.i_class WHEN 14 THEN tpcds_partitioned_parquet_snap.item.i_class WHEN 15 THEN NULL WHEN 16 THEN NULL END ASC NULLS LAST, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN 9 THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN 10 THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN 11 THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN 12 THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN 13 THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN 14 THEN NULL WHEN 15 THEN NULL WHEN 16 THEN NULL END ASC NULLS LAST, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.item.i_product_name WHEN 9 THEN tpcds_partitioned_parquet_snap.item.i_product_name WHEN 10 THEN tpcds_partitioned_parquet_snap.item.i_product_name WHEN 11 THEN tpcds_partitioned_parquet_snap.item.i_product_name WHEN 12 THEN tpcds_partitioned_parquet_snap.item.i_product_name WHEN 13 THEN NULL WHEN 14 THEN NULL WHEN 15 THEN NULL WHEN 16 THEN NULL END ASC NULLS LAST, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.date_dim.d_year WHEN 9 THEN tpcds_partitioned_parquet_snap.date_dim.d_year WHEN 10 THEN tpcds_partitioned_parquet_snap.date_dim.d_year WHEN 11 THEN tpcds_partitioned_parquet_snap.date_dim.d_year WHEN 12 THEN NULL WHEN 13 THEN NULL WHEN 14 THEN NULL WHEN 15 THEN NULL WHEN 16 THEN NULL END ASC NULLS LAST, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.date_dim.d_qoy WHEN 9 THEN tpcds_partitioned_parquet_snap.date_dim.d_qoy WHEN 10 THEN tpcds_partitioned_parquet_snap.date_dim.d_qoy WHEN 11 THEN NULL WHEN 12 THEN NULL WHEN 13 THEN NULL WHEN 14 THEN NULL WHEN 15 THEN NULL WHEN 16 THEN NULL END ASC NULLS LAST, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.date_dim.d_moy WHEN 9 THEN tpcds_partitioned_parquet_snap.date_dim.d_moy WHEN 10 THEN NULL WHEN 11 THEN NULL WHEN 12 THEN NULL WHEN 13 THEN NULL WHEN 14 THEN NULL WHEN 15 THEN NULL WHEN 16 THEN NULL END ASC NULLS LAST, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.store.s_store_id WHEN 9 THEN NULL WHEN 10 THEN NULL WHEN 11 THEN NULL WHEN 12 THEN NULL WHEN 13 THEN NULL WHEN 14 THEN NULL WHEN 15 THEN NULL WHEN 16 THEN NULL END ASC NULLS LAST, aggif(valid_tid(16,8,9,10,11,12,13,14,15) IN (8, 9, 10, 11, 12, 13, 14, 15, 16), CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 9 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 10 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 11 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 12 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 13 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 14 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 15 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 16 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) END) ASC NULLS LAST, rank() ASC NULLS LAST
| mem-estimate=9.38KB mem-reservation=0B thread-reservation=0
| tuple-ids=24 row-size=96B cardinality=100 cost=21075768581
| tuple-ids=24 row-size=96B cardinality=100 cost=861
| in pipelines: 12(GETNEXT), 09(OPEN)
|
11:SELECT
| predicates: rank() <= CAST(100 AS BIGINT)
| mem-estimate=0B mem-reservation=0B thread-reservation=0
| tuple-ids=22,21 row-size=100B cardinality=1.24G cost=12376017920
| tuple-ids=22,21 row-size=100B cardinality=200 cost=200
| in pipelines: 09(GETNEXT)
|
10:ANALYTIC
@@ -71,13 +71,15 @@ PLAN-ROOT SINK
| order by: aggif(valid_tid(16,8,9,10,11,12,13,14,15) IN (8, 9, 10, 11, 12, 13, 14, 15, 16), CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 9 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 10 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 11 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 12 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 13 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 14 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 15 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 16 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) END) DESC
| window: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
| mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
| tuple-ids=22,21 row-size=100B cardinality=12.38G cost=12376017920
| tuple-ids=22,21 row-size=100B cardinality=200 cost=200
| in pipelines: 09(GETNEXT)
|
09:SORT
09:TOP-N
| order by: CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 9 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 10 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 11 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 12 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 13 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 14 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 15 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 16 THEN NULL END ASC NULLS LAST, aggif(valid_tid(16,8,9,10,11,12,13,14,15) IN (8, 9, 10, 11, 12, 13, 14, 15, 16), CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 9 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 10 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 11 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 12 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 13 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 14 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 15 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 16 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) END) DESC NULLS FIRST
| mem-estimate=50.00GB mem-reservation=12.00MB spill-buffer=2.00MB thread-reservation=0
| tuple-ids=22 row-size=92B cardinality=12.38G cost=54567099909
| limit with ties: 200
| source expr: rank() <= CAST(100 AS BIGINT)
| mem-estimate=17.97KB mem-reservation=0B thread-reservation=0
| tuple-ids=22 row-size=92B cardinality=200 cost=233936768446
| in pipelines: 09(GETNEXT), 08(OPEN)
|
08:AGGREGATE [FINALIZE]
@@ -188,36 +190,36 @@ PLAN-ROOT SINK
tuple-ids=0 row-size=20B cardinality=8.64G(filtered from 8.64G) cost=2488301335
in pipelines: 00(GETNEXT)
---- DISTRIBUTEDPLAN
Max Per-Host Resource Reservation: Memory=3.79GB Threads=25
Per-Host Resource Estimates: Memory=138.28GB
Max Per-Host Resource Reservation: Memory=4.70GB Threads=27
Per-Host Resource Estimates: Memory=104.39GB
F06:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
| Per-Instance Resources: mem-estimate=4.29MB mem-reservation=4.00MB thread-reservation=1
| Per-Instance Resources: mem-estimate=4.10MB mem-reservation=4.00MB thread-reservation=1
| max-parallelism=1 segment-costs=[1044] cpu-comparison-result=120 [max(1 (self) vs 120 (sum children))]
PLAN-ROOT SINK
| output exprs: CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 9 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 10 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 11 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 12 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 13 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 14 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 15 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 16 THEN NULL END, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.item.i_class WHEN 9 THEN tpcds_partitioned_parquet_snap.item.i_class WHEN 10 THEN tpcds_partitioned_parquet_snap.item.i_class WHEN 11 THEN tpcds_partitioned_parquet_snap.item.i_class WHEN 12 THEN tpcds_partitioned_parquet_snap.item.i_class WHEN 13 THEN tpcds_partitioned_parquet_snap.item.i_class WHEN 14 THEN tpcds_partitioned_parquet_snap.item.i_class WHEN 15 THEN NULL WHEN 16 THEN NULL END, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN 9 THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN 10 THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN 11 THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN 12 THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN 13 THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN 14 THEN NULL WHEN 15 THEN NULL WHEN 16 THEN NULL END, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.item.i_product_name WHEN 9 THEN tpcds_partitioned_parquet_snap.item.i_product_name WHEN 10 THEN tpcds_partitioned_parquet_snap.item.i_product_name WHEN 11 THEN tpcds_partitioned_parquet_snap.item.i_product_name WHEN 12 THEN tpcds_partitioned_parquet_snap.item.i_product_name WHEN 13 THEN NULL WHEN 14 THEN NULL WHEN 15 THEN NULL WHEN 16 THEN NULL END, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.date_dim.d_year WHEN 9 THEN tpcds_partitioned_parquet_snap.date_dim.d_year WHEN 10 THEN tpcds_partitioned_parquet_snap.date_dim.d_year WHEN 11 THEN tpcds_partitioned_parquet_snap.date_dim.d_year WHEN 12 THEN NULL WHEN 13 THEN NULL WHEN 14 THEN NULL WHEN 15 THEN NULL WHEN 16 THEN NULL END, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.date_dim.d_qoy WHEN 9 THEN tpcds_partitioned_parquet_snap.date_dim.d_qoy WHEN 10 THEN tpcds_partitioned_parquet_snap.date_dim.d_qoy WHEN 11 THEN NULL WHEN 12 THEN NULL WHEN 13 THEN NULL WHEN 14 THEN NULL WHEN 15 THEN NULL WHEN 16 THEN NULL END, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.date_dim.d_moy WHEN 9 THEN tpcds_partitioned_parquet_snap.date_dim.d_moy WHEN 10 THEN NULL WHEN 11 THEN NULL WHEN 12 THEN NULL WHEN 13 THEN NULL WHEN 14 THEN NULL WHEN 15 THEN NULL WHEN 16 THEN NULL END, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.store.s_store_id WHEN 9 THEN NULL WHEN 10 THEN NULL WHEN 11 THEN NULL WHEN 12 THEN NULL WHEN 13 THEN NULL WHEN 14 THEN NULL WHEN 15 THEN NULL WHEN 16 THEN NULL END, aggif(valid_tid(16,8,9,10,11,12,13,14,15) IN (8, 9, 10, 11, 12, 13, 14, 15, 16), CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 9 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 10 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 11 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 12 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 13 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 14 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 15 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 16 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) END), rank()
| mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0 cost=1000
|
19:MERGING-EXCHANGE [UNPARTITIONED]
20:MERGING-EXCHANGE [UNPARTITIONED]
| order by: CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 9 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 10 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 11 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 12 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 13 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 14 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 15 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 16 THEN NULL END ASC NULLS LAST, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.item.i_class WHEN 9 THEN tpcds_partitioned_parquet_snap.item.i_class WHEN 10 THEN tpcds_partitioned_parquet_snap.item.i_class WHEN 11 THEN tpcds_partitioned_parquet_snap.item.i_class WHEN 12 THEN tpcds_partitioned_parquet_snap.item.i_class WHEN 13 THEN tpcds_partitioned_parquet_snap.item.i_class WHEN 14 THEN tpcds_partitioned_parquet_snap.item.i_class WHEN 15 THEN NULL WHEN 16 THEN NULL END ASC NULLS LAST, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN 9 THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN 10 THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN 11 THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN 12 THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN 13 THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN 14 THEN NULL WHEN 15 THEN NULL WHEN 16 THEN NULL END ASC NULLS LAST, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.item.i_product_name WHEN 9 THEN tpcds_partitioned_parquet_snap.item.i_product_name WHEN 10 THEN tpcds_partitioned_parquet_snap.item.i_product_name WHEN 11 THEN tpcds_partitioned_parquet_snap.item.i_product_name WHEN 12 THEN tpcds_partitioned_parquet_snap.item.i_product_name WHEN 13 THEN NULL WHEN 14 THEN NULL WHEN 15 THEN NULL WHEN 16 THEN NULL END ASC NULLS LAST, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.date_dim.d_year WHEN 9 THEN tpcds_partitioned_parquet_snap.date_dim.d_year WHEN 10 THEN tpcds_partitioned_parquet_snap.date_dim.d_year WHEN 11 THEN tpcds_partitioned_parquet_snap.date_dim.d_year WHEN 12 THEN NULL WHEN 13 THEN NULL WHEN 14 THEN NULL WHEN 15 THEN NULL WHEN 16 THEN NULL END ASC NULLS LAST, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.date_dim.d_qoy WHEN 9 THEN tpcds_partitioned_parquet_snap.date_dim.d_qoy WHEN 10 THEN tpcds_partitioned_parquet_snap.date_dim.d_qoy WHEN 11 THEN NULL WHEN 12 THEN NULL WHEN 13 THEN NULL WHEN 14 THEN NULL WHEN 15 THEN NULL WHEN 16 THEN NULL END ASC NULLS LAST, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.date_dim.d_moy WHEN 9 THEN tpcds_partitioned_parquet_snap.date_dim.d_moy WHEN 10 THEN NULL WHEN 11 THEN NULL WHEN 12 THEN NULL WHEN 13 THEN NULL WHEN 14 THEN NULL WHEN 15 THEN NULL WHEN 16 THEN NULL END ASC NULLS LAST, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.store.s_store_id WHEN 9 THEN NULL WHEN 10 THEN NULL WHEN 11 THEN NULL WHEN 12 THEN NULL WHEN 13 THEN NULL WHEN 14 THEN NULL WHEN 15 THEN NULL WHEN 16 THEN NULL END ASC NULLS LAST, aggif(valid_tid(16,8,9,10,11,12,13,14,15) IN (8, 9, 10, 11, 12, 13, 14, 15, 16), CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 9 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 10 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 11 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 12 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 13 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 14 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 15 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 16 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) END) ASC NULLS LAST, rank() ASC NULLS LAST
| limit: 100
| mem-estimate=293.91KB mem-reservation=0B thread-reservation=0
| mem-estimate=98.59KB mem-reservation=0B thread-reservation=0
| tuple-ids=24 row-size=96B cardinality=100 cost=44
| in pipelines: 12(GETNEXT)
|
F05:PLAN FRAGMENT [HASH(CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 9 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 10 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 11 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 12 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 13 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 14 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 15 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 16 THEN NULL END)] hosts=10 instances=30 (adjusted from 120)
Per-Instance Resources: mem-estimate=16.68GB mem-reservation=16.00MB thread-reservation=1
max-parallelism=6120 segment-costs=[61179606532, 45827804421, 282] cpu-comparison-result=120 [max(30 (self) vs 120 (sum children))]
F05:PLAN FRAGMENT [HASH(CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 9 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 10 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 11 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 12 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 13 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 14 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 15 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 16 THEN NULL END)] hosts=10 instances=10 (adjusted from 120)
Per-Instance Resources: mem-estimate=5.30MB mem-reservation=4.00MB thread-reservation=1
max-parallelism=10 segment-costs=[967, 1261, 282] cpu-comparison-result=120 [max(10 (self) vs 120 (sum children))]
12:TOP-N [LIMIT=100]
| order by: CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 9 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 10 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 11 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 12 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 13 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 14 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 15 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 16 THEN NULL END ASC NULLS LAST, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.item.i_class WHEN 9 THEN tpcds_partitioned_parquet_snap.item.i_class WHEN 10 THEN tpcds_partitioned_parquet_snap.item.i_class WHEN 11 THEN tpcds_partitioned_parquet_snap.item.i_class WHEN 12 THEN tpcds_partitioned_parquet_snap.item.i_class WHEN 13 THEN tpcds_partitioned_parquet_snap.item.i_class WHEN 14 THEN tpcds_partitioned_parquet_snap.item.i_class WHEN 15 THEN NULL WHEN 16 THEN NULL END ASC NULLS LAST, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN 9 THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN 10 THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN 11 THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN 12 THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN 13 THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN 14 THEN NULL WHEN 15 THEN NULL WHEN 16 THEN NULL END ASC NULLS LAST, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.item.i_product_name WHEN 9 THEN tpcds_partitioned_parquet_snap.item.i_product_name WHEN 10 THEN tpcds_partitioned_parquet_snap.item.i_product_name WHEN 11 THEN tpcds_partitioned_parquet_snap.item.i_product_name WHEN 12 THEN tpcds_partitioned_parquet_snap.item.i_product_name WHEN 13 THEN NULL WHEN 14 THEN NULL WHEN 15 THEN NULL WHEN 16 THEN NULL END ASC NULLS LAST, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.date_dim.d_year WHEN 9 THEN tpcds_partitioned_parquet_snap.date_dim.d_year WHEN 10 THEN tpcds_partitioned_parquet_snap.date_dim.d_year WHEN 11 THEN tpcds_partitioned_parquet_snap.date_dim.d_year WHEN 12 THEN NULL WHEN 13 THEN NULL WHEN 14 THEN NULL WHEN 15 THEN NULL WHEN 16 THEN NULL END ASC NULLS LAST, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.date_dim.d_qoy WHEN 9 THEN tpcds_partitioned_parquet_snap.date_dim.d_qoy WHEN 10 THEN tpcds_partitioned_parquet_snap.date_dim.d_qoy WHEN 11 THEN NULL WHEN 12 THEN NULL WHEN 13 THEN NULL WHEN 14 THEN NULL WHEN 15 THEN NULL WHEN 16 THEN NULL END ASC NULLS LAST, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.date_dim.d_moy WHEN 9 THEN tpcds_partitioned_parquet_snap.date_dim.d_moy WHEN 10 THEN NULL WHEN 11 THEN NULL WHEN 12 THEN NULL WHEN 13 THEN NULL WHEN 14 THEN NULL WHEN 15 THEN NULL WHEN 16 THEN NULL END ASC NULLS LAST, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.store.s_store_id WHEN 9 THEN NULL WHEN 10 THEN NULL WHEN 11 THEN NULL WHEN 12 THEN NULL WHEN 13 THEN NULL WHEN 14 THEN NULL WHEN 15 THEN NULL WHEN 16 THEN NULL END ASC NULLS LAST, aggif(valid_tid(16,8,9,10,11,12,13,14,15) IN (8, 9, 10, 11, 12, 13, 14, 15, 16), CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 9 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 10 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 11 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 12 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 13 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 14 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 15 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 16 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) END) ASC NULLS LAST, rank() ASC NULLS LAST
| mem-estimate=9.38KB mem-reservation=0B thread-reservation=0
| tuple-ids=24 row-size=96B cardinality=100 cost=21075768581
| in pipelines: 12(GETNEXT), 09(OPEN)
| tuple-ids=24 row-size=96B cardinality=100 cost=861
| in pipelines: 12(GETNEXT), 19(OPEN)
|
11:SELECT
| predicates: rank() <= CAST(100 AS BIGINT)
| mem-estimate=0B mem-reservation=0B thread-reservation=0
| tuple-ids=22,21 row-size=100B cardinality=1.24G cost=12376017920
| in pipelines: 09(GETNEXT)
| tuple-ids=22,21 row-size=100B cardinality=200 cost=200
| in pipelines: 19(GETNEXT)
|
10:ANALYTIC
| functions: rank()
@@ -225,27 +227,36 @@ max-parallelism=6120 segment-costs=[61179606532, 45827804421, 282] cpu-compariso
| order by: aggif(valid_tid(16,8,9,10,11,12,13,14,15) IN (8, 9, 10, 11, 12, 13, 14, 15, 16), CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 9 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 10 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 11 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 12 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 13 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 14 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 15 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 16 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) END) DESC
| window: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
| mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
| tuple-ids=22,21 row-size=100B cardinality=12.38G cost=12376017920
| in pipelines: 09(GETNEXT)
| tuple-ids=22,21 row-size=100B cardinality=200 cost=200
| in pipelines: 19(GETNEXT)
|
09:SORT
19:TOP-N
| order by: CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 9 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 10 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 11 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 12 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 13 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 14 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 15 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 16 THEN NULL END ASC NULLS LAST, aggif(valid_tid(16,8,9,10,11,12,13,14,15) IN (8, 9, 10, 11, 12, 13, 14, 15, 16), CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 9 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 10 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 11 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 12 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 13 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 14 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 15 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 16 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) END) DESC NULLS FIRST
| mem-estimate=16.67GB mem-reservation=12.00MB spill-buffer=2.00MB thread-reservation=0
| tuple-ids=22 row-size=92B cardinality=12.38G cost=54567099909
| in pipelines: 09(GETNEXT), 08(OPEN)
| limit with ties: 200
| mem-estimate=17.97KB mem-reservation=0B thread-reservation=0
| tuple-ids=22 row-size=92B cardinality=200 cost=861
| in pipelines: 19(GETNEXT), 09(OPEN)
|
18:EXCHANGE [HASH(CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 9 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 10 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 11 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 12 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 13 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 14 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 15 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 16 THEN NULL END)]
| mem-estimate=12.81MB mem-reservation=0B thread-reservation=0
| tuple-ids=17 row-size=92B cardinality=12.38G cost=6612506623
| in pipelines: 08(GETNEXT)
| mem-estimate=1.28MB mem-reservation=0B thread-reservation=0
| tuple-ids=22 row-size=92B cardinality=200 cost=106
| in pipelines: 09(GETNEXT)
|
F04:PLAN FRAGMENT [HASH(CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_category) WHEN 9 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_category) WHEN 10 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_category) WHEN 11 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_category) WHEN 12 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_category) WHEN 13 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_category) WHEN 14 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_category) WHEN 15 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_category) WHEN 16 THEN murmur_hash(NULL) END,CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_class) WHEN 9 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_class) WHEN 10 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_class) WHEN 11 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_class) WHEN 12 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_class) WHEN 13 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_class) WHEN 14 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_class) WHEN 15 THEN murmur_hash(NULL) WHEN 16 THEN murmur_hash(NULL) END,CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_brand) WHEN 9 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_brand) WHEN 10 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_brand) WHEN 11 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_brand) WHEN 12 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_brand) WHEN 13 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_brand) WHEN 14 THEN murmur_hash(NULL) WHEN 15 THEN murmur_hash(NULL) WHEN 16 THEN murmur_hash(NULL) END,CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_product_name) WHEN 9 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_product_name) WHEN 10 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_product_name) WHEN 11 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_product_name) WHEN 12 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_product_name) WHEN 13 THEN murmur_hash(NULL) WHEN 14 THEN murmur_hash(NULL) WHEN 15 THEN murmur_hash(NULL) WHEN 16 THEN murmur_hash(NULL) END,CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN murmur_hash(tpcds_partitioned_parquet_snap.date_dim.d_year) WHEN 9 THEN murmur_hash(tpcds_partitioned_parquet_snap.date_dim.d_year) WHEN 10 THEN murmur_hash(tpcds_partitioned_parquet_snap.date_dim.d_year) WHEN 11 THEN murmur_hash(tpcds_partitioned_parquet_snap.date_dim.d_year) WHEN 12 THEN murmur_hash(NULL) WHEN 13 THEN murmur_hash(NULL) WHEN 14 THEN murmur_hash(NULL) WHEN 15 THEN murmur_hash(NULL) WHEN 16 THEN murmur_hash(NULL) END,CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN murmur_hash(tpcds_partitioned_parquet_snap.date_dim.d_qoy) WHEN 9 THEN murmur_hash(tpcds_partitioned_parquet_snap.date_dim.d_qoy) WHEN 10 THEN murmur_hash(tpcds_partitioned_parquet_snap.date_dim.d_qoy) WHEN 11 THEN murmur_hash(NULL) WHEN 12 THEN murmur_hash(NULL) WHEN 13 THEN murmur_hash(NULL) WHEN 14 THEN murmur_hash(NULL) WHEN 15 THEN murmur_hash(NULL) WHEN 16 THEN murmur_hash(NULL) END,CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN murmur_hash(tpcds_partitioned_parquet_snap.date_dim.d_moy) WHEN 9 THEN murmur_hash(tpcds_partitioned_parquet_snap.date_dim.d_moy) WHEN 10 THEN murmur_hash(NULL) WHEN 11 THEN murmur_hash(NULL) WHEN 12 THEN murmur_hash(NULL) WHEN 13 THEN murmur_hash(NULL) WHEN 14 THEN murmur_hash(NULL) WHEN 15 THEN murmur_hash(NULL) WHEN 16 THEN murmur_hash(NULL) END,CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN murmur_hash(tpcds_partitioned_parquet_snap.store.s_store_id) WHEN 9 THEN murmur_hash(NULL) WHEN 10 THEN murmur_hash(NULL) WHEN 11 THEN murmur_hash(NULL) WHEN 12 THEN murmur_hash(NULL) WHEN 13 THEN murmur_hash(NULL) WHEN 14 THEN murmur_hash(NULL) WHEN 15 THEN murmur_hash(NULL) WHEN 16 THEN murmur_hash(NULL) END)] hosts=10 instances=70 (adjusted from 120)
Per-Instance Resources: mem-estimate=7.14GB mem-reservation=244.75MB thread-reservation=1
max-parallelism=26630 segment-costs=[266227423359, 73050686125, 233936768446, 1288] cpu-comparison-result=120 [max(70 (self) vs 120 (sum children))]
09:TOP-N
| order by: CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 9 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 10 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 11 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 12 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 13 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 14 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 15 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 16 THEN NULL END ASC NULLS LAST, aggif(valid_tid(16,8,9,10,11,12,13,14,15) IN (8, 9, 10, 11, 12, 13, 14, 15, 16), CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 9 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 10 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 11 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 12 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 13 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 14 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 15 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 16 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) END) DESC NULLS FIRST
| limit with ties: 200
| source expr: rank() <= CAST(100 AS BIGINT)
| mem-estimate=17.97KB mem-reservation=0B thread-reservation=0
| tuple-ids=22 row-size=92B cardinality=200 cost=233936768446
| in pipelines: 09(GETNEXT), 08(OPEN)
|
F04:PLAN FRAGMENT [HASH(CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_category) WHEN 9 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_category) WHEN 10 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_category) WHEN 11 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_category) WHEN 12 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_category) WHEN 13 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_category) WHEN 14 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_category) WHEN 15 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_category) WHEN 16 THEN murmur_hash(NULL) END,CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_class) WHEN 9 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_class) WHEN 10 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_class) WHEN 11 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_class) WHEN 12 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_class) WHEN 13 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_class) WHEN 14 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_class) WHEN 15 THEN murmur_hash(NULL) WHEN 16 THEN murmur_hash(NULL) END,CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_brand) WHEN 9 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_brand) WHEN 10 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_brand) WHEN 11 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_brand) WHEN 12 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_brand) WHEN 13 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_brand) WHEN 14 THEN murmur_hash(NULL) WHEN 15 THEN murmur_hash(NULL) WHEN 16 THEN murmur_hash(NULL) END,CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_product_name) WHEN 9 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_product_name) WHEN 10 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_product_name) WHEN 11 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_product_name) WHEN 12 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_product_name) WHEN 13 THEN murmur_hash(NULL) WHEN 14 THEN murmur_hash(NULL) WHEN 15 THEN murmur_hash(NULL) WHEN 16 THEN murmur_hash(NULL) END,CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN murmur_hash(tpcds_partitioned_parquet_snap.date_dim.d_year) WHEN 9 THEN murmur_hash(tpcds_partitioned_parquet_snap.date_dim.d_year) WHEN 10 THEN murmur_hash(tpcds_partitioned_parquet_snap.date_dim.d_year) WHEN 11 THEN murmur_hash(tpcds_partitioned_parquet_snap.date_dim.d_year) WHEN 12 THEN murmur_hash(NULL) WHEN 13 THEN murmur_hash(NULL) WHEN 14 THEN murmur_hash(NULL) WHEN 15 THEN murmur_hash(NULL) WHEN 16 THEN murmur_hash(NULL) END,CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN murmur_hash(tpcds_partitioned_parquet_snap.date_dim.d_qoy) WHEN 9 THEN murmur_hash(tpcds_partitioned_parquet_snap.date_dim.d_qoy) WHEN 10 THEN murmur_hash(tpcds_partitioned_parquet_snap.date_dim.d_qoy) WHEN 11 THEN murmur_hash(NULL) WHEN 12 THEN murmur_hash(NULL) WHEN 13 THEN murmur_hash(NULL) WHEN 14 THEN murmur_hash(NULL) WHEN 15 THEN murmur_hash(NULL) WHEN 16 THEN murmur_hash(NULL) END,CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN murmur_hash(tpcds_partitioned_parquet_snap.date_dim.d_moy) WHEN 9 THEN murmur_hash(tpcds_partitioned_parquet_snap.date_dim.d_moy) WHEN 10 THEN murmur_hash(NULL) WHEN 11 THEN murmur_hash(NULL) WHEN 12 THEN murmur_hash(NULL) WHEN 13 THEN murmur_hash(NULL) WHEN 14 THEN murmur_hash(NULL) WHEN 15 THEN murmur_hash(NULL) WHEN 16 THEN murmur_hash(NULL) END,CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN murmur_hash(tpcds_partitioned_parquet_snap.store.s_store_id) WHEN 9 THEN murmur_hash(NULL) WHEN 10 THEN murmur_hash(NULL) WHEN 11 THEN murmur_hash(NULL) WHEN 12 THEN murmur_hash(NULL) WHEN 13 THEN murmur_hash(NULL) WHEN 14 THEN murmur_hash(NULL) WHEN 15 THEN murmur_hash(NULL) WHEN 16 THEN murmur_hash(NULL) END)] hosts=10 instances=30 (adjusted from 120)
Per-Instance Resources: mem-estimate=11.98GB mem-reservation=244.75MB thread-reservation=1
max-parallelism=26630 segment-costs=[266227423359, 73050686125, 79701558405] cpu-comparison-result=120 [max(30 (self) vs 120 (sum children))]
08:AGGREGATE [FINALIZE]
| output: aggif(valid_tid(16,8,9,10,11,12,13,14,15) IN (CAST(8 AS INT), CAST(9 AS INT), CAST(10 AS INT), CAST(11 AS INT), CAST(12 AS INT), CAST(13 AS INT), CAST(14 AS INT), CAST(15 AS INT), CAST(16 AS INT)), CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN CAST(8 AS INT) THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN CAST(9 AS INT) THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN CAST(10 AS INT) THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN CAST(11 AS INT) THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN CAST(12 AS INT) THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN CAST(13 AS INT) THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN CAST(14 AS INT) THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN CAST(15 AS INT) THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN CAST(16 AS INT) THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) END)
| group by: CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN CAST(8 AS INT) THEN tpcds_partitioned_parquet_snap.item.i_category WHEN CAST(9 AS INT) THEN tpcds_partitioned_parquet_snap.item.i_category WHEN CAST(10 AS INT) THEN tpcds_partitioned_parquet_snap.item.i_category WHEN CAST(11 AS INT) THEN tpcds_partitioned_parquet_snap.item.i_category WHEN CAST(12 AS INT) THEN tpcds_partitioned_parquet_snap.item.i_category WHEN CAST(13 AS INT) THEN tpcds_partitioned_parquet_snap.item.i_category WHEN CAST(14 AS INT) THEN tpcds_partitioned_parquet_snap.item.i_category WHEN CAST(15 AS INT) THEN tpcds_partitioned_parquet_snap.item.i_category WHEN CAST(16 AS INT) THEN NULL END, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN CAST(8 AS INT) THEN tpcds_partitioned_parquet_snap.item.i_class WHEN CAST(9 AS INT) THEN tpcds_partitioned_parquet_snap.item.i_class WHEN CAST(10 AS INT) THEN tpcds_partitioned_parquet_snap.item.i_class WHEN CAST(11 AS INT) THEN tpcds_partitioned_parquet_snap.item.i_class WHEN CAST(12 AS INT) THEN tpcds_partitioned_parquet_snap.item.i_class WHEN CAST(13 AS INT) THEN tpcds_partitioned_parquet_snap.item.i_class WHEN CAST(14 AS INT) THEN tpcds_partitioned_parquet_snap.item.i_class WHEN CAST(15 AS INT) THEN NULL WHEN CAST(16 AS INT) THEN NULL END, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN CAST(8 AS INT) THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN CAST(9 AS INT) THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN CAST(10 AS INT) THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN CAST(11 AS INT) THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN CAST(12 AS INT) THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN CAST(13 AS INT) THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN CAST(14 AS INT) THEN NULL WHEN CAST(15 AS INT) THEN NULL WHEN CAST(16 AS INT) THEN NULL END, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN CAST(8 AS INT) THEN tpcds_partitioned_parquet_snap.item.i_product_name WHEN CAST(9 AS INT) THEN tpcds_partitioned_parquet_snap.item.i_product_name WHEN CAST(10 AS INT) THEN tpcds_partitioned_parquet_snap.item.i_product_name WHEN CAST(11 AS INT) THEN tpcds_partitioned_parquet_snap.item.i_product_name WHEN CAST(12 AS INT) THEN tpcds_partitioned_parquet_snap.item.i_product_name WHEN CAST(13 AS INT) THEN NULL WHEN CAST(14 AS INT) THEN NULL WHEN CAST(15 AS INT) THEN NULL WHEN CAST(16 AS INT) THEN NULL END, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN CAST(8 AS INT) THEN tpcds_partitioned_parquet_snap.date_dim.d_year WHEN CAST(9 AS INT) THEN tpcds_partitioned_parquet_snap.date_dim.d_year WHEN CAST(10 AS INT) THEN tpcds_partitioned_parquet_snap.date_dim.d_year WHEN CAST(11 AS INT) THEN tpcds_partitioned_parquet_snap.date_dim.d_year WHEN CAST(12 AS INT) THEN NULL WHEN CAST(13 AS INT) THEN NULL WHEN CAST(14 AS INT) THEN NULL WHEN CAST(15 AS INT) THEN NULL WHEN CAST(16 AS INT) THEN NULL END, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN CAST(8 AS INT) THEN tpcds_partitioned_parquet_snap.date_dim.d_qoy WHEN CAST(9 AS INT) THEN tpcds_partitioned_parquet_snap.date_dim.d_qoy WHEN CAST(10 AS INT) THEN tpcds_partitioned_parquet_snap.date_dim.d_qoy WHEN CAST(11 AS INT) THEN NULL WHEN CAST(12 AS INT) THEN NULL WHEN CAST(13 AS INT) THEN NULL WHEN CAST(14 AS INT) THEN NULL WHEN CAST(15 AS INT) THEN NULL WHEN CAST(16 AS INT) THEN NULL END, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN CAST(8 AS INT) THEN tpcds_partitioned_parquet_snap.date_dim.d_moy WHEN CAST(9 AS INT) THEN tpcds_partitioned_parquet_snap.date_dim.d_moy WHEN CAST(10 AS INT) THEN NULL WHEN CAST(11 AS INT) THEN NULL WHEN CAST(12 AS INT) THEN NULL WHEN CAST(13 AS INT) THEN NULL WHEN CAST(14 AS INT) THEN NULL WHEN CAST(15 AS INT) THEN NULL WHEN CAST(16 AS INT) THEN NULL END, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN CAST(8 AS INT) THEN tpcds_partitioned_parquet_snap.store.s_store_id WHEN CAST(9 AS INT) THEN NULL WHEN CAST(10 AS INT) THEN NULL WHEN CAST(11 AS INT) THEN NULL WHEN CAST(12 AS INT) THEN NULL WHEN CAST(13 AS INT) THEN NULL WHEN CAST(14 AS INT) THEN NULL WHEN CAST(15 AS INT) THEN NULL WHEN CAST(16 AS INT) THEN NULL END, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN CAST(8 AS INT) THEN CAST(8 AS INT) WHEN CAST(9 AS INT) THEN CAST(9 AS INT) WHEN CAST(10 AS INT) THEN CAST(10 AS INT) WHEN CAST(11 AS INT) THEN CAST(11 AS INT) WHEN CAST(12 AS INT) THEN CAST(12 AS INT) WHEN CAST(13 AS INT) THEN CAST(13 AS INT) WHEN CAST(14 AS INT) THEN CAST(14 AS INT) WHEN CAST(15 AS INT) THEN CAST(15 AS INT) WHEN CAST(16 AS INT) THEN CAST(16 AS INT) END
| mem-estimate=669.83MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
| mem-estimate=578.50MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
| tuple-ids=17 row-size=92B cardinality=12.38G cost=73050686125
| in pipelines: 08(GETNEXT), 17(OPEN)
|
@@ -277,7 +288,7 @@ max-parallelism=26630 segment-costs=[266227423359, 73050686125, 79701558405] cpu
| Class 8
| output: sum:merge(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0))
| group by: NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL
| mem-estimate=11.33GB mem-reservation=210.75MB thread-reservation=0
| mem-estimate=6.57GB mem-reservation=210.75MB thread-reservation=0
| tuple-ids=8N,9N,10N,11N,12N,13N,14N,15N,16N row-size=1.09KB cardinality=12.38G cost=97683994945
| in pipelines: 17(GETNEXT), 00(OPEN)
|
@@ -288,7 +299,7 @@ max-parallelism=26630 segment-costs=[266227423359, 73050686125, 79701558405] cpu
|
F00:PLAN FRAGMENT [RANDOM] hosts=10 instances=120
Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB
Per-Instance Resources: mem-estimate=4.31GB mem-reservation=219.00MB thread-reservation=1
Per-Instance Resources: mem-estimate=4.49GB mem-reservation=219.00MB thread-reservation=1
max-parallelism=1824 segment-costs=[257851731293, 2328086935759] cpu-comparison-result=120 [max(120 (self) vs 36 (sum children))]
07:AGGREGATE [STREAMING]
| Class 0
@@ -442,36 +453,36 @@ max-parallelism=1824 segment-costs=[257851731293, 2328086935759] cpu-comparison-
tuple-ids=0 row-size=20B cardinality=8.64G(filtered from 8.64G) cost=2488301335
in pipelines: 00(GETNEXT)
---- PARALLELPLANS
Max Per-Host Resource Reservation: Memory=3.79GB Threads=25
Per-Host Resource Estimates: Memory=138.28GB
Max Per-Host Resource Reservation: Memory=4.70GB Threads=27
Per-Host Resource Estimates: Memory=104.39GB
F06:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
| Per-Instance Resources: mem-estimate=4.29MB mem-reservation=4.00MB thread-reservation=1
| Per-Instance Resources: mem-estimate=4.10MB mem-reservation=4.00MB thread-reservation=1
| max-parallelism=1 segment-costs=[1044] cpu-comparison-result=120 [max(1 (self) vs 120 (sum children))]
PLAN-ROOT SINK
| output exprs: CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 9 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 10 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 11 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 12 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 13 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 14 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 15 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 16 THEN NULL END, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.item.i_class WHEN 9 THEN tpcds_partitioned_parquet_snap.item.i_class WHEN 10 THEN tpcds_partitioned_parquet_snap.item.i_class WHEN 11 THEN tpcds_partitioned_parquet_snap.item.i_class WHEN 12 THEN tpcds_partitioned_parquet_snap.item.i_class WHEN 13 THEN tpcds_partitioned_parquet_snap.item.i_class WHEN 14 THEN tpcds_partitioned_parquet_snap.item.i_class WHEN 15 THEN NULL WHEN 16 THEN NULL END, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN 9 THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN 10 THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN 11 THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN 12 THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN 13 THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN 14 THEN NULL WHEN 15 THEN NULL WHEN 16 THEN NULL END, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.item.i_product_name WHEN 9 THEN tpcds_partitioned_parquet_snap.item.i_product_name WHEN 10 THEN tpcds_partitioned_parquet_snap.item.i_product_name WHEN 11 THEN tpcds_partitioned_parquet_snap.item.i_product_name WHEN 12 THEN tpcds_partitioned_parquet_snap.item.i_product_name WHEN 13 THEN NULL WHEN 14 THEN NULL WHEN 15 THEN NULL WHEN 16 THEN NULL END, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.date_dim.d_year WHEN 9 THEN tpcds_partitioned_parquet_snap.date_dim.d_year WHEN 10 THEN tpcds_partitioned_parquet_snap.date_dim.d_year WHEN 11 THEN tpcds_partitioned_parquet_snap.date_dim.d_year WHEN 12 THEN NULL WHEN 13 THEN NULL WHEN 14 THEN NULL WHEN 15 THEN NULL WHEN 16 THEN NULL END, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.date_dim.d_qoy WHEN 9 THEN tpcds_partitioned_parquet_snap.date_dim.d_qoy WHEN 10 THEN tpcds_partitioned_parquet_snap.date_dim.d_qoy WHEN 11 THEN NULL WHEN 12 THEN NULL WHEN 13 THEN NULL WHEN 14 THEN NULL WHEN 15 THEN NULL WHEN 16 THEN NULL END, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.date_dim.d_moy WHEN 9 THEN tpcds_partitioned_parquet_snap.date_dim.d_moy WHEN 10 THEN NULL WHEN 11 THEN NULL WHEN 12 THEN NULL WHEN 13 THEN NULL WHEN 14 THEN NULL WHEN 15 THEN NULL WHEN 16 THEN NULL END, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.store.s_store_id WHEN 9 THEN NULL WHEN 10 THEN NULL WHEN 11 THEN NULL WHEN 12 THEN NULL WHEN 13 THEN NULL WHEN 14 THEN NULL WHEN 15 THEN NULL WHEN 16 THEN NULL END, aggif(valid_tid(16,8,9,10,11,12,13,14,15) IN (8, 9, 10, 11, 12, 13, 14, 15, 16), CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 9 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 10 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 11 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 12 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 13 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 14 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 15 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 16 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) END), rank()
| mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0 cost=1000
|
19:MERGING-EXCHANGE [UNPARTITIONED]
20:MERGING-EXCHANGE [UNPARTITIONED]
| order by: CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 9 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 10 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 11 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 12 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 13 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 14 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 15 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 16 THEN NULL END ASC NULLS LAST, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.item.i_class WHEN 9 THEN tpcds_partitioned_parquet_snap.item.i_class WHEN 10 THEN tpcds_partitioned_parquet_snap.item.i_class WHEN 11 THEN tpcds_partitioned_parquet_snap.item.i_class WHEN 12 THEN tpcds_partitioned_parquet_snap.item.i_class WHEN 13 THEN tpcds_partitioned_parquet_snap.item.i_class WHEN 14 THEN tpcds_partitioned_parquet_snap.item.i_class WHEN 15 THEN NULL WHEN 16 THEN NULL END ASC NULLS LAST, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN 9 THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN 10 THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN 11 THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN 12 THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN 13 THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN 14 THEN NULL WHEN 15 THEN NULL WHEN 16 THEN NULL END ASC NULLS LAST, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.item.i_product_name WHEN 9 THEN tpcds_partitioned_parquet_snap.item.i_product_name WHEN 10 THEN tpcds_partitioned_parquet_snap.item.i_product_name WHEN 11 THEN tpcds_partitioned_parquet_snap.item.i_product_name WHEN 12 THEN tpcds_partitioned_parquet_snap.item.i_product_name WHEN 13 THEN NULL WHEN 14 THEN NULL WHEN 15 THEN NULL WHEN 16 THEN NULL END ASC NULLS LAST, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.date_dim.d_year WHEN 9 THEN tpcds_partitioned_parquet_snap.date_dim.d_year WHEN 10 THEN tpcds_partitioned_parquet_snap.date_dim.d_year WHEN 11 THEN tpcds_partitioned_parquet_snap.date_dim.d_year WHEN 12 THEN NULL WHEN 13 THEN NULL WHEN 14 THEN NULL WHEN 15 THEN NULL WHEN 16 THEN NULL END ASC NULLS LAST, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.date_dim.d_qoy WHEN 9 THEN tpcds_partitioned_parquet_snap.date_dim.d_qoy WHEN 10 THEN tpcds_partitioned_parquet_snap.date_dim.d_qoy WHEN 11 THEN NULL WHEN 12 THEN NULL WHEN 13 THEN NULL WHEN 14 THEN NULL WHEN 15 THEN NULL WHEN 16 THEN NULL END ASC NULLS LAST, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.date_dim.d_moy WHEN 9 THEN tpcds_partitioned_parquet_snap.date_dim.d_moy WHEN 10 THEN NULL WHEN 11 THEN NULL WHEN 12 THEN NULL WHEN 13 THEN NULL WHEN 14 THEN NULL WHEN 15 THEN NULL WHEN 16 THEN NULL END ASC NULLS LAST, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.store.s_store_id WHEN 9 THEN NULL WHEN 10 THEN NULL WHEN 11 THEN NULL WHEN 12 THEN NULL WHEN 13 THEN NULL WHEN 14 THEN NULL WHEN 15 THEN NULL WHEN 16 THEN NULL END ASC NULLS LAST, aggif(valid_tid(16,8,9,10,11,12,13,14,15) IN (8, 9, 10, 11, 12, 13, 14, 15, 16), CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 9 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 10 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 11 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 12 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 13 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 14 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 15 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 16 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) END) ASC NULLS LAST, rank() ASC NULLS LAST
| limit: 100
| mem-estimate=293.91KB mem-reservation=0B thread-reservation=0
| mem-estimate=98.59KB mem-reservation=0B thread-reservation=0
| tuple-ids=24 row-size=96B cardinality=100 cost=44
| in pipelines: 12(GETNEXT)
|
F05:PLAN FRAGMENT [HASH(CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 9 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 10 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 11 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 12 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 13 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 14 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 15 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 16 THEN NULL END)] hosts=10 instances=30 (adjusted from 120)
Per-Instance Resources: mem-estimate=16.68GB mem-reservation=16.00MB thread-reservation=1
max-parallelism=6120 segment-costs=[61179606532, 45827804421, 282] cpu-comparison-result=120 [max(30 (self) vs 120 (sum children))]
F05:PLAN FRAGMENT [HASH(CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 9 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 10 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 11 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 12 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 13 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 14 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 15 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 16 THEN NULL END)] hosts=10 instances=10 (adjusted from 120)
Per-Instance Resources: mem-estimate=5.30MB mem-reservation=4.00MB thread-reservation=1
max-parallelism=10 segment-costs=[967, 1261, 282] cpu-comparison-result=120 [max(10 (self) vs 120 (sum children))]
12:TOP-N [LIMIT=100]
| order by: CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 9 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 10 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 11 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 12 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 13 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 14 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 15 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 16 THEN NULL END ASC NULLS LAST, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.item.i_class WHEN 9 THEN tpcds_partitioned_parquet_snap.item.i_class WHEN 10 THEN tpcds_partitioned_parquet_snap.item.i_class WHEN 11 THEN tpcds_partitioned_parquet_snap.item.i_class WHEN 12 THEN tpcds_partitioned_parquet_snap.item.i_class WHEN 13 THEN tpcds_partitioned_parquet_snap.item.i_class WHEN 14 THEN tpcds_partitioned_parquet_snap.item.i_class WHEN 15 THEN NULL WHEN 16 THEN NULL END ASC NULLS LAST, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN 9 THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN 10 THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN 11 THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN 12 THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN 13 THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN 14 THEN NULL WHEN 15 THEN NULL WHEN 16 THEN NULL END ASC NULLS LAST, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.item.i_product_name WHEN 9 THEN tpcds_partitioned_parquet_snap.item.i_product_name WHEN 10 THEN tpcds_partitioned_parquet_snap.item.i_product_name WHEN 11 THEN tpcds_partitioned_parquet_snap.item.i_product_name WHEN 12 THEN tpcds_partitioned_parquet_snap.item.i_product_name WHEN 13 THEN NULL WHEN 14 THEN NULL WHEN 15 THEN NULL WHEN 16 THEN NULL END ASC NULLS LAST, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.date_dim.d_year WHEN 9 THEN tpcds_partitioned_parquet_snap.date_dim.d_year WHEN 10 THEN tpcds_partitioned_parquet_snap.date_dim.d_year WHEN 11 THEN tpcds_partitioned_parquet_snap.date_dim.d_year WHEN 12 THEN NULL WHEN 13 THEN NULL WHEN 14 THEN NULL WHEN 15 THEN NULL WHEN 16 THEN NULL END ASC NULLS LAST, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.date_dim.d_qoy WHEN 9 THEN tpcds_partitioned_parquet_snap.date_dim.d_qoy WHEN 10 THEN tpcds_partitioned_parquet_snap.date_dim.d_qoy WHEN 11 THEN NULL WHEN 12 THEN NULL WHEN 13 THEN NULL WHEN 14 THEN NULL WHEN 15 THEN NULL WHEN 16 THEN NULL END ASC NULLS LAST, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.date_dim.d_moy WHEN 9 THEN tpcds_partitioned_parquet_snap.date_dim.d_moy WHEN 10 THEN NULL WHEN 11 THEN NULL WHEN 12 THEN NULL WHEN 13 THEN NULL WHEN 14 THEN NULL WHEN 15 THEN NULL WHEN 16 THEN NULL END ASC NULLS LAST, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.store.s_store_id WHEN 9 THEN NULL WHEN 10 THEN NULL WHEN 11 THEN NULL WHEN 12 THEN NULL WHEN 13 THEN NULL WHEN 14 THEN NULL WHEN 15 THEN NULL WHEN 16 THEN NULL END ASC NULLS LAST, aggif(valid_tid(16,8,9,10,11,12,13,14,15) IN (8, 9, 10, 11, 12, 13, 14, 15, 16), CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 9 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 10 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 11 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 12 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 13 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 14 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 15 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 16 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) END) ASC NULLS LAST, rank() ASC NULLS LAST
| mem-estimate=9.38KB mem-reservation=0B thread-reservation=0
| tuple-ids=24 row-size=96B cardinality=100 cost=21075768581
| in pipelines: 12(GETNEXT), 09(OPEN)
| tuple-ids=24 row-size=96B cardinality=100 cost=861
| in pipelines: 12(GETNEXT), 19(OPEN)
|
11:SELECT
| predicates: rank() <= CAST(100 AS BIGINT)
| mem-estimate=0B mem-reservation=0B thread-reservation=0
| tuple-ids=22,21 row-size=100B cardinality=1.24G cost=12376017920
| in pipelines: 09(GETNEXT)
| tuple-ids=22,21 row-size=100B cardinality=200 cost=200
| in pipelines: 19(GETNEXT)
|
10:ANALYTIC
| functions: rank()
@@ -479,27 +490,36 @@ max-parallelism=6120 segment-costs=[61179606532, 45827804421, 282] cpu-compariso
| order by: aggif(valid_tid(16,8,9,10,11,12,13,14,15) IN (8, 9, 10, 11, 12, 13, 14, 15, 16), CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 9 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 10 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 11 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 12 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 13 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 14 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 15 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 16 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) END) DESC
| window: RANGE BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
| mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0
| tuple-ids=22,21 row-size=100B cardinality=12.38G cost=12376017920
| in pipelines: 09(GETNEXT)
| tuple-ids=22,21 row-size=100B cardinality=200 cost=200
| in pipelines: 19(GETNEXT)
|
09:SORT
19:TOP-N
| order by: CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 9 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 10 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 11 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 12 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 13 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 14 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 15 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 16 THEN NULL END ASC NULLS LAST, aggif(valid_tid(16,8,9,10,11,12,13,14,15) IN (8, 9, 10, 11, 12, 13, 14, 15, 16), CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 9 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 10 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 11 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 12 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 13 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 14 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 15 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 16 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) END) DESC NULLS FIRST
| mem-estimate=16.67GB mem-reservation=12.00MB spill-buffer=2.00MB thread-reservation=0
| tuple-ids=22 row-size=92B cardinality=12.38G cost=54567099909
| in pipelines: 09(GETNEXT), 08(OPEN)
| limit with ties: 200
| mem-estimate=17.97KB mem-reservation=0B thread-reservation=0
| tuple-ids=22 row-size=92B cardinality=200 cost=861
| in pipelines: 19(GETNEXT), 09(OPEN)
|
18:EXCHANGE [HASH(CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 9 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 10 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 11 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 12 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 13 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 14 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 15 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 16 THEN NULL END)]
| mem-estimate=12.81MB mem-reservation=0B thread-reservation=0
| tuple-ids=17 row-size=92B cardinality=12.38G cost=6612506623
| in pipelines: 08(GETNEXT)
| mem-estimate=1.28MB mem-reservation=0B thread-reservation=0
| tuple-ids=22 row-size=92B cardinality=200 cost=106
| in pipelines: 09(GETNEXT)
|
F04:PLAN FRAGMENT [HASH(CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_category) WHEN 9 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_category) WHEN 10 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_category) WHEN 11 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_category) WHEN 12 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_category) WHEN 13 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_category) WHEN 14 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_category) WHEN 15 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_category) WHEN 16 THEN murmur_hash(NULL) END,CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_class) WHEN 9 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_class) WHEN 10 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_class) WHEN 11 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_class) WHEN 12 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_class) WHEN 13 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_class) WHEN 14 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_class) WHEN 15 THEN murmur_hash(NULL) WHEN 16 THEN murmur_hash(NULL) END,CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_brand) WHEN 9 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_brand) WHEN 10 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_brand) WHEN 11 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_brand) WHEN 12 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_brand) WHEN 13 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_brand) WHEN 14 THEN murmur_hash(NULL) WHEN 15 THEN murmur_hash(NULL) WHEN 16 THEN murmur_hash(NULL) END,CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_product_name) WHEN 9 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_product_name) WHEN 10 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_product_name) WHEN 11 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_product_name) WHEN 12 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_product_name) WHEN 13 THEN murmur_hash(NULL) WHEN 14 THEN murmur_hash(NULL) WHEN 15 THEN murmur_hash(NULL) WHEN 16 THEN murmur_hash(NULL) END,CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN murmur_hash(tpcds_partitioned_parquet_snap.date_dim.d_year) WHEN 9 THEN murmur_hash(tpcds_partitioned_parquet_snap.date_dim.d_year) WHEN 10 THEN murmur_hash(tpcds_partitioned_parquet_snap.date_dim.d_year) WHEN 11 THEN murmur_hash(tpcds_partitioned_parquet_snap.date_dim.d_year) WHEN 12 THEN murmur_hash(NULL) WHEN 13 THEN murmur_hash(NULL) WHEN 14 THEN murmur_hash(NULL) WHEN 15 THEN murmur_hash(NULL) WHEN 16 THEN murmur_hash(NULL) END,CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN murmur_hash(tpcds_partitioned_parquet_snap.date_dim.d_qoy) WHEN 9 THEN murmur_hash(tpcds_partitioned_parquet_snap.date_dim.d_qoy) WHEN 10 THEN murmur_hash(tpcds_partitioned_parquet_snap.date_dim.d_qoy) WHEN 11 THEN murmur_hash(NULL) WHEN 12 THEN murmur_hash(NULL) WHEN 13 THEN murmur_hash(NULL) WHEN 14 THEN murmur_hash(NULL) WHEN 15 THEN murmur_hash(NULL) WHEN 16 THEN murmur_hash(NULL) END,CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN murmur_hash(tpcds_partitioned_parquet_snap.date_dim.d_moy) WHEN 9 THEN murmur_hash(tpcds_partitioned_parquet_snap.date_dim.d_moy) WHEN 10 THEN murmur_hash(NULL) WHEN 11 THEN murmur_hash(NULL) WHEN 12 THEN murmur_hash(NULL) WHEN 13 THEN murmur_hash(NULL) WHEN 14 THEN murmur_hash(NULL) WHEN 15 THEN murmur_hash(NULL) WHEN 16 THEN murmur_hash(NULL) END,CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN murmur_hash(tpcds_partitioned_parquet_snap.store.s_store_id) WHEN 9 THEN murmur_hash(NULL) WHEN 10 THEN murmur_hash(NULL) WHEN 11 THEN murmur_hash(NULL) WHEN 12 THEN murmur_hash(NULL) WHEN 13 THEN murmur_hash(NULL) WHEN 14 THEN murmur_hash(NULL) WHEN 15 THEN murmur_hash(NULL) WHEN 16 THEN murmur_hash(NULL) END)] hosts=10 instances=70 (adjusted from 120)
Per-Instance Resources: mem-estimate=7.14GB mem-reservation=244.75MB thread-reservation=1
max-parallelism=26630 segment-costs=[266227423359, 73050686125, 233936768446, 1288] cpu-comparison-result=120 [max(70 (self) vs 120 (sum children))]
09:TOP-N
| order by: CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 9 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 10 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 11 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 12 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 13 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 14 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 15 THEN tpcds_partitioned_parquet_snap.item.i_category WHEN 16 THEN NULL END ASC NULLS LAST, aggif(valid_tid(16,8,9,10,11,12,13,14,15) IN (8, 9, 10, 11, 12, 13, 14, 15, 16), CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 9 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 10 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 11 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 12 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 13 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 14 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 15 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN 16 THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) END) DESC NULLS FIRST
| limit with ties: 200
| source expr: rank() <= CAST(100 AS BIGINT)
| mem-estimate=17.97KB mem-reservation=0B thread-reservation=0
| tuple-ids=22 row-size=92B cardinality=200 cost=233936768446
| in pipelines: 09(GETNEXT), 08(OPEN)
|
F04:PLAN FRAGMENT [HASH(CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_category) WHEN 9 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_category) WHEN 10 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_category) WHEN 11 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_category) WHEN 12 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_category) WHEN 13 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_category) WHEN 14 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_category) WHEN 15 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_category) WHEN 16 THEN murmur_hash(NULL) END,CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_class) WHEN 9 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_class) WHEN 10 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_class) WHEN 11 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_class) WHEN 12 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_class) WHEN 13 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_class) WHEN 14 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_class) WHEN 15 THEN murmur_hash(NULL) WHEN 16 THEN murmur_hash(NULL) END,CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_brand) WHEN 9 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_brand) WHEN 10 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_brand) WHEN 11 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_brand) WHEN 12 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_brand) WHEN 13 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_brand) WHEN 14 THEN murmur_hash(NULL) WHEN 15 THEN murmur_hash(NULL) WHEN 16 THEN murmur_hash(NULL) END,CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_product_name) WHEN 9 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_product_name) WHEN 10 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_product_name) WHEN 11 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_product_name) WHEN 12 THEN murmur_hash(tpcds_partitioned_parquet_snap.item.i_product_name) WHEN 13 THEN murmur_hash(NULL) WHEN 14 THEN murmur_hash(NULL) WHEN 15 THEN murmur_hash(NULL) WHEN 16 THEN murmur_hash(NULL) END,CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN murmur_hash(tpcds_partitioned_parquet_snap.date_dim.d_year) WHEN 9 THEN murmur_hash(tpcds_partitioned_parquet_snap.date_dim.d_year) WHEN 10 THEN murmur_hash(tpcds_partitioned_parquet_snap.date_dim.d_year) WHEN 11 THEN murmur_hash(tpcds_partitioned_parquet_snap.date_dim.d_year) WHEN 12 THEN murmur_hash(NULL) WHEN 13 THEN murmur_hash(NULL) WHEN 14 THEN murmur_hash(NULL) WHEN 15 THEN murmur_hash(NULL) WHEN 16 THEN murmur_hash(NULL) END,CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN murmur_hash(tpcds_partitioned_parquet_snap.date_dim.d_qoy) WHEN 9 THEN murmur_hash(tpcds_partitioned_parquet_snap.date_dim.d_qoy) WHEN 10 THEN murmur_hash(tpcds_partitioned_parquet_snap.date_dim.d_qoy) WHEN 11 THEN murmur_hash(NULL) WHEN 12 THEN murmur_hash(NULL) WHEN 13 THEN murmur_hash(NULL) WHEN 14 THEN murmur_hash(NULL) WHEN 15 THEN murmur_hash(NULL) WHEN 16 THEN murmur_hash(NULL) END,CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN murmur_hash(tpcds_partitioned_parquet_snap.date_dim.d_moy) WHEN 9 THEN murmur_hash(tpcds_partitioned_parquet_snap.date_dim.d_moy) WHEN 10 THEN murmur_hash(NULL) WHEN 11 THEN murmur_hash(NULL) WHEN 12 THEN murmur_hash(NULL) WHEN 13 THEN murmur_hash(NULL) WHEN 14 THEN murmur_hash(NULL) WHEN 15 THEN murmur_hash(NULL) WHEN 16 THEN murmur_hash(NULL) END,CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN 8 THEN murmur_hash(tpcds_partitioned_parquet_snap.store.s_store_id) WHEN 9 THEN murmur_hash(NULL) WHEN 10 THEN murmur_hash(NULL) WHEN 11 THEN murmur_hash(NULL) WHEN 12 THEN murmur_hash(NULL) WHEN 13 THEN murmur_hash(NULL) WHEN 14 THEN murmur_hash(NULL) WHEN 15 THEN murmur_hash(NULL) WHEN 16 THEN murmur_hash(NULL) END)] hosts=10 instances=30 (adjusted from 120)
Per-Instance Resources: mem-estimate=11.98GB mem-reservation=244.75MB thread-reservation=1
max-parallelism=26630 segment-costs=[266227423359, 73050686125, 79701558405] cpu-comparison-result=120 [max(30 (self) vs 120 (sum children))]
08:AGGREGATE [FINALIZE]
| output: aggif(valid_tid(16,8,9,10,11,12,13,14,15) IN (CAST(8 AS INT), CAST(9 AS INT), CAST(10 AS INT), CAST(11 AS INT), CAST(12 AS INT), CAST(13 AS INT), CAST(14 AS INT), CAST(15 AS INT), CAST(16 AS INT)), CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN CAST(8 AS INT) THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN CAST(9 AS INT) THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN CAST(10 AS INT) THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN CAST(11 AS INT) THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN CAST(12 AS INT) THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN CAST(13 AS INT) THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN CAST(14 AS INT) THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN CAST(15 AS INT) THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) WHEN CAST(16 AS INT) THEN sum(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0)) END)
| group by: CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN CAST(8 AS INT) THEN tpcds_partitioned_parquet_snap.item.i_category WHEN CAST(9 AS INT) THEN tpcds_partitioned_parquet_snap.item.i_category WHEN CAST(10 AS INT) THEN tpcds_partitioned_parquet_snap.item.i_category WHEN CAST(11 AS INT) THEN tpcds_partitioned_parquet_snap.item.i_category WHEN CAST(12 AS INT) THEN tpcds_partitioned_parquet_snap.item.i_category WHEN CAST(13 AS INT) THEN tpcds_partitioned_parquet_snap.item.i_category WHEN CAST(14 AS INT) THEN tpcds_partitioned_parquet_snap.item.i_category WHEN CAST(15 AS INT) THEN tpcds_partitioned_parquet_snap.item.i_category WHEN CAST(16 AS INT) THEN NULL END, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN CAST(8 AS INT) THEN tpcds_partitioned_parquet_snap.item.i_class WHEN CAST(9 AS INT) THEN tpcds_partitioned_parquet_snap.item.i_class WHEN CAST(10 AS INT) THEN tpcds_partitioned_parquet_snap.item.i_class WHEN CAST(11 AS INT) THEN tpcds_partitioned_parquet_snap.item.i_class WHEN CAST(12 AS INT) THEN tpcds_partitioned_parquet_snap.item.i_class WHEN CAST(13 AS INT) THEN tpcds_partitioned_parquet_snap.item.i_class WHEN CAST(14 AS INT) THEN tpcds_partitioned_parquet_snap.item.i_class WHEN CAST(15 AS INT) THEN NULL WHEN CAST(16 AS INT) THEN NULL END, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN CAST(8 AS INT) THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN CAST(9 AS INT) THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN CAST(10 AS INT) THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN CAST(11 AS INT) THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN CAST(12 AS INT) THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN CAST(13 AS INT) THEN tpcds_partitioned_parquet_snap.item.i_brand WHEN CAST(14 AS INT) THEN NULL WHEN CAST(15 AS INT) THEN NULL WHEN CAST(16 AS INT) THEN NULL END, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN CAST(8 AS INT) THEN tpcds_partitioned_parquet_snap.item.i_product_name WHEN CAST(9 AS INT) THEN tpcds_partitioned_parquet_snap.item.i_product_name WHEN CAST(10 AS INT) THEN tpcds_partitioned_parquet_snap.item.i_product_name WHEN CAST(11 AS INT) THEN tpcds_partitioned_parquet_snap.item.i_product_name WHEN CAST(12 AS INT) THEN tpcds_partitioned_parquet_snap.item.i_product_name WHEN CAST(13 AS INT) THEN NULL WHEN CAST(14 AS INT) THEN NULL WHEN CAST(15 AS INT) THEN NULL WHEN CAST(16 AS INT) THEN NULL END, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN CAST(8 AS INT) THEN tpcds_partitioned_parquet_snap.date_dim.d_year WHEN CAST(9 AS INT) THEN tpcds_partitioned_parquet_snap.date_dim.d_year WHEN CAST(10 AS INT) THEN tpcds_partitioned_parquet_snap.date_dim.d_year WHEN CAST(11 AS INT) THEN tpcds_partitioned_parquet_snap.date_dim.d_year WHEN CAST(12 AS INT) THEN NULL WHEN CAST(13 AS INT) THEN NULL WHEN CAST(14 AS INT) THEN NULL WHEN CAST(15 AS INT) THEN NULL WHEN CAST(16 AS INT) THEN NULL END, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN CAST(8 AS INT) THEN tpcds_partitioned_parquet_snap.date_dim.d_qoy WHEN CAST(9 AS INT) THEN tpcds_partitioned_parquet_snap.date_dim.d_qoy WHEN CAST(10 AS INT) THEN tpcds_partitioned_parquet_snap.date_dim.d_qoy WHEN CAST(11 AS INT) THEN NULL WHEN CAST(12 AS INT) THEN NULL WHEN CAST(13 AS INT) THEN NULL WHEN CAST(14 AS INT) THEN NULL WHEN CAST(15 AS INT) THEN NULL WHEN CAST(16 AS INT) THEN NULL END, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN CAST(8 AS INT) THEN tpcds_partitioned_parquet_snap.date_dim.d_moy WHEN CAST(9 AS INT) THEN tpcds_partitioned_parquet_snap.date_dim.d_moy WHEN CAST(10 AS INT) THEN NULL WHEN CAST(11 AS INT) THEN NULL WHEN CAST(12 AS INT) THEN NULL WHEN CAST(13 AS INT) THEN NULL WHEN CAST(14 AS INT) THEN NULL WHEN CAST(15 AS INT) THEN NULL WHEN CAST(16 AS INT) THEN NULL END, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN CAST(8 AS INT) THEN tpcds_partitioned_parquet_snap.store.s_store_id WHEN CAST(9 AS INT) THEN NULL WHEN CAST(10 AS INT) THEN NULL WHEN CAST(11 AS INT) THEN NULL WHEN CAST(12 AS INT) THEN NULL WHEN CAST(13 AS INT) THEN NULL WHEN CAST(14 AS INT) THEN NULL WHEN CAST(15 AS INT) THEN NULL WHEN CAST(16 AS INT) THEN NULL END, CASE valid_tid(16,8,9,10,11,12,13,14,15) WHEN CAST(8 AS INT) THEN CAST(8 AS INT) WHEN CAST(9 AS INT) THEN CAST(9 AS INT) WHEN CAST(10 AS INT) THEN CAST(10 AS INT) WHEN CAST(11 AS INT) THEN CAST(11 AS INT) WHEN CAST(12 AS INT) THEN CAST(12 AS INT) WHEN CAST(13 AS INT) THEN CAST(13 AS INT) WHEN CAST(14 AS INT) THEN CAST(14 AS INT) WHEN CAST(15 AS INT) THEN CAST(15 AS INT) WHEN CAST(16 AS INT) THEN CAST(16 AS INT) END
| mem-estimate=669.83MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
| mem-estimate=578.50MB mem-reservation=34.00MB spill-buffer=2.00MB thread-reservation=0
| tuple-ids=17 row-size=92B cardinality=12.38G cost=73050686125
| in pipelines: 08(GETNEXT), 17(OPEN)
|
@@ -531,7 +551,7 @@ max-parallelism=26630 segment-costs=[266227423359, 73050686125, 79701558405] cpu
| Class 8
| output: sum:merge(coalesce(multiply(tpcds_partitioned_parquet_snap.store_sales.ss_sales_price, tpcds_partitioned_parquet_snap.store_sales.ss_quantity), 0))
| group by: NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL
| mem-estimate=11.33GB mem-reservation=210.75MB thread-reservation=0
| mem-estimate=6.57GB mem-reservation=210.75MB thread-reservation=0
| tuple-ids=8N,9N,10N,11N,12N,13N,14N,15N,16N row-size=1.09KB cardinality=12.38G cost=97683994945
| in pipelines: 17(GETNEXT), 00(OPEN)
|
@@ -542,7 +562,7 @@ max-parallelism=26630 segment-costs=[266227423359, 73050686125, 79701558405] cpu
|
F00:PLAN FRAGMENT [RANDOM] hosts=10 instances=120
Per-Host Shared Resources: mem-estimate=3.00MB mem-reservation=3.00MB thread-reservation=0 runtime-filters-memory=3.00MB
Per-Instance Resources: mem-estimate=4.31GB mem-reservation=219.00MB thread-reservation=1
Per-Instance Resources: mem-estimate=4.49GB mem-reservation=219.00MB thread-reservation=1
max-parallelism=1824 segment-costs=[257851731293, 2328086935759] cpu-comparison-result=120 [max(120 (self) vs 36 (sum children))]
07:AGGREGATE [STREAMING]
| Class 0

View File

@@ -42,7 +42,7 @@ Max Per-Host Resource Reservation: Memory=37.40MB Threads=1
Per-Host Resource Estimates: Memory=115MB
F00:PLAN FRAGMENT [UNPARTITIONED] hosts=1 instances=1
| Per-Instance Resources: mem-estimate=114.81MB mem-reservation=37.40MB thread-reservation=1 runtime-filters-memory=5.00MB
| max-parallelism=1 segment-costs=[11583582477, 49, 11797402272, 8242, 3284, 9372, 500]
| max-parallelism=1 segment-costs=[11583582477, 79, 44672813328, 8242, 3284, 9372, 500]
PLAN-ROOT SINK
| output exprs: aggif(valid_tid(19,20,21) IN (19, 20, 21), CASE valid_tid(19,20,21) WHEN 19 THEN sum(tpcds_partitioned_parquet_snap.store_sales.ss_net_profit) WHEN 20 THEN sum(tpcds_partitioned_parquet_snap.store_sales.ss_net_profit) WHEN 21 THEN sum(tpcds_partitioned_parquet_snap.store_sales.ss_net_profit) END), CASE valid_tid(19,20,21) WHEN 19 THEN tpcds_partitioned_parquet_snap.store.s_state WHEN 20 THEN tpcds_partitioned_parquet_snap.store.s_state WHEN 21 THEN NULL END, CASE valid_tid(19,20,21) WHEN 19 THEN tpcds_partitioned_parquet_snap.store.s_county WHEN 20 THEN NULL WHEN 21 THEN NULL END, add(CAST(aggif(valid_tid(19,20,21) IN (19, 20, 21), CASE valid_tid(19,20,21) WHEN 19 THEN 0 WHEN 20 THEN 0 WHEN 21 THEN 1 END) AS SMALLINT), CAST(aggif(valid_tid(19,20,21) IN (19, 20, 21), CASE valid_tid(19,20,21) WHEN 19 THEN 0 WHEN 20 THEN 1 WHEN 21 THEN 1 END) AS SMALLINT)), rank()
| mem-estimate=4.00MB mem-reservation=4.00MB spill-buffer=2.00MB thread-reservation=0 cost=500
@@ -86,7 +86,7 @@ PLAN-ROOT SINK
| output: sum(tpcds_partitioned_parquet_snap.store_sales.ss_net_profit)
| group by: NULL, NULL
| mem-estimate=30.00MB mem-reservation=5.81MB thread-reservation=0
| tuple-ids=19N,20N,21N row-size=138B cardinality=1.36K cost=7460644055
| tuple-ids=19N,20N,21N row-size=138B cardinality=1.36K cost=35616411022
| in pipelines: 15(GETNEXT), 00(OPEN)
|
14:HASH JOIN [INNER JOIN]
@@ -94,7 +94,7 @@ PLAN-ROOT SINK
| fk/pk conjuncts: tpcds_partitioned_parquet_snap.store_sales.ss_store_sk = tpcds_partitioned_parquet_snap.store.s_store_sk
| runtime filters: RF000[bloom] <- tpcds_partitioned_parquet_snap.store.s_store_sk, RF001[min_max] <- tpcds_partitioned_parquet_snap.store.s_store_sk
| mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
| tuple-ids=0,2,4,17,16 row-size=102B cardinality=1.81G cost=803070546
| tuple-ids=0,2,4,17,16 row-size=102B cardinality=8.64G cost=3781700983
| in pipelines: 00(GETNEXT), 03(OPEN)
|
|--13:HASH JOIN [INNER JOIN]
@@ -102,13 +102,13 @@ PLAN-ROOT SINK
| | fk/pk conjuncts: none
| | runtime filters: RF004[bloom] <- tpcds_partitioned_parquet_snap.store_0.s_state, RF005[min_max] <- tpcds_partitioned_parquet_snap.store_0.s_state
| | mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
| | tuple-ids=4,17,16 row-size=82B cardinality=140 cost=64
| | tuple-ids=4,17,16 row-size=82B cardinality=1.35K cost=619
| | in pipelines: 03(GETNEXT), 10(OPEN)
| |
| |--12:SELECT
| | | predicates: rank() <= CAST(5 AS BIGINT)
| | | mem-estimate=0B mem-reservation=0B thread-reservation=0
| | | tuple-ids=17,16 row-size=38B cardinality=3 cost=29
| | | tuple-ids=17,16 row-size=38B cardinality=29 cost=29
| | | in pipelines: 10(GETNEXT)
| | |
| | 11:ANALYTIC
@@ -120,10 +120,13 @@ PLAN-ROOT SINK
| | | tuple-ids=17,16 row-size=38B cardinality=29 cost=29
| | | in pipelines: 10(GETNEXT)
| | |
| | 10:SORT
| | | order by: tpcds_partitioned_parquet_snap.store_0.s_state ASC NULLS LAST, sum(tpcds_partitioned_parquet_snap.store_sales_0.ss_net_profit) DESC NULLS FIRST
| | 10:TOP-N
| | | partition by: tpcds_partitioned_parquet_snap.store_0.s_state
| | | order by: sum(tpcds_partitioned_parquet_snap.store_sales_0.ss_net_profit) DESC NULLS FIRST
| | | partition limit: 5 (include ties)
| | | source expr: rank() <= CAST(5 AS BIGINT)
| | | mem-estimate=12.00MB mem-reservation=12.00MB spill-buffer=2.00MB thread-reservation=0
| | | tuple-ids=17 row-size=30B cardinality=29 cost=49
| | | tuple-ids=17 row-size=30B cardinality=29 cost=79
| | | in pipelines: 10(GETNEXT), 09(OPEN)
| | |
| | 09:AGGREGATE [FINALIZE]
@@ -192,7 +195,7 @@ PLAN-ROOT SINK
| columns: all
| extrapolated-rows=disabled max-scan-range-rows=1.35K
| mem-estimate=16.00MB mem-reservation=24.00KB thread-reservation=0
| tuple-ids=4 row-size=44B cardinality=140(filtered from 1.35K) cost=387
| tuple-ids=4 row-size=44B cardinality=1.35K cost=387
| in pipelines: 03(GETNEXT)
|
02:HASH JOIN [INNER JOIN]
@@ -200,7 +203,7 @@ PLAN-ROOT SINK
| fk/pk conjuncts: tpcds_partitioned_parquet_snap.store_sales.ss_sold_date_sk = tpcds_partitioned_parquet_snap.date_dim.d_date_sk
| runtime filters: RF002[bloom] <- tpcds_partitioned_parquet_snap.date_dim.d_date_sk
| mem-estimate=1.94MB mem-reservation=1.94MB spill-buffer=64.00KB thread-reservation=0
| tuple-ids=0,2 row-size=20B cardinality=1.85G(filtered from 8.64G) cost=2040693841
| tuple-ids=0,2 row-size=20B cardinality=8.64G cost=3781706938
| in pipelines: 00(GETNEXT), 01(OPEN)
|
|--01:SCAN HDFS [tpcds_partitioned_parquet_snap.date_dim]
@@ -225,7 +228,7 @@ PLAN-ROOT SINK
columns: all
extrapolated-rows=disabled max-scan-range-rows=390.22M
mem-estimate=16.00MB mem-reservation=8.00MB thread-reservation=0
tuple-ids=0 row-size=12B cardinality=1.85G(filtered from 8.64G) cost=1492980801
tuple-ids=0 row-size=12B cardinality=8.64G cost=1492980801
in pipelines: 00(GETNEXT)
---- DISTRIBUTEDPLAN
Max Per-Host Resource Reservation: Memory=437.66MB Threads=38
@@ -305,7 +308,7 @@ max-parallelism=10 segment-costs=[365183, 8242, 4743] cpu-comparison-result=142
F00:PLAN FRAGMENT [RANDOM] hosts=10 instances=120
Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB
Per-Instance Resources: mem-estimate=51.86MB mem-reservation=14.00MB thread-reservation=1
max-parallelism=1530 segment-costs=[15280143040, 1707075] cpu-comparison-result=142 [max(120 (self) vs 142 (sum children))]
max-parallelism=1824 segment-costs=[44673526136, 1707075] cpu-comparison-result=142 [max(120 (self) vs 142 (sum children))]
15:AGGREGATE [STREAMING]
| Class 0
| output: sum(tpcds_partitioned_parquet_snap.store_sales.ss_net_profit)
@@ -317,7 +320,7 @@ max-parallelism=1530 segment-costs=[15280143040, 1707075] cpu-comparison-result=
| output: sum(tpcds_partitioned_parquet_snap.store_sales.ss_net_profit)
| group by: NULL, NULL
| mem-estimate=30.00MB mem-reservation=6.00MB thread-reservation=0
| tuple-ids=19N,20N,21N row-size=138B cardinality=163.68K cost=7461379103
| tuple-ids=19N,20N,21N row-size=138B cardinality=163.68K cost=35617146069
| in pipelines: 00(GETNEXT)
|
14:HASH JOIN [INNER JOIN, BROADCAST]
@@ -325,32 +328,32 @@ max-parallelism=1530 segment-costs=[15280143040, 1707075] cpu-comparison-result=
| hash predicates: tpcds_partitioned_parquet_snap.store_sales.ss_store_sk = tpcds_partitioned_parquet_snap.store.s_store_sk
| fk/pk conjuncts: tpcds_partitioned_parquet_snap.store_sales.ss_store_sk = tpcds_partitioned_parquet_snap.store.s_store_sk
| mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
| tuple-ids=0,2,17,16,4 row-size=102B cardinality=1.81G cost=2544083503
| tuple-ids=0,2,17,16,4 row-size=102B cardinality=8.64G cost=3781699633
| in pipelines: 00(GETNEXT), 10(OPEN)
|
|--F11:PLAN FRAGMENT [RANDOM] hosts=10 instances=10
| | Per-Instance Resources: mem-estimate=24.39MB mem-reservation=24.25MB thread-reservation=1 runtime-filters-memory=1.00MB
| | max-parallelism=10 segment-costs=[320] cpu-comparison-result=131 [max(20 (self) vs 131 (sum children))]
| | Per-Instance Resources: mem-estimate=25.27MB mem-reservation=24.25MB thread-reservation=1 runtime-filters-memory=1.00MB
| | max-parallelism=10 segment-costs=[3140] cpu-comparison-result=131 [max(20 (self) vs 131 (sum children))]
| JOIN BUILD
| | join-table-id=00 plan-id=01 cohort-id=01
| | build expressions: tpcds_partitioned_parquet_snap.store.s_store_sk
| | runtime filters: RF000[bloom] <- tpcds_partitioned_parquet_snap.store.s_store_sk, RF001[min_max] <- tpcds_partitioned_parquet_snap.store.s_store_sk
| | mem-estimate=23.25MB mem-reservation=23.25MB spill-buffer=64.00KB thread-reservation=0 cost=140
| | mem-estimate=23.25MB mem-reservation=23.25MB spill-buffer=64.00KB thread-reservation=0 cost=1350
| |
| 27:EXCHANGE [BROADCAST]
| | mem-estimate=139.65KB mem-reservation=0B thread-reservation=0
| | tuple-ids=17,16,4 row-size=82B cardinality=140 cost=180
| | mem-estimate=1.02MB mem-reservation=0B thread-reservation=0
| | tuple-ids=17,16,4 row-size=82B cardinality=1.35K cost=1790
| | in pipelines: 10(GETNEXT)
| |
| F07:PLAN FRAGMENT [HASH(tpcds_partitioned_parquet_snap.store_0.s_state)] hosts=10 instances=10 (adjusted from 120)
| Per-Instance Resources: mem-estimate=391.79KB mem-reservation=0B thread-reservation=1
| max-parallelism=10 segment-costs=[66]
| max-parallelism=10 segment-costs=[647]
| 13:HASH JOIN [INNER JOIN, PARTITIONED]
| | hash-table-id=01
| | hash predicates: tpcds_partitioned_parquet_snap.store_0.s_state = tpcds_partitioned_parquet_snap.store.s_state
| | fk/pk conjuncts: none
| | mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
| | tuple-ids=17,16,4 row-size=82B cardinality=140 cost=26
| | tuple-ids=17,16,4 row-size=82B cardinality=1.35K cost=252
| | in pipelines: 10(GETNEXT), 03(OPEN)
| |
| |--F12:PLAN FRAGMENT [HASH(tpcds_partitioned_parquet_snap.store_0.s_state)] hosts=10 instances=10 (adjusted from 120)
@@ -381,16 +384,16 @@ max-parallelism=1530 segment-costs=[15280143040, 1707075] cpu-comparison-result=
| |
| 25:EXCHANGE [HASH(tpcds_partitioned_parquet_snap.store_0.s_state)]
| | mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
| | tuple-ids=17,16 row-size=38B cardinality=3 cost=0
| | tuple-ids=17,16 row-size=38B cardinality=29 cost=9
| | in pipelines: 10(GETNEXT)
| |
| F05:PLAN FRAGMENT [HASH(tpcds_partitioned_parquet_snap.store_0.s_state)] hosts=10 instances=10 (adjusted from 120)
| Per-Instance Resources: mem-estimate=26.00MB mem-reservation=17.94MB thread-reservation=1
| max-parallelism=10 segment-costs=[1959, 49, 68] cpu-comparison-result=120 [max(10 (self) vs 120 (sum children))]
| max-parallelism=10 segment-costs=[1959, 79, 158] cpu-comparison-result=120 [max(10 (self) vs 120 (sum children))]
| 12:SELECT
| | predicates: rank() <= CAST(5 AS BIGINT)
| | mem-estimate=0B mem-reservation=0B thread-reservation=0
| | tuple-ids=17,16 row-size=38B cardinality=3 cost=29
| | tuple-ids=17,16 row-size=38B cardinality=29 cost=29
| | in pipelines: 10(GETNEXT)
| |
| 11:ANALYTIC
@@ -402,10 +405,13 @@ max-parallelism=1530 segment-costs=[15280143040, 1707075] cpu-comparison-result=
| | tuple-ids=17,16 row-size=38B cardinality=29 cost=29
| | in pipelines: 10(GETNEXT)
| |
| 10:SORT
| | order by: tpcds_partitioned_parquet_snap.store_0.s_state ASC NULLS LAST, sum(tpcds_partitioned_parquet_snap.store_sales_0.ss_net_profit) DESC NULLS FIRST
| 10:TOP-N
| | partition by: tpcds_partitioned_parquet_snap.store_0.s_state
| | order by: sum(tpcds_partitioned_parquet_snap.store_sales_0.ss_net_profit) DESC NULLS FIRST
| | partition limit: 5 (include ties)
| | source expr: rank() <= CAST(5 AS BIGINT)
| | mem-estimate=12.00MB mem-reservation=12.00MB spill-buffer=2.00MB thread-reservation=0
| | tuple-ids=17 row-size=30B cardinality=29 cost=49
| | tuple-ids=17 row-size=30B cardinality=29 cost=79
| | in pipelines: 10(GETNEXT), 24(OPEN)
| |
| 24:AGGREGATE [FINALIZE]
@@ -643,7 +649,7 @@ max-parallelism=10 segment-costs=[365183, 8242, 4743] cpu-comparison-result=142
F00:PLAN FRAGMENT [RANDOM] hosts=10 instances=120
Per-Host Shared Resources: mem-estimate=2.00MB mem-reservation=2.00MB thread-reservation=0 runtime-filters-memory=2.00MB
Per-Instance Resources: mem-estimate=51.86MB mem-reservation=14.00MB thread-reservation=1
max-parallelism=1530 segment-costs=[15280143040, 1707075] cpu-comparison-result=142 [max(120 (self) vs 142 (sum children))]
max-parallelism=1824 segment-costs=[44673526136, 1707075] cpu-comparison-result=142 [max(120 (self) vs 142 (sum children))]
15:AGGREGATE [STREAMING]
| Class 0
| output: sum(tpcds_partitioned_parquet_snap.store_sales.ss_net_profit)
@@ -655,7 +661,7 @@ max-parallelism=1530 segment-costs=[15280143040, 1707075] cpu-comparison-result=
| output: sum(tpcds_partitioned_parquet_snap.store_sales.ss_net_profit)
| group by: NULL, NULL
| mem-estimate=30.00MB mem-reservation=6.00MB thread-reservation=0
| tuple-ids=19N,20N,21N row-size=138B cardinality=163.68K cost=7461379103
| tuple-ids=19N,20N,21N row-size=138B cardinality=163.68K cost=35617146069
| in pipelines: 00(GETNEXT)
|
14:HASH JOIN [INNER JOIN, BROADCAST]
@@ -663,32 +669,32 @@ max-parallelism=1530 segment-costs=[15280143040, 1707075] cpu-comparison-result=
| hash predicates: tpcds_partitioned_parquet_snap.store_sales.ss_store_sk = tpcds_partitioned_parquet_snap.store.s_store_sk
| fk/pk conjuncts: tpcds_partitioned_parquet_snap.store_sales.ss_store_sk = tpcds_partitioned_parquet_snap.store.s_store_sk
| mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
| tuple-ids=0,2,17,16,4 row-size=102B cardinality=1.81G cost=2544083503
| tuple-ids=0,2,17,16,4 row-size=102B cardinality=8.64G cost=3781699633
| in pipelines: 00(GETNEXT), 10(OPEN)
|
|--F11:PLAN FRAGMENT [RANDOM] hosts=10 instances=10
| | Per-Instance Resources: mem-estimate=24.39MB mem-reservation=24.25MB thread-reservation=1 runtime-filters-memory=1.00MB
| | max-parallelism=10 segment-costs=[320] cpu-comparison-result=131 [max(20 (self) vs 131 (sum children))]
| | Per-Instance Resources: mem-estimate=25.27MB mem-reservation=24.25MB thread-reservation=1 runtime-filters-memory=1.00MB
| | max-parallelism=10 segment-costs=[3140] cpu-comparison-result=131 [max(20 (self) vs 131 (sum children))]
| JOIN BUILD
| | join-table-id=00 plan-id=01 cohort-id=01
| | build expressions: tpcds_partitioned_parquet_snap.store.s_store_sk
| | runtime filters: RF000[bloom] <- tpcds_partitioned_parquet_snap.store.s_store_sk, RF001[min_max] <- tpcds_partitioned_parquet_snap.store.s_store_sk
| | mem-estimate=23.25MB mem-reservation=23.25MB spill-buffer=64.00KB thread-reservation=0 cost=140
| | mem-estimate=23.25MB mem-reservation=23.25MB spill-buffer=64.00KB thread-reservation=0 cost=1350
| |
| 27:EXCHANGE [BROADCAST]
| | mem-estimate=139.65KB mem-reservation=0B thread-reservation=0
| | tuple-ids=17,16,4 row-size=82B cardinality=140 cost=180
| | mem-estimate=1.02MB mem-reservation=0B thread-reservation=0
| | tuple-ids=17,16,4 row-size=82B cardinality=1.35K cost=1790
| | in pipelines: 10(GETNEXT)
| |
| F07:PLAN FRAGMENT [HASH(tpcds_partitioned_parquet_snap.store_0.s_state)] hosts=10 instances=10 (adjusted from 120)
| Per-Instance Resources: mem-estimate=391.79KB mem-reservation=0B thread-reservation=1
| max-parallelism=10 segment-costs=[66]
| max-parallelism=10 segment-costs=[647]
| 13:HASH JOIN [INNER JOIN, PARTITIONED]
| | hash-table-id=01
| | hash predicates: tpcds_partitioned_parquet_snap.store_0.s_state = tpcds_partitioned_parquet_snap.store.s_state
| | fk/pk conjuncts: none
| | mem-estimate=0B mem-reservation=0B spill-buffer=64.00KB thread-reservation=0
| | tuple-ids=17,16,4 row-size=82B cardinality=140 cost=26
| | tuple-ids=17,16,4 row-size=82B cardinality=1.35K cost=252
| | in pipelines: 10(GETNEXT), 03(OPEN)
| |
| |--F12:PLAN FRAGMENT [HASH(tpcds_partitioned_parquet_snap.store_0.s_state)] hosts=10 instances=10 (adjusted from 120)
@@ -719,16 +725,16 @@ max-parallelism=1530 segment-costs=[15280143040, 1707075] cpu-comparison-result=
| |
| 25:EXCHANGE [HASH(tpcds_partitioned_parquet_snap.store_0.s_state)]
| | mem-estimate=16.00KB mem-reservation=0B thread-reservation=0
| | tuple-ids=17,16 row-size=38B cardinality=3 cost=0
| | tuple-ids=17,16 row-size=38B cardinality=29 cost=9
| | in pipelines: 10(GETNEXT)
| |
| F05:PLAN FRAGMENT [HASH(tpcds_partitioned_parquet_snap.store_0.s_state)] hosts=10 instances=10 (adjusted from 120)
| Per-Instance Resources: mem-estimate=26.00MB mem-reservation=17.94MB thread-reservation=1
| max-parallelism=10 segment-costs=[1959, 49, 68] cpu-comparison-result=120 [max(10 (self) vs 120 (sum children))]
| max-parallelism=10 segment-costs=[1959, 79, 158] cpu-comparison-result=120 [max(10 (self) vs 120 (sum children))]
| 12:SELECT
| | predicates: rank() <= CAST(5 AS BIGINT)
| | mem-estimate=0B mem-reservation=0B thread-reservation=0
| | tuple-ids=17,16 row-size=38B cardinality=3 cost=29
| | tuple-ids=17,16 row-size=38B cardinality=29 cost=29
| | in pipelines: 10(GETNEXT)
| |
| 11:ANALYTIC
@@ -740,10 +746,13 @@ max-parallelism=1530 segment-costs=[15280143040, 1707075] cpu-comparison-result=
| | tuple-ids=17,16 row-size=38B cardinality=29 cost=29
| | in pipelines: 10(GETNEXT)
| |
| 10:SORT
| | order by: tpcds_partitioned_parquet_snap.store_0.s_state ASC NULLS LAST, sum(tpcds_partitioned_parquet_snap.store_sales_0.ss_net_profit) DESC NULLS FIRST
| 10:TOP-N
| | partition by: tpcds_partitioned_parquet_snap.store_0.s_state
| | order by: sum(tpcds_partitioned_parquet_snap.store_sales_0.ss_net_profit) DESC NULLS FIRST
| | partition limit: 5 (include ties)
| | source expr: rank() <= CAST(5 AS BIGINT)
| | mem-estimate=12.00MB mem-reservation=12.00MB spill-buffer=2.00MB thread-reservation=0
| | tuple-ids=17 row-size=30B cardinality=29 cost=49
| | tuple-ids=17 row-size=30B cardinality=29 cost=79
| | in pipelines: 10(GETNEXT), 24(OPEN)
| |
| 24:AGGREGATE [FINALIZE]