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/11 20:27:54 UTC

[14/16] incubator-quickstep git commit: MinMaxStatistics

MinMaxStatistics


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

Branch: refs/heads/LIP-for-tpch
Commit: 9b90665796962baa9a80ef5240db99d6b38a9016
Parents: ebdc041
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Thu Aug 11 03:21:30 2016 -0500
Committer: Jianqiao Zhu <ji...@cs.wisc.edu>
Committed: Thu Aug 11 15:23:00 2016 -0500

----------------------------------------------------------------------
 catalog/Catalog.proto                 | 10 +--
 catalog/CatalogRelationStatistics.cpp | 50 ++++++++++++---
 catalog/CatalogRelationStatistics.hpp | 93 +++++++++++++++++++++++-----
 cli/CommandExecutor.cpp               | 98 ++++++++++++++++++++++--------
 cli/QuickstepCli.cpp                  |  8 +--
 5 files changed, 201 insertions(+), 58 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9b906657/catalog/Catalog.proto
----------------------------------------------------------------------
diff --git a/catalog/Catalog.proto b/catalog/Catalog.proto
index a4cc3ce..6423a7c 100644
--- a/catalog/Catalog.proto
+++ b/catalog/Catalog.proto
@@ -85,13 +85,15 @@ message CatalogRelationConstraints {
 }
 
 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/9b906657/catalog/CatalogRelationStatistics.cpp
----------------------------------------------------------------------
diff --git a/catalog/CatalogRelationStatistics.cpp b/catalog/CatalogRelationStatistics.cpp
index 6a51570..04939a5 100644
--- a/catalog/CatalogRelationStatistics.cpp
+++ b/catalog/CatalogRelationStatistics.cpp
@@ -20,30 +20,60 @@
 #include "catalog/CatalogRelationStatistics.hpp"
 
 #include "catalog/Catalog.pb.h"
+#include "types/NullType.hpp"
 
 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/9b906657/catalog/CatalogRelationStatistics.hpp
----------------------------------------------------------------------
diff --git a/catalog/CatalogRelationStatistics.hpp b/catalog/CatalogRelationStatistics.hpp
index f2056f3..532ea46 100644
--- a/catalog/CatalogRelationStatistics.hpp
+++ b/catalog/CatalogRelationStatistics.hpp
@@ -26,8 +26,13 @@
 
 #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"
 
+#include "glog/logging.h"
+
 namespace quickstep {
 
 /** \addtogroup Catalog
@@ -44,7 +49,7 @@ class CatalogRelationStatistics {
    * @brief Constructor.
    **/
   CatalogRelationStatistics()
-      : num_tuples_(0) {}
+      : num_tuples_(kNullValue) {}
 
   /**
    * @brief Reconstruct a CatalogRelationStatistics object from its serialized
@@ -68,8 +73,12 @@ class CatalogRelationStatistics {
    *
    * @param num_tuples The number of tuples statistic.
    */
-  void setNumTuples(std::size_t num_tuples) {
-    num_tuples_ = num_tuples;
+  void setNumTuples(const std::size_t num_tuples) {
+    num_tuples_ = LongType::InstanceNonNullable().makeValue(&num_tuples);
+  }
+
+  bool hasNumTuples() const {
+    return !num_tuples_.isNull();
   }
 
   /**
@@ -78,7 +87,8 @@ 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>();
   }
 
   /**
@@ -87,8 +97,15 @@ class CatalogRelationStatistics {
    * @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);
+  }
+
+  bool hasNumDistinctValues(const attribute_id attr_id) const {
+    const auto stats = getColumnStats(attr_id);
+    return (stats != nullptr && !stats->num_distinct_values.isNull());
   }
 
   /**
@@ -98,21 +115,65 @@ class CatalogRelationStatistics {
    * @return The number of distinct values statistic for the column. Returns 0
    *         if the statistic is not set.
    */
-  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;
-    }
+  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>();
+  }
+
+  void setMinValue(const attribute_id attr_id,
+                   const TypedValue &min_value) {
+    column_stats_[attr_id].min_value = min_value;
+  }
+
+  bool hasMinValue(const attribute_id attr_id) const {
+    const auto stats = getColumnStats(attr_id);
+    return (stats != nullptr && !stats->min_value.isNull());
+  }
+
+  const TypedValue& getMinValue(const attribute_id attr_id) const {
+    DCHECK(hasMinValue(attr_id));
+    return column_stats_.at(attr_id).min_value;
+  }
+
+  void setMaxValue(const attribute_id attr_id,
+                   const TypedValue &max_value) {
+    column_stats_[attr_id].max_value = max_value;
+  }
+
+  bool hasMaxValue(const attribute_id attr_id) const {
+    const auto stats = getColumnStats(attr_id);
+    return (stats != nullptr && !stats->max_value.isNull());
+  }
+
+  const TypedValue& getMaxValue(const attribute_id attr_id) const {
+    DCHECK(hasMaxValue(attr_id));
+    return column_stats_.at(attr_id).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/9b906657/cli/CommandExecutor.cpp
----------------------------------------------------------------------
diff --git a/cli/CommandExecutor.cpp b/cli/CommandExecutor.cpp
index f27f1ef..feafec1 100644
--- a/cli/CommandExecutor.cpp
+++ b/cli/CommandExecutor.cpp
@@ -196,10 +196,7 @@ void executeDescribeTable(
   }
 }
 
