diff --git a/axiom/optimizer/Cost.cpp b/axiom/optimizer/Cost.cpp index 0981dfd8..c313b5eb 100644 --- a/axiom/optimizer/Cost.cpp +++ b/axiom/optimizer/Cost.cpp @@ -36,11 +36,11 @@ void History::updateFromFile(const std::string& path) { } float shuffleCost(const ColumnVector& columns) { - return byteSize(columns); + return byteSize(columns) * Costs::kByteShuffleCost; } float shuffleCost(const ExprVector& exprs) { - return byteSize(exprs); + return byteSize(exprs) * Costs::kByteShuffleCost; } float selfCost(ExprCP expr) { diff --git a/axiom/optimizer/Cost.h b/axiom/optimizer/Cost.h index 8f1b99f8..31a0dc50 100644 --- a/axiom/optimizer/Cost.h +++ b/axiom/optimizer/Cost.h @@ -26,6 +26,8 @@ struct NodePrediction { float cardinality; /// Peak total memory for the top node. float peakMemory{0}; + /// CPU estimate in optimizer internal units. + float cpu{0}; }; /// Interface to historical query cost and cardinality @@ -103,23 +105,42 @@ struct Costs { : kLargeHashCost; } + static float hashBuildCost(float cardinality) { + // To build, a row is written once and read at least once. A write is ~2 + // redreads. + return 3 * hashProbeCost(cardinality); + } + static constexpr float kKeyCompareCost = 6; // ~30 instructions to find, decode and an compare static constexpr float kArrayProbeCost = 2; // ~10 instructions. - static constexpr float kSmallHashCost = 10; // 50 instructions - static constexpr float kLargeHashCost = 40; // 2 LLC misses + static constexpr float kSmallHashCost = 4; // 50 instructions + static constexpr float kLargeHashCost = 12; // 2 LLC misses static constexpr float kColumnRowCost = 5; static constexpr float kColumnByteCost = 0.1; /// Cost of hash function on one column. - static constexpr float kHashColumnCost = 0.5; + static constexpr float kHashColumnCost = 0.3; /// Cost of getting a column from a hash table - static constexpr float kHashExtractColumnCost = 0.5; + static constexpr float kHashExtractColumnCost = 0.3; + + /// Cost of sum/min/max. A little more than getting a value from the a hash + /// table. + static constexpr float kSimpleAggregateCost = kHashExtractColumnCost * 1.5; + + /// Bytes of overhead for a hash table row: ~12 bytes for the table and ~12 + /// bytes for the row. + static constexpr float kHashRowBytes = 24; /// Minimal cost of calling a filter function, e.g. comparing two numeric /// exprss. static constexpr float kMinimumFilterCost = 2; + + // Multiplier to apply to shuffle byte volume to get CPU cost. A + // complete cost model will need to consider the count of + // destinations, number of partition keys etc. + static constexpr float kByteShuffleCost = 0.3; }; /// Returns shuffle cost for a single row. Depends on the number of types of diff --git a/axiom/optimizer/DerivedTable.cpp b/axiom/optimizer/DerivedTable.cpp index 3376a83e..bb69b75e 100644 --- a/axiom/optimizer/DerivedTable.cpp +++ b/axiom/optimizer/DerivedTable.cpp @@ -296,7 +296,9 @@ void DerivedTable::import( PlanObjectCP firstTable, const PlanObjectSet& superTables, const std::vector& existences, - float existsFanout) { + float existsFanout, + PlanObjectSet extraConjuncts, + PlanObjectSet projected) { tableSet = superTables; tables = superTables.toObjects(); @@ -342,11 +344,24 @@ void DerivedTable::import( noImportOfExists = true; } + conjuncts = extraConjuncts.toObjects(); + if (firstTable->is(PlanType::kDerivedTableNode)) { importJoinsIntoFirstDt(firstTable->as()); } else { fullyImported = superTables; } + + // If there are many tables or we have a wrapped dt, columns should be + // explicitly projected out. + if (tables.size() > 1 || tables[0]->is(PlanType::kDerivedTableNode)) { + projected.forEach([&](auto column) { + if (std::find(columns.begin(), columns.end(), column) == columns.end()) { + exprs.push_back(column); + columns.push_back(column); + } + }); + } linkTablesToJoins(); } @@ -489,6 +504,12 @@ ExprCP replaceInputs(ExprCP expr, const T& source, const U& target) { } // namespace +bool DerivedTable::isWrapOnly() const { + return tables.size() == 1 && tables[0]->is(PlanType::kDerivedTableNode) && + !hasLimit() && !hasOrderBy() && conjuncts.empty() && !hasAggregation() && + exprs.empty(); +} + ExprCP DerivedTable::exportExpr(ExprCP expr) { return replaceInputs(expr, exprs, columns); } @@ -498,7 +519,7 @@ ExprCP DerivedTable::importExpr(ExprCP expr) { } void DerivedTable::importJoinsIntoFirstDt(const DerivedTable* firstDt) { - if (tables.size() == 1 && tables[0]->is(PlanType::kDerivedTableNode)) { + if (isWrapOnly()) { flattenDt(tables[0]->as()); return; } @@ -518,6 +539,9 @@ void DerivedTable::importJoinsIntoFirstDt(const DerivedTable* firstDt) { } auto* newFirst = make(*firstDt->as()); + newFirst->cname = firstDt->as()->cname; + newFirst->conjuncts = firstDt->conjuncts; + int32_t previousNumJoins = newFirst->joins.size(); for (auto& join : joins) { auto other = join->otherSide(firstDt); if (!other) { @@ -557,6 +581,7 @@ void DerivedTable::importJoinsIntoFirstDt(const DerivedTable* firstDt) { } } else { auto* chainDt = make(); + chainDt->cname = toName(fmt::format("rdt{}", chainDt->id())); PlanObjectSet chainSet; chainSet.add(other); if (fullyImported) { @@ -583,6 +608,9 @@ void DerivedTable::importJoinsIntoFirstDt(const DerivedTable* firstDt) { } } + for (auto i = previousNumJoins; i < newFirst->joins.size(); ++i) { + newFirst->joins[i]->guessFanout(); + } VELOX_CHECK_EQ(tables.size(), 1); for (auto i = 0; i < initialTables.size(); ++i) { if (!newFirst->fullyImported.contains(initialTables[i])) { @@ -599,12 +627,15 @@ void DerivedTable::flattenDt(const DerivedTable* dt) { tableSet = dt->tableSet; joins = dt->joins; joinOrder = dt->joinOrder; + conjuncts = dt->conjuncts; columns = dt->columns; exprs = dt->exprs; fullyImported = dt->fullyImported; importedExistences.unionSet(dt->importedExistences); aggregation = dt->aggregation; having = dt->having; + limit = dt->limit; + offset = dt->offset; } void DerivedTable::makeProjection(const ExprVector& exprs) { diff --git a/axiom/optimizer/DerivedTable.h b/axiom/optimizer/DerivedTable.h index e544c634..7180a682 100644 --- a/axiom/optimizer/DerivedTable.h +++ b/axiom/optimizer/DerivedTable.h @@ -165,7 +165,9 @@ struct DerivedTable : public PlanObject { PlanObjectCP firstTable, const PlanObjectSet& superTables, const std::vector& existences, - float existsFanout = 1); + float existsFanout = 1, + PlanObjectSet extraConjuncts = PlanObjectSet(), + PlanObjectSet columns = PlanObjectSet()); /// Return a copy of 'expr', replacing references to this DT's 'columns' with /// corresponding 'exprs'. @@ -212,6 +214,10 @@ struct DerivedTable : public PlanObject { return limit >= 0; } + // True if contains one derived table in 'tables' and adds no change to its + // result set. + bool isWrapOnly() const; + void addJoinedBy(JoinEdgeP join); /// Memoizes plans for 'this' and fills in 'cardinality'. Needed before adding diff --git a/axiom/optimizer/Optimization.cpp b/axiom/optimizer/Optimization.cpp index cbadc2c9..8f25f410 100644 --- a/axiom/optimizer/Optimization.cpp +++ b/axiom/optimizer/Optimization.cpp @@ -56,7 +56,6 @@ Optimization::Optimization( toGraph_.setDtOutput(root_, *logicalPlan_); } -// static PlanAndStats Optimization::toVeloxPlan( const logical_plan::LogicalPlanNode& logicalPlan, velox::memory::MemoryPool& pool, @@ -89,9 +88,21 @@ PlanAndStats Optimization::toVeloxPlan( std::move(runnerOptions)}; auto best = opt.bestPlan(); + opt.trace(OptimizerOptions::kRetained, 0, best->cost, *best->op); return opt.toVeloxPlan(best->op); } +std::string Optimization::memoString() const { + std::stringstream out; + for (auto& [key, planSet] : memo_) { + out << key.toString() << " plans= " << std::endl; + for (auto& plan : planSet.plans) { + out << plan->toString(true) << std::endl; + } + } + return out.str(); +} + void Optimization::trace( uint32_t event, int32_t id, @@ -140,7 +151,15 @@ void reducingJoinsRecursive( resultFunc = {}) { bool isLeaf = true; for (auto join : joinedBy(candidate)) { - if (join->leftOptional() || join->rightOptional()) { + if (join->isLeftOuter() && candidate == join->rightTable() && + candidate->is(PlanType::kDerivedTableNode)) { + // One can restrict the build of the optional side by a + // restriction on the probe. This happens specially when value + // subqueries are represented as optional sides of left + // oj. These are often aggregations and htere is no point + // creating values for groups that can't be probed. + ; + } else if (join->leftOptional() || join->rightOptional()) { continue; } JoinSide other = join->sideOf(candidate, true); @@ -192,6 +211,16 @@ void reducingJoinsRecursive( } } +bool allowReducingInnerJoins(const JoinCandidate& candidate) { + if (!candidate.join->isInner()) { + return false; + } + if (candidate.tables[0]->is(PlanType::kDerivedTableNode)) { + return false; + } + return true; +} + // For an inner join, see if can bundle reducing joins on the build. std::optional reducingJoins( const PlanState& state, @@ -202,7 +231,7 @@ std::optional reducingJoins( float fanout = candidate.fanout; PlanObjectSet reducingSet; - if (candidate.join->isInner()) { + if (allowReducingInnerJoins(candidate)) { PlanObjectSet visited = state.placed; VELOX_DCHECK(!candidate.tables.empty()); visited.add(candidate.tables[0]); @@ -317,23 +346,24 @@ void forJoinedTables(const PlanState& state, Func func) { } bool addExtraEdges(PlanState& state, JoinCandidate& candidate) { - // See if there are more join edges from the first of 'candidate' to already - // placed tables. Fill in the non-redundant equalities into the join edge. - // Make a new edge if the edge would be altered. + // See if there are more join edges from any of 'candidate' inner joined + // tables to already placed tables. Fill in the non-redundant equalities into + // the join edge. Make a new edge if the edge would be altered. auto* originalJoin = candidate.join; - auto* table = candidate.tables[0]; - for (auto* otherJoin : joinedBy(table)) { - if (otherJoin == originalJoin || !otherJoin->isInner()) { - continue; - } - auto [otherTable, fanout] = otherJoin->otherTable(table); - if (!state.dt->hasTable(otherTable)) { - continue; - } - if (candidate.isDominantEdge(state, otherJoin)) { - return false; + for (auto* table : candidate.tables) { + for (auto* otherJoin : joinedBy(table)) { + if (otherJoin == originalJoin || !otherJoin->isInner()) { + continue; + } + auto [otherTable, fanout] = otherJoin->otherTable(table); + if (!state.dt->hasTable(otherTable)) { + continue; + } + if (candidate.isDominantEdge(state, otherJoin)) { + break; + } + candidate.addEdge(state, otherJoin, table); } - candidate.addEdge(state, otherJoin); } return true; } @@ -913,7 +943,8 @@ void Optimization::addAggregation( std::move(finalGroupingKeys), std::move(aggregates), velox::core::AggregationNode::Step::kFinal, - aggPlan->columns()); + aggPlan->columns(), + partialAgg); state.addCost(*finalAgg); plan = finalAgg; @@ -1017,6 +1048,21 @@ void Optimization::joinByIndex( } } +namespace { +// Given a MemoKey for a build side, picks the deterministic conjuncts from +// 'state.dt' that are fully defined in terms of 'key.tables'. +void gatherConjunctsForKey(PlanState& state, MemoKey& key) { + for (auto& conjunct : state.dt->conjuncts) { + if (conjunct->containsFunction(FunctionSet::kNonDeterministic)) { + continue; + } + if (conjunct->allTables().isSubset(key.tables)) { + key.extraConjuncts.add(conjunct); + } + } +} +} // namespace + void Optimization::joinByHash( const RelationOpPtr& plan, const JoinCandidate& candidate, @@ -1046,7 +1092,13 @@ void Optimization::joinByHash( buildTables.add(buildTable); } + // The build side dt does not need to produce columns that it uses + // internally, only the columns that are downstream if we consider + // the build to be placed. So, provisionally mark build side tables + // as placed for the downstreamColumns(). + state.placed.unionSet(buildTables); buildColumns.intersect(state.downstreamColumns()); + state.placed.except(buildTables); buildColumns.unionColumns(build.keys); buildColumns.unionSet(buildFilterColumns); state.columns.unionSet(buildColumns); @@ -1054,6 +1106,9 @@ void Optimization::joinByHash( MemoKey memoKey{ candidate.tables[0], buildColumns, buildTables, candidate.existences}; + if (candidate.join->isInner()) { + gatherConjunctsForKey(state, memoKey); + } Distribution forBuild; if (plan->distribution().isGather()) { forBuild = Distribution::gather(); @@ -1075,7 +1130,7 @@ void Optimization::joinByHash( } else { state.placed.unionSet(buildTables); } - + state.placed.unionSet(memoKey.extraConjuncts); PlanState buildState(state.optimization, state.dt, buildPlan); RelationOpPtr buildInput = buildPlan->op; RelationOpPtr probeInput = plan; @@ -1238,7 +1293,6 @@ void Optimization::joinByHashRight( buildColumns.unionObjects(buildInput->columns()); const auto leftJoinType = probe.leftJoinType(); - const auto fanout = fanoutJoinTypeLimit(leftJoinType, candidate.fanout); // Change the join type to the right join variant. const auto rightJoinType = reverseJoinType(leftJoinType); @@ -1246,10 +1300,39 @@ void Optimization::joinByHashRight( leftJoinType != rightJoinType, "Join type does not have right hash join variant"); + float markTrueFraction = Value::kUnknown; const bool buildOnly = rightJoinType == velox::core::JoinType::kRightSemiFilter || rightJoinType == velox::core::JoinType::kRightSemiProject; + // Initialize fanout to invalid value, check that it is assigned after the + // below switch. + float fanout = -1; + switch (rightJoinType) { + case velox::core::JoinType::kRightSemiFilter: + fanout = 1.0 / candidate.fanout; + break; + case velox::core::JoinType::kRightSemiProject: + markTrueFraction = 1 / fanout; + fanout = state.cost.cardinality < 1 + ? 1 + : state.cost.cardinality / probePlan->cost.cardinality; + break; + case velox::core::JoinType::kRight: + // A right oj produces every probe side row plus unhit build side rows. + // rlFanout is the approximation but never < 1. + fanout = std::max(candidate.join->rlFanout(), 1); + break; + case velox::core::JoinType::kLeft: + // A right oj reversed produces every right side row and never limits the + // lrFanout. + fanout = std::max(candidate.join->lrFanout(), 1); + break; + default: + VELOX_UNREACHABLE("Bad right join type {}", rightJoinType); + } + VELOX_CHECK_GE(fanout, 0); + ColumnVector columns; PlanObjectSet columnSet; ColumnCP mark = nullptr; @@ -1272,7 +1355,7 @@ void Optimization::joinByHashRight( if (mark) { const_cast(&mark->value())->trueFraction = - std::min(1, candidate.fanout); + std::min(1, markTrueFraction); columns.push_back(mark); } @@ -1451,7 +1534,7 @@ void Optimization::placeDerivedTable(DerivedTableCP from, PlanState& state) { state.columns.unionSet(dtColumns); MemoKey key; - key.columns = std::move(dtColumns); + key.columns = dtColumns; key.firstTable = from; key.tables.add(from); @@ -1479,7 +1562,7 @@ void Optimization::placeDerivedTable(DerivedTableCP from, PlanState& state) { if (reduction < 0.9) { key.tables = reducingSet; - key.columns = state.downstreamColumns(); + key.columns = dtColumns; plan = makePlan(key, Distribution{}, PlanObjectSet{}, 1, state, ignore); // Not all reducing joins are necessarily retained in the plan. Only mark // the ones fully imported as placed. @@ -1768,7 +1851,7 @@ void Optimization::makeJoins(RelationOpPtr plan, PlanState& state) { } addPostprocess(dt, plan, state); - auto kept = state.plans.addPlan(plan, state); + auto kept = state.plans.addPlan(plan, state, isSingleWorker_); trace( kept ? OptimizerOptions::kRetained : OptimizerOptions::kExceededBest, dt->id(), @@ -1780,6 +1863,13 @@ void Optimization::makeJoins(RelationOpPtr plan, PlanState& state) { std::vector nextJoins; nextJoins.reserve(candidates.size()); for (auto& candidate : candidates) { + if (candidate.tables.size() > 1) { + // When there are multiple tables on the build side, we need to consider + // all edges that go from already placed tables to the bushy build side. + // So far, we have only filled in the edges for the first in + // 'candidate.tables'. + addExtraEdges(state, candidate); + } addJoin(candidate, plan, state, nextJoins); } @@ -1894,7 +1984,13 @@ PlanP Optimization::makeDtPlan( auto dt = make(); dt->cname = newCName("tmp_dt"); dt->import( - *state.dt, key.firstTable, key.tables, key.existences, existsFanout); + *state.dt, + key.firstTable, + key.tables, + key.existences, + existsFanout, + key.extraConjuncts, + key.columns); PlanState inner(*this, dt); if (key.firstTable->is(PlanType::kDerivedTableNode)) { diff --git a/axiom/optimizer/Optimization.h b/axiom/optimizer/Optimization.h index 4aca5ca7..999dd20e 100644 --- a/axiom/optimizer/Optimization.h +++ b/axiom/optimizer/Optimization.h @@ -173,6 +173,8 @@ class Optimization { void trace(uint32_t event, int32_t id, const PlanCost& cost, RelationOp& plan) const; + std::string memoString() const; + private: // Retrieves or makes a plan from 'key'. 'key' specifies a set of top level // joined tables or a hash join build side table or join. diff --git a/axiom/optimizer/ParallelExpr.cpp b/axiom/optimizer/ParallelExpr.cpp index 9a4c5b03..95bbe24b 100644 --- a/axiom/optimizer/ParallelExpr.cpp +++ b/axiom/optimizer/ParallelExpr.cpp @@ -172,13 +172,13 @@ velox::core::PlanNodePtr ToVelox::makeParallelProject( group->emplace_back(toTypedExpr(expr)); if (expr->is(PlanType::kColumnExpr)) { - names.push_back(expr->as()->outputName()); + names.push_back(sanitizeFieldName(expr->as()->outputName())); } else { names.push_back(fmt::format("__temp{}", expr->id())); } auto fieldAccess = std::make_shared( - group->back()->type(), names.back()); + group->back()->type(), sanitizeFieldName(names.back())); projectedExprs_[expr] = fieldAccess; } @@ -365,7 +365,7 @@ velox::core::PlanNodePtr ToVelox::maybeParallelProject( finalExprs.reserve(exprs.size()); for (auto i = 0; i < exprs.size(); ++i) { - names.emplace_back(columns[i]->outputName()); + names.emplace_back(sanitizeFieldName(columns[i]->outputName())); finalExprs.emplace_back(toTypedExpr(exprs[i])); } diff --git a/axiom/optimizer/Plan.cpp b/axiom/optimizer/Plan.cpp index 0b681f0b..581ce142 100644 --- a/axiom/optimizer/Plan.cpp +++ b/axiom/optimizer/Plan.cpp @@ -27,9 +27,22 @@ namespace { bool isSingleWorker() { return queryCtx()->optimization()->runnerOptions().numWorkers == 1; } - } // namespace +std::string MemoKey::toString() const { + std::stringstream out; + out << "{MemoKey Columns: "; + out << columns.toString(1) << " Tables " << tables.toString(1) + << " extraConjuncts=" << extraConjuncts.toString(1) << " "; + if (!existences.empty()) { + out << std::endl << " existences="; + for (auto& existence : existences) { + out << " exists= " << existence.toString(1) << std::endl; + } + } + return out.str(); +} + PlanState::PlanState(Optimization& optimization, DerivedTableCP dt) : optimization(optimization), dt(dt), @@ -121,6 +134,9 @@ std::string Plan::toString(bool detail) const { void PlanState::addCost(RelationOp& op) { cost.cost += op.cost().totalCost(); cost.cardinality = op.cost().resultCardinality(); + if (std::isnan(cost.cost) || std::isnan(cost.cardinality)) { + printf("bing\n"); + } } bool PlanState::mayConsiderNext(PlanObjectCP table) const { @@ -298,9 +314,15 @@ std::string PlanState::printPlan(RelationOpPtr op, bool detail) const { return plan->toString(detail); } -PlanP PlanSet::addPlan(RelationOpPtr plan, PlanState& state) { +PlanP PlanSet::addPlan( + RelationOpPtr plan, + PlanState& state, + bool isSingleWorker) { int32_t replaceIndex = -1; - const float shuffle = shuffleCost(plan->columns()) * state.cost.cardinality; + bool isRoot = state.dt->id() == 0; + const float shuffle = isSingleWorker + ? 0 + : shuffleCost(plan->columns()) * state.cost.cardinality; if (!plans.empty()) { // Compare with existing. If there is one with same distribution and new is @@ -315,7 +337,9 @@ PlanP PlanSet::addPlan(RelationOpPtr plan, PlanState& state) { const bool newIsBetter = old->isStateBetter(state); const bool newIsBetterWithShuffle = old->isStateBetter(state, shuffle); - const bool sameDist = + // We do not differentiate plans by their result distribution for root + // plans or single worker plans. + const bool sameDist = isRoot || isSingleWorker || old->op->distribution().isSamePartition(plan->distribution()); const bool sameOrder = old->op->distribution().isSameOrder(plan->distribution()); @@ -436,8 +460,10 @@ bool hasEqual(ExprCP key, const ExprVector& keys) { } } // namespace -void JoinCandidate::addEdge(PlanState& state, JoinEdgeP edge) { - auto* joined = tables[0]; +void JoinCandidate::addEdge( + PlanState& state, + JoinEdgeP edge, + PlanObjectCP joined) { auto newTableSide = edge->sideOf(joined); auto newPlacedSide = edge->sideOf(joined, true); VELOX_CHECK_NOT_NULL(newPlacedSide.table); @@ -518,6 +544,9 @@ bool NextJoin::isWorse(const NextJoin& other) const { size_t MemoKey::hash() const { size_t hash = tables.hash(); + if (!extraConjuncts.empty()) { + hash = velox::bits::commutativeHashMix(hash, extraConjuncts.hash()); + } for (auto& exists : existences) { hash = velox::bits::commutativeHashMix(hash, exists.hash()); } @@ -526,7 +555,7 @@ size_t MemoKey::hash() const { bool MemoKey::operator==(const MemoKey& other) const { if (firstTable == other.firstTable && columns == other.columns && - tables == other.tables) { + tables == other.tables && extraConjuncts == other.extraConjuncts) { if (existences.size() != other.existences.size()) { return false; } diff --git a/axiom/optimizer/Plan.h b/axiom/optimizer/Plan.h index 99d3b36d..4e52a5dc 100644 --- a/axiom/optimizer/Plan.h +++ b/axiom/optimizer/Plan.h @@ -96,9 +96,14 @@ struct PlanSet { } /// Compares 'plan' to already seen plans and retains it if it is - /// interesting, e.g. better than the best so far or has an interesting - /// order. Returns the plan if retained, nullptr if not. - PlanP addPlan(RelationOpPtr plan, PlanState& state); + /// interesting, e.g. better than the best so far or has an + /// interesting order or distribution. Returns the plan if retained, + /// nullptr if not. If 'isSingleWorker' is true, does not consider + /// the cost of shuffling the result when comparing plan costs, + /// i.e. if a plan + shuffle is cheaper than than an existing plan + /// with a different distribution, then the new cheaper plan always + /// wins. + PlanP addPlan(RelationOpPtr plan, PlanState& state, bool isSingleWorker); }; /// Represents the next table/derived table to join. May consist of several @@ -111,10 +116,10 @@ struct JoinCandidate { /// Second is the other side. std::pair joinSides() const; - /// Adds 'edge' to the set of joins between the new table and already placed + /// Adds 'edge' to the set of joins between 'joined' and already placed /// tables. a.k = b.k and c.k = b.k2 and c.k3 = a.k2. When placing c after a /// and b the edges to both a and b must be combined. - void addEdge(PlanState& state, JoinEdgeP edge); + void addEdge(PlanState& state, JoinEdgeP edge, PlanObjectCP joined); /// True if 'edge' has all the equalities to placed columns that 'join' of /// 'this' has and has more equalities. @@ -332,7 +337,11 @@ struct PlanStateSaver { /// result. For example, if a reducing join is moved below a group by, /// unless it is known never to have duplicates, it must become a /// semijoin and the original join must still stay in place in case -/// there were duplicates. +/// there were duplicates. 'extraConjuncts' is conjuncts from the dt containing +/// 'tables' which only depend on 'tables'. The typical use is that the key +/// represents a hash join build nd the conjuncts are non-join edge and +/// non-single table deterministic conjuncts that can be evaluated before the +/// build. struct MemoKey { bool operator==(const MemoKey& other) const; @@ -342,6 +351,9 @@ struct MemoKey { PlanObjectSet columns; PlanObjectSet tables; std::vector existences; + PlanObjectSet extraConjuncts; + + std::string toString() const; }; } // namespace facebook::axiom::optimizer diff --git a/axiom/optimizer/QueryGraph.h b/axiom/optimizer/QueryGraph.h index 6e85f24b..b06fad08 100644 --- a/axiom/optimizer/QueryGraph.h +++ b/axiom/optimizer/QueryGraph.h @@ -584,6 +584,10 @@ class JoinEdge { return rightExists_ || (markColumn_ != nullptr); } + bool isLeftOuter() const { + return rightOptional_ && !leftOptional_ && !isSemi() && !isAnti(); + } + bool isAnti() const { return rightNotExists_; } diff --git a/axiom/optimizer/RelationOp.cpp b/axiom/optimizer/RelationOp.cpp index 78d940b3..ded5bb95 100644 --- a/axiom/optimizer/RelationOp.cpp +++ b/axiom/optimizer/RelationOp.cpp @@ -470,6 +470,9 @@ std::string Join::toString(bool recursive, bool detail) const { << joinTypeLabel(joinType); printCost(detail, out); if (detail) { + out << "left: " << itemsToString(leftKeys.data(), leftKeys.size()) + << " = right: " << itemsToString(rightKeys.data(), rightKeys.size()) + << std::endl; out << "columns: " << itemsToString(columns().data(), columns().size()) << std::endl; } @@ -559,21 +562,123 @@ Unnest::Unnest( cost_.inputCardinality = inputCardinality(); } +namespace { +double partialFlushInterval( + double totalInput, + double numDistinct, + double maxDistinct) { + // Handle edge cases + if (maxDistinct >= numDistinct) { + return totalInput; + } + VELOX_CHECK_GT(maxDistinct, 0); + + // The expected number of samples to see k out of n distinct values + // follows from the coupon collector problem: + // E[k] = n * (1/n + 1/(n-1) + ... + 1/(n-k+1)) + + double n = numDistinct; + double k = maxDistinct; + + // Approximate the partial harmonic sum using logarithms (constant time): + // H(n,k) = Σ(i=0 to k-1) 1/(n-i) ≈ ln(n) - ln(n-k) = ln(n/(n-k)) + // This uses the integral approximation: ∫_{n-k}^n 1/x dx + double harmonicSum = std::log(n / (n - k)); + + // Expected number of samples in a uniform distribution + double expectedSamples = n * harmonicSum; + + // Scale by the ratio of total input to distinct values + // to account for non-uniform distribution + double scalingFactor = totalInput / numDistinct; + + return expectedSamples * scalingFactor; +} + +// Predicts the number of distinct values expected after sampling numRows items +// from a population with numDistinct distinct values. +// numRows: the count of samples that are initially seen +// numDistinct: the count of distinct values in the full population +// numSamples: the total count of samples in the population (unused in basic +// formula) Returns: predicted number of distinct values seen after numRows +// inputs +double expectedNumDistincts(double numRows, double numDistinct) { + if (numDistinct <= 0 || numRows <= 0) { + return 0.0f; + } + + // Using the coupon collector formula: + // Expected distinct values = d * (1 - (1 - 1/d)^n) + // where d is total distinct values and n is number of samples + return numDistinct * (1.0 - std::pow(1.0 - (1.0 / numDistinct), numRows)); +} +} // namespace + Aggregation::Aggregation( RelationOpPtr input, ExprVector groupingKeysVector, AggregateVector aggregatesVector, velox::core::AggregationNode::Step step, - ColumnVector columns) + ColumnVector columns, + const Aggregation* partial) : RelationOp{RelType::kAggregation, std::move(input), std::move(columns)}, groupingKeys{std::move(groupingKeysVector)}, aggregates{std::move(aggregatesVector)}, step{step} { cost_.inputCardinality = inputCardinality(); - float cardinality = 1; + // inputBeforePartial is the input cardinality before the partial aggregation + int64_t inputBeforePartial = partial != nullptr + ? partial->cost_.inputCardinality + : cost_.inputCardinality; + + auto numKeys = groupingKeys.size(); + + auto* optimization = queryCtx()->optimization(); + auto& veloxQueryCtx = optimization->veloxQueryCtx(); + const float maxPartialAggregationMemory = + veloxQueryCtx->queryConfig().maxPartialAggregationMemoryUsage(); + const float abandonPartialAggregationMinRows = + veloxQueryCtx->queryConfig().abandonPartialAggregationMinRows(); + const float abandonPartialAggregationMinPct = + veloxQueryCtx->queryConfig().abandonPartialAggregationMinPct(); + + const auto& runnerOptions = optimization->runnerOptions(); + int32_t width = runnerOptions.numWorkers * runnerOptions.numDrivers; + + if (numKeys > 0) { + setCostWithGroups( + inputBeforePartial, + width, + maxPartialAggregationMemory, + abandonPartialAggregationMinRows, + abandonPartialAggregationMinPct); + } else { + // Global aggregation (no grouping keys) + // Avoid division by zero + float safeInputCardinality = + std::max(1.0f, static_cast(cost_.inputCardinality)); + + cost_.unitCost = aggregates.size() * Costs::kSimpleAggregateCost; + cost_.fanout = 1.0f / safeInputCardinality; + } +} + +void Aggregation::setCostWithGroups( + int64_t inputBeforePartial, + int32_t width, + float maxPartialAggregationMemory, + float abandonPartialAggregationMinRows, + float abandonPartialAggregationMinPct) { + // Avoid division by zero + float safeInputBeforePartial = + std::max(1.0f, static_cast(inputBeforePartial)); + + auto numKeys = groupingKeys.size(); + + double maxCardinality = 1; for (auto key : groupingKeys) { - cardinality *= key->value().cardinality; + maxCardinality *= key->value().cardinality; } // The estimated output is input minus the times an input is a @@ -582,16 +687,50 @@ Aggregation::Aggregation( // potentially distinct keys and n is the number of elements in the // input. This approaches d as n goes to infinity. The chance of one in d // being unique after n values is 1 - (1/d)^n. - auto nOut = cardinality - - cardinality * - std::pow(1.0F - (1.0F / cardinality), input_->resultCardinality()); + auto nOut = expectedNumDistincts(maxCardinality, safeInputBeforePartial); - cost_.fanout = nOut / cost_.inputCardinality; - const auto numGrouppingKeys = static_cast(groupingKeys.size()); - cost_.unitCost = numGrouppingKeys * Costs::hashProbeCost(nOut); - - float rowBytes = byteSize(groupingKeys) + byteSize(aggregates); - cost_.totalBytes = nOut * rowBytes; + float rowBytes = + byteSize(groupingKeys) + byteSize(aggregates) + Costs::kHashRowBytes; + float partialCapacity = maxPartialAggregationMemory / rowBytes; + if (partialCapacity > nOut) { + partialCapacity = nOut; + } + auto maxInTable = step == velox::core::AggregationNode::Step::kPartial + ? partialCapacity + : nOut; + auto aggCost = aggregates.size() * Costs::kSimpleAggregateCost + + 2 * Costs::hashProbeCost(maxInTable); + + auto initialDistincts = + expectedNumDistincts(abandonPartialAggregationMinRows, nOut); + auto partialInput = + partialFlushInterval(safeInputBeforePartial, nOut, partialCapacity); + auto partialFanout = partialCapacity / partialInput; + + if ((safeInputBeforePartial > abandonPartialAggregationMinRows * width && + initialDistincts > abandonPartialAggregationMinRows * + (abandonPartialAggregationMinPct / 100)) || + (safeInputBeforePartial > nOut * 5 && + partialFanout > (abandonPartialAggregationMinPct / 100))) { + // Partial agg does not reduce. + partialFanout = 1; + } + if (step == velox::core::AggregationNode::Step::kPartial) { + cost_.fanout = partialFanout; + if (partialFanout == 1) { + cost_.unitCost = 0.1 * rowBytes; + } else { + cost_.unitCost = Costs::kHashColumnCost * numKeys + + Costs::hashProbeCost(partialCapacity) + aggCost; + cost_.totalBytes = partialCapacity * rowBytes; + } + } else { + cost_.totalBytes = nOut * rowBytes; + // auto in = cost_.inputCardinality / partialFanout; + cost_.unitCost = + Costs::kHashColumnCost * numKeys + Costs::hashProbeCost(nOut) + aggCost; + cost_.fanout = nOut / (safeInputBeforePartial * partialFanout); + } } std::string Unnest::toString(bool recursive, bool detail) const { @@ -679,7 +818,7 @@ HashBuild::HashBuild(RelationOpPtr input, ExprVector keysVector, PlanP plan) const auto numKeys = static_cast(keys.size()); const auto numColumns = static_cast(columns().size()); cost_.unitCost = numKeys * Costs::kHashColumnCost + - Costs::hashProbeCost(cost_.inputCardinality) + + Costs::hashBuildCost(cost_.inputCardinality) + numColumns * Costs::kHashExtractColumnCost * 2; cost_.totalBytes = cost_.inputCardinality * byteSize(columns()); } @@ -700,16 +839,31 @@ void HashBuild::accept( visitor.visit(*this, context); } +std::optional filterCardinality(ExprCP expr) { + // Covers the special case of a mark semijoin cardinality passed in + // trueFraction of mark column. + if (expr->value().trueFraction != Value::kUnknown) { + return expr->value().trueFraction; + } + return std::nullopt; +} + Filter::Filter(RelationOpPtr input, ExprVector exprs) : RelationOp{RelType::kFilter, std::move(input)}, exprs_{std::move(exprs)} { cost_.inputCardinality = inputCardinality(); const auto numExprs = static_cast(exprs_.size()); cost_.unitCost = Costs::kMinimumFilterCost * numExprs; - // We assume each filter selects 4/5. Small effect makes it so - // join and scan selectivities that are better known have more - // influence on plan cardinality. To be filled in from history. - cost_.fanout = std::pow(0.8F, numExprs); + cost_.fanout = 1; + for (auto& conjunct : exprs_) { + auto maybeCardinality = filterCardinality(conjunct); + // We assume each unknown filter selects 4/5. Small effect makes it so + // join and scan selectivities that are better known have more + // influence on plan cardinality. To be filled in from history. + + cost_.fanout *= + maybeCardinality.has_value() ? maybeCardinality.value() : 0.8; + } } const QGString& Filter::historyKey() const { @@ -821,7 +975,20 @@ OrderBy::OrderBy( limit{limit}, offset{offset} { cost_.inputCardinality = inputCardinality(); - cost_.fanout = 1; + if (limit == -1) { + cost_.fanout = 1; + } else { + const auto cardinality = static_cast(limit); + if (cost_.inputCardinality <= cardinality) { + // Input cardinality does not exceed the limit. The limit is no-op. + // Doesn't change cardinality. + cost_.fanout = 1; + } else { + // Input cardinality exceeds the limit. Calculate fanout to ensure that + // fanout * limit = input-cardinality. + cost_.fanout = cardinality / cost_.inputCardinality; + } + } // TODO Fill in cost_.unitCost and others. } diff --git a/axiom/optimizer/RelationOp.h b/axiom/optimizer/RelationOp.h index 90e22667..daa1dac2 100644 --- a/axiom/optimizer/RelationOp.h +++ b/axiom/optimizer/RelationOp.h @@ -529,7 +529,8 @@ struct Aggregation : public RelationOp { ExprVector groupingKeys, AggregateVector aggregates, velox::core::AggregationNode::Step step, - ColumnVector columns); + ColumnVector columns, + const Aggregation* partial = nullptr); const ExprVector groupingKeys; const AggregateVector aggregates; @@ -542,6 +543,14 @@ struct Aggregation : public RelationOp { void accept( const RelationOpVisitor& visitor, RelationOpVisitorContext& context) const override; + + private: + void setCostWithGroups( + int64_t inputBeforePartial, + int32_t width, + float maxPartialAggregationMemory, + float abandonPartialAggregationMinRows, + float abandonPartialAggregationMinPct); }; /// Represents an order by. The order is given by the distribution. diff --git a/axiom/optimizer/Schema.cpp b/axiom/optimizer/Schema.cpp index e265c567..8dbd584a 100644 --- a/axiom/optimizer/Schema.cpp +++ b/axiom/optimizer/Schema.cpp @@ -84,8 +84,10 @@ SchemaTableCP Schema::findTable( auto& tableColumns = connectorTable->columnMap(); schemaColumns.reserve(tableColumns.size()); for (const auto& [columnName, tableColumn] : tableColumns) { - const auto cardinality = static_cast(tableColumn->approxNumDistinct( - static_cast(connectorTable->numRows()))); + const auto cardinality = std::max( + tableColumn->approxNumDistinct( + static_cast(connectorTable->numRows())), + 1.0f); Value value(toType(tableColumn->type()), cardinality); auto* column = make(toName(columnName), nullptr, value); schemaColumns[column->name()] = column; diff --git a/axiom/optimizer/Schema.h b/axiom/optimizer/Schema.h index 561ae83e..ba1bcd8b 100644 --- a/axiom/optimizer/Schema.h +++ b/axiom/optimizer/Schema.h @@ -39,6 +39,8 @@ using NameMap = std::unordered_map< /// Represents constraints on a column value or intermediate result. struct Value { + static constexpr float kUnknown = -1; + Value(const velox::Type* type, float cardinality) : type{type}, cardinality{cardinality} {} @@ -56,8 +58,8 @@ struct Value { // Estimate of true fraction for booleans. 0 means always // false. This is an estimate and 1 or 0 do not allow pruning - // dependent code paths. - float trueFraction{1}; + // dependent code paths. kUnknown + float trueFraction{kUnknown}; // 0 means no nulls, 0.5 means half are null. float nullFraction{0}; @@ -285,7 +287,7 @@ float baseSelectivity(PlanObjectCP object); struct SchemaTable { explicit SchemaTable(const connector::Table& connectorTable) : connectorTable{&connectorTable}, - cardinality{static_cast(connectorTable.numRows())} {} + cardinality{std::max(connectorTable.numRows(), 1.0f)} {} ColumnGroupCP addIndex( const connector::TableLayout& layout, diff --git a/axiom/optimizer/ToGraph.cpp b/axiom/optimizer/ToGraph.cpp index a5279f4e..51e295f0 100644 --- a/axiom/optimizer/ToGraph.cpp +++ b/axiom/optimizer/ToGraph.cpp @@ -1213,7 +1213,10 @@ void ToGraph::translateJoin(const lp::JoinNode& join) { // TODO Allow mixing Unnest with Join in a single DT. // https://github.com/facebookexperimental/verax/issues/286 - const auto allowedInDt = allow(lp::NodeKind::kJoin); + const auto kAllowScan = + allow(lp::NodeKind::kTableScan) | allow(lp::NodeKind::kValues); + const auto allowedInDt = + allow(lp::NodeKind::kJoin) | allow(lp::NodeKind::kFilter) | kAllowScan; makeQueryGraph(*joinLeft, allowedInDt); // For an inner join a join tree on the right can be flattened, for all other @@ -1226,11 +1229,12 @@ void ToGraph::translateJoin(const lp::JoinNode& join) { isNondeterministicWrap_ = false; } + makeQueryGraph( *joinRight, (isInner && !queryCtx()->optimization()->options().syntacticJoinOrder) ? allowedInDt - : 0); + : kAllowScan); if (previousDt) { finalizeDt(*joinRight, previousDt); @@ -2035,6 +2039,14 @@ void ToGraph::makeQueryGraph( } isNondeterministicWrap_ = false; + + // For example on the right of left outer join, a filter must not go to + // the enclosing dt but must make its own dt. + if (!contains(allowedInDt, lp::NodeKind::kFilter)) { + wrapInDt(node); + return; + } + makeQueryGraph(*node.onlyInput(), allowedInDt); if (currentDt_->hasLimit()) { diff --git a/axiom/optimizer/ToVelox.cpp b/axiom/optimizer/ToVelox.cpp index 95c91b88..231728a0 100644 --- a/axiom/optimizer/ToVelox.cpp +++ b/axiom/optimizer/ToVelox.cpp @@ -15,6 +15,9 @@ */ #include "axiom/optimizer/ToVelox.h" +#include +#include +#include "axiom/optimizer/DerivedTablePrinter.h" #include "axiom/optimizer/FunctionRegistry.h" #include "axiom/optimizer/Optimization.h" #include "velox/core/PlanConsistencyChecker.h" @@ -53,6 +56,12 @@ ToVelox::ToVelox( isSingle_{options.numWorkers == 1}, subscript_{FunctionRegistry::instance()->subscript()} {} +std::string sanitizeFieldName(std::string_view name) { + std::string result(name); + std::replace(result.begin(), result.end(), '.', '_'); + return result; +} + namespace { std::vector columnSubfields( @@ -204,6 +213,11 @@ PlanAndStats ToVelox::toVeloxPlan( const runner::MultiFragmentPlan::Options& options) { options_ = options; + auto* opt = queryCtx()->optimization(); + if ((opt->options().traceFlags & OptimizerOptions::kRetained) != 0) { + std::cout << "Velox Plan: " << plan->toString(true, false) << std::endl; + } + prediction_.clear(); nodeHistory_.clear(); @@ -253,7 +267,7 @@ velox::RowTypePtr ToVelox::makeOutputType(const ColumnVector& columns) const { } auto name = makeVeloxExprWithNoAlias_ ? std::string(column->name()) : column->outputName(); - names.push_back(name); + names.push_back(sanitizeFieldName(name)); types.push_back(toTypePtr(columns[i]->value().type)); } return ROW(std::move(names), std::move(types)); @@ -313,7 +327,7 @@ velox::core::TypedExprPtr stepToGetter( auto& type = arg->type()->childAt( arg->type()->as().getChildIdx(step.field)); return std::make_shared( - type, arg, step.field); + type, arg, sanitizeFieldName(step.field)); } auto& type = arg->type()->childAt(step.id); return std::make_shared( @@ -426,10 +440,10 @@ velox::core::TypedExprPtr ToVelox::toTypedExpr(ExprCP expr) { auto it = columnAlteredTypes_.find(column); if (it != columnAlteredTypes_.end()) { return std::make_shared( - it->second, name); + it->second, sanitizeFieldName(name)); } return std::make_shared( - toTypePtr(expr->value().type), name); + toTypePtr(expr->value().type), sanitizeFieldName(name)); } case PlanType::kCallExpr: { std::vector inputs; @@ -471,7 +485,7 @@ velox::core::TypedExprPtr ToVelox::toTypedExpr(ExprCP expr) { return std::make_shared( toTypePtr(expr->value().type), toTypedExpr(expr->as()->base()), - field); + sanitizeFieldName(field)); } return std::make_shared( toTypePtr(expr->value().type), @@ -496,7 +510,7 @@ velox::core::TypedExprPtr ToVelox::toTypedExpr(ExprCP expr) { std::vector names; std::vector types; for (auto& c : lambda->args()) { - names.push_back(c->toString()); + names.push_back(sanitizeFieldName(c->toString())); types.push_back(toTypePtr(c->value().type)); } return std::make_shared( @@ -615,7 +629,7 @@ velox::core::FieldAccessTypedExprPtr ToVelox::toFieldRef(ExprCP expr) { auto column = expr->as(); return std::make_shared( - toTypePtr(column->value().type), column->outputName()); + toTypePtr(column->value().type), sanitizeFieldName(column->outputName())); } std::vector ToVelox::toFieldRefs( @@ -839,7 +853,7 @@ velox::RowTypePtr skylineStruct(BaseTableCP baseTable, ColumnCP column) { const auto& first = path->steps()[0]; auto name = first.field ? std::string{first.field} : fmt::format("{}", first.id); - names.push_back(name); + names.push_back(sanitizeFieldName(name)); types.push_back(valueType); }); @@ -862,7 +876,7 @@ velox::RowTypePtr ToVelox::subfieldPushdownScanType( } top.add(topColumn); topColumns.push_back(topColumn); - names.push_back(topColumn->name()); + names.push_back(sanitizeFieldName(topColumn->name())); if (isMapAsStruct(baseTable->schemaTable->name(), topColumn->name())) { types.push_back(skylineStruct(baseTable, topColumn)); typeMap[topColumn] = types.back(); @@ -874,7 +888,7 @@ velox::RowTypePtr ToVelox::subfieldPushdownScanType( continue; } topColumns.push_back(column); - names.push_back(column->name()); + names.push_back(sanitizeFieldName(column->name())); types.push_back(toTypePtr(column->value().type)); } } @@ -890,7 +904,7 @@ velox::core::PlanNodePtr ToVelox::makeSubfieldProjections( std::vector names; std::vector exprs; for (auto* column : scan.columns()) { - names.push_back(column->outputName()); + names.push_back(sanitizeFieldName(column->outputName())); exprs.push_back(toTypedExpr(column)); } return std::make_shared( @@ -943,7 +957,8 @@ velox::core::TypedExprPtr toAndWithAliases( for (const auto& column : baseTable->columns) { auto name = column->name(); mapping[name] = std::make_shared( - toTypePtr(column->value().type), column->outputName()); + toTypePtr(column->value().type), + sanitizeFieldName(column->outputName())); if (usedFieldNames.contains(name)) { if (!columnSet.contains(column)) { @@ -1070,7 +1085,7 @@ velox::core::PlanNodePtr ToVelox::makeProject( names.reserve(numOutputs); exprs.reserve(numOutputs); for (auto i = 0; i < numOutputs; ++i) { - names.push_back(project.columns()[i]->outputName()); + names.push_back(sanitizeFieldName(project.columns()[i]->outputName())); exprs.push_back(toTypedExpr(project.exprs()[i])); } @@ -1190,7 +1205,8 @@ velox::core::PlanNodePtr ToVelox::makeAggregation( type, aggregate->name(), std::make_shared( - toTypePtr(aggregate->intermediateType()), aggregateNames.back())); + toTypePtr(aggregate->intermediateType()), + sanitizeFieldName(aggregateNames.back()))); aggregates.push_back({.call = call, .rawInputTypes = rawInputTypes}); } } @@ -1217,7 +1233,7 @@ velox::core::PlanNodePtr ToVelox::makeAggregation( } } - return std::make_shared( + auto result = std::make_shared( nextId(), op.step, keys, @@ -1226,6 +1242,8 @@ velox::core::PlanNodePtr ToVelox::makeAggregation( aggregates, false, input); + makePredictionAndHistory(result->id(), &op); + return result; } velox::core::PlanNodePtr ToVelox::makeRepartition( @@ -1270,6 +1288,7 @@ velox::core::PlanNodePtr ToVelox::makeRepartition( exchangeSerdeKind_, sourcePlan); } + makePredictionAndHistory(source.fragment.planNode->id(), &repartition); if (exchange == nullptr) { exchange = std::make_shared( @@ -1390,7 +1409,7 @@ velox::core::PlanNodePtr ToVelox::makeWrite( inputNames.reserve(tableWrite.inputColumns.size()); inputTypes.reserve(tableWrite.inputColumns.size()); for (const auto* column : tableWrite.inputColumns) { - inputNames.push_back(column->as()->outputName()); + inputNames.push_back(sanitizeFieldName(column->as()->outputName())); inputTypes.push_back(toTypePtr(column->value().type)); } @@ -1451,7 +1470,10 @@ void ToVelox::makePredictionAndHistory( const velox::core::PlanNodeId& id, const RelationOp* op) { nodeHistory_[id] = op->historyKey(); - prediction_[id] = NodePrediction{.cardinality = op->resultCardinality()}; + prediction_[id] = NodePrediction{ + .cardinality = op->resultCardinality(), + .peakMemory = op->cost().totalBytes, + .cpu = op->cost().totalCost()}; } velox::core::PlanNodePtr ToVelox::makeFragment( @@ -1504,4 +1526,8 @@ extern std::string planString(const runner::MultiFragmentPlan* plan) { return plan->toString(true); } +extern std::string dtString(const DerivedTable* dt) { + return DerivedTablePrinter::toText(*dt); +} + } // namespace facebook::axiom::optimizer diff --git a/axiom/optimizer/ToVelox.h b/axiom/optimizer/ToVelox.h index 0c4b0ef9..1cc42eca 100644 --- a/axiom/optimizer/ToVelox.h +++ b/axiom/optimizer/ToVelox.h @@ -293,4 +293,8 @@ class ToVelox { runner::FinishWrite finishWrite_; }; +/// Replaces dots with underscores so that field names can be read back as names +/// and not as getters. +std::string sanitizeFieldName(std::string_view name); + } // namespace facebook::axiom::optimizer diff --git a/axiom/optimizer/VeloxHistory.cpp b/axiom/optimizer/VeloxHistory.cpp index bad637d8..134be355 100644 --- a/axiom/optimizer/VeloxHistory.cpp +++ b/axiom/optimizer/VeloxHistory.cpp @@ -127,8 +127,11 @@ bool VeloxHistory::setLeafSelectivity( return true; } + // When finding no hits, do not make a selectivity of 0 because this makes /0 + // or *0 and *0 is 0, which makes any subsequent operations 0 regardless of + // cost. So as not to underflow, count non-existent as 09 rows. table.filterSelectivity = - static_cast(sample.second) / static_cast(sample.first); + std::max(0.9f, sample.second) / static_cast(sample.first); recordLeafSelectivity(string, table.filterSelectivity, false); bool trace = (options.traceFlags & OptimizerOptions::kSample) != 0; diff --git a/axiom/optimizer/tests/AxiomSqlBenchmark.cpp b/axiom/optimizer/tests/AxiomSqlBenchmark.cpp index 38013f91..07385df8 100644 --- a/axiom/optimizer/tests/AxiomSqlBenchmark.cpp +++ b/axiom/optimizer/tests/AxiomSqlBenchmark.cpp @@ -20,6 +20,7 @@ #include #include #include +#include #include "axiom/connectors/SchemaResolver.h" #include "axiom/connectors/hive/LocalHiveConnectorMetadata.h" #include "axiom/connectors/tpch/TpchConnectorMetadata.h" @@ -64,6 +65,11 @@ DEFINE_bool(use_mmap, false, "Use mmap for buffers and cache"); DEFINE_uint32(optimizer_trace, 0, "Optimizer trace level"); +DEFINE_bool( + enable_reducing_existences, + true, + "Enable adding reducing semijoins into hash builds,a aggregations etc."); + DEFINE_bool(print_logical_plan, false, "Print logical plan (optimizer input)"); DEFINE_bool(print_plan, false, "Print optimizer results"); @@ -580,6 +586,11 @@ class VeloxRunner : public velox::QueryBenchmarkBase { auto session = std::make_shared(queryCtx->queryId()); + axiom::optimizer::OptimizerOptions optimizerOptions; + optimizerOptions.traceFlags = FLAGS_optimizer_trace; + optimizerOptions.enableReducingExistences = + FLAGS_enable_reducing_existences; + optimizer::Optimization optimization( session, *logicalPlan, @@ -587,7 +598,7 @@ class VeloxRunner : public velox::QueryBenchmarkBase { *history_, queryCtx, evaluator, - {.traceFlags = FLAGS_optimizer_trace}, + optimizerOptions, opts); if (checkDerivedTable && !checkDerivedTable(*optimization.rootDt())) { @@ -616,6 +627,32 @@ class VeloxRunner : public velox::QueryBenchmarkBase { static void printPlanWithStats( runner::LocalRunner& runner, const optimizer::NodePredictionMap& estimates) { + // Calculate predicted CPU total + float predictedCpu = 0; + for (auto& pair : estimates) { + predictedCpu += pair.second.cpu; + } + + // Get actual CPU timings from TaskStats + folly::F14FastMap nodeCpuNanos; + int64_t cpuNanos = 0; + + // Get TaskStats from runner and convert to PlanNodeStats + auto taskStats = runner.stats(); + for (const auto& taskStat : taskStats) { + auto planStats = velox::exec::toPlanStats(taskStat); + + // For each plan node, sum up CPU from addInput, getOutput, and finish + for (const auto& [nodeId, stats] : planStats) { + int64_t nodeCpu = stats.addInputTiming.cpuNanos + + stats.getOutputTiming.cpuNanos + stats.finishTiming.cpuNanos; + + // Accumulate (PlanNodeIds may not be unique across tasks) + nodeCpuNanos[nodeId] += nodeCpu; + cpuNanos += nodeCpu; + } + } + std::cout << runner.printPlanWithStats([&](const core::PlanNodeId& nodeId, std::string_view indentation, std::ostream& out) { @@ -623,9 +660,64 @@ class VeloxRunner : public velox::QueryBenchmarkBase { if (it != estimates.end()) { out << indentation << "Estimate: " << it->second.cardinality << " rows, " << succinctBytes(it->second.peakMemory) - << " peak memory" << std::endl; + << " peak memory, predicted cpu=" << std::fixed + << std::setprecision(2) << (it->second.cpu * 100.0f / predictedCpu) + << "%"; + + // Add actual CPU percentage + auto cpuIt = nodeCpuNanos.find(nodeId); + if (cpuIt != nodeCpuNanos.end() && cpuNanos > 0) { + out << ", actual cpu=" << std::fixed << std::setprecision(2) + << (static_cast(cpuIt->second) * 100.0f / cpuNanos) << "%"; + } + + out << std::endl; } }); + + // Print runtime stats grouped by operator if requested + if (FLAGS_include_custom_stats) { + std::cout << "\n" << std::string(80, '=') << "\n"; + std::cout << "Runtime Stats by Operator:\n"; + std::cout << std::string(80, '=') << "\n"; + + // Collect all runtime stats grouped by node ID + std::map> + allNodeStats; + + for (const auto& taskStat : taskStats) { + auto planStats = velox::exec::toPlanStats(taskStat); + + for (auto& [nodeId, stats] : planStats) { + if (!stats.customStats.empty()) { + // Merge custom stats for this node + for (const auto& [statName, statValue] : stats.customStats) { + auto& nodeStatMap = allNodeStats[nodeId]; + auto it = nodeStatMap.find(statName); + if (it == nodeStatMap.end()) { + nodeStatMap[statName] = statValue; + } else { + it->second.merge(statValue); + } + } + } + } + } + + // Print collected stats + for (const auto& [nodeId, customStats] : allNodeStats) { + std::cout << "\nNode " << nodeId << ":\n"; + for (const auto& [statName, statValue] : customStats) { + std::cout << " " << statName << ": " << statValue.toString() << "\n"; + } + } + + if (allNodeStats.empty()) { + std::cout << "\nNo custom runtime stats available.\n"; + } + + std::cout << std::string(80, '=') << "\n"; + } } std::shared_ptr makeRunner( diff --git a/axiom/optimizer/tests/CMakeLists.txt b/axiom/optimizer/tests/CMakeLists.txt index e570bf3a..c6c9a55b 100644 --- a/axiom/optimizer/tests/CMakeLists.txt +++ b/axiom/optimizer/tests/CMakeLists.txt @@ -44,7 +44,7 @@ target_link_libraries( gtest_main ) -add_library(axiom_optimizer_tests_plan_matcher PlanMatcher.cpp) +add_library(axiom_optimizer_tests_plan_matcher PlanMatcher.cpp PlanMatcherGenerator.cpp) target_link_libraries( axiom_optimizer_tests_plan_matcher @@ -94,6 +94,7 @@ add_executable( HiveQueriesTest.cpp PrecomputeProjectionTest.cpp PlanTest.cpp + PlanMatcherGeneratorTest.cpp RelationOpPrinterTest.cpp SetTest.cpp UnnestTest.cpp @@ -145,6 +146,43 @@ target_link_libraries( velox_parse_utils ) +target_link_libraries( + axiom_sql + axiom_runner_local_runner + axiom_runner_multifragment_plan + axiom_optimizer + axiom_hive_connector_metadata + axiom_tpch_connector_metadata + axiom_sql_presto_parser + velox_exec_test_lib + velox_dwio_common + velox_dwio_parquet_reader + velox_dwio_native_parquet_reader + velox_parse_parser + velox_parse_expression + velox_parse_utils +) + +add_executable(axiom_sql_benchmark AxiomSqlBenchmark.cpp linenoise/linenoise.c) + +target_link_libraries( + axiom_sql_benchmark + axiom_runner_local_runner + axiom_runner_multifragment_plan + axiom_optimizer + axiom_hive_connector_metadata + axiom_tpch_connector_metadata + axiom_sql_presto_parser + velox_query_benchmark + velox_exec_test_lib + velox_dwio_common + velox_dwio_parquet_reader + velox_dwio_native_parquet_reader + velox_parse_parser + velox_parse_expression + velox_parse_utils +) + add_subdirectory(utils) add_executable(axiom_model_test ModelTest.cpp) diff --git a/axiom/optimizer/tests/Console.cpp b/axiom/optimizer/tests/Console.cpp index 4a5af21b..fbfa0b86 100644 --- a/axiom/optimizer/tests/Console.cpp +++ b/axiom/optimizer/tests/Console.cpp @@ -31,6 +31,16 @@ DEFINE_uint64( DEFINE_uint32(optimizer_trace, 0, "Optimizer trace level"); +DEFINE_bool( + syntactic_join_order, + false, + "Disable cost-based join order selection and use syntactic order from query"); + +DEFINE_bool( + include_runtime_stats, + false, + "Include operator-specific runtime statistics in EXPLAIN ANALYZE output"); + DEFINE_int32(max_rows, 100, "Max number of printed result rows"); DEFINE_int32(num_workers, 4, "Number of in-process workers"); @@ -225,6 +235,8 @@ void Console::runNoThrow(std::string_view sql) { .numDrivers = FLAGS_num_drivers, .splitTargetBytes = FLAGS_split_target_bytes, .optimizerTraceFlags = FLAGS_optimizer_trace, + .includeRuntimeStats = FLAGS_include_runtime_stats, + .syntacticJoinOrder = FLAGS_syntactic_join_order, }); }, timing); diff --git a/axiom/optimizer/tests/HiveQueriesTestBase.cpp b/axiom/optimizer/tests/HiveQueriesTestBase.cpp index 7053e79f..f1869db4 100644 --- a/axiom/optimizer/tests/HiveQueriesTestBase.cpp +++ b/axiom/optimizer/tests/HiveQueriesTestBase.cpp @@ -57,7 +57,7 @@ RowTypePtr HiveQueriesTestBase::getSchema(std::string_view tableName) { ->type(); } -void HiveQueriesTestBase::checkResults( +axiom::optimizer::PlanAndStats HiveQueriesTestBase::checkResults( std::string_view sql, const core::PlanNodePtr& referencePlan) { SCOPED_TRACE(sql); @@ -65,11 +65,13 @@ void HiveQueriesTestBase::checkResults( auto statement = prestoParser_->parse(sql); - ASSERT_TRUE(statement->isSelect()); + VELOX_CHECK(statement->isSelect()); auto logicalPlan = statement->as<::axiom::sql::presto::SelectStatement>()->plan(); - checkSame(logicalPlan, referencePlan); + auto plan = planVelox(logicalPlan, {.numWorkers = 1, .numDrivers = 1}); + checkSame(plan, referencePlan); + return plan; } void HiveQueriesTestBase::checkResults( @@ -90,4 +92,21 @@ void HiveQueriesTestBase::checkSingleNodePlan( ASSERT_TRUE(matcher->match(fragments.at(0).fragment.planNode)); } +void HiveQueriesTestBase::explain( + std::string_view sql, + std::string* shortRel, + std::string* longRel, + std::string* graph, + const runner::MultiFragmentPlan::Options& runnerOptions, + const OptimizerOptions& optimizerOptions) { + auto statement = prestoParser_->parse(sql); + + VELOX_CHECK(statement->isSelect()); + auto logicalPlan = + statement->as<::axiom::sql::presto::SelectStatement>()->plan(); + + QueryTestBase::explain( + logicalPlan, shortRel, longRel, graph, runnerOptions, optimizerOptions); +} + } // namespace facebook::axiom::optimizer::test diff --git a/axiom/optimizer/tests/HiveQueriesTestBase.h b/axiom/optimizer/tests/HiveQueriesTestBase.h index 65965bcf..7f0c8c64 100644 --- a/axiom/optimizer/tests/HiveQueriesTestBase.h +++ b/axiom/optimizer/tests/HiveQueriesTestBase.h @@ -36,7 +36,7 @@ class HiveQueriesTestBase : public test::QueryTestBase { /// Returns a schema of a TPC-H table. velox::RowTypePtr getSchema(std::string_view tableName); - void checkResults( + axiom::optimizer::PlanAndStats checkResults( std::string_view sql, const velox::core::PlanNodePtr& referencePlan); @@ -50,6 +50,14 @@ class HiveQueriesTestBase : public test::QueryTestBase { return *prestoParser_; } + void explain( + std::string_view sql, + std::string* shortRel, + std::string* longRel, + std::string* graph, + const runner::MultiFragmentPlan::Options& runnerOptions = {}, + const OptimizerOptions& optimizerOptions = {}); + private: inline static std::shared_ptr gTempDirectory; diff --git a/axiom/optimizer/tests/PlanMatcherGenerator.cpp b/axiom/optimizer/tests/PlanMatcherGenerator.cpp new file mode 100644 index 00000000..aa6e4502 --- /dev/null +++ b/axiom/optimizer/tests/PlanMatcherGenerator.cpp @@ -0,0 +1,505 @@ +/* + * Copyright (c) Meta Platforms, Inc. and its affiliates. + * + * Licensed 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. + */ + +#include "axiom/optimizer/tests/PlanMatcherGenerator.h" +#include +#include +#include +#include "velox/core/PlanNode.h" + +namespace facebook::velox::core { + +namespace { + +/// Escapes a string for use in C++ string literals. +std::string escapeString(const std::string& str) { + std::ostringstream oss; + for (char c : str) { + switch (c) { + case '"': + oss << "\\\""; + break; + case '\\': + oss << "\\\\"; + break; + case '\n': + oss << "\\n"; + break; + case '\t': + oss << "\\t"; + break; + default: + oss << c; + } + } + return oss.str(); +} + +/// Generates a vector literal from a vector of strings. +std::string generateVectorLiteral(const std::vector& items) { + if (items.empty()) { + return "{}"; + } + + std::ostringstream oss; + oss << "{"; + for (size_t i = 0; i < items.size(); ++i) { + if (i > 0) { + oss << ", "; + } + oss << "\"" << escapeString(items[i]) << "\""; + } + oss << "}"; + return oss.str(); +} + +/// Generates code for a TableScanNode. +std::string generateTableScanCode(const TableScanNode& node) { + const auto& tableName = node.tableHandle()->name(); + + // Get the column names and types + + std::ostringstream oss; + oss << ".tableScan(\"" << escapeString(tableName) << "\")"; + + return oss.str(); +} + +/// Generates code for a ValuesNode. +std::string generateValuesCode(const ValuesNode& node) { + return ".values()"; +} + +/// Generates code for a FilterNode. +std::string generateFilterCode(const FilterNode& node) { + const auto& predicate = node.filter(); + std::ostringstream oss; + oss << ".filter(\"" << escapeString(predicate->toString()) << "\")"; + return oss.str(); +} + +/// Generates code for a ProjectNode. +std::string generateProjectCode(const ProjectNode& node) { + const auto& projections = node.projections(); + const auto& names = node.names(); + + std::vector expressions; + for (size_t i = 0; i < projections.size(); ++i) { + std::string exprStr = projections[i]->toString(); + // Add alias if the name differs from the expression + if (names[i] != exprStr) { + exprStr += " AS " + names[i]; + } + expressions.push_back(exprStr); + } + + std::ostringstream oss; + oss << ".project(" << generateVectorLiteral(expressions) << ")"; + return oss.str(); +} + +/// Generates code for a ParallelProjectNode. +std::string generateParallelProjectCode(const ParallelProjectNode& node) { + const auto& projections = node.projections(); + const auto& names = node.names(); + + std::vector expressions; + for (size_t i = 0; i < projections.size(); ++i) { + std::string exprStr = projections[i]->toString(); + if (names[i] != exprStr) { + exprStr += " AS " + names[i]; + } + expressions.push_back(exprStr); + } + + std::ostringstream oss; + oss << ".parallelProject(" << generateVectorLiteral(expressions) << ")"; + return oss.str(); +} + +/// Generates code for an AggregationNode. +std::string generateAggregationCode(const AggregationNode& node) { + const auto& groupingKeys = node.groupingKeys(); + const auto& aggregates = node.aggregates(); + const auto& aggregateNames = node.aggregateNames(); + const auto step = node.step(); + + std::vector groupingKeyStrs; + for (const auto& key : groupingKeys) { + groupingKeyStrs.push_back(key->toString()); + } + + std::vector aggregateStrs; + for (size_t i = 0; i < aggregates.size(); ++i) { + std::string aggStr = aggregates[i].call->toString(); + if (aggregateNames[i] != aggStr) { + aggStr += " AS " + aggregateNames[i]; + } + aggregateStrs.push_back(aggStr); + } + + std::ostringstream oss; + + // Determine the aggregation type + std::string stepMethod; + switch (step) { + case AggregationNode::Step::kSingle: + stepMethod = "singleAggregation"; + break; + case AggregationNode::Step::kPartial: + stepMethod = "partialAggregation"; + break; + case AggregationNode::Step::kFinal: + stepMethod = "finalAggregation"; + break; + case AggregationNode::Step::kIntermediate: + stepMethod = "aggregation"; // Use generic for intermediate + break; + } + + oss << "." << stepMethod << "("; + oss << generateVectorLiteral(groupingKeyStrs); + oss << ", "; + oss << generateVectorLiteral(aggregateStrs); + oss << ")"; + + return oss.str(); +} + +/// Generates code for an UnnestNode. +std::string generateUnnestCode(const UnnestNode& node) { + const auto& replicateVars = node.replicateVariables(); + const auto& unnestVars = node.unnestVariables(); + + std::vector replicateStrs; + for (const auto& var : replicateVars) { + replicateStrs.push_back(var->toString()); + } + + std::vector unnestStrs; + for (const auto& var : unnestVars) { + unnestStrs.push_back(var->toString()); + } + + std::ostringstream oss; + oss << ".unnest("; + oss << generateVectorLiteral(replicateStrs); + oss << ", "; + oss << generateVectorLiteral(unnestStrs); + oss << ")"; + + return oss.str(); +} + +/// Generates code for a LimitNode. +std::string generateLimitCode(const LimitNode& node) { + const auto offset = node.offset(); + const auto count = node.count(); + const auto isPartial = node.isPartial(); + + std::ostringstream oss; + if (isPartial) { + oss << ".partialLimit(" << offset << ", " << count << ")"; + } else { + oss << ".finalLimit(" << offset << ", " << count << ")"; + } + + return oss.str(); +} + +/// Generates code for a TopNNode. +std::string generateTopNCode(const TopNNode& node) { + const auto count = node.count(); + + std::ostringstream oss; + oss << ".topN(" << count << ")"; + + return oss.str(); +} + +/// Generates code for an OrderByNode. +std::string generateOrderByCode(const OrderByNode& node) { + const auto& sortingKeys = node.sortingKeys(); + const auto& sortingOrders = node.sortingOrders(); + + std::vector orderingStrs; + for (size_t i = 0; i < sortingKeys.size(); ++i) { + std::ostringstream oss; + oss << sortingKeys[i]->toString(); + oss << " " << (sortingOrders[i].isAscending() ? "ASC" : "DESC"); + oss << " NULLS " << (sortingOrders[i].isNullsFirst() ? "FIRST" : "LAST"); + orderingStrs.push_back(oss.str()); + } + + std::ostringstream oss; + oss << ".orderBy(" << generateVectorLiteral(orderingStrs) << ")"; + + return oss.str(); +} + +/// Generates code for a LocalPartitionNode. +std::string generateLocalPartitionCode(const LocalPartitionNode& node) { + return ".localPartition()"; +} + +/// Generates code for a LocalMergeNode. +std::string generateLocalMergeCode(const LocalMergeNode& node) { + return ".localMerge()"; +} + +/// Generates code for a PartitionedOutputNode. +std::string generatePartitionedOutputCode(const PartitionedOutputNode& node) { + return ".partitionedOutput()"; +} + +/// Generates code for an ExchangeNode. +std::string generateExchangeCode(const ExchangeNode& node) { + return ".exchange()"; +} + +/// Generates code for a MergeExchangeNode. +std::string generateMergeExchangeCode(const MergeExchangeNode& node) { + return ".mergeExchange()"; +} + +/// Generates code for a TableWriteNode. +std::string generateTableWriteCode(const TableWriteNode& node) { + return ".tableWrite()"; +} + +/// Forward declarations +std::string generatePlanMatcherCodeImpl( + const PlanNodePtr& planNode, + std::unordered_map& rightMatchers); +void generateJoinMatchers( + const PlanNodePtr& planNode, + std::vector& matchers, + int& matcherCounter, + std::unordered_map& rightMatchers); + +/// Generates the matcher variable name for a join side. +std::string getJoinMatcherVarName(int counter) { + if (counter == 0) { + return "rightMatcher"; + } + return "rightMatcher" + std::to_string(counter); +} + +/// Collects all join right-side matchers in the tree. +void generateJoinMatchers( + const PlanNodePtr& planNode, + std::vector& matchers, + int& matcherCounter, + std::unordered_map& rightMatchers) { + if (auto* joinNode = dynamic_cast(planNode.get())) { + VELOX_CHECK_EQ( + joinNode->sources().size(), 2, "HashJoinNode must have 2 sources"); + + // Recursively process left side for nested joins + generateJoinMatchers( + joinNode->sources()[0], matchers, matcherCounter, rightMatchers); + + // Generate matcher for the right side + const auto& rightSource = joinNode->sources()[1]; + + // First, collect any nested joins on the right side + generateJoinMatchers(rightSource, matchers, matcherCounter, rightMatchers); + + // Then generate the matcher for this right side + std::string rightMatcherCode = + generatePlanMatcherCodeImpl(rightSource, rightMatchers); + std::string matcherVar = getJoinMatcherVarName(matcherCounter++); + + // Record the mapping from right child PlanNode to matcher variable name + rightMatchers[rightSource.get()] = matcherVar; + + std::ostringstream oss; + oss << "auto " << matcherVar << " = core::PlanMatcherBuilder()"; + oss << rightMatcherCode; + oss << ".build();"; + + matchers.push_back(oss.str()); + } else { + // Recursively process sources + for (const auto& source : planNode->sources()) { + generateJoinMatchers(source, matchers, matcherCounter, rightMatchers); + } + } +} + +/// Generates code for a HashJoinNode (inline call only, not the right matcher). +std::string generateHashJoinCode( + const HashJoinNode& node, + const std::unordered_map& rightMatchers) { + // Look up the matcher variable name for this join's right child + const auto& rightSource = node.sources()[1]; + auto it = rightMatchers.find(rightSource.get()); + VELOX_CHECK(it != rightMatchers.end(), "Right matcher not found for join"); + std::string matcherVar = it->second; + + const auto joinType = node.joinType(); + std::ostringstream oss; + oss << ".hashJoin(" << matcherVar << ", velox::core::JoinType::"; + + // Map JoinType to its enum name + switch (joinType) { + case JoinType::kInner: + oss << "kInner"; + break; + case JoinType::kLeft: + oss << "kLeft"; + break; + case JoinType::kRight: + oss << "kRight"; + break; + case JoinType::kFull: + oss << "kFull"; + break; + case JoinType::kLeftSemiProject: + oss << "kLeftSemiProject"; + break; + case JoinType::kRightSemiProject: + oss << "kRightSemiProject"; + break; + case JoinType::kLeftSemiFilter: + oss << "kLeftSemiFilter"; + break; + case JoinType::kRightSemiFilter: + oss << "kRightSemiFilter"; + break; + case JoinType::kAnti: + oss << "kAnti"; + break; + default: + oss << "kInner"; // Default fallback + } + + oss << ")"; + + return oss.str(); +} + +/// Recursive implementation of generatePlanMatcherCode +std::string generatePlanMatcherCodeImpl( + const PlanNodePtr& planNode, + std::unordered_map& rightMatchers) { + std::ostringstream result; + + // Process sources first (post-order traversal for non-join nodes) + const auto& sources = planNode->sources(); + + // Special handling for joins which have two sources + if (auto* joinNode = dynamic_cast(planNode.get())) { + // For joins, we generate the left source inline + // The right source matcher is generated separately + if (!sources.empty()) { + result << generatePlanMatcherCodeImpl(sources[0], rightMatchers); + } + result << generateHashJoinCode(*joinNode, rightMatchers); + } else { + // For non-join nodes, process the first source recursively + if (!sources.empty()) { + result << generatePlanMatcherCodeImpl(sources[0], rightMatchers); + } + + // Generate code for the current node + if (auto* tableScan = dynamic_cast(planNode.get())) { + result << generateTableScanCode(*tableScan); + } else if (auto* values = dynamic_cast(planNode.get())) { + result << generateValuesCode(*values); + } else if (auto* filter = dynamic_cast(planNode.get())) { + result << generateFilterCode(*filter); + } else if ( + auto* parallelProject = + dynamic_cast(planNode.get())) { + result << generateParallelProjectCode(*parallelProject); + } else if ( + auto* project = dynamic_cast(planNode.get())) { + result << generateProjectCode(*project); + } else if ( + auto* agg = dynamic_cast(planNode.get())) { + result << generateAggregationCode(*agg); + } else if (auto* unnest = dynamic_cast(planNode.get())) { + result << generateUnnestCode(*unnest); + } else if (auto* limit = dynamic_cast(planNode.get())) { + result << generateLimitCode(*limit); + } else if (auto* topN = dynamic_cast(planNode.get())) { + result << generateTopNCode(*topN); + } else if ( + auto* orderBy = dynamic_cast(planNode.get())) { + result << generateOrderByCode(*orderBy); + } else if ( + auto* localPartition = + dynamic_cast(planNode.get())) { + result << generateLocalPartitionCode(*localPartition); + } else if ( + auto* localMerge = + dynamic_cast(planNode.get())) { + result << generateLocalMergeCode(*localMerge); + } else if ( + auto* partitionedOutput = + dynamic_cast(planNode.get())) { + result << generatePartitionedOutputCode(*partitionedOutput); + } else if ( + auto* mergeExchange = + dynamic_cast(planNode.get())) { + result << generateMergeExchangeCode(*mergeExchange); + } else if ( + auto* exchange = dynamic_cast(planNode.get())) { + result << generateExchangeCode(*exchange); + } else if ( + auto* tableWrite = + dynamic_cast(planNode.get())) { + result << generateTableWriteCode(*tableWrite); + } else { + // For unknown node types, add a comment + result << " // Unknown node type: " + << folly::demangle(typeid(*planNode).name()); + } + } + + return result.str(); +} + +} // namespace + +std::string generatePlanMatcherCode( + const PlanNodePtr& planNode, + const std::string& builderVarName) { + std::ostringstream oss; + + // Create the map to track right child PlanNode -> matcher variable name + std::unordered_map rightMatchers; + + // First, collect all join right-side matchers + std::vector joinMatchers; + int matcherCounter = 0; + generateJoinMatchers(planNode, joinMatchers, matcherCounter, rightMatchers); + + // Generate the join matchers first + for (const auto& matcher : joinMatchers) { + oss << matcher << "\n\n"; + } + + // Then generate the main matcher + oss << "auto " << builderVarName << " = core::PlanMatcherBuilder()"; + oss << generatePlanMatcherCodeImpl(planNode, rightMatchers); + oss << ".build();\n"; + + return oss.str(); +} + +} // namespace facebook::velox::core diff --git a/axiom/optimizer/tests/PlanMatcherGenerator.h b/axiom/optimizer/tests/PlanMatcherGenerator.h new file mode 100644 index 00000000..167d68da --- /dev/null +++ b/axiom/optimizer/tests/PlanMatcherGenerator.h @@ -0,0 +1,46 @@ +/* + * Copyright (c) Meta Platforms, Inc. and its affiliates. + * + * Licensed 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. + */ + +#pragma once + +#include +#include "velox/core/PlanNode.h" + +namespace facebook::velox::core { + +/// Generates C++ code that constructs a PlanMatcher tree matching the given +/// PlanNode tree. +/// +/// @param planNode The PlanNode tree to generate matcher code for. +/// @param builderVarName The name of the PlanMatcherBuilder variable to use +/// in the generated code (default: "builder"). +/// @return A string containing C++ code that uses PlanMatcherBuilder to +/// construct a matcher tree. +/// +/// Example: +/// auto plan = ...; // some PlanNode tree +/// std::string code = generatePlanMatcherCode(plan); +/// // code will contain something like: +/// // core::PlanMatcherBuilder() +/// // .tableScan("t") +/// // .filter("a > 10") +/// // .project({"a", "b"}) +/// // .build(); +std::string generatePlanMatcherCode( + const PlanNodePtr& planNode, + const std::string& builderVarName = "builder"); + +} // namespace facebook::velox::core diff --git a/axiom/optimizer/tests/PlanMatcherGenerator.md b/axiom/optimizer/tests/PlanMatcherGenerator.md new file mode 100644 index 00000000..ea008e21 --- /dev/null +++ b/axiom/optimizer/tests/PlanMatcherGenerator.md @@ -0,0 +1,195 @@ +# PlanMatcherGenerator + +A utility to generate C++ code for constructing `PlanMatcher` trees from `PlanNode` trees. + +## Overview + +The `PlanMatcherGenerator` takes a `velox::core::PlanNode` tree and generates C++ code that uses `PlanMatcherBuilder` to construct an equivalent `PlanMatcher` tree. This is useful for: + +- Creating test assertions from actual query plans +- Documenting expected plan structures +- Quickly generating matcher code instead of writing it manually + +## API + +### Main Function + +```cpp +std::string generatePlanMatcherCode( + const PlanNodePtr& planNode, + const std::string& builderVarName = "builder"); +``` + +**Parameters:** +- `planNode`: The root of the PlanNode tree to convert +- `builderVarName`: Optional name for the builder variable (default: "builder") + +**Returns:** A string containing C++ code that constructs a PlanMatcher tree + +## Usage Example + +```cpp +#include "axiom/optimizer/tests/PlanMatcherGenerator.h" + +// Create or obtain a PlanNode tree +auto logicalPlan = toLogicalPlan("SELECT name, age * 2 FROM users WHERE age > 18"); +auto plan = toSingleNodePlan(logicalPlan); + +// Generate the matcher code +std::string matcherCode = facebook::velox::core::generatePlanMatcherCode(plan); + +// Print the generated code +std::cout << matcherCode << std::endl; +``` + +**Output:** +```cpp +auto builder = core::PlanMatcherBuilder() + .tableScan("users") + .filter("age > 18") + .project({"name", "age * 2 AS c1"}) + .build(); +``` + +## Supported Node Types + +The generator supports the following PlanNode types: + +### Leaf Nodes +- **TableScanNode**: Generates `.tableScan("table_name")` +- **ValuesNode**: Generates `.values()` +- **ExchangeNode**: Generates `.exchange()` +- **MergeExchangeNode**: Generates `.mergeExchange()` + +### Unary Operators (Single Input) +- **FilterNode**: Generates `.filter("predicate")` +- **ProjectNode**: Generates `.project({"expr1", "expr2 AS alias"})` +- **ParallelProjectNode**: Generates `.parallelProject({"expr1", "expr2"})` +- **AggregationNode**: Generates aggregation calls based on step: + - `.singleAggregation({"key1", "key2"}, {"agg1", "agg2"})` + - `.partialAggregation({"key1"}, {"count(*) AS cnt"})` + - `.finalAggregation({"key1"}, {"count(a0) AS cnt"})` +- **UnnestNode**: Generates `.unnest({"replicate_cols"}, {"unnest_cols"})` +- **LimitNode**: Generates `.partialLimit(offset, count)` or `.finalLimit(offset, count)` +- **TopNNode**: Generates `.topN(count)` +- **OrderByNode**: Generates `.orderBy({"col1 ASC NULLS FIRST", "col2 DESC"})` +- **LocalPartitionNode**: Generates `.localPartition()` +- **LocalMergeNode**: Generates `.localMerge()` +- **PartitionedOutputNode**: Generates `.partitionedOutput()` +- **TableWriteNode**: Generates `.tableWrite()` + +### Binary Operators (Two Inputs) +- **HashJoinNode**: Generates code with a separate matcher for the right side: + ```cpp + auto rightMatcher = core::PlanMatcherBuilder() + .tableScan("right_table") + .build(); + + auto builder = core::PlanMatcherBuilder() + .tableScan("left_table") + .hashJoin(rightMatcher, JoinType::kInner) + .build(); + ``` + +## Integration + +### Adding to Your Test File + +1. Include the header: + ```cpp + #include "axiom/optimizer/tests/PlanMatcherGenerator.h" + ``` + +2. Generate matcher code for your plan: + ```cpp + auto plan = ...; // your PlanNode + auto code = facebook::velox::core::generatePlanMatcherCode(plan); + std::cout << code << std::endl; + ``` + +3. Copy the generated code into your test: + ```cpp + TEST_F(MyTest, testQuery) { + auto plan = toSingleNodePlan(toLogicalPlan("SELECT ...")); + + // Generated code goes here: + auto matcher = core::PlanMatcherBuilder() + .tableScan("my_table") + .filter("condition") + .build(); + + AXIOM_ASSERT_PLAN(plan, matcher); + } + ``` + +## Advanced Usage + +### Customizing the Builder Variable Name + +```cpp +std::string code = generatePlanMatcherCode(plan, "myMatcher"); +// Generates: auto myMatcher = core::PlanMatcherBuilder()... +``` + +### Handling Complex Joins + +For join nodes, the generator creates a separate matcher for the right side: + +```cpp +// Input: A join between two table scans +auto plan = ...; // SELECT * FROM a JOIN b ON a.id = b.id + +// Generated output: +auto rightMatcher = core::PlanMatcherBuilder() + .tableScan("b") + .build(); + +auto builder = core::PlanMatcherBuilder() + .tableScan("a") + .hashJoin(rightMatcher, JoinType::kInner) + .build(); +``` + +## Implementation Details + +The generator uses the following approach: + +1. **Post-order Traversal**: Processes the PlanNode tree from leaves to root +2. **Dynamic Type Detection**: Uses `dynamic_cast` to identify node types +3. **Expression Conversion**: Converts `TypedExpr` objects to string representations +4. **Code Generation**: Builds up the PlanMatcherBuilder chain + +### Node Type Handling + +For each node type, the generator: +- Extracts relevant properties (predicates, expressions, keys, etc.) +- Formats them as C++ string literals or vectors +- Generates the appropriate PlanMatcherBuilder method call + +## Limitations + +1. **Expression Fidelity**: The generator uses `toString()` on expressions, which may not exactly match the original SQL syntax +2. **Type Information**: Currently doesn't generate detailed type information for table scans +3. **Complex Filters**: Subfield filters in HiveScan nodes are not yet supported +4. **Unknown Node Types**: Generates a comment for unsupported node types + +## Future Enhancements + +Potential improvements: +- Add support for more specialized matchers (e.g., `hiveScan` with filters) +- Include output type information where relevant +- Better formatting for complex expressions +- Support for window functions and other advanced features + +## Files + +- `PlanMatcherGenerator.h`: Header file with function declaration +- `PlanMatcherGenerator.cpp`: Implementation +- `PlanMatcherGeneratorTest.cpp`: Unit tests +- `PlanMatcherGeneratorExample.cpp`: Usage examples + +## See Also + +- `PlanMatcher.h`: PlanMatcher interface and PlanMatcherBuilder +- `PlanMatcher.cpp`: PlanMatcher implementations +- `velox/core/PlanNode.h`: PlanNode definitions diff --git a/axiom/optimizer/tests/PlanMatcherGeneratorExample.cpp b/axiom/optimizer/tests/PlanMatcherGeneratorExample.cpp new file mode 100644 index 00000000..9702ba45 --- /dev/null +++ b/axiom/optimizer/tests/PlanMatcherGeneratorExample.cpp @@ -0,0 +1,75 @@ +/* + * Copyright (c) Meta Platforms, Inc. and its affiliates. + * + * Licensed 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. + */ + +#include +#include "axiom/optimizer/tests/PlanMatcherGenerator.h" + +/// Example usage of generatePlanMatcherCode +/// +/// This example shows how to use the PlanMatcherGenerator to convert a +/// PlanNode tree into C++ code that constructs an equivalent PlanMatcher. +/// +/// Usage: +/// // Given a PlanNode tree: +/// auto plan = ...; // your PlanNode tree +/// +/// // Generate the matcher code: +/// std::string matcherCode = +/// facebook::velox::core::generatePlanMatcherCode(plan); +/// +/// // Print or save the generated code: +/// std::cout << matcherCode << std::endl; +/// +/// The generated code will look something like: +/// +/// auto builder = core::PlanMatcherBuilder() +/// .tableScan("my_table") +/// .filter("age > 18") +/// .project({"name", "age * 2 AS double_age"}) +/// .singleAggregation({"name"}, {"count(*) AS cnt"}) +/// .build(); +/// +/// You can then copy this code into your test file and use it to match plans. + +int main() { + std::cout << "PlanMatcherGenerator - Example Usage" << std::endl; + std::cout << "=====================================" << std::endl; + std::cout << std::endl; + std::cout << "To use the PlanMatcherGenerator:" << std::endl; + std::cout << std::endl; + std::cout << "1. Include the header:" << std::endl; + std::cout << " #include \"axiom/optimizer/tests/PlanMatcherGenerator.h\"" + << std::endl; + std::cout << std::endl; + std::cout << "2. Call the function with your PlanNode:" << std::endl; + std::cout << " auto plan = ...; // your PlanNode tree" << std::endl; + std::cout + << " std::string code = facebook::velox::core::generatePlanMatcherCode(plan);" + << std::endl; + std::cout << std::endl; + std::cout << "3. Use the generated code in your tests:" << std::endl; + std::cout << " std::cout << code << std::endl;" << std::endl; + std::cout << std::endl; + std::cout << "Example output:" << std::endl; + std::cout << " auto matcher = core::PlanMatcherBuilder()" << std::endl; + std::cout << " .tableScan(\"employees\")" << std::endl; + std::cout << " .filter(\"salary > 50000\")" << std::endl; + std::cout << " .project({\"name\", \"salary\"})" << std::endl; + std::cout << " .build();" << std::endl; + std::cout << std::endl; + + return 0; +} diff --git a/axiom/optimizer/tests/PlanMatcherGeneratorTest.cpp b/axiom/optimizer/tests/PlanMatcherGeneratorTest.cpp new file mode 100644 index 00000000..c500b1f3 --- /dev/null +++ b/axiom/optimizer/tests/PlanMatcherGeneratorTest.cpp @@ -0,0 +1,97 @@ +/* + * Copyright (c) Meta Platforms, Inc. and its affiliates. + * + * Licensed 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. + */ + +#include "axiom/optimizer/tests/PlanMatcherGenerator.h" +#include +#include +#include "axiom/logical_plan/PlanBuilder.h" +#include "axiom/optimizer/tests/HiveQueriesTestBase.h" +#include "velox/exec/tests/utils/PlanBuilder.h" + +namespace facebook::axiom::optimizer { +namespace { + +using namespace facebook::velox; +namespace lp = facebook::axiom::logical_plan; + +class PlanMatcherGeneratorTest : public test::HiveQueriesTestBase {}; + +TEST_F(PlanMatcherGeneratorTest, filterProject) { + // Test with an actual SQL query - parse and get logical plan + auto statement = + prestoParser().parse("SELECT n_name FROM nation WHERE n_regionkey > 2"); + ASSERT_TRUE(statement->isSelect()); + auto logicalPlan = + statement->as<::axiom::sql::presto::SelectStatement>()->plan(); + + // Convert to single node plan + auto plan = toSingleNodePlan(logicalPlan); + + // Generate the matcher code + auto code = core::generatePlanMatcherCode(plan); + + std::cout + << "Generated code for 'SELECT n_name FROM nation WHERE n_regionkey > 2':\n" + << code << std::endl; + + EXPECT_FALSE(code.empty()); +} + +TEST_F(PlanMatcherGeneratorTest, aggregation) { + // Test with an aggregation query + auto statement = prestoParser().parse( + "SELECT n_regionkey, count(*) FROM nation GROUP BY n_regionkey"); + ASSERT_TRUE(statement->isSelect()); + auto logicalPlan = + statement->as<::axiom::sql::presto::SelectStatement>()->plan(); + + // Convert to single node plan + auto plan = toSingleNodePlan(logicalPlan); + + // Generate the matcher code + auto code = core::generatePlanMatcherCode(plan); + + std::cout + << "Generated code for 'SELECT n_regionkey, count(*) FROM nation GROUP BY n_regionkey':\n" + << code << std::endl; + + EXPECT_FALSE(code.empty()); + EXPECT_TRUE(code.find("Aggregation") != std::string::npos); +} + +TEST_F(PlanMatcherGeneratorTest, join) { + // Test with a join query + auto statement = prestoParser().parse( + "SELECT n_name, r_name FROM nation, region WHERE n_regionkey = r_regionkey"); + ASSERT_TRUE(statement->isSelect()); + auto logicalPlan = + statement->as<::axiom::sql::presto::SelectStatement>()->plan(); + + // Convert to single node plan + auto plan = toSingleNodePlan(logicalPlan); + + // Generate the matcher code + auto code = core::generatePlanMatcherCode(plan); + + std::cout << "Generated code for join query:\n" << code << std::endl; + + EXPECT_FALSE(code.empty()); + EXPECT_TRUE(code.find("hashJoin") != std::string::npos); + EXPECT_TRUE(code.find("rightMatcher") != std::string::npos); +} + +} // namespace +} // namespace facebook::axiom::optimizer diff --git a/axiom/optimizer/tests/PlanTest.cpp b/axiom/optimizer/tests/PlanTest.cpp index f80a0d4d..94ddfa71 100644 --- a/axiom/optimizer/tests/PlanTest.cpp +++ b/axiom/optimizer/tests/PlanTest.cpp @@ -26,6 +26,9 @@ #include "velox/exec/tests/utils/TpchQueryBuilder.h" #include "velox/type/tests/SubfieldFiltersBuilder.h" +#define AXIOM_ASSERT_PLAN(plan, matcher) \ + ASSERT_TRUE(matcher->match(plan)) << plan->toString(true, true); + namespace facebook::axiom::optimizer { namespace { @@ -1115,6 +1118,123 @@ TEST_F(PlanTest, lambdaArgs) { AXIOM_ASSERT_PLAN(plan, matcher); } +TEST_F(PlanTest, joinWithFilterOverLimit) { + testConnector_->addTable("t", ROW({"a", "b", "c"}, BIGINT())); + testConnector_->addTable("u", ROW({"x", "y", "z"}, BIGINT())); + + lp::PlanBuilder::Context ctx(kTestConnectorId); + auto logicalPlan = + lp::PlanBuilder(ctx) + .tableScan("t") + .limit(100) + .filter("b > 50") + .join( + lp::PlanBuilder(ctx).tableScan("u").limit(50).filter("y < 100"), + "a = x", + lp::JoinType::kInner) + .build(); + + { + auto plan = toSingleNodePlan(logicalPlan); + auto matcher = core::PlanMatcherBuilder() + .tableScan("t") + .limit() + .filter("b > 50") + .hashJoin( + core::PlanMatcherBuilder() + .tableScan("u") + .limit() + .filter("y < 100") + .build()) + .build(); + + AXIOM_ASSERT_PLAN(plan, matcher); + } +} + +TEST_F(PlanTest, outerJoinWithInnerJoin) { + testConnector_->addTable("t", ROW({"a", "b", "c"}, BIGINT())); + testConnector_->addTable("v", ROW({"vx", "vy", "vz"}, BIGINT())); + testConnector_->addTable("u", ROW({"x", "y", "z"}, BIGINT())); + + lp::PlanBuilder::Context ctx(kTestConnectorId); + auto logicalPlan = lp::PlanBuilder(ctx) + .tableScan("t") + .filter("b > 50") + .join( + lp::PlanBuilder(ctx).tableScan("u").join( + lp::PlanBuilder(ctx).tableScan("v"), + "x = vx", + lp::JoinType::kInner), + "a = x", + lp::JoinType::kLeft) + .build(); + + { + SCOPED_TRACE("left join with inner join on right"); + + auto plan = toSingleNodePlan(logicalPlan); + + // The expected plan is a right oj with T, which has the filter on the build + // side and the wider row with no filter on the probe side. + auto matcher = + core::PlanMatcherBuilder() + .tableScan("u") + .hashJoin(core::PlanMatcherBuilder().tableScan("v").build()) + .hashJoin( + core::PlanMatcherBuilder() + .tableScan("t") + .filter("b > 50") + + .build()) + + .build(); + AXIOM_ASSERT_PLAN(plan, matcher); + } + + logicalPlan = lp::PlanBuilder(ctx) + .tableScan("t") + .filter("b > 50") + .aggregate({"a", "b"}, {"sum(c)"}) + .join( + lp::PlanBuilder(ctx) + .tableScan("u") + .join( + lp::PlanBuilder(ctx).tableScan("v"), + "x = vx", + lp::JoinType::kInner) + .filter("not(x = vy)"), + "a = x", + lp::JoinType::kLeft) + .build(); + + { + SCOPED_TRACE("Aggregation left join filter over inner join"); + auto plan = toSingleNodePlan(logicalPlan); + + // Expect a right oj with the aggregation and filter on the build side and + // the join to the right on probe. The aggregation is expected to be the + // narrower row. + auto matcher = + core::PlanMatcherBuilder() + .tableScan("u") + .hashJoin(core::PlanMatcherBuilder().tableScan("v").build()) + .filter() + .hashJoin( + core::PlanMatcherBuilder() + .tableScan("t") + .filter() + .aggregation() + .build()) + .project() + .build(); + + AXIOM_ASSERT_PLAN(plan, matcher); + } +} + +#undef AXIOM_ASSERT_PLAN + } // namespace } // namespace facebook::axiom::optimizer diff --git a/axiom/optimizer/tests/QueryTestBase.cpp b/axiom/optimizer/tests/QueryTestBase.cpp index 6878b91b..e00b4d72 100644 --- a/axiom/optimizer/tests/QueryTestBase.cpp +++ b/axiom/optimizer/tests/QueryTestBase.cpp @@ -15,7 +15,10 @@ */ #include "axiom/optimizer/tests/QueryTestBase.h" +#include +#include #include "axiom/connectors/SchemaResolver.h" +#include "axiom/optimizer/DerivedTablePrinter.h" #include "axiom/optimizer/Optimization.h" #include "axiom/optimizer/Plan.h" #include "axiom/optimizer/VeloxHistory.h" @@ -280,4 +283,123 @@ velox::core::PlanNodePtr QueryTestBase::toSingleNodePlan( return plan->fragments().at(0).fragment.planNode; } +void QueryTestBase::checkPlanText( + const velox::core::PlanNodePtr& plan, + const std::vector& expected, + bool negative) { + // Convert plan to string with details + auto planString = plan->toString(true, true); + + // Split into lines + std::vector lines; + std::istringstream stream(planString); + std::string line; + while (std::getline(stream, line)) { + lines.push_back(line); + } + + // Search for patterns in order + size_t currentLine = 0; + size_t matchedPattern = 0; + int lastMatchLine = -1; + + for (size_t i = 0; i < expected.size(); ++i) { + RE2 pattern(expected[i]); + if (!pattern.ok()) { + FAIL() << "Invalid regex pattern at index " << i << ": " << expected[i] + << " - " << pattern.error(); + return; + } + + bool found = false; + for (size_t lineIdx = currentLine; lineIdx < lines.size(); ++lineIdx) { + if (RE2::PartialMatch(lines[lineIdx], pattern)) { + found = true; + lastMatchLine = lineIdx; + currentLine = lineIdx + 1; + matchedPattern = i + 1; + break; + } + } + + if (!found) { + // Pattern not found + if (!negative) { + // Expected to find all patterns - fail + std::string errorMsg = + fmt::format("Pattern {} not found: '{}'\n", i, expected[i]); + if (lastMatchLine >= 0) { + errorMsg += fmt::format( + "Previous pattern ({}) matched at line {}\n", + i - 1, + lastMatchLine); + } + errorMsg += fmt::format("\nFull plan:\n{}", planString); + FAIL() << errorMsg; + } else { + // Expected NOT to find all patterns - success (at least one missing) + return; + } + } + } + + // All patterns were found in order + if (negative) { + // Expected NOT to find all patterns - fail + FAIL() << fmt::format( + "All {} patterns were found in order (expected them NOT to be found):\n{}", + expected.size(), + planString); + } + // else: success (all patterns found and negative=false) +} + +void QueryTestBase::explain( + const logical_plan::LogicalPlanNodePtr& query, + std::string* shortRel, + std::string* longRel, + std::string* graph, + const runner::MultiFragmentPlan::Options& runnerOptions, + const OptimizerOptions& optimizerOptions) { + auto& queryCtx = getQueryCtx(); + + auto allocator = std::make_unique(optimizerPool_.get()); + auto context = std::make_unique(*allocator); + optimizer::queryCtx() = context.get(); + SCOPE_EXIT { + optimizer::queryCtx() = nullptr; + }; + exec::SimpleExpressionEvaluator evaluator( + queryCtx.get(), optimizerPool_.get()); + + auto session = std::make_shared(queryCtx->queryId()); + + connector::SchemaResolver schemaResolver; + + optimizer::Optimization opt( + session, + *query, + schemaResolver, + *history_, + queryCtx, + evaluator, + optimizerOptions, + runnerOptions); + + auto best = opt.bestPlan(); + + if (shortRel) { + *shortRel = best->op->toString(true, false); + } + + if (longRel) { + *longRel = best->op->toString(true, true); + } + + if (graph) { + auto rootDt = opt.rootDt(); + *graph = DerivedTablePrinter::toText(*rootDt); + } +} + } // namespace facebook::axiom::optimizer::test diff --git a/axiom/optimizer/tests/QueryTestBase.h b/axiom/optimizer/tests/QueryTestBase.h index c06a1a3c..8ec09124 100644 --- a/axiom/optimizer/tests/QueryTestBase.h +++ b/axiom/optimizer/tests/QueryTestBase.h @@ -126,6 +126,25 @@ class QueryTestBase : public runner::test::LocalRunnerTestBase { const logical_plan::LogicalPlanNodePtr& logicalPlan, int32_t numDrivers = 1); + /// Checks that a plan contains (or doesn't contain) expected patterns. + /// @param plan The plan node to check. + /// @param expected Vector of regex patterns (RE2) to search for. + /// @param negative If false (default), expects all patterns to be found in + /// order. If true, expects that NOT all patterns are found in order (i.e., + /// at least one pattern is missing or out of order). + void checkPlanText( + const velox::core::PlanNodePtr& plan, + const std::vector& expected, + bool negative = false); + + void explain( + const logical_plan::LogicalPlanNodePtr& query, + std::string* shortRel, + std::string* longRel, + std::string* graph, + const runner::MultiFragmentPlan::Options& runnerOptions = {}, + const OptimizerOptions& optimizerOptions = {}); + void checkSameSingleNode( const logical_plan::LogicalPlanNodePtr& planNode, const velox::core::PlanNodePtr& referencePlan, diff --git a/axiom/optimizer/tests/SqlQueryRunner.cpp b/axiom/optimizer/tests/SqlQueryRunner.cpp index 1a4e0e3a..2a6c4da8 100644 --- a/axiom/optimizer/tests/SqlQueryRunner.cpp +++ b/axiom/optimizer/tests/SqlQueryRunner.cpp @@ -22,6 +22,12 @@ #include "axiom/optimizer/Plan.h" #include "axiom/optimizer/RelationOpPrinter.h" #include "velox/common/file/FileSystems.h" +#include "velox/connectors/hive/HiveConnector.h" +#include "velox/dwio/dwrf/RegisterDwrfReader.h" +#include "velox/dwio/dwrf/RegisterDwrfWriter.h" +#include "velox/dwio/parquet/RegisterParquetReader.h" +#include "velox/dwio/parquet/RegisterParquetWriter.h" +#include "velox/exec/PlanNodeStats.h" #include "velox/exec/tests/utils/LocalExchangeSource.h" #include "velox/expression/Expr.h" #include "velox/functions/prestosql/aggregates/RegisterAggregateFunctions.h" @@ -217,22 +223,110 @@ std::string SqlQueryRunner::runExplain( case presto::ExplainStatement::Type::kExecutable: return optimize(statement.plan(), newQuery(options), options).toString(); } + VELOX_UNREACHABLE(); } namespace { std::string printPlanWithStats( runner::LocalRunner& runner, - const optimizer::NodePredictionMap& estimates) { - return runner.printPlanWithStats([&](const velox::core::PlanNodeId& nodeId, - std::string_view indentation, - std::ostream& out) { + const optimizer::NodePredictionMap& estimates, + bool includeRuntimeStats = false) { + // Calculate predicted CPU total + float predictedCpu = 0; + for (auto& pair : estimates) { + predictedCpu += pair.second.cpu; + } + + // Get actual CPU timings from TaskStats + folly::F14FastMap nodeCpuNanos; + int64_t cpuNanos = 0; + + // Get TaskStats from runner and convert to PlanNodeStats + auto taskStats = runner.stats(); + for (const auto& taskStat : taskStats) { + auto planStats = velox::exec::toPlanStats(taskStat); + + // For each plan node, sum up CPU from addInput, getOutput, and finish + for (const auto& [nodeId, stats] : planStats) { + int64_t nodeCpu = stats.addInputTiming.cpuNanos + + stats.getOutputTiming.cpuNanos + stats.finishTiming.cpuNanos; + + // Accumulate (PlanNodeIds may not be unique across tasks) + nodeCpuNanos[nodeId] += nodeCpu; + cpuNanos += nodeCpu; + } + } + + std::stringstream result; + result << runner.printPlanWithStats([&](const velox::core::PlanNodeId& nodeId, + std::string_view indentation, + std::ostream& out) { auto it = estimates.find(nodeId); if (it != estimates.end()) { out << indentation << "Estimate: " << it->second.cardinality << " rows, " - << velox::succinctBytes(it->second.peakMemory) << " peak memory" - << std::endl; + << velox::succinctBytes(it->second.peakMemory) + << " peak memory, predicted cpu=" << std::fixed + << std::setprecision(2) << (it->second.cpu * 100.0f / predictedCpu) + << "%"; + + // Add actual CPU percentage + auto cpuIt = nodeCpuNanos.find(nodeId); + if (cpuIt != nodeCpuNanos.end() && cpuNanos > 0) { + out << ", actual cpu=" << std::fixed << std::setprecision(2) + << (static_cast(cpuIt->second) * 100.0f / cpuNanos) << "%"; + } + + out << std::endl; } }); + + // Print runtime stats grouped by operator if requested + if (includeRuntimeStats) { + result << "\n" << std::string(80, '=') << "\n"; + result << "Runtime Stats by Operator:\n"; + result << std::string(80, '=') << "\n"; + + // Collect all runtime stats grouped by node ID + std::map< + velox::core::PlanNodeId, + std::unordered_map> + allNodeStats; + + for (const auto& taskStat : taskStats) { + auto planStats = velox::exec::toPlanStats(taskStat); + + for (auto& [nodeId, stats] : planStats) { + if (!stats.customStats.empty()) { + // Merge custom stats for this node + for (const auto& [statName, statValue] : stats.customStats) { + auto& nodeStatMap = allNodeStats[nodeId]; + auto it = nodeStatMap.find(statName); + if (it == nodeStatMap.end()) { + nodeStatMap[statName] = statValue; + } else { + it->second.merge(statValue); + } + } + } + } + } + + // Print collected stats + for (const auto& [nodeId, customStats] : allNodeStats) { + result << "\nNode " << nodeId << ":\n"; + for (const auto& [statName, statValue] : customStats) { + result << " " << statName << ": " << statValue.toString() << "\n"; + } + } + + if (allNodeStats.empty()) { + result << "\nNo custom runtime stats available.\n"; + } + + result << std::string(80, '=') << "\n"; + } + + return result.str(); } } // namespace @@ -250,7 +344,8 @@ std::string SqlQueryRunner::runExplainAnalyze( auto results = fetchResults(*runner); std::stringstream out; - out << printPlanWithStats(*runner, planAndStats.prediction); + out << printPlanWithStats( + *runner, planAndStats.prediction, options.includeRuntimeStats); return out.str(); } @@ -279,6 +374,11 @@ optimizer::PlanAndStats SqlQueryRunner::optimize( auto session = std::make_shared(queryCtx->queryId()); + optimizer::OptimizerOptions optimizerOptions; + optimizerOptions.traceFlags = options.optimizerTraceFlags; + optimizerOptions.enableReducingExistences = options.enableReducingExistences; + optimizerOptions.syntacticJoinOrder = options.syntacticJoinOrder; + optimizer::Optimization optimization( session, *logicalPlan, @@ -286,7 +386,7 @@ optimizer::PlanAndStats SqlQueryRunner::optimize( *history_, queryCtx, evaluator, - {.traceFlags = options.optimizerTraceFlags}, + optimizerOptions, opts); if (checkDerivedTable && !checkDerivedTable(*optimization.rootDt())) { diff --git a/axiom/optimizer/tests/SqlQueryRunner.h b/axiom/optimizer/tests/SqlQueryRunner.h index 47cf1908..0cad357b 100644 --- a/axiom/optimizer/tests/SqlQueryRunner.h +++ b/axiom/optimizer/tests/SqlQueryRunner.h @@ -47,6 +47,9 @@ class SqlQueryRunner { int32_t numDrivers{4}; uint64_t splitTargetBytes{16 << 20}; uint32_t optimizerTraceFlags{0}; + bool enableReducingExistences{true}; + bool includeRuntimeStats{false}; + bool syntacticJoinOrder{false}; }; SqlResult run(std::string_view sql, const RunOptions& options); diff --git a/axiom/optimizer/tests/TpchPlanTest.cpp b/axiom/optimizer/tests/TpchPlanTest.cpp index 95d3dc5d..1bb6e8e8 100644 --- a/axiom/optimizer/tests/TpchPlanTest.cpp +++ b/axiom/optimizer/tests/TpchPlanTest.cpp @@ -16,13 +16,18 @@ #include #include +#include +#include +#include #include "axiom/logical_plan/ExprApi.h" #include "axiom/logical_plan/PlanBuilder.h" #include "axiom/optimizer/tests/HiveQueriesTestBase.h" +#include "axiom/optimizer/tests/PlanMatcherGenerator.h" #include "velox/dwio/common/tests/utils/DataFiles.h" #include "velox/exec/tests/utils/TpchQueryBuilder.h" DEFINE_int32(num_repeats, 1, "Number of repeats for optimization timing"); +DEFINE_bool(record_plans, false, "Record plan checkers to files"); DECLARE_uint32(optimizer_trace); DECLARE_string(history_save_path); @@ -33,8 +38,31 @@ namespace { using namespace facebook::velox; namespace lp = facebook::axiom::logical_plan; +struct CheckerKey { + int32_t queryNo; + int32_t numWorkers; + int32_t numDrivers; + + bool operator==(const CheckerKey& other) const { + return queryNo == other.queryNo && numWorkers == other.numWorkers && + numDrivers == other.numDrivers; + } +}; + +struct CheckerKeyHash { + std::size_t operator()(const CheckerKey& key) const { + // Combine hash values using a simple hash combination technique + std::size_t h1 = std::hash{}(key.queryNo); + std::size_t h2 = std::hash{}(key.numWorkers); + std::size_t h3 = std::hash{}(key.numDrivers); + return h1 ^ (h2 << 1) ^ (h3 << 2); + } +}; + class TpchPlanTest : public virtual test::HiveQueriesTestBase { protected: + using PlanChecker = std::function; + static void SetUpTestCase() { test::HiveQueriesTestBase::SetUpTestCase(); } @@ -58,9 +86,133 @@ class TpchPlanTest : public virtual test::HiveQueriesTestBase { HiveQueriesTestBase::TearDown(); } + void setChecker( + int32_t queryNo, + int32_t numWorkers, + int32_t numDrivers, + PlanChecker checker) { + checkers_[CheckerKey{queryNo, numWorkers, numDrivers}] = std::move(checker); + } + + PlanChecker* + getChecker(int32_t queryNo, int32_t numWorkers, int32_t numDrivers) { + auto it = checkers_.find(CheckerKey{queryNo, numWorkers, numDrivers}); + if (it != checkers_.end()) { + return &it->second; + } + return nullptr; + } + + void recordPlanCheckerStart(int32_t queryNo) { + if (!FLAGS_record_plans) { + return; + } + + auto filename = fmt::format("check_{}.inc", queryNo); + std::ofstream file(filename); + + if (!file.is_open()) { + LOG(ERROR) << "Failed to open file: " << filename; + return; + } + + file << "void defineCheckers" << queryNo << "() {\n"; + file.close(); + } + + void recordPlanCheckerEnd(int32_t queryNo) { + if (!FLAGS_record_plans) { + return; + } + + auto filename = fmt::format("check_{}.inc", queryNo); + std::ofstream file(filename, std::ios::app); + + if (!file.is_open()) { + LOG(ERROR) << "Failed to open file: " << filename; + return; + } + + file << "}\n"; + file.close(); + } + + void recordPlanChecker( + int32_t queryNo, + const lp::LogicalPlanNodePtr& logicalPlan, + const PlanAndStats& planAndStats, + int32_t numWorkers, + int32_t numDrivers) { + if (!FLAGS_record_plans) { + return; + } + + auto filename = fmt::format("check_{}.inc", queryNo); + std::ofstream file(filename, std::ios::app); + + if (!file.is_open()) { + LOG(ERROR) << "Failed to open file: " << filename; + return; + } + + // Get the short RelationOp representation + std::string shortRel; + QueryTestBase::explain( + logicalPlan, + &shortRel, + nullptr, + nullptr, + runner::MultiFragmentPlan::Options{ + .numWorkers = numWorkers, .numDrivers = numDrivers}); + + file << "// Configuration: numWorkers=" << numWorkers + << ", numDrivers=" << numDrivers << "\n"; + file << "setChecker(" << queryNo << ", " << numWorkers << ", " << numDrivers + << ", [](const PlanAndStats& planAndStats) {\n"; + + // Add the plan as a comment for readability + file << " // Plan:\n"; + std::istringstream planStream(shortRel); + std::string line; + while (std::getline(planStream, line)) { + file << " // " << line << "\n"; + } + file << "\n"; + + const auto& fragments = planAndStats.plan->fragments(); + for (size_t i = 0; i < fragments.size(); ++i) { + const auto& fragment = fragments[i]; + const auto& topNode = fragment.fragment.planNode; + + file << " // Fragment " << i << "\n"; + file << " {\n"; + file << " " << velox::core::generatePlanMatcherCode(topNode, "matcher") + << ";\n"; + file << " EXPECT_TRUE(matcher->match(planAndStats.plan->fragments()[" + << i << "].fragment.planNode));\n"; + file << " }\n"; + } + + file << "});\n\n"; + file.close(); + } + void checkTpch(int32_t query, const lp::LogicalPlanNodePtr& logicalPlan) { auto referencePlan = referenceBuilder_->getQueryPlan(query).plan; - checkSame(logicalPlan, referencePlan); + + // Define configurations to test + std::vector> configs = { + {1, 1}, {1, 4}, {4, 1}, {4, 4}}; + + // Loop over configs and test the plan against reference builder for each + for (const auto& [numWorkers, numDrivers] : configs) { + // Generate plan with this config and test against reference builder + planVelox( + logicalPlan, + runner::MultiFragmentPlan::Options{ + .numWorkers = numWorkers, .numDrivers = numDrivers}); + checkSame(logicalPlan, referencePlan); + } } static std::string readSqlFromFile(const std::string& filePath) { @@ -111,10 +263,65 @@ class TpchPlanTest : public virtual test::HiveQueriesTestBase { void checkTpchSql(int32_t query) { auto sql = readTpchSql(query); auto referencePlan = referenceBuilder_->getQueryPlan(query).plan; - checkResults(sql, referencePlan); + + // Define configurations to test + std::vector> configs = { + {1, 1}, {1, 4}, {4, 1}, {4, 4}}; + + // Loop over configs + for (size_t i = 0; i < configs.size(); ++i) { + const auto& [numWorkers, numDrivers] = configs[i]; + bool isFirstConfig = (i == 0); + bool isLastConfig = (i == configs.size() - 1); + + // First run the SQL query and compare with reference builder + auto logicalPlan = parseTpchSql(query); + auto planAndStats = planVelox( + logicalPlan, + runner::MultiFragmentPlan::Options{ + .numWorkers = numWorkers, .numDrivers = numDrivers}); + + // Check the optimized query against the reference builder + checkSame(logicalPlan, referencePlan); + + // If FLAGS_record_plans is set, initialize the recording at first config + if (FLAGS_record_plans && isFirstConfig) { + recordPlanCheckerStart(query); + } + + // If FLAGS_record_plans is set, record the checker + if (FLAGS_record_plans) { + recordPlanChecker( + query, logicalPlan, planAndStats, numWorkers, numDrivers); + } + + // If FLAGS_record_plans is set, after recording with the last config, + // finalize the recording + if (FLAGS_record_plans && isLastConfig) { + recordPlanCheckerEnd(query); + } + + // If FLAGS_record_plans is not set and there is a checker for the query + // and config, run the checker + if (!FLAGS_record_plans) { + auto* checker = getChecker(query, numWorkers, numDrivers); + if (checker) { + (*checker)(planAndStats); + } + } + + // Continue loop with the next config + } } std::unique_ptr referenceBuilder_; + std::unordered_map checkers_; + +#include "check_1.inc" +#include "check_13.inc" +#include "check_17.inc" +#include "check_3.inc" +#include "check_4.inc" }; TEST_F(TpchPlanTest, stats) { @@ -127,10 +334,12 @@ TEST_F(TpchPlanTest, stats) { auto planAndStats = planVelox(logicalPlan); auto stats = planAndStats.prediction; - ASSERT_EQ(stats.size(), 1); - ASSERT_EQ(stats.begin()->first, logicalPlan->id()); - ASSERT_EQ(stats.begin()->second.cardinality, cardinality); + // We expect a prediction for the table scan and + ASSERT_EQ(stats.size(), 2); + + // Node ids start at 0, the scan is always first. + ASSERT_EQ(stats.at("0").cardinality, cardinality); }; verifyStats("region", 5); @@ -140,6 +349,7 @@ TEST_F(TpchPlanTest, stats) { } TEST_F(TpchPlanTest, q01) { + defineCheckers1(); auto logicalPlan = lp::PlanBuilder() .tableScan(exec::test::kHiveConnectorId, "lineitem") @@ -169,6 +379,7 @@ TEST_F(TpchPlanTest, q02) { } TEST_F(TpchPlanTest, q03) { + defineCheckers3(); lp::PlanBuilder::Context context{exec::test::kHiveConnectorId}; auto logicalPlan = lp::PlanBuilder(context) @@ -193,6 +404,7 @@ TEST_F(TpchPlanTest, q03) { } TEST_F(TpchPlanTest, q04) { + defineCheckers4(); checkTpchSql(4); } @@ -453,6 +665,7 @@ TEST_F(TpchPlanTest, q12) { } TEST_F(TpchPlanTest, q13) { + defineCheckers13(); lp::PlanBuilder::Context context{exec::test::kHiveConnectorId}; auto logicalPlan = lp::PlanBuilder(context) @@ -502,6 +715,7 @@ TEST_F(TpchPlanTest, q16) { } TEST_F(TpchPlanTest, q17) { + defineCheckers17(); checkTpchSql(17); } @@ -546,14 +760,6 @@ TEST_F(TpchPlanTest, q19) { } TEST_F(TpchPlanTest, q20) { - // TODO Fix the plan when 'enableReducingExistences' is true. - const bool originalEnableReducingExistences = - optimizerOptions_.enableReducingExistences; - optimizerOptions_.enableReducingExistences = false; - SCOPE_EXIT { - optimizerOptions_.enableReducingExistences = - originalEnableReducingExistences; - }; checkTpchSql(20); } @@ -588,6 +794,113 @@ TEST_F(TpchPlanTest, DISABLED_makePlans) { } } +TEST_F(TpchPlanTest, supplierAggregationJoin) { + using namespace facebook::velox; + + auto lineitemType = ROW({{"l_suppkey", BIGINT()}, {"l_quantity", DOUBLE()}}); + + auto supplierType = ROW( + {{"s_suppkey", BIGINT()}, + {"s_name", VARCHAR()}, + {"s_acctbal", DOUBLE()}, + {"s_nationkey", BIGINT()}}); + + auto nationType = ROW({{"n_nationkey", BIGINT()}, {"n_name", VARCHAR()}}); + + // Create shared plan node ID generator + auto planNodeIdGenerator = std::make_shared(); + + // Test 1: Simple supplier join + { + auto sql = + "select s_name, s_acctbal, volume " + "from (select l_suppkey, sum(l_quantity) as volume from lineitem group by l_suppkey), supplier " + "where s_suppkey = l_suppkey and s_acctbal < 100"; + SCOPED_TRACE(sql); + + // Build reference plan: lineitem scan -> aggregation -> hash join with + // filtered supplier -> project + auto referencePlan = + exec::test::PlanBuilder(planNodeIdGenerator, pool()) + .localPartition( + {}, + {exec::test::PlanBuilder(planNodeIdGenerator, pool()) + .tableScan("lineitem", lineitemType) + .singleAggregation({"l_suppkey"}, {"sum(l_quantity)"}) + .planNode()}) + .hashJoin( + {"l_suppkey"}, + {"s_suppkey"}, + exec::test::PlanBuilder(planNodeIdGenerator, pool()) + .tableScan("supplier", supplierType) + .filter("s_acctbal < 100.0") + .planNode(), + "", + {"s_name", "s_acctbal", "a0"}) + .project({"s_name", "s_acctbal", "a0 as volume"}) + .planNode(); + + auto sqlPlan = checkResults(sql, referencePlan); + + // Verify the optimized plan contains aggregation above filter semijoin + ASSERT_EQ(1, sqlPlan.plan->fragments().size()); + auto sqlPlanNode = sqlPlan.plan->fragments().at(0).fragment.planNode; + + checkPlanText(sqlPlanNode, {"Aggregation", "LEFT SEMI \\(FILTER\\)"}); + } + + // Test 2: Supplier join with nation filter + { + auto sql = + "select s_name, s_acctbal, volume " + "from (select l_suppkey, sum(l_quantity) as volume from lineitem group by l_suppkey), supplier, nation " + "where s_suppkey = l_suppkey and s_acctbal - 100 < 0 and s_nationkey = n_nationkey and n_name = 'FRANCE'"; + SCOPED_TRACE(sql); + + // Build reference plan: lineitem scan -> aggregation -> hash join with + // supplier joined to nation and filtered -> project + auto referencePlan = + exec::test::PlanBuilder(planNodeIdGenerator, pool()) + .localPartition( + {}, + {exec::test::PlanBuilder(planNodeIdGenerator, pool()) + .tableScan("lineitem", lineitemType) + .singleAggregation({"l_suppkey"}, {"sum(l_quantity)"}) + .planNode()}) + .hashJoin( + {"l_suppkey"}, + {"s_suppkey"}, + exec::test::PlanBuilder(planNodeIdGenerator, pool()) + .tableScan("supplier", supplierType) + .hashJoin( + {"s_nationkey"}, + {"n_nationkey"}, + exec::test::PlanBuilder(planNodeIdGenerator, pool()) + .tableScan("nation", nationType) + .filter("n_name = 'FRANCE'") + .planNode(), + "", + {"s_suppkey", "s_name", "s_acctbal"}) + .filter("s_acctbal - 100.0 < 0.0") + .planNode(), + "", + {"s_name", "s_acctbal", "a0"}) + .project({"s_name", "s_acctbal", "a0 as volume"}) + .planNode(); + + auto sqlPlan = checkResults(sql, referencePlan); + + // Verify the optimized plan contains aggregation above filter semijoin + // with nation joined to supplier + ASSERT_EQ(1, sqlPlan.plan->fragments().size()); + auto sqlPlanNode = sqlPlan.plan->fragments().at(0).fragment.planNode; + + checkPlanText( + sqlPlanNode, + {"Aggregation", "LEFT SEMI \\(FILTER\\)", "supplier", "nation"}); + } +} + } // namespace } // namespace facebook::axiom::optimizer diff --git a/axiom/optimizer/tests/check_1.inc b/axiom/optimizer/tests/check_1.inc new file mode 100644 index 00000000..ab2578f2 --- /dev/null +++ b/axiom/optimizer/tests/check_1.inc @@ -0,0 +1,243 @@ +/* + * Copyright (c) Meta Platforms, Inc. and its affiliates. + * + * Licensed 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. + */ +void defineCheckers1() { + // Configuration: numWorkers=1, numDrivers=1 + setChecker(1, 1, 1, [](const PlanAndStats& planAndStats) { + // Plan: + // lineitem t2 project 7 columns SINGLE agg order by 2 columns project 10 + // columns + + // Fragment 0 + { + auto matcher = + core::PlanMatcherBuilder() + .tableScan("lineitem") + .project( + {"\"l_returnflag\" AS l_returnflag", + "\"l_linestatus\" AS l_linestatus", + "\"l_quantity\" AS l_quantity", + "\"l_extendedprice\" AS l_extendedprice", + "multiply(\"l_extendedprice\",minus(1,\"l_discount\")) AS dt1___p39", + "multiply(multiply(\"l_extendedprice\",minus(1,\"l_discount\")),plus(\"l_tax\",1)) AS dt1___p46", + "\"l_discount\" AS l_discount"}) + .singleAggregation( + {"\"l_returnflag\"", "\"l_linestatus\""}, + {"sum(\"l_quantity\") AS sum_qty", + "sum(\"l_extendedprice\") AS sum_base_price", + "sum(\"dt1___p39\") AS sum_disc_price", + "sum(\"dt1___p46\") AS sum_charge", + "avg(\"l_quantity\") AS avg_qty", + "avg(\"l_extendedprice\") AS avg_price", + "avg(\"l_discount\") AS avg_disc", + "count() AS count_order"}) + .orderBy( + {"\"l_returnflag\" ASC NULLS LAST", + "\"l_linestatus\" ASC NULLS LAST"}) + .build(); + ; + EXPECT_TRUE( + matcher->match(planAndStats.plan->fragments()[0].fragment.planNode)); + } + }); + + // Configuration: numWorkers=1, numDrivers=4 + setChecker(1, 1, 4, [](const PlanAndStats& planAndStats) { + // Plan: + // lineitem t2 project 7 columns PARTIAL agg FINAL agg order by 2 columns + // project 10 columns + + // Fragment 0 + { + auto matcher = + core::PlanMatcherBuilder() + .tableScan("lineitem") + .project( + {"\"l_returnflag\" AS l_returnflag", + "\"l_linestatus\" AS l_linestatus", + "\"l_quantity\" AS l_quantity", + "\"l_extendedprice\" AS l_extendedprice", + "multiply(\"l_extendedprice\",minus(1,\"l_discount\")) AS dt1___p39", + "multiply(multiply(\"l_extendedprice\",minus(1,\"l_discount\")),plus(\"l_tax\",1)) AS dt1___p46", + "\"l_discount\" AS l_discount"}) + .partialAggregation( + {"\"l_returnflag\"", "\"l_linestatus\""}, + {"sum(\"l_quantity\") AS sum_qty", + "sum(\"l_extendedprice\") AS sum_base_price", + "sum(\"dt1___p39\") AS sum_disc_price", + "sum(\"dt1___p46\") AS sum_charge", + "avg(\"l_quantity\") AS avg_qty", + "avg(\"l_extendedprice\") AS avg_price", + "avg(\"l_discount\") AS avg_disc", + "count() AS count_order"}) + .localPartition() + .finalAggregation( + {"\"l_returnflag\"", "\"l_linestatus\""}, + {"sum(\"sum_qty\") AS sum_qty", + "sum(\"sum_base_price\") AS sum_base_price", + "sum(\"sum_disc_price\") AS sum_disc_price", + "sum(\"sum_charge\") AS sum_charge", + "avg(\"avg_qty\") AS avg_qty", + "avg(\"avg_price\") AS avg_price", + "avg(\"avg_disc\") AS avg_disc", + "count(\"count_order\") AS count_order"}) + .orderBy( + {"\"l_returnflag\" ASC NULLS LAST", + "\"l_linestatus\" ASC NULLS LAST"}) + .localMerge() + .build(); + ; + EXPECT_TRUE( + matcher->match(planAndStats.plan->fragments()[0].fragment.planNode)); + } + }); + + // Configuration: numWorkers=4, numDrivers=1 + setChecker(1, 4, 1, [](const PlanAndStats& planAndStats) { + // Plan: + // lineitem t2 project 7 columns PARTIAL agg repartition FINAL agg order + // by 2 columns project 10 columns + + // Fragment 0 + { + auto matcher = + core::PlanMatcherBuilder() + .tableScan("lineitem") + .project( + {"\"l_returnflag\" AS l_returnflag", + "\"l_linestatus\" AS l_linestatus", + "\"l_quantity\" AS l_quantity", + "\"l_extendedprice\" AS l_extendedprice", + "multiply(\"l_extendedprice\",minus(1,\"l_discount\")) AS dt1___p39", + "multiply(multiply(\"l_extendedprice\",minus(1,\"l_discount\")),plus(\"l_tax\",1)) AS dt1___p46", + "\"l_discount\" AS l_discount"}) + .partialAggregation( + {"\"l_returnflag\"", "\"l_linestatus\""}, + {"sum(\"l_quantity\") AS sum_qty", + "sum(\"l_extendedprice\") AS sum_base_price", + "sum(\"dt1___p39\") AS sum_disc_price", + "sum(\"dt1___p46\") AS sum_charge", + "avg(\"l_quantity\") AS avg_qty", + "avg(\"l_extendedprice\") AS avg_price", + "avg(\"l_discount\") AS avg_disc", + "count() AS count_order"}) + .partitionedOutput() + .build(); + ; + EXPECT_TRUE( + matcher->match(planAndStats.plan->fragments()[0].fragment.planNode)); + } + // Fragment 1 + { + auto matcher = core::PlanMatcherBuilder() + .exchange() + .finalAggregation( + {"\"l_returnflag\"", "\"l_linestatus\""}, + {"sum(\"sum_qty\") AS sum_qty", + "sum(\"sum_base_price\") AS sum_base_price", + "sum(\"sum_disc_price\") AS sum_disc_price", + "sum(\"sum_charge\") AS sum_charge", + "avg(\"avg_qty\") AS avg_qty", + "avg(\"avg_price\") AS avg_price", + "avg(\"avg_disc\") AS avg_disc", + "count(\"count_order\") AS count_order"}) + .orderBy( + {"\"l_returnflag\" ASC NULLS LAST", + "\"l_linestatus\" ASC NULLS LAST"}) + .localMerge() + .partitionedOutput() + .build(); + ; + EXPECT_TRUE( + matcher->match(planAndStats.plan->fragments()[1].fragment.planNode)); + } + // Fragment 2 + { + auto matcher = core::PlanMatcherBuilder().mergeExchange().build(); + ; + EXPECT_TRUE( + matcher->match(planAndStats.plan->fragments()[2].fragment.planNode)); + } + }); + + // Configuration: numWorkers=4, numDrivers=4 + setChecker(1, 4, 4, [](const PlanAndStats& planAndStats) { + // Plan: + // lineitem t2 project 7 columns PARTIAL agg repartition FINAL agg order + // by 2 columns project 10 columns + + // Fragment 0 + { + auto matcher = + core::PlanMatcherBuilder() + .tableScan("lineitem") + .project( + {"\"l_returnflag\" AS l_returnflag", + "\"l_linestatus\" AS l_linestatus", + "\"l_quantity\" AS l_quantity", + "\"l_extendedprice\" AS l_extendedprice", + "multiply(\"l_extendedprice\",minus(1,\"l_discount\")) AS dt1___p39", + "multiply(multiply(\"l_extendedprice\",minus(1,\"l_discount\")),plus(\"l_tax\",1)) AS dt1___p46", + "\"l_discount\" AS l_discount"}) + .partialAggregation( + {"\"l_returnflag\"", "\"l_linestatus\""}, + {"sum(\"l_quantity\") AS sum_qty", + "sum(\"l_extendedprice\") AS sum_base_price", + "sum(\"dt1___p39\") AS sum_disc_price", + "sum(\"dt1___p46\") AS sum_charge", + "avg(\"l_quantity\") AS avg_qty", + "avg(\"l_extendedprice\") AS avg_price", + "avg(\"l_discount\") AS avg_disc", + "count() AS count_order"}) + .partitionedOutput() + .build(); + ; + EXPECT_TRUE( + matcher->match(planAndStats.plan->fragments()[0].fragment.planNode)); + } + // Fragment 1 + { + auto matcher = core::PlanMatcherBuilder() + .exchange() + .localPartition() + .finalAggregation( + {"\"l_returnflag\"", "\"l_linestatus\""}, + {"sum(\"sum_qty\") AS sum_qty", + "sum(\"sum_base_price\") AS sum_base_price", + "sum(\"sum_disc_price\") AS sum_disc_price", + "sum(\"sum_charge\") AS sum_charge", + "avg(\"avg_qty\") AS avg_qty", + "avg(\"avg_price\") AS avg_price", + "avg(\"avg_disc\") AS avg_disc", + "count(\"count_order\") AS count_order"}) + .orderBy( + {"\"l_returnflag\" ASC NULLS LAST", + "\"l_linestatus\" ASC NULLS LAST"}) + .localMerge() + .partitionedOutput() + .build(); + ; + EXPECT_TRUE( + matcher->match(planAndStats.plan->fragments()[1].fragment.planNode)); + } + // Fragment 2 + { + auto matcher = core::PlanMatcherBuilder().mergeExchange().build(); + ; + EXPECT_TRUE( + matcher->match(planAndStats.plan->fragments()[2].fragment.planNode)); + } + }); +} diff --git a/axiom/optimizer/tests/check_13.inc b/axiom/optimizer/tests/check_13.inc new file mode 100644 index 00000000..7cdf3137 --- /dev/null +++ b/axiom/optimizer/tests/check_13.inc @@ -0,0 +1,251 @@ +/* + * Copyright (c) Meta Platforms, Inc. and its affiliates. + * + * Licensed 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. + */ +void defineCheckers13() { + // Configuration: numWorkers=1, numDrivers=1 + setChecker(13, 1, 1, [](const PlanAndStats& planAndStats) { + // Plan: + // orders t3*H right (customer t2 Build ) SINGLE agg project 1 columns + // SINGLE agg order by 2 columns project 2 columns + + // Fragment 0 + { + auto rightMatcher = + core::PlanMatcherBuilder().tableScan("customer").build(); + + auto matcher = + core::PlanMatcherBuilder() + .tableScan("orders") + .hashJoin(rightMatcher, velox::core::JoinType::kRight) + .singleAggregation( + {"\"c_custkey\""}, {"count(\"o_orderkey\") AS count"}) + .project({"\"count\" AS c_count"}) + .singleAggregation({"\"c_count\""}, {"count() AS custdist"}) + .orderBy( + {"\"custdist\" DESC NULLS LAST", + "\"c_count\" DESC NULLS LAST"}) + .build(); + ; + EXPECT_TRUE( + matcher->match(planAndStats.plan->fragments()[0].fragment.planNode)); + } + }); + + // Configuration: numWorkers=1, numDrivers=4 + setChecker(13, 1, 4, [](const PlanAndStats& planAndStats) { + // Plan: + // orders t3*H right (customer t2 Build ) PARTIAL agg FINAL agg project 1 + // columns PARTIAL agg FINAL agg order by 2 columns project 2 columns + + // Fragment 0 + { + auto rightMatcher = + core::PlanMatcherBuilder().tableScan("customer").build(); + + auto matcher = + core::PlanMatcherBuilder() + .tableScan("orders") + .hashJoin(rightMatcher, velox::core::JoinType::kRight) + .partialAggregation( + {"\"c_custkey\""}, {"count(\"o_orderkey\") AS count"}) + .localPartition() + .finalAggregation( + {"\"c_custkey\""}, {"count(\"count\") AS count"}) + .project({"\"count\" AS c_count"}) + .partialAggregation({"\"c_count\""}, {"count() AS custdist"}) + .localPartition() + .finalAggregation( + {"\"c_count\""}, {"count(\"custdist\") AS custdist"}) + .orderBy( + {"\"custdist\" DESC NULLS LAST", + "\"c_count\" DESC NULLS LAST"}) + .localMerge() + .build(); + ; + EXPECT_TRUE( + matcher->match(planAndStats.plan->fragments()[0].fragment.planNode)); + } + }); + + // Configuration: numWorkers=4, numDrivers=1 + setChecker(13, 4, 1, [](const PlanAndStats& planAndStats) { + // Plan: + // orders t3 repartition *H right (customer t2 repartition Build ) PARTIAL + // agg repartition FINAL agg project 1 columns PARTIAL agg repartition + // FINAL agg order by 2 columns project 2 columns + + // Fragment 0 + { + auto matcher = core::PlanMatcherBuilder() + .tableScan("orders") + .partitionedOutput() + .build(); + ; + EXPECT_TRUE( + matcher->match(planAndStats.plan->fragments()[0].fragment.planNode)); + } + // Fragment 1 + { + auto matcher = core::PlanMatcherBuilder() + .tableScan("customer") + .partitionedOutput() + .build(); + ; + EXPECT_TRUE( + matcher->match(planAndStats.plan->fragments()[1].fragment.planNode)); + } + // Fragment 2 + { + auto rightMatcher = core::PlanMatcherBuilder().exchange().build(); + + auto matcher = + core::PlanMatcherBuilder() + .exchange() + .hashJoin(rightMatcher, velox::core::JoinType::kRight) + .partialAggregation( + {"\"c_custkey\""}, {"count(\"o_orderkey\") AS count"}) + .partitionedOutput() + .build(); + ; + EXPECT_TRUE( + matcher->match(planAndStats.plan->fragments()[2].fragment.planNode)); + } + // Fragment 3 + { + auto matcher = + core::PlanMatcherBuilder() + .exchange() + .finalAggregation( + {"\"c_custkey\""}, {"count(\"count\") AS count"}) + .project({"\"count\" AS c_count"}) + .partialAggregation({"\"c_count\""}, {"count() AS custdist"}) + .partitionedOutput() + .build(); + ; + EXPECT_TRUE( + matcher->match(planAndStats.plan->fragments()[3].fragment.planNode)); + } + // Fragment 4 + { + auto matcher = + core::PlanMatcherBuilder() + .exchange() + .finalAggregation( + {"\"c_count\""}, {"count(\"custdist\") AS custdist"}) + .orderBy( + {"\"custdist\" DESC NULLS LAST", + "\"c_count\" DESC NULLS LAST"}) + .localMerge() + .partitionedOutput() + .build(); + ; + EXPECT_TRUE( + matcher->match(planAndStats.plan->fragments()[4].fragment.planNode)); + } + // Fragment 5 + { + auto matcher = core::PlanMatcherBuilder().mergeExchange().build(); + ; + EXPECT_TRUE( + matcher->match(planAndStats.plan->fragments()[5].fragment.planNode)); + } + }); + + // Configuration: numWorkers=4, numDrivers=4 + setChecker(13, 4, 4, [](const PlanAndStats& planAndStats) { + // Plan: + // orders t3 repartition *H right (customer t2 repartition Build ) PARTIAL + // agg repartition FINAL agg project 1 columns PARTIAL agg repartition + // FINAL agg order by 2 columns project 2 columns + + // Fragment 0 + { + auto matcher = core::PlanMatcherBuilder() + .tableScan("orders") + .partitionedOutput() + .build(); + ; + EXPECT_TRUE( + matcher->match(planAndStats.plan->fragments()[0].fragment.planNode)); + } + // Fragment 1 + { + auto matcher = core::PlanMatcherBuilder() + .tableScan("customer") + .partitionedOutput() + .build(); + ; + EXPECT_TRUE( + matcher->match(planAndStats.plan->fragments()[1].fragment.planNode)); + } + // Fragment 2 + { + auto rightMatcher = core::PlanMatcherBuilder().exchange().build(); + + auto matcher = + core::PlanMatcherBuilder() + .exchange() + .hashJoin(rightMatcher, velox::core::JoinType::kRight) + .partialAggregation( + {"\"c_custkey\""}, {"count(\"o_orderkey\") AS count"}) + .partitionedOutput() + .build(); + ; + EXPECT_TRUE( + matcher->match(planAndStats.plan->fragments()[2].fragment.planNode)); + } + // Fragment 3 + { + auto matcher = + core::PlanMatcherBuilder() + .exchange() + .localPartition() + .finalAggregation( + {"\"c_custkey\""}, {"count(\"count\") AS count"}) + .project({"\"count\" AS c_count"}) + .partialAggregation({"\"c_count\""}, {"count() AS custdist"}) + .partitionedOutput() + .build(); + ; + EXPECT_TRUE( + matcher->match(planAndStats.plan->fragments()[3].fragment.planNode)); + } + // Fragment 4 + { + auto matcher = + core::PlanMatcherBuilder() + .exchange() + .localPartition() + .finalAggregation( + {"\"c_count\""}, {"count(\"custdist\") AS custdist"}) + .orderBy( + {"\"custdist\" DESC NULLS LAST", + "\"c_count\" DESC NULLS LAST"}) + .localMerge() + .partitionedOutput() + .build(); + ; + EXPECT_TRUE( + matcher->match(planAndStats.plan->fragments()[4].fragment.planNode)); + } + // Fragment 5 + { + auto matcher = core::PlanMatcherBuilder().mergeExchange().build(); + ; + EXPECT_TRUE( + matcher->match(planAndStats.plan->fragments()[5].fragment.planNode)); + } + }); +} diff --git a/axiom/optimizer/tests/check_3.inc b/axiom/optimizer/tests/check_3.inc new file mode 100644 index 00000000..41ea7005 --- /dev/null +++ b/axiom/optimizer/tests/check_3.inc @@ -0,0 +1,279 @@ +/* + * Copyright (c) Meta Platforms, Inc. and its affiliates. + * + * Licensed 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. + */ +void defineCheckers3() { + // Configuration: numWorkers=1, numDrivers=1 + setChecker(3, 1, 1, [](const PlanAndStats& planAndStats) { + // Plan: + // lineitem t4*H (orders t3*H (customer t2 Build ) project 3 columns + // Build ) project 4 columns SINGLE agg order by 2 columns project 4 + // columns + + // Fragment 0 + { + auto rightMatcher = + core::PlanMatcherBuilder().tableScan("customer").build(); + + auto rightMatcher1 = + core::PlanMatcherBuilder() + .tableScan("orders") + .hashJoin(rightMatcher, velox::core::JoinType::kInner) + .build(); + + auto matcher = + core::PlanMatcherBuilder() + .tableScan("lineitem") + .hashJoin(rightMatcher1, velox::core::JoinType::kInner) + .project( + {"\"l_orderkey\" AS l_orderkey", + "\"o_orderdate\" AS o_orderdate", + "\"o_shippriority\" AS o_shippriority", + "multiply(\"l_extendedprice\",minus(1,\"l_discount\")) AS dt1___p61"}) + .singleAggregation( + {"\"l_orderkey\"", "\"o_orderdate\"", "\"o_shippriority\""}, + {"sum(\"dt1___p61\") AS revenue"}) + .topN(10) + .project( + {"\"l_orderkey\" AS l_orderkey", + "\"revenue\" AS revenue", + "\"o_orderdate\" AS o_orderdate", + "\"o_shippriority\" AS o_shippriority"}) + .build(); + ; + EXPECT_TRUE( + matcher->match(planAndStats.plan->fragments()[0].fragment.planNode)); + } + }); + + // Configuration: numWorkers=1, numDrivers=4 + setChecker(3, 1, 4, [](const PlanAndStats& planAndStats) { + // Plan: + // lineitem t4*H (orders t3*H (customer t2 Build ) project 3 columns + // Build ) project 4 columns PARTIAL agg FINAL agg order by 2 columns + // project 4 columns + + // Fragment 0 + { + auto rightMatcher = + core::PlanMatcherBuilder().tableScan("customer").build(); + + auto rightMatcher1 = + core::PlanMatcherBuilder() + .tableScan("orders") + .hashJoin(rightMatcher, velox::core::JoinType::kInner) + .build(); + + auto matcher = + core::PlanMatcherBuilder() + .tableScan("lineitem") + .hashJoin(rightMatcher1, velox::core::JoinType::kInner) + .project( + {"\"l_orderkey\" AS l_orderkey", + "\"o_orderdate\" AS o_orderdate", + "\"o_shippriority\" AS o_shippriority", + "multiply(\"l_extendedprice\",minus(1,\"l_discount\")) AS dt1___p61"}) + .partialAggregation( + {"\"l_orderkey\"", "\"o_orderdate\"", "\"o_shippriority\""}, + {"sum(\"dt1___p61\") AS revenue"}) + .localPartition() + .finalAggregation( + {"\"l_orderkey\"", "\"o_orderdate\"", "\"o_shippriority\""}, + {"sum(\"revenue\") AS revenue"}) + .topN(10) + .localMerge() + .finalLimit(0, 10) + .project( + {"\"l_orderkey\" AS l_orderkey", + "\"revenue\" AS revenue", + "\"o_orderdate\" AS o_orderdate", + "\"o_shippriority\" AS o_shippriority"}) + .build(); + ; + EXPECT_TRUE( + matcher->match(planAndStats.plan->fragments()[0].fragment.planNode)); + } + }); + + // Configuration: numWorkers=4, numDrivers=1 + setChecker(3, 4, 1, [](const PlanAndStats& planAndStats) { + // Plan: + // lineitem t4*H (orders t3*H (customer t2 broadcast Build ) project 3 + // columns broadcast Build ) project 4 columns PARTIAL agg repartition + // FINAL agg order by 2 columns project 4 columns + + // Fragment 0 + { + auto matcher = core::PlanMatcherBuilder() + .tableScan("customer") + .partitionedOutput() + .build(); + ; + EXPECT_TRUE( + matcher->match(planAndStats.plan->fragments()[0].fragment.planNode)); + } + // Fragment 1 + { + auto rightMatcher = core::PlanMatcherBuilder().exchange().build(); + + auto matcher = core::PlanMatcherBuilder() + .tableScan("orders") + .hashJoin(rightMatcher, velox::core::JoinType::kInner) + .partitionedOutput() + .build(); + ; + EXPECT_TRUE( + matcher->match(planAndStats.plan->fragments()[1].fragment.planNode)); + } + // Fragment 2 + { + auto rightMatcher = core::PlanMatcherBuilder().exchange().build(); + + auto matcher = + core::PlanMatcherBuilder() + .tableScan("lineitem") + .hashJoin(rightMatcher, velox::core::JoinType::kInner) + .project( + {"\"l_orderkey\" AS l_orderkey", + "\"o_orderdate\" AS o_orderdate", + "\"o_shippriority\" AS o_shippriority", + "multiply(\"l_extendedprice\",minus(1,\"l_discount\")) AS dt1___p61"}) + .partialAggregation( + {"\"l_orderkey\"", "\"o_orderdate\"", "\"o_shippriority\""}, + {"sum(\"dt1___p61\") AS revenue"}) + .partitionedOutput() + .build(); + ; + EXPECT_TRUE( + matcher->match(planAndStats.plan->fragments()[2].fragment.planNode)); + } + // Fragment 3 + { + auto matcher = + core::PlanMatcherBuilder() + .exchange() + .finalAggregation( + {"\"l_orderkey\"", "\"o_orderdate\"", "\"o_shippriority\""}, + {"sum(\"revenue\") AS revenue"}) + .topN(10) + .localMerge() + .partitionedOutput() + .build(); + ; + EXPECT_TRUE( + matcher->match(planAndStats.plan->fragments()[3].fragment.planNode)); + } + // Fragment 4 + { + auto matcher = core::PlanMatcherBuilder() + .mergeExchange() + .finalLimit(0, 10) + .project( + {"\"l_orderkey\" AS l_orderkey", + "\"revenue\" AS revenue", + "\"o_orderdate\" AS o_orderdate", + "\"o_shippriority\" AS o_shippriority"}) + .build(); + ; + EXPECT_TRUE( + matcher->match(planAndStats.plan->fragments()[4].fragment.planNode)); + } + }); + + // Configuration: numWorkers=4, numDrivers=4 + setChecker(3, 4, 4, [](const PlanAndStats& planAndStats) { + // Plan: + // lineitem t4*H (orders t3*H (customer t2 broadcast Build ) project 3 + // columns broadcast Build ) project 4 columns PARTIAL agg repartition + // FINAL agg order by 2 columns project 4 columns + + // Fragment 0 + { + auto matcher = core::PlanMatcherBuilder() + .tableScan("customer") + .partitionedOutput() + .build(); + ; + EXPECT_TRUE( + matcher->match(planAndStats.plan->fragments()[0].fragment.planNode)); + } + // Fragment 1 + { + auto rightMatcher = core::PlanMatcherBuilder().exchange().build(); + + auto matcher = core::PlanMatcherBuilder() + .tableScan("orders") + .hashJoin(rightMatcher, velox::core::JoinType::kInner) + .partitionedOutput() + .build(); + ; + EXPECT_TRUE( + matcher->match(planAndStats.plan->fragments()[1].fragment.planNode)); + } + // Fragment 2 + { + auto rightMatcher = core::PlanMatcherBuilder().exchange().build(); + + auto matcher = + core::PlanMatcherBuilder() + .tableScan("lineitem") + .hashJoin(rightMatcher, velox::core::JoinType::kInner) + .project( + {"\"l_orderkey\" AS l_orderkey", + "\"o_orderdate\" AS o_orderdate", + "\"o_shippriority\" AS o_shippriority", + "multiply(\"l_extendedprice\",minus(1,\"l_discount\")) AS dt1___p61"}) + .partialAggregation( + {"\"l_orderkey\"", "\"o_orderdate\"", "\"o_shippriority\""}, + {"sum(\"dt1___p61\") AS revenue"}) + .partitionedOutput() + .build(); + ; + EXPECT_TRUE( + matcher->match(planAndStats.plan->fragments()[2].fragment.planNode)); + } + // Fragment 3 + { + auto matcher = + core::PlanMatcherBuilder() + .exchange() + .localPartition() + .finalAggregation( + {"\"l_orderkey\"", "\"o_orderdate\"", "\"o_shippriority\""}, + {"sum(\"revenue\") AS revenue"}) + .topN(10) + .localMerge() + .partitionedOutput() + .build(); + ; + EXPECT_TRUE( + matcher->match(planAndStats.plan->fragments()[3].fragment.planNode)); + } + // Fragment 4 + { + auto matcher = core::PlanMatcherBuilder() + .mergeExchange() + .finalLimit(0, 10) + .project( + {"\"l_orderkey\" AS l_orderkey", + "\"revenue\" AS revenue", + "\"o_orderdate\" AS o_orderdate", + "\"o_shippriority\" AS o_shippriority"}) + .build(); + ; + EXPECT_TRUE( + matcher->match(planAndStats.plan->fragments()[4].fragment.planNode)); + } + }); +} diff --git a/axiom/optimizer/tests/check_4.inc b/axiom/optimizer/tests/check_4.inc new file mode 100644 index 00000000..389cd0cc --- /dev/null +++ b/axiom/optimizer/tests/check_4.inc @@ -0,0 +1,241 @@ +/* + * Copyright (c) Meta Platforms, Inc. and its affiliates. + * + * Licensed 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. + */ +void defineCheckers4() { + // Configuration: numWorkers=1, numDrivers=1 + setChecker(4, 1, 1, [](const PlanAndStats& planAndStats) { + // Plan: + // lineitem t4 project 1 columns *H right exists-flag (orders t2 Build ) + // filter 1 exprs SINGLE agg order by 1 columns project 2 columns + + // Fragment 0 + { + auto rightMatcher = + core::PlanMatcherBuilder().tableScan("orders").build(); + + auto matcher = + core::PlanMatcherBuilder() + .tableScan("lineitem") + .hashJoin(rightMatcher, velox::core::JoinType::kRightSemiProject) + .filter("\"dt1___mark0\"") + .singleAggregation( + {"\"o_orderpriority\""}, {"count() AS order_count"}) + .orderBy({"\"o_orderpriority\" ASC NULLS LAST"}) + .build(); + ; + EXPECT_TRUE( + matcher->match(planAndStats.plan->fragments()[0].fragment.planNode)); + } + }); + + // Configuration: numWorkers=1, numDrivers=4 + setChecker(4, 1, 4, [](const PlanAndStats& planAndStats) { + // Plan: + // lineitem t4 project 1 columns *H right exists-flag (orders t2 Build ) + // filter 1 exprs PARTIAL agg FINAL agg order by 1 columns project 2 + // columns + + // Fragment 0 + { + auto rightMatcher = + core::PlanMatcherBuilder().tableScan("orders").build(); + + auto matcher = + core::PlanMatcherBuilder() + .tableScan("lineitem") + .hashJoin(rightMatcher, velox::core::JoinType::kRightSemiProject) + .filter("\"dt1___mark0\"") + .partialAggregation( + {"\"o_orderpriority\""}, {"count() AS order_count"}) + .localPartition() + .finalAggregation( + {"\"o_orderpriority\""}, + {"count(\"order_count\") AS order_count"}) + .orderBy({"\"o_orderpriority\" ASC NULLS LAST"}) + .localMerge() + .build(); + ; + EXPECT_TRUE( + matcher->match(planAndStats.plan->fragments()[0].fragment.planNode)); + } + }); + + // Configuration: numWorkers=4, numDrivers=1 + setChecker(4, 4, 1, [](const PlanAndStats& planAndStats) { + // Plan: + // lineitem t4*H exists (orders t2 broadcast Build ) project 1 columns + // repartition *H right exists-flag (orders t2 repartition Build ) filter + // 1 exprs PARTIAL agg repartition FINAL agg order by 1 columns project 2 + // columns + + // Fragment 0 + { + auto matcher = core::PlanMatcherBuilder() + .tableScan("orders") + .partitionedOutput() + .build(); + ; + EXPECT_TRUE( + matcher->match(planAndStats.plan->fragments()[0].fragment.planNode)); + } + // Fragment 1 + { + auto rightMatcher = core::PlanMatcherBuilder().exchange().build(); + + auto matcher = + core::PlanMatcherBuilder() + .tableScan("lineitem") + .hashJoin(rightMatcher, velox::core::JoinType::kLeftSemiFilter) + .partitionedOutput() + .build(); + ; + EXPECT_TRUE( + matcher->match(planAndStats.plan->fragments()[1].fragment.planNode)); + } + // Fragment 2 + { + auto matcher = core::PlanMatcherBuilder() + .tableScan("orders") + .partitionedOutput() + .build(); + ; + EXPECT_TRUE( + matcher->match(planAndStats.plan->fragments()[2].fragment.planNode)); + } + // Fragment 3 + { + auto rightMatcher = core::PlanMatcherBuilder().exchange().build(); + + auto matcher = + core::PlanMatcherBuilder() + .exchange() + .hashJoin(rightMatcher, velox::core::JoinType::kRightSemiProject) + .filter("\"dt1___mark0\"") + .partialAggregation( + {"\"o_orderpriority\""}, {"count() AS order_count"}) + .partitionedOutput() + .build(); + ; + EXPECT_TRUE( + matcher->match(planAndStats.plan->fragments()[3].fragment.planNode)); + } + // Fragment 4 + { + auto matcher = core::PlanMatcherBuilder() + .exchange() + .finalAggregation( + {"\"o_orderpriority\""}, + {"count(\"order_count\") AS order_count"}) + .orderBy({"\"o_orderpriority\" ASC NULLS LAST"}) + .localMerge() + .partitionedOutput() + .build(); + ; + EXPECT_TRUE( + matcher->match(planAndStats.plan->fragments()[4].fragment.planNode)); + } + // Fragment 5 + { + auto matcher = core::PlanMatcherBuilder().mergeExchange().build(); + ; + EXPECT_TRUE( + matcher->match(planAndStats.plan->fragments()[5].fragment.planNode)); + } + }); + + // Configuration: numWorkers=4, numDrivers=4 + setChecker(4, 4, 4, [](const PlanAndStats& planAndStats) { + // Plan: + // lineitem t4*H exists (orders t2 broadcast Build ) project 1 columns + // repartition *H right exists-flag (orders t2 repartition Build ) filter + // 1 exprs PARTIAL agg repartition FINAL agg order by 1 columns project 2 + // columns + + // Fragment 0 + { + auto matcher = core::PlanMatcherBuilder() + .tableScan("orders") + .partitionedOutput() + .build(); + ; + EXPECT_TRUE( + matcher->match(planAndStats.plan->fragments()[0].fragment.planNode)); + } + // Fragment 1 + { + auto rightMatcher = core::PlanMatcherBuilder().exchange().build(); + + auto matcher = + core::PlanMatcherBuilder() + .tableScan("lineitem") + .hashJoin(rightMatcher, velox::core::JoinType::kLeftSemiFilter) + .partitionedOutput() + .build(); + ; + EXPECT_TRUE( + matcher->match(planAndStats.plan->fragments()[1].fragment.planNode)); + } + // Fragment 2 + { + auto matcher = core::PlanMatcherBuilder() + .tableScan("orders") + .partitionedOutput() + .build(); + ; + EXPECT_TRUE( + matcher->match(planAndStats.plan->fragments()[2].fragment.planNode)); + } + // Fragment 3 + { + auto rightMatcher = core::PlanMatcherBuilder().exchange().build(); + + auto matcher = + core::PlanMatcherBuilder() + .exchange() + .hashJoin(rightMatcher, velox::core::JoinType::kRightSemiProject) + .filter("\"dt1___mark0\"") + .partialAggregation( + {"\"o_orderpriority\""}, {"count() AS order_count"}) + .partitionedOutput() + .build(); + ; + EXPECT_TRUE( + matcher->match(planAndStats.plan->fragments()[3].fragment.planNode)); + } + // Fragment 4 + { + auto matcher = core::PlanMatcherBuilder() + .exchange() + .localPartition() + .finalAggregation( + {"\"o_orderpriority\""}, + {"count(\"order_count\") AS order_count"}) + .orderBy({"\"o_orderpriority\" ASC NULLS LAST"}) + .localMerge() + .partitionedOutput() + .build(); + ; + EXPECT_TRUE( + matcher->match(planAndStats.plan->fragments()[4].fragment.planNode)); + } + // Fragment 5 + { + auto matcher = core::PlanMatcherBuilder().mergeExchange().build(); + ; + EXPECT_TRUE( + matcher->match(planAndStats.plan->fragments()[5].fragment.planNode)); + } + }); +} diff --git a/axiom/optimizer/tests/tpch.md b/axiom/optimizer/tests/tpch.md new file mode 100644 index 00000000..9fb1a105 --- /dev/null +++ b/axiom/optimizer/tests/tpch.md @@ -0,0 +1,244 @@ +# TPC-H Plans + +This document covers TPC-H with Verax under different configurations of execution (single node and scale-out) and different data layouts, as in bucketed with optional sorting on keys. + +We wil extend this as support is added to the different data layouts and access methods. We begin with analyzing multithreaded single node execution. Subsequent sections will cover differences brought by distributed execution or different data layouts. + +## Environment + +We run the single node experiments using axiom_sql on a laptop against a 30G dataset with Parquet and snappy. To repeat the below runs, use the query formulations in tpch.queries. To see plans and stats, use + +SQL> flag optimizer_trace = 15; +SQL> flag include_runtime_stats = true; +SQL flag num_workers = 1; +SQL> flag num_drivers = ; + +We set num_workers = 1 to force single node plans. These run faster on a single system and are easier to read when we do not have exchanges between stages. + +## Single Node Multithreaded Execution + +###Q1 + +The query is a simple scan of lineitem with a very low cardinality +group by. All the work is in the table scan (65%( and partial +aggregation (30%). + + + + +###Q2 + +The query takes partsupp info for a set of parts and a set of +suppliers limited by geography. the selection on supplier is 1/5 and +the selection on part is 1/300. So we first join partsupp with part +and then with supplier. Then we get to the correlated subquery. This +is flattened into a left join against a group by on the correlation +keys which are p.p_partkey and s.s_suppkey. The aggregate is the min +of cost. When building the group by, we add a semijoin with part that +comes from the probe side so that we do not calculate the min cost for +parts that will in any case not be probed. + +The one-line plan is: +partsupp t4*H (part t2 Build )*H (supplier t3*H (nation t5*H (region t6 Build ) project 2 columns Build ) project 7 columns Build )*H left (partsupp t8*H exists (part t2 Build )*H (supplier t9*H (nation t10*H (region t11 Build ) project 1 columns Build ) project 1 columns Build ) PARTIAL agg FINAL agg project 1 columns Build ) order by 4 columns project 8 columns + + +Note the exists with part in the subquery right of the left outer +join. The existence being more selective than the supplier join, we do +the existence first. This lan is likely the best or very close. The +group by from hte subquery is smaller than partsupp, so doing this as +a left outer join instead of right hand is better. + + + +###Q3 + +The query is straightforward to do by hash. We select 1/5 of customer +and 1/2 of orders. We first join orders x customer, build on that and +then probe on lineitem. There is nticorrelation between the date +filters on lineitem and orders but that does not affect the best plan +choice. + +###Q4 + +The trick in q4 is using a right hand semijoin to do the exists. If we +probed with orders and built on lineitem, we would get a much larger +build side. But using the right had semijoin, we get to build on the +smaller side. If we had shared key order between lineitem and orders +we could look at other plans but in the hash based plan space we have +the best outcome. + +Q5 + +The filters are on region and order date. There is also a diamond between supplier and customer, this being that they have the same nation. +We get the plan: + +lineitem t4*H (supplier t5*H (nation t6*H (region t7 Build ) project 2 columns Build ) project 4 columns Build )*H (orders t3 Build )*H (customer t2 Build ) project 2 columns PARTIAL agg FINAL agg order by 1 columns project 2 columns + + +Lineitem is the driving table that is joined to 1/5 of supplier and +then 1/7 of orders. Finally we join with customer on c_custkey and +c_nationkey. The build of customer could have been restricted on +c_nationkey being in the range of s_nationkey but we did not pick this +restriction because this would have gone through a single equality in +a join edge of two equalities. The plan is otherwise good and the +extra reduction on customer ends up not being very important. This is +a possible enhancement for completeness. + +###Q6 + +We have a single scan, so there are no query optimization choices here. + +###Q7 + +The query has filters on lineitem ship date and on customer and +supplier nation. The trick is to understand the customer from france +and supplier from Germany or customer from Germany and supplier from +France construct. Customers not from France or Germany and suppliers +not from France or Germany cannot be part of the result. The condition +is broken up and pushed down into the scans of nation that are a +reducing join against both customer and supplier. The plan that we +get: + +lineitem t3*H (supplier t2*H (nation t6 Build ) project 2 columns Build )*H (orders t4*H (customer t5*H (nation t7 Build ) project 2 columns Build ) project 2 columns Build ) filter 1 exprs project 4 columns PARTIAL agg FINAL agg order by 3 columns project 4 columns + + +first joins with supplier because this is the smaller table and the reduction is the same as the one with orders, i.e. 2/25 in both cases. The biger table is slower to probe so we reduce with the smaller one first. + +###Q8 + +The query has filters on region, selecting 1/5 of supplier or customer +and the most selective filter on part, plus 2/7 selection on orders. + +The join order of first joining with part and then orders joined with customer makes sense, doing the more reducing join first. At the tail we have supplier and the supplier's nation. + + lineitem t4*H (part t2 Build )*H (orders t5*H (customer t6*H (nation t7*H (region t9 Build ) project 1 columns Build ) project 1 columns Build ) project 2 columns Build ) + *H (supplier t3 Build )*H (nation t8 Build ) project 3 columns PARTIAL agg FINAL agg order by 1 columns project 2 columns + +###Q9 + +The plan is a natural join of lineitem, orders, part, partsupp, supplier and nation. The only selection is 1/17 of part. + +The outcome is actually quite ingenious. One would think we should begin with lineitem x part. Instead we get: + +orders t6*H +(partsupp t5*H (lineitem t4*H (part t2 Build ) project 6 columns Build ) project 7 columns Build ) +*H (supplier t3 Build )*H (nation t7 Build ) project 3 columns PARTIAL agg FINAL agg order by 2 columns project 3 columns + + +We get a complicated build side that has 1/17 of lineitem as a build +and partsupp as a probe. Since partsupp is 1/8 of lineitem, building +on the join of lineitem and part builds on the smaller, as one +should. Then we probe this with orders, which is 1/4th of lineitem, +thus the biggest table. The rest is just 1:1 joins to supplier and +nation. + + +###Q10 + +The join order is as expected, from largest to smallest. The trick +question here is that the grouping keys are functionally dependent on +c_custkey, so do not need to figure in hte group by at all. This +information is not known because the schema does not have primary key +information, so we cannot take advantage of this. + +lineitem t4*H (orders t3 Build )*H (customer t2 Build )*H (nation t5 Build ) project 8 columns PARTIAL agg FINAL agg order by 1 columns project 8 columns + + + +###Q11 + +The join order is the usual, from large to small. The only particularity is the non-correlated subquery that repeats the same join steps. An optimization opportunity could be to reuse build sides but this is not something that Velox plans support at this time. Also, Practical need for this is not very high. + +###Q12 + + +In this query, we end up building on lineitem since the filters on it +make it the smaller of the two tables. Everything else is +unsurprising. + +###Q13 + +This query has only two possible plans, left and right hand outer +join. We correctly produce the right outer join, building on the left, +i.e. customer, as it is much smaller than orders. + + +###Q14 + +The only noteworthy aspect is that we build on lineitem since its +filters (1 month out of 7 years) make it smaller than part. + +###Q15 + +The plan + +lineitem t4 project 2 columns PARTIAL agg FINAL agg project 2 columns *H (lineitem t6 project 2 columns PARTIAL agg FINAL agg project 1 columns PARTIAL agg FINAL agg project 1 columns Build )*H (supplier t2 Build ) order by 1 columns project 5 columns + +We join with the aggregation on lineitem, which is the most selective join available for the first lineitem, , then we join with supplier. + +###Q16 + +The join is biggest table first, with part joined first because it is quite selective, more so than the exists with supplier. + +Q17 + +The trick here is that we have a correlated subquery that flattens +into a group by that aggregates over all of lineitem. We correctly +observe that only lineitems with a very specific part will occur on +the probe side, so we copy the restriction inside the group by as a +semijoin (exists). + +lineitem t2*H (part t3 Build )*H left (lineitem t5*H exists (part t3 Build ) PARTIAL agg FINAL agg project 2 columns Build ) filter 1 exprs PARTIAL agg FINAL agg project 1 columns + +###Q18 + + + +##$#Q19 + +The trick is to extract common pieces to push down into the scan of +lineitem and part from the or of three ands in the single where clause. +We extract the join condition that is present in all three disjuncts +of the or. Then we extract an or to push dowbn into the scan of part +and lineitem. We build on part, as it is the smaller table. + +###Q20 + +This is one of the harder queries. The main filters are 1/25 of +supplier nations, 1/7 years of lineitem and ~1/20 of part. The larger +compute is the subquery with lineitem that adds up the volume for part +and supplier combinations. + +The subquery flattens into a left oj with a group by derived table on the right. + + + + lineitem t7 PARTIAL agg FINAL agg project 3 columns *H right + (partsupp t5*H exists-flag (part t11 project 1 columns Build ) filter 1 exprs *H exists (supplier t2*H (nation t3 Build ) project 1 columns Build ) Build ) + filter 1 exprs project 1 columns *H right exists-flag + (supplier t2*H (nation t3 Build ) Build ) + filter 1 exprs order by 1 columns project 2 columns + + +We should have the exists with part inside the aggregation on +lineitem. To do this we need to consider existences as well as inner +joins for when gathering reducing joins. + +###Q21 + +The key insight is that the exists and not exists will apply to only +1/50 or so of lineitem. The worst plan would be to build hash tables +on the contents of the subqueries. + +The plan we get builds on lineitem l1 joined to supplier joined to nation joined to orders. This is probed in a right hand semijoin by the scan of lineite l2 and this is built and then probed by lineitem l3 in a right semijoin. The rigright semijoin flags marks the build side rows that get probed by the semijoined side and once the probe is complete, flags are produced for al build side rows to indicate if they were hit. + +The plan is as expected. + +lineitem t7 project 2 columns *H right exists-flag (lineitem t9 project 2 columns *H right exists-flag (lineitem t3*H (supplier t2*H (nation t5 Build ) project 2 columns Build )*H (orders t4 Build ) Build ) filter 1 exprs Build ) filter 1 exprs PARTIAL agg FINAL agg order by 2 columns project 2 columns + +###Q22 + + +The query is straightforward, with the not exists resolved with a +right semijoin and the non-correlated subquery becoming a cross join +to the one row result set of the non-grouped aggregation.