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/10/28 19:34:24 UTC

[11/11] incubator-quickstep git commit: Analyze min/max stat for each column.

Analyze min/max stat for each column.


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

Branch: refs/heads/exact-filter
Commit: f10f5677b7b7c03ad684967a3dc0ea0cfc41502a
Parents: 7f0067b
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Thu Oct 27 14:16:32 2016 -0500
Committer: Jianqiao Zhu <ji...@cs.wisc.edu>
Committed: Fri Oct 28 14:31:52 2016 -0500

----------------------------------------------------------------------
 catalog/Catalog.proto                           |  10 +-
 catalog/CatalogRelationStatistics.cpp           |  49 ++++--
 catalog/CatalogRelationStatistics.hpp           | 151 ++++++++++++++++---
 cli/CommandExecutor.cpp                         |  34 ++++-
 cli/QuickstepCli.cpp                            |   6 +
 .../cost_model/StarSchemaSimpleCostModel.cpp    |  16 +-
 6 files changed, 220 insertions(+), 46 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f10f5677/catalog/Catalog.proto
----------------------------------------------------------------------
diff --git a/catalog/Catalog.proto b/catalog/Catalog.proto
index 90ce37e..58de363 100644
--- a/catalog/Catalog.proto
+++ b/catalog/Catalog.proto
@@ -81,13 +81,15 @@ message IndexScheme {
 }
 
 message CatalogRelationStatistics {
-  optional fixed64 num_tuples = 1;
+  optional TypedValue num_tuples = 1;
 
-  message NumDistinctValuesEntry {
+  message ColumnStats {
     required int32 attr_id = 1;
-    required fixed64 num_distinct_values = 2;
+    optional TypedValue num_distinct_values = 2;
+    optional TypedValue min_value = 3;
+    optional TypedValue max_value = 4;
   }
-  repeated NumDistinctValuesEntry num_distinct_values_map = 2;
+  repeated ColumnStats column_stats = 2;
 }
 
 message CatalogRelationSchema {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f10f5677/catalog/CatalogRelationStatistics.cpp
----------------------------------------------------------------------
diff --git a/catalog/CatalogRelationStatistics.cpp b/catalog/CatalogRelationStatistics.cpp
index 6a51570..0f67234 100644
--- a/catalog/CatalogRelationStatistics.cpp
+++ b/catalog/CatalogRelationStatistics.cpp
@@ -23,27 +23,56 @@
 
 namespace quickstep {
 
+const TypedValue CatalogRelationStatistics::kNullValue =
+    NullType::InstanceNullable().makeNullValue();
+
 CatalogRelationStatistics::CatalogRelationStatistics(
     const serialization::CatalogRelationStatistics &proto) {
   if (proto.has_num_tuples()) {
-    num_tuples_ = proto.num_tuples();
+    num_tuples_ = TypedValue::ReconstructFromProto(proto.num_tuples());
+  } else {
+    num_tuples_ = kNullValue;
   }
-  for (int i = 0; i < proto.num_distinct_values_map_size(); ++i) {
-    const auto &entry = proto.num_distinct_values_map(i);
-    num_distinct_values_map_.emplace(entry.attr_id(),
-                                     entry.num_distinct_values());
+
+  for (int i = 0; i < proto.column_stats_size(); ++i) {
+    const auto &stat_proto = proto.column_stats(i);
+    auto &stat = column_stats_[stat_proto.attr_id()];
+    if (stat_proto.has_num_distinct_values()) {
+      stat.num_distinct_values =
+          TypedValue::ReconstructFromProto(stat_proto.num_distinct_values());
+    }
+    if (stat_proto.has_min_value()) {
+      stat.min_value =
+          TypedValue::ReconstructFromProto(stat_proto.min_value());
+    }
+    if (stat_proto.has_max_value()) {
+      stat.max_value =
+          TypedValue::ReconstructFromProto(stat_proto.max_value());
+    }
   }
 }
 
 serialization::CatalogRelationStatistics CatalogRelationStatistics::getProto() const {
   serialization::CatalogRelationStatistics proto;
-  if (num_tuples_ != 0) {
-    proto.set_num_tuples(num_tuples_);
+  if (!num_tuples_.isNull()) {
+    proto.mutable_num_tuples()->CopyFrom(num_tuples_.getProto());
   }
-  for (const auto &pair : num_distinct_values_map_) {
-    auto entry = proto.add_num_distinct_values_map();
+  for (const auto &pair : column_stats_) {
+    auto entry = proto.add_column_stats();
     entry->set_attr_id(pair.first);
-    entry->set_num_distinct_values(pair.second);
+    const auto &stat = pair.second;
+    if (!stat.num_distinct_values.isNull()) {
+      entry->mutable_num_distinct_values()->CopyFrom(
+          stat.num_distinct_values.getProto());
+    }
+    if (!stat.min_value.isNull()) {
+      entry->mutable_min_value()->CopyFrom(
+          stat.min_value.getProto());
+    }
+    if (!stat.max_value.isNull()) {
+      entry->mutable_max_value()->CopyFrom(
+          stat.max_value.getProto());
+    }
   }
   return proto;
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f10f5677/catalog/CatalogRelationStatistics.hpp
----------------------------------------------------------------------
diff --git a/catalog/CatalogRelationStatistics.hpp b/catalog/CatalogRelationStatistics.hpp
index f2056f3..ecb3110 100644
--- a/catalog/CatalogRelationStatistics.hpp
+++ b/catalog/CatalogRelationStatistics.hpp
@@ -26,6 +26,9 @@
 
 #include "catalog/Catalog.pb.h"
 #include "catalog/CatalogTypedefs.hpp"
+#include "types/LongType.hpp"
+#include "types/NullType.hpp"
+#include "types/TypedValue.hpp"
 #include "utility/Macros.hpp"
 
 namespace quickstep {
@@ -44,7 +47,7 @@ class CatalogRelationStatistics {
    * @brief Constructor.
    **/
   CatalogRelationStatistics()
-      : num_tuples_(0) {}
+      : num_tuples_(kNullValue) {}
 
   /**
    * @brief Reconstruct a CatalogRelationStatistics object from its serialized
@@ -64,12 +67,12 @@ class CatalogRelationStatistics {
   serialization::CatalogRelationStatistics getProto() const;
 
   /**
-   * @brief Set the number of tuples statistic.
+   * @brief Check whether the number of tuples statistic is available.
    *
-   * @param num_tuples The number of tuples statistic.
+   * @return True if the number of tuples statistic is available, false otherwise.
    */
-  void setNumTuples(std::size_t num_tuples) {
-    num_tuples_ = num_tuples;
+  bool hasNumTuples() const {
+    return !num_tuples_.isNull();
   }
 
   /**
@@ -78,41 +81,145 @@ class CatalogRelationStatistics {
    * @return The number of tuples. Returns 0 if the statistic is not set.
    */
   std::size_t getNumTuples() const {
-    return num_tuples_;
+    DCHECK(hasNumTuples());
+    return num_tuples_.getLiteral<std::int64_t>();
+  }
+
+  /**
+   * @brief Set the number of tuples statistic.
+   *
+   * @param num_tuples The number of tuples statistic.
+   */
+  void setNumTuples(const std::size_t num_tuples) {
+    num_tuples_ = LongType::InstanceNonNullable().makeValue(&num_tuples);
+  }
+
+  /**
+   * @brief Check whether the number of distinct values statistic is available
+   *        for a column.
+   *
+   * @param The id of the column.
+   * @return True if the number of distinct values statistic is available,
+   *         false otherwise.
+   */
+  bool hasNumDistinctValues(const attribute_id attr_id) const {
+    const ColumnStats *stats = getColumnStats(attr_id);
+    return (stats != nullptr && !stats->num_distinct_values.isNull());
+  }
+
+  /**
+   * @brief Get the number of distinct values statistic for a column.
+   *
+   * @param The id of the column.
+   * @return The number of distinct values statistic for the column.
+   */
+  std::size_t getNumDistinctValues(const attribute_id attr_id) const {
+    DCHECK(hasNumDistinctValues(attr_id));
+    return column_stats_.at(attr_id).num_distinct_values.getLiteral<std::int64_t>();
   }
 
   /**
-   * @brief Set the number of distinct values statistic for a column (catalog attribute).
+   * @brief Set the number of distinct values statistic for a column.
    *
    * @param attr_id The id of the column.
    * @param num_distinct_values The number of distinct values statistic.
    */
-  void setNumDistinctValues(attribute_id attr_id, std::size_t num_distinct_values) {
-    num_distinct_values_map_[attr_id] = num_distinct_values;
+  void setNumDistinctValues(const attribute_id attr_id,
+                            const std::size_t num_distinct_values) {
+    column_stats_[attr_id].num_distinct_values =
+        LongType::InstanceNonNullable().makeValue(&num_distinct_values);
   }
 
   /**
-   * @brief Get the number of distinct values statistic for a column (catalog attribute).
+   * @brief Check whether the minimum value statistic is available for a column.
    *
    * @param The id of the column.
-   * @return The number of distinct values statistic for the column. Returns 0
-   *         if the statistic is not set.
+   * @return True if the minimum value statistic is available, false otherwise.
    */
-  std::size_t getNumDistinctValues(attribute_id attr_id) const {
-    const auto it = num_distinct_values_map_.find(attr_id);
-    if (it == num_distinct_values_map_.end()) {
-      return static_cast<std::size_t>(0);
-    } else {
-      return it->second;
-    }
+  bool hasMinValue(const attribute_id attr_id) const {
+    const ColumnStats *stats = getColumnStats(attr_id);
+    return (stats != nullptr && !stats->min_value.isNull());
+  }
+
+  /**
+   * @brief Get the minimum value statistic for a column.
+   *
+   * @param The id of the column.
+   * @return The minimum value statistic for the column.
+   */
+  const TypedValue& getMinValue(const attribute_id attr_id) const {
+    DCHECK(hasMinValue(attr_id));
+    return column_stats_.at(attr_id).min_value;
+  }
+
+  /**
+   * @brief Set the minimum value statistic for a column.
+   *
+   * @param The id of the column.
+   * @return The minimum value statistic for the column.
+   */
+  void setMinValue(const attribute_id attr_id,
+                   const TypedValue &min_value) {
+    column_stats_[attr_id].min_value = min_value;
+  }
+
+  /**
+   * @brief Check whether the maximum value statistic is available for a column.
+   *
+   * @param The id of the column.
+   * @return True if the maximum value statistic is available, false otherwise.
+   */
+  bool hasMaxValue(const attribute_id attr_id) const {
+    const ColumnStats *stats = getColumnStats(attr_id);
+    return (stats != nullptr && !stats->max_value.isNull());
+  }
+
+  /**
+   * @brief Get the maximum value statistic for a column.
+   *
+   * @param The id of the column.
+   * @return The maximum value statistic for the column.
+   */
+  const TypedValue& getMaxValue(const attribute_id attr_id) const {
+    DCHECK(hasMaxValue(attr_id));
+    return column_stats_.at(attr_id).max_value;
+  }
+
+  /**
+   * @brief Set the maximum value statistic for a column.
+   *
+   * @param The id of the column.
+   * @return The maximum value statistic for the column.
+   */
+  void setMaxValue(const attribute_id attr_id,
+                   const TypedValue &max_value) {
+    column_stats_[attr_id].max_value = max_value;
   }
 
  private:
+  struct ColumnStats {
+    ColumnStats()
+        : num_distinct_values(kNullValue),
+          min_value(kNullValue),
+          max_value(kNullValue) {
+    }
+    TypedValue num_distinct_values;
+    TypedValue min_value;
+    TypedValue max_value;
+  };
+
+  inline const ColumnStats* getColumnStats(const attribute_id attr_id) const {
+    const auto it = column_stats_.find(attr_id);
+    return (it == column_stats_.end() ? nullptr : &it->second);
+  }
+
+  static const TypedValue kNullValue;
+
   // Total number of tuples in the relation.
-  std::size_t num_tuples_;
+  TypedValue num_tuples_;
 
-  // Number of distinct values for each column.
-  std::unordered_map<attribute_id, std::size_t> num_distinct_values_map_;
+  // Statistics for each column.
+  std::unordered_map<attribute_id, ColumnStats> column_stats_;
 
   DISALLOW_COPY_AND_ASSIGN(CatalogRelationStatistics);
 };

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f10f5677/cli/CommandExecutor.cpp
----------------------------------------------------------------------
diff --git a/cli/CommandExecutor.cpp b/cli/CommandExecutor.cpp
index fea4b74..aa4a039 100644
--- a/cli/CommandExecutor.cpp
+++ b/cli/CommandExecutor.cpp
@@ -33,6 +33,8 @@
 #include "catalog/CatalogRelationSchema.hpp"
 #include "cli/DropRelation.hpp"
 #include "cli/PrintToScreen.hpp"
+#include "expressions/aggregation/AggregateFunctionMax.hpp"
+#include "expressions/aggregation/AggregateFunctionMin.hpp"
 #include "parser/ParseStatement.hpp"
 #include "parser/ParseString.hpp"
 #include "parser/SqlParserWrapper.hpp"
@@ -322,9 +324,26 @@ void executeAnalyze(const PtrVector<ParseString> *arguments,
 
     // Get the number of distinct values for each column.
     for (const CatalogAttribute &attribute : relation) {
+      const Type &attr_type = attribute.getType();
+      bool is_min_applicable =
+          AggregateFunctionMin::Instance().canApplyToTypes({&attr_type});
+      bool is_max_applicable =
+          AggregateFunctionMax::Instance().canApplyToTypes({&attr_type});
+
       std::string query_string = "SELECT COUNT(DISTINCT ";
       query_string.append(attribute.getName());
-      query_string.append(") FROM ");
+      query_string.append(")");
+      if (is_min_applicable) {
+        query_string.append(", MIN(");
+        query_string.append(attribute.getName());
+        query_string.append(")");
+      }
+      if (is_max_applicable) {
+        query_string.append(", MAX(");
+        query_string.append(attribute.getName());
+        query_string.append(")");
+      }
+      query_string.append(" FROM ");
       query_string.append(relation.getName());
       query_string.append(";");
 
@@ -340,9 +359,18 @@ void executeAnalyze(const PtrVector<ParseString> *arguments,
       auto *stat = mutable_relation->getStatisticsMutable();
       const attribute_id attr_id = attribute.getID();
 
-      DCHECK(results[0].getTypeID() == TypeID::kLong);
+      auto results_it = results.begin();
+      DCHECK(results_it->getTypeID() == TypeID::kLong);
       stat->setNumDistinctValues(attr_id,
-                                 results[0].getLiteral<std::int64_t>());
+                                 results_it->getLiteral<std::int64_t>());
+      if (is_min_applicable) {
+        ++results_it;
+        stat->setMinValue(attr_id, *results_it);
+      }
+      if (is_max_applicable) {
+        ++results_it;
+        stat->setMaxValue(attr_id, *results_it);
+      }
     }
 
     // Get the number of tuples for the relation.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f10f5677/cli/QuickstepCli.cpp
----------------------------------------------------------------------
diff --git a/cli/QuickstepCli.cpp b/cli/QuickstepCli.cpp
index 8269197..b89a0f3 100644
--- a/cli/QuickstepCli.cpp
+++ b/cli/QuickstepCli.cpp
@@ -84,6 +84,7 @@ typedef quickstep::LineReaderDumb LineReaderImpl;
 #include "gflags/gflags.h"
 
 #include "glog/logging.h"
+#include "google/protobuf/text_format.h"
 
 #include "tmb/address.h"
 #include "tmb/id_typedefs.h"
@@ -300,6 +301,11 @@ int main(int argc, char* argv[]) {
     LOG(FATAL) << "NON-STANDARD EXCEPTION DURING STARTUP";
   }
 
+  std::string str;
+  google::protobuf::TextFormat::PrintToString(
+      query_processor->getDefaultDatabase()->getProto(), &str);
+  std::cerr << str << "\n";
+
   // Parse the CPU affinities for workers and the preloader thread, if enabled
   // to warm up the buffer pool.
   const vector<int> worker_cpu_affinities =

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/f10f5677/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp b/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
index 1075739..d8682f8 100644
--- a/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
+++ b/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
@@ -25,6 +25,7 @@
 #include <vector>
 
 #include "catalog/CatalogRelation.hpp"
+#include "catalog/CatalogRelationStatistics.hpp"
 #include "query_optimizer/expressions/AttributeReference.hpp"
 #include "query_optimizer/expressions/ComparisonExpression.hpp"
 #include "query_optimizer/expressions/ExprId.hpp"
@@ -104,10 +105,11 @@ std::size_t StarSchemaSimpleCostModel::estimateCardinalityForTopLevelPlan(
 
 std::size_t StarSchemaSimpleCostModel::estimateCardinalityForTableReference(
     const P::TableReferencePtr &physical_plan) {
-  std::size_t num_tuples = physical_plan->relation()->getStatistics().getNumTuples();
-  if (num_tuples == 0) {
-    num_tuples = physical_plan->relation()->estimateTupleCardinality();
-  }
+  const CatalogRelation *relation = physical_plan->relation();
+  const CatalogRelationStatistics &stat = relation->getStatistics();
+  const std::size_t num_tuples =
+      stat.hasNumTuples() ? stat.getNumTuples()
+                          : relation->estimateTupleCardinality();
   return num_tuples;
 }
 
@@ -384,9 +386,9 @@ std::size_t StarSchemaSimpleCostModel::getNumDistinctValues(
   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) {
-      std::size_t num_distinct_values = relation.getStatistics().getNumDistinctValues(i);
-      if (num_distinct_values > 0) {
-        return num_distinct_values;
+      const CatalogRelationStatistics &stat = relation.getStatistics();
+      if (stat.hasNumDistinctValues(i)) {
+        return stat.getNumDistinctValues(i);
       }
       break;
     }