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 2016/05/27 03:23:09 UTC

[18/50] [abbrv] incubator-quickstep git commit: Operator that returns the cardinality of a relation

Operator that returns the cardinality of a relation


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

Branch: refs/heads/quickstep_gen_stats
Commit: 4fb3856ef07581b576639d70d2a024f797f17167
Parents: 6a5a4c9
Author: rogersjeffreyl <rl...@cs.wisc.edu>
Authored: Tue May 17 15:40:31 2016 -0500
Committer: rogersjeffreyl <rl...@cs.wisc.edu>
Committed: Tue May 17 15:40:31 2016 -0500

----------------------------------------------------------------------
 catalog/Catalog.proto                  |  1 +
 catalog/CatalogRelation.cpp            |  4 ++++
 catalog/CatalogRelation.hpp            | 19 +++++++++++++++++++
 query_optimizer/CMakeLists.txt         |  1 +
 query_optimizer/ExecutionGenerator.cpp |  9 +++++++++
 relational_operators/CMakeLists.txt    | 11 +++++++++++
 6 files changed, 45 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4fb3856e/catalog/Catalog.proto
----------------------------------------------------------------------
diff --git a/catalog/Catalog.proto b/catalog/Catalog.proto
index 81e28cf..f975c96 100644
--- a/catalog/Catalog.proto
+++ b/catalog/Catalog.proto
@@ -99,6 +99,7 @@ message CatalogRelation {
     optional IndexScheme index_scheme = 18;
     optional PartitionScheme partition_scheme = 19;
     optional NUMAPlacementScheme placement_scheme = 20;
+    optional uint32 num_tuples = 21;
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4fb3856e/catalog/CatalogRelation.cpp
----------------------------------------------------------------------
diff --git a/catalog/CatalogRelation.cpp b/catalog/CatalogRelation.cpp
index 36f82d9..b6c5bb7 100644
--- a/catalog/CatalogRelation.cpp
+++ b/catalog/CatalogRelation.cpp
@@ -180,6 +180,10 @@ serialization::CatalogRelationSchema CatalogRelation::getProto() const {
   return proto;
 }
 
+void CatalogRelation::setNumTuples(std::size_t num_tuples) const {
+  num_tuples_ = num_tuples;
+}
+
 void CatalogRelation::setPartitionScheme(PartitionScheme* partition_scheme) {
   DCHECK_EQ(0u, size_blocks());
   partition_scheme_.reset(partition_scheme);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4fb3856e/catalog/CatalogRelation.hpp
----------------------------------------------------------------------
diff --git a/catalog/CatalogRelation.hpp b/catalog/CatalogRelation.hpp
index 312f3b4..87de204 100644
--- a/catalog/CatalogRelation.hpp
+++ b/catalog/CatalogRelation.hpp
@@ -80,6 +80,7 @@ class CatalogRelation : public CatalogRelationSchema {
                   bool temporary = false)
       : CatalogRelationSchema(parent, name, id, temporary),
         default_layout_(nullptr) {
+        num_tuples_ = 0;
   }
 
   /**
@@ -190,6 +191,22 @@ class CatalogRelation : public CatalogRelationSchema {
 #endif  // QUICKSTEP_HAVE_LIBNUMA
 
   /**
+   * @brief Get the number of tuples in the catalog relation.
+   *
+   * @return The number of tuples in the catalog relation
+   **/
+  const std::size_t getNumTuples() const {
+    return num_tuples_;
+  }
+
+  /**
+   * @brief Set the number of tuples for catalog relation.
+   *
+   * @param num_tuples The number of tuples in the catalog relation.
+   **/
+  void setNumTuples(std::size_t num_tuples) const;
+
+  /**
    * @brief Check if an index scheme is available for the relation.
    *
    * @return True if the relation has a index scheme, false otherwise.
@@ -397,6 +414,8 @@ class CatalogRelation : public CatalogRelationSchema {
   // Mutex for locking the index scheme.
   alignas(kCacheLineBytes) mutable SpinSharedMutex<false> index_scheme_mutex_;
 
+  mutable std::size_t num_tuples_;
+
 #ifdef QUICKSTEP_HAVE_LIBNUMA
   // NUMA placement scheme object which has the mapping between the partitions
   // of the relation and the NUMA nodes/sockets. It also maintains a mapping

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4fb3856e/query_optimizer/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/CMakeLists.txt b/query_optimizer/CMakeLists.txt
index feaecb3..747cc29 100644
--- a/query_optimizer/CMakeLists.txt
+++ b/query_optimizer/CMakeLists.txt
@@ -111,6 +111,7 @@ target_link_libraries(quickstep_queryoptimizer_ExecutionGenerator
                       quickstep_relationaloperators_DestroyHashOperator
                       quickstep_relationaloperators_DropTableOperator
                       quickstep_relationaloperators_FinalizeAggregationOperator
+                      quickstep_relationaloperators_GenerateNumRowsStatsOperator
                       quickstep_relationaloperators_HashJoinOperator
                       quickstep_relationaloperators_InsertOperator
                       quickstep_relationaloperators_NestedLoopsJoinOperator

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4fb3856e/query_optimizer/ExecutionGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.cpp b/query_optimizer/ExecutionGenerator.cpp
index 7209cfa..a312bff 100644
--- a/query_optimizer/ExecutionGenerator.cpp
+++ b/query_optimizer/ExecutionGenerator.cpp
@@ -91,6 +91,7 @@
 #include "relational_operators/DestroyHashOperator.hpp"
 #include "relational_operators/DropTableOperator.hpp"
 #include "relational_operators/FinalizeAggregationOperator.hpp"
+#include "relational_operators/GenerateNumRowsStatsOperator.hpp"
 #include "relational_operators/HashJoinOperator.hpp"
 #include "relational_operators/InsertOperator.hpp"
 #include "relational_operators/NestedLoopsJoinOperator.hpp"
@@ -947,6 +948,14 @@ void ExecutionGenerator::convertCopyFrom(
   execution_plan_->addDirectDependency(save_blocks_operator_index,
                                        scan_operator_index,
                                        false /* is_pipeline_breaker */);
+
+  const QueryPlan::DAGNodeIndex num_rows_operator_index =
+      execution_plan_->addRelationalOperator(
+          new GenerateNumRowsStatsOperator( output_relation));
+  insert_destination_proto->set_relational_op_index(num_rows_operator_index);
+  execution_plan_->addDirectDependency(num_rows_operator_index,
+                                       scan_operator_index,
+                                       true /* is_pipeline_breaker */);
 }
 
 void ExecutionGenerator::convertCreateIndex(

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/4fb3856e/relational_operators/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/relational_operators/CMakeLists.txt b/relational_operators/CMakeLists.txt
index eec5300..e211630 100644
--- a/relational_operators/CMakeLists.txt
+++ b/relational_operators/CMakeLists.txt
@@ -34,6 +34,9 @@ add_library(quickstep_relationaloperators_DropTableOperator DropTableOperator.cp
 add_library(quickstep_relationaloperators_FinalizeAggregationOperator
             FinalizeAggregationOperator.cpp
             FinalizeAggregationOperator.hpp)
+add_library(quickstep_relationaloperators_GenerateNumRowsStatsOperator
+            GenerateNumRowsStatsOperator.cpp
+            GenerateNumRowsStatsOperator.hpp)
 add_library(quickstep_relationaloperators_HashJoinOperator HashJoinOperator.cpp HashJoinOperator.hpp)
 add_library(quickstep_relationaloperators_InsertOperator InsertOperator.cpp InsertOperator.hpp)
 add_library(quickstep_relationaloperators_NestedLoopsJoinOperator
@@ -159,6 +162,13 @@ target_link_libraries(quickstep_relationaloperators_FinalizeAggregationOperator
                       quickstep_storage_AggregationOperationState
                       quickstep_utility_Macros
                       tmb)
+target_link_libraries(quickstep_relationaloperators_GenerateNumRowsStatsOperator
+                      glog
+                      quickstep_catalog_CatalogRelation
+                      quickstep_cli_PrintToScreen
+                      quickstep_relationaloperators_RelationalOperator
+                      quickstep_utility_Macros
+                      tmb)
 target_link_libraries(quickstep_relationaloperators_HashJoinOperator
                       gflags_nothreads-static
                       glog
@@ -446,6 +456,7 @@ target_link_libraries(quickstep_relationaloperators
                       quickstep_relationaloperators_DestroyHashOperator
                       quickstep_relationaloperators_DropTableOperator
                       quickstep_relationaloperators_FinalizeAggregationOperator
+                      quickstep_relationaloperators_GenerateNumRowsStatsOperator
                       quickstep_relationaloperators_HashJoinOperator
                       quickstep_relationaloperators_InsertOperator
                       quickstep_relationaloperators_NestedLoopsJoinOperator