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 07:02:29 UTC

incubator-quickstep git commit: Push down low cost disjunctive predicates to filter the stored relations early

Repository: incubator-quickstep
Updated Branches:
  refs/heads/push-down-disjunctive-predicate [created] f3e06b528


Push down low cost disjunctive predicates to filter the stored relations early


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

Branch: refs/heads/push-down-disjunctive-predicate
Commit: f3e06b528255e60cb94060fd53fff709ff4fe428
Parents: 0780b84
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Mon Jan 30 01:02:19 2017 -0600
Committer: Jianqiao Zhu <ji...@cs.wisc.edu>
Committed: Mon Jan 30 01:02:19 2017 -0600

----------------------------------------------------------------------
 query_optimizer/CMakeLists.txt                  |   1 +
 query_optimizer/PhysicalGenerator.cpp           |   2 +
 query_optimizer/rules/CMakeLists.txt            |  23 ++
 .../PushDownLowCostDisjunctivePredicate.cpp     | 213 +++++++++++++++++++
 .../PushDownLowCostDisjunctivePredicate.hpp     |  98 +++++++++
 5 files changed, 337 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f3e06b52/query_optimizer/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/CMakeLists.txt b/query_optimizer/CMakeLists.txt
index b6c794d..3bb60d4 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_PushDownLowCostDisjunctivePredicate
                       quickstep_queryoptimizer_rules_StarSchemaHashJoinOrderOptimization
                       quickstep_queryoptimizer_rules_SwapProbeBuild
                       quickstep_queryoptimizer_strategy_Aggregate

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f3e06b52/query_optimizer/PhysicalGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/PhysicalGenerator.cpp b/query_optimizer/PhysicalGenerator.cpp
index 7cb97dc..03ae913 100644
--- a/query_optimizer/PhysicalGenerator.cpp
+++ b/query_optimizer/PhysicalGenerator.cpp
@@ -27,6 +27,7 @@
 #include "query_optimizer/logical/Logical.hpp"
 #include "query_optimizer/physical/Physical.hpp"
 #include "query_optimizer/rules/AttachLIPFilters.hpp"
+#include "query_optimizer/rules/PushDownLowCostDisjunctivePredicate.hpp"
 #include "query_optimizer/rules/PruneColumns.hpp"
 #include "query_optimizer/rules/StarSchemaHashJoinOrderOptimization.hpp"
 #include "query_optimizer/rules/SwapProbeBuild.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 PushDownLowCostDisjunctivePredicate());
   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/f3e06b52/query_optimizer/rules/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/CMakeLists.txt b/query_optimizer/rules/CMakeLists.txt
index 7fffadc..b62e99e 100644
--- a/query_optimizer/rules/CMakeLists.txt
+++ b/query_optimizer/rules/CMakeLists.txt
@@ -24,6 +24,9 @@ add_library(quickstep_queryoptimizer_rules_CollapseProject CollapseProject.cpp C
 add_library(quickstep_queryoptimizer_rules_GenerateJoins GenerateJoins.cpp GenerateJoins.hpp)
 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_PushDownLowCostDisjunctivePredicate
+            PushDownLowCostDisjunctivePredicate.cpp
+            PushDownLowCostDisjunctivePredicate.hpp)
 add_library(quickstep_queryoptimizer_rules_PushDownSemiAntiJoin PushDownSemiAntiJoin.cpp PushDownSemiAntiJoin.hpp)
 add_library(quickstep_queryoptimizer_rules_Rule ../../empty_src.cpp Rule.hpp)
 add_library(quickstep_queryoptimizer_rules_RuleHelper RuleHelper.cpp RuleHelper.hpp)
@@ -110,6 +113,25 @@ target_link_libraries(quickstep_queryoptimizer_rules_PushDownFilter
                       quickstep_queryoptimizer_rules_RuleHelper
                       quickstep_queryoptimizer_rules_TopDownRule
                       quickstep_utility_Macros)
