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 2016/08/04 01:56:35 UTC

[06/13] incubator-quickstep git commit: Implemented hashjoin optimization class and removed the logic from ExecutionGenerator.

Implemented hashjoin optimization class and removed the logic from ExecutionGenerator.


Project: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/commit/a61b99e9
Tree: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/tree/a61b99e9
Diff: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/diff/a61b99e9

Branch: refs/heads/LIP-for-tpch
Commit: a61b99e9e1fcbbe84c60d63b8277cbb67e518030
Parents: ccea2ff
Author: Hakan Memisoglu <ha...@gmail.com>
Authored: Mon Aug 1 16:39:07 2016 -0500
Committer: Jianqiao Zhu <ji...@cs.wisc.edu>
Committed: Wed Aug 3 03:10:58 2016 -0500

----------------------------------------------------------------------
 query_optimizer/CMakeLists.txt                  |  1 +
 query_optimizer/ExecutionGenerator.cpp          | 19 +-----
 query_optimizer/PhysicalGenerator.cpp           |  2 +
 query_optimizer/physical/HashJoin.hpp           |  1 +
 query_optimizer/rules/BottomUpRule.hpp          | 10 ++++
 query_optimizer/rules/CMakeLists.txt            | 12 ++++
 query_optimizer/rules/SwapProbeBuild.cpp        | 62 ++++++++++++++++++++
 query_optimizer/rules/SwapProbeBuild.hpp        | 48 +++++++++++++++
 .../tests/physical_generator/Select.test        | 36 ++++++------
 9 files changed, 156 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a61b99e9/query_optimizer/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/CMakeLists.txt b/query_optimizer/CMakeLists.txt
