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

[46/50] incubator-quickstep git commit: Add min/max statistics and the exactness flag into CatalogRelationStatistics.

Add min/max statistics and the exactness flag into CatalogRelationStatistics.


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

Branch: refs/heads/quickstep_partition_parser_support
Commit: 9fcb0aca86cf0e63b04e307534f65644877ee327
Parents: 1b0328b
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Thu Oct 27 14:16:32 2016 -0500
Committer: jianqiao <ji...@node-2.jianqiao.quickstep-pg0.wisc.cloudlab.us>
Committed: Tue Dec 20 09:19:50 2016 -0600

----------------------------------------------------------------------
 catalog/CMakeLists.txt                          |   3 +
 catalog/Catalog.proto                           |  11 +-
 catalog/CatalogRelationStatistics.cpp           |  60 +++++--
 catalog/CatalogRelationStatistics.hpp           | 174 ++++++++++++++++---
 cli/CMakeLists.txt                              |   3 +
 cli/CommandExecutor.cpp                         | 105 ++++++++---
 query_optimizer/ExecutionGenerator.cpp          |  36 ++--
 query_optimizer/cost_model/CMakeLists.txt       |   1 +
 query_optimizer/cost_model/SimpleCostModel.cpp  |  13 +-
 .../cost_model/StarSchemaSimpleCostModel.cpp    |  16 +-
 relational_operators/CMakeLists.txt             |   1 +
 relational_operators/SaveBlocksOperator.cpp     |  11 ++
 relational_operators/SaveBlocksOperator.hpp     |   7 +
 13 files changed, 353 insertions(+), 88 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9fcb0aca/catalog/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/catalog/CMakeLists.txt b/catalog/CMakeLists.txt
