You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@quickstep.apache.org by ji...@apache.org on 2017/01/30 00:47:46 UTC

incubator-quickstep git commit: Reduce the number of group-by attributes by pulling tables up aggregations

Repository: incubator-quickstep
Updated Branches:
  refs/heads/reduce-group-by-attrs [created] 04db5cd77


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/04db5cd7
Tree: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/tree/04db5cd7
Diff: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/diff/04db5cd7

Branch: refs/heads/reduce-group-by-attrs
Commit: 04db5cd770472e9ca3ee02514727a2b9940ce1ca
Parents: 3210500
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Sun Jan 29 18:36:14 2017 -0600
Committer: Jianqiao Zhu <ji...@cs.wisc.edu>
Committed: Sun Jan 29 18:47:32 2017 -0600

----------------------------------------------------------------------
 query_optimizer/CMakeLists.txt                  |   1 +
 query_optimizer/Optimizer.cpp                   |   3 +-
 query_optimizer/Optimizer.hpp                   |   2 -
 query_optimizer/PhysicalGenerator.cpp           |   2 +
 query_optimizer/PhysicalGenerator.hpp           |  11 +-
 query_optimizer/cost_model/CMakeLists.txt       |   1 +
 .../cost_model/StarSchemaSimpleCostModel.cpp    |  79 ++++++-
 .../cost_model/StarSchemaSimpleCostModel.hpp    |  21 ++
 query_optimizer/rules/CMakeLists.txt            |  23 ++
 .../rules/ReduceGroupByAttributes.cpp           | 211 +++++++++++++++++++
 .../rules/ReduceGroupByAttributes.hpp           | 143 +++++++++++++
 query_optimizer/tests/OptimizerTest.cpp         |   2 +-
 12 files changed, 484 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/04db5cd7/query_optimizer/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/CMakeLists.txt b/query_optimizer/CMakeLists.txt
