Skip to content
This repository has been archived by the owner on May 9, 2024. It is now read-only.

Commit

Permalink
Support partitioned aggregation.
Browse files Browse the repository at this point in the history
Signed-off-by: ienkovich <[email protected]>
  • Loading branch information
ienkovich committed Jun 28, 2023
1 parent aab3f75 commit e1d6f60
Show file tree
Hide file tree
Showing 37 changed files with 1,479 additions and 142 deletions.
39 changes: 39 additions & 0 deletions omniscidb/ConfigBuilder/ConfigBuilder.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -183,6 +183,39 @@ bool ConfigBuilder::parseCommandLineArgs(int argc,
po::value<size_t>(&config_->exec.group_by.large_ndv_multiplier)
->default_value(config_->exec.group_by.large_ndv_multiplier),
"A multiplier applied to NDV estimator buffer size for large ranges.");
opt_desc.add_options()(
"enable-cpu-partitioned-groupby",
po::value<bool>(&config_->exec.group_by.enable_cpu_partitioned_groupby)
->default_value(config_->exec.group_by.enable_cpu_partitioned_groupby)
->implicit_value(true),
"Enable partitioned aggregation on CPU.");
opt_desc.add_options()(
"groupby-partitioning-buffer-size-threshold",
po::value<size_t>(&config_->exec.group_by.partitioning_buffer_size_threshold)
->default_value(config_->exec.group_by.partitioning_buffer_size_threshold),
"Minimal estimated output buffer size to enable partitioned aggregation.");
opt_desc.add_options()(
"groupby-partitioning-group-size-threshold",
po::value<double>(&config_->exec.group_by.partitioning_group_size_threshold)
->default_value(config_->exec.group_by.partitioning_group_size_threshold),
"Maximum average estimated number of rows per output group to enable partitioned "
"aggregation.");
opt_desc.add_options()("groupby-min-partitions",
po::value<size_t>(&config_->exec.group_by.min_partitions)
->default_value(config_->exec.group_by.min_partitions),
"A minimal number of partitions to be used for partitioned "
"aggregation. 0 value is used for auto-detection.");
opt_desc.add_options()("groupby-max-partitions",
po::value<size_t>(&config_->exec.group_by.max_partitions)
->default_value(config_->exec.group_by.max_partitions),
"A maximum number of partitions to be used for partitioned "
"aggregation.");
opt_desc.add_options()(
"groupby-partitioning-target-buffer-size",
po::value<size_t>(&config_->exec.group_by.partitioning_buffer_target_size)
->default_value(config_->exec.group_by.partitioning_buffer_target_size),
"A preferred aggregation output buffer size used to compute number of partitions "
"to use.");

// exec.window
opt_desc.add_options()("enable-window-functions",
Expand Down Expand Up @@ -349,6 +382,12 @@ bool ConfigBuilder::parseCommandLineArgs(int argc,
->implicit_value(true),
"Enable multi-fragment intermediate results to improve execution parallelism for "
"queries with multiple execution steps");
opt_desc.add_options()(
"enable-multifrag-execution-result",
po::value<bool>(&config_->exec.enable_multifrag_execution_result)
->default_value(config_->exec.enable_multifrag_execution_result)
->implicit_value(true),
"Enable multi-fragment final execution result");
opt_desc.add_options()("gpu-block-size",
po::value<size_t>(&config_->exec.override_gpu_block_size)
->default_value(config_->exec.override_gpu_block_size),
Expand Down
11 changes: 11 additions & 0 deletions omniscidb/IR/ExprCollector.h
Original file line number Diff line number Diff line change
Expand Up @@ -25,10 +25,21 @@ class ExprCollector : public ExprVisitor<void> {
return collect(expr.get(), std::forward<Ts>(args)...);
}

template <typename... Ts>
static ResultType collect(const ExprPtrVector& exprs, Ts&&... args) {
CollectorType collector(std::forward<Ts>(args)...);
for (auto& expr : exprs) {
collector.visit(expr.get());
}
return std::move(collector.result_);
}

ResultType& result() { return result_; }
const ResultType& result() const { return result_; }

protected:
using BaseClass = ExprCollector<ResultType, CollectorType>;

ResultType result_;
};

Expand Down
1 change: 1 addition & 0 deletions omniscidb/IR/ExprVisitor.h
Original file line number Diff line number Diff line change
Expand Up @@ -115,6 +115,7 @@ class ExprVisitor {
if (auto agg = expr->as<AggExpr>()) {
return visitAggExpr(agg);
}
CHECK(false) << "Unhandled expr: " << expr->toString();
return defaultResult(expr);
}

Expand Down
101 changes: 98 additions & 3 deletions omniscidb/IR/Node.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -284,6 +284,90 @@ void LogicalUnion::checkForMatchingMetaInfoTypes() const {
}
}

size_t ShuffleFunction::hash() const {
size_t res = 0;
boost::hash_combine(res, kind);
boost::hash_combine(res, partitions);
return res;
}