index dd4ef99..143d12f 100644
--- a/catalog/CMakeLists.txt
+++ b/catalog/CMakeLists.txt
@@ -120,6 +120,9 @@ target_link_libraries(quickstep_catalog_CatalogRelation
 target_link_libraries(quickstep_catalog_CatalogRelationStatistics
                       quickstep_catalog_CatalogTypedefs
                       quickstep_catalog_Catalog_proto
+                      quickstep_types_LongType
+                      quickstep_types_NullType
+                      quickstep_types_TypedValue
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_catalog_IndexScheme
                       glog

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9fcb0aca/catalog/Catalog.proto
----------------------------------------------------------------------
diff --git a/catalog/Catalog.proto b/catalog/Catalog.proto
index 90ce37e..a54862f 100644
--- a/catalog/Catalog.proto
+++ b/catalog/Catalog.proto
@@ -81,13 +81,16 @@ message IndexScheme {
 }
 
 message CatalogRelationStatistics {
-  optional fixed64 num_tuples = 1;
+  optional bool is_exact = 1;
+  optional TypedValue num_tuples = 2;
 
-  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 = 3;
 }
 
 message CatalogRelationSchema {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9fcb0aca/catalog/CatalogRelationStatistics.cpp
----------------------------------------------------------------------
diff --git a/catalog/CatalogRelationStatistics.cpp b/catalog/CatalogRelationStatistics.cpp
index 6a51570..f3f224d 100644
--- a/catalog/CatalogRelationStatistics.cpp
+++ b/catalog/CatalogRelationStatistics.cpp
@@ -23,28 +23,68 @@
 
 namespace quickstep {
 
+const TypedValue CatalogRelationStatistics::kNullValue =
+    NullType::InstanceNullable().makeNullValue();
+
 CatalogRelationStatistics::CatalogRelationStatistics(
     const serialization::CatalogRelationStatistics &proto) {
+  if (proto.has_is_exact()) {
+    is_exact_ = proto.is_exact();
+  } else {
+    is_exact_ = false;
+  }
+
   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_);
+
+  proto.set_is_exact(is_exact_);
+
+  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/9fcb0aca/catalog/CatalogRelationStatistics.hpp
----------------------------------------------------------------------
diff --git a/catalog/CatalogRelationStatistics.hpp b/catalog/CatalogRelationStatistics.hpp
index f2056f3..df95231 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,8 @@ class CatalogRelationStatistics {
    * @brief Constructor.
    **/
   CatalogRelationStatistics()
-      : num_tuples_(0) {}
+      : is_exact_(false),
+        num_tuples_(kNullValue) {}
 
   /**
    * @brief Reconstruct a CatalogRelationStatistics object from its serialized
@@ -64,12 +68,31 @@ class CatalogRelationStatistics {
   serialization::CatalogRelationStatistics getProto() const;
 
   /**
-   * @brief Set the number of tuples statistic.
+   * @brief Check whether the statistics are exact for the relation.
    *
-   * @param num_tuples The number of tuples statistic.
+   * return True if the statistics are exact for the relation, false otherwise.
    */
-  void setNumTuples(std::size_t num_tuples) {
-    num_tuples_ = num_tuples;
+  bool isExact() const {
+    return is_exact_;
+  }
+
+  /**
+   * @brief Set the boolean flag that indicates whether the statistics are exact
+   *        for the relation.
+   *
+   * @param Exactness of the statistics.
+   */
+  void setExactness(const bool is_exact) {
+    is_exact_ = is_exact;
+  }
+
+  /**
+   * @brief Check whether the number of tuples statistic is available.
+   *
+   * @return True if the number of tuples statistic is available, false otherwise.
+   */
+  bool hasNumTuples() const {
+    return !num_tuples_.isNull();
   }
 
   /**
@@ -78,41 +101,148 @@ 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 Set the number of distinct values statistic for a column (catalog attribute).
+   * @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.
    *
    * @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;
+
+  // Whether the statistics are exact (i.e. up-to-date) for the relation.
+  bool is_exact_;
+
   // 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/9fcb0aca/cli/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/cli/CMakeLists.txt b/cli/CMakeLists.txt
index be13c82..8c7fe2d 100644
--- a/cli/CMakeLists.txt
+++ b/cli/CMakeLists.txt
@@ -89,8 +89,11 @@ target_link_libraries(quickstep_cli_CommandExecutor
                       quickstep_catalog_CatalogDatabase
                       quickstep_catalog_CatalogRelation
                       quickstep_catalog_CatalogRelationSchema
+                      quickstep_catalog_CatalogRelationStatistics
                       quickstep_cli_DropRelation
                       quickstep_cli_PrintToScreen
+                      quickstep_expressions_aggregation_AggregateFunctionMax
+                      quickstep_expressions_aggregation_AggregateFunctionMin
                       quickstep_parser_ParseStatement
                       quickstep_parser_SqlParserWrapper
                       quickstep_queryoptimizer_QueryHandle

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9fcb0aca/cli/CommandExecutor.cpp
----------------------------------------------------------------------
diff --git a/cli/CommandExecutor.cpp b/cli/CommandExecutor.cpp
index 3c510e7..78bff98 100644
--- a/cli/CommandExecutor.cpp
+++ b/cli/CommandExecutor.cpp
@@ -31,8 +31,11 @@
 #include "catalog/CatalogDatabase.hpp"
 #include "catalog/CatalogRelation.hpp"
 #include "catalog/CatalogRelationSchema.hpp"
+#include "catalog/CatalogRelationStatistics.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"
@@ -69,7 +72,7 @@ namespace {
 
 namespace C = ::quickstep::cli;
 
-void executeDescribeDatabase(
+void ExecuteDescribeDatabase(
     const PtrVector<ParseString> *arguments,
     const CatalogDatabase &catalog_database,
     StorageManager *storage_manager,
@@ -131,7 +134,7 @@ void executeDescribeDatabase(
   }
 }
 
-void executeDescribeTable(
+void ExecuteDescribeTable(
     const PtrVector<ParseString> *arguments,
     const CatalogDatabase &catalog_database, FILE *out) {
   const ParseString &table_name = arguments->front();
@@ -199,7 +202,7 @@ void executeDescribeTable(
 /**
  * @brief A helper function that executes a SQL query to obtain a row of results.
  */
-inline std::vector<TypedValue> executeQueryForSingleRow(
+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,
@@ -270,7 +273,7 @@ inline std::vector<TypedValue> executeQueryForSingleRow(
 /**
  * @brief A helper function that executes a SQL query to obtain a scalar result.
  */
-inline TypedValue executeQueryForSingleResult(
+inline TypedValue ExecuteQueryForSingleResult(
     const tmb::client_id main_thread_client_id,
     const tmb::client_id foreman_client_id,
     const std::string &query_string,
@@ -279,7 +282,7 @@ inline TypedValue executeQueryForSingleResult(
     QueryProcessor *query_processor,
     SqlParserWrapper *parser_wrapper) {
   std::vector<TypedValue> results =
-      executeQueryForSingleRow(main_thread_client_id,
+      ExecuteQueryForSingleRow(main_thread_client_id,
                                foreman_client_id,
                                query_string,
                                bus,
@@ -290,7 +293,21 @@ inline TypedValue executeQueryForSingleResult(
   return results[0];
 }
 
-void executeAnalyze(const PtrVector<ParseString> *arguments,
+/**
+ * @brief A helper function for escaping quotes (i.e. ' or ").
+ */
+std::string EscapeQuotes(const std::string &str, const char quote) {
+  std::string ret;
+  for (const char c : str) {
+    ret.push_back(c);
+    if (c == quote) {
+      ret.push_back(c);
+    }
+  }
+  return ret;
+}
+
+void ExecuteAnalyze(const PtrVector<ParseString> *arguments,
                     const tmb::client_id main_thread_client_id,
                     const tmb::client_id foreman_client_id,
                     MessageBus *bus,
@@ -321,17 +338,42 @@ void executeAnalyze(const PtrVector<ParseString> *arguments,
 
     CatalogRelation *mutable_relation =
         query_processor->getDefaultDatabase()->getRelationByIdMutable(relation.getID());
+    CatalogRelationStatistics *mutable_stat =
+        mutable_relation->getStatisticsMutable();
+
+    const std::string rel_name = EscapeQuotes(relation.getName(), '"');
 
     // Get the number of distinct values for each column.
     for (const CatalogAttribute &attribute : relation) {
-      std::string query_string = "SELECT COUNT(DISTINCT ";
-      query_string.append(attribute.getName());
-      query_string.append(") FROM ");
-      query_string.append(relation.getName());
-      query_string.append(";");
+      const std::string attr_name = EscapeQuotes(attribute.getName(), '"');
+      const Type &attr_type = attribute.getType();
+      bool is_min_applicable =
+          AggregateFunctionMin::Instance().canApplyToTypes({&attr_type});
+      bool is_max_applicable =
+          AggregateFunctionMax::Instance().canApplyToTypes({&attr_type});
+
+      // NOTE(jianqiao): Note that the relation name and the attribute names may
+      // contain non-letter characters, e.g. CREATE TABLE "with space"("1" int).
+      // So here we need to format the names with double quotes (").
+      std::string query_string = "SELECT COUNT(DISTINCT \"";
+      query_string.append(attr_name);
+      query_string.append("\")");
+      if (is_min_applicable) {
+        query_string.append(", MIN(\"");
+        query_string.append(attr_name);
+        query_string.append("\")");
+      }
+      if (is_max_applicable) {
+        query_string.append(", MAX(\"");
+        query_string.append(attr_name);
+        query_string.append("\")");
+      }
+      query_string.append(" FROM \"");
+      query_string.append(rel_name);
+      query_string.append("\";");
 
       std::vector<TypedValue> results =
-          executeQueryForSingleRow(main_thread_client_id,
+          ExecuteQueryForSingleRow(main_thread_client_id,
                                    foreman_client_id,
                                    query_string,
                                    bus,
@@ -339,21 +381,29 @@ void executeAnalyze(const PtrVector<ParseString> *arguments,
                                    query_processor,
                                    parser_wrapper.get());
 
-      auto *stat = mutable_relation->getStatisticsMutable();
-      const attribute_id attr_id = attribute.getID();
+      auto results_it = results.begin();
+      DCHECK(results_it->getTypeID() == TypeID::kLong);
 
-      DCHECK(results[0].getTypeID() == TypeID::kLong);
-      stat->setNumDistinctValues(attr_id,
-                                 results[0].getLiteral<std::int64_t>());
+      const attribute_id attr_id = attribute.getID();
+      mutable_stat->setNumDistinctValues(attr_id,
+                                         results_it->getLiteral<std::int64_t>());
+      if (is_min_applicable) {
+        ++results_it;
+        mutable_stat->setMinValue(attr_id, *results_it);
+      }
+      if (is_max_applicable) {
+        ++results_it;
+        mutable_stat->setMaxValue(attr_id, *results_it);
+      }
     }
 
     // Get the number of tuples for the relation.
-    std::string query_string = "SELECT COUNT(*) FROM ";
-    query_string.append(relation.getName());
-    query_string.append(";");
+    std::string query_string = "SELECT COUNT(*) FROM \"";
+    query_string.append(rel_name);
+    query_string.append("\";");
 
     TypedValue num_tuples =
-        executeQueryForSingleResult(main_thread_client_id,
+        ExecuteQueryForSingleResult(main_thread_client_id,
                                     foreman_client_id,
                                     query_string,
                                     bus,
@@ -362,8 +412,9 @@ void executeAnalyze(const PtrVector<ParseString> *arguments,
                                     parser_wrapper.get());
 
     DCHECK(num_tuples.getTypeID() == TypeID::kLong);
-    mutable_relation->getStatisticsMutable()->setNumTuples(
-        num_tuples.getLiteral<std::int64_t>());
+    mutable_stat->setNumTuples(num_tuples.getLiteral<std::int64_t>());
+
+    mutable_stat->setExactness(true);
 
     fprintf(out, "done\n");
     fflush(out);
@@ -386,15 +437,15 @@ void executeCommand(const ParseStatement &statement,
   const PtrVector<ParseString> *arguments = command.arguments();
   const std::string &command_str = command.command()->value();
   if (command_str == C::kDescribeDatabaseCommand) {
-    executeDescribeDatabase(arguments, catalog_database, storage_manager, out);
+    ExecuteDescribeDatabase(arguments, catalog_database, storage_manager, out);
   } else if (command_str == C::kDescribeTableCommand) {
     if (arguments->size() == 0) {
-      executeDescribeDatabase(arguments, catalog_database, storage_manager, out);
+      ExecuteDescribeDatabase(arguments, catalog_database, storage_manager, out);
     } else {
-      executeDescribeTable(arguments, catalog_database, out);
+      ExecuteDescribeTable(arguments, catalog_database, out);
     }
   } else if (command_str == C::kAnalyzeCommand) {
-    executeAnalyze(arguments,
+    ExecuteAnalyze(arguments,
                    main_thread_client_id,
                    foreman_client_id,
                    bus,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9fcb0aca/query_optimizer/ExecutionGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.cpp b/query_optimizer/ExecutionGenerator.cpp
index 5a2c450..d24f498 100644
--- a/query_optimizer/ExecutionGenerator.cpp
+++ b/query_optimizer/ExecutionGenerator.cpp
@@ -924,9 +924,11 @@ void ExecutionGenerator::convertCopyFrom(
               insert_destination_index));
   insert_destination_proto->set_relational_op_index(scan_operator_index);
 
+  CatalogRelation *mutable_output_relation =
+      catalog_database_->getRelationByIdMutable(output_relation->getID());
   const QueryPlan::DAGNodeIndex save_blocks_operator_index =
       execution_plan_->addRelationalOperator(
-          new SaveBlocksOperator(query_handle_->query_id()));
+          new SaveBlocksOperator(query_handle_->query_id(), mutable_output_relation));
   execution_plan_->addDirectDependency(save_blocks_operator_index,
                                        scan_operator_index,
                                        false /* is_pipeline_breaker */);
@@ -1038,13 +1040,16 @@ void ExecutionGenerator::convertDeleteTuples(
   const CatalogRelationInfo *input_relation_info =
       findRelationInfoOutputByPhysical(physical_plan->input());
   DCHECK(input_relation_info != nullptr);
+
+  const CatalogRelation *input_relation = input_relation_info->relation;
+
   if (execution_predicate == nullptr ||
       (execution_predicate->hasStaticResult() &&
        execution_predicate->getStaticResult())) {
     const QueryPlan::DAGNodeIndex drop_table_index =
         execution_plan_->addRelationalOperator(
             new DropTableOperator(query_handle_->query_id(),
-                                  *input_relation_info->relation,
+                                  *input_relation,
                                   catalog_database_,
                                   true /* only_drop_blocks */));
     if (!input_relation_info->isStoredRelation()) {
@@ -1059,7 +1064,7 @@ void ExecutionGenerator::convertDeleteTuples(
     const QueryPlan::DAGNodeIndex delete_tuples_index =
         execution_plan_->addRelationalOperator(
             new DeleteOperator(query_handle_->query_id(),
-                               *input_relation_info->relation,
+                               *input_relation,
                                execution_predicate_index,
                                input_relation_info->isStoredRelation()));
 
@@ -1069,9 +1074,11 @@ void ExecutionGenerator::convertDeleteTuples(
                                            false /* is_pipeline_breaker */);
     }
 
+    CatalogRelation *mutable_relation =
+        catalog_database_->getRelationByIdMutable(input_relation->getID());
     const QueryPlan::DAGNodeIndex save_blocks_index =
         execution_plan_->addRelationalOperator(
-            new SaveBlocksOperator(query_handle_->query_id()));
+            new SaveBlocksOperator(query_handle_->query_id(), mutable_relation));
     execution_plan_->addDirectDependency(save_blocks_index,
                                          delete_tuples_index,
                                          false /* is_pipeline_breaker */);
@@ -1150,9 +1157,11 @@ void ExecutionGenerator::convertInsertTuple(
                              tuple_index));
   insert_destination_proto->set_relational_op_index(insert_operator_index);
 
+  CatalogRelation *mutable_relation =
+      catalog_database_->getRelationByIdMutable(input_relation.getID());
   const QueryPlan::DAGNodeIndex save_blocks_index =
       execution_plan_->addRelationalOperator(
-          new SaveBlocksOperator(query_handle_->query_id()));
+          new SaveBlocksOperator(query_handle_->query_id(), mutable_relation));
   if (!input_relation_info->isStoredRelation()) {
     execution_plan_->addDirectDependency(insert_operator_index,
                                          input_relation_info->producer_operator_index,
@@ -1201,6 +1210,7 @@ void ExecutionGenerator::convertInsertSelection(
 
   const CatalogRelationInfo *selection_relation_info =
       findRelationInfoOutputByPhysical(physical_plan->selection());
+  const CatalogRelation *selection_relation = selection_relation_info->relation;
 
   // Prepare the attributes, which are output columns of the selection relation.
   std::vector<attribute_id> attributes;
@@ -1221,7 +1231,7 @@ void ExecutionGenerator::convertInsertSelection(
   // physical plan by modifying class Physical.
   SelectOperator *insert_selection_op =
       new SelectOperator(query_handle_->query_id(),
-                         *selection_relation_info->relation,
+                         *selection_relation,
                          destination_relation,
                          insert_destination_index,
                          QueryContext::kInvalidPredicateId,
@@ -1232,8 +1242,11 @@ void ExecutionGenerator::convertInsertSelection(
       execution_plan_->addRelationalOperator(insert_selection_op);
   insert_destination_proto->set_relational_op_index(insert_selection_index);
 
+  CatalogRelation *mutable_relation =
+      catalog_database_->getRelationByIdMutable(selection_relation->getID());
   const QueryPlan::DAGNodeIndex save_blocks_index =
-      execution_plan_->addRelationalOperator(new SaveBlocksOperator(query_handle_->query_id()));
+      execution_plan_->addRelationalOperator(
+          new SaveBlocksOperator(query_handle_->query_id(), mutable_relation));
 
   if (!selection_relation_info->isStoredRelation()) {
     execution_plan_->addDirectDependency(insert_selection_index,
@@ -1253,7 +1266,8 @@ void ExecutionGenerator::convertUpdateTable(
       findRelationInfoOutputByPhysical(physical_plan->input());
   DCHECK(input_relation_info != nullptr);
 
-  const relation_id input_rel_id = input_relation_info->relation->getID();
+  const CatalogRelation *input_relation = input_relation_info->relation;
+  const relation_id input_rel_id = input_relation->getID();
 
   // Create InsertDestination proto.
   const QueryContext::insert_destination_id relocation_destination_index =
@@ -1303,8 +1317,7 @@ void ExecutionGenerator::convertUpdateTable(
   const QueryPlan::DAGNodeIndex update_operator_index =
       execution_plan_->addRelationalOperator(new UpdateOperator(
           query_handle_->query_id(),
-          *catalog_database_->getRelationById(
-              input_rel_id),
+          *input_relation,
           relocation_destination_index,
           execution_predicate_index,
           update_group_index));
@@ -1312,7 +1325,8 @@ void ExecutionGenerator::convertUpdateTable(
 
   const QueryPlan::DAGNodeIndex save_blocks_index =
       execution_plan_->addRelationalOperator(
-          new SaveBlocksOperator(query_handle_->query_id()));
+          new SaveBlocksOperator(query_handle_->query_id(),
+                                 catalog_database_->getRelationByIdMutable(input_rel_id)));
   if (!input_relation_info->isStoredRelation()) {
     execution_plan_->addDirectDependency(update_operator_index,
                                          input_relation_info->producer_operator_index,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9fcb0aca/query_optimizer/cost_model/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/CMakeLists.txt b/query_optimizer/cost_model/CMakeLists.txt
index 032e34c..90133e7 100644
--- a/query_optimizer/cost_model/CMakeLists.txt
+++ b/query_optimizer/cost_model/CMakeLists.txt
@@ -48,6 +48,7 @@ target_link_libraries(quickstep_queryoptimizer_costmodel_SimpleCostModel
 target_link_libraries(quickstep_queryoptimizer_costmodel_StarSchemaSimpleCostModel
                       glog
                       quickstep_catalog_CatalogRelation
+                      quickstep_catalog_CatalogRelationStatistics
                       quickstep_queryoptimizer_costmodel_CostModel
                       quickstep_queryoptimizer_expressions_AttributeReference
                       quickstep_queryoptimizer_expressions_ComparisonExpression

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9fcb0aca/query_optimizer/cost_model/SimpleCostModel.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/SimpleCostModel.cpp b/query_optimizer/cost_model/SimpleCostModel.cpp
index bf6da6a..7808898 100644
--- a/query_optimizer/cost_model/SimpleCostModel.cpp
+++ b/query_optimizer/cost_model/SimpleCostModel.cpp
@@ -94,13 +94,12 @@ std::size_t SimpleCostModel::estimateCardinalityForTopLevelPlan(
 
 std::size_t SimpleCostModel::estimateCardinalityForTableReference(
     const P::TableReferencePtr &physical_plan) {
-  const std::size_t num_tuples_in_relation =
-      physical_plan->relation()->getStatistics().getNumTuples();
-  if (num_tuples_in_relation == 0) {
-    return physical_plan->relation()->estimateTupleCardinality();
-  } else {
-    return num_tuples_in_relation;
-  }
+  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;
 }
 
 std::size_t SimpleCostModel::estimateCardinalityForSelection(

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9fcb0aca/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp b/query_optimizer/cost_model/StarSchemaSimpleCostModel.cpp
index dfafa7d..75b1b2b 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/cost_model/CostModel.hpp"
 #include "query_optimizer/expressions/AttributeReference.hpp"
 #include "query_optimizer/expressions/ComparisonExpression.hpp"
@@ -105,10 +106,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;
 }
 
@@ -385,9 +387,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;
     }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9fcb0aca/relational_operators/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/relational_operators/CMakeLists.txt b/relational_operators/CMakeLists.txt
index 0735bce..9e4b1b6 100644
--- a/relational_operators/CMakeLists.txt
+++ b/relational_operators/CMakeLists.txt
@@ -303,6 +303,7 @@ target_link_libraries(quickstep_relationaloperators_SampleOperator
                       tmb)
 target_link_libraries(quickstep_relationaloperators_SaveBlocksOperator
                       glog
+                      quickstep_catalog_CatalogRelation
                       quickstep_catalog_CatalogTypedefs
                       quickstep_queryexecution_WorkOrderProtosContainer
                       quickstep_queryexecution_WorkOrdersContainer

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9fcb0aca/relational_operators/SaveBlocksOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/SaveBlocksOperator.cpp b/relational_operators/SaveBlocksOperator.cpp
index 04d4211..a5a96c4 100644
--- a/relational_operators/SaveBlocksOperator.cpp
+++ b/relational_operators/SaveBlocksOperator.cpp
@@ -21,6 +21,7 @@
 
 #include <vector>
 
+#include "catalog/CatalogRelation.hpp"
 #include "query_execution/WorkOrderProtosContainer.hpp"
 #include "query_execution/WorkOrdersContainer.hpp"
 #include "relational_operators/WorkOrder.pb.h"
@@ -70,6 +71,16 @@ void SaveBlocksOperator::feedInputBlock(const block_id input_block_id, const rel
   destination_block_ids_.push_back(input_block_id);
 }
 
+void SaveBlocksOperator::updateCatalogOnCompletion() {
+  // Note(jianqiao): We need to reset the exactness flag whenever a stored
+  // relation gets changed. Given the pre-condition that all the data manipulation
+  // operations (insert, delete, update, copy) use this SaveBlocksOperator as a
+  // common routine. It is valid to put the updating call here to minimize the
+  // changes to other relational operators (e.g. InsertOperator, DeleteOperator,
+  // TextScanOperator, SelectOperator, etc.).
+  relation_->getStatisticsMutable()->setExactness(false);
+}
+
 void SaveBlocksWorkOrder::execute() {
   // It may happen that the block gets saved to disk as a result of an eviction,
   // before this invocation. In either case, we don't care about the return

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9fcb0aca/relational_operators/SaveBlocksOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SaveBlocksOperator.hpp b/relational_operators/SaveBlocksOperator.hpp
index e84b641..a8d5327 100644
--- a/relational_operators/SaveBlocksOperator.hpp
+++ b/relational_operators/SaveBlocksOperator.hpp
@@ -38,6 +38,7 @@ namespace tmb { class MessageBus; }
 
 namespace quickstep {
 
+class CatalogRelation;
 class QueryContext;
 class StorageManager;
 class WorkOrderProtosContainer;
@@ -56,13 +57,16 @@ class SaveBlocksOperator : public RelationalOperator {
    * @brief Constructor for saving only modified blocks in a relation.
    *
    * @param query_id The ID of the query to which this operator belongs.
+   * @param relation The relation whose blocks will be saved.
    * @param force If true, force writing of all blocks to disk, otherwise only
    *        write dirty blocks.
    **/
   explicit SaveBlocksOperator(const std::size_t query_id,
+                              CatalogRelation *relation,
                               const bool force = false)
       : RelationalOperator(query_id),
         force_(force),
+        relation_(relation),
         num_workorders_generated_(0) {}
 
   ~SaveBlocksOperator() override {}
@@ -89,9 +93,12 @@ class SaveBlocksOperator : public RelationalOperator {
     }
   }
 
+  void updateCatalogOnCompletion() override;
+
  private:
   const bool force_;
 
+  CatalogRelation *relation_;
   std::vector<block_id> destination_block_ids_;
 
   std::vector<block_id>::size_type num_workorders_generated_;