index a56b714..c55881f 100644
--- a/query_optimizer/CMakeLists.txt
+++ b/query_optimizer/CMakeLists.txt
@@ -199,6 +199,7 @@ target_link_libraries(quickstep_queryoptimizer_PhysicalGenerator
                       quickstep_queryoptimizer_physical_Physical
                       quickstep_queryoptimizer_rules_PruneColumns
                       quickstep_queryoptimizer_rules_StarSchemaHashJoinOrderOptimization
+                      quickstep_queryoptimizer_rules_SwapProbeBuild
                       quickstep_queryoptimizer_strategy_Aggregate
                       quickstep_queryoptimizer_strategy_Join
                       quickstep_queryoptimizer_strategy_OneToOne

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a61b99e9/query_optimizer/ExecutionGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.cpp b/query_optimizer/ExecutionGenerator.cpp
index 88103df..fb24489 100644
--- a/query_optimizer/ExecutionGenerator.cpp
+++ b/query_optimizer/ExecutionGenerator.cpp
@@ -604,6 +604,8 @@ void ExecutionGenerator::convertHashJoin(const P::HashJoinPtr &physical_plan) {
   const CatalogRelation *referenced_stored_probe_relation = nullptr;
   const CatalogRelation *referenced_stored_build_relation = nullptr;
 
+  std::size_t build_cardinality = cost_model_->estimateCardinality(build_physical);
+
   bool any_probe_attributes_nullable = false;
   bool any_build_attributes_nullable = false;
 
@@ -671,23 +673,6 @@ void ExecutionGenerator::convertHashJoin(const P::HashJoinPtr &physical_plan) {
     key_types.push_back(&left_attribute_type);
   }
 
-  std::size_t probe_cardinality = cost_model_->estimateCardinality(probe_physical);
-  std::size_t build_cardinality = cost_model_->estimateCardinality(build_physical);
-  // For inner join, we may swap the probe table and the build table.
-  if (physical_plan->join_type() == P::HashJoin::JoinType::kInnerJoin)  {
-    // Choose the smaller table as the inner build table,
-    // and the other one as the outer probe table.
-    if (probe_cardinality < build_cardinality) {
-      // Switch the probe and build physical nodes.
-      std::swap(probe_physical, build_physical);
-      std::swap(probe_cardinality, build_cardinality);
-      std::swap(probe_attribute_ids, build_attribute_ids);
-      std::swap(any_probe_attributes_nullable, any_build_attributes_nullable);
-      std::swap(probe_original_attribute_ids, build_original_attribute_ids);
-      std::swap(referenced_stored_probe_relation, referenced_stored_build_relation);
-    }
-  }
-
   // Convert the residual predicate proto.
   QueryContext::predicate_id residual_predicate_index = QueryContext::kInvalidPredicateId;
   if (physical_plan->residual_predicate()) {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a61b99e9/query_optimizer/PhysicalGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/PhysicalGenerator.cpp b/query_optimizer/PhysicalGenerator.cpp
index 75a7bc9..897b212 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/PruneColumns.hpp"
 #include "query_optimizer/rules/StarSchemaHashJoinOrderOptimization.hpp"
+#include "query_optimizer/rules/SwapProbeBuild.hpp"
 #include "query_optimizer/strategy/Aggregate.hpp"
 #include "query_optimizer/strategy/Join.hpp"
 #include "query_optimizer/strategy/OneToOne.hpp"
@@ -98,6 +99,7 @@ P::PhysicalPtr PhysicalGenerator::optimizePlan() {
     rules.emplace_back(new StarSchemaHashJoinOrderOptimization());
   }
   rules.emplace_back(new PruneColumns());
+  rules.emplace_back(new SwapProbeBuild());
 
   for (std::unique_ptr<Rule<P::Physical>> &rule : rules) {
     physical_plan_ = rule->apply(physical_plan_);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a61b99e9/query_optimizer/physical/HashJoin.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/physical/HashJoin.hpp b/query_optimizer/physical/HashJoin.hpp
index b904b5f..988b139 100644
--- a/query_optimizer/physical/HashJoin.hpp
+++ b/query_optimizer/physical/HashJoin.hpp
@@ -20,6 +20,7 @@
 #ifndef QUICKSTEP_QUERY_OPTIMIZER_PHYSICAL_HASHJOIN_HPP_
 #define QUICKSTEP_QUERY_OPTIMIZER_PHYSICAL_HASHJOIN_HPP_
 
+#include <cstddef>
 #include <memory>
 #include <string>
 #include <type_traits>

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a61b99e9/query_optimizer/rules/BottomUpRule.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/BottomUpRule.hpp b/query_optimizer/rules/BottomUpRule.hpp
index 0acc199..f98dadf 100644
--- a/query_optimizer/rules/BottomUpRule.hpp
+++ b/query_optimizer/rules/BottomUpRule.hpp
@@ -54,6 +54,7 @@ class BottomUpRule : public Rule<TreeType> {
   TreeNodePtr apply(const TreeNodePtr &tree) override {
     DCHECK(tree != nullptr);
 
+    init(tree);
     std::vector<std::shared_ptr<const TreeType>> new_children;
     bool has_changed_children = false;
     for (const std::shared_ptr<const TreeType> &child : tree->children()) {
@@ -80,6 +81,15 @@ class BottomUpRule : public Rule<TreeType> {
    */
   virtual TreeNodePtr applyToNode(const TreeNodePtr &node) = 0;
 
+  /**
+   * @brief Override this method to implement the initialization code
+   *        for the rule.
+   *
+   * @param input The input tree.
+   */
+  virtual void init(const TreeNodePtr &input) {
+  }
+
  private:
   DISALLOW_COPY_AND_ASSIGN(BottomUpRule);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a61b99e9/query_optimizer/rules/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/CMakeLists.txt b/query_optimizer/rules/CMakeLists.txt
index 1990174..04a9814 100644
--- a/query_optimizer/rules/CMakeLists.txt
+++ b/query_optimizer/rules/CMakeLists.txt
@@ -29,6 +29,7 @@ add_library(quickstep_queryoptimizer_rules_RuleHelper RuleHelper.cpp RuleHelper.
 add_library(quickstep_queryoptimizer_rules_StarSchemaHashJoinOrderOptimization
             StarSchemaHashJoinOrderOptimization.cpp
             StarSchemaHashJoinOrderOptimization.hpp)
+add_library(quickstep_queryoptimizer_rules_SwapProbeBuild SwapProbeBuild.cpp SwapProbeBuild.hpp)
 add_library(quickstep_queryoptimizer_rules_TopDownRule ../../empty_src.cpp TopDownRule.hpp)
 add_library(quickstep_queryoptimizer_rules_UpdateExpression UpdateExpression.cpp UpdateExpression.hpp)
 add_library(quickstep_queryoptimizer_rules_UnnestSubqueries UnnestSubqueries.cpp UnnestSubqueries.hpp)
@@ -127,6 +128,16 @@ target_link_libraries(quickstep_queryoptimizer_rules_StarSchemaHashJoinOrderOpti
                       quickstep_queryoptimizer_physical_TopLevelPlan
                       quickstep_queryoptimizer_rules_Rule
                       quickstep_utility_Macros)
+target_link_libraries(quickstep_queryoptimizer_rules_SwapProbeBuild
+                      quickstep_queryoptimizer_costmodel_SimpleCostModel
+                      quickstep_queryoptimizer_expressions_AttributeReference
+                      quickstep_queryoptimizer_physical_HashJoin
+                      quickstep_queryoptimizer_physical_PatternMatcher
+                      quickstep_queryoptimizer_physical_Physical
+                      quickstep_queryoptimizer_physical_TopLevelPlan
+                      quickstep_queryoptimizer_rules_BottomUpRule
+                      quickstep_queryoptimizer_rules_Rule
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_queryoptimizer_rules_TopDownRule
                       quickstep_queryoptimizer_rules_Rule
                       quickstep_utility_Macros)
@@ -185,6 +196,7 @@ target_link_libraries(quickstep_queryoptimizer_rules
                       quickstep_queryoptimizer_rules_Rule
                       quickstep_queryoptimizer_rules_RuleHelper
                       quickstep_queryoptimizer_rules_StarSchemaHashJoinOrderOptimization
+                      quickstep_queryoptimizer_rules_SwapProbeBuild
                       quickstep_queryoptimizer_rules_TopDownRule
                       quickstep_queryoptimizer_rules_UpdateExpression
                       quickstep_queryoptimizer_rules_UnnestSubqueries)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a61b99e9/query_optimizer/rules/SwapProbeBuild.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/SwapProbeBuild.cpp b/query_optimizer/rules/SwapProbeBuild.cpp
new file mode 100644
index 0000000..4b4b38f
--- /dev/null
+++ b/query_optimizer/rules/SwapProbeBuild.cpp
@@ -0,0 +1,62 @@
+#include "query_optimizer/rules/SwapProbeBuild.hpp"
+
+#include <cstddef>
+#include <memory>
+#include <vector>
+
+#include "query_optimizer/expressions/AttributeReference.hpp"
+#include "query_optimizer/physical/HashJoin.hpp"
+#include "query_optimizer/physical/PatternMatcher.hpp"
+#include "query_optimizer/physical/Physical.hpp"
+#include "query_optimizer/physical/TopLevelPlan.hpp"
+#include "query_optimizer/rules/Rule.hpp"
+
+
+namespace quickstep {
+namespace optimizer {
+
+P::PhysicalPtr SwapProbeBuild::applyToNode(const P::PhysicalPtr &input) {
+  P::HashJoinPtr hash_join;
+
+  if (P::SomeHashJoin::MatchesWithConditionalCast(input, &hash_join)
+      && hash_join->join_type() == P::HashJoin::JoinType::kInnerJoin) {
+    P::PhysicalPtr left = hash_join->left();
+    P::PhysicalPtr right = hash_join->right();
+
+    std::size_t left_cardinality = cost_model_->estimateCardinality(left);
+    std::size_t right_cardinality = cost_model_->estimateCardinality(right);
+
+    if (right_cardinality > left_cardinality) {
+      std::vector<E::AttributeReferencePtr> left_join_attributes = hash_join->left_join_attributes();
+      std::vector<E::AttributeReferencePtr> right_join_attributes = hash_join->right_join_attributes();
+
+      P::PhysicalPtr output = P::HashJoin::Create(right,
+                                                  left,
+                                                  right_join_attributes,
+                                                  left_join_attributes,
+                                                  hash_join->residual_predicate(),
+                                                  hash_join->project_expressions(),
+                                                  hash_join->join_type());
+      LOG_APPLYING_RULE(input, output);
+      return output;
+    }
+  }
+
+  LOG_IGNORING_RULE(input);
+  return input;
+}
+
+void SwapProbeBuild::init(const P::PhysicalPtr &input) {
+  if (cost_model_ == nullptr) {
+    P::TopLevelPlanPtr top_level;
+    if (P::SomeTopLevelPlan::MatchesWithConditionalCast(input, &top_level)) {
+      cost_model_.reset(new C::SimpleCostModel(top_level->shared_subplans()));
+    } else {
+      std::vector<P::PhysicalPtr> plans = {input};
+      cost_model_.reset(new C::SimpleCostModel(plans));
+    }
+  }
+}
+
+}  // namespace optimizer
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a61b99e9/query_optimizer/rules/SwapProbeBuild.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/SwapProbeBuild.hpp b/query_optimizer/rules/SwapProbeBuild.hpp
new file mode 100644
index 0000000..da8fdf9
--- /dev/null
+++ b/query_optimizer/rules/SwapProbeBuild.hpp
@@ -0,0 +1,48 @@
+#ifndef QUICKSTEP_QUERY_OPTIMIZER_RULES_SWAP_PROBE_BUILD_HPP_
+#define QUICKSTEP_QUERY_OPTIMIZER_RULES_SWAP_PROBE_BUILD_HPP_
+
+#include <memory>
+#include <string>
+
+#include "query_optimizer/physical/Physical.hpp"
+#include "query_optimizer/rules/Rule.hpp"
+#include "query_optimizer/rules/BottomUpRule.hpp"
+#include "query_optimizer/cost_model/SimpleCostModel.hpp"
+#include "utility/Macros.hpp"
+
+namespace quickstep {
+namespace optimizer {
+
+/** \addtogroup OptimizerRules
+ *  @{
+ */
+
+namespace P = ::quickstep::optimizer::physical;
+namespace E = ::quickstep::optimizer::expressions;
+namespace C = ::quickstep::optimizer::cost;
+
+/**
+ * @brief Rule that applies to a physical plan to arrange probe and
+ *        build side based on the cardinalities.
+ */
+class SwapProbeBuild : public BottomUpRule<P::Physical> {
+ public:
+  SwapProbeBuild() {
+  }
+
+  std::string getName() const override { return "SwapProbeBuild"; }
+
+ protected:
+  P::PhysicalPtr applyToNode(const P::PhysicalPtr &input) override;
+  void init(const P::PhysicalPtr &input) override;
+
+ private:
+  std::unique_ptr<C::SimpleCostModel> cost_model_;
+
+  DISALLOW_COPY_AND_ASSIGN(SwapProbeBuild);
+};
+
+}  // namespace optimizer
+}  // namespace quickstep
+
+#endif

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/a61b99e9/query_optimizer/tests/physical_generator/Select.test
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/physical_generator/Select.test b/query_optimizer/tests/physical_generator/Select.test
index d99916c..76c5683 100644
--- a/query_optimizer/tests/physical_generator/Select.test
+++ b/query_optimizer/tests/physical_generator/Select.test
@@ -2232,10 +2232,7 @@ TopLevelPlan
 [Physical Plan]
 TopLevelPlan
 +-plan=HashJoin
-| +-left=TableReference[relation=b]
-| | +-AttributeReference[id=0,name=w,relation=b,type=Int]
-| | +-AttributeReference[id=1,name=x,relation=b,type=Int]
-| +-right=Aggregate
+| +-left=Aggregate
 | | +-input=TableReference[relation=c]
 | | | +-AttributeReference[id=2,name=x,relation=c,type=Int]
 | | | +-AttributeReference[id=3,name=y,relation=c,type=Int]
@@ -2245,6 +2242,9 @@ TopLevelPlan
 | |   +-Alias[id=4,name=,alias=$aggregate0,relation=$aggregate,type=Long NULL]
 | |     +-AggregateFunction[function=SUM]
 | |       +-AttributeReference[id=3,name=y,relation=c,type=Int]
+| +-right=TableReference[relation=b]
+| | +-AttributeReference[id=0,name=w,relation=b,type=Int]
+| | +-AttributeReference[id=1,name=x,relation=b,type=Int]
 | +-project_expressions=
 | | +-Alias[id=5,name=,alias=(x+SubqueryExpression),relation=,type=Long NULL]
 | |   +-Add
@@ -2252,9 +2252,9 @@ TopLevelPlan
 | |     +-AttributeReference[id=4,name=,alias=$aggregate0,relation=$aggregate,
 | |       type=Long NULL]
 | +-left_join_attributes=
-| | +-AttributeReference[id=0,name=w,relation=b,type=Int]
+| | +-AttributeReference[id=2,name=x,relation=c,type=Int]
 | +-right_join_attributes=
-|   +-AttributeReference[id=2,name=x,relation=c,type=Int]
+|   +-AttributeReference[id=0,name=w,relation=b,type=Int]
 +-output_attributes=
   +-AttributeReference[id=5,name=,alias=(x+SubqueryExpression),relation=,
     type=Long NULL]
@@ -2369,10 +2369,7 @@ TopLevelPlan
 [Physical Plan]
 TopLevelPlan
 +-plan=HashJoin
-| +-left=TableReference[relation=b]
-| | +-AttributeReference[id=0,name=w,relation=b,type=Int]
-| | +-AttributeReference[id=1,name=x,relation=b,type=Int]
-| +-right=Aggregate
+| +-left=Aggregate
 | | +-input=TableReference[relation=c]
 | | | +-AttributeReference[id=2,name=x,relation=c,type=Int]
 | | | +-AttributeReference[id=3,name=y,relation=c,type=Int]
@@ -2382,6 +2379,9 @@ TopLevelPlan
 | |   +-Alias[id=4,name=,alias=$aggregate0,relation=$aggregate,type=Long NULL]
 | |     +-AggregateFunction[function=SUM]
 | |       +-AttributeReference[id=3,name=y,relation=c,type=Int]
+| +-right=TableReference[relation=b]
+| | +-AttributeReference[id=0,name=w,relation=b,type=Int]
+| | +-AttributeReference[id=1,name=x,relation=b,type=Int]
 | +-residual_predicate=Greater
 | | +-AttributeReference[id=1,name=x,relation=b,type=Int]
 | | +-AttributeReference[id=4,name=,alias=$aggregate0,relation=$aggregate,
@@ -2390,9 +2390,9 @@ TopLevelPlan
 | | +-AttributeReference[id=0,name=w,relation=b,type=Int]
 | | +-AttributeReference[id=1,name=x,relation=b,type=Int]
 | +-left_join_attributes=
-| | +-AttributeReference[id=0,name=w,relation=b,type=Int]
+| | +-AttributeReference[id=2,name=x,relation=c,type=Int]
 | +-right_join_attributes=
-|   +-AttributeReference[id=2,name=x,relation=c,type=Int]
+|   +-AttributeReference[id=0,name=w,relation=b,type=Int]
 +-output_attributes=
   +-AttributeReference[id=0,name=w,relation=b,type=Int]
   +-AttributeReference[id=1,name=x,relation=b,type=Int]
@@ -2476,10 +2476,7 @@ TopLevelPlan
 [Physical Plan]
 TopLevelPlan
 +-plan=HashJoin
-| +-left=TableReference[relation=b]
-| | +-AttributeReference[id=0,name=w,relation=b,type=Int]
-| | +-AttributeReference[id=1,name=x,relation=b,type=Int]
-| +-right=NestedLoopsJoin
+| +-left=NestedLoopsJoin
 | | +-left=Aggregate
 | | | +-input=TableReference[relation=c]
 | | | | +-AttributeReference[id=2,name=x,relation=c,type=Int]
@@ -2518,6 +2515,9 @@ TopLevelPlan
 | |       | type=Long NULL]
 | |       +-AttributeReference[id=9,name=,alias=$aggregate0,relation=$aggregate,
 | |         type=Long NULL]
+| +-right=TableReference[relation=b]
+| | +-AttributeReference[id=0,name=w,relation=b,type=Int]
+| | +-AttributeReference[id=1,name=x,relation=b,type=Int]
 | +-project_expressions=
 | | +-Alias[id=11,name=,alias=(x+SubqueryExpression),relation=,type=Long NULL]
 | |   +-Add
@@ -2525,9 +2525,9 @@ TopLevelPlan
 | |     +-AttributeReference[id=10,name=,alias=(SUM(y)+SubqueryExpression),
 | |       relation=,type=Long NULL]
 | +-left_join_attributes=
-| | +-AttributeReference[id=0,name=w,relation=b,type=Int]
+| | +-AttributeReference[id=2,name=x,relation=c,type=Int]
 | +-right_join_attributes=
-|   +-AttributeReference[id=2,name=x,relation=c,type=Int]
+|   +-AttributeReference[id=0,name=w,relation=b,type=Int]
 +-output_attributes=
   +-AttributeReference[id=11,name=,alias=(x+SubqueryExpression),relation=,
     type=Long NULL]