index b6c794d..587288e 100644
--- a/query_optimizer/CMakeLists.txt
+++ b/query_optimizer/CMakeLists.txt
@@ -207,6 +207,7 @@ target_link_libraries(quickstep_queryoptimizer_PhysicalGenerator
                       quickstep_queryoptimizer_physical_Physical
                       quickstep_queryoptimizer_rules_AttachLIPFilters
                       quickstep_queryoptimizer_rules_PruneColumns
+                      quickstep_queryoptimizer_rules_ReduceGroupByAttributes
                       quickstep_queryoptimizer_rules_StarSchemaHashJoinOrderOptimization
                       quickstep_queryoptimizer_rules_SwapProbeBuild
                       quickstep_queryoptimizer_strategy_Aggregate

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/04db5cd7/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/04db5cd7/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/04db5cd7/query_optimizer/PhysicalGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/PhysicalGenerator.cpp b/query_optimizer/PhysicalGenerator.cpp
index 7cb97dc..b2094e2 100644
--- a/query_optimizer/PhysicalGenerator.cpp
+++ b/query_optimizer/PhysicalGenerator.cpp
@@ -28,6 +28,7 @@
 #include "query_optimizer/physical/Physical.hpp"
 #include "query_optimizer/rules/AttachLIPFilters.hpp"
 #include "query_optimizer/rules/PruneColumns.hpp"
+#include "query_optimizer/rules/ReduceGroupByAttributes.hpp"
 #include "query_optimizer/rules/StarSchemaHashJoinOrderOptimization.hpp"
 #include "query_optimizer/rules/SwapProbeBuild.hpp"
 #include "query_optimizer/strategy/Aggregate.hpp"
@@ -103,6 +104,7 @@ P::PhysicalPtr PhysicalGenerator::generateInitialPlan(
 P::PhysicalPtr PhysicalGenerator::optimizePlan() {
   std::vector<std::unique_ptr<Rule<P::Physical>>> rules;
   rules.emplace_back(new PruneColumns());
+  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/04db5cd7/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::LogicalPtr, physical::PhysicalPtr> logical_to_physical_map_;
 
+  OptimizerContext *optimizer_context_;
+
   /**
    * @brief The complete physical plan.
    */

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/04db5cd7/query_optimizer/cost_model/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/CMakeLists.txt b/query_optimizer/cost_model/CMakeLists.txt
index 90133e7..9463b0a 100644
--- a/query_optimizer/cost_model/CMakeLists.txt
+++ b/query_optimizer/cost_model/CMakeLists.txt
@@ -49,6 +49,7 @@ target_link_libraries(quickstep_queryoptimizer_costmodel_StarSchemaSimpleCostMod
                       glog
                       quickstep_catalog_CatalogRelation
                       quickstep_catalog_CatalogRelationStatistics
+                      quickstep_catalog_CatalogTypedefs
                       quickstep_queryoptimizer_costmodel_CostModel
                       quickstep_queryoptimizer_expressions_AttributeReference
                       quickstep_queryoptimizer_expressions_ComparisonExpression

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/04db5cd7/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp b/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
index 75b1b2b..5109cb4 100644
--- a/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
+++ b/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
@@ -21,11 +21,11 @@
 
 #include <algorithm>
 #include <memory>
-#include <unordered_map>
 #include <vector>
 
 #include "catalog/CatalogRelation.hpp"
 #include "catalog/CatalogRelationStatistics.hpp"
+#include "catalog/CatalogTypedefs.hpp"
 #include "query_optimizer/cost_model/CostModel.hpp"
 #include "query_optimizer/expressions/AttributeReference.hpp"
 #include "query_optimizer/expressions/ComparisonExpression.hpp"
@@ -383,18 +383,79 @@ double StarSchemaSimpleCostModel::estimateSelectivityForPredicate(
 std::size_t StarSchemaSimpleCostModel::getNumDistinctValues(
     const E::ExprId attribute_id,
     const P::TableReferencePtr &table_reference) {
-  const CatalogRelation &relation = *table_reference->relation();
-  const std::vector<E::AttributeReferencePtr> &attributes = table_reference->attribute_list();
-  for (std::size_t i = 0; i < attributes.size(); ++i) {
-    if (attributes[i]->id() == attribute_id) {
-      const CatalogRelationStatistics &stat = relation.getStatistics();
-      if (stat.hasNumDistinctValues(i)) {
-        return stat.getNumDistinctValues(i);
+  const auto rel_attr_id =
+      findCatalogRelationAttributeId(table_reference, attribute_id);
+  if (rel_attr_id != kInvalidAttributeID) {
+    const CatalogRelationStatistics &stat =
+        table_reference->relation()->getStatistics();
+    if (stat.hasNumDistinctValues(rel_attr_id)) {
+      return stat.getNumDistinctValues(rel_attr_id);
+    }
+  }
+  return estimateCardinalityForTableReference(table_reference);
+}
+
+bool StarSchemaSimpleCostModel::impliesUniqueAttributes(
+    const P::PhysicalPtr &physical_plan,
+    const std::vector<E::AttributeReferencePtr> &attributes) {
+  switch (physical_plan->getPhysicalType()) {
+    case P::PhysicalType::kAggregate: {
+      const P::AggregatePtr &aggregate =
+          std::static_pointer_cast<const P::Aggregate>(physical_plan);
+      return E::SubsetOfExpressions(aggregate->grouping_expressions(), attributes);
+    }
+    case P::PhysicalType::kHashJoin: {
+      const P::HashJoinPtr &hash_join =
+          std::static_pointer_cast<const P::HashJoin>(physical_plan);
+      bool unique_from_left =
+          impliesUniqueAttributes(hash_join->right(), hash_join->right_join_attributes())
+              && impliesUniqueAttributes(hash_join->left(), attributes);
+      bool unique_from_right =
+          impliesUniqueAttributes(hash_join->left(), hash_join->left_join_attributes())
+              && impliesUniqueAttributes(hash_join->right(), attributes);
+      return unique_from_left || unique_from_right;
+    }
+    case P::PhysicalType::kTableReference: {
+      const P::TableReferencePtr &table_reference =
+          std::static_pointer_cast<const P::TableReference>(physical_plan);
+      const CatalogRelationStatistics &stat =
+          table_reference->relation()->getStatistics();
+      if (stat.hasNumTuples()) {
+        const std::size_t num_tuples = stat.getNumTuples();
+        for (const auto &attr : attributes) {
+          const attribute_id rel_attr_id =
+              findCatalogRelationAttributeId(table_reference, attr->id());
+          if (rel_attr_id != kInvalidAttributeID &&
+              stat.hasNumDistinctValues(rel_attr_id) &&
+              stat.getNumDistinctValues(rel_attr_id) == num_tuples) {
+            return true;
+          }
+        }
       }
+      return false;
+    }
+    case P::PhysicalType::kSample:  // Fall through
+    case P::PhysicalType::kSelection:
+    case P::PhysicalType::kSort: {
+      DCHECK_EQ(physical_plan->getNumChildren(), 1u);
+      return impliesUniqueAttributes(physical_plan->children()[0], attributes);
+    }
+    default:
       break;
+  }
+  return false;
+}
+
+attribute_id StarSchemaSimpleCostModel::findCatalogRelationAttributeId(
+    const physical::TableReferencePtr &table_reference,
+    const expressions::ExprId expr_id) {
+  const auto &attribute_list = table_reference->attribute_list();
+  for (std::size_t i = 0; i < attribute_list.size(); ++i) {
+    if (attribute_list[i]->id() == expr_id) {
+      return i;
     }
   }
-  return estimateCardinalityForTableReference(table_reference);
+  return kInvalidAttributeID;
 }
 
 }  // namespace cost

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/04db5cd7/query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp b/query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp
index 6f6aa29..d750ca0 100644
--- a/query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp
+++ b/query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp
@@ -23,7 +23,9 @@
 #include <cstddef>
 #include <vector>
 
+#include "catalog/CatalogTypedefs.hpp"
 #include "query_optimizer/cost_model/CostModel.hpp"
+#include "query_optimizer/expressions/AttributeReference.hpp"
 #include "query_optimizer/expressions/ExprId.hpp"
 #include "query_optimizer/expressions/Predicate.hpp"
 #include "query_optimizer/physical/Aggregate.hpp"
@@ -105,6 +107,21 @@ class StarSchemaSimpleCostModel : public CostModel {
   double estimateSelectivityForFilterPredicate(
       const physical::PhysicalPtr &physical_plan);
 
+  /**
+   * @brief Check whether a set of attributes are unique (i.e. have distinct
+   *        values) for a relation.
+   *
+   * @param physical_plan The physical plan that corresponds to a relation.
+   * @param attributes The set of attributes to be checked. Note that each
+   *        attribute in this set must be an output attribute of the physical
+   *        plan.
+   * @return True if it is guaranteed that the attributes are unique; false
+   *         otherwise.
+   */
+  bool impliesUniqueAttributes(
+      const physical::PhysicalPtr &physical_plan,
+      const std::vector<expressions::AttributeReferencePtr> &attributes);
+
  private:
   std::size_t estimateCardinalityForAggregate(
       const physical::AggregatePtr &physical_plan);
@@ -144,6 +161,10 @@ class StarSchemaSimpleCostModel : public CostModel {
   std::size_t getNumDistinctValues(const expressions::ExprId attribute_id,
                                    const physical::TableReferencePtr &table_reference);
 
+  // For a table reference attribute, find its correponding catalog attribute.
+  attribute_id findCatalogRelationAttributeId(
+      const physical::TableReferencePtr &table_reference,
+      const expressions::ExprId expr_id);
 
   DISALLOW_COPY_AND_ASSIGN(StarSchemaSimpleCostModel);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/04db5cd7/query_optimizer/rules/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/CMakeLists.txt b/query_optimizer/rules/CMakeLists.txt
index 7fffadc..7176a4a 100644
--- a/query_optimizer/rules/CMakeLists.txt
+++ b/query_optimizer/rules/CMakeLists.txt
@@ -25,6 +25,9 @@ add_library(quickstep_queryoptimizer_rules_GenerateJoins GenerateJoins.cpp Gener
 add_library(quickstep_queryoptimizer_rules_PruneColumns PruneColumns.cpp PruneColumns.hpp)
 add_library(quickstep_queryoptimizer_rules_PushDownFilter PushDownFilter.cpp PushDownFilter.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_Rule ../../empty_src.cpp Rule.hpp)
 add_library(quickstep_queryoptimizer_rules_RuleHelper RuleHelper.cpp RuleHelper.hpp)
 add_library(quickstep_queryoptimizer_rules_StarSchemaHashJoinOrderOptimization
@@ -118,6 +121,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
+                      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_types_TypeID
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_queryoptimizer_rules_Rule
                       glog
                       quickstep_utility_Macros)
@@ -213,6 +235,7 @@ target_link_libraries(quickstep_queryoptimizer_rules
                       quickstep_queryoptimizer_rules_PruneColumns
                       quickstep_queryoptimizer_rules_PushDownFilter
                       quickstep_queryoptimizer_rules_PushDownSemiAntiJoin
+                      quickstep_queryoptimizer_rules_ReduceGroupByAttributes
                       quickstep_queryoptimizer_rules_Rule
                       quickstep_queryoptimizer_rules_RuleHelper
                       quickstep_queryoptimizer_rules_StarSchemaHashJoinOrderOptimization

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/04db5cd7/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..99e17b3
--- /dev/null
+++ b/query_optimizer/rules/ReduceGroupByAttributes.cpp
@@ -0,0 +1,211 @@
+/**
+ * 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 <algorithm>
+#include <map>
+#include <vector>
+#include <unordered_set>
+#include <utility>
+
+#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 "types/TypeID.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+namespace optimizer {
+
+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<const P::TopLevelPlan>(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<P::PhysicalPtr> 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<P::TableReferencePtr, std::vector<E::AttributeReferencePtr>> 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<E::ExprId> erased_grouping_attr_ids;
+  std::vector<std::pair<P::TableReferencePtr, E::AttributeReferencePtr>> 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<E::AttributeReferencePtr> &attributes = pair.second;
+    // TODO(jianqiao): find a cost-based metic instead of hard-coding the threshold
+    // number of group-by attributes.
+    if (attributes.size() <= 3u) {
+      continue;
+    }
+
+    std::vector<AttributeInfo> 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<const AttributeInfo *> 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<E::NamedExpressionPtr> 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<E::NamedExpressionPtr> 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<E::AttributeReferencePtr> 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/04db5cd7/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 <cstddef>
+#include <memory>
+#include <string>
+#include <unordered_map>
+#include <utility>
+
+#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<physical::Physical> {
+ 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::StarSchemaSimpleCostModel> cost_model_;
+
+  // Maps an attribute's id to the TableReference that generates the attribute.
+  std::unordered_map<expressions::ExprId,
+                     std::pair<physical::TableReferencePtr,
+                               expressions::AttributeReferencePtr>> 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/04db5cd7/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,