std::string ShuffleFunction::toString() const {
std::stringstream ss;
ss << kind << "(" << partitions << ")";
return ss.str();
}

std::ostream& operator<<(std::ostream& os, const ShuffleFunction& fn) {
os << fn.toString();
return os;
}

std::ostream& operator<<(std::ostream& os, ShuffleFunction::Kind kind) {
os << ::toString(kind);
return os;
}

Shuffle::Shuffle(ExprPtrVector keys,
ExprPtr expr,
std::string field,
ShuffleFunction fn,
NodePtr input)
: keys_(keys), exprs_({std::move(expr)}), fields_({std::move(field)}), fn_(fn) {
inputs_.emplace_back(std::move(input));
}

Shuffle::Shuffle(ExprPtrVector keys,
ExprPtrVector exprs,
std::vector<std::string> fields,
ShuffleFunction fn,
std::vector<NodePtr> inputs)
: keys_(keys), exprs_(std::move(exprs)), fields_(std::move(fields)), fn_(fn) {
inputs_ = std::move(inputs);
}

std::string Shuffle::toString() const {
return cat(::typeName(this),
getIdString(),
"(keys=",
::toString(keys_),
", exprs=",
::toString(exprs_),
", fields",
::toString(fields_),
", fn=",
::toString(fn_),
")");
}

size_t Shuffle::toHash() const {
if (!hash_) {
hash_ = typeid(Shuffle).hash_code();
for (auto& expr : keys_) {
boost::hash_combine(*hash_, expr->hash());
}
for (auto& expr : exprs_) {
boost::hash_combine(*hash_, expr->hash());
}
for (auto& field : fields_) {
boost::hash_combine(*hash_, field);
}
boost::hash_combine(*hash_, fn_.hash());
for (auto& node : inputs_) {
boost::hash_combine(*hash_, node->toHash());
}
}
return *hash_;
}

void Shuffle::rewriteExprs(hdk::ir::ExprRewriter& rewriter) {
for (size_t i = 0; i < keys_.size(); ++i) {
keys_[i] = rewriter.visit(keys_[i].get());
}
for (size_t i = 0; i < exprs_.size(); ++i) {
exprs_[i] = rewriter.visit(exprs_[i].get());
}
}