+target_link_libraries(quickstep_queryoptimizer_rules_PushDownLowCostDisjunctivePredicate
+                      quickstep_queryoptimizer_costmodel_StarSchemaSimpleCostModel
+                      quickstep_queryoptimizer_expressions_AttributeReference
+                      quickstep_queryoptimizer_expressions_ExpressionUtil
+                      quickstep_queryoptimizer_expressions_LogicalAnd
+                      quickstep_queryoptimizer_expressions_LogicalOr
+                      quickstep_queryoptimizer_expressions_PatternMatcher
+                      quickstep_queryoptimizer_expressions_Predicate
+                      quickstep_queryoptimizer_physical_Aggregate
+                      quickstep_queryoptimizer_physical_HashJoin
+                      quickstep_queryoptimizer_physical_NestedLoopsJoin
+                      quickstep_queryoptimizer_physical_PatternMatcher
+                      quickstep_queryoptimizer_physical_Physical
+                      quickstep_queryoptimizer_physical_PhysicalType
+                      quickstep_queryoptimizer_physical_Selection
+                      quickstep_queryoptimizer_physical_TableReference
+                      quickstep_queryoptimizer_physical_TopLevelPlan
+                      quickstep_queryoptimizer_rules_Rule
+                      quickstep_utility_Macros)
 target_link_libraries(quickstep_queryoptimizer_rules_PushDownSemiAntiJoin
                       quickstep_queryoptimizer_expressions_AttributeReference
                       quickstep_queryoptimizer_expressions_ExpressionUtil
@@ -212,6 +234,7 @@ target_link_libraries(quickstep_queryoptimizer_rules
                       quickstep_queryoptimizer_rules_GenerateJoins
                       quickstep_queryoptimizer_rules_PruneColumns
                       quickstep_queryoptimizer_rules_PushDownFilter
+                      quickstep_queryoptimizer_rules_PushDownLowCostDisjunctivePredicate
                       quickstep_queryoptimizer_rules_PushDownSemiAntiJoin
                       quickstep_queryoptimizer_rules_Rule
                       quickstep_queryoptimizer_rules_RuleHelper

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f3e06b52/query_optimizer/rules/PushDownLowCostDisjunctivePredicate.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/PushDownLowCostDisjunctivePredicate.cpp b/query_optimizer/rules/PushDownLowCostDisjunctivePredicate.cpp
new file mode 100644
index 0000000..5a7bbde
--- /dev/null
+++ b/query_optimizer/rules/PushDownLowCostDisjunctivePredicate.cpp
@@ -0,0 +1,213 @@
+/**
+ * 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/PushDownLowCostDisjunctivePredicate.hpp"
+
+#include <cstddef>
+#include <vector>
+
+#include "query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp"
+#include "query_optimizer/expressions/AttributeReference.hpp"
+#include "query_optimizer/expressions/ExpressionUtil.hpp"
+#include "query_optimizer/expressions/LogicalAnd.hpp"
+#include "query_optimizer/expressions/LogicalOr.hpp"
+#include "query_optimizer/expressions/PatternMatcher.hpp"
+#include "query_optimizer/expressions/Predicate.hpp"
+#include "query_optimizer/physical/Aggregate.hpp"
+#include "query_optimizer/physical/HashJoin.hpp"
+#include "query_optimizer/physical/NestedLoopsJoin.hpp"
+#include "query_optimizer/physical/PatternMatcher.hpp"
+#include "query_optimizer/physical/Physical.hpp"
+#include "query_optimizer/physical/PhysicalType.hpp"
+#include "query_optimizer/physical/Selection.hpp"
+#include "query_optimizer/physical/TableReference.hpp"
+#include "query_optimizer/physical/TopLevelPlan.hpp"
+
+#include "glog/logging.h"
+
+namespace quickstep {
+namespace optimizer {
+
+namespace E = ::quickstep::optimizer::expressions;
+namespace P = ::quickstep::optimizer::physical;
+
+P::PhysicalPtr PushDownLowCostDisjunctivePredicate::apply(const P::PhysicalPtr &input) {
+  DCHECK(input->getPhysicalType() == P::PhysicalType::kTopLevelPlan);
+
+  const P::TopLevelPlanPtr top_level_plan =
+     std::static_pointer_cast<const P::TopLevelPlan>(input);
+  cost_model_.reset(
+      new cost::StarSchemaSimpleCostModel(
+          top_level_plan->shared_subplans()));
+
+  collectApplicablePredicates(input);
+
+  if (!applicable_predicates_.empty()) {
+    // Apply the selected predicates to stored relations.
+    return attachPredicates(input);
+  } else {
+    return input;
+  }
+}
+
+void PushDownLowCostDisjunctivePredicate::collectApplicablePredicates(
+    const physical::PhysicalPtr &input) {
+  P::TableReferencePtr table_reference;
+  if (P::SomeTableReference::MatchesWithConditionalCast(input, &table_reference)) {
+    // Consider only stored relations with small cardinality as targets.
+    if (cost_model_->estimateCardinality(input) <= 100u) {
+      applicable_nodes_.emplace_back(input, &table_reference->attribute_list());
+    }
+    return;
+  }
+
+  for (const auto &child : input->children()) {
+    collectApplicablePredicates(child);
+  }
+
+  E::PredicatePtr filter_predicate = nullptr;
+  switch (input->getPhysicalType()) {
+    case P::PhysicalType::kAggregate: {
+      filter_predicate =
+          std::static_pointer_cast<const P::Aggregate>(input)->filter_predicate();
+      break;
+    }
+    case P::PhysicalType::kHashJoin: {
+      const P::HashJoinPtr hash_join =
+          std::static_pointer_cast<const P::HashJoin>(input);
+      if (hash_join->join_type() == P::HashJoin::JoinType::kInnerJoin) {
+        filter_predicate = hash_join->residual_predicate();
+      }
+      break;
+    }
+    case P::PhysicalType::kNestedLoopsJoin: {
+      filter_predicate =
+          std::static_pointer_cast<const P::NestedLoopsJoin>(input)->join_predicate();
+      break;
+    }
+    case P::PhysicalType::kSelection: {
+      filter_predicate =
+          std::static_pointer_cast<const P::Selection>(input)->filter_predicate();
+      break;
+    }
+    default:
+      break;
+  }
+
+  E::LogicalOrPtr disjunctive_predicate;
+  if (filter_predicate == nullptr ||
+      !E::SomeLogicalOr::MatchesWithConditionalCast(filter_predicate, &disjunctive_predicate)) {
+    return;
+  }
+
+  // Consider only disjunctive normal form, i.e. disjunction of conjunctions.
+  // Divide the disjunctive components into groups.
+  std::vector<std::vector<E::PredicatePtr>> candidate_predicates;
+  std::vector<std::vector<std::vector<E::AttributeReferencePtr>>> candidate_attributes;
+  for (const auto &conjunctive_predicate : disjunctive_predicate->operands()) {
+    candidate_predicates.emplace_back();
+    candidate_attributes.emplace_back();
+    E::LogicalAndPtr logical_and;
+    if (E::SomeLogicalAnd::MatchesWithConditionalCast(conjunctive_predicate, &logical_and)) {
+      for (const auto &predicate : logical_and->operands()) {
+        candidate_predicates.back().emplace_back(predicate);
+        candidate_attributes.back().emplace_back(
+            predicate->getReferencedAttributes());
+      }
+    } else {
+      candidate_predicates.back().emplace_back(conjunctive_predicate);
+      candidate_attributes.back().emplace_back(
+          conjunctive_predicate->getReferencedAttributes());
+    }
+  }
+
+  // Checks whether each group of predicates can be applied to small-cardinality
+  // stored relations.
+  for (const auto &node_pair : applicable_nodes_) {
+    const std::vector<E::AttributeReferencePtr> &target_attributes = *node_pair.second;
+    std::vector<E::PredicatePtr> selected_disj_preds;
+    for (std::size_t i = 0; i < candidate_predicates.size(); ++i) {
+      const auto &cand_preds = candidate_predicates[i];
+      const auto &cand_attrs = candidate_attributes[i];
+
+      std::vector<E::PredicatePtr> selected_conj_preds;
+      for (std::size_t j = 0; j < cand_preds.size(); ++j) {
+        if (E::SubsetOfExpressions(cand_attrs[j], target_attributes)) {
+          selected_conj_preds.emplace_back(cand_preds[j]);
+        }
+      }
+      if (!selected_conj_preds.empty()) {
+        selected_disj_preds.emplace_back(
+            CreateConjunctive(selected_conj_preds));
+      }
+    }
+    if (!selected_disj_preds.empty()) {
+      applicable_predicates_[node_pair.first].add(
+          CreateDisjunctive(selected_disj_preds));
+    }
+  }
+}
+
+P::PhysicalPtr PushDownLowCostDisjunctivePredicate::attachPredicates(
+    const P::PhysicalPtr &input) const {
+  std::vector<P::PhysicalPtr> new_children;
+  for (const P::PhysicalPtr &child : input->children()) {
+    const P::PhysicalPtr new_child = attachPredicates(child);
+    new_children.push_back(new_child);
+  }
+
+  const P::PhysicalPtr output =
+      new_children == input->children() ? input
+                                        : input->copyWithNewChildren(new_children);
+
+  const auto &node_it = applicable_predicates_.find(input);
+  if (node_it != applicable_predicates_.end()) {
+    const E::PredicatePtr filter_predicate =
+        CreateConjunctive(node_it->second.predicates);
+    return P::Selection::Create(output,
+                                E::ToNamedExpressions(output->getOutputAttributes()),
+                                filter_predicate);
+  }
+
+  return output;
+}
+
+E::PredicatePtr PushDownLowCostDisjunctivePredicate::CreateConjunctive(
+    const std::vector<E::PredicatePtr> predicates) {
+  DCHECK_GE(predicates.size(), 1u);
+  if (predicates.size() == 1) {
+    return predicates.front();
+  } else {
+    return E::LogicalAnd::Create(predicates);
+  }
+}
+
+E::PredicatePtr PushDownLowCostDisjunctivePredicate::CreateDisjunctive(
+    const std::vector<E::PredicatePtr> predicates) {
+  DCHECK_GE(predicates.size(), 1u);
+  if (predicates.size() == 1) {
+    return predicates.front();
+  } else {
+    return E::LogicalOr::Create(predicates);
+  }
+}
+
+
+}  // namespace optimizer
+}  // namespace quickstep

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f3e06b52/query_optimizer/rules/PushDownLowCostDisjunctivePredicate.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/rules/PushDownLowCostDisjunctivePredicate.hpp b/query_optimizer/rules/PushDownLowCostDisjunctivePredicate.hpp
new file mode 100644
index 0000000..bddf2d8
--- /dev/null
+++ b/query_optimizer/rules/PushDownLowCostDisjunctivePredicate.hpp
@@ -0,0 +1,98 @@
+/**
+ * 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_PUSH_DOWN_LOW_COST_DISJUNCTIVE_PREDICATE_HPP_
+#define QUICKSTEP_QUERY_OPTIMIZER_RULES_PUSH_DOWN_LOW_COST_DISJUNCTIVE_PREDICATE_HPP_
+
+#include <cstddef>
+#include <map>
+#include <memory>
+#include <string>
+#include <utility>
+#include <vector>
+
+#include "query_optimizer/cost_model/StarSchemaSimpleCostModel.hpp"
+#include "query_optimizer/expressions/AttributeReference.hpp"
+#include "query_optimizer/expressions/Predicate.hpp"
+#include "query_optimizer/physical/Physical.hpp"
+#include "query_optimizer/rules/Rule.hpp"
+#include "utility/Macros.hpp"
+
+namespace quickstep {
+namespace optimizer {
+
+/** \addtogroup OptimizerRules
+ *  @{
+ */
+
+/**
+ * @brief Rule that applies to a physical plan to push down low-cost disjunctive
+ *        predicate.
+ */
+class PushDownLowCostDisjunctivePredicate : public Rule<physical::Physical> {
+ public:
+  /**
+   * @brief Constructor.
+   */
+  PushDownLowCostDisjunctivePredicate() {}
+
+  ~PushDownLowCostDisjunctivePredicate() override {}
+
+  std::string getName() const override {
+    return "PushDownLowCostDisjunctivePredicate";
+  }
+
+  physical::PhysicalPtr apply(const physical::PhysicalPtr &input) override;
+
+ private:
+  struct PredicateInfo {
+    PredicateInfo() {}
+    inline void add(expressions::PredicatePtr predicate) {
+      predicates.emplace_back(predicate);
+    }
+    std::vector<expressions::PredicatePtr> predicates;
+  };
+
+  void collectApplicablePredicates(const physical::PhysicalPtr &input);
+  physical::PhysicalPtr attachPredicates(const physical::PhysicalPtr &input) const;
+
+  static expressions::PredicatePtr CreateConjunctive(
+      const std::vector<expressions::PredicatePtr> predicates);
+
+  static expressions::PredicatePtr CreateDisjunctive(
+      const std::vector<expressions::PredicatePtr> predicates);
+
+  std::unique_ptr<cost::StarSchemaSimpleCostModel> cost_model_;
+
+  std::vector<std::pair<physical::PhysicalPtr,
+                        const std::vector<expressions::AttributeReferencePtr> *>> applicable_nodes_;
+  std::map<physical::PhysicalPtr, PredicateInfo> applicable_predicates_;
+
+  // Currently consider only stored relations with small cardinality as targets.
+  static constexpr std::size_t kCardinalityThreshold = 100u;
+
+  DISALLOW_COPY_AND_ASSIGN(PushDownLowCostDisjunctivePredicate);
+};
+
+/** @} */
+
+}  // namespace optimizer
+}  // namespace quickstep
+
+#endif  // QUICKSTEP_QUERY_OPTIMIZER_RULES_PUSH_DOWN_LOW_COST_DISJUNCTIVE_PREDICATE_HPP_