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/12/16 20:13:17 UTC

incubator-quickstep git commit: Add min/max statistics and support the exactness flag for CatalogRelationStatistics

Repository: incubator-quickstep
Updated Branches:
  refs/heads/min-max-stats f8feba19b -> c8ef1991e


Add min/max statistics and support the exactness flag for 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/c8ef1991
Tree: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/tree/c8ef1991
Diff: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/diff/c8ef1991

Branch: refs/heads/min-max-stats
Commit: c8ef1991e62ecffdd54b1347bb5acdda784fc44d
Parents: f8feba1
Author: Jianqiao Zhu <ji...@cs.wisc.edu>
Authored: Fri Dec 16 14:12:58 2016 -0600
Committer: Jianqiao Zhu <ji...@cs.wisc.edu>
Committed: Fri Dec 16 14:12:58 2016 -0600

----------------------------------------------------------------------
 catalog/CMakeLists.txt                      |  3 ++
 catalog/Catalog.proto                       |  5 ++--
 catalog/CatalogRelationStatistics.cpp       | 11 ++++++++
 catalog/CatalogRelationStatistics.hpp       | 25 +++++++++++++++-
 cli/CMakeLists.txt                          |  3 ++
 cli/CommandExecutor.cpp                     | 21 ++++++++------
 query_optimizer/ExecutionGenerator.cpp      | 36 ++++++++++++++++--------
 query_optimizer/cost_model/CMakeLists.txt   |  1 +
 relational_operators/CMakeLists.txt         |  1 +
 relational_operators/SaveBlocksOperator.cpp |  5 ++++
 relational_operators/SaveBlocksOperator.hpp |  7 +++++
 11 files changed, 95 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c8ef1991/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/c8ef1991/catalog/Catalog.proto
----------------------------------------------------------------------
diff --git a/catalog/Catalog.proto b/catalog/Catalog.proto
index 58de363..a54862f 100644
--- a/catalog/Catalog.proto
+++ b/catalog/Catalog.proto
@@ -81,7 +81,8 @@ message IndexScheme {
 }
 
 message CatalogRelationStatistics {
-  optional TypedValue num_tuples = 1;
+  optional bool is_exact = 1;
+  optional TypedValue num_tuples = 2;
 
   message ColumnStats {
     required int32 attr_id = 1;
@@ -89,7 +90,7 @@ message CatalogRelationStatistics {
     optional TypedValue min_value = 3;
     optional TypedValue max_value = 4;
   }
-  repeated ColumnStats column_stats = 2;
+  repeated ColumnStats column_stats = 3;
 }
 
 message CatalogRelationSchema {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c8ef1991/catalog/CatalogRelationStatistics.cpp
----------------------------------------------------------------------
diff --git a/catalog/CatalogRelationStatistics.cpp b/catalog/CatalogRelationStatistics.cpp
index 0f67234..f3f224d 100644
--- a/catalog/CatalogRelationStatistics.cpp
+++ b/catalog/CatalogRelationStatistics.cpp
@@ -28,6 +28,12 @@ const TypedValue CatalogRelationStatistics::kNullValue =
 
 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_ = TypedValue::ReconstructFromProto(proto.num_tuples());
   } else {
@@ -54,9 +60,13 @@ CatalogRelationStatistics::CatalogRelationStatistics(
 
 serialization::CatalogRelationStatistics CatalogRelationStatistics::getProto() const {
   serialization::CatalogRelationStatistics proto;
+
+  proto.set_is_exact(is_exact_);
+
   if (!num_tuples_.isNull()) {
     proto.mutable_num_tuples()->CopyFrom(num_tuples_.getProto());
   }
+
   for (const auto &pair : column_stats_) {
     auto entry = proto.add_column_stats();
     entry->set_attr_id(pair.first);
@@ -74,6 +84,7 @@ serialization::CatalogRelationStatistics CatalogRelationStatistics::getProto() c
           stat.max_value.getProto());
     }
   }
+
   return proto;
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c8ef1991/catalog/CatalogRelationStatistics.hpp
----------------------------------------------------------------------
diff --git a/catalog/CatalogRelationStatistics.hpp b/catalog/CatalogRelationStatistics.hpp
index ecb3110..df95231 100644
--- a/catalog/CatalogRelationStatistics.hpp
+++ b/catalog/CatalogRelationStatistics.hpp
@@ -47,7 +47,8 @@ class CatalogRelationStatistics {
    * @brief Constructor.
    **/
   CatalogRelationStatistics()
-      : num_tuples_(kNullValue) {}
+      : is_exact_(false),
+        num_tuples_(kNullValue) {}
 
   /**
    * @brief Reconstruct a CatalogRelationStatistics object from its serialized
@@ -67,6 +68,25 @@ class CatalogRelationStatistics {
   serialization::CatalogRelationStatistics getProto() const;
 
   /**
+   * @brief Check whether the statistics are exact for the relation.
+   *
+   * return True if the statistics are exact for the relation, false otherwise.
+   */
+  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.
@@ -215,6 +235,9 @@ class CatalogRelationStatistics {
 
   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.
   TypedValue num_tuples_;
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c8ef1991/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/c8ef1991/cli/CommandExecutor.cpp
----------------------------------------------------------------------
diff --git a/cli/CommandExecutor.cpp b/cli/CommandExecutor.cpp
index 3e1312a..377ca73 100644
--- a/cli/CommandExecutor.cpp
+++ b/cli/CommandExecutor.cpp
@@ -31,6 +31,7 @@
 #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"
@@ -323,6 +324,8 @@ void executeAnalyze(const PtrVector<ParseString> *arguments,
 
     CatalogRelation *mutable_relation =
         query_processor->getDefaultDatabase()->getRelationByIdMutable(relation.getID());
+    CatalogRelationStatistics *mutable_stat =
+        mutable_relation->getStatisticsMutable();
 
     // Get the number of distinct values for each column.
     for (const CatalogAttribute &attribute : relation) {
@@ -358,20 +361,19 @@ 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);
-      stat->setNumDistinctValues(attr_id,
-                                 results_it->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;
-        stat->setMinValue(attr_id, *results_it);
+        mutable_stat->setMinValue(attr_id, *results_it);
       }
       if (is_max_applicable) {
         ++results_it;
-        stat->setMaxValue(attr_id, *results_it);
+        mutable_stat->setMaxValue(attr_id, *results_it);
       }
     }
 
@@ -390,8 +392,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);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/c8ef1991/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/c8ef1991/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/c8ef1991/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/c8ef1991/relational_operators/SaveBlocksOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/SaveBlocksOperator.cpp b/relational_operators/SaveBlocksOperator.cpp
index 04d4211..ba27faa 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,10 @@ void SaveBlocksOperator::feedInputBlock(const block_id input_block_id, const rel
   destination_block_ids_.push_back(input_block_id);
 }
 
+void SaveBlocksOperator::updateCatalogOnCompletion() {
+  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/c8ef1991/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_;