namespace {

void collectNodes(NodePtr node, std::vector<NodePtr> nodes) {
Expand Down Expand Up @@ -318,7 +402,7 @@ void QueryDag::resetQueryExecutionState() {
// TODO: always simply use node->size()
size_t getNodeColumnCount(const Node* node) {
// Nodes that don't depend on input.
if (is_one_of<Scan, Project, Aggregate, LogicalUnion, LogicalValues>(node)) {
if (is_one_of<Scan, Project, Aggregate, LogicalUnion, LogicalValues, Shuffle>(node)) {
return node->size();
}

Expand Down Expand Up @@ -356,7 +440,8 @@ ExprPtrVector getNodeColumnRefs(const Node* node) {
LogicalValues,
Filter,
Sort,
Join>(node)) {
Join,
Shuffle>(node)) {
return genColumnRefs(node, getNodeColumnCount(node));
}

Expand All @@ -374,7 +459,8 @@ ExprPtr getNodeColumnRef(const Node* node, unsigned index) {
LogicalValues,
Filter,
Sort,
Join>(node)) {
Join,
Shuffle>(node)) {
return makeExpr<ColumnRef>(getColumnType(node, index), node, index);
}

Expand Down Expand Up @@ -458,3 +544,12 @@ const Type* getColumnType(const Node* node, size_t col_idx) {
}

} // namespace hdk::ir

std::string toString(hdk::ir::ShuffleFunction::Kind kind) {
switch (kind) {
case hdk::ir::ShuffleFunction::kHash:
return "Hash";
}
LOG(FATAL) << "Invalid shuffle kind.";
return "";
}
82 changes: 80 additions & 2 deletions omniscidb/IR/Node.h
Original file line number Diff line number Diff line change
Expand Up @@ -377,7 +377,9 @@ class Aggregate : public Node {
NodePtr input)
: groupby_count_(groupby_count)
, aggs_(std::move(aggs))
, fields_(std::move(fields)) {
, fields_(std::move(fields))
, partitioned_(false)
, buffer_entry_count_hint_(0) {
inputs_.emplace_back(std::move(input));
}

Expand Down Expand Up @@ -406,6 +408,12 @@ class Aggregate : public Node {

void setAggExprs(ExprPtrVector new_aggs) { aggs_ = std::move(new_aggs); }

bool isPartitioned() const { return partitioned_; }
void setPartitioned(bool val) { partitioned_ = val; }

size_t bufferEntryCountHint() const { return buffer_entry_count_hint_; }
void setBufferEntryCountHint(size_t val) { buffer_entry_count_hint_ = val; }

void rewriteExprs(hdk::ir::ExprRewriter& rewriter) override;

std::string toString() const override {
Expand All @@ -417,6 +425,7 @@ class Aggregate : public Node {
::toString(aggs_),
", fields=",
::toString(fields_),
(partitioned_ ? ", partitioned" : ""),
", inputs=",
inputsToString(inputs_),
")");
Expand Down Expand Up @@ -445,6 +454,8 @@ class Aggregate : public Node {
const size_t groupby_count_;
ExprPtrVector aggs_;
std::vector<std::string> fields_;
bool partitioned_;
size_t buffer_entry_count_hint_;
};

class Join : public Node {
Expand Down Expand Up @@ -593,7 +604,9 @@ class TranslatedJoin : public Node {
CHECK(false);
return nullptr;
}
const std::string& getFieldName(size_t i) const override { CHECK(false); }
const std::string& getFieldName(size_t i) const override {
throw std::runtime_error("Unexpected call to TranslatedJoin::getFieldName.");
}
std::vector<const ColumnVar*> getJoinCols(bool lhs) const {
if (lhs) {
return lhs_join_cols_;
Expand Down Expand Up @@ -853,6 +866,69 @@ class LogicalUnion : public Node {
bool const is_all_;
};
struct ShuffleFunction {
enum Kind {
kHash,
};
Kind kind;
size_t partitions;
size_t hash() const;
std::string toString() const;
};
std::ostream& operator<<(std::ostream& os, const ShuffleFunction& fn);
std::ostream& operator<<(std::ostream& os, ShuffleFunction::Kind kind);
class Shuffle : public Node {
public:
Shuffle(ExprPtrVector keys,
ExprPtr expr,
std::string field,
ShuffleFunction fn,
NodePtr input);
Shuffle(ExprPtrVector keys,
ExprPtrVector exprs,
std::vector<std::string> fields,
ShuffleFunction fn,
std::vector<NodePtr> input);
Shuffle(const Shuffle& other) = default;
const ExprPtrVector& keys() const { return keys_; }
const ExprPtrVector& exprs() const { return exprs_; }
const std::vector<std::string>& fields() const { return fields_; }
ShuffleFunction fn() const { return fn_; }
size_t size() const override { return exprs_.size(); }
// Shuffle node can be used for computing partition sizes and perform
// actual partitioning. The first version uses COUNT aggregte as its
// only target expression.
bool isCount() const {
return exprs_.size() == (size_t)1 && exprs_.front()->is<AggExpr>();
}
std::string toString() const override;
size_t toHash() const override;
void rewriteExprs(hdk::ir::ExprRewriter& rewriter) override;
std::shared_ptr<Node> deepCopy() const override {
return std::make_shared<Shuffle>(*this);
}
const std::string& getFieldName(size_t i) const override {
CHECK_LT(i, fields_.size());
return fields_[i];
}
private:
ExprPtrVector keys_;
ExprPtrVector exprs_;
std::vector<std::string> fields_;
ShuffleFunction fn_;
};
class QueryNotSupported : public std::runtime_error {
public:
QueryNotSupported(const std::string& reason) : std::runtime_error(reason) {}
Expand Down Expand Up @@ -921,3 +997,5 @@ size_t getNodeColumnCount(const Node* node);
ExprPtr getJoinInputColumnRef(const ColumnRef* col_ref);
} // namespace hdk::ir
std::string toString(hdk::ir::ShuffleFunction::Kind kind);
3 changes: 3 additions & 0 deletions omniscidb/QueryEngine/CardinalityEstimator.cpp
Original file line number Diff line number Diff line change
Expand Up @@ -114,6 +114,9 @@ RelAlgExecutionUnit create_count_all_execution_unit(
ra_exe_unit.hash_table_build_plan_dag,
ra_exe_unit.table_id_to_node_map,
ra_exe_unit.union_all,
ra_exe_unit.shuffle_fn,
ra_exe_unit.partition_offsets_col,
ra_exe_unit.partitioned_aggregation,
ra_exe_unit.cost_model,
{}}; // TODO(bagrorg): should we use costmodel here?
}
Expand Down
16 changes: 16 additions & 0 deletions omniscidb/QueryEngine/CardinalityEstimator.h
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,22 @@ class CardinalityEstimationRequired : public std::runtime_error {
const int64_t range_;
};

class RequestPartitionedAggregation : public std::runtime_error {
public:
RequestPartitionedAggregation(size_t entry_size, size_t estimated_buffer_entries)
: std::runtime_error("RequestPartitionedAggregation")
, entry_size_(entry_size)
, estimated_buffer_entries_(estimated_buffer_entries) {}

size_t entrySize() const { return entry_size_; }
size_t estimatedBufferEntries() const { return estimated_buffer_entries_; }
size_t estimatedBufferSize() const { return entry_size_ * estimated_buffer_entries_; }

private:
size_t entry_size_;
size_t estimated_buffer_entries_;
};

RelAlgExecutionUnit create_ndv_execution_unit(const RelAlgExecutionUnit& ra_exe_unit,
SchemaProvider* schema_provider,
const Config& config,
Expand Down
Loading

0 comments on commit e1d6f60

Please sign in to comment.