-/**
- * @brief A helper function that executes a SQL query to obtain a scalar result.
- */
-inline TypedValue executeQueryForSingleResult(
+inline std::vector<TypedValue> executeQueryForSingleRow(
     const tmb::client_id main_thread_client_id,
     const tmb::client_id foreman_client_id,
     const std::string &query_string,
@@ -232,22 +229,29 @@ inline TypedValue executeQueryForSingleResult(
   const CatalogRelation *query_result_relation = query_handle->getQueryResultRelation();
   DCHECK(query_result_relation != nullptr);
 
-  TypedValue value;
+  std::vector<TypedValue> values;
   {
     std::vector<block_id> blocks = query_result_relation->getBlocksSnapshot();
     DCHECK_EQ(1u, blocks.size());
+
     BlockReference block = storage_manager->getBlock(blocks[0], *query_result_relation);
     const TupleStorageSubBlock &tuple_store = block->getTupleStorageSubBlock();
     DCHECK_EQ(1, tuple_store.numTuples());
-    DCHECK_EQ(1u, tuple_store.getRelation().size());
 
+    const std::size_t num_columns = tuple_store.getRelation().size();
     if (tuple_store.isPacked()) {
-      value = tuple_store.getAttributeValueTyped(0, 0);
+      for (std::size_t i = 0; i < num_columns; ++i) {
+        values.emplace_back(tuple_store.getAttributeValueTyped(0, i));
+        values[i].ensureNotReference();
+      }
     } else {
       std::unique_ptr<TupleIdSequence> existence_map(tuple_store.getExistenceMap());
-      value = tuple_store.getAttributeValueTyped(*existence_map->begin(), 0);
+      for (std::size_t i = 0; i < num_columns; ++i) {
+        values.emplace_back(
+            tuple_store.getAttributeValueTyped(*existence_map->begin(), i));
+        values[i].ensureNotReference();
+      }
     }
-    value.ensureNotReference();
   }
 
   // Drop the result relation.
@@ -255,7 +259,31 @@ inline TypedValue executeQueryForSingleResult(
                      query_processor->getDefaultDatabase(),
                      query_processor->getStorageManager());
 
-  return value;
+  return values;
+}
+
+/**
+ * @brief A helper function that executes a SQL query to obtain a scalar result.
+ */
+inline TypedValue executeQueryForSingleResult(
+    const tmb::client_id main_thread_client_id,
+    const tmb::client_id foreman_client_id,
+    const std::string &query_string,
+    tmb::MessageBus *bus,
+    StorageManager *storage_manager,
+    QueryProcessor *query_processor,
+    SqlParserWrapper *parser_wrapper) {
+  std::vector<TypedValue> results =
+      executeQueryForSingleRow(
+          main_thread_client_id,
+          foreman_client_id,
+          query_string,
+          bus,
+          storage_manager,
+          query_processor,
+          parser_wrapper);
+  DCHECK_EQ(1u, results.size());
+  return results[0];
 }
 
 void executeAnalyze(const PtrVector<ParseString> *arguments,
@@ -292,25 +320,47 @@ 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_max_applicable =
+          (attr_type.getSuperTypeID() == Type::SuperTypeID::kNumeric);
+
       std::string query_string = "SELECT COUNT(DISTINCT ";
       query_string.append(attribute.getName());
-      query_string.append(") FROM ");
+      query_string.append(")");
+      if (is_min_max_applicable) {
+        query_string.append(", MIN(");
+        query_string.append(attribute.getName());
+        query_string.append("), MAX(");
+        query_string.append(attribute.getName());
+        query_string.append(")");
+      }
+      query_string.append(" FROM ");
       query_string.append(relation.getName());
       query_string.append(";");
 
-      TypedValue num_distinct_values =
-          executeQueryForSingleResult(main_thread_client_id,
-                                      foreman_client_id,
-                                      query_string,
-                                      bus,
-                                      storage_manager,
-                                      query_processor,
-                                      parser_wrapper.get());
-
-      DCHECK(num_distinct_values.getTypeID() == TypeID::kLong);
-      mutable_relation->getStatisticsMutable()->setNumDistinctValues(
-          attribute.getID(),
-          num_distinct_values.getLiteral<std::int64_t>());
+      std::vector<TypedValue> results =
+          executeQueryForSingleRow(main_thread_client_id,
+                                   foreman_client_id,
+                                   query_string,
+                                   bus,
+                                   storage_manager,
+                                   query_processor,
+                                   parser_wrapper.get());
+
+      auto *stat = mutable_relation->getStatisticsMutable();
+      const attribute_id attr_id = attribute.getID();
+
+      DCHECK(results[0].getTypeID() == TypeID::kLong);
+      stat->setNumDistinctValues(attr_id,
+                                 results[0].getLiteral<std::int64_t>());
+
+      if (is_min_max_applicable) {
+        DCHECK_GE(results.size(), 3u);
+        DCHECK(results[1].getTypeID() == attr_type.getTypeID());
+        DCHECK(results[2].getTypeID() == attr_type.getTypeID());
+        stat->setMinValue(attr_id, results[1]);
+        stat->setMaxValue(attr_id, results[2]);
+      }
     }
 
     // Get the number of tuples for the relation.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9b906657/cli/QuickstepCli.cpp
----------------------------------------------------------------------
diff --git a/cli/QuickstepCli.cpp b/cli/QuickstepCli.cpp
index e7ffc0c..bf0e2c7 100644
--- a/cli/QuickstepCli.cpp
+++ b/cli/QuickstepCli.cpp
@@ -347,10 +347,10 @@ int main(int argc, char* argv[]) {
 
 //  addPrimaryKeyInfoForTPCHTables(query_processor->getDefaultDatabase());
 //  addPrimaryKeyInfoForSSBTables(query_processor->getDefaultDatabase());
-//  std::string proto_str;
-//  google::protobuf::TextFormat::PrintToString(
-//      query_processor->getDefaultDatabase()->getProto(), &proto_str);
-//  std::cerr << proto_str << "\n";
+  std::string proto_str;
+  google::protobuf::TextFormat::PrintToString(
+      query_processor->getDefaultDatabase()->getProto(), &proto_str);
+  std::cerr << proto_str << "\n";
 //  query_processor->markCatalogAltered();
 //  query_processor->saveCatalog();