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:22:54 UTC

[03/50] [abbrv] incubator-quickstep git commit: Added query ID to relational operators and Aggregation op.

Added query ID to relational operators and Aggregation op.


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

Branch: refs/heads/query-id-operator-workorder
Commit: e29e232f1dff27308881abe7b07f4c34b26a8170
Parents: abfc5f2
Author: Harshad Deshmukh <ha...@cs.wisc.edu>
Authored: Sun May 1 23:01:39 2016 -0500
Committer: Harshad Deshmukh <ha...@cs.wisc.edu>
Committed: Sun May 1 23:01:39 2016 -0500

----------------------------------------------------------------------
 query_optimizer/ExecutionGenerator.cpp                      | 3 ++-
 relational_operators/AggregationOperator.hpp                | 7 +++++--
 relational_operators/RelationalOperator.hpp                 | 7 +++++--
 relational_operators/tests/AggregationOperator_unittest.cpp | 4 ++--
 4 files changed, 14 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/e29e232f/query_optimizer/ExecutionGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.cpp b/query_optimizer/ExecutionGenerator.cpp
index c34f084..c66410a 100644
--- a/query_optimizer/ExecutionGenerator.cpp
+++ b/query_optimizer/ExecutionGenerator.cpp
@@ -1347,7 +1347,8 @@ void ExecutionGenerator::convertAggregate(
           new AggregationOperator(
               *input_relation_info->relation,
               input_relation_info->isStoredRelation(),
-              aggr_state_index));
+              aggr_state_index,
+              query_handle_->query_id()));
   if (!input_relation_info->isStoredRelation()) {
     execution_plan_->addDirectDependency(aggregation_operator_index,
                                          input_relation_info->producer_operator_index,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/e29e232f/relational_operators/AggregationOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/AggregationOperator.hpp b/relational_operators/AggregationOperator.hpp
index 0e74dfc..b9cffd8 100644
--- a/relational_operators/AggregationOperator.hpp
+++ b/relational_operators/AggregationOperator.hpp
@@ -58,11 +58,14 @@ class AggregationOperator : public RelationalOperator {
    *        is fully available to the operator before it can start generating
    *        workorders.
    * @param aggr_state_index The index of the AggregationState in QueryContext.
+   * @param query_id The ID of this query.
    **/
   AggregationOperator(const CatalogRelation &input_relation,
                       bool input_relation_is_stored,
-                      const QueryContext::aggregation_state_id aggr_state_index)
-      : input_relation_is_stored_(input_relation_is_stored),
+                      const QueryContext::aggregation_state_id aggr_state_index,
+                      const std::size_t query_id)
+      : RelationalOperator(query_id),
+        input_relation_is_stored_(input_relation_is_stored),
         input_relation_block_ids_(input_relation_is_stored ? input_relation.getBlocksSnapshot()
                                                            : std::vector<block_id>()),
         aggr_state_index_(aggr_state_index),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/e29e232f/relational_operators/RelationalOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/RelationalOperator.hpp b/relational_operators/RelationalOperator.hpp
index 75fde17..10aa4aa 100644
--- a/relational_operators/RelationalOperator.hpp
+++ b/relational_operators/RelationalOperator.hpp
@@ -211,13 +211,16 @@ class RelationalOperator {
    * @param blocking_dependencies_met If those dependencies which break the
    *        pipeline have been met.
    **/
-  explicit RelationalOperator(bool blocking_dependencies_met = false)
+  explicit RelationalOperator(const std::size_t query_id = 0,
+                              const bool blocking_dependencies_met = false)
       : blocking_dependencies_met_(blocking_dependencies_met),
-        done_feeding_input_relation_(false) {}
+        done_feeding_input_relation_(false),
+        query_id_(query_id) {}
 
   bool blocking_dependencies_met_;
   bool done_feeding_input_relation_;
   std::size_t op_index_;
+  const std::size_t query_id_;
 
  private:
   DISALLOW_COPY_AND_ASSIGN(RelationalOperator);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/e29e232f/relational_operators/tests/AggregationOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/AggregationOperator_unittest.cpp b/relational_operators/tests/AggregationOperator_unittest.cpp
index f2207c2..2c408df 100644
--- a/relational_operators/tests/AggregationOperator_unittest.cpp
+++ b/relational_operators/tests/AggregationOperator_unittest.cpp
@@ -270,7 +270,7 @@ class AggregationOperatorTest : public ::testing::Test {
     aggr_state_proto->set_estimated_num_entries(estimated_entries);
 
     // Create Operators.
-    op_.reset(new AggregationOperator(*table_, true, aggr_state_index));
+    op_.reset(new AggregationOperator(*table_, true, aggr_state_index, 0));
 
     // Setup the InsertDestination proto in the query context proto.
     const QueryContext::insert_destination_id insert_destination_index =
@@ -352,7 +352,7 @@ class AggregationOperatorTest : public ::testing::Test {
         serialization::HashTableImplType::LINEAR_OPEN_ADDRESSING);
 
     // Create Operators.
-    op_.reset(new AggregationOperator(*table_, true, aggr_state_index));
+    op_.reset(new AggregationOperator(*table_, true, aggr_state_index, 0));
 
     // Setup the InsertDestination proto in the query context proto.
     const QueryContext::insert_destination_id insert_destination_index =