Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id CB228200C23 for ; Wed, 8 Feb 2017 00:47:04 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id C9AF1160B68; Tue, 7 Feb 2017 23:47:04 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 7C9E0160B3E for ; Wed, 8 Feb 2017 00:47:03 +0100 (CET) Received: (qmail 74403 invoked by uid 500); 7 Feb 2017 23:47:02 -0000 Mailing-List: contact commits-help@quickstep.incubator.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@quickstep.incubator.apache.org Delivered-To: mailing list commits@quickstep.incubator.apache.org Received: (qmail 74394 invoked by uid 99); 7 Feb 2017 23:47:02 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd1-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 07 Feb 2017 23:47:02 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd1-us-west.apache.org (ASF Mail Server at spamd1-us-west.apache.org) with ESMTP id 41054C23AD for ; Tue, 7 Feb 2017 23:47:02 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd1-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -6.219 X-Spam-Level: X-Spam-Status: No, score=-6.219 tagged_above=-999 required=6.31 tests=[KAM_ASCII_DIVIDERS=0.8, KAM_LAZY_DOMAIN_SECURITY=1, RCVD_IN_DNSWL_HI=-5, RCVD_IN_MSPIKE_H3=-0.01, RCVD_IN_MSPIKE_WL=-0.01, RP_MATCHES_RCVD=-2.999] autolearn=disabled Received: from mx1-lw-eu.apache.org ([10.40.0.8]) by localhost (spamd1-us-west.apache.org [10.40.0.7]) (amavisd-new, port 10024) with ESMTP id 9PmYex3RlxK6 for ; Tue, 7 Feb 2017 23:47:00 +0000 (UTC) Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx1-lw-eu.apache.org (ASF Mail Server at mx1-lw-eu.apache.org) with SMTP id DB0F25F254 for ; Tue, 7 Feb 2017 23:46:58 +0000 (UTC) Received: (qmail 74380 invoked by uid 99); 7 Feb 2017 23:46:58 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 07 Feb 2017 23:46:58 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 14C97DFBDB; Tue, 7 Feb 2017 23:46:58 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: jignesh@apache.org To: commits@quickstep.incubator.apache.org Message-Id: X-Mailer: ASF-Git Admin Mailer Subject: incubator-quickstep git commit: Reduce the number of group-by attributes by pulling tables up aggregations Date: Tue, 7 Feb 2017 23:46:58 +0000 (UTC) archived-at: Tue, 07 Feb 2017 23:47:05 -0000 Repository: incubator-quickstep Updated Branches: refs/heads/master 2d89e4fbf -> f46ae1512 Reduce the number of group-by attributes by pulling tables up aggregations Project: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/commit/f46ae151 Tree: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/tree/f46ae151 Diff: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/diff/f46ae151 Branch: refs/heads/master Commit: f46ae1512e5b4665b855d9b3201d42773fa899e8 Parents: 2d89e4f Author: Jianqiao Zhu Authored: Sun Jan 29 18:36:14 2017 -0600 Committer: Jignesh Patel Committed: Tue Feb 7 17:45:58 2017 -0600 ---------------------------------------------------------------------- query_optimizer/CMakeLists.txt | 1 + query_optimizer/Optimizer.cpp | 3 +- query_optimizer/Optimizer.hpp | 2 - query_optimizer/PhysicalGenerator.cpp | 3 + query_optimizer/PhysicalGenerator.hpp | 11 +- query_optimizer/rules/CMakeLists.txt | 23 ++ .../rules/ReduceGroupByAttributes.cpp | 217 +++++++++++++++++++ .../rules/ReduceGroupByAttributes.hpp | 143 ++++++++++++ query_optimizer/tests/OptimizerTest.cpp | 2 +- .../tests/OptimizerTextTestRunner.cpp | 7 +- .../tests/OptimizerTextTestRunner.hpp | 3 +- 11 files changed, 405 insertions(+), 10 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f46ae151/query_optimizer/CMakeLists.txt ---------------------------------------------------------------------- diff --git a/query_optimizer/CMakeLists.txt b/query_optimizer/CMakeLists.txt index 8b8fa3c..e750a1e 100644 --- a/query_optimizer/CMakeLists.txt +++ b/query_optimizer/CMakeLists.txt @@ -216,6 +216,7 @@ target_link_libraries(quickstep_queryoptimizer_PhysicalGenerator quickstep_queryoptimizer_rules_InjectJoinFilters quickstep_queryoptimizer_rules_PruneColumns quickstep_queryoptimizer_rules_PushDownLowCostDisjunctivePredicate + quickstep_queryoptimizer_rules_ReduceGroupByAttributes quickstep_queryoptimizer_rules_ReorderColumns quickstep_queryoptimizer_rules_StarSchemaHashJoinOrderOptimization quickstep_queryoptimizer_rules_SwapProbeBuild http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f46ae151/query_optimizer/Optimizer.cpp ---------------------------------------------------------------------- diff --git a/query_optimizer/Optimizer.cpp b/query_optimizer/Optimizer.cpp index b14c938..1b91574 100644 --- a/query_optimizer/Optimizer.cpp +++ b/query_optimizer/Optimizer.cpp @@ -30,10 +30,11 @@ void Optimizer::generateQueryHandle(const ParseStatement &parse_statement, OptimizerContext *optimizer_context, QueryHandle *query_handle) { LogicalGenerator logical_generator(optimizer_context); + PhysicalGenerator physical_generator(optimizer_context); ExecutionGenerator execution_generator(catalog_database, query_handle); execution_generator.generatePlan( - physical_generator_.generatePlan( + physical_generator.generatePlan( logical_generator.generatePlan(*catalog_database, parse_statement))); } http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f46ae151/query_optimizer/Optimizer.hpp ---------------------------------------------------------------------- diff --git a/query_optimizer/Optimizer.hpp b/query_optimizer/Optimizer.hpp index 36f956a..227dd04 100644 --- a/query_optimizer/Optimizer.hpp +++ b/query_optimizer/Optimizer.hpp @@ -70,8 +70,6 @@ class Optimizer { QueryHandle *query_handle); private: - PhysicalGenerator physical_generator_; - DISALLOW_COPY_AND_ASSIGN(Optimizer); }; http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f46ae151/query_optimizer/PhysicalGenerator.cpp ---------------------------------------------------------------------- diff --git a/query_optimizer/PhysicalGenerator.cpp b/query_optimizer/PhysicalGenerator.cpp index 5dc0ffb..1b68f49 100644 --- a/query_optimizer/PhysicalGenerator.cpp +++ b/query_optimizer/PhysicalGenerator.cpp @@ -30,6 +30,7 @@ #include "query_optimizer/rules/InjectJoinFilters.hpp" #include "query_optimizer/rules/PruneColumns.hpp" #include "query_optimizer/rules/PushDownLowCostDisjunctivePredicate.hpp" +#include "query_optimizer/rules/ReduceGroupByAttributes.hpp" #include "query_optimizer/rules/ReorderColumns.hpp" #include "query_optimizer/rules/StarSchemaHashJoinOrderOptimization.hpp" #include "query_optimizer/rules/SwapProbeBuild.hpp" @@ -127,6 +128,8 @@ P::PhysicalPtr PhysicalGenerator::optimizePlan() { // general FusePhysical optimization) in the future. rules.emplace_back(new PushDownLowCostDisjunctivePredicate()); + rules.emplace_back(new ReduceGroupByAttributes(optimizer_context_)); + if (FLAGS_reorder_hash_joins) { rules.emplace_back(new StarSchemaHashJoinOrderOptimization()); rules.emplace_back(new PruneColumns()); http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f46ae151/query_optimizer/PhysicalGenerator.hpp ---------------------------------------------------------------------- diff --git a/query_optimizer/PhysicalGenerator.hpp b/query_optimizer/PhysicalGenerator.hpp index 886a173..42fea86 100644 --- a/query_optimizer/PhysicalGenerator.hpp +++ b/query_optimizer/PhysicalGenerator.hpp @@ -33,6 +33,8 @@ namespace quickstep { namespace optimizer { +class OptimizerContext; + /** \addtogroup QueryOptimizer * @{ */ @@ -43,9 +45,12 @@ namespace optimizer { class PhysicalGenerator : public LogicalToPhysicalMapper { public: /** - * @brief Constructor + * @brief Constructor. + * + * @param optimizer_context The optimizer context. */ - PhysicalGenerator() { + explicit PhysicalGenerator(OptimizerContext *optimizer_context) + : optimizer_context_(optimizer_context) { createStrategies(); } @@ -125,6 +130,8 @@ class PhysicalGenerator : public LogicalToPhysicalMapper { */ std::unordered_map logical_to_physical_map_; + OptimizerContext *optimizer_context_; + /** * @brief The complete physical plan. */ http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f46ae151/query_optimizer/rules/CMakeLists.txt ---------------------------------------------------------------------- diff --git a/query_optimizer/rules/CMakeLists.txt b/query_optimizer/rules/CMakeLists.txt index 223c78c..029d816 100644 --- a/query_optimizer/rules/CMakeLists.txt +++ b/query_optimizer/rules/CMakeLists.txt @@ -29,6 +29,9 @@ add_library(quickstep_queryoptimizer_rules_PushDownLowCostDisjunctivePredicate PushDownLowCostDisjunctivePredicate.cpp PushDownLowCostDisjunctivePredicate.hpp) add_library(quickstep_queryoptimizer_rules_PushDownSemiAntiJoin PushDownSemiAntiJoin.cpp PushDownSemiAntiJoin.hpp) +add_library(quickstep_queryoptimizer_rules_ReduceGroupByAttributes + ReduceGroupByAttributes.cpp + ReduceGroupByAttributes.hpp) add_library(quickstep_queryoptimizer_rules_ReorderColumns ReorderColumns.cpp ReorderColumns.hpp) add_library(quickstep_queryoptimizer_rules_Rule ../../empty_src.cpp Rule.hpp) add_library(quickstep_queryoptimizer_rules_RuleHelper RuleHelper.cpp RuleHelper.hpp) @@ -143,6 +146,25 @@ target_link_libraries(quickstep_queryoptimizer_rules_PushDownSemiAntiJoin quickstep_queryoptimizer_logical_PatternMatcher quickstep_queryoptimizer_rules_TopDownRule quickstep_utility_Macros) +target_link_libraries(quickstep_queryoptimizer_rules_ReduceGroupByAttributes + ${GFLAGS_LIB_NAME} + quickstep_catalog_CatalogRelation + quickstep_queryoptimizer_OptimizerContext + quickstep_queryoptimizer_costmodel_StarSchemaSimpleCostModel + quickstep_queryoptimizer_expressions_AttributeReference + quickstep_queryoptimizer_expressions_ExprId + quickstep_queryoptimizer_expressions_ExpressionUtil + quickstep_queryoptimizer_expressions_NamedExpression + quickstep_queryoptimizer_physical_Aggregate + quickstep_queryoptimizer_physical_HashJoin + quickstep_queryoptimizer_physical_PatternMatcher + quickstep_queryoptimizer_physical_Physical + quickstep_queryoptimizer_physical_PhysicalType + quickstep_queryoptimizer_physical_TableReference + quickstep_queryoptimizer_physical_TopLevelPlan + quickstep_queryoptimizer_rules_PruneColumns + quickstep_queryoptimizer_rules_Rule + quickstep_utility_Macros) target_link_libraries(quickstep_queryoptimizer_rules_ReorderColumns quickstep_queryoptimizer_expressions_AttributeReference quickstep_queryoptimizer_expressions_ExprId @@ -272,6 +294,7 @@ target_link_libraries(quickstep_queryoptimizer_rules quickstep_queryoptimizer_rules_PushDownFilter quickstep_queryoptimizer_rules_PushDownLowCostDisjunctivePredicate quickstep_queryoptimizer_rules_PushDownSemiAntiJoin + quickstep_queryoptimizer_rules_ReduceGroupByAttributes quickstep_queryoptimizer_rules_ReorderColumns quickstep_queryoptimizer_rules_Rule quickstep_queryoptimizer_rules_RuleHelper http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f46ae151/query_optimizer/rules/ReduceGroupByAttributes.cpp ---------------------------------------------------------------------- diff --git a/query_optimizer/rules/ReduceGroupByAttributes.cpp b/query_optimizer/rules/ReduceGroupByAttributes.cpp new file mode 100644 index 0000000..dcdd27a --- /dev/null +++ b/query_optimizer/rules/ReduceGroupByAttributes.cpp @@ -0,0 +1,217 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + **/ + +#include "query_optimizer/rules/ReduceGroupByAttributes.hpp" + +#include +#include +#include +#include +#include + +#include "catalog/CatalogRelation.hpp" +#include "query_optimizer/OptimizerContext.hpp" +#include "query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp" +#include "query_optimizer/expressions/AttributeReference.hpp" +#include "query_optimizer/expressions/ExprId.hpp" +#include "query_optimizer/expressions/ExpressionUtil.hpp" +#include "query_optimizer/expressions/NamedExpression.hpp" +#include "query_optimizer/physical/Aggregate.hpp" +#include "query_optimizer/physical/HashJoin.hpp" +#include "query_optimizer/physical/PatternMatcher.hpp" +#include "query_optimizer/physical/Physical.hpp" +#include "query_optimizer/physical/PhysicalType.hpp" +#include "query_optimizer/physical/TableReference.hpp" +#include "query_optimizer/physical/TopLevelPlan.hpp" +#include "query_optimizer/rules/PruneColumns.hpp" + +#include "gflags/gflags.h" + +#include "glog/logging.h" + +namespace quickstep { +namespace optimizer { + +DEFINE_uint64(reduce_group_by_attributes_threshold, 3u, + "The threshold for a stored relation's number of attributes in a " + "group-by clause for the ReduceGroupByAttributes optimization " + "rule to pull the stored relation up the aggregation"); + +namespace E = ::quickstep::optimizer::expressions; +namespace P = ::quickstep::optimizer::physical; + +P::PhysicalPtr ReduceGroupByAttributes::apply(const P::PhysicalPtr &input) { + DCHECK(input->getPhysicalType() == P::PhysicalType::kTopLevelPlan); + cost_model_.reset(new cost::StarSchemaSimpleCostModel( + std::static_pointer_cast(input)->shared_subplans())); + + P::PhysicalPtr output = applyInternal(input); + if (output != input) { + output = PruneColumns().apply(output); + } + return output; +} + +P::PhysicalPtr ReduceGroupByAttributes::applyInternal(const P::PhysicalPtr &input) { + std::vector new_children; + for (const P::PhysicalPtr &child : input->children()) { + new_children.push_back(applyInternal(child)); + } + + if (new_children != input->children()) { + return applyToNode(input->copyWithNewChildren(new_children)); + } else { + return applyToNode(input); + } +} + +P::PhysicalPtr ReduceGroupByAttributes::applyToNode(const P::PhysicalPtr &input) { + P::TableReferencePtr table_reference; + if (P::SomeTableReference::MatchesWithConditionalCast(input, &table_reference)) { + // Collect the attributes-to-TableReference mapping info. + for (const auto &attr : table_reference->attribute_list()) { + source_.emplace(attr->id(), std::make_pair(table_reference, attr)); + } + return input; + } + + P::AggregatePtr aggregate; + if (!P::SomeAggregate::MatchesWithConditionalCast(input, &aggregate) || + aggregate->grouping_expressions().size() <= 1u) { + return input; + } + + // Divide the group-by attributes into groups based on their source table. + std::map> table_attributes; + for (const auto &expr : aggregate->grouping_expressions()) { + const auto source_it = source_.find(expr->id()); + if (source_it != source_.end()) { + table_attributes[source_it->second.first].emplace_back(source_it->second.second); + } + } + + std::unordered_set erased_grouping_attr_ids; + std::vector> hoisted_tables; + + // For each group (i.e. each source table), if it is profitable then we pull + // the table up the aggregation. + for (const auto &pair : table_attributes) { + const P::TableReferencePtr table = pair.first; + const std::vector &attributes = pair.second; + // TODO(jianqiao): find a cost-based metic instead of hard-coding the threshold + // number of group-by attributes. + if (attributes.size() <= FLAGS_reduce_group_by_attributes_threshold) { + continue; + } + + std::vector attr_infos; + for (const auto &attr : attributes) { + attr_infos.emplace_back(attr, + cost_model_->impliesUniqueAttributes(table, {attr}), + !attr->getValueType().isVariableLength(), + attr->getValueType().maximumByteLength()); + } + + std::vector attr_info_refs; + for (const auto &info : attr_infos) { + attr_info_refs.emplace_back(&info); + } + std::sort(attr_info_refs.begin(), + attr_info_refs.end(), + AttributeInfo::IsBetterThan); + + const AttributeInfo &best_candidate = *attr_info_refs.front(); + if (!best_candidate.is_unique) { + // Cannot find a key attribute. Give up pulling this table up. + continue; + } + + const E::AttributeReferencePtr key_attribute = best_candidate.attribute; + hoisted_tables.emplace_back(table, key_attribute); + + for (const auto &attr : attributes) { + if (attr->id() != key_attribute->id()) { + erased_grouping_attr_ids.emplace(attr->id()); + } + } + } + + if (erased_grouping_attr_ids.empty()) { + return input; + } + + // Reconstuct the Aggregate node with reduced group-by attributes and then + // construct HashJoin nodes on top of the Aggregate. + std::vector reduced_grouping_expressions; + for (const auto &expr : aggregate->grouping_expressions()) { + if (erased_grouping_attr_ids.find(expr->id()) == erased_grouping_attr_ids.end()) { + reduced_grouping_expressions.emplace_back(expr); + } + } + + const P::AggregatePtr new_aggregate = + P::Aggregate::Create(aggregate->input(), + reduced_grouping_expressions, + aggregate->aggregate_expressions(), + aggregate->filter_predicate()); + + P::PhysicalPtr output = new_aggregate; + std::vector project_expressions = + E::ToNamedExpressions(output->getOutputAttributes()); + for (const auto &pair : hoisted_tables) { + const P::TableReferencePtr &source_table = pair.first; + const E::AttributeReferencePtr &probe_attribute = pair.second; + + E::AttributeReferencePtr build_attribute; + std::vector new_attribute_list; + for (const auto &attr : source_table->attribute_list()) { + if (attr->id() == probe_attribute->id()) { + build_attribute = + E::AttributeReference::Create(optimizer_context_->nextExprId(), + attr->attribute_name(), + attr->attribute_alias(), + attr->relation_name(), + attr->getValueType(), + E::AttributeReferenceScope::kLocal); + new_attribute_list.emplace_back(build_attribute); + } else { + new_attribute_list.emplace_back(attr); + project_expressions.emplace_back(attr); + } + } + + DCHECK(build_attribute != nullptr); + const P::TableReferencePtr build_side_table = + P::TableReference::Create(source_table->relation(), + source_table->relation()->getName(), + new_attribute_list); + output = P::HashJoin::Create(output, + build_side_table, + {probe_attribute}, + {build_attribute}, + nullptr, + project_expressions, + P::HashJoin::JoinType::kInnerJoin); + } + + return output; +} + +} // namespace optimizer +} // namespace quickstep http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f46ae151/query_optimizer/rules/ReduceGroupByAttributes.hpp ---------------------------------------------------------------------- diff --git a/query_optimizer/rules/ReduceGroupByAttributes.hpp b/query_optimizer/rules/ReduceGroupByAttributes.hpp new file mode 100644 index 0000000..5a1f295 --- /dev/null +++ b/query_optimizer/rules/ReduceGroupByAttributes.hpp @@ -0,0 +1,143 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + **/ + +#ifndef QUICKSTEP_QUERY_OPTIMIZER_RULES_REDUCE_GROUP_BY_ATTRIBUTES_HPP_ +#define QUICKSTEP_QUERY_OPTIMIZER_RULES_REDUCE_GROUP_BY_ATTRIBUTES_HPP_ + +#include +#include +#include +#include +#include + +#include "query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp" +#include "query_optimizer/expressions/AttributeReference.hpp" +#include "query_optimizer/expressions/ExprId.hpp" +#include "query_optimizer/physical/Physical.hpp" +#include "query_optimizer/physical/TableReference.hpp" +#include "query_optimizer/rules/Rule.hpp" +#include "utility/Macros.hpp" + +namespace quickstep { +namespace optimizer { + +class OptimizerContext; + +/** + * @brief Rule that applies to a physical plan to reduce the number of group-by + * attributes for Aggregate nodes (to improve performance) by pulling + * joins up the aggregations. + * + * For example, let R be a relation with PRIMARY KEY x and attributes y, z. Let + * S be a relation with FOREIGN KEY u refering to R(x) and attribute v. Then the + * optimization rule will transform the physical plan: + * Aggregate( + * [input relation]: HashJoin( + * [probe relation]: S + * [build relation]: R + * [join expression]: S.u = R.x + * [project attributes]: v, x, y, z + * ) + * [aggregate expression]: SUM(v) AS sum_v + * [group-by attributes]: x, y, z + * ) + * + * into: + * HashJoin( + * [probe relation]: Aggregate( + * [input relation]: S + * [aggregate expression]: SUM(v) AS sum_v + * [group-by attribute]: u + * ) AS T + * [build relation]: R + * [join expression]: T.u = R.x + * [project attributes]: sum_v, x, y, z + * ) + */ +class ReduceGroupByAttributes : public Rule { + public: + /** + * @brief Constructor. + * + * @param optimizer_context The optimizer context. + */ + explicit ReduceGroupByAttributes(OptimizerContext *optimizer_context) + : optimizer_context_(optimizer_context) {} + + ~ReduceGroupByAttributes() override {} + + std::string getName() const override { + return "ReduceGroupByAttributes"; + } + + physical::PhysicalPtr apply(const physical::PhysicalPtr &input) override; + + private: + struct AttributeInfo { + AttributeInfo(const expressions::AttributeReferencePtr &attribute_in, + const bool is_unique_in, + const bool is_fixed_length_in, + const std::size_t maximum_size_in) + : attribute(attribute_in), + is_unique(is_unique_in), + is_fixed_length(is_fixed_length_in), + maximum_size(maximum_size_in) {} + + // In the situation that there are multiple attributes that can serve as the + // group-by key, we define an ordering based on aggregation performance (e.g. + // it is faster to do aggregation with a fix-length attribute as the group-by + // key than with a variable-length attribute). + inline static bool IsBetterThan(const AttributeInfo *lhs, + const AttributeInfo *rhs) { + if (lhs->is_unique != rhs->is_unique) { + return lhs->is_unique; + } + if (lhs->is_fixed_length != rhs->is_fixed_length) { + return lhs->is_fixed_length; + } + if (lhs->maximum_size != rhs->maximum_size) { + return lhs->maximum_size < rhs->maximum_size; + } + return lhs->attribute->id() < rhs->attribute->id(); + } + + const expressions::AttributeReferencePtr attribute; + const bool is_unique; + const bool is_fixed_length; + const std::size_t maximum_size; + }; + + physical::PhysicalPtr applyInternal(const physical::PhysicalPtr &input); + physical::PhysicalPtr applyToNode(const physical::PhysicalPtr &input); + + OptimizerContext *optimizer_context_; + std::unique_ptr cost_model_; + + // Maps an attribute's id to the TableReference that generates the attribute. + std::unordered_map> source_; + + DISALLOW_COPY_AND_ASSIGN(ReduceGroupByAttributes); +}; + +} // namespace optimizer +} // namespace quickstep + +#endif // QUICKSTEP_QUERY_OPTIMIZER_RULES_REDUCE_GROUP_BY_ATTRIBUTES_HPP_ http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f46ae151/query_optimizer/tests/OptimizerTest.cpp ---------------------------------------------------------------------- diff --git a/query_optimizer/tests/OptimizerTest.cpp b/query_optimizer/tests/OptimizerTest.cpp index 3838638..7eb7a11 100644 --- a/query_optimizer/tests/OptimizerTest.cpp +++ b/query_optimizer/tests/OptimizerTest.cpp @@ -62,7 +62,7 @@ OptimizerTest::OptimizerTest() catalog_database_( new CatalogDatabase(catalog_.get(), "TestDatabase" /* name */, 0)), optimizer_context_(new OptimizerContext), - physical_generator_(new PhysicalGenerator()) {} + physical_generator_(new PhysicalGenerator(optimizer_context_.get())) {} E::AliasPtr OptimizerTest::createAlias(const E::ExpressionPtr &expression, const std::string &attribute_name, http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f46ae151/query_optimizer/tests/OptimizerTextTestRunner.cpp ---------------------------------------------------------------------- diff --git a/query_optimizer/tests/OptimizerTextTestRunner.cpp b/query_optimizer/tests/OptimizerTextTestRunner.cpp index b9238c9..cb8f153 100644 --- a/query_optimizer/tests/OptimizerTextTestRunner.cpp +++ b/query_optimizer/tests/OptimizerTextTestRunner.cpp @@ -80,7 +80,7 @@ void OptimizerTextTestRunner::runTestCase(const std::string &input, } if (output_physical_plan) { physical_plan = - generatePhysicalPlan(optimized_logical_plan); + generatePhysicalPlan(optimized_logical_plan, &optimizer_context); ++num_options; } @@ -126,8 +126,9 @@ logical::LogicalPtr OptimizerTextTestRunner::generateLogicalPlan( } physical::PhysicalPtr OptimizerTextTestRunner::generatePhysicalPlan( - const logical::LogicalPtr &logical_plan) { - PhysicalGenerator physical_generator; + const logical::LogicalPtr &logical_plan, + OptimizerContext *optimizer_context) { + PhysicalGenerator physical_generator(optimizer_context); return physical_generator.generatePlan(logical_plan); } http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f46ae151/query_optimizer/tests/OptimizerTextTestRunner.hpp ---------------------------------------------------------------------- diff --git a/query_optimizer/tests/OptimizerTextTestRunner.hpp b/query_optimizer/tests/OptimizerTextTestRunner.hpp index 27fa14f..d8f604b 100644 --- a/query_optimizer/tests/OptimizerTextTestRunner.hpp +++ b/query_optimizer/tests/OptimizerTextTestRunner.hpp @@ -73,7 +73,8 @@ class OptimizerTextTestRunner : public TextBasedTestRunner { OptimizerContext *optimizer_context); physical::PhysicalPtr generatePhysicalPlan( - const logical::LogicalPtr &logical_plan); + const logical::LogicalPtr &logical_plan, + OptimizerContext *optimizer_context); SqlParserWrapper sql_parser_; TestDatabaseLoader test_database_loader_;