You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@quickstep.apache.org by hb...@apache.org on 2016/06/08 20:49:00 UTC

[01/16] incubator-quickstep git commit: Added query ID to relational operators and Aggregation op.

Repository: incubator-quickstep
Updated Branches:
  refs/heads/query-manager-used-in-foreman bef0ae1d8 -> e8ead8610


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-manager-used-in-foreman
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 =


[02/16] incubator-quickstep git commit: Added query ID to WorkOrder base class and Agg work order.

Posted by hb...@apache.org.
Added query ID to WorkOrder base class and Agg work order.


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

Branch: refs/heads/query-manager-used-in-foreman
Commit: 9a141fad18f6c9714635471a9aaab66a8b5debfc
Parents: e29e232
Author: Harshad Deshmukh <ha...@cs.wisc.edu>
Authored: Sun May 1 23:13:48 2016 -0500
Committer: Harshad Deshmukh <ha...@cs.wisc.edu>
Committed: Sun May 1 23:13:48 2016 -0500

----------------------------------------------------------------------
 relational_operators/AggregationOperator.cpp | 12 ++++++++----
 relational_operators/AggregationOperator.hpp |  4 +++-
 relational_operators/WorkOrder.hpp           |  4 +++-
 relational_operators/WorkOrderFactory.cpp    |  1 +
 4 files changed, 15 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9a141fad/relational_operators/AggregationOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/AggregationOperator.cpp b/relational_operators/AggregationOperator.cpp
index 1b935ee..94ba901 100644
--- a/relational_operators/AggregationOperator.cpp
+++ b/relational_operators/AggregationOperator.cpp
@@ -38,8 +38,10 @@ bool AggregationOperator::getAllWorkOrders(
     if (!started_) {
       for (const block_id input_block_id : input_relation_block_ids_) {
         container->addNormalWorkOrder(
-            new AggregationWorkOrder(input_block_id,
-                                     query_context->getAggregationState(aggr_state_index_)),
+            new AggregationWorkOrder(
+                input_block_id,
+                query_id_,
+                query_context->getAggregationState(aggr_state_index_)),
             op_index_);
       }
       started_ = true;
@@ -48,8 +50,10 @@ bool AggregationOperator::getAllWorkOrders(
   } else {
     while (num_workorders_generated_ < input_relation_block_ids_.size()) {
       container->addNormalWorkOrder(
-          new AggregationWorkOrder(input_relation_block_ids_[num_workorders_generated_],
-                                   query_context->getAggregationState(aggr_state_index_)),
+          new AggregationWorkOrder(
+              input_relation_block_ids_[num_workorders_generated_],
+              query_id_,
+              query_context->getAggregationState(aggr_state_index_)),
           op_index_);
       ++num_workorders_generated_;
     }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9a141fad/relational_operators/AggregationOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/AggregationOperator.hpp b/relational_operators/AggregationOperator.hpp
index b9cffd8..2b2a954 100644
--- a/relational_operators/AggregationOperator.hpp
+++ b/relational_operators/AggregationOperator.hpp
@@ -113,8 +113,10 @@ class AggregationWorkOrder : public WorkOrder {
    * @param state The AggregationState to use.
    **/
   AggregationWorkOrder(const block_id input_block_id,
+                       const std::size_t query_id,
                        AggregationOperationState *state)
-      : input_block_id_(input_block_id),
+      : WorkOrder(query_id),
+        input_block_id_(input_block_id),
         state_(DCHECK_NOTNULL(state)) {}
 
   ~AggregationWorkOrder() override {}

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9a141fad/relational_operators/WorkOrder.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrder.hpp b/relational_operators/WorkOrder.hpp
index 42cec2a..b179d8e 100644
--- a/relational_operators/WorkOrder.hpp
+++ b/relational_operators/WorkOrder.hpp
@@ -286,8 +286,10 @@ class WorkOrder {
   }
 
  protected:
-  WorkOrder() {}
+  WorkOrder(const std::size_t query_id = 0)
+      : query_id_(query_id) {}
 
+  const std::size_t query_id_;
   // A vector of preferred NUMA node IDs where this workorder should be executed.
   // These node IDs typically indicate the NUMA node IDs of the input(s) of the
   // workorder. Derived classes should ensure that there are no duplicate entries

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9a141fad/relational_operators/WorkOrderFactory.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrderFactory.cpp b/relational_operators/WorkOrderFactory.cpp
index 964c11c..d34d535 100644
--- a/relational_operators/WorkOrderFactory.cpp
+++ b/relational_operators/WorkOrderFactory.cpp
@@ -71,6 +71,7 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
       LOG(INFO) << "Creating AggregationWorkOrder";
       return new AggregationWorkOrder(
           proto.GetExtension(serialization::AggregationWorkOrder::block_id),
+          0,  // TODO(harshad) - Replace this with true query ID.
           query_context->getAggregationState(
               proto.GetExtension(serialization::AggregationWorkOrder::aggr_state_index)));
     }


[07/16] incubator-quickstep git commit: Added query ID to hash join operator.

Posted by hb...@apache.org.
Added query ID to hash join operator.


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

Branch: refs/heads/query-manager-used-in-foreman
Commit: 06d35016a65b394404e17ead321012ad55298246
Parents: 0712707
Author: Harshad Deshmukh <hb...@apache.org>
Authored: Sun Jun 5 21:01:17 2016 -0500
Committer: Harshad Deshmukh <hb...@apache.org>
Committed: Sun Jun 5 21:01:17 2016 -0500

----------------------------------------------------------------------
 query_optimizer/ExecutionGenerator.cpp          |  1 +
 .../tests/ExecutionHeuristics_unittest.cpp      |  6 ++--
 relational_operators/HashJoinOperator.cpp       |  4 +++
 relational_operators/HashJoinOperator.hpp       | 31 +++++++++++++++-----
 relational_operators/WorkOrderFactory.cpp       |  4 +++
 .../tests/HashJoinOperator_unittest.cpp         | 18 ++++++++----
 6 files changed, 49 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/06d35016/query_optimizer/ExecutionGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.cpp b/query_optimizer/ExecutionGenerator.cpp
index 54a81e3..f5e47bd 100644
--- a/query_optimizer/ExecutionGenerator.cpp
+++ b/query_optimizer/ExecutionGenerator.cpp
@@ -788,6 +788,7 @@ void ExecutionGenerator::convertHashJoin(const P::HashJoinPtr &physical_plan) {
               join_hash_table_index,
               residual_predicate_index,
               project_expressions_group_index,
+              query_handle_->query_id(),
               is_selection_on_build.get(),
               join_type));
   insert_destination_proto->set_relational_op_index(join_operator_index);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/06d35016/query_optimizer/tests/ExecutionHeuristics_unittest.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/ExecutionHeuristics_unittest.cpp b/query_optimizer/tests/ExecutionHeuristics_unittest.cpp
index 12acaff..a08a476 100644
--- a/query_optimizer/tests/ExecutionHeuristics_unittest.cpp
+++ b/query_optimizer/tests/ExecutionHeuristics_unittest.cpp
@@ -79,7 +79,8 @@ class ExecutionHeuristicsTest : public ::testing::Test {
                                                                 true,
                                                                 build_attribute_ids,
                                                                 false,
-                                                                join_hash_table_index));
+                                                                join_hash_table_index,
+                                                                0  /* dummy query ID */));
     return build_operator_index;
   }
 
@@ -100,7 +101,8 @@ class ExecutionHeuristicsTest : public ::testing::Test {
                                                                0,
                                                                join_hash_table_index,
                                                                0,
-                                                               0));
+                                                               0,
+                                                               0  /* dummy query ID */));
     return join_operator_index;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/06d35016/relational_operators/HashJoinOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/HashJoinOperator.cpp b/relational_operators/HashJoinOperator.cpp
index aa03794..d8c7304 100644
--- a/relational_operators/HashJoinOperator.cpp
+++ b/relational_operators/HashJoinOperator.cpp
@@ -298,6 +298,7 @@ bool HashJoinOperator::getAllNonOuterJoinWorkOrders(
                                      residual_predicate,
                                      selection,
                                      hash_table,
+                                     query_id_,
                                      output_destination,
                                      storage_manager),
               op_index_);
@@ -316,6 +317,7 @@ bool HashJoinOperator::getAllNonOuterJoinWorkOrders(
                                    residual_predicate,
                                    selection,
                                    hash_table,
+                                   query_id_,
                                    output_destination,
                                    storage_manager),
             op_index_);
@@ -356,6 +358,7 @@ bool HashJoinOperator::getAllOuterJoinWorkOrders(
                   selection,
                   is_selection_on_build_,
                   hash_table,
+                  query_id_,
                   output_destination,
                   storage_manager),
               op_index_);
@@ -375,6 +378,7 @@ bool HashJoinOperator::getAllOuterJoinWorkOrders(
                 selection,
                 is_selection_on_build_,
                 hash_table,
+                query_id_,
                 output_destination,
                 storage_manager),
             op_index_);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/06d35016/relational_operators/HashJoinOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/HashJoinOperator.hpp b/relational_operators/HashJoinOperator.hpp
index fcc087a..b0a26dc 100644
--- a/relational_operators/HashJoinOperator.hpp
+++ b/relational_operators/HashJoinOperator.hpp
@@ -108,6 +108,7 @@ class HashJoinOperator : public RelationalOperator {
    *        corresponding to the attributes of the relation referred by
    *        output_relation_id. Each Scalar is evaluated for the joined tuples,
    *        and the resulting value is inserted into the join result.
+   * @param query_id The ID of the query.
    * @param is_selection_on_build Whether each selection Scalar is using attributes
    *        from the build relation as input. Can be NULL for inner/semi/anti
    *        joins since this information is not utilized by these joins.
@@ -123,9 +124,11 @@ class HashJoinOperator : public RelationalOperator {
                    const QueryContext::join_hash_table_id hash_table_index,
                    const QueryContext::predicate_id residual_predicate_index,
                    const QueryContext::scalar_group_id selection_index,
+                   const std::size_t query_id,
                    const std::vector<bool> *is_selection_on_build = nullptr,
                    const JoinType join_type = JoinType::kInnerJoin)
-      : build_relation_(build_relation),
+      : RelationalOperator(query_id),
+        build_relation_(build_relation),
         probe_relation_(probe_relation),
         probe_relation_is_stored_(probe_relation_is_stored),
         join_key_attributes_(join_key_attributes),
@@ -254,9 +257,11 @@ class HashInnerJoinWorkOrder : public WorkOrder {
                          const Predicate *residual_predicate,
                          const std::vector<std::unique_ptr<const Scalar>> &selection,
                          const JoinHashTable &hash_table,
+                         const std::size_t query_id,
                          InsertDestination *output_destination,
                          StorageManager *storage_manager)
-      : build_relation_(build_relation),
+      : WorkOrder(query_id),
+        build_relation_(build_relation),
         probe_relation_(probe_relation),
         join_key_attributes_(join_key_attributes),
         any_join_key_attributes_nullable_(any_join_key_attributes_nullable),
@@ -297,9 +302,11 @@ class HashInnerJoinWorkOrder : public WorkOrder {
                          const Predicate *residual_predicate,
                          const std::vector<std::unique_ptr<const Scalar>> &selection,
                          const JoinHashTable &hash_table,
+                         const std::size_t query_id,
                          InsertDestination *output_destination,
                          StorageManager *storage_manager)
-      : build_relation_(build_relation),
+      : WorkOrder(query_id),
+        build_relation_(build_relation),
         probe_relation_(probe_relation),
         join_key_attributes_(std::move(join_key_attributes)),
         any_join_key_attributes_nullable_(any_join_key_attributes_nullable),
@@ -377,9 +384,11 @@ class HashSemiJoinWorkOrder : public WorkOrder {
                         const Predicate *residual_predicate,
                         const std::vector<std::unique_ptr<const Scalar>> &selection,
                         const JoinHashTable &hash_table,
+                        const std::size_t query_id,
                         InsertDestination *output_destination,
                         StorageManager *storage_manager)
-      : build_relation_(build_relation),
+      : WorkOrder(query_id),
+        build_relation_(build_relation),
         probe_relation_(probe_relation),
         join_key_attributes_(join_key_attributes),
         any_join_key_attributes_nullable_(any_join_key_attributes_nullable),
@@ -420,9 +429,11 @@ class HashSemiJoinWorkOrder : public WorkOrder {
                         const Predicate *residual_predicate,
                         const std::vector<std::unique_ptr<const Scalar>> &selection,
                         const JoinHashTable &hash_table,
+                        const std::size_t query_id,
                         InsertDestination *output_destination,
                         StorageManager *storage_manager)
-      : build_relation_(build_relation),
+      : WorkOrder(query_id),
+        build_relation_(build_relation),
         probe_relation_(probe_relation),
         join_key_attributes_(std::move(join_key_attributes)),
         any_join_key_attributes_nullable_(any_join_key_attributes_nullable),
@@ -493,9 +504,11 @@ class HashAntiJoinWorkOrder : public WorkOrder {
                         const Predicate *residual_predicate,
                         const std::vector<std::unique_ptr<const Scalar>> &selection,
                         const JoinHashTable &hash_table,
+                        const std::size_t query_id,
                         InsertDestination *output_destination,
                         StorageManager *storage_manager)
-      : build_relation_(build_relation),
+      : WorkOrder(query_id),
+        build_relation_(build_relation),
         probe_relation_(probe_relation),
         join_key_attributes_(join_key_attributes),
         any_join_key_attributes_nullable_(any_join_key_attributes_nullable),
@@ -536,9 +549,11 @@ class HashAntiJoinWorkOrder : public WorkOrder {
                         const Predicate *residual_predicate,
                         const std::vector<std::unique_ptr<const Scalar>> &selection,
                         const JoinHashTable &hash_table,
+                        const std::size_t query_id,
                         InsertDestination *output_destination,
                         StorageManager *storage_manager)
-      : build_relation_(build_relation),
+      : WorkOrder(query_id),
+        build_relation_(build_relation),
         probe_relation_(probe_relation),
         join_key_attributes_(std::move(join_key_attributes)),
         any_join_key_attributes_nullable_(any_join_key_attributes_nullable),
@@ -602,6 +617,7 @@ class HashOuterJoinWorkOrder : public WorkOrder {
    *        is using attributes from the build relation as input. Note that the
    *        length of this vector should equal the length of \p selection.
    * @param lookup_block_id The block id of the probe_relation.
+   * @param query_id The ID of the query.
    * @param output_destination The InsertDestination to insert the join results.
    * @param storage_manager The StorageManager to use.
    **/
@@ -613,6 +629,7 @@ class HashOuterJoinWorkOrder : public WorkOrder {
                          const std::vector<std::unique_ptr<const Scalar>> &selection,
                          const std::vector<bool> &is_selection_on_build,
                          const JoinHashTable &hash_table,
+                         const std::size_t query_id,
                          InsertDestination *output_destination,
                          StorageManager *storage_manager)
       : build_relation_(build_relation),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/06d35016/relational_operators/WorkOrderFactory.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrderFactory.cpp b/relational_operators/WorkOrderFactory.cpp
index 60e110c..b91fe2d 100644
--- a/relational_operators/WorkOrderFactory.cpp
+++ b/relational_operators/WorkOrderFactory.cpp
@@ -195,6 +195,7 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
               residual_predicate,
               selection,
               hash_table,
+              proto.query_id(),
               output_destination,
               storage_manager);
         }
@@ -209,6 +210,7 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
               residual_predicate,
               selection,
               hash_table,
+              proto.query_id(),
               output_destination,
               storage_manager);
         }
@@ -231,6 +233,7 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
               selection,
               move(is_selection_on_build),
               hash_table,
+              proto.query_id(),
               output_destination,
               storage_manager);
         }
@@ -245,6 +248,7 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
               residual_predicate,
               selection,
               hash_table,
+              proto.query_id(),
               output_destination,
               storage_manager);
         }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/06d35016/relational_operators/tests/HashJoinOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/HashJoinOperator_unittest.cpp b/relational_operators/tests/HashJoinOperator_unittest.cpp
index f1b18e7..dfec228 100644
--- a/relational_operators/tests/HashJoinOperator_unittest.cpp
+++ b/relational_operators/tests/HashJoinOperator_unittest.cpp
@@ -369,7 +369,8 @@ TEST_P(HashJoinOperatorTest, LongKeyHashJoinTest) {
                            output_destination_index,
                            join_hash_table_index,
                            QueryContext::kInvalidPredicateId /* residual_predicate_index */,
-                           selection_index));
+                           selection_index,
+                           0  /* dummy query ID */));
 
   // Set up the QueryContext.
   query_context_.reset(new QueryContext(query_context_proto,
@@ -516,7 +517,8 @@ TEST_P(HashJoinOperatorTest, IntDuplicateKeyHashJoinTest) {
                            output_destination_index,
                            join_hash_table_index,
                            QueryContext::kInvalidPredicateId /* residual_predicate_index */,
-                           selection_index));
+                           selection_index,
+                           0  /* dummy query ID */));
 
   // Set up the QueryContext.
   query_context_.reset(new QueryContext(query_context_proto,
@@ -671,7 +673,8 @@ TEST_P(HashJoinOperatorTest, CharKeyCartesianProductHashJoinTest) {
                            output_destination_index,
                            join_hash_table_index,
                            QueryContext::kInvalidPredicateId /* residual_predicate_index */,
-                           selection_index));
+                           selection_index,
+                           0  /* dummy query ID */));
 
   // Set up the QueryContext.
   query_context_.reset(new QueryContext(query_context_proto,
@@ -811,7 +814,8 @@ TEST_P(HashJoinOperatorTest, VarCharDuplicateKeyHashJoinTest) {
                            output_destination_index,
                            join_hash_table_index,
                            QueryContext::kInvalidPredicateId /* residual_predicate_index */,
-                           selection_index));
+                           selection_index,
+                           0  /* dummy query ID */));
 
   // Set up the QueryContext.
   query_context_.reset(new QueryContext(query_context_proto,
@@ -985,7 +989,8 @@ TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinTest) {
                            output_destination_index,
                            join_hash_table_index,
                            QueryContext::kInvalidPredicateId /* residual_predicate_index */,
-                           selection_index));
+                           selection_index,
+                           0  /* dummy query ID */));
 
   // Set up the QueryContext.
   query_context_.reset(new QueryContext(query_context_proto,
@@ -1170,7 +1175,8 @@ TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinWithResidualPredicateTest) {
                            output_destination_index,
                            join_hash_table_index,
                            residual_pred_index,
-                           selection_index));
+                           selection_index,
+                           0  /* dummy query ID */));
 
   // Set up the QueryContext.
   query_context_.reset(new QueryContext(query_context_proto,


[15/16] incubator-quickstep git commit: Reordered query ID in operators and work orders.

Posted by hb...@apache.org.
Reordered query ID in operators and work orders.

Reordered query ID in CreateIndex.

Reordered query ID in delete.

Reordered query ID in destroy hash.

Reordered query ID in Drop table.

Reordered query ID in Finalize aggregate

Reordered query ID in hash join

Reordered query ID in insert op

Reordered query ID in nested loops

Reordered query ID in rebuild

Reordered query ID in sample

Reordered query ID in save blocks

Reordered query ID in select

Reordered query ID in sort merge

Reordered query ID in sort run

Reordered query ID in table gen

Reorder query ID in text scan

Reordered query ID in update


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

Branch: refs/heads/query-manager-used-in-foreman
Commit: d67f61e1651cb68243d78728cc1ce6c68bb08fb1
Parents: 15a2114
Author: Harshad Deshmukh <hb...@apache.org>
Authored: Wed Jun 8 09:36:18 2016 -0500
Committer: Harshad Deshmukh <hb...@apache.org>
Committed: Wed Jun 8 11:38:06 2016 -0500

----------------------------------------------------------------------
 query_execution/Foreman.cpp                     |  12 +-
 query_execution/QueryManager.cpp                |  12 +-
 query_optimizer/ExecutionGenerator.cpp          | 189 +++++++--------
 .../tests/ExecutionHeuristics_unittest.cpp      |  29 +--
 relational_operators/BuildHashOperator.cpp      |   6 +-
 relational_operators/BuildHashOperator.hpp      |  20 +-
 relational_operators/CreateIndexOperator.hpp    |   6 +-
 relational_operators/DeleteOperator.cpp         |   8 +-
 relational_operators/DeleteOperator.hpp         |  14 +-
 relational_operators/DestroyHashOperator.cpp    |   2 +-
 relational_operators/DestroyHashOperator.hpp    |  12 +-
 relational_operators/DropTableOperator.hpp      |   6 +-
 .../FinalizeAggregationOperator.hpp             |  11 +-
 relational_operators/HashJoinOperator.cpp       |  31 +--
 relational_operators/HashJoinOperator.hpp       | 227 ++++++++++---------
 relational_operators/InsertOperator.hpp         |  11 +-
 .../NestedLoopsJoinOperator.cpp                 |  45 ++--
 .../NestedLoopsJoinOperator.hpp                 |  52 +++--
 relational_operators/RebuildWorkOrder.hpp       |  15 +-
 relational_operators/SampleOperator.cpp         |  51 +++--
 relational_operators/SampleOperator.hpp         |  26 ++-
 relational_operators/SaveBlocksOperator.cpp     |   2 +-
 relational_operators/SaveBlocksOperator.hpp     |   6 +-
 relational_operators/SelectOperator.cpp         |  27 ++-
 relational_operators/SelectOperator.hpp         |  56 ++---
 relational_operators/SortMergeRunOperator.cpp   |   2 +-
 relational_operators/SortMergeRunOperator.hpp   |  27 +--
 .../SortRunGenerationOperator.cpp               |   6 +-
 .../SortRunGenerationOperator.hpp               |  26 ++-
 relational_operators/TableGeneratorOperator.cpp |   2 +-
 relational_operators/TableGeneratorOperator.hpp |  20 +-
 relational_operators/TextScanOperator.cpp       |  21 +-
 relational_operators/TextScanOperator.hpp       |  32 +--
 relational_operators/UpdateOperator.cpp         |  22 +-
 relational_operators/UpdateOperator.hpp         |  37 +--
 relational_operators/WorkOrderFactory.cpp       |  38 ++--
 .../tests/AggregationOperator_unittest.cpp      |  18 +-
 .../tests/HashJoinOperator_unittest.cpp         | 180 ++++++++-------
 .../tests/SortMergeRunOperator_unittest.cpp     |  13 +-
 .../SortRunGenerationOperator_unittest.cpp      |  15 +-
 .../tests/TextScanOperator_unittest.cpp         |   6 +-
 41 files changed, 695 insertions(+), 646 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d67f61e1/query_execution/Foreman.cpp
----------------------------------------------------------------------
diff --git a/query_execution/Foreman.cpp b/query_execution/Foreman.cpp
index b358f70..7705819 100644
--- a/query_execution/Foreman.cpp
+++ b/query_execution/Foreman.cpp
@@ -521,12 +521,12 @@ void Foreman::getRebuildWorkOrders(const dag_node_index index, WorkOrdersContain
     // Note: The query ID used below is dummy for now, it will be replaced with
     // the true query ID when QueryManager gets used in Foreman.
     container->addRebuildWorkOrder(
-        new RebuildWorkOrder(move(partially_filled_block_refs[i]),
-                            index,
-                            op.getOutputRelationID(),
-                            foreman_client_id_,
-                            0,
-                            bus_),
+        new RebuildWorkOrder(0,
+                             move(partially_filled_block_refs[i]),
+                             index,
+                             op.getOutputRelationID(),
+                             foreman_client_id_,
+                             bus_),
         index);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d67f61e1/query_execution/QueryManager.cpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryManager.cpp b/query_execution/QueryManager.cpp
index 21f5820..e4e4c9d 100644
--- a/query_execution/QueryManager.cpp
+++ b/query_execution/QueryManager.cpp
@@ -457,12 +457,12 @@ void QueryManager::getRebuildWorkOrders(const dag_node_index index,
        i < partially_filled_block_refs.size();
        ++i) {
     container->addRebuildWorkOrder(
-        new RebuildWorkOrder(std::move(partially_filled_block_refs[i]),
-                            index,
-                            op.getOutputRelationID(),
-                            foreman_client_id_,
-                            query_id_,
-                            bus_),
+        new RebuildWorkOrder(query_id_,
+                             std::move(partially_filled_block_refs[i]),
+                             index,
+                             op.getOutputRelationID(),
+                             foreman_client_id_,
+                             bus_),
         index);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d67f61e1/query_optimizer/ExecutionGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.cpp b/query_optimizer/ExecutionGenerator.cpp
index 56625fd..99c2a21 100644
--- a/query_optimizer/ExecutionGenerator.cpp
+++ b/query_optimizer/ExecutionGenerator.cpp
@@ -193,8 +193,8 @@ void ExecutionGenerator::generatePlan(const P::PhysicalPtr &physical_plan) {
     }
     const QueryPlan::DAGNodeIndex drop_table_index =
         execution_plan_->addRelationalOperator(
-            new DropTableOperator(*temporary_relation,
-                                  query_handle_->query_id(),
+            new DropTableOperator(query_handle_->query_id(),
+                                  *temporary_relation,
                                   optimizer_context_->catalog_database(),
                                   false /* only_drop_blocks */));
     DCHECK(!temporary_relation_info.isStoredRelation());
@@ -411,13 +411,14 @@ void ExecutionGenerator::convertSample(const P::SamplePtr &physical_sample) {
       findRelationInfoOutputByPhysical(physical_sample->input());
   DCHECK(input_relation_info != nullptr);
 
-  SampleOperator *sample_op = new SampleOperator(*input_relation_info->relation,
-                                                 *output_relation,
-                                                 insert_destination_index,
-                                                 input_relation_info->isStoredRelation(),
-                                                 physical_sample->is_block_sample(),
-                                                 physical_sample->percentage(),
-                                                 query_handle_->query_id());
+  SampleOperator *sample_op =
+      new SampleOperator(query_handle_->query_id(),
+                         *input_relation_info->relation,
+                         *output_relation,
+                         insert_destination_index,
+                         input_relation_info->isStoredRelation(),
+                         physical_sample->is_block_sample(),
+                         physical_sample->percentage());
   const QueryPlan::DAGNodeIndex sample_index =
       execution_plan_->addRelationalOperator(sample_op);
   insert_destination_proto->set_relational_op_index(sample_index);
@@ -526,22 +527,22 @@ void ExecutionGenerator::convertSelection(
   // doesn't require any expression evaluation or intermediate copies) if
   // possible.
   std::vector<attribute_id> attributes;
-  SelectOperator *op
-      = convertSimpleProjection(project_expressions_group_index, &attributes)
-        ? new SelectOperator(*input_relation_info->relation,
-                             *output_relation,
-                             insert_destination_index,
-                             execution_predicate_index,
-                             move(attributes),
-                             input_relation_info->isStoredRelation(),
-                             query_handle_->query_id())
-        : new SelectOperator(*input_relation_info->relation,
-                             *output_relation,
-                             insert_destination_index,
-                             execution_predicate_index,
-                             project_expressions_group_index,
-                             input_relation_info->isStoredRelation(),
-                             query_handle_->query_id());
+  SelectOperator *op =
+      convertSimpleProjection(project_expressions_group_index, &attributes)
+          ? new SelectOperator(query_handle_->query_id(),
+                               *input_relation_info->relation,
+                               *output_relation,
+                               insert_destination_index,
+                               execution_predicate_index,
+                               move(attributes),
+                               input_relation_info->isStoredRelation())
+          : new SelectOperator(query_handle_->query_id(),
+                               *input_relation_info->relation,
+                               *output_relation,
+                               insert_destination_index,
+                               execution_predicate_index,
+                               project_expressions_group_index,
+                               input_relation_info->isStoredRelation());
 
   const QueryPlan::DAGNodeIndex select_index =
       execution_plan_->addRelationalOperator(op);
@@ -741,12 +742,12 @@ void ExecutionGenerator::convertHashJoin(const P::HashJoinPtr &physical_plan) {
   const QueryPlan::DAGNodeIndex build_operator_index =
       execution_plan_->addRelationalOperator(
           new BuildHashOperator(
+              query_handle_->query_id(),
               *build_relation_info->relation,
               build_relation_info->isStoredRelation(),
               build_attribute_ids,
               any_build_attributes_nullable,
-              join_hash_table_index,
-              query_handle_->query_id()));
+              join_hash_table_index));
 
   // Create InsertDestination proto.
   const CatalogRelation *output_relation = nullptr;
@@ -782,6 +783,7 @@ void ExecutionGenerator::convertHashJoin(const P::HashJoinPtr &physical_plan) {
   const QueryPlan::DAGNodeIndex join_operator_index =
       execution_plan_->addRelationalOperator(
           new HashJoinOperator(
+              query_handle_->query_id(),
               *build_relation_info->relation,
               *probe_operator_info->relation,
               probe_operator_info->isStoredRelation(),
@@ -792,14 +794,13 @@ void ExecutionGenerator::convertHashJoin(const P::HashJoinPtr &physical_plan) {
               join_hash_table_index,
               residual_predicate_index,
               project_expressions_group_index,
-              query_handle_->query_id(),
               is_selection_on_build.get(),
               join_type));
   insert_destination_proto->set_relational_op_index(join_operator_index);
 
   const QueryPlan::DAGNodeIndex destroy_operator_index =
       execution_plan_->addRelationalOperator(new DestroyHashOperator(
-          join_hash_table_index, query_handle_->query_id()));
+          query_handle_->query_id(), join_hash_table_index));
 
   if (!build_relation_info->isStoredRelation()) {
     execution_plan_->addDirectDependency(build_operator_index,
@@ -885,16 +886,16 @@ void ExecutionGenerator::convertNestedLoopsJoin(
 
   // Create and add a NestedLoopsJoin operator.
   const QueryPlan::DAGNodeIndex join_operator_index =
-      execution_plan_->addRelationalOperator(new NestedLoopsJoinOperator(
-          *left_relation_info->relation,
-          *right_relation_info->relation,
-          *output_relation,
-          insert_destination_index,
-          execution_join_predicate_index,
-          project_expressions_group_index,
-          left_relation_info->isStoredRelation(),
-          right_relation_info->isStoredRelation(),
-          query_handle_->query_id()));
+      execution_plan_->addRelationalOperator(
+          new NestedLoopsJoinOperator(query_handle_->query_id(),
+                                      *left_relation_info->relation,
+                                      *right_relation_info->relation,
+                                      *output_relation,
+                                      insert_destination_index,
+                                      execution_join_predicate_index,
+                                      project_expressions_group_index,
+                                      left_relation_info->isStoredRelation(),
+                                      right_relation_info->isStoredRelation()));
   insert_destination_proto->set_relational_op_index(join_operator_index);
 
   if (!left_relation_info->isStoredRelation()) {
@@ -940,13 +941,13 @@ void ExecutionGenerator::convertCopyFrom(
   const QueryPlan::DAGNodeIndex scan_operator_index =
       execution_plan_->addRelationalOperator(
           new TextScanOperator(
+              query_handle_->query_id(),
               physical_plan->file_name(),
               physical_plan->column_delimiter(),
               physical_plan->escape_strings(),
               FLAGS_parallelize_load,
               *output_relation,
-              insert_destination_index,
-              query_handle_->query_id()));
+              insert_destination_index));
   insert_destination_proto->set_relational_op_index(scan_operator_index);
 
   const QueryPlan::DAGNodeIndex save_blocks_operator_index =
@@ -997,10 +998,11 @@ void ExecutionGenerator::convertCreateIndex(
     // Check if the given index description is valid.
     THROW_SQL_ERROR() << "The index with given properties cannot be created.";
   }
-  execution_plan_->addRelationalOperator(new CreateIndexOperator(input_relation,
-                                                                 physical_plan->index_name(),
-                                                                 query_handle_->query_id(),
-                                                                 std::move(index_description)));
+  execution_plan_->addRelationalOperator(
+      new CreateIndexOperator(query_handle_->query_id(),
+                              input_relation,
+                              physical_plan->index_name(),
+                              std::move(index_description)));
 }
 
 void ExecutionGenerator::convertCreateTable(
@@ -1067,8 +1069,8 @@ void ExecutionGenerator::convertDeleteTuples(
        execution_predicate->getStaticResult())) {
     const QueryPlan::DAGNodeIndex drop_table_index =
         execution_plan_->addRelationalOperator(
-            new DropTableOperator(*input_relation_info->relation,
-                                  query_handle_->query_id(),
+            new DropTableOperator(query_handle_->query_id(),
+                                  *input_relation_info->relation,
                                   optimizer_context_->catalog_database(),
                                   true /* only_drop_blocks */));
     if (!input_relation_info->isStoredRelation()) {
@@ -1081,11 +1083,12 @@ void ExecutionGenerator::convertDeleteTuples(
     query_context_proto_->add_predicates()->CopyFrom(execution_predicate->getProto());
 
     const QueryPlan::DAGNodeIndex delete_tuples_index =
-        execution_plan_->addRelationalOperator(new DeleteOperator(
-            *input_relation_info->relation,
-            execution_predicate_index,
-            input_relation_info->isStoredRelation(),
-            query_handle_->query_id()));
+        execution_plan_->addRelationalOperator(
+            new DeleteOperator(query_handle_->query_id(),
+                               *input_relation_info->relation,
+                               execution_predicate_index,
+                               input_relation_info->isStoredRelation()));
+
     if (!input_relation_info->isStoredRelation()) {
       execution_plan_->addDirectDependency(delete_tuples_index,
                                            input_relation_info->producer_operator_index,
@@ -1111,8 +1114,8 @@ void ExecutionGenerator::convertDropTable(
 #endif
 
   execution_plan_->addRelationalOperator(
-      new DropTableOperator(catalog_relation,
-                            query_handle_->query_id(),
+      new DropTableOperator(query_handle_->query_id(),
+                            catalog_relation,
                             optimizer_context_->catalog_database()));
 }
 
@@ -1164,10 +1167,10 @@ void ExecutionGenerator::convertInsertTuple(
 
   const QueryPlan::DAGNodeIndex insert_operator_index =
       execution_plan_->addRelationalOperator(
-          new InsertOperator(input_relation,
+          new InsertOperator(query_handle_->query_id(),
+                             input_relation,
                              insert_destination_index,
-                             tuple_index,
-                             query_handle_->query_id()));
+                             tuple_index));
   insert_destination_proto->set_relational_op_index(insert_operator_index);
 
   const QueryPlan::DAGNodeIndex save_blocks_index =
@@ -1240,13 +1243,13 @@ void ExecutionGenerator::convertInsertSelection(
   // optimization is to enable specifying a specific output relation for each
   // physical plan by modifying class Physical.
   SelectOperator *insert_selection_op =
-      new SelectOperator(*selection_relation_info->relation,
+      new SelectOperator(query_handle_->query_id(),
+                         *selection_relation_info->relation,
                          destination_relation,
                          insert_destination_index,
                          QueryContext::kInvalidPredicateId,
                          move(attributes),
-                         selection_relation_info->isStoredRelation(),
-                         query_handle_->query_id());
+                         selection_relation_info->isStoredRelation());
 
   const QueryPlan::DAGNodeIndex insert_selection_index =
       execution_plan_->addRelationalOperator(insert_selection_op);
@@ -1321,13 +1324,13 @@ void ExecutionGenerator::convertUpdateTable(
   }
 
   const QueryPlan::DAGNodeIndex update_operator_index =
-      execution_plan_->addRelationalOperator(
-          new UpdateOperator(
-              *optimizer_context_->catalog_database()->getRelationById(input_rel_id),
-              relocation_destination_index,
-              execution_predicate_index,
-              update_group_index,
-              query_handle_->query_id()));
+      execution_plan_->addRelationalOperator(new UpdateOperator(
+          query_handle_->query_id(),
+          *optimizer_context_->catalog_database()->getRelationById(
+              input_rel_id),
+          relocation_destination_index,
+          execution_predicate_index,
+          update_group_index));
   relocation_destination_proto->set_relational_op_index(update_operator_index);
 
   const QueryPlan::DAGNodeIndex save_blocks_index =
@@ -1451,10 +1454,11 @@ void ExecutionGenerator::convertAggregate(
 
   const QueryPlan::DAGNodeIndex finalize_aggregation_operator_index =
       execution_plan_->addRelationalOperator(
-          new FinalizeAggregationOperator(aggr_state_index,
+          new FinalizeAggregationOperator(query_handle_->query_id(),
+                                          aggr_state_index,
                                           *output_relation,
-                                          insert_destination_index,
-                                          query_handle_->query_id()));
+                                          insert_destination_index));
+
   insert_destination_proto->set_relational_op_index(finalize_aggregation_operator_index);
 
   execution_plan_->addDirectDependency(finalize_aggregation_operator_index,
@@ -1500,13 +1504,13 @@ void ExecutionGenerator::convertSort(const P::SortPtr &physical_sort) {
   const CatalogRelationInfo *input_relation_info =
       findRelationInfoOutputByPhysical(physical_sort->input());
   const QueryPlan::DAGNodeIndex run_generator_index =
-      execution_plan_->addRelationalOperator(
-          new SortRunGenerationOperator(*input_relation_info->relation,
-                                        *initial_runs_relation,
-                                        initial_runs_destination_id,
-                                        sort_run_gen_config_id,
-                                        input_relation_info->isStoredRelation(),
-                                        query_handle_->query_id()));
+      execution_plan_->addRelationalOperator(new SortRunGenerationOperator(
+          query_handle_->query_id(),
+          *input_relation_info->relation,
+          *initial_runs_relation,
+          initial_runs_destination_id,
+          sort_run_gen_config_id,
+          input_relation_info->isStoredRelation()));
   if (!input_relation_info->isStoredRelation()) {
     execution_plan_->addDirectDependency(run_generator_index,
                                          input_relation_info->producer_operator_index,
@@ -1554,17 +1558,18 @@ void ExecutionGenerator::convertSort(const P::SortPtr &physical_sort) {
 
   // TODO(qzeng): Make the merge factor configurable.
   const QueryPlan::DAGNodeIndex merge_run_operator_index =
-      execution_plan_->addRelationalOperator(
-          new SortMergeRunOperator(*initial_runs_relation,
-                                   *sorted_relation,
-                                   sorted_output_destination_id,
-                                   *merged_runs_relation,
-                                   merged_runs_destination_id,
-                                   sort_merge_run_config_id,
-                                   64 /* merge_factor */,
-                                   physical_sort->limit(),
-                                   false /* input_relation_is_stored */,
-                                   query_handle_->query_id()));
+      execution_plan_->addRelationalOperator(new SortMergeRunOperator(
+          query_handle_->query_id(),
+          *initial_runs_relation,
+          *sorted_relation,
+          sorted_output_destination_id,
+          *merged_runs_relation,
+          merged_runs_destination_id,
+          sort_merge_run_config_id,
+          64 /* merge_factor */,
+          physical_sort->limit(),
+          false /* input_relation_is_stored */));
+
   execution_plan_->addDirectDependency(merge_run_operator_index,
                                        run_generator_index,
                                        false /* is_pipeline_breaker */);
@@ -1577,8 +1582,8 @@ void ExecutionGenerator::convertSort(const P::SortPtr &physical_sort) {
   const QueryPlan::DAGNodeIndex drop_merged_runs_index =
       execution_plan_->addRelationalOperator(
           new DropTableOperator(
-              *merged_runs_relation,
               query_handle_->query_id(),
+              *merged_runs_relation,
               optimizer_context_->catalog_database(),
               false /* only_drop_blocks */));
   execution_plan_->addDirectDependency(
@@ -1614,10 +1619,10 @@ void ExecutionGenerator::convertTableGenerator(
       physical_tablegen->generator_function_handle()->getProto());
 
   TableGeneratorOperator *op =
-      new TableGeneratorOperator(*output_relation,
+      new TableGeneratorOperator(query_handle_->query_id(),
+                                 *output_relation,
                                  insert_destination_index,
-                                 generator_function_index,
-                                 query_handle_->query_id());
+                                 generator_function_index);
 
   const QueryPlan::DAGNodeIndex tablegen_index =
       execution_plan_->addRelationalOperator(op);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d67f61e1/query_optimizer/tests/ExecutionHeuristics_unittest.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/ExecutionHeuristics_unittest.cpp b/query_optimizer/tests/ExecutionHeuristics_unittest.cpp
index a08a476..39130b8 100644
--- a/query_optimizer/tests/ExecutionHeuristics_unittest.cpp
+++ b/query_optimizer/tests/ExecutionHeuristics_unittest.cpp
@@ -75,12 +75,12 @@ class ExecutionHeuristicsTest : public ::testing::Test {
     std::vector<attribute_id> build_attribute_ids;
     build_attribute_ids.push_back(build_attribute_id);
     QueryPlan::DAGNodeIndex build_operator_index =
-        query_plan->addRelationalOperator(new BuildHashOperator(*build_relation,
+        query_plan->addRelationalOperator(new BuildHashOperator(0,  /* dummy query ID */
+                                                                *build_relation,
                                                                 true,
                                                                 build_attribute_ids,
                                                                 false,
-                                                                join_hash_table_index,
-                                                                0  /* dummy query ID */));
+                                                                join_hash_table_index));
     return build_operator_index;
   }
 
@@ -92,17 +92,18 @@ class ExecutionHeuristicsTest : public ::testing::Test {
     std::vector<attribute_id> probe_attribute_ids;
     probe_attribute_ids.push_back(probe_attribute_id);
     QueryPlan::DAGNodeIndex join_operator_index =
-        query_plan->addRelationalOperator(new HashJoinOperator(*build_relation,
-                                                               *probe_relation,
-                                                               true,
-                                                               probe_attribute_ids,
-                                                               false,
-                                                               *probe_relation,
-                                                               0,
-                                                               join_hash_table_index,
-                                                               0,
-                                                               0,
-                                                               0  /* dummy query ID */));
+        query_plan->addRelationalOperator(
+            new HashJoinOperator(0 /* dummy query ID */,
+                                 *build_relation,
+                                 *probe_relation,
+                                 true,
+                                 probe_attribute_ids,
+                                 false,
+                                 *probe_relation,
+                                 0,
+                                 join_hash_table_index,
+                                 0,
+                                 0));
     return join_operator_index;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d67f61e1/relational_operators/BuildHashOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/BuildHashOperator.cpp b/relational_operators/BuildHashOperator.cpp
index c6f6f96..9dc4afe 100644
--- a/relational_operators/BuildHashOperator.cpp
+++ b/relational_operators/BuildHashOperator.cpp
@@ -69,11 +69,11 @@ bool BuildHashOperator::getAllWorkOrders(
     if (!started_) {
       for (const block_id input_block_id : input_relation_block_ids_) {
         container->addNormalWorkOrder(
-            new BuildHashWorkOrder(input_relation_,
+            new BuildHashWorkOrder(query_id_,
+                                   input_relation_,
                                    join_key_attributes_,
                                    any_join_key_attributes_nullable_,
                                    input_block_id,
-                                   query_id_,
                                    hash_table,
                                    storage_manager),
             op_index_);
@@ -85,11 +85,11 @@ bool BuildHashOperator::getAllWorkOrders(
     while (num_workorders_generated_ < input_relation_block_ids_.size()) {
       container->addNormalWorkOrder(
           new BuildHashWorkOrder(
+              query_id_,
               input_relation_,
               join_key_attributes_,
               any_join_key_attributes_nullable_,
               input_relation_block_ids_[num_workorders_generated_],
-              query_id_,
               hash_table,
               storage_manager),
           op_index_);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d67f61e1/relational_operators/BuildHashOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/BuildHashOperator.hpp b/relational_operators/BuildHashOperator.hpp
index 5a46d8b..50dd7d6 100644
--- a/relational_operators/BuildHashOperator.hpp
+++ b/relational_operators/BuildHashOperator.hpp
@@ -59,6 +59,7 @@ class BuildHashOperator : public RelationalOperator {
   /**
    * @brief Constructor.
    *
+   * @param query_id The ID of the query to which this operator belongs.
    * @param input_relation The relation to build hash table on.
    * @param input_relation_is_stored If input_relation is a stored relation and
    *        is fully available to the operator before it can start generating
@@ -69,14 +70,13 @@ class BuildHashOperator : public RelationalOperator {
    * @param hash_table_index The index of the JoinHashTable in QueryContext.
    *        The HashTable's key Type(s) should be the Type(s) of the
    *        join_key_attributes in input_relation.
-   * @param query_id The ID of the query to which this operator belongs.
    **/
-  BuildHashOperator(const CatalogRelation &input_relation,
+  BuildHashOperator(const std::size_t query_id,
+                    const CatalogRelation &input_relation,
                     const bool input_relation_is_stored,
                     const std::vector<attribute_id> &join_key_attributes,
                     const bool any_join_key_attributes_nullable,
-                    const QueryContext::join_hash_table_id hash_table_index,
-                    const std::size_t query_id)
+                    const QueryContext::join_hash_table_id hash_table_index)
     : RelationalOperator(query_id),
       input_relation_(input_relation),
       input_relation_is_stored_(input_relation_is_stored),
@@ -131,20 +131,20 @@ class BuildHashWorkOrder : public WorkOrder {
   /**
    * @brief Constructor.
    *
+   * @param query_id The ID of the query.
    * @param input_relation The relation to build hash table on.
    * @param join_key_attributes The IDs of equijoin attributes in
    *        input_relation.
    * @param any_join_key_attributes_nullable If any attribute is nullable.
    * @param build_block_id The block id.
-   * @param query_id The ID of the query.
    * @param hash_table The JoinHashTable to use.
    * @param storage_manager The StorageManager to use.
    **/
-  BuildHashWorkOrder(const CatalogRelationSchema &input_relation,
+  BuildHashWorkOrder(const std::size_t query_id,
+                     const CatalogRelationSchema &input_relation,
                      const std::vector<attribute_id> &join_key_attributes,
                      const bool any_join_key_attributes_nullable,
                      const block_id build_block_id,
-                     const std::size_t query_id,
                      JoinHashTable *hash_table,
                      StorageManager *storage_manager)
       : WorkOrder(query_id),
@@ -158,20 +158,20 @@ class BuildHashWorkOrder : public WorkOrder {
   /**
    * @brief Constructor for the distributed version.
    *
+   * @param query_id The ID of the query.
    * @param input_relation The relation to build hash table on.
    * @param join_key_attributes The IDs of equijoin attributes in
    *        input_relation.
    * @param any_join_key_attributes_nullable If any attribute is nullable.
-   * @param query_id The ID of the query.
    * @param build_block_id The block id.
    * @param hash_table The JoinHashTable to use.
    * @param storage_manager The StorageManager to use.
    **/
-  BuildHashWorkOrder(const CatalogRelationSchema &input_relation,
+  BuildHashWorkOrder(const std::size_t query_id,
+                     const CatalogRelationSchema &input_relation,
                      std::vector<attribute_id> &&join_key_attributes,
                      const bool any_join_key_attributes_nullable,
                      const block_id build_block_id,
-                     const std::size_t query_id,
                      JoinHashTable *hash_table,
                      StorageManager *storage_manager)
       : WorkOrder(query_id),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d67f61e1/relational_operators/CreateIndexOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/CreateIndexOperator.hpp b/relational_operators/CreateIndexOperator.hpp
index ede3f02..11a01ae 100644
--- a/relational_operators/CreateIndexOperator.hpp
+++ b/relational_operators/CreateIndexOperator.hpp
@@ -51,14 +51,14 @@ class CreateIndexOperator : public RelationalOperator {
   /**
    * @brief Constructor.
    *
+   * @param query_id The ID of the query to which this operator belongs.
    * @param relation The relation to create index upon.
    * @param index_name The index to create.
-   * @param query_id The ID of the query to which this operator belongs.
    * @param index_description The index_description associated with this index.
    **/
-  CreateIndexOperator(CatalogRelation *relation,
+  CreateIndexOperator(const std::size_t query_id,
+                      CatalogRelation *relation,
                       const std::string &index_name,
-                      const std::size_t query_id,
                       IndexSubBlockDescription &&index_description)  // NOLINT(whitespace/operators)
       : RelationalOperator(query_id),
         relation_(DCHECK_NOTNULL(relation)),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d67f61e1/relational_operators/DeleteOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/DeleteOperator.cpp b/relational_operators/DeleteOperator.cpp
index 94169ed..15dc9e3 100644
--- a/relational_operators/DeleteOperator.cpp
+++ b/relational_operators/DeleteOperator.cpp
@@ -54,13 +54,13 @@ bool DeleteOperator::getAllWorkOrders(
     if (!started_) {
       for (const block_id input_block_id : relation_block_ids_) {
         container->addNormalWorkOrder(
-            new DeleteWorkOrder(relation_,
+            new DeleteWorkOrder(query_id_,
+                                relation_,
                                 input_block_id,
                                 predicate,
                                 storage_manager,
                                 op_index_,
                                 scheduler_client_id,
-                                query_id_,
                                 bus),
             op_index_);
       }
@@ -70,13 +70,13 @@ bool DeleteOperator::getAllWorkOrders(
   } else {
     while (num_workorders_generated_ < relation_block_ids_.size()) {
       container->addNormalWorkOrder(
-          new DeleteWorkOrder(relation_,
+          new DeleteWorkOrder(query_id_,
+                              relation_,
                               relation_block_ids_[num_workorders_generated_],
                               predicate,
                               storage_manager,
                               op_index_,
                               scheduler_client_id,
-                              query_id_,
                               bus),
           op_index_);
       ++num_workorders_generated_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d67f61e1/relational_operators/DeleteOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/DeleteOperator.hpp b/relational_operators/DeleteOperator.hpp
index ba1f825..c55f585 100644
--- a/relational_operators/DeleteOperator.hpp
+++ b/relational_operators/DeleteOperator.hpp
@@ -55,18 +55,18 @@ class DeleteOperator : public RelationalOperator {
   /**
    * @brief Constructor.
    *
+   * @param query_id The ID of the query to which this operator belongs.
    * @param relation The relation to perform the DELETE over.
    * @param predicate_index The index of Predicate in QueryContext. All tuples
    *        matching pred will be deleted (If kInvalidPredicateId, then all
    *        tuples will be deleted).
    * @param relation_is_stored If relation is a stored relation and is fully
    *        available to the operator before it can start generating workorders.
-   * @param query_id The ID of the query to which this operator belongs.
    **/
-  DeleteOperator(const CatalogRelation &relation,
+  DeleteOperator(const std::size_t query_id,
+                 const CatalogRelation &relation,
                  const QueryContext::predicate_id predicate_index,
-                 const bool relation_is_stored,
-                 const std::size_t query_id)
+                 const bool relation_is_stored)
      :  RelationalOperator(query_id),
         relation_(relation),
         predicate_index_(predicate_index),
@@ -122,6 +122,7 @@ class DeleteWorkOrder : public WorkOrder {
   /**
    * @brief Constructor.
    *
+   * @param query_id The ID of the query to which this workorder belongs.
    * @param input_relation The relation to perform the DELETE over.
    * @param input_block_id The block Id.
    * @param predicate All tuples matching \c predicate will be deleted (If
@@ -130,16 +131,15 @@ class DeleteWorkOrder : public WorkOrder {
    * @param delete_operator_index The index of the Delete Operator in the query
    *        plan DAG.
    * @param scheduler_client_id The TMB client ID of the scheduler thread.
-   * @param query_id The ID of the query to which this workorder belongs.
    * @param bus A pointer to the TMB.
    **/
-  DeleteWorkOrder(const CatalogRelationSchema &input_relation,
+  DeleteWorkOrder(const std::size_t query_id,
+                  const CatalogRelationSchema &input_relation,
                   const block_id input_block_id,
                   const Predicate *predicate,
                   StorageManager *storage_manager,
                   const std::size_t delete_operator_index,
                   const tmb::client_id scheduler_client_id,
-                  const std::size_t query_id,
                   MessageBus *bus)
       : WorkOrder(query_id),
         input_relation_(input_relation),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d67f61e1/relational_operators/DestroyHashOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/DestroyHashOperator.cpp b/relational_operators/DestroyHashOperator.cpp
index c92117a..8aa40b4 100644
--- a/relational_operators/DestroyHashOperator.cpp
+++ b/relational_operators/DestroyHashOperator.cpp
@@ -33,7 +33,7 @@ bool DestroyHashOperator::getAllWorkOrders(
   if (blocking_dependencies_met_ && !work_generated_) {
     work_generated_ = true;
     container->addNormalWorkOrder(
-        new DestroyHashWorkOrder(hash_table_index_, query_id_, query_context),
+        new DestroyHashWorkOrder(query_id_, hash_table_index_, query_context),
         op_index_);
   }
   return work_generated_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d67f61e1/relational_operators/DestroyHashOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/DestroyHashOperator.hpp b/relational_operators/DestroyHashOperator.hpp
index 086c279..7d8acb7 100644
--- a/relational_operators/DestroyHashOperator.hpp
+++ b/relational_operators/DestroyHashOperator.hpp
@@ -46,11 +46,11 @@ class DestroyHashOperator : public RelationalOperator {
   /**
    * @brief Constructor.
    *
-   * @param hash_table_index The index of the JoinHashTable in QueryContext.
    * @param query_id The ID of the query to which this operator belongs.
+   * @param hash_table_index The index of the JoinHashTable in QueryContext.
    **/
-  DestroyHashOperator(const QueryContext::join_hash_table_id hash_table_index,
-                      const std::size_t query_id)
+  DestroyHashOperator(const std::size_t query_id,
+                      const QueryContext::join_hash_table_id hash_table_index)
       : RelationalOperator(query_id),
         hash_table_index_(hash_table_index),
         work_generated_(false) {}
@@ -78,12 +78,12 @@ class DestroyHashWorkOrder : public WorkOrder {
   /**
    * @brief Constructor.
    *
-   * @param hash_table_index The index of the JoinHashTable in QueryContext.
    * @param query_id The ID of the query to which this WorkOrder belongs.
+   * @param hash_table_index The index of the JoinHashTable in QueryContext.
    * @param query_context The QueryContext to use.
    **/
-  DestroyHashWorkOrder(const QueryContext::join_hash_table_id hash_table_index,
-                       const std::size_t query_id,
+  DestroyHashWorkOrder(const std::size_t query_id,
+                       const QueryContext::join_hash_table_id hash_table_index,
                        QueryContext *query_context)
       : WorkOrder(query_id),
         hash_table_index_(hash_table_index),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d67f61e1/relational_operators/DropTableOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/DropTableOperator.hpp b/relational_operators/DropTableOperator.hpp
index 0bbb718..a0a8d6e 100644
--- a/relational_operators/DropTableOperator.hpp
+++ b/relational_operators/DropTableOperator.hpp
@@ -55,14 +55,14 @@ class DropTableOperator : public RelationalOperator {
   /**
    * @brief Constructor.
    *
-   * @param relation The relation to drop.
    * @param query_id The ID of the query to which this operator belongs.
+   * @param relation The relation to drop.
    * @param database The databse where to drop \c relation.
    * @param only_drop_blocks If true, only drop the blocks belonging to \c
    *        relation, but leave \c relation in \c database.
    **/
-  DropTableOperator(const CatalogRelation &relation,
-                    const std::size_t query_id,
+  DropTableOperator(const std::size_t query_id,
+                    const CatalogRelation &relation,
                     CatalogDatabase *database,
                     const bool only_drop_blocks = false)
       : RelationalOperator(query_id),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d67f61e1/relational_operators/FinalizeAggregationOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/FinalizeAggregationOperator.hpp b/relational_operators/FinalizeAggregationOperator.hpp
index 51e55dc..e8a403f 100644
--- a/relational_operators/FinalizeAggregationOperator.hpp
+++ b/relational_operators/FinalizeAggregationOperator.hpp
@@ -54,16 +54,17 @@ class FinalizeAggregationOperator : public RelationalOperator {
    * @brief Constructor for finalizing aggregation state and writing output
    * tuples.  The actual aggregation is computed by the AggregationOperator.
    *
+   * @param query_id The ID of the query to which this operator belongs.
    * @param aggr_state_index The index of the AggregationState in QueryContext.
    * @param output_relation The output relation.
    * @param output_destination_index The index of the InsertDestination in the
    *        QueryContext to insert aggregation results.
-   * @param query_id The ID of the query to which this operator belongs.
    */
-  FinalizeAggregationOperator(const QueryContext::aggregation_state_id aggr_state_index,
-                              const CatalogRelation &output_relation,
-                              const QueryContext::insert_destination_id output_destination_index,
-                              const std::size_t query_id)
+  FinalizeAggregationOperator(
+      const std::size_t query_id,
+      const QueryContext::aggregation_state_id aggr_state_index,
+      const CatalogRelation &output_relation,
+      const QueryContext::insert_destination_id output_destination_index)
       : RelationalOperator(query_id),
         aggr_state_index_(aggr_state_index),
         output_relation_(output_relation),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d67f61e1/relational_operators/HashJoinOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/HashJoinOperator.cpp b/relational_operators/HashJoinOperator.cpp
index d8c7304..b89cfb3 100644
--- a/relational_operators/HashJoinOperator.cpp
+++ b/relational_operators/HashJoinOperator.cpp
@@ -290,7 +290,8 @@ bool HashJoinOperator::getAllNonOuterJoinWorkOrders(
       if (!started_) {
         for (const block_id probe_block_id : probe_relation_block_ids_) {
           container->addNormalWorkOrder(
-              new JoinWorkOrderClass(build_relation_,
+              new JoinWorkOrderClass(query_id_,
+                                     build_relation_,
                                      probe_relation_,
                                      join_key_attributes_,
                                      any_join_key_attributes_nullable_,
@@ -298,7 +299,6 @@ bool HashJoinOperator::getAllNonOuterJoinWorkOrders(
                                      residual_predicate,
                                      selection,
                                      hash_table,
-                                     query_id_,
                                      output_destination,
                                      storage_manager),
               op_index_);
@@ -309,17 +309,18 @@ bool HashJoinOperator::getAllNonOuterJoinWorkOrders(
     } else {
       while (num_workorders_generated_ < probe_relation_block_ids_.size()) {
         container->addNormalWorkOrder(
-            new JoinWorkOrderClass(build_relation_,
-                                   probe_relation_,
-                                   join_key_attributes_,
-                                   any_join_key_attributes_nullable_,
-                                   probe_relation_block_ids_[num_workorders_generated_],
-                                   residual_predicate,
-                                   selection,
-                                   hash_table,
-                                   query_id_,
-                                   output_destination,
-                                   storage_manager),
+            new JoinWorkOrderClass(
+                query_id_,
+                build_relation_,
+                probe_relation_,
+                join_key_attributes_,
+                any_join_key_attributes_nullable_,
+                probe_relation_block_ids_[num_workorders_generated_],
+                residual_predicate,
+                selection,
+                hash_table,
+                output_destination,
+                storage_manager),
             op_index_);
         ++num_workorders_generated_;
       }  // end while
@@ -350,6 +351,7 @@ bool HashJoinOperator::getAllOuterJoinWorkOrders(
         for (const block_id probe_block_id : probe_relation_block_ids_) {
           container->addNormalWorkOrder(
               new HashOuterJoinWorkOrder(
+                  query_id_,
                   build_relation_,
                   probe_relation_,
                   join_key_attributes_,
@@ -358,7 +360,6 @@ bool HashJoinOperator::getAllOuterJoinWorkOrders(
                   selection,
                   is_selection_on_build_,
                   hash_table,
-                  query_id_,
                   output_destination,
                   storage_manager),
               op_index_);
@@ -370,6 +371,7 @@ bool HashJoinOperator::getAllOuterJoinWorkOrders(
       while (num_workorders_generated_ < probe_relation_block_ids_.size()) {
         container->addNormalWorkOrder(
             new HashOuterJoinWorkOrder(
+                query_id_,
                 build_relation_,
                 probe_relation_,
                 join_key_attributes_,
@@ -378,7 +380,6 @@ bool HashJoinOperator::getAllOuterJoinWorkOrders(
                 selection,
                 is_selection_on_build_,
                 hash_table,
-                query_id_,
                 output_destination,
                 storage_manager),
             op_index_);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d67f61e1/relational_operators/HashJoinOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/HashJoinOperator.hpp b/relational_operators/HashJoinOperator.hpp
index 825f360..621b8cf 100644
--- a/relational_operators/HashJoinOperator.hpp
+++ b/relational_operators/HashJoinOperator.hpp
@@ -85,6 +85,7 @@ class HashJoinOperator : public RelationalOperator {
    *       dependent on the selectivity of the predicates involved. The decision
    *       is left to the query optimizer.
    *
+   * @param query_id The ID of the query.
    * @param build_relation The relation that the hash table was originally
    *        built on (i.e. the inner relation in the join).
    * @param probe_relation The relation to probe the hash table with (i.e. the
@@ -108,25 +109,25 @@ class HashJoinOperator : public RelationalOperator {
    *        corresponding to the attributes of the relation referred by
    *        output_relation_id. Each Scalar is evaluated for the joined tuples,
    *        and the resulting value is inserted into the join result.
-   * @param query_id The ID of the query.
    * @param is_selection_on_build Whether each selection Scalar is using attributes
    *        from the build relation as input. Can be NULL for inner/semi/anti
    *        joins since this information is not utilized by these joins.
    * @param join_type The type of join corresponding to this operator.
    **/
-  HashJoinOperator(const CatalogRelation &build_relation,
-                   const CatalogRelation &probe_relation,
-                   const bool probe_relation_is_stored,
-                   const std::vector<attribute_id> &join_key_attributes,
-                   const bool any_join_key_attributes_nullable,
-                   const CatalogRelation &output_relation,
-                   const QueryContext::insert_destination_id output_destination_index,
-                   const QueryContext::join_hash_table_id hash_table_index,
-                   const QueryContext::predicate_id residual_predicate_index,
-                   const QueryContext::scalar_group_id selection_index,
-                   const std::size_t query_id,
-                   const std::vector<bool> *is_selection_on_build = nullptr,
-                   const JoinType join_type = JoinType::kInnerJoin)
+  HashJoinOperator(
+      const std::size_t query_id,
+      const CatalogRelation &build_relation,
+      const CatalogRelation &probe_relation,
+      const bool probe_relation_is_stored,
+      const std::vector<attribute_id> &join_key_attributes,
+      const bool any_join_key_attributes_nullable,
+      const CatalogRelation &output_relation,
+      const QueryContext::insert_destination_id output_destination_index,
+      const QueryContext::join_hash_table_id hash_table_index,
+      const QueryContext::predicate_id residual_predicate_index,
+      const QueryContext::scalar_group_id selection_index,
+      const std::vector<bool> *is_selection_on_build = nullptr,
+      const JoinType join_type = JoinType::kInnerJoin)
       : RelationalOperator(query_id),
         build_relation_(build_relation),
         probe_relation_(probe_relation),
@@ -230,6 +231,7 @@ class HashInnerJoinWorkOrder : public WorkOrder {
   /**
    * @brief Constructor.
    *
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param build_relation The relation that the hash table was originally built
    *        on (i.e. the inner relation in the join).
    * @param probe_relation The relation to probe the hash table with (i.e. the
@@ -246,21 +248,21 @@ class HashInnerJoinWorkOrder : public WorkOrder {
    *        in \c output_destination. Each Scalar is evaluated for the joined
    *        tuples, and the resulting value is inserted into the join result.
    * @param hash_table The JoinHashTable to use.
-   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to insert the join results.
    * @param storage_manager The StorageManager to use.
    **/
-  HashInnerJoinWorkOrder(const CatalogRelationSchema &build_relation,
-                         const CatalogRelationSchema &probe_relation,
-                         const std::vector<attribute_id> &join_key_attributes,
-                         const bool any_join_key_attributes_nullable,
-                         const block_id lookup_block_id,
-                         const Predicate *residual_predicate,
-                         const std::vector<std::unique_ptr<const Scalar>> &selection,
-                         const JoinHashTable &hash_table,
-                         const std::size_t query_id,
-                         InsertDestination *output_destination,
-                         StorageManager *storage_manager)
+  HashInnerJoinWorkOrder(
+      const std::size_t query_id,
+      const CatalogRelationSchema &build_relation,
+      const CatalogRelationSchema &probe_relation,
+      const std::vector<attribute_id> &join_key_attributes,
+      const bool any_join_key_attributes_nullable,
+      const block_id lookup_block_id,
+      const Predicate *residual_predicate,
+      const std::vector<std::unique_ptr<const Scalar>> &selection,
+      const JoinHashTable &hash_table,
+      InsertDestination *output_destination,
+      StorageManager *storage_manager)
       : WorkOrder(query_id),
         build_relation_(build_relation),
         probe_relation_(probe_relation),
@@ -276,6 +278,7 @@ class HashInnerJoinWorkOrder : public WorkOrder {
   /**
    * @brief Constructor for the distributed version.
    *
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param build_relation The relation that the hash table was originally built
    *        on (i.e. the inner relation in the join).
    * @param probe_relation The relation to probe the hash table with (i.e. the
@@ -292,21 +295,21 @@ class HashInnerJoinWorkOrder : public WorkOrder {
    *        in \c output_destination. Each Scalar is evaluated for the joined
    *        tuples, and the resulting value is inserted into the join result.
    * @param hash_table The JoinHashTable to use.
-   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to insert the join results.
    * @param storage_manager The StorageManager to use.
    **/
-  HashInnerJoinWorkOrder(const CatalogRelationSchema &build_relation,
-                         const CatalogRelationSchema &probe_relation,
-                         std::vector<attribute_id> &&join_key_attributes,
-                         const bool any_join_key_attributes_nullable,
-                         const block_id lookup_block_id,
-                         const Predicate *residual_predicate,
-                         const std::vector<std::unique_ptr<const Scalar>> &selection,
-                         const JoinHashTable &hash_table,
-                         const std::size_t query_id,
-                         InsertDestination *output_destination,
-                         StorageManager *storage_manager)
+  HashInnerJoinWorkOrder(
+      const std::size_t query_id,
+      const CatalogRelationSchema &build_relation,
+      const CatalogRelationSchema &probe_relation,
+      std::vector<attribute_id> &&join_key_attributes,
+      const bool any_join_key_attributes_nullable,
+      const block_id lookup_block_id,
+      const Predicate *residual_predicate,
+      const std::vector<std::unique_ptr<const Scalar>> &selection,
+      const JoinHashTable &hash_table,
+      InsertDestination *output_destination,
+      StorageManager *storage_manager)
       : WorkOrder(query_id),
         build_relation_(build_relation),
         probe_relation_(probe_relation),
@@ -359,6 +362,7 @@ class HashSemiJoinWorkOrder : public WorkOrder {
   /**
    * @brief Constructor.
    *
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param build_relation The relation that the hash table was originally built
    *        on (i.e. the inner relation in the join).
    * @param probe_relation The relation to probe the hash table with (i.e. the
@@ -375,21 +379,21 @@ class HashSemiJoinWorkOrder : public WorkOrder {
    *        in \c output_destination. Each Scalar is evaluated for the joined
    *        tuples, and the resulting value is inserted into the join result.
    * @param hash_table The JoinHashTable to use.
-   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to insert the join results.
    * @param storage_manager The StorageManager to use.
    **/
-  HashSemiJoinWorkOrder(const CatalogRelationSchema &build_relation,
-                        const CatalogRelationSchema &probe_relation,
-                        const std::vector<attribute_id> &join_key_attributes,
-                        const bool any_join_key_attributes_nullable,
-                        const block_id lookup_block_id,
-                        const Predicate *residual_predicate,
-                        const std::vector<std::unique_ptr<const Scalar>> &selection,
-                        const JoinHashTable &hash_table,
-                        const std::size_t query_id,
-                        InsertDestination *output_destination,
-                        StorageManager *storage_manager)
+  HashSemiJoinWorkOrder(
+      const std::size_t query_id,
+      const CatalogRelationSchema &build_relation,
+      const CatalogRelationSchema &probe_relation,
+      const std::vector<attribute_id> &join_key_attributes,
+      const bool any_join_key_attributes_nullable,
+      const block_id lookup_block_id,
+      const Predicate *residual_predicate,
+      const std::vector<std::unique_ptr<const Scalar>> &selection,
+      const JoinHashTable &hash_table,
+      InsertDestination *output_destination,
+      StorageManager *storage_manager)
       : WorkOrder(query_id),
         build_relation_(build_relation),
         probe_relation_(probe_relation),
@@ -405,6 +409,7 @@ class HashSemiJoinWorkOrder : public WorkOrder {
   /**
    * @brief Constructor for the distributed version.
    *
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param build_relation The relation that the hash table was originally built
    *        on (i.e. the inner relation in the join).
    * @param probe_relation The relation to probe the hash table with (i.e. the
@@ -421,21 +426,21 @@ class HashSemiJoinWorkOrder : public WorkOrder {
    *        in \c output_destination. Each Scalar is evaluated for the joined
    *        tuples, and the resulting value is inserted into the join result.
    * @param hash_table The JoinHashTable to use.
-   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to insert the join results.
    * @param storage_manager The StorageManager to use.
    **/
-  HashSemiJoinWorkOrder(const CatalogRelationSchema &build_relation,
-                        const CatalogRelationSchema &probe_relation,
-                        std::vector<attribute_id> &&join_key_attributes,
-                        const bool any_join_key_attributes_nullable,
-                        const block_id lookup_block_id,
-                        const Predicate *residual_predicate,
-                        const std::vector<std::unique_ptr<const Scalar>> &selection,
-                        const JoinHashTable &hash_table,
-                        const std::size_t query_id,
-                        InsertDestination *output_destination,
-                        StorageManager *storage_manager)
+  HashSemiJoinWorkOrder(
+      const std::size_t query_id,
+      const CatalogRelationSchema &build_relation,
+      const CatalogRelationSchema &probe_relation,
+      std::vector<attribute_id> &&join_key_attributes,
+      const bool any_join_key_attributes_nullable,
+      const block_id lookup_block_id,
+      const Predicate *residual_predicate,
+      const std::vector<std::unique_ptr<const Scalar>> &selection,
+      const JoinHashTable &hash_table,
+      InsertDestination *output_destination,
+      StorageManager *storage_manager)
       : WorkOrder(query_id),
         build_relation_(build_relation),
         probe_relation_(probe_relation),
@@ -481,6 +486,7 @@ class HashAntiJoinWorkOrder : public WorkOrder {
   /**
    * @brief Constructor.
    *
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param build_relation The relation that the hash table was originally built
    *        on (i.e. the inner relation in the join).
    * @param probe_relation The relation to probe the hash table with (i.e. the
@@ -497,21 +503,21 @@ class HashAntiJoinWorkOrder : public WorkOrder {
    *        in \c output_destination. Each Scalar is evaluated for the joined
    *        tuples, and the resulting value is inserted into the join result.
    * @param hash_table The JoinHashTable to use.
-   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to insert the join results.
    * @param storage_manager The StorageManager to use.
    **/
-  HashAntiJoinWorkOrder(const CatalogRelationSchema &build_relation,
-                        const CatalogRelationSchema &probe_relation,
-                        const std::vector<attribute_id> &join_key_attributes,
-                        const bool any_join_key_attributes_nullable,
-                        const block_id lookup_block_id,
-                        const Predicate *residual_predicate,
-                        const std::vector<std::unique_ptr<const Scalar>> &selection,
-                        const JoinHashTable &hash_table,
-                        const std::size_t query_id,
-                        InsertDestination *output_destination,
-                        StorageManager *storage_manager)
+  HashAntiJoinWorkOrder(
+      const std::size_t query_id,
+      const CatalogRelationSchema &build_relation,
+      const CatalogRelationSchema &probe_relation,
+      const std::vector<attribute_id> &join_key_attributes,
+      const bool any_join_key_attributes_nullable,
+      const block_id lookup_block_id,
+      const Predicate *residual_predicate,
+      const std::vector<std::unique_ptr<const Scalar>> &selection,
+      const JoinHashTable &hash_table,
+      InsertDestination *output_destination,
+      StorageManager *storage_manager)
       : WorkOrder(query_id),
         build_relation_(build_relation),
         probe_relation_(probe_relation),
@@ -527,6 +533,7 @@ class HashAntiJoinWorkOrder : public WorkOrder {
   /**
    * @brief Constructor for the distributed version.
    *
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param build_relation The relation that the hash table was originally built
    *        on (i.e. the inner relation in the join).
    * @param probe_relation The relation to probe the hash table with (i.e. the
@@ -543,21 +550,21 @@ class HashAntiJoinWorkOrder : public WorkOrder {
    *        in \c output_destination. Each Scalar is evaluated for the joined
    *        tuples, and the resulting value is inserted into the join result.
    * @param hash_table The JoinHashTable to use.
-   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to insert the join results.
    * @param storage_manager The StorageManager to use.
    **/
-  HashAntiJoinWorkOrder(const CatalogRelationSchema &build_relation,
-                        const CatalogRelationSchema &probe_relation,
-                        std::vector<attribute_id> &&join_key_attributes,
-                        const bool any_join_key_attributes_nullable,
-                        const block_id lookup_block_id,
-                        const Predicate *residual_predicate,
-                        const std::vector<std::unique_ptr<const Scalar>> &selection,
-                        const JoinHashTable &hash_table,
-                        const std::size_t query_id,
-                        InsertDestination *output_destination,
-                        StorageManager *storage_manager)
+  HashAntiJoinWorkOrder(
+      const std::size_t query_id,
+      const CatalogRelationSchema &build_relation,
+      const CatalogRelationSchema &probe_relation,
+      std::vector<attribute_id> &&join_key_attributes,
+      const bool any_join_key_attributes_nullable,
+      const block_id lookup_block_id,
+      const Predicate *residual_predicate,
+      const std::vector<std::unique_ptr<const Scalar>> &selection,
+      const JoinHashTable &hash_table,
+      InsertDestination *output_destination,
+      StorageManager *storage_manager)
       : WorkOrder(query_id),
         build_relation_(build_relation),
         probe_relation_(probe_relation),
@@ -627,17 +634,18 @@ class HashOuterJoinWorkOrder : public WorkOrder {
    * @param output_destination The InsertDestination to insert the join results.
    * @param storage_manager The StorageManager to use.
    **/
-  HashOuterJoinWorkOrder(const CatalogRelationSchema &build_relation,
-                         const CatalogRelationSchema &probe_relation,
-                         const std::vector<attribute_id> &join_key_attributes,
-                         const bool any_join_key_attributes_nullable,
-                         const block_id lookup_block_id,
-                         const std::vector<std::unique_ptr<const Scalar>> &selection,
-                         const std::vector<bool> &is_selection_on_build,
-                         const JoinHashTable &hash_table,
-                         const std::size_t query_id,
-                         InsertDestination *output_destination,
-                         StorageManager *storage_manager)
+  HashOuterJoinWorkOrder(
+      const std::size_t query_id,
+      const CatalogRelationSchema &build_relation,
+      const CatalogRelationSchema &probe_relation,
+      const std::vector<attribute_id> &join_key_attributes,
+      const bool any_join_key_attributes_nullable,
+      const block_id lookup_block_id,
+      const std::vector<std::unique_ptr<const Scalar>> &selection,
+      const std::vector<bool> &is_selection_on_build,
+      const JoinHashTable &hash_table,
+      InsertDestination *output_destination,
+      StorageManager *storage_manager)
       : WorkOrder(query_id),
         build_relation_(build_relation),
         probe_relation_(probe_relation),
@@ -653,6 +661,7 @@ class HashOuterJoinWorkOrder : public WorkOrder {
   /**
    * @brief Constructor for the distributed version.
    *
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param build_relation The relation that the hash table was originally built
    *        on (i.e. the inner relation in the join).
    * @param probe_relation The relation to probe the hash table with (i.e. the
@@ -668,21 +677,21 @@ class HashOuterJoinWorkOrder : public WorkOrder {
    *        is using attributes from the build relation as input. Note that the
    *        length of this vector should equal the length of \p selection.
    * @param hash_table The JoinHashTable to use.
-   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to insert the join results.
    * @param storage_manager The StorageManager to use.
    **/
-  HashOuterJoinWorkOrder(const CatalogRelationSchema &build_relation,
-                         const CatalogRelationSchema &probe_relation,
-                         std::vector<attribute_id> &&join_key_attributes,
-                         const bool any_join_key_attributes_nullable,
-                         const block_id lookup_block_id,
-                         const std::vector<std::unique_ptr<const Scalar>> &selection,
-                         std::vector<bool> &&is_selection_on_build,
-                         const JoinHashTable &hash_table,
-                         const std::size_t query_id,
-                         InsertDestination *output_destination,
-                         StorageManager *storage_manager)
+  HashOuterJoinWorkOrder(
+      const std::size_t query_id,
+      const CatalogRelationSchema &build_relation,
+      const CatalogRelationSchema &probe_relation,
+      std::vector<attribute_id> &&join_key_attributes,
+      const bool any_join_key_attributes_nullable,
+      const block_id lookup_block_id,
+      const std::vector<std::unique_ptr<const Scalar>> &selection,
+      std::vector<bool> &&is_selection_on_build,
+      const JoinHashTable &hash_table,
+      InsertDestination *output_destination,
+      StorageManager *storage_manager)
       : WorkOrder(query_id),
         build_relation_(build_relation),
         probe_relation_(probe_relation),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d67f61e1/relational_operators/InsertOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/InsertOperator.hpp b/relational_operators/InsertOperator.hpp
index ccef444..51c606d 100644
--- a/relational_operators/InsertOperator.hpp
+++ b/relational_operators/InsertOperator.hpp
@@ -53,16 +53,17 @@ class InsertOperator : public RelationalOperator {
   /**
    * @brief Constructor.
    *
+   * @param query_id The ID of the query to which this operator belongs.
    * @param output_relation_id The output relation.
    * @param output_destination_index The index of the InsertDestination in the
    *        QueryContext to insert the tuple.
    * @param tuple_index The index of the tuple to insert in the QueryContext.
-   * @param query_id The ID of the query to which this operator belongs.
    **/
-  InsertOperator(const CatalogRelation &output_relation,
-                 const QueryContext::insert_destination_id output_destination_index,
-                 const QueryContext::tuple_id tuple_index,
-                 const std::size_t query_id)
+  InsertOperator(
+      const std::size_t query_id,
+      const CatalogRelation &output_relation,
+      const QueryContext::insert_destination_id output_destination_index,
+      const QueryContext::tuple_id tuple_index)
       : RelationalOperator(query_id),
         output_relation_(output_relation),
         output_destination_index_(output_destination_index),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d67f61e1/relational_operators/NestedLoopsJoinOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/NestedLoopsJoinOperator.cpp b/relational_operators/NestedLoopsJoinOperator.cpp
index 317cc5d..5a47fca 100644
--- a/relational_operators/NestedLoopsJoinOperator.cpp
+++ b/relational_operators/NestedLoopsJoinOperator.cpp
@@ -76,15 +76,17 @@ bool NestedLoopsJoinOperator::getAllWorkOrders(
       for (const block_id left_block_id : left_relation_block_ids_) {
         for (const block_id right_block_id : right_relation_block_ids_) {
           container->addNormalWorkOrder(
-              new NestedLoopsJoinWorkOrder(left_input_relation_,
-                                           right_input_relation_,
-                                           left_block_id,
-                                           right_block_id,
-                                           query_context->getPredicate(join_predicate_index_),
-                                           query_context->getScalarGroup(selection_index_),
-                                           query_id_,
-                                           query_context->getInsertDestination(output_destination_index_),
-                                           storage_manager),
+              new NestedLoopsJoinWorkOrder(
+                  query_id_,
+                  left_input_relation_,
+                  right_input_relation_,
+                  left_block_id,
+                  right_block_id,
+                  query_context->getPredicate(join_predicate_index_),
+                  query_context->getScalarGroup(selection_index_),
+                  query_context->getInsertDestination(
+                      output_destination_index_),
+                  storage_manager),
               op_index_);
         }
       }
@@ -166,15 +168,16 @@ std::size_t NestedLoopsJoinOperator::getAllWorkOrdersHelperBothNotStored(WorkOrd
          right_index < right_max;
          ++right_index) {
       container->addNormalWorkOrder(
-          new NestedLoopsJoinWorkOrder(left_input_relation_,
-                                       right_input_relation_,
-                                       left_relation_block_ids_[left_index],
-                                       right_relation_block_ids_[right_index],
-                                       query_context->getPredicate(join_predicate_index_),
-                                       query_context->getScalarGroup(selection_index_),
-                                       query_id_,
-                                       query_context->getInsertDestination(output_destination_index_),
-                                       storage_manager),
+          new NestedLoopsJoinWorkOrder(
+              query_id_,
+              left_input_relation_,
+              right_input_relation_,
+              left_relation_block_ids_[left_index],
+              right_relation_block_ids_[right_index],
+              query_context->getPredicate(join_predicate_index_),
+              query_context->getScalarGroup(selection_index_),
+              query_context->getInsertDestination(output_destination_index_),
+              storage_manager),
           op_index_);
     }
   }
@@ -201,13 +204,13 @@ bool NestedLoopsJoinOperator::getAllWorkOrdersHelperOneStored(WorkOrdersContaine
       for (const block_id left_block_id : left_relation_block_ids_) {
         container->addNormalWorkOrder(
             new NestedLoopsJoinWorkOrder(
+                query_id_,
                 left_input_relation_,
                 right_input_relation_,
                 left_block_id,
                 right_relation_block_ids_[right_index],
                 join_predicate,
                 selection,
-                query_id_,
                 output_destination,
                 storage_manager),
             op_index_);
@@ -221,13 +224,13 @@ bool NestedLoopsJoinOperator::getAllWorkOrdersHelperOneStored(WorkOrdersContaine
          ++left_index) {
       for (const block_id right_block_id : right_relation_block_ids_) {
         container->addNormalWorkOrder(
-            new NestedLoopsJoinWorkOrder(left_input_relation_,
+            new NestedLoopsJoinWorkOrder(query_id_,
+                                         left_input_relation_,
                                          right_input_relation_,
                                          left_relation_block_ids_[left_index],
                                          right_block_id,
                                          join_predicate,
                                          selection,
-                                         query_id_,
                                          output_destination,
                                          storage_manager),
             op_index_);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d67f61e1/relational_operators/NestedLoopsJoinOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/NestedLoopsJoinOperator.hpp b/relational_operators/NestedLoopsJoinOperator.hpp
index f165442..0b13842 100644
--- a/relational_operators/NestedLoopsJoinOperator.hpp
+++ b/relational_operators/NestedLoopsJoinOperator.hpp
@@ -59,6 +59,7 @@ class NestedLoopsJoinOperator : public RelationalOperator {
   /**
    * @brief Constructor.
    *
+   * @param query_id The ID of the query to which this operator belongs.
    * @param left_input_relation The first relation in the join (order is not
    *        actually important).
    * @param right_input_relation The second relation in the join (order is not
@@ -76,17 +77,17 @@ class NestedLoopsJoinOperator : public RelationalOperator {
    * @param left_relation_is_stored If left_input_relation is a stored relation.
    * @param right_relation_is_stored If right_input_relation is a stored
    *                                 relation.
-   * @param query_id The ID of the query to which this operator belongs.
    **/
-  NestedLoopsJoinOperator(const CatalogRelation &left_input_relation,
-                          const CatalogRelation &right_input_relation,
-                          const CatalogRelation &output_relation,
-                          const QueryContext::insert_destination_id output_destination_index,
-                          const QueryContext::predicate_id join_predicate_index,
-                          const QueryContext::scalar_group_id selection_index,
-                          bool left_relation_is_stored,
-                          bool right_relation_is_stored,
-                          const std::size_t query_id)
+  NestedLoopsJoinOperator(
+      const std::size_t query_id,
+      const CatalogRelation &left_input_relation,
+      const CatalogRelation &right_input_relation,
+      const CatalogRelation &output_relation,
+      const QueryContext::insert_destination_id output_destination_index,
+      const QueryContext::predicate_id join_predicate_index,
+      const QueryContext::scalar_group_id selection_index,
+      bool left_relation_is_stored,
+      bool right_relation_is_stored)
       : RelationalOperator(query_id),
         left_input_relation_(left_input_relation),
         right_input_relation_(right_input_relation),
@@ -96,10 +97,12 @@ class NestedLoopsJoinOperator : public RelationalOperator {
         selection_index_(selection_index),
         left_relation_is_stored_(left_relation_is_stored),
         right_relation_is_stored_(right_relation_is_stored),
-        left_relation_block_ids_(left_relation_is_stored ? left_input_relation.getBlocksSnapshot()
-                                                         : std::vector<block_id>()),
-        right_relation_block_ids_(right_relation_is_stored ? right_input_relation.getBlocksSnapshot()
-                                                           : std::vector<block_id>()),
+        left_relation_block_ids_(left_relation_is_stored
+                                     ? left_input_relation.getBlocksSnapshot()
+                                     : std::vector<block_id>()),
+        right_relation_block_ids_(right_relation_is_stored
+                                      ? right_input_relation.getBlocksSnapshot()
+                                      : std::vector<block_id>()),
         num_left_workorders_generated_(0),
         num_right_workorders_generated_(0),
         done_feeding_left_relation_(false),
@@ -222,6 +225,7 @@ class NestedLoopsJoinWorkOrder : public WorkOrder {
   /**
    * @brief Constructor.
    *
+   * @param query_id The ID of the query to which this operator belongs.
    * @param left_input_relation The first relation in the join (order is not
    *        actually important).
    * @param right_input_relation The second relation in the join (order is not
@@ -233,19 +237,19 @@ class NestedLoopsJoinWorkOrder : public WorkOrder {
    * @param selection A list of Scalars corresponding to the relation attributes
    *        in \c output_destination. Each Scalar is evaluated for the joined
    *        tuples, and the resulting value is inserted into the join result.
-   * @param query_id The ID of the query to which this operator belongs.
    * @param output_destination The InsertDestination to insert the join results.
    * @param storage_manager The StorageManager to use.
    **/
-  NestedLoopsJoinWorkOrder(const CatalogRelationSchema &left_input_relation,
-                           const CatalogRelationSchema &right_input_relation,
-                           const block_id left_block_id,
-                           const block_id right_block_id,
-                           const Predicate *join_predicate,
-                           const std::vector<std::unique_ptr<const Scalar>> &selection,
-                           const std::size_t query_id,
-                           InsertDestination *output_destination,
-                           StorageManager *storage_manager)
+  NestedLoopsJoinWorkOrder(
+      const std::size_t query_id,
+      const CatalogRelationSchema &left_input_relation,
+      const CatalogRelationSchema &right_input_relation,
+      const block_id left_block_id,
+      const block_id right_block_id,
+      const Predicate *join_predicate,
+      const std::vector<std::unique_ptr<const Scalar>> &selection,
+      InsertDestination *output_destination,
+      StorageManager *storage_manager)
       : WorkOrder(query_id),
         left_input_relation_(left_input_relation),
         right_input_relation_(right_input_relation),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d67f61e1/relational_operators/RebuildWorkOrder.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/RebuildWorkOrder.hpp b/relational_operators/RebuildWorkOrder.hpp
index ae876ba..86f8eaf 100644
--- a/relational_operators/RebuildWorkOrder.hpp
+++ b/relational_operators/RebuildWorkOrder.hpp
@@ -48,6 +48,7 @@ class RebuildWorkOrder : public WorkOrder {
   /**
    * @brief Constructor.
    *
+   * @param query_id The ID of the query to which this RebuildWorkOrder belongs.
    * @param block_ref A MutableBlockReference to the storage block which needs
    *        to be rebuilt.
    * @param input_operator_index The index of the relational operator in the
@@ -55,15 +56,15 @@ class RebuildWorkOrder : public WorkOrder {
    * @param input_relation_id The ID of the CatalogRelation to which the given
    *        storage block belongs to.
    * @param scheduler_client_id The TMB client ID of the scheduler thread.
-   * @param query_id The ID of the query to which this RebuildWorkOrder belongs.
    * @param bus A pointer to the TMB.
    **/
-  RebuildWorkOrder(MutableBlockReference &&block_ref,
-                   const std::size_t input_operator_index,
-                   const relation_id input_relation_id,
-                   const client_id scheduler_client_id,
-                   const std::size_t query_id,
-                   MessageBus *bus)
+  RebuildWorkOrder(
+      const std::size_t query_id,
+      MutableBlockReference &&block_ref,  // NOLINT(whitespace/operators)
+      const std::size_t input_operator_index,
+      const relation_id input_relation_id,
+      const client_id scheduler_client_id,
+      MessageBus *bus)
       : WorkOrder(query_id),
         block_ref_(std::move(block_ref)),
         input_operator_index_(input_operator_index),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d67f61e1/relational_operators/SampleOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/SampleOperator.cpp b/relational_operators/SampleOperator.cpp
index b318ce4..6669148 100644
--- a/relational_operators/SampleOperator.cpp
+++ b/relational_operators/SampleOperator.cpp
@@ -54,11 +54,11 @@ bool SampleOperator::getAllWorkOrders(
         for (const block_id input_block_id : input_relation_block_ids_) {
           if (distribution(generator) <= probability) {
             container->addNormalWorkOrder(
-                new SampleWorkOrder(input_relation_,
+                new SampleWorkOrder(query_id_,
+                                    input_relation_,
                                     input_block_id,
                                     is_block_sample_,
                                     percentage_,
-                                    query_id_,
                                     output_destination,
                                     storage_manager),
                 op_index_);
@@ -68,15 +68,14 @@ bool SampleOperator::getAllWorkOrders(
         // Add all the blocks for tuple sampling which would handle
         // the sampling from each block
         for (const block_id input_block_id : input_relation_block_ids_) {
-          container->addNormalWorkOrder(
-              new SampleWorkOrder(input_relation_,
-                                  input_block_id,
-                                  is_block_sample_,
-                                  percentage_,
-                                  query_id_,
-                                  output_destination,
-                                  storage_manager),
-              op_index_);
+          container->addNormalWorkOrder(new SampleWorkOrder(query_id_,
+                                                            input_relation_,
+                                                            input_block_id,
+                                                            is_block_sample_,
+                                                            percentage_,
+                                                            output_destination,
+                                                            storage_manager),
+                                        op_index_);
         }
       }
       started_ = true;
@@ -87,13 +86,14 @@ bool SampleOperator::getAllWorkOrders(
           while (num_workorders_generated_ < input_relation_block_ids_.size()) {
             if (distribution(generator) <= probability) {
               container->addNormalWorkOrder(
-                  new SampleWorkOrder(input_relation_,
-                                      input_relation_block_ids_[num_workorders_generated_],
-                                      is_block_sample_,
-                                      percentage_,
-                                      query_id_,
-                                      output_destination,
-                                      storage_manager),
+                  new SampleWorkOrder(
+                      query_id_,
+                      input_relation_,
+                      input_relation_block_ids_[num_workorders_generated_],
+                      is_block_sample_,
+                      percentage_,
+                      output_destination,
+                      storage_manager),
                   op_index_);
             ++num_workorders_generated_;
           }
@@ -101,13 +101,14 @@ bool SampleOperator::getAllWorkOrders(
       } else  {
         while (num_workorders_generated_ < input_relation_block_ids_.size()) {
           container->addNormalWorkOrder(
-              new SampleWorkOrder(input_relation_,
-                                  input_relation_block_ids_[num_workorders_generated_],
-                                  is_block_sample_,
-                                  percentage_,
-                                  query_id_,
-                                  output_destination,
-                                  storage_manager),
+              new SampleWorkOrder(
+                  query_id_,
+                  input_relation_,
+                  input_relation_block_ids_[num_workorders_generated_],
+                  is_block_sample_,
+                  percentage_,
+                  output_destination,
+                  storage_manager),
               op_index_);
           ++num_workorders_generated_;
         }



[05/16] incubator-quickstep git commit: Checking if Travis Ubuntu uses pre-built clang-3.7 instead of downloading.

Posted by hb...@apache.org.
Checking if Travis Ubuntu uses pre-built clang-3.7 instead of downloading.


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

Branch: refs/heads/query-manager-used-in-foreman
Commit: bf9150e81a2a1fa59ed9053ce7657a66d9c14f9a
Parents: 69c6551
Author: Harshad Deshmukh <hb...@apache.org>
Authored: Fri Jun 3 15:40:34 2016 -0500
Committer: Harshad Deshmukh <hb...@apache.org>
Committed: Fri Jun 3 15:40:34 2016 -0500

----------------------------------------------------------------------
 .travis.yml | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/bf9150e8/.travis.yml
----------------------------------------------------------------------
diff --git a/.travis.yml b/.travis.yml
index 483a02b..065cfe4 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -77,11 +77,11 @@ addons:
   apt:
     sources:
       - ubuntu-toolchain-r-test
-      - llvm-toolchain-precise-3.7
+#      - llvm-toolchain-precise-3.7
     packages:
       - gcc-5
       - g++-5
-      - clang-3.7
+#      - clang-3.7
       - binutils-gold
       - libprotobuf-dev
       - protobuf-compiler


[11/16] incubator-quickstep git commit: Manually resetting the changes in the travis YAML.

Posted by hb...@apache.org.
Manually resetting the changes in the travis YAML.


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

Branch: refs/heads/query-manager-used-in-foreman
Commit: 9f1008a0d07013ccbcc3b9c56be90f05b8219462
Parents: 3055379
Author: Harshad Deshmukh <hb...@apache.org>
Authored: Mon Jun 6 15:31:35 2016 -0500
Committer: Harshad Deshmukh <hb...@apache.org>
Committed: Mon Jun 6 15:31:35 2016 -0500

----------------------------------------------------------------------
 .travis.yml | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/9f1008a0/.travis.yml
----------------------------------------------------------------------
diff --git a/.travis.yml b/.travis.yml
index 065cfe4..483a02b 100644
--- a/.travis.yml
+++ b/.travis.yml
@@ -77,11 +77,11 @@ addons:
   apt:
     sources:
       - ubuntu-toolchain-r-test
-#      - llvm-toolchain-precise-3.7
+      - llvm-toolchain-precise-3.7
     packages:
       - gcc-5
       - g++-5
-#      - clang-3.7
+      - clang-3.7
       - binutils-gold
       - libprotobuf-dev
       - protobuf-compiler


[08/16] incubator-quickstep git commit: Query ID added in CreateTable, Index, Delete and DestroyHash

Posted by hb...@apache.org.
Query ID added in CreateTable, Index, Delete and DestroyHash


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

Branch: refs/heads/query-manager-used-in-foreman
Commit: 58d5ccbb46a4e456c2361397f09a7fc33c49bf99
Parents: 06d3501
Author: Harshad Deshmukh <hb...@apache.org>
Authored: Sun Jun 5 23:09:51 2016 -0500
Committer: Harshad Deshmukh <hb...@apache.org>
Committed: Sun Jun 5 23:09:51 2016 -0500

----------------------------------------------------------------------
 query_optimizer/ExecutionGenerator.cpp                  | 11 +++++++----
 relational_operators/CreateIndexOperator.hpp            |  8 ++++++--
 relational_operators/CreateTableOperator.hpp            | 10 +++++++---
 relational_operators/DeleteOperator.cpp                 |  2 ++
 relational_operators/DeleteOperator.hpp                 | 12 +++++++++---
 relational_operators/DestroyHashOperator.cpp            |  5 +++--
 relational_operators/DestroyHashOperator.hpp            |  9 +++++++--
 relational_operators/WorkOrder.hpp                      |  2 +-
 relational_operators/WorkOrderFactory.cpp               |  5 ++++-
 .../tests/HashJoinOperator_unittest.cpp                 | 12 ++++++------
 10 files changed, 52 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/58d5ccbb/query_optimizer/ExecutionGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.cpp b/query_optimizer/ExecutionGenerator.cpp
index f5e47bd..60389cb 100644
--- a/query_optimizer/ExecutionGenerator.cpp
+++ b/query_optimizer/ExecutionGenerator.cpp
@@ -794,8 +794,8 @@ void ExecutionGenerator::convertHashJoin(const P::HashJoinPtr &physical_plan) {
   insert_destination_proto->set_relational_op_index(join_operator_index);
 
   const QueryPlan::DAGNodeIndex destroy_operator_index =
-      execution_plan_->addRelationalOperator(
-          new DestroyHashOperator(join_hash_table_index));
+      execution_plan_->addRelationalOperator(new DestroyHashOperator(
+          join_hash_table_index, query_handle_->query_id()));
 
   if (!build_relation_info->isStoredRelation()) {
     execution_plan_->addDirectDependency(build_operator_index,
@@ -993,6 +993,7 @@ void ExecutionGenerator::convertCreateIndex(
   }
   execution_plan_->addRelationalOperator(new CreateIndexOperator(input_relation,
                                                                  physical_plan->index_name(),
+                                                                 query_handle_->query_id(),
                                                                  std::move(index_description)));
 }
 
@@ -1033,7 +1034,8 @@ void ExecutionGenerator::convertCreateTable(
   }
 
   execution_plan_->addRelationalOperator(
-      new CreateTableOperator(catalog_relation.release(),
+      new CreateTableOperator(query_handle_->query_id(),
+                              catalog_relation.release(),
                               optimizer_context_->catalog_database()));
 }
 
@@ -1075,7 +1077,8 @@ void ExecutionGenerator::convertDeleteTuples(
         execution_plan_->addRelationalOperator(new DeleteOperator(
             *input_relation_info->relation,
             execution_predicate_index,
-            input_relation_info->isStoredRelation()));
+            input_relation_info->isStoredRelation(),
+            query_handle_->query_id()));
     if (!input_relation_info->isStoredRelation()) {
       execution_plan_->addDirectDependency(delete_tuples_index,
                                            input_relation_info->producer_operator_index,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/58d5ccbb/relational_operators/CreateIndexOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/CreateIndexOperator.hpp b/relational_operators/CreateIndexOperator.hpp
index 2bfacc4..cb614ad 100644
--- a/relational_operators/CreateIndexOperator.hpp
+++ b/relational_operators/CreateIndexOperator.hpp
@@ -18,6 +18,7 @@
 #ifndef QUICKSTEP_RELATIONAL_OPERATORS_CREATE_INDEX_OPERATOR_HPP_
 #define QUICKSTEP_RELATIONAL_OPERATORS_CREATE_INDEX_OPERATOR_HPP_
 
+#include <cstddef>
 #include <string>
 
 #include "catalog/CatalogRelation.hpp"
@@ -52,19 +53,22 @@ class CreateIndexOperator : public RelationalOperator {
    *
    * @param relation The relation to create index upon.
    * @param index_name The index to create.
+   * @param The ID of the query to which this operator belongs.
    * @param index_description The index_description associated with this index.
    **/
   CreateIndexOperator(CatalogRelation *relation,
                       const std::string &index_name,
+                      const std::size_t query_id,
                       IndexSubBlockDescription &&index_description)  // NOLINT(whitespace/operators)
-      : relation_(DCHECK_NOTNULL(relation)),
+      : RelationalOperator(query_id),
+        relation_(DCHECK_NOTNULL(relation)),
         index_name_(index_name),
         index_description_(index_description) {}
 
   ~CreateIndexOperator() override {}
 
   /**
-   * @note no WorkOrder generated for this operator.
+   * @note No WorkOrder generated for this operator.
    **/
   bool getAllWorkOrders(WorkOrdersContainer *container,
                         QueryContext *query_context,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/58d5ccbb/relational_operators/CreateTableOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/CreateTableOperator.hpp b/relational_operators/CreateTableOperator.hpp
index 98f3253..1439a2b 100644
--- a/relational_operators/CreateTableOperator.hpp
+++ b/relational_operators/CreateTableOperator.hpp
@@ -18,6 +18,7 @@
 #ifndef QUICKSTEP_RELATIONAL_OPERATORS_CREATE_TABLE_OPERATOR_HPP_
 #define QUICKSTEP_RELATIONAL_OPERATORS_CREATE_TABLE_OPERATOR_HPP_
 
+#include <cstddef>
 #include <memory>
 
 #include "catalog/CatalogRelation.hpp"
@@ -49,20 +50,23 @@ class CreateTableOperator : public RelationalOperator {
   /**
    * @brief Constructor.
    *
+   * @param The ID of the query to which this operator belongs.
    * @param relation The relation to add. This CreateTableOperator owns
    *        relation until the WorkOrder it produces is successfully executed,
    *        at which point it is owned by database.
    * @param database The database to add a relation to.
    **/
-  CreateTableOperator(CatalogRelation *relation,
+  CreateTableOperator(const std::size_t query_id,
+                      CatalogRelation *relation,
                       CatalogDatabase *database)
-      : relation_(DCHECK_NOTNULL(relation)),
+      : RelationalOperator(query_id),
+        relation_(DCHECK_NOTNULL(relation)),
         database_(DCHECK_NOTNULL(database)) {}
 
   ~CreateTableOperator() override {}
 
   /**
-   * @note no WorkOrder generated for this operator.
+   * @note No WorkOrder generated for this operator.
    **/
   bool getAllWorkOrders(WorkOrdersContainer *container,
                         QueryContext *query_context,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/58d5ccbb/relational_operators/DeleteOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/DeleteOperator.cpp b/relational_operators/DeleteOperator.cpp
index 2c2c6de..94169ed 100644
--- a/relational_operators/DeleteOperator.cpp
+++ b/relational_operators/DeleteOperator.cpp
@@ -60,6 +60,7 @@ bool DeleteOperator::getAllWorkOrders(
                                 storage_manager,
                                 op_index_,
                                 scheduler_client_id,
+                                query_id_,
                                 bus),
             op_index_);
       }
@@ -75,6 +76,7 @@ bool DeleteOperator::getAllWorkOrders(
                               storage_manager,
                               op_index_,
                               scheduler_client_id,
+                              query_id_,
                               bus),
           op_index_);
       ++num_workorders_generated_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/58d5ccbb/relational_operators/DeleteOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/DeleteOperator.hpp b/relational_operators/DeleteOperator.hpp
index 1d44552..9b6a44d 100644
--- a/relational_operators/DeleteOperator.hpp
+++ b/relational_operators/DeleteOperator.hpp
@@ -61,11 +61,14 @@ class DeleteOperator : public RelationalOperator {
    *        tuples will be deleted).
    * @param relation_is_stored If relation is a stored relation and is fully
    *        available to the operator before it can start generating workorders.
+   * @param The ID of the query to which this operator belongs.
    **/
   DeleteOperator(const CatalogRelation &relation,
                  const QueryContext::predicate_id predicate_index,
-                 const bool relation_is_stored)
-     :  relation_(relation),
+                 const bool relation_is_stored,
+                 const std::size_t query_id)
+     :  RelationalOperator(query_id),
+        relation_(relation),
         predicate_index_(predicate_index),
         relation_is_stored_(relation_is_stored),
         started_(false),
@@ -127,6 +130,7 @@ class DeleteWorkOrder : public WorkOrder {
    * @param delete_operator_index The index of the Delete Operator in the query
    *        plan DAG.
    * @param scheduler_client_id The TMB client ID of the scheduler thread.
+   * @param The ID of the query to which this operator belongs.
    * @param bus A pointer to the TMB.
    **/
   DeleteWorkOrder(const CatalogRelationSchema &input_relation,
@@ -135,8 +139,10 @@ class DeleteWorkOrder : public WorkOrder {
                   StorageManager *storage_manager,
                   const std::size_t delete_operator_index,
                   const tmb::client_id scheduler_client_id,
+                  const std::size_t query_id,
                   MessageBus *bus)
-      : input_relation_(input_relation),
+      : WorkOrder(query_id),
+        input_relation_(input_relation),
         input_block_id_(input_block_id),
         predicate_(predicate),
         storage_manager_(DCHECK_NOTNULL(storage_manager)),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/58d5ccbb/relational_operators/DestroyHashOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/DestroyHashOperator.cpp b/relational_operators/DestroyHashOperator.cpp
index c2220d5..c92117a 100644
--- a/relational_operators/DestroyHashOperator.cpp
+++ b/relational_operators/DestroyHashOperator.cpp
@@ -32,8 +32,9 @@ bool DestroyHashOperator::getAllWorkOrders(
     tmb::MessageBus *bus) {
   if (blocking_dependencies_met_ && !work_generated_) {
     work_generated_ = true;
-    container->addNormalWorkOrder(new DestroyHashWorkOrder(hash_table_index_, query_context),
-                                  op_index_);
+    container->addNormalWorkOrder(
+        new DestroyHashWorkOrder(hash_table_index_, query_id_, query_context),
+        op_index_);
   }
   return work_generated_;
 }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/58d5ccbb/relational_operators/DestroyHashOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/DestroyHashOperator.hpp b/relational_operators/DestroyHashOperator.hpp
index 46331ba..d799e05 100644
--- a/relational_operators/DestroyHashOperator.hpp
+++ b/relational_operators/DestroyHashOperator.hpp
@@ -47,9 +47,12 @@ class DestroyHashOperator : public RelationalOperator {
    * @brief Constructor.
    *
    * @param hash_table_index The index of the JoinHashTable in QueryContext.
+   * @param The ID of the query to which this operator belongs.
    **/
-  explicit DestroyHashOperator(const QueryContext::join_hash_table_id hash_table_index)
-      : hash_table_index_(hash_table_index),
+  DestroyHashOperator(const QueryContext::join_hash_table_id hash_table_index,
+                      const std::size_t query_id)
+      : RelationalOperator(query_id),
+        hash_table_index_(hash_table_index),
         work_generated_(false) {}
 
   ~DestroyHashOperator() override {}
@@ -76,9 +79,11 @@ class DestroyHashWorkOrder : public WorkOrder {
    * @brief Constructor.
    *
    * @param hash_table_index The index of the JoinHashTable in QueryContext.
+   * @param The ID of the query to which this operator belongs.
    * @param query_context The QueryContext to use.
    **/
   DestroyHashWorkOrder(const QueryContext::join_hash_table_id hash_table_index,
+                       const std::size_t query_id,
                        QueryContext *query_context)
       : hash_table_index_(hash_table_index),
         query_context_(DCHECK_NOTNULL(query_context)) {}

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/58d5ccbb/relational_operators/WorkOrder.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrder.hpp b/relational_operators/WorkOrder.hpp
index b179d8e..5f8ad3f 100644
--- a/relational_operators/WorkOrder.hpp
+++ b/relational_operators/WorkOrder.hpp
@@ -286,7 +286,7 @@ class WorkOrder {
   }
 
  protected:
-  WorkOrder(const std::size_t query_id = 0)
+  explicit WorkOrder(const std::size_t query_id = 0)
       : query_id_(query_id) {}
 
   const std::size_t query_id_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/58d5ccbb/relational_operators/WorkOrderFactory.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrderFactory.cpp b/relational_operators/WorkOrderFactory.cpp
index b91fe2d..9828ab9 100644
--- a/relational_operators/WorkOrderFactory.cpp
+++ b/relational_operators/WorkOrderFactory.cpp
@@ -110,12 +110,15 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
           storage_manager,
           proto.GetExtension(serialization::DeleteWorkOrder::operator_index),
           shiftboss_client_id,
+          proto.query_id(),
           bus);
     }
     case serialization::DESTROY_HASH: {
       LOG(INFO) << "Creating DestroyHashWorkOrder";
       return new DestroyHashWorkOrder(
-          proto.GetExtension(serialization::DestroyHashWorkOrder::join_hash_table_index),
+          proto.GetExtension(
+              serialization::DestroyHashWorkOrder::join_hash_table_index),
+          proto.query_id(),
           query_context);
     }
     case serialization::DROP_TABLE: {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/58d5ccbb/relational_operators/tests/HashJoinOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/HashJoinOperator_unittest.cpp b/relational_operators/tests/HashJoinOperator_unittest.cpp
index dfec228..4ef5a5c 100644
--- a/relational_operators/tests/HashJoinOperator_unittest.cpp
+++ b/relational_operators/tests/HashJoinOperator_unittest.cpp
@@ -423,7 +423,7 @@ TEST_P(HashJoinOperatorTest, LongKeyHashJoinTest) {
   }
 
   // Create cleaner operator.
-  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(join_hash_table_index));
+  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(join_hash_table_index, 0  /* dummy query ID */));
   cleaner->informAllBlockingDependenciesMet();
   fetchAndExecuteWorkOrders(cleaner.get());
 
@@ -592,7 +592,7 @@ TEST_P(HashJoinOperatorTest, IntDuplicateKeyHashJoinTest) {
   }
 
   // Create cleaner operator.
-  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(join_hash_table_index));
+  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(join_hash_table_index, 0  /* dummy query ID */));
   cleaner->informAllBlockingDependenciesMet();
   fetchAndExecuteWorkOrders(cleaner.get());
 
@@ -727,7 +727,7 @@ TEST_P(HashJoinOperatorTest, CharKeyCartesianProductHashJoinTest) {
   }
 
   // Create cleaner operator.
-  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(join_hash_table_index));
+  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(join_hash_table_index, 0  /* dummy query ID */));
   cleaner->informAllBlockingDependenciesMet();
   fetchAndExecuteWorkOrders(cleaner.get());
 
@@ -893,7 +893,7 @@ TEST_P(HashJoinOperatorTest, VarCharDuplicateKeyHashJoinTest) {
   }
 
   // Create the cleaner operator.
-  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(join_hash_table_index));
+  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(join_hash_table_index, 0  /* dummy query ID */));
   cleaner->informAllBlockingDependenciesMet();
   fetchAndExecuteWorkOrders(cleaner.get());
 
@@ -1068,7 +1068,7 @@ TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinTest) {
   }
 
   // Create cleaner operator.
-  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(join_hash_table_index));
+  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(join_hash_table_index, 0  /* dummy query ID */));
   cleaner->informAllBlockingDependenciesMet();
   fetchAndExecuteWorkOrders(cleaner.get());
 
@@ -1254,7 +1254,7 @@ TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinWithResidualPredicateTest) {
   }
 
   // Create cleaner operator.
-  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(join_hash_table_index));
+  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(join_hash_table_index, 0  /* dummy query ID */));
   cleaner->informAllBlockingDependenciesMet();
   fetchAndExecuteWorkOrders(cleaner.get());
 


[13/16] incubator-quickstep git commit: Reordered query ID in aggregation operator

Posted by hb...@apache.org.
Reordered query ID in aggregation operator


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

Branch: refs/heads/query-manager-used-in-foreman
Commit: 15a2114a2c64c05f8a1b23831496fcf606a38d5d
Parents: eb3e73f
Author: Harshad Deshmukh <hb...@apache.org>
Authored: Wed Jun 8 09:14:20 2016 -0500
Committer: Harshad Deshmukh <hb...@apache.org>
Committed: Wed Jun 8 09:14:20 2016 -0500

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


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/15a2114a/query_optimizer/ExecutionGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.cpp b/query_optimizer/ExecutionGenerator.cpp
index 30dfa8e..56625fd 100644
--- a/query_optimizer/ExecutionGenerator.cpp
+++ b/query_optimizer/ExecutionGenerator.cpp
@@ -1429,10 +1429,11 @@ void ExecutionGenerator::convertAggregate(
   const QueryPlan::DAGNodeIndex aggregation_operator_index =
       execution_plan_->addRelationalOperator(
           new AggregationOperator(
+              query_handle_->query_id(),
               *input_relation_info->relation,
               input_relation_info->isStoredRelation(),
-              aggr_state_index,
-              query_handle_->query_id()));
+              aggr_state_index));
+
   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/15a2114a/relational_operators/AggregationOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/AggregationOperator.cpp b/relational_operators/AggregationOperator.cpp
index 94ba901..7252541 100644
--- a/relational_operators/AggregationOperator.cpp
+++ b/relational_operators/AggregationOperator.cpp
@@ -39,8 +39,8 @@ bool AggregationOperator::getAllWorkOrders(
       for (const block_id input_block_id : input_relation_block_ids_) {
         container->addNormalWorkOrder(
             new AggregationWorkOrder(
-                input_block_id,
                 query_id_,
+                input_block_id,
                 query_context->getAggregationState(aggr_state_index_)),
             op_index_);
       }
@@ -51,8 +51,8 @@ bool AggregationOperator::getAllWorkOrders(
     while (num_workorders_generated_ < input_relation_block_ids_.size()) {
       container->addNormalWorkOrder(
           new AggregationWorkOrder(
-              input_relation_block_ids_[num_workorders_generated_],
               query_id_,
+              input_relation_block_ids_[num_workorders_generated_],
               query_context->getAggregationState(aggr_state_index_)),
           op_index_);
       ++num_workorders_generated_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/15a2114a/relational_operators/AggregationOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/AggregationOperator.hpp b/relational_operators/AggregationOperator.hpp
index 0fbc381..f340d4e 100644
--- a/relational_operators/AggregationOperator.hpp
+++ b/relational_operators/AggregationOperator.hpp
@@ -53,17 +53,17 @@ class AggregationOperator : public RelationalOperator {
    * @brief Constructor for aggregating with arbitrary expressions in projection
    *        list.
    *
+   * @param query_id The ID of this query.
    * @param input_relation The relation to perform aggregation over.
    * @param input_relation_is_stored If input_relation is a stored relation and
    *        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,
+  AggregationOperator(const std::size_t query_id,
+                      const CatalogRelation &input_relation,
                       bool input_relation_is_stored,
-                      const QueryContext::aggregation_state_id aggr_state_index,
-                      const std::size_t query_id)
+                      const QueryContext::aggregation_state_id aggr_state_index)
       : RelationalOperator(query_id),
         input_relation_is_stored_(input_relation_is_stored),
         input_relation_block_ids_(input_relation_is_stored ? input_relation.getBlocksSnapshot()
@@ -109,12 +109,12 @@ class AggregationWorkOrder : public WorkOrder {
   /**
    * @brief Constructor
    *
-   * @param input_block_id The block id.
    * @param query_id The ID of this query.
+   * @param input_block_id The block id.
    * @param state The AggregationState to use.
    **/
-  AggregationWorkOrder(const block_id input_block_id,
-                       const std::size_t query_id,
+  AggregationWorkOrder(const std::size_t query_id,
+                       const block_id input_block_id,
                        AggregationOperationState *state)
       : WorkOrder(query_id),
         input_block_id_(input_block_id),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/15a2114a/relational_operators/tests/AggregationOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/AggregationOperator_unittest.cpp b/relational_operators/tests/AggregationOperator_unittest.cpp
index ace7951..5f3a434 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, 0));
+    op_.reset(new AggregationOperator(0, *table_, true, aggr_state_index));
 
     // Setup the InsertDestination proto in the query context proto.
     const QueryContext::insert_destination_id insert_destination_index =
@@ -354,7 +354,7 @@ class AggregationOperatorTest : public ::testing::Test {
         serialization::HashTableImplType::LINEAR_OPEN_ADDRESSING);
 
     // Create Operators.
-    op_.reset(new AggregationOperator(*table_, true, aggr_state_index, 0));
+    op_.reset(new AggregationOperator(0, *table_, true, aggr_state_index));
 
     // Setup the InsertDestination proto in the query context proto.
     const QueryContext::insert_destination_id insert_destination_index =


[16/16] incubator-quickstep git commit: Merge branch 'reorder-query-id-param' into query-manager-used-in-foreman

Posted by hb...@apache.org.
Merge branch 'reorder-query-id-param' into query-manager-used-in-foreman

Conflicts:
	query_execution/Foreman.cpp
	query_execution/QueryManager.cpp
	query_execution/tests/Foreman_unittest.cpp
	relational_operators/RebuildWorkOrder.hpp
	relational_operators/UpdateOperator.cpp
	relational_operators/UpdateOperator.hpp


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

Branch: refs/heads/query-manager-used-in-foreman
Commit: e8ead86103341a34ac7449ed416d1dbba67496a7
Parents: bef0ae1 d67f61e
Author: Harshad Deshmukh <hb...@apache.org>
Authored: Wed Jun 8 15:46:28 2016 -0500
Committer: Harshad Deshmukh <hb...@apache.org>
Committed: Wed Jun 8 15:46:28 2016 -0500

----------------------------------------------------------------------
 cli/QuickstepCli.cpp                            |   1 -
 query_execution/AdmitRequestMessage.hpp         |   2 -
 query_execution/Foreman.cpp                     |   3 -
 query_execution/QueryExecutionUtil.hpp          |   6 +-
 query_execution/QueryManager.cpp                |  14 +-
 query_execution/WorkOrdersContainer.hpp         |  10 +-
 query_execution/tests/Foreman_unittest.cpp      | 945 +++++++++++++++++++
 query_execution/tests/QueryManager_unittest.cpp |   2 +-
 .../tests/WorkOrdersContainer_unittest.cpp      |  18 +-
 query_optimizer/ExecutionGenerator.cpp          | 174 ++--
 .../tests/ExecutionHeuristics_unittest.cpp      |  25 +-
 relational_operators/AggregationOperator.cpp    |  12 +-
 relational_operators/AggregationOperator.hpp    |  14 +-
 relational_operators/BuildHashOperator.cpp      |   4 +-
 relational_operators/BuildHashOperator.hpp      |  21 +-
 relational_operators/CreateIndexOperator.hpp    |  10 +-
 relational_operators/CreateTableOperator.hpp    |  10 +-
 relational_operators/DeleteOperator.cpp         |   8 +-
 relational_operators/DeleteOperator.hpp         |  18 +-
 relational_operators/DestroyHashOperator.cpp    |   5 +-
 relational_operators/DestroyHashOperator.hpp    |  14 +-
 relational_operators/DropTableOperator.cpp      |   3 +-
 relational_operators/DropTableOperator.hpp      |  15 +-
 .../FinalizeAggregationOperator.cpp             |   6 +-
 .../FinalizeAggregationOperator.hpp             |  20 +-
 relational_operators/HashJoinOperator.cpp       |  27 +-
 relational_operators/HashJoinOperator.hpp       | 242 +++--
 relational_operators/InsertOperator.cpp         |   6 +-
 relational_operators/InsertOperator.hpp         |  20 +-
 .../NestedLoopsJoinOperator.cpp                 |  41 +-
 .../NestedLoopsJoinOperator.hpp                 |  54 +-
 relational_operators/RebuildWorkOrder.hpp       |  19 +-
 relational_operators/RelationalOperator.hpp     |   8 +-
 relational_operators/SampleOperator.cpp         |  46 +-
 relational_operators/SampleOperator.hpp         |  31 +-
 relational_operators/SaveBlocksOperator.cpp     |   1 +
 relational_operators/SaveBlocksOperator.hpp     |  14 +-
 relational_operators/SelectOperator.cpp         |  23 +-
 relational_operators/SelectOperator.hpp         |  60 +-
 relational_operators/SortMergeRunOperator.cpp   |   1 +
 relational_operators/SortMergeRunOperator.hpp   |  29 +-
 .../SortRunGenerationOperator.cpp               |   4 +-
 .../SortRunGenerationOperator.hpp               |  28 +-
 relational_operators/TableGeneratorOperator.cpp |   7 +-
 relational_operators/TableGeneratorOperator.hpp |  23 +-
 relational_operators/TextScanOperator.cpp       |  27 +-
 relational_operators/TextScanOperator.hpp       |  34 +-
 relational_operators/UpdateOperator.cpp         |  22 +-
 relational_operators/UpdateOperator.hpp         |  42 +-
 relational_operators/WorkOrder.hpp              |  16 +-
 relational_operators/WorkOrder.proto            |   1 +
 relational_operators/WorkOrderFactory.cpp       |  35 +-
 .../tests/AggregationOperator_unittest.cpp      |  18 +-
 .../tests/HashJoinOperator_unittest.cpp         | 156 +--
 .../tests/SortMergeRunOperator_unittest.cpp     |  11 +-
 .../SortRunGenerationOperator_unittest.cpp      |  16 +-
 .../tests/TextScanOperator_unittest.cpp         |   5 +-
 57 files changed, 1825 insertions(+), 602 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/e8ead861/cli/QuickstepCli.cpp
----------------------------------------------------------------------
diff --cc cli/QuickstepCli.cpp
index 6f954fe,558d6eb..d65eb89
--- a/cli/QuickstepCli.cpp
+++ b/cli/QuickstepCli.cpp
@@@ -402,25 -389,14 +402,24 @@@ int main(int argc, char* argv[]) 
          }
  
          DCHECK(query_handle->getQueryPlanMutable() != nullptr);
 -        foreman.setQueryPlan(query_handle->getQueryPlanMutable()->getQueryPlanDAGMutable());
 -
 -        foreman.reconstructQueryContextFromProto(query_handle->getQueryContextProto());
 -
 +        // TODO(harshad) - In the future when queries are not admitted
 +        // immediately, calculate their waiting time separately.
-         LOG(INFO) << "Address of query handle in QuickstepCli: " << query_handle.get();
 +        start = std::chrono::steady_clock::now();
 +        const tmb::MessageBus::SendStatus send_status =
 +            QueryExecutionUtil::ConstructAndSendAdmitRequestMessage(
 +                main_thread_client_id,
 +                foreman.getBusClientID(),
 +                query_handle.get(),
 +                &bus);
 +        if (send_status != tmb::MessageBus::SendStatus::kOK) {
 +          fprintf(stderr, "\nQuery %s could not be admitted to the system\n", command_string->c_str());
 +          continue;
 +        }
          try {
 -          start = std::chrono::steady_clock::now();
 -          foreman.start();
 -          foreman.join();
 +          const AnnotatedMessage annotated_msg =
 +              bus.Receive(main_thread_client_id, 0, true);
 +          const TaggedMessage &tagged_message = annotated_msg.tagged_message;
 +          DCHECK_EQ(kWorkloadCompletionMessage, tagged_message.message_type());
            end = std::chrono::steady_clock::now();
  
            const CatalogRelation *query_result_relation = query_handle->getQueryResultRelation();

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/e8ead861/query_execution/AdmitRequestMessage.hpp
----------------------------------------------------------------------
diff --cc query_execution/AdmitRequestMessage.hpp
index e2a1077,0000000..e33b354
mode 100644,000000..100644
--- a/query_execution/AdmitRequestMessage.hpp
+++ b/query_execution/AdmitRequestMessage.hpp
@@@ -1,75 -1,0 +1,73 @@@
 +/**
 + *   Copyright 2016, Quickstep Research Group, Computer Sciences Department,
 + *     University of Wisconsin\u2014Madison.
 + *
 + *   Licensed under the Apache License, Version 2.0 (the "License");
 + *   you may not use this file except in compliance with the License.
 + *   You may obtain a copy of the License at
 + *
 + *       http://www.apache.org/licenses/LICENSE-2.0
 + *
 + *   Unless required by applicable law or agreed to in writing, software
 + *   distributed under the License is distributed on an "AS IS" BASIS,
 + *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + *   See the License for the specific language governing permissions and
 + *   limitations under the License.
 + **/
 +
 +#ifndef QUICKSTEP_QUERY_EXECUTION_ADMIT_REQUEST_MESSAGE_HPP_
 +#define QUICKSTEP_QUERY_EXECUTION_ADMIT_REQUEST_MESSAGE_HPP_
 +
 +#include <vector>
 +
 +#include "utility/Macros.hpp"
 +
 +namespace quickstep {
 +
 +class QueryHandle;
 +
 +/** \addtogroup QueryExecution
 + *  @{
 + */
 +
 +/**
 + * @brief A message requesting a query or queries to be admitted to the system.
 + **/
 +class AdmitRequestMessage {
 + public:
 +  /**
 +   * @brief Constructor.
 +   *
 +   * @param query_handles The handles of the queries requesting to be admitted
 +   *        to the system.
 +   **/
 +  explicit AdmitRequestMessage(const std::vector<QueryHandle*> &query_handles)
 +      : query_handles_(query_handles) {}
 +
 +  /**
 +   * @brief Constructor for requesting single query admission.
 +   *
 +   * @param query_handle The handle of the query requesting to be admitted.
 +   **/
 +  explicit AdmitRequestMessage(QueryHandle *query_handle) {
 +    query_handles_.push_back(query_handle);
 +  }
 +
 +  /**
 +   * @brief Get the query handles from this message.
 +   **/
 +  const std::vector<QueryHandle*>& getQueryHandles() const {
-     LOG(INFO) << "Query handle in getQueryHandles(): " << query_handles_.front()
-       << " [0] " << query_handles_[0];
 +    return query_handles_;
 +  }
 +
 + private:
 +  std::vector<QueryHandle*> query_handles_;
 +
 +  DISALLOW_COPY_AND_ASSIGN(AdmitRequestMessage);
 +};
 +
 +/** @} */
 +
 +}  // namespace quickstep
 +
 +#endif  // QUICKSTEP_QUERY_EXECUTION_ADMIT_REQUEST_MESSAGE_HPP_

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/e8ead861/query_execution/Foreman.cpp
----------------------------------------------------------------------
diff --cc query_execution/Foreman.cpp
index 6cec70a,7705819..3609120
--- a/query_execution/Foreman.cpp
+++ b/query_execution/Foreman.cpp
@@@ -97,43 -58,161 +97,40 @@@ void Foreman::run() 
      // We can pin the foreman thread to a CPU if specified.
      ThreadUtil::BindToCPU(cpu_id_);
    }
 -  initializeState();
 -
 -  DEBUG_ASSERT(query_dag_ != nullptr);
 -  const dag_node_index dag_size = query_dag_->size();
 -
 -  // Collect all the workorders from all the relational operators in the DAG.
 -  for (dag_node_index index = 0; index < dag_size; ++index) {
 -    if (checkAllBlockingDependenciesMet(index)) {
 -      query_dag_->getNodePayloadMutable(index)->informAllBlockingDependenciesMet();
 -      processOperator(index, false);
 -    }
 -  }
 -
 -  // Dispatch the WorkOrders generated so far.
 -  dispatchWorkerMessages(0, 0);
 -}
 -
 -void Foreman::processWorkOrderCompleteMessage(const dag_node_index op_index,
 -                                              const size_t worker_thread_index) {
 -  query_exec_state_->decrementNumQueuedWorkOrders(op_index);
 -
 -  // As the given worker finished executing a WorkOrder, decrement its number
 -  // of queued WorkOrders.
 -  workers_->decrementNumQueuedWorkOrders(worker_thread_index);
 -
 -  // Check if new work orders are available and fetch them if so.
 -  fetchNormalWorkOrders(op_index);
 -
 -  if (checkRebuildRequired(op_index)) {
 -    if (checkNormalExecutionOver(op_index)) {
 -      if (!checkRebuildInitiated(op_index)) {
 -        if (initiateRebuild(op_index)) {
 -          // Rebuild initiated and completed right away.
 -          markOperatorFinished(op_index);
 -        } else {
 -          // Rebuild under progress.
 -        }
 -      } else if (checkRebuildOver(op_index)) {
 -        // Rebuild was under progress and now it is over.
 -        markOperatorFinished(op_index);
 -      }
 -    } else {
 -      // Normal execution under progress for this operator.
 -    }
 -  } else if (checkOperatorExecutionOver(op_index)) {
 -    // Rebuild not required for this operator and its normal execution is
 -    // complete.
 -    markOperatorFinished(op_index);
 -  }
 -
 -  for (const pair<dag_node_index, bool> &dependent_link :
 -       query_dag_->getDependents(op_index)) {
 -    const dag_node_index dependent_op_index = dependent_link.first;
 -    if (checkAllBlockingDependenciesMet(dependent_op_index)) {
 -      // Process the dependent operator (of the operator whose WorkOrder
 -      // was just executed) for which all the dependencies have been met.
 -      processOperator(dependent_op_index, true);
 -    }
 -  }
 -
 -  // Dispatch the WorkerMessages to the workers. We prefer to start the search
 -  // for the schedulable WorkOrders beginning from 'op_index'. The first
 -  // candidate worker to receive the next WorkOrder is the one that sent the
 -  // response message to Foreman.
 -  dispatchWorkerMessages(worker_thread_index, op_index);
 -}
 -
 -void Foreman::processRebuildWorkOrderCompleteMessage(const dag_node_index op_index,
 -                                                     const size_t worker_thread_index) {
 -  query_exec_state_->decrementNumRebuildWorkOrders(op_index);
 -  workers_->decrementNumQueuedWorkOrders(worker_thread_index);
 -
 -  if (checkRebuildOver(op_index)) {
 -    markOperatorFinished(op_index);
 -
 -    for (const pair<dag_node_index, bool> &dependent_link :
 -         query_dag_->getDependents(op_index)) {
 -      const dag_node_index dependent_op_index = dependent_link.first;
 -      if (checkAllBlockingDependenciesMet(dependent_op_index)) {
 -        processOperator(dependent_op_index, true);
 -      }
 -    }
 -  }
 -
 -  // Dispatch the WorkerMessages to the workers. We prefer to start the search
 -  // for the schedulable WorkOrders beginning from 'op_index'. The first
 -  // candidate worker to receive the next WorkOrder is the one that sent the
 -  // response message to Foreman.
 -  dispatchWorkerMessages(worker_thread_index, op_index);
 -}
 -
 -void Foreman::processDataPipelineMessage(const dag_node_index op_index,
 -                                         const block_id block,
 -                                         const relation_id rel_id) {
 -  for (const dag_node_index consumer_index :
 -       output_consumers_[op_index]) {
 -    // Feed the streamed block to the consumer. Note that 'output_consumers_'
 -    // only contain those dependents of operator with index = op_index which are
 -    // eligible to receive streamed input.
 -    query_dag_->getNodePayloadMutable(consumer_index)->feedInputBlock(block, rel_id);
 -    // Because of the streamed input just fed, check if there are any new
 -    // WorkOrders available and if so, fetch them.
 -    fetchNormalWorkOrders(consumer_index);
 -  }
 -
 -  // Dispatch the WorkerMessages to the workers. We prefer to start the search
 -  // for the schedulable WorkOrders beginning from 'op_index'. The first
 -  // candidate worker to receive the next WorkOrder is the one that sent the
 -  // response message to Foreman.
 -  // TODO(zuyu): Improve the data locality for the next WorkOrder.
 -  dispatchWorkerMessages(0, op_index);
 -}
 -
 -void Foreman::processFeedbackMessage(const WorkOrder::FeedbackMessage &msg) {
 -  RelationalOperator *op =
 -      query_dag_->getNodePayloadMutable(msg.header().rel_op_index);
 -  op->receiveFeedbackMessage(msg);
 -}
 -
 -void Foreman::run() {
 -  // Initialize before for Foreman eventloop.
 -  initialize();
  
    // Event loop
 -  while (!query_exec_state_->hasQueryExecutionFinished()) {
 +  for (;;) {
      // Receive() causes this thread to sleep until next message is received.
 -    AnnotatedMessage annotated_msg = bus_->Receive(foreman_client_id_, 0, true);
 +    const AnnotatedMessage annotated_msg =
 +        bus_->Receive(foreman_client_id_, 0, true);
      const TaggedMessage &tagged_message = annotated_msg.tagged_message;
 -    switch (tagged_message.message_type()) {
 -      case kWorkOrderCompleteMessage: {
 -        serialization::WorkOrderCompletionMessage proto;
 -        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
 -
 -        processWorkOrderCompleteMessage(proto.operator_index(), proto.worker_thread_index());
 -        break;
 -      }
 -      case kRebuildWorkOrderCompleteMessage: {
 -        serialization::WorkOrderCompletionMessage proto;
 -        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
 -
 -        processRebuildWorkOrderCompleteMessage(proto.operator_index(), proto.worker_thread_index());
 +    const tmb::message_type_id message_type = tagged_message.message_type();
 +    switch (message_type) {
 +      case kCatalogRelationNewBlockMessage:  // Fall through
 +      case kDataPipelineMessage:
 +      case kRebuildWorkOrderCompleteMessage:
 +      case kWorkOrderCompleteMessage:
 +      case kWorkOrderFeedbackMessage:
 +      case kWorkOrdersAvailableMessage: {
 +        policy_enforcer_->processMessage(tagged_message);
          break;
        }
 -      case kCatalogRelationNewBlockMessage: {
 -        serialization::CatalogRelationNewBlockMessage proto;
 -        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
 -
 -        const block_id block = proto.block_id();
 -
 -        CatalogRelation *relation =
 -            static_cast<CatalogDatabase*>(catalog_database_)->getRelationByIdMutable(proto.relation_id());
 -        relation->addBlock(block);
 -
 -        if (proto.has_partition_id()) {
 -          relation->getPartitionSchemeMutable()->addBlockToPartition(proto.partition_id(), block);
 +      case kAdmitRequestMessage: {
 +        const AdmitRequestMessage *msg =
 +            static_cast<const AdmitRequestMessage *>(tagged_message.message());
 +        const vector<QueryHandle *> &query_handles = msg->getQueryHandles();
 +
-         LOG(INFO) << "Address of query handle in foreman front: " << query_handles.front() <<
-             " [0]: " << query_handles[0];
 +        DCHECK(!query_handles.empty());
 +        bool all_queries_admitted = true;
 +        if (query_handles.size() == 1u) {
-           LOG(INFO) << "Address of query handle in foreman: " << query_handles.front();
 +          all_queries_admitted =
 +              policy_enforcer_->admitQuery(query_handles.front());
 +        } else {
 +          all_queries_admitted = policy_enforcer_->admitQueries(query_handles);
 +        }
 +        if (!all_queries_admitted) {
 +          LOG(WARNING) << "The scheduler could not admit all the queries";
 +          // TODO(harshad) - Inform the main thread about the failure.
          }
          break;
        }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/e8ead861/query_execution/QueryExecutionUtil.hpp
----------------------------------------------------------------------
diff --cc query_execution/QueryExecutionUtil.hpp
index 267bbe6,a8b6a38..50f277e
--- a/query_execution/QueryExecutionUtil.hpp
+++ b/query_execution/QueryExecutionUtil.hpp
@@@ -65,67 -60,6 +65,65 @@@ class QueryExecutionUtil 
                       std::move(tagged_message));
    }
  
 +  /**
 +   * @brief Construct and send an AdmitRequestMessage from a given sender to a
 +   *        given recipient.
 +   *
 +   * @param sender_id The TMB client ID of the sender.
 +   * @param receiver_id The TMB client ID of the receiver.
 +   * @param query_handle The QueryHandle used in the AdmitRequestMessage.
 +   * @param bus A pointer to the TMB.
 +   * @param tagged_message A moved from reference to the tagged message.
 +   *
 +   * @return A status code indicating the result of the message delivery.
 +   *         The caller should ensure that the status is SendStatus::kOK.
 +   **/
 +  static tmb::MessageBus::SendStatus ConstructAndSendAdmitRequestMessage(
 +      const tmb::client_id sender_id,
 +      const tmb::client_id receiver_id,
 +      QueryHandle *query_handle,
 +      MessageBus *bus) {
-     LOG(INFO) << "Address of query handle in QExecUtil: " << query_handle;
-     std::unique_ptr<AdmitRequestMessage> request_message(new AdmitRequestMessage(query_handle));
-     const std::vector<QueryHandle *> &query_handles = request_message->getQueryHandles();
-       LOG(INFO) << "Address of query handle in foreman front: " << query_handles.front() << " [0]: " << query_handles[0];
++    std::unique_ptr<AdmitRequestMessage> request_message(
++        new AdmitRequestMessage(query_handle));
 +    const std::size_t size_of_request_msg = sizeof(*request_message);
 +    TaggedMessage admit_tagged_message(
 +        request_message.release(), size_of_request_msg, kAdmitRequestMessage);
 +
 +    return QueryExecutionUtil::SendTMBMessage(
 +        bus, sender_id, receiver_id, std::move(admit_tagged_message));
 +  }
 +
 +  /**
 +   * @brief Broadcast a poison message from a given sender.
 +   *
 +   * @note This message will be received by all the clients that have registered
 +   *       as recipients of the poison message type.
 +   *
 +   * @param sender_id The TMB client ID of the sender.
 +   * @param bus A pointer to the TMB.
 +   **/
 +  static void BroadcastPoisonMessage(const tmb::client_id sender_id,
 +                                     tmb::MessageBus *bus) {
 +    // Terminate all threads.
 +    // The sender thread broadcasts poison message to the workers and foreman.
 +    // Each worker dies after receiving poison message. The order of workers'
 +    // death is irrelavant.
 +    MessageStyle style;
 +    style.Broadcast(true);
 +    Address address;
 +    address.All(true);
 +    std::unique_ptr<WorkerMessage> poison_message(WorkerMessage::PoisonMessage());
 +    TaggedMessage poison_tagged_message(poison_message.get(),
 +                                        sizeof(*poison_message),
 +                                        kPoisonMessage);
 +
 +    const tmb::MessageBus::SendStatus send_status = bus->Send(
 +        sender_id, address, style, std::move(poison_tagged_message));
 +    CHECK(send_status == tmb::MessageBus::SendStatus::kOK) <<
 +       "Broadcast poison message from sender with TMB client ID " << sender_id
 +       << " failed";
 +  }
 +
   private:
    /**
     * @brief Constructor. Made private to avoid instantiation.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/e8ead861/query_execution/WorkOrdersContainer.hpp
----------------------------------------------------------------------
diff --cc query_execution/WorkOrdersContainer.hpp
index a1c4288,eb9aedd..3b93729
--- a/query_execution/WorkOrdersContainer.hpp
+++ b/query_execution/WorkOrdersContainer.hpp
@@@ -48,14 -46,10 +48,11 @@@ class WorkOrdersContainer 
     *
     * @param num_operators Number of operators in the query DAG.
     * @param num_numa_nodes Number of NUMA nodes in the system.
-    * @param query_id The ID of the query.
     **/
    WorkOrdersContainer(const std::size_t num_operators,
-                       const std::size_t num_numa_nodes,
-                       const std::size_t query_id)
+                       const std::size_t num_numa_nodes)
 -      : num_operators_(num_operators), num_numa_nodes_(num_numa_nodes) {
 +      : num_operators_(num_operators),
-         num_numa_nodes_(num_numa_nodes),
-         query_id_(query_id) {
++        num_numa_nodes_(num_numa_nodes) {
      DEBUG_ASSERT(num_operators != 0);
      for (std::size_t op = 0; op < num_operators; ++op) {
        normal_workorders_.push_back(
@@@ -226,9 -220,8 +223,8 @@@
     * @param operator_index The index of the operator in the query DAG.
     **/
    void addNormalWorkOrder(WorkOrder *workorder, const std::size_t operator_index) {
 -    DEBUG_ASSERT(workorder != nullptr);
 -    DEBUG_ASSERT(operator_index < num_operators_);
 +    DCHECK(workorder != nullptr);
 +    DCHECK(operator_index < num_operators_);
-     workorder->setQueryID(query_id_);
      normal_workorders_[operator_index].addWorkOrder(workorder);
    }
  
@@@ -245,9 -238,8 +241,8 @@@
     **/
    void addRebuildWorkOrder(WorkOrder *workorder,
                             const std::size_t operator_index) {
 -    DEBUG_ASSERT(workorder != nullptr);
 -    DEBUG_ASSERT(operator_index < num_operators_);
 +    DCHECK(workorder != nullptr);
 +    DCHECK(operator_index < num_operators_);
-     workorder->setQueryID(query_id_);
      rebuild_workorders_[operator_index].addWorkOrder(workorder);
    }
  

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/e8ead861/query_execution/tests/QueryManager_unittest.cpp
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/e8ead861/query_execution/tests/WorkOrdersContainer_unittest.cpp
----------------------------------------------------------------------
diff --cc query_execution/tests/WorkOrdersContainer_unittest.cpp
index 865f01f,cf133c4..cb583ab
--- a/query_execution/tests/WorkOrdersContainer_unittest.cpp
+++ b/query_execution/tests/WorkOrdersContainer_unittest.cpp
@@@ -72,8 -72,7 +72,8 @@@ TEST(WorkOrdersContainerTest, ZeroNUMAN
    // they get inserted and retrieved correctly.
    std::vector<int> numa_node_ids;
    // A container for one operator and no NUMA nodes.
 +  const std::size_t query_id = 0;
-   WorkOrdersContainer w(1, 0, query_id);
+   WorkOrdersContainer w(1, 0);
  
    EXPECT_EQ(0u, w.getNumNormalWorkOrders(0));
    EXPECT_EQ(0u, w.getNumRebuildWorkOrders(0));
@@@ -128,8 -123,7 +128,8 @@@ TEST(WorkOrdersContainerTest, ZeroNUMAN
    // if they get inserted and retrieved correctly and the order of retrieval.
    // A container for one operator and no NUMA nodes.
    std::vector<int> numa_node_ids;
 +  const std::size_t query_id = 0;
-   WorkOrdersContainer w(1, 0, query_id);
+   WorkOrdersContainer w(1, 0);
  
    EXPECT_EQ(0u, w.getNumNormalWorkOrders(0));
    EXPECT_EQ(0u, w.getNumRebuildWorkOrders(0));
@@@ -198,8 -190,7 +198,8 @@@ TEST(WorkOrdersContainerTest, MultipleN
    const std::size_t kNUMANodesUsed = numa_node_ids.size();
  
    // A container for one operator and kNUMANodes.
 +  const std::size_t query_id = 0;
-   WorkOrdersContainer w(1, kNUMANodes, query_id);
+   WorkOrdersContainer w(1, kNUMANodes);
  
    for (std::size_t i = 0; i < kNUMANodesUsed; ++i) {
      std::vector<int> curr_numa_node;
@@@ -303,8 -291,7 +303,8 @@@ TEST(WorkOrdersContainerTest, AllTypesW
    const std::size_t kNUMANodesUsed = numa_nodes.size();
  
    // Create the container.
 +  const std::size_t query_id = 0;
-   WorkOrdersContainer w(1, kNUMANodes, query_id);
+   WorkOrdersContainer w(1, kNUMANodes);
  
    w.addNormalWorkOrder(&multiple_numa_work_order, 0);
  
@@@ -443,8 -427,7 +443,8 @@@ TEST(WorkOrdersContainerTest, MultipleO
    const std::size_t kNUMANodes = numa_node_ids.size();
  
    // Create the container.
 +  const std::size_t query_id = 0;
-   WorkOrdersContainer w(kNumOperators, kNUMANodes, query_id);
+   WorkOrdersContainer w(kNumOperators, kNUMANodes);
  
    std::vector<std::size_t> operator_ids;
    for (std::size_t i = 0; i < kNumOperators; ++i) {
@@@ -640,8 -620,7 +640,8 @@@ TEST(WorkOrdersContainerTest, MultipleO
    const std::size_t kNUMANodes = numa_node_ids.size();
  
    // Create the container.
 +  const std::size_t query_id = 0;
-   WorkOrdersContainer w(kNumOperators, kNUMANodes, query_id);
+   WorkOrdersContainer w(kNumOperators, kNUMANodes);
  
    std::vector<std::size_t> operator_ids;
    for (std::size_t i = 0; i < kNumOperators; ++i) {
@@@ -796,8 -772,7 +796,8 @@@ TEST(WorkOrdersContainerTest, Retrieval
    numa_node_ids.push_back(0);
    const std::size_t kNumWorkOrdersPerType = 100;
  
 +  const std::size_t query_id = 0;
-   WorkOrdersContainer w(1, 2, query_id);
+   WorkOrdersContainer w(1, 2);
  
    std::vector<int> single_numa_node_workorder_ids;
    std::vector<int> multiple_numa_node_workorder_ids;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/e8ead861/query_optimizer/ExecutionGenerator.cpp
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/e8ead861/relational_operators/DeleteOperator.cpp
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/e8ead861/relational_operators/DeleteOperator.hpp
----------------------------------------------------------------------
diff --cc relational_operators/DeleteOperator.hpp
index a239f42,c55f585..fdc9b00
--- a/relational_operators/DeleteOperator.hpp
+++ b/relational_operators/DeleteOperator.hpp
@@@ -159,8 -162,6 +162,7 @@@ class DeleteWorkOrder : public WorkOrde
    StorageManager *storage_manager_;
  
    const std::size_t delete_operator_index_;
-   const std::size_t query_id_;
 +
    const tmb::client_id scheduler_client_id_;
    MessageBus *bus_;
  

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/e8ead861/relational_operators/RebuildWorkOrder.hpp
----------------------------------------------------------------------
diff --cc relational_operators/RebuildWorkOrder.hpp
index fef2cc9,86f8eaf..d11fe7d
--- a/relational_operators/RebuildWorkOrder.hpp
+++ b/relational_operators/RebuildWorkOrder.hpp
@@@ -55,16 -56,17 +56,18 @@@ class RebuildWorkOrder : public WorkOrd
     * @param input_relation_id The ID of the CatalogRelation to which the given
     *        storage block belongs to.
     * @param scheduler_client_id The TMB client ID of the scheduler thread.
 +   * @param query_id The ID of the query.
     * @param bus A pointer to the TMB.
     **/
-   RebuildWorkOrder(MutableBlockReference &&block_ref,
-                    const std::size_t input_operator_index,
-                    const relation_id input_relation_id,
-                    const client_id scheduler_client_id,
-                    const std::size_t query_id,
-                    MessageBus *bus)
-       : block_ref_(std::move(block_ref)),
+   RebuildWorkOrder(
+       const std::size_t query_id,
+       MutableBlockReference &&block_ref,  // NOLINT(whitespace/operators)
+       const std::size_t input_operator_index,
+       const relation_id input_relation_id,
+       const client_id scheduler_client_id,
+       MessageBus *bus)
+       : WorkOrder(query_id),
+         block_ref_(std::move(block_ref)),
          input_operator_index_(input_operator_index),
          input_relation_id_(input_relation_id),
          scheduler_client_id_(scheduler_client_id),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/e8ead861/relational_operators/SortMergeRunOperator.cpp
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/e8ead861/relational_operators/TextScanOperator.cpp
----------------------------------------------------------------------
diff --cc relational_operators/TextScanOperator.cpp
index d1f1932,5acecbf..3899af4
--- a/relational_operators/TextScanOperator.cpp
+++ b/relational_operators/TextScanOperator.cpp
@@@ -604,7 -609,6 +609,7 @@@ void TextSplitWorkOrder::execute() 
  
    // Notify the operator about the completion of this Work Order.
    FeedbackMessage msg(TextScanOperator::kSplitWorkOrderCompletionMessage,
-                       getQueryID(),
++                      query_id_,
                        operator_index_,
                        nullptr /* payload */,
                        0 /* payload_size */,
@@@ -666,7 -670,6 +671,7 @@@ void TextSplitWorkOrder::sendBlobInfoTo
  
    const tmb::client_id worker_thread_client_id = ClientIDMap::Instance()->getValue();
    FeedbackMessage feedback_msg(TextScanOperator::kNewTextBlobMessage,
-                                getQueryID(),
++                               query_id_,
                                 operator_index_,
                                 payload,
                                 payload_size);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/e8ead861/relational_operators/TextScanOperator.hpp
----------------------------------------------------------------------
diff --cc relational_operators/TextScanOperator.hpp
index f87b530,3cda65b..4fd5c04
--- a/relational_operators/TextScanOperator.hpp
+++ b/relational_operators/TextScanOperator.hpp
@@@ -372,8 -381,6 +381,7 @@@ class TextSplitWorkOrder : public WorkO
    StorageManager *storage_manager_;
  
    const std::size_t operator_index_;  // Opeartor index.
-   const std::size_t query_id_;  // query ID.
 +
    const tmb::client_id scheduler_client_id_;  // The scheduler's TMB client ID.
    MessageBus *bus_;
  

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/e8ead861/relational_operators/UpdateOperator.cpp
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/e8ead861/relational_operators/UpdateOperator.hpp
----------------------------------------------------------------------
diff --cc relational_operators/UpdateOperator.hpp
index 9673229,cebb9b5..b4f9b9d
--- a/relational_operators/UpdateOperator.hpp
+++ b/relational_operators/UpdateOperator.hpp
@@@ -174,8 -181,6 +181,7 @@@ class UpdateWorkOrder : public WorkOrde
    StorageManager *storage_manager_;
  
    const std::size_t update_operator_index_;
-   const std::size_t query_id_;
 +
    const tmb::client_id scheduler_client_id_;
    MessageBus *bus_;
  

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/e8ead861/relational_operators/WorkOrder.hpp
----------------------------------------------------------------------
diff --cc relational_operators/WorkOrder.hpp
index fd4b0f1,059865d..df195cc
--- a/relational_operators/WorkOrder.hpp
+++ b/relational_operators/WorkOrder.hpp
@@@ -292,25 -285,16 +292,23 @@@ class WorkOrder 
          " receiver thread with TMB client ID " << receiver_id;
    }
  
 +  /**
 +   * @brief Get the ID of the query which this WorkOder belongs to.
 +   **/
 +  inline const std::size_t getQueryID() const {
 +    return query_id_;
 +  }
 +
+  protected:
    /**
-    * @brief Set the ID of the query which the WorkOrder belongs to.
+    * @brief Constructor.
     *
-    * @param query_id The query ID.
+    * @param query_id The ID of the query to which this WorkOrder belongs.
     **/
-   void setQueryID(const std::size_t query_id) {
-     query_id_ = query_id;
-   }
- 
-  protected:
-   WorkOrder() {}
+   explicit WorkOrder(const std::size_t query_id)
+       : query_id_(query_id) {}
  
+   const std::size_t query_id_;
    // A vector of preferred NUMA node IDs where this workorder should be executed.
    // These node IDs typically indicate the NUMA node IDs of the input(s) of the
    // workorder. Derived classes should ensure that there are no duplicate entries

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/e8ead861/relational_operators/tests/AggregationOperator_unittest.cpp
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/e8ead861/relational_operators/tests/HashJoinOperator_unittest.cpp
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/e8ead861/relational_operators/tests/SortMergeRunOperator_unittest.cpp
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/e8ead861/relational_operators/tests/SortRunGenerationOperator_unittest.cpp
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/e8ead861/relational_operators/tests/TextScanOperator_unittest.cpp
----------------------------------------------------------------------


[06/16] incubator-quickstep git commit: Merge branch 'master' into query-id-operator-workorder

Posted by hb...@apache.org.
Merge branch 'master' into query-id-operator-workorder


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

Branch: refs/heads/query-manager-used-in-foreman
Commit: 0712707724f0632fc076cfbd69fffc5aae8aecdd
Parents: d2841af bf9150e
Author: Harshad Deshmukh <hb...@apache.org>
Authored: Sat Jun 4 17:55:40 2016 -0500
Committer: Harshad Deshmukh <hb...@apache.org>
Committed: Sat Jun 4 17:55:40 2016 -0500

----------------------------------------------------------------------
 .travis.yml                                     |    4 +-
 CMakeLists.txt                                  |   17 +-
 DISCLAIMER                                      |   11 +
 README.md                                       |   25 +-
 catalog/CMakeLists.txt                          |   11 +
 catalog/Catalog.proto                           |   13 +-
 catalog/CatalogRelation.cpp                     |   11 +
 catalog/CatalogRelation.hpp                     |   43 +-
 catalog/CatalogRelationStatistics.cpp           |   49 +
 catalog/CatalogRelationStatistics.hpp           |  122 +
 cli/CMakeLists.txt                              |   23 +-
 cli/CommandExecutor.cpp                         |  171 +-
 cli/CommandExecutor.hpp                         |   21 +-
 cli/DefaultsConfigurator.hpp                    |   16 +
 cli/InputParserUtil.cpp                         |    1 +
 cli/PrintToScreen.cpp                           |   33 +
 cli/PrintToScreen.hpp                           |   31 +
 cli/QuickstepCli.cpp                            |   34 +-
 cli/tests/CommandExecutorTestRunner.cpp         |    3 +
 cli/tests/command_executor/D.test               |   35 +-
 cli/tests/command_executor/Dt.test              |   36 +-
 .../aggregation/AggregationConcreteHandle.hpp   |  105 +
 expressions/aggregation/AggregationHandle.hpp   |   15 +-
 .../aggregation/AggregationHandleAvg.cpp        |    9 +
 .../aggregation/AggregationHandleAvg.hpp        |    4 +
 .../aggregation/AggregationHandleCount.cpp      |   11 +
 .../aggregation/AggregationHandleCount.hpp      |    4 +
 .../aggregation/AggregationHandleDistinct.hpp   |    7 +
 .../aggregation/AggregationHandleMax.cpp        |    9 +
 .../aggregation/AggregationHandleMax.hpp        |    4 +
 .../aggregation/AggregationHandleMin.cpp        |    9 +
 .../aggregation/AggregationHandleMin.hpp        |    4 +
 .../aggregation/AggregationHandleSum.cpp        |    9 +
 .../aggregation/AggregationHandleSum.hpp        |    4 +
 expressions/aggregation/CMakeLists.txt          |    2 +
 .../tests/AggregationHandleAvg_unittest.cpp     |  109 +
 .../tests/AggregationHandleCount_unittest.cpp   |  126 +-
 .../tests/AggregationHandleMax_unittest.cpp     |  122 +
 .../tests/AggregationHandleMin_unittest.cpp     |  121 +
 .../tests/AggregationHandleSum_unittest.cpp     |  124 +
 parser/ParseBasicExpressions.cpp                |   31 +
 parser/ParseBasicExpressions.hpp                |   77 +
 parser/ParseExpression.hpp                      |    1 +
 parser/SqlLexer.lpp                             |    2 +
 parser/SqlParser.ypp                            |   16 +
 parser/preprocessed/SqlLexer_gen.cpp            | 1173 +++----
 parser/preprocessed/SqlLexer_gen.hpp            |    2 +-
 parser/preprocessed/SqlParser_gen.cpp           | 2920 +++++++++---------
 parser/preprocessed/SqlParser_gen.hpp           |  126 +-
 query_execution/BlockLocator.cpp                |  223 ++
 query_execution/BlockLocator.hpp                |  125 +
 query_execution/CMakeLists.txt                  |   50 +
 query_execution/QueryContext.cpp                |   16 +-
 query_execution/QueryContext.hpp                |   57 +-
 query_execution/QueryContext.proto              |   18 +-
 query_execution/QueryExecutionMessages.proto    |   34 +
 query_execution/QueryExecutionTypedefs.hpp      |   16 +
 query_execution/tests/BlockLocator_unittest.cpp |  270 ++
 query_optimizer/CMakeLists.txt                  |   16 +-
 query_optimizer/ExecutionGenerator.cpp          |  126 +-
 query_optimizer/ExecutionGenerator.hpp          |    5 +-
 query_optimizer/ExecutionHeuristics.cpp         |  127 +
 query_optimizer/ExecutionHeuristics.hpp         |  155 +
 query_optimizer/PhysicalGenerator.cpp           |   25 +
 query_optimizer/QueryProcessor.hpp              |   10 +
 query_optimizer/cost_model/CMakeLists.txt       |   32 +-
 .../cost_model/StarSchemaSimpleCostModel.cpp    |  258 ++
 .../cost_model/StarSchemaSimpleCostModel.hpp    |  115 +
 query_optimizer/physical/Aggregate.cpp          |   14 +
 query_optimizer/resolver/CMakeLists.txt         |    1 +
 query_optimizer/resolver/Resolver.cpp           |   32 +
 query_optimizer/rules/CMakeLists.txt            |   20 +-
 .../StarSchemaHashJoinOrderOptimization.cpp     |  309 ++
 .../StarSchemaHashJoinOrderOptimization.hpp     |  136 +
 query_optimizer/tests/CMakeLists.txt            |   17 +
 .../tests/ExecutionHeuristics_unittest.cpp      |  301 ++
 query_optimizer/tests/OptimizerTextTest.cpp     |   14 +
 .../tests/execution_generator/Select.test       |   56 +-
 query_optimizer/tests/resolver/Select.test      |   78 +
 relational_operators/HashJoinOperator.cpp       |    2 +-
 storage/AggregationOperationState.cpp           |  112 +-
 storage/AggregationOperationState.hpp           |    9 +
 storage/AggregationOperationState.proto         |    5 +
 storage/CMakeLists.txt                          |  117 +-
 storage/DataExchange.proto                      |   31 +
 storage/DataExchangerAsync.cpp                  |  165 +
 storage/DataExchangerAsync.hpp                  |   97 +
 storage/FileManager.hpp                         |    9 +-
 storage/FileManagerHdfs.cpp                     |   57 +-
 storage/FileManagerPosix.cpp                    |   55 +-
 storage/FileManagerPosix.hpp                    |    6 +-
 storage/FileManagerWindows.cpp                  |   53 +-
 storage/FileManagerWindows.hpp                  |    4 +-
 storage/HashTable.hpp                           |  104 +
 storage/HashTable.proto                         |    8 +
 storage/HashTableFactory.hpp                    |   44 +-
 storage/HashTablePool.hpp                       |  166 +
 storage/PreloaderThread.cpp                     |   98 +-
 storage/PreloaderThread.hpp                     |   30 +
 storage/SMAIndexSubBlock.cpp                    |   48 +-
 storage/SMAIndexSubBlock.hpp                    |    3 +-
 .../SimpleScalarSeparateChainingHashTable.hpp   |   10 +-
 storage/StorageBlock.cpp                        |    5 +
 storage/StorageBlock.hpp                        |   11 +-
 storage/StorageConstants.hpp                    |    4 +-
 storage/StorageManager.cpp                      |  312 +-
 storage/StorageManager.hpp                      |  135 +-
 storage/tests/DataExchange_unittest.cpp         |  240 ++
 storage/tests/SMAIndexSubBlock_unittest.cpp     |   52 +
 third_party/iwyu/iwyu_helper.py                 |    3 +-
 transaction/AccessMode.hpp                      |   76 +-
 transaction/CMakeLists.txt                      |   59 +-
 transaction/CycleDetector.cpp                   |  120 +
 transaction/CycleDetector.hpp                   |   83 +
 transaction/DeadLockDetector.cpp                |  177 ++
 transaction/DeadLockDetector.hpp                |  156 +
 transaction/DirectedGraph.hpp                   |   56 +-
 transaction/LockManager.cpp                     |  237 ++
 transaction/LockManager.hpp                     |  128 +
 transaction/LockTable.cpp                       |   22 +-
 transaction/LockTable.hpp                       |   36 +-
 transaction/ResourceId.hpp                      |    9 +-
 transaction/StronglyConnectedComponents.cpp     |    1 -
 transaction/Transaction.cpp                     |   48 -
 transaction/TransactionTable.cpp                |    8 +-
 transaction/TransactionTable.hpp                |   11 +-
 transaction/tests/AccessMode_unittest.cpp       |   12 +-
 transaction/tests/CycleDetector_unittest.cpp    |  157 +
 transaction/tests/DeadLockDetector_unittest.cpp |   96 +
 transaction/tests/DirectedGraph_unittest.cpp    |  113 +-
 transaction/tests/LockRequest_unittest.cpp      |    4 +-
 transaction/tests/LockTable_unittest.cpp        |   63 +-
 transaction/tests/Lock_unittest.cpp             |   13 +-
 .../StronglyConnectedComponents_unittest.cpp    |   15 +-
 transaction/tests/TransactionTable_unittest.cpp |  102 +-
 types/operations/Operation.proto                |    9 +
 .../operations/unary_operations/CMakeLists.txt  |   22 +
 .../unary_operations/SubstringOperation.cpp     |  214 ++
 .../unary_operations/SubstringOperation.hpp     |  234 ++
 .../unary_operations/UnaryOperation.cpp         |    4 +
 .../unary_operations/UnaryOperationFactory.cpp  |   12 +
 .../unary_operations/UnaryOperationID.cpp       |    6 +-
 .../unary_operations/UnaryOperationID.hpp       |    3 +
 utility/BloomFilter.hpp                         |  198 +-
 utility/BloomFilter.proto                       |   30 +
 utility/CMakeLists.txt                          |   28 +
 utility/PlanVisualizer.cpp                      |  161 +
 utility/PlanVisualizer.hpp                      |   94 +
 utility/ScopedBuffer.hpp                        |   19 +-
 utility/StringUtil.cpp                          |   14 +
 utility/StringUtil.hpp                          |   48 +-
 validate_cmakelists.py                          |    5 +-
 152 files changed, 10723 insertions(+), 2687 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/07127077/query_optimizer/ExecutionGenerator.cpp
----------------------------------------------------------------------


[10/16] incubator-quickstep git commit: Doxygen and minor bug fixes.

Posted by hb...@apache.org.
Doxygen and minor bug fixes.


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

Branch: refs/heads/query-manager-used-in-foreman
Commit: 305537904824d01cf49211c5d82f9f70bcec1151
Parents: 1290dd7
Author: Harshad Deshmukh <hb...@apache.org>
Authored: Mon Jun 6 15:15:31 2016 -0500
Committer: Harshad Deshmukh <hb...@apache.org>
Committed: Mon Jun 6 15:15:31 2016 -0500

----------------------------------------------------------------------
 relational_operators/AggregationOperator.hpp         |  1 +
 relational_operators/BuildHashOperator.hpp           |  2 +-
 relational_operators/CreateIndexOperator.hpp         |  2 +-
 relational_operators/CreateTableOperator.hpp         |  2 +-
 relational_operators/DeleteOperator.hpp              |  4 ++--
 relational_operators/DestroyHashOperator.hpp         |  7 ++++---
 relational_operators/DropTableOperator.hpp           |  4 ++--
 relational_operators/FinalizeAggregationOperator.hpp |  4 ++--
 relational_operators/HashJoinOperator.hpp            | 11 +++++++++--
 relational_operators/InsertOperator.hpp              |  4 ++--
 relational_operators/NestedLoopsJoinOperator.hpp     |  4 ++--
 relational_operators/RebuildWorkOrder.hpp            |  2 +-
 relational_operators/RelationalOperator.hpp          |  1 +
 relational_operators/SampleOperator.hpp              |  2 +-
 relational_operators/SaveBlocksOperator.hpp          |  3 ++-
 relational_operators/SortRunGenerationOperator.hpp   |  2 +-
 relational_operators/TableGeneratorOperator.hpp      |  2 +-
 relational_operators/TextScanOperator.hpp            |  6 +++---
 relational_operators/UpdateOperator.hpp              |  2 +-
 relational_operators/WorkOrder.hpp                   |  5 +++++
 20 files changed, 43 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/30553790/relational_operators/AggregationOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/AggregationOperator.hpp b/relational_operators/AggregationOperator.hpp
index 2b2a954..0fbc381 100644
--- a/relational_operators/AggregationOperator.hpp
+++ b/relational_operators/AggregationOperator.hpp
@@ -110,6 +110,7 @@ class AggregationWorkOrder : public WorkOrder {
    * @brief Constructor
    *
    * @param input_block_id The block id.
+   * @param query_id The ID of this query.
    * @param state The AggregationState to use.
    **/
   AggregationWorkOrder(const block_id input_block_id,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/30553790/relational_operators/BuildHashOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/BuildHashOperator.hpp b/relational_operators/BuildHashOperator.hpp
index 973a214..5a46d8b 100644
--- a/relational_operators/BuildHashOperator.hpp
+++ b/relational_operators/BuildHashOperator.hpp
@@ -69,7 +69,7 @@ class BuildHashOperator : public RelationalOperator {
    * @param hash_table_index The index of the JoinHashTable in QueryContext.
    *        The HashTable's key Type(s) should be the Type(s) of the
    *        join_key_attributes in input_relation.
-   * @param The ID of the query to which this operator belongs.
+   * @param query_id The ID of the query to which this operator belongs.
    **/
   BuildHashOperator(const CatalogRelation &input_relation,
                     const bool input_relation_is_stored,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/30553790/relational_operators/CreateIndexOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/CreateIndexOperator.hpp b/relational_operators/CreateIndexOperator.hpp
index cb614ad..ede3f02 100644
--- a/relational_operators/CreateIndexOperator.hpp
+++ b/relational_operators/CreateIndexOperator.hpp
@@ -53,7 +53,7 @@ class CreateIndexOperator : public RelationalOperator {
    *
    * @param relation The relation to create index upon.
    * @param index_name The index to create.
-   * @param The ID of the query to which this operator belongs.
+   * @param query_id The ID of the query to which this operator belongs.
    * @param index_description The index_description associated with this index.
    **/
   CreateIndexOperator(CatalogRelation *relation,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/30553790/relational_operators/CreateTableOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/CreateTableOperator.hpp b/relational_operators/CreateTableOperator.hpp
index 1439a2b..60bcef4 100644
--- a/relational_operators/CreateTableOperator.hpp
+++ b/relational_operators/CreateTableOperator.hpp
@@ -50,7 +50,7 @@ class CreateTableOperator : public RelationalOperator {
   /**
    * @brief Constructor.
    *
-   * @param The ID of the query to which this operator belongs.
+   * @param query_id The ID of the query to which this operator belongs.
    * @param relation The relation to add. This CreateTableOperator owns
    *        relation until the WorkOrder it produces is successfully executed,
    *        at which point it is owned by database.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/30553790/relational_operators/DeleteOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/DeleteOperator.hpp b/relational_operators/DeleteOperator.hpp
index 9b6a44d..ba1f825 100644
--- a/relational_operators/DeleteOperator.hpp
+++ b/relational_operators/DeleteOperator.hpp
@@ -61,7 +61,7 @@ class DeleteOperator : public RelationalOperator {
    *        tuples will be deleted).
    * @param relation_is_stored If relation is a stored relation and is fully
    *        available to the operator before it can start generating workorders.
-   * @param The ID of the query to which this operator belongs.
+   * @param query_id The ID of the query to which this operator belongs.
    **/
   DeleteOperator(const CatalogRelation &relation,
                  const QueryContext::predicate_id predicate_index,
@@ -130,7 +130,7 @@ class DeleteWorkOrder : public WorkOrder {
    * @param delete_operator_index The index of the Delete Operator in the query
    *        plan DAG.
    * @param scheduler_client_id The TMB client ID of the scheduler thread.
-   * @param The ID of the query to which this operator belongs.
+   * @param query_id The ID of the query to which this workorder belongs.
    * @param bus A pointer to the TMB.
    **/
   DeleteWorkOrder(const CatalogRelationSchema &input_relation,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/30553790/relational_operators/DestroyHashOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/DestroyHashOperator.hpp b/relational_operators/DestroyHashOperator.hpp
index d799e05..086c279 100644
--- a/relational_operators/DestroyHashOperator.hpp
+++ b/relational_operators/DestroyHashOperator.hpp
@@ -47,7 +47,7 @@ class DestroyHashOperator : public RelationalOperator {
    * @brief Constructor.
    *
    * @param hash_table_index The index of the JoinHashTable in QueryContext.
-   * @param The ID of the query to which this operator belongs.
+   * @param query_id The ID of the query to which this operator belongs.
    **/
   DestroyHashOperator(const QueryContext::join_hash_table_id hash_table_index,
                       const std::size_t query_id)
@@ -79,13 +79,14 @@ class DestroyHashWorkOrder : public WorkOrder {
    * @brief Constructor.
    *
    * @param hash_table_index The index of the JoinHashTable in QueryContext.
-   * @param The ID of the query to which this operator belongs.
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param query_context The QueryContext to use.
    **/
   DestroyHashWorkOrder(const QueryContext::join_hash_table_id hash_table_index,
                        const std::size_t query_id,
                        QueryContext *query_context)
-      : hash_table_index_(hash_table_index),
+      : WorkOrder(query_id),
+        hash_table_index_(hash_table_index),
         query_context_(DCHECK_NOTNULL(query_context)) {}
 
   ~DestroyHashWorkOrder() override {}

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/30553790/relational_operators/DropTableOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/DropTableOperator.hpp b/relational_operators/DropTableOperator.hpp
index e5fc989..0bbb718 100644
--- a/relational_operators/DropTableOperator.hpp
+++ b/relational_operators/DropTableOperator.hpp
@@ -56,7 +56,7 @@ class DropTableOperator : public RelationalOperator {
    * @brief Constructor.
    *
    * @param relation The relation to drop.
-   * @param The ID of the query to which this operator belongs.
+   * @param query_id The ID of the query to which this operator belongs.
    * @param database The databse where to drop \c relation.
    * @param only_drop_blocks If true, only drop the blocks belonging to \c
    *        relation, but leave \c relation in \c database.
@@ -99,7 +99,7 @@ class DropTableWorkOrder : public WorkOrder {
   /**
    * @brief Constructor.
    *
-   * @param The ID of the query to which this operator belongs.
+   * @param query_id The ID of the query to which this operator belongs.
    * @param blocks The blocks to drop.
    * @param storage_manager The StorageManager to use.
    * @param rel_id The relation id to drop.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/30553790/relational_operators/FinalizeAggregationOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/FinalizeAggregationOperator.hpp b/relational_operators/FinalizeAggregationOperator.hpp
index da93e34..51e55dc 100644
--- a/relational_operators/FinalizeAggregationOperator.hpp
+++ b/relational_operators/FinalizeAggregationOperator.hpp
@@ -58,7 +58,7 @@ class FinalizeAggregationOperator : public RelationalOperator {
    * @param output_relation The output relation.
    * @param output_destination_index The index of the InsertDestination in the
    *        QueryContext to insert aggregation results.
-   * @param The ID of the query to which this operator belongs.
+   * @param query_id The ID of the query to which this operator belongs.
    */
   FinalizeAggregationOperator(const QueryContext::aggregation_state_id aggr_state_index,
                               const CatalogRelation &output_relation,
@@ -105,7 +105,7 @@ class FinalizeAggregationWorkOrder : public WorkOrder {
    *
    * @note InsertWorkOrder takes ownership of \c state.
    *
-   * @param The ID of the query to which this operator belongs.
+   * @param query_id The ID of the query to which this operator belongs.
    * @param state The AggregationState to use.
    * @param output_destination The InsertDestination to insert aggregation
    *        results.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/30553790/relational_operators/HashJoinOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/HashJoinOperator.hpp b/relational_operators/HashJoinOperator.hpp
index b0a26dc..37c23b8 100644
--- a/relational_operators/HashJoinOperator.hpp
+++ b/relational_operators/HashJoinOperator.hpp
@@ -246,6 +246,7 @@ class HashInnerJoinWorkOrder : public WorkOrder {
    *        in \c output_destination. Each Scalar is evaluated for the joined
    *        tuples, and the resulting value is inserted into the join result.
    * @param hash_table The JoinHashTable to use.
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to insert the join results.
    * @param storage_manager The StorageManager to use.
    **/
@@ -291,6 +292,7 @@ class HashInnerJoinWorkOrder : public WorkOrder {
    *        in \c output_destination. Each Scalar is evaluated for the joined
    *        tuples, and the resulting value is inserted into the join result.
    * @param hash_table The JoinHashTable to use.
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to insert the join results.
    * @param storage_manager The StorageManager to use.
    **/
@@ -373,6 +375,7 @@ class HashSemiJoinWorkOrder : public WorkOrder {
    *        in \c output_destination. Each Scalar is evaluated for the joined
    *        tuples, and the resulting value is inserted into the join result.
    * @param hash_table The JoinHashTable to use.
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to insert the join results.
    * @param storage_manager The StorageManager to use.
    **/
@@ -418,6 +421,7 @@ class HashSemiJoinWorkOrder : public WorkOrder {
    *        in \c output_destination. Each Scalar is evaluated for the joined
    *        tuples, and the resulting value is inserted into the join result.
    * @param hash_table The JoinHashTable to use.
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to insert the join results.
    * @param storage_manager The StorageManager to use.
    **/
@@ -493,6 +497,7 @@ class HashAntiJoinWorkOrder : public WorkOrder {
    *        in \c output_destination. Each Scalar is evaluated for the joined
    *        tuples, and the resulting value is inserted into the join result.
    * @param hash_table The JoinHashTable to use.
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to insert the join results.
    * @param storage_manager The StorageManager to use.
    **/
@@ -538,6 +543,7 @@ class HashAntiJoinWorkOrder : public WorkOrder {
    *        in \c output_destination. Each Scalar is evaluated for the joined
    *        tuples, and the resulting value is inserted into the join result.
    * @param hash_table The JoinHashTable to use.
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to insert the join results.
    * @param storage_manager The StorageManager to use.
    **/
@@ -617,7 +623,7 @@ class HashOuterJoinWorkOrder : public WorkOrder {
    *        is using attributes from the build relation as input. Note that the
    *        length of this vector should equal the length of \p selection.
    * @param lookup_block_id The block id of the probe_relation.
-   * @param query_id The ID of the query.
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to insert the join results.
    * @param storage_manager The StorageManager to use.
    **/
@@ -632,7 +638,8 @@ class HashOuterJoinWorkOrder : public WorkOrder {
                          const std::size_t query_id,
                          InsertDestination *output_destination,
                          StorageManager *storage_manager)
-      : build_relation_(build_relation),
+      : WorkOrder(query_id),
+        build_relation_(build_relation),
         probe_relation_(probe_relation),
         join_key_attributes_(join_key_attributes),
         any_join_key_attributes_nullable_(any_join_key_attributes_nullable),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/30553790/relational_operators/InsertOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/InsertOperator.hpp b/relational_operators/InsertOperator.hpp
index 5ac0051..ccef444 100644
--- a/relational_operators/InsertOperator.hpp
+++ b/relational_operators/InsertOperator.hpp
@@ -57,7 +57,7 @@ class InsertOperator : public RelationalOperator {
    * @param output_destination_index The index of the InsertDestination in the
    *        QueryContext to insert the tuple.
    * @param tuple_index The index of the tuple to insert in the QueryContext.
-   * @param The ID of the query to which this operator belongs.
+   * @param query_id The ID of the query to which this operator belongs.
    **/
   InsertOperator(const CatalogRelation &output_relation,
                  const QueryContext::insert_destination_id output_destination_index,
@@ -104,7 +104,7 @@ class InsertWorkOrder : public WorkOrder {
    *
    * @note InsertWorkOrder takes ownership of \c tuple.
    *
-   * @param The ID of the query to which this operator belongs.
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to insert the tuple.
    * @param tuple The tuple to insert.
    **/

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/30553790/relational_operators/NestedLoopsJoinOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/NestedLoopsJoinOperator.hpp b/relational_operators/NestedLoopsJoinOperator.hpp
index 9d81d10..f165442 100644
--- a/relational_operators/NestedLoopsJoinOperator.hpp
+++ b/relational_operators/NestedLoopsJoinOperator.hpp
@@ -76,7 +76,7 @@ class NestedLoopsJoinOperator : public RelationalOperator {
    * @param left_relation_is_stored If left_input_relation is a stored relation.
    * @param right_relation_is_stored If right_input_relation is a stored
    *                                 relation.
-   * @param The ID of the query to which this operator belongs.
+   * @param query_id The ID of the query to which this operator belongs.
    **/
   NestedLoopsJoinOperator(const CatalogRelation &left_input_relation,
                           const CatalogRelation &right_input_relation,
@@ -233,7 +233,7 @@ class NestedLoopsJoinWorkOrder : public WorkOrder {
    * @param selection A list of Scalars corresponding to the relation attributes
    *        in \c output_destination. Each Scalar is evaluated for the joined
    *        tuples, and the resulting value is inserted into the join result.
-   * @param The ID of the query to which this operator belongs.
+   * @param query_id The ID of the query to which this operator belongs.
    * @param output_destination The InsertDestination to insert the join results.
    * @param storage_manager The StorageManager to use.
    **/

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/30553790/relational_operators/RebuildWorkOrder.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/RebuildWorkOrder.hpp b/relational_operators/RebuildWorkOrder.hpp
index 88e5248..2695a62 100644
--- a/relational_operators/RebuildWorkOrder.hpp
+++ b/relational_operators/RebuildWorkOrder.hpp
@@ -55,7 +55,7 @@ class RebuildWorkOrder : public WorkOrder {
    * @param input_relation_id The ID of the CatalogRelation to which the given
    *        storage block belongs to.
    * @param scheduler_client_id The TMB client ID of the scheduler thread.
-   * @param The ID of the query to which this operator belongs.
+   * @param query_id The ID of the query to which this RebuildWorkOrder belongs.
    * @param bus A pointer to the TMB.
    **/
   RebuildWorkOrder(MutableBlockReference &&block_ref,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/30553790/relational_operators/RelationalOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/RelationalOperator.hpp b/relational_operators/RelationalOperator.hpp
index 10aa4aa..aa93018 100644
--- a/relational_operators/RelationalOperator.hpp
+++ b/relational_operators/RelationalOperator.hpp
@@ -208,6 +208,7 @@ class RelationalOperator {
   /**
    * @brief Constructor
    *
+   * @param query_id The ID of the query to which this operator belongs.
    * @param blocking_dependencies_met If those dependencies which break the
    *        pipeline have been met.
    **/

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/30553790/relational_operators/SampleOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SampleOperator.hpp b/relational_operators/SampleOperator.hpp
index 7f88eea..f65f28a 100644
--- a/relational_operators/SampleOperator.hpp
+++ b/relational_operators/SampleOperator.hpp
@@ -65,7 +65,7 @@ class SampleOperator : public RelationalOperator {
    *        workorders.
    * @param is_block_sample Flag indicating whether the sample type is block or tuple.
    * @param percentage The percentage of data to be sampled.
-   * @param The ID of the query to which this operator belongs.
+   * @param query_id The ID of the query to which this operator belongs.
    **/
   SampleOperator(const CatalogRelation &input_relation,
                  const CatalogRelationSchema &output_relation,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/30553790/relational_operators/SaveBlocksOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SaveBlocksOperator.hpp b/relational_operators/SaveBlocksOperator.hpp
index 3bcc6a7..f4650bb 100644
--- a/relational_operators/SaveBlocksOperator.hpp
+++ b/relational_operators/SaveBlocksOperator.hpp
@@ -106,7 +106,8 @@ class SaveBlocksWorkOrder : public WorkOrder {
                       const bool force,
                       const std::size_t query_id,
                       StorageManager *storage_manager)
-      : save_block_id_(save_block_id),
+      : WorkOrder(query_id),
+        save_block_id_(save_block_id),
         force_(force),
         storage_manager_(DCHECK_NOTNULL(storage_manager)) {}
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/30553790/relational_operators/SortRunGenerationOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SortRunGenerationOperator.hpp b/relational_operators/SortRunGenerationOperator.hpp
index 7618790..3da9813 100644
--- a/relational_operators/SortRunGenerationOperator.hpp
+++ b/relational_operators/SortRunGenerationOperator.hpp
@@ -155,7 +155,7 @@ class SortRunGenerationWorkOrder : public WorkOrder {
    * @param input_block_id The block id.
    * @param sort_config The Sort configuration specifying ORDER BY, ordering,
    *        and null ordering.
-   * @param query_id The ID of the query to which this operator belongs.
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to store the sorted blocks
    *        of runs.
    * @param storage_manager The StorageManager to use.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/30553790/relational_operators/TableGeneratorOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/TableGeneratorOperator.hpp b/relational_operators/TableGeneratorOperator.hpp
index 27efcad..bfc70c5 100644
--- a/relational_operators/TableGeneratorOperator.hpp
+++ b/relational_operators/TableGeneratorOperator.hpp
@@ -114,7 +114,7 @@ class TableGeneratorWorkOrder : public WorkOrder {
    * @brief Constructor.
    *
    * @param generator_function The GeneratorFunctionHandle to use.
-   * @param query_id The ID of the query to which this operator belongs.
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to insert the generated
    *        output.
    **/

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/30553790/relational_operators/TextScanOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/TextScanOperator.hpp b/relational_operators/TextScanOperator.hpp
index ad03f6a..1d0c04f 100644
--- a/relational_operators/TextScanOperator.hpp
+++ b/relational_operators/TextScanOperator.hpp
@@ -205,7 +205,7 @@ class TextScanWorkOrder : public WorkOrder {
    *        the text file.
    * @param process_escape_sequences Whether to decode escape sequences in the
    *        text file.
-   * @param query_id The ID of the query to which this operator belongs.
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to insert tuples.
    * @param storage_manager The StorageManager to use.
    **/
@@ -226,7 +226,7 @@ class TextScanWorkOrder : public WorkOrder {
    *        the text file.
    * @param process_escape_sequences Whether to decode escape sequences in the
    *        text file.
-   * @param query_id The ID of the query to which this operator belongs.
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to write the read tuples.
    * @param storage_manager The StorageManager to use.
    */
@@ -325,7 +325,7 @@ class TextSplitWorkOrder : public WorkOrder {
    * @param filename File to split into row-aligned blobs.
    * @param process_escape_sequences Whether to decode escape sequences in the
    *        text file.
-   * @param query_id The ID of the query to which this operator belongs.
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param storage_manager The StorageManager to use.
    * @param operator_index Operator index of the current operator. This is used
    *                       to send new-work available message to Foreman.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/30553790/relational_operators/UpdateOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/UpdateOperator.hpp b/relational_operators/UpdateOperator.hpp
index f1de0eb..ba2d6cf 100644
--- a/relational_operators/UpdateOperator.hpp
+++ b/relational_operators/UpdateOperator.hpp
@@ -133,7 +133,7 @@ class UpdateWorkOrder : public WorkOrder {
    * @param assignments The assignments (the map of attribute_ids to Scalars)
    *        which should be evaluated to get the new value for the corresponding
    *        attribute.
-   * @param query_id The ID of the query to which this operator belongs.
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param input_block_id The block id.
    * @param relocation_destination The InsertDestination to relocate tuples
    *        which can not be updated in-place.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/30553790/relational_operators/WorkOrder.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrder.hpp b/relational_operators/WorkOrder.hpp
index 5f8ad3f..f159cc2 100644
--- a/relational_operators/WorkOrder.hpp
+++ b/relational_operators/WorkOrder.hpp
@@ -286,6 +286,11 @@ class WorkOrder {
   }
 
  protected:
+  /**
+   * @brief Constructor.
+   *
+   * @param query_id The ID of the query to which this WorkOrder belongs.
+   **/
   explicit WorkOrder(const std::size_t query_id = 0)
       : query_id_(query_id) {}
 


[03/16] incubator-quickstep git commit: Added query ID to BuildHash operator and WorkOrders.

Posted by hb...@apache.org.
Added query ID to BuildHash operator and WorkOrders.


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

Branch: refs/heads/query-manager-used-in-foreman
Commit: ab8d21e2416d73afe077c0b1457d0477ade78e00
Parents: 9a141fa
Author: Harshad Deshmukh <ha...@cs.wisc.edu>
Authored: Mon May 2 23:19:00 2016 -0500
Committer: Harshad Deshmukh <ha...@cs.wisc.edu>
Committed: Mon May 2 23:19:00 2016 -0500

----------------------------------------------------------------------
 query_optimizer/ExecutionGenerator.cpp            |  3 ++-
 relational_operators/BuildHashOperator.cpp        |  2 ++
 relational_operators/BuildHashOperator.hpp        | 17 +++++++++++++----
 relational_operators/WorkOrder.proto              |  1 +
 relational_operators/WorkOrderFactory.cpp         |  3 ++-
 .../tests/HashJoinOperator_unittest.cpp           | 18 ++++++++++++------
 6 files changed, 32 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ab8d21e2/query_optimizer/ExecutionGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.cpp b/query_optimizer/ExecutionGenerator.cpp
index c66410a..d7dc568 100644
--- a/query_optimizer/ExecutionGenerator.cpp
+++ b/query_optimizer/ExecutionGenerator.cpp
@@ -698,7 +698,8 @@ void ExecutionGenerator::convertHashJoin(const P::HashJoinPtr &physical_plan) {
               build_relation_info->isStoredRelation(),
               build_attribute_ids,
               any_build_attributes_nullable,
-              join_hash_table_index));
+              join_hash_table_index,
+              query_handle_->query_id()));
 
   // Create InsertDestination proto.
   const CatalogRelation *output_relation = nullptr;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ab8d21e2/relational_operators/BuildHashOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/BuildHashOperator.cpp b/relational_operators/BuildHashOperator.cpp
index df92159..c6f6f96 100644
--- a/relational_operators/BuildHashOperator.cpp
+++ b/relational_operators/BuildHashOperator.cpp
@@ -73,6 +73,7 @@ bool BuildHashOperator::getAllWorkOrders(
                                    join_key_attributes_,
                                    any_join_key_attributes_nullable_,
                                    input_block_id,
+                                   query_id_,
                                    hash_table,
                                    storage_manager),
             op_index_);
@@ -88,6 +89,7 @@ bool BuildHashOperator::getAllWorkOrders(
               join_key_attributes_,
               any_join_key_attributes_nullable_,
               input_relation_block_ids_[num_workorders_generated_],
+              query_id_,
               hash_table,
               storage_manager),
           op_index_);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ab8d21e2/relational_operators/BuildHashOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/BuildHashOperator.hpp b/relational_operators/BuildHashOperator.hpp
index f9d830f..973a214 100644
--- a/relational_operators/BuildHashOperator.hpp
+++ b/relational_operators/BuildHashOperator.hpp
@@ -69,13 +69,16 @@ class BuildHashOperator : public RelationalOperator {
    * @param hash_table_index The index of the JoinHashTable in QueryContext.
    *        The HashTable's key Type(s) should be the Type(s) of the
    *        join_key_attributes in input_relation.
+   * @param The ID of the query to which this operator belongs.
    **/
   BuildHashOperator(const CatalogRelation &input_relation,
                     const bool input_relation_is_stored,
                     const std::vector<attribute_id> &join_key_attributes,
                     const bool any_join_key_attributes_nullable,
-                    const QueryContext::join_hash_table_id hash_table_index)
-    : input_relation_(input_relation),
+                    const QueryContext::join_hash_table_id hash_table_index,
+                    const std::size_t query_id)
+    : RelationalOperator(query_id),
+      input_relation_(input_relation),
       input_relation_is_stored_(input_relation_is_stored),
       join_key_attributes_(join_key_attributes),
       any_join_key_attributes_nullable_(any_join_key_attributes_nullable),
@@ -133,6 +136,7 @@ class BuildHashWorkOrder : public WorkOrder {
    *        input_relation.
    * @param any_join_key_attributes_nullable If any attribute is nullable.
    * @param build_block_id The block id.
+   * @param query_id The ID of the query.
    * @param hash_table The JoinHashTable to use.
    * @param storage_manager The StorageManager to use.
    **/
@@ -140,9 +144,11 @@ class BuildHashWorkOrder : public WorkOrder {
                      const std::vector<attribute_id> &join_key_attributes,
                      const bool any_join_key_attributes_nullable,
                      const block_id build_block_id,
+                     const std::size_t query_id,
                      JoinHashTable *hash_table,
                      StorageManager *storage_manager)
-      : input_relation_(input_relation),
+      : WorkOrder(query_id),
+        input_relation_(input_relation),
         join_key_attributes_(join_key_attributes),
         any_join_key_attributes_nullable_(any_join_key_attributes_nullable),
         build_block_id_(build_block_id),
@@ -156,6 +162,7 @@ class BuildHashWorkOrder : public WorkOrder {
    * @param join_key_attributes The IDs of equijoin attributes in
    *        input_relation.
    * @param any_join_key_attributes_nullable If any attribute is nullable.
+   * @param query_id The ID of the query.
    * @param build_block_id The block id.
    * @param hash_table The JoinHashTable to use.
    * @param storage_manager The StorageManager to use.
@@ -164,9 +171,11 @@ class BuildHashWorkOrder : public WorkOrder {
                      std::vector<attribute_id> &&join_key_attributes,
                      const bool any_join_key_attributes_nullable,
                      const block_id build_block_id,
+                     const std::size_t query_id,
                      JoinHashTable *hash_table,
                      StorageManager *storage_manager)
-      : input_relation_(input_relation),
+      : WorkOrder(query_id),
+        input_relation_(input_relation),
         join_key_attributes_(std::move(join_key_attributes)),
         any_join_key_attributes_nullable_(any_join_key_attributes_nullable),
         build_block_id_(build_block_id),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ab8d21e2/relational_operators/WorkOrder.proto
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrder.proto b/relational_operators/WorkOrder.proto
index 8ed2080..42fc9ae 100644
--- a/relational_operators/WorkOrder.proto
+++ b/relational_operators/WorkOrder.proto
@@ -48,6 +48,7 @@ enum WorkOrderType {
 
 message WorkOrder {
   required WorkOrderType work_order_type = 1;
+  required uint32 query_id = 2;
 
   // The convention for extension numbering is that extensions for a particular
   // WorkOrderID should begin from (operator_type + 1) * 16.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ab8d21e2/relational_operators/WorkOrderFactory.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrderFactory.cpp b/relational_operators/WorkOrderFactory.cpp
index d34d535..79ec633 100644
--- a/relational_operators/WorkOrderFactory.cpp
+++ b/relational_operators/WorkOrderFactory.cpp
@@ -71,7 +71,7 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
       LOG(INFO) << "Creating AggregationWorkOrder";
       return new AggregationWorkOrder(
           proto.GetExtension(serialization::AggregationWorkOrder::block_id),
-          0,  // TODO(harshad) - Replace this with true query ID.
+          proto.query_id(),
           query_context->getAggregationState(
               proto.GetExtension(serialization::AggregationWorkOrder::aggr_state_index)));
     }
@@ -89,6 +89,7 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
           move(join_key_attributes),
           proto.GetExtension(serialization::BuildHashWorkOrder::any_join_key_attributes_nullable),
           proto.GetExtension(serialization::BuildHashWorkOrder::block_id),
+          proto.query_id(),
           query_context->getJoinHashTable(
               proto.GetExtension(serialization::BuildHashWorkOrder::join_hash_table_index)),
           storage_manager);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/ab8d21e2/relational_operators/tests/HashJoinOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/HashJoinOperator_unittest.cpp b/relational_operators/tests/HashJoinOperator_unittest.cpp
index 333c3f0..f1b18e7 100644
--- a/relational_operators/tests/HashJoinOperator_unittest.cpp
+++ b/relational_operators/tests/HashJoinOperator_unittest.cpp
@@ -336,7 +336,8 @@ TEST_P(HashJoinOperatorTest, LongKeyHashJoinTest) {
                             true /* is_stored */,
                             std::vector<attribute_id>(1, dim_col_long.getID()),
                             dim_col_long.getType().isNullable(),
-                            join_hash_table_index));
+                            join_hash_table_index,
+                            0));  // dummy query ID
 
   // Create the prober operator with one selection attribute.
   const QueryContext::scalar_group_id selection_index = query_context_proto.scalar_groups_size();
@@ -477,7 +478,8 @@ TEST_P(HashJoinOperatorTest, IntDuplicateKeyHashJoinTest) {
                             true /* is_stored */,
                             std::vector<attribute_id>(1, dim_col_int.getID()),
                             dim_col_int.getType().isNullable(),
-                            join_hash_table_index));
+                            join_hash_table_index,
+                            0));  // dummy query ID
 
   // Create the prober operator with two selection attributes.
   const QueryContext::scalar_group_id selection_index = query_context_proto.scalar_groups_size();
@@ -636,7 +638,8 @@ TEST_P(HashJoinOperatorTest, CharKeyCartesianProductHashJoinTest) {
                             true /* is_stored */,
                             std::vector<attribute_id>(1, dim_col_char.getID()),
                             dim_col_char.getType().isNullable(),
-                            join_hash_table_index));
+                            join_hash_table_index,
+                            0));  // dummy query ID.
 
   // Create prober operator with one selection attribute.
   const QueryContext::scalar_group_id selection_index = query_context_proto.scalar_groups_size();
@@ -770,7 +773,8 @@ TEST_P(HashJoinOperatorTest, VarCharDuplicateKeyHashJoinTest) {
                             true /* is_stored */,
                             std::vector<attribute_id>(1, dim_col_varchar.getID()),
                             dim_col_varchar.getType().isNullable(),
-                            join_hash_table_index));
+                            join_hash_table_index,
+                            0));  // dummy query ID.
 
   // Create prober operator with two selection attributes.
   const QueryContext::scalar_group_id selection_index = query_context_proto.scalar_groups_size();
@@ -939,7 +943,8 @@ TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinTest) {
                             true /* is_stored */,
                             dim_key_attrs,
                             dim_col_long.getType().isNullable() || dim_col_varchar.getType().isNullable(),
-                            join_hash_table_index));
+                            join_hash_table_index,
+                            0));  // dummy query ID.
 
   // Create the prober operator with two selection attributes.
   const QueryContext::scalar_group_id selection_index = query_context_proto.scalar_groups_size();
@@ -1113,7 +1118,8 @@ TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinWithResidualPredicateTest) {
                             true /* is_stored */,
                             dim_key_attrs,
                             dim_col_long.getType().isNullable() || dim_col_varchar.getType().isNullable(),
-                            join_hash_table_index));
+                            join_hash_table_index,
+                            0));  // dummy query ID.
 
   // Create prober operator with two selection attributes.
   const QueryContext::scalar_group_id selection_index = query_context_proto.scalar_groups_size();


[04/16] incubator-quickstep git commit: Merge branch 'master' into query-id-operator-workorder

Posted by hb...@apache.org.
Merge branch 'master' into query-id-operator-workorder


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

Branch: refs/heads/query-manager-used-in-foreman
Commit: d2841af736babbe4769a356ffaa55e94e4f4fa44
Parents: ab8d21e 29a71ac
Author: Harshad Deshmukh <ha...@cs.wisc.edu>
Authored: Mon May 2 23:19:56 2016 -0500
Committer: Harshad Deshmukh <ha...@cs.wisc.edu>
Committed: Mon May 2 23:19:56 2016 -0500

----------------------------------------------------------------------
 catalog/CMakeLists.txt                    |   5 +-
 catalog/CatalogRelation.hpp               |   9 +
 parser/ParseIndexProperties.hpp           |   2 +-
 parser/tests/Index.test                   |  11 +
 relational_operators/CMakeLists.txt       |   8 +
 relational_operators/HashJoinOperator.cpp |   5 +-
 relational_operators/SelectOperator.cpp   | 129 +++++++--
 relational_operators/SelectOperator.hpp   | 115 +++++++-
 relational_operators/WorkOrder.proto      |  98 +++----
 relational_operators/WorkOrderFactory.cpp | 362 +++++++++----------------
 10 files changed, 400 insertions(+), 344 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d2841af7/relational_operators/WorkOrder.proto
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d2841af7/relational_operators/WorkOrderFactory.cpp
----------------------------------------------------------------------


[12/16] incubator-quickstep git commit: Code review suggestions

Posted by hb...@apache.org.
Code review suggestions

- Made query ID as uint64 in the WorkOrder proto.
- Removed the default value set for query ID in the base WorkOrder
  constructor.


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

Branch: refs/heads/query-manager-used-in-foreman
Commit: eb3e73f7fcbb169f4403b4eceb39182221b43962
Parents: 9f1008a
Author: Harshad Deshmukh <hb...@apache.org>
Authored: Mon Jun 6 19:19:19 2016 -0500
Committer: Harshad Deshmukh <hb...@apache.org>
Committed: Mon Jun 6 19:19:19 2016 -0500

----------------------------------------------------------------------
 query_execution/tests/Foreman_unittest.cpp             | 2 +-
 query_execution/tests/QueryManager_unittest.cpp        | 2 +-
 query_execution/tests/WorkOrdersContainer_unittest.cpp | 2 +-
 relational_operators/HashJoinOperator.hpp              | 9 ++++++---
 relational_operators/RebuildWorkOrder.hpp              | 3 ++-
 relational_operators/WorkOrder.hpp                     | 2 +-
 relational_operators/WorkOrder.proto                   | 2 +-
 7 files changed, 13 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/eb3e73f7/query_execution/tests/Foreman_unittest.cpp
----------------------------------------------------------------------
diff --git a/query_execution/tests/Foreman_unittest.cpp b/query_execution/tests/Foreman_unittest.cpp
index 47cc641..d2f43a4 100644
--- a/query_execution/tests/Foreman_unittest.cpp
+++ b/query_execution/tests/Foreman_unittest.cpp
@@ -61,7 +61,7 @@ namespace quickstep {
 class MockWorkOrder : public WorkOrder {
  public:
   explicit MockWorkOrder(const int op_index)
-      : op_index_(op_index) {}
+      : WorkOrder(0), op_index_(op_index) {}
 
   void execute() override {
     VLOG(3) << "WorkOrder[" << op_index_ << "] executing.";

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/eb3e73f7/query_execution/tests/QueryManager_unittest.cpp
----------------------------------------------------------------------
diff --git a/query_execution/tests/QueryManager_unittest.cpp b/query_execution/tests/QueryManager_unittest.cpp
index 1b9be48..80876f2 100644
--- a/query_execution/tests/QueryManager_unittest.cpp
+++ b/query_execution/tests/QueryManager_unittest.cpp
@@ -62,7 +62,7 @@ namespace quickstep {
 class MockWorkOrder : public WorkOrder {
  public:
   explicit MockWorkOrder(const int op_index)
-      : op_index_(op_index) {}
+      : WorkOrder(0), op_index_(op_index) {}
 
   void execute() override {
     VLOG(3) << "WorkOrder[" << op_index_ << "] executing.";

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/eb3e73f7/query_execution/tests/WorkOrdersContainer_unittest.cpp
----------------------------------------------------------------------
diff --git a/query_execution/tests/WorkOrdersContainer_unittest.cpp b/query_execution/tests/WorkOrdersContainer_unittest.cpp
index d7db9a6..cf133c4 100644
--- a/query_execution/tests/WorkOrdersContainer_unittest.cpp
+++ b/query_execution/tests/WorkOrdersContainer_unittest.cpp
@@ -30,7 +30,7 @@ namespace quickstep {
 class MockNUMAWorkOrder : public WorkOrder {
  public:
   MockNUMAWorkOrder(const int id, const std::vector<int> &numa_nodes)
-      : id_(id) {
+      : WorkOrder(0), id_(id) {
     for (int numa_node : numa_nodes) {
       preferred_numa_nodes_.push_back(numa_node);
     }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/eb3e73f7/relational_operators/HashJoinOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/HashJoinOperator.hpp b/relational_operators/HashJoinOperator.hpp
index 37c23b8..825f360 100644
--- a/relational_operators/HashJoinOperator.hpp
+++ b/relational_operators/HashJoinOperator.hpp
@@ -660,14 +660,15 @@ class HashOuterJoinWorkOrder : public WorkOrder {
    * @param join_key_attributes The IDs of equijoin attributes in \c
    *        probe_relation.
    * @param any_join_key_attributes_nullable If any attribute is nullable.
-   * @param hash_table The JoinHashTable to use.
+   * @param lookup_block_id The block id of the probe_relation.
    * @param selection A list of Scalars corresponding to the relation attributes
    *        in \c output_destination. Each Scalar is evaluated for the joined
    *        tuples, and the resulting value is inserted into the join result.
    * @param is_selection_on_build Whether each Scalar in the \p selection vector
    *        is using attributes from the build relation as input. Note that the
    *        length of this vector should equal the length of \p selection.
-   * @param lookup_block_id The block id of the probe_relation.
+   * @param hash_table The JoinHashTable to use.
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to insert the join results.
    * @param storage_manager The StorageManager to use.
    **/
@@ -679,9 +680,11 @@ class HashOuterJoinWorkOrder : public WorkOrder {
                          const std::vector<std::unique_ptr<const Scalar>> &selection,
                          std::vector<bool> &&is_selection_on_build,
                          const JoinHashTable &hash_table,
+                         const std::size_t query_id,
                          InsertDestination *output_destination,
                          StorageManager *storage_manager)
-      : build_relation_(build_relation),
+      : WorkOrder(query_id),
+        build_relation_(build_relation),
         probe_relation_(probe_relation),
         join_key_attributes_(std::move(join_key_attributes)),
         any_join_key_attributes_nullable_(any_join_key_attributes_nullable),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/eb3e73f7/relational_operators/RebuildWorkOrder.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/RebuildWorkOrder.hpp b/relational_operators/RebuildWorkOrder.hpp
index 2695a62..ae876ba 100644
--- a/relational_operators/RebuildWorkOrder.hpp
+++ b/relational_operators/RebuildWorkOrder.hpp
@@ -64,7 +64,8 @@ class RebuildWorkOrder : public WorkOrder {
                    const client_id scheduler_client_id,
                    const std::size_t query_id,
                    MessageBus *bus)
-      : block_ref_(std::move(block_ref)),
+      : WorkOrder(query_id),
+        block_ref_(std::move(block_ref)),
         input_operator_index_(input_operator_index),
         input_relation_id_(input_relation_id),
         scheduler_client_id_(scheduler_client_id),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/eb3e73f7/relational_operators/WorkOrder.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrder.hpp b/relational_operators/WorkOrder.hpp
index f159cc2..059865d 100644
--- a/relational_operators/WorkOrder.hpp
+++ b/relational_operators/WorkOrder.hpp
@@ -291,7 +291,7 @@ class WorkOrder {
    *
    * @param query_id The ID of the query to which this WorkOrder belongs.
    **/
-  explicit WorkOrder(const std::size_t query_id = 0)
+  explicit WorkOrder(const std::size_t query_id)
       : query_id_(query_id) {}
 
   const std::size_t query_id_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/eb3e73f7/relational_operators/WorkOrder.proto
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrder.proto b/relational_operators/WorkOrder.proto
index 697c09c..fd731f7 100644
--- a/relational_operators/WorkOrder.proto
+++ b/relational_operators/WorkOrder.proto
@@ -45,7 +45,7 @@ enum WorkOrderType {
 
 message WorkOrder {
   required WorkOrderType work_order_type = 1;
-  required uint32 query_id = 2;
+  required uint64 query_id = 2;
 
   // The convention for extension numbering is that extensions for a particular
   // WorkOrderID should begin from (operator_type + 1) * 16.


[09/16] incubator-quickstep git commit: Added query ID to all the remaining operators and work orders.

Posted by hb...@apache.org.
Added query ID to all the remaining operators and work orders.


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

Branch: refs/heads/query-manager-used-in-foreman
Commit: 1290dd70302d1896bbc6e9100175fceb4d7137f5
Parents: 58d5ccb
Author: Harshad Deshmukh <hb...@apache.org>
Authored: Mon Jun 6 14:32:08 2016 -0500
Committer: Harshad Deshmukh <hb...@apache.org>
Committed: Mon Jun 6 14:32:08 2016 -0500

----------------------------------------------------------------------
 query_execution/Foreman.cpp                     |  3 ++
 query_execution/QueryManager.cpp                |  1 +
 query_optimizer/ExecutionGenerator.cpp          | 50 +++++++++++++-------
 relational_operators/DropTableOperator.cpp      |  3 +-
 relational_operators/DropTableOperator.hpp      | 13 +++--
 .../FinalizeAggregationOperator.cpp             |  6 ++-
 .../FinalizeAggregationOperator.hpp             | 15 ++++--
 relational_operators/InsertOperator.cpp         |  6 ++-
 relational_operators/InsertOperator.hpp         | 15 ++++--
 .../NestedLoopsJoinOperator.cpp                 |  4 ++
 .../NestedLoopsJoinOperator.hpp                 | 12 +++--
 relational_operators/RebuildWorkOrder.hpp       |  2 +
 relational_operators/SampleOperator.cpp         |  7 ++-
 relational_operators/SampleOperator.hpp         | 13 +++--
 relational_operators/SaveBlocksOperator.cpp     |  1 +
 relational_operators/SaveBlocksOperator.hpp     |  9 +++-
 relational_operators/SelectOperator.cpp         |  4 ++
 relational_operators/SelectOperator.hpp         | 24 +++++++---
 relational_operators/SortMergeRunOperator.cpp   |  1 +
 relational_operators/SortMergeRunOperator.hpp   | 12 +++--
 .../SortRunGenerationOperator.cpp               |  2 +
 .../SortRunGenerationOperator.hpp               | 12 +++--
 relational_operators/TableGeneratorOperator.cpp |  7 ++-
 relational_operators/TableGeneratorOperator.hpp | 13 +++--
 relational_operators/TextScanOperator.cpp       | 11 ++++-
 relational_operators/TextScanOperator.hpp       | 16 +++++--
 relational_operators/UpdateOperator.cpp         |  1 +
 relational_operators/UpdateOperator.hpp         | 12 +++--
 relational_operators/WorkOrderFactory.cpp       | 21 ++++++--
 .../tests/AggregationOperator_unittest.cpp      | 12 +++--
 .../tests/SortMergeRunOperator_unittest.cpp     |  6 ++-
 .../SortRunGenerationOperator_unittest.cpp      |  3 +-
 .../tests/TextScanOperator_unittest.cpp         |  3 +-
 33 files changed, 240 insertions(+), 80 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1290dd70/query_execution/Foreman.cpp
----------------------------------------------------------------------
diff --git a/query_execution/Foreman.cpp b/query_execution/Foreman.cpp
index 304c429..b358f70 100644
--- a/query_execution/Foreman.cpp
+++ b/query_execution/Foreman.cpp
@@ -518,11 +518,14 @@ void Foreman::getRebuildWorkOrders(const dag_node_index index, WorkOrdersContain
   for (vector<MutableBlockReference>::size_type i = 0;
        i < partially_filled_block_refs.size();
        ++i) {
+    // Note: The query ID used below is dummy for now, it will be replaced with
+    // the true query ID when QueryManager gets used in Foreman.
     container->addRebuildWorkOrder(
         new RebuildWorkOrder(move(partially_filled_block_refs[i]),
                             index,
                             op.getOutputRelationID(),
                             foreman_client_id_,
+                            0,
                             bus_),
         index);
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1290dd70/query_execution/QueryManager.cpp
----------------------------------------------------------------------
diff --git a/query_execution/QueryManager.cpp b/query_execution/QueryManager.cpp
index 02c5d4c..21f5820 100644
--- a/query_execution/QueryManager.cpp
+++ b/query_execution/QueryManager.cpp
@@ -461,6 +461,7 @@ void QueryManager::getRebuildWorkOrders(const dag_node_index index,
                             index,
                             op.getOutputRelationID(),
                             foreman_client_id_,
+                            query_id_,
                             bus_),
         index);
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1290dd70/query_optimizer/ExecutionGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.cpp b/query_optimizer/ExecutionGenerator.cpp
index 60389cb..30dfa8e 100644
--- a/query_optimizer/ExecutionGenerator.cpp
+++ b/query_optimizer/ExecutionGenerator.cpp
@@ -194,6 +194,7 @@ void ExecutionGenerator::generatePlan(const P::PhysicalPtr &physical_plan) {
     const QueryPlan::DAGNodeIndex drop_table_index =
         execution_plan_->addRelationalOperator(
             new DropTableOperator(*temporary_relation,
+                                  query_handle_->query_id(),
                                   optimizer_context_->catalog_database(),
                                   false /* only_drop_blocks */));
     DCHECK(!temporary_relation_info.isStoredRelation());
@@ -415,7 +416,8 @@ void ExecutionGenerator::convertSample(const P::SamplePtr &physical_sample) {
                                                  insert_destination_index,
                                                  input_relation_info->isStoredRelation(),
                                                  physical_sample->is_block_sample(),
-                                                 physical_sample->percentage());
+                                                 physical_sample->percentage(),
+                                                 query_handle_->query_id());
   const QueryPlan::DAGNodeIndex sample_index =
       execution_plan_->addRelationalOperator(sample_op);
   insert_destination_proto->set_relational_op_index(sample_index);
@@ -531,13 +533,15 @@ void ExecutionGenerator::convertSelection(
                              insert_destination_index,
                              execution_predicate_index,
                              move(attributes),
-                             input_relation_info->isStoredRelation())
+                             input_relation_info->isStoredRelation(),
+                             query_handle_->query_id())
         : new SelectOperator(*input_relation_info->relation,
                              *output_relation,
                              insert_destination_index,
                              execution_predicate_index,
                              project_expressions_group_index,
-                             input_relation_info->isStoredRelation());
+                             input_relation_info->isStoredRelation(),
+                             query_handle_->query_id());
 
   const QueryPlan::DAGNodeIndex select_index =
       execution_plan_->addRelationalOperator(op);
@@ -889,7 +893,8 @@ void ExecutionGenerator::convertNestedLoopsJoin(
           execution_join_predicate_index,
           project_expressions_group_index,
           left_relation_info->isStoredRelation(),
-          right_relation_info->isStoredRelation()));
+          right_relation_info->isStoredRelation(),
+          query_handle_->query_id()));
   insert_destination_proto->set_relational_op_index(join_operator_index);
 
   if (!left_relation_info->isStoredRelation()) {
@@ -940,12 +945,13 @@ void ExecutionGenerator::convertCopyFrom(
               physical_plan->escape_strings(),
               FLAGS_parallelize_load,
               *output_relation,
-              insert_destination_index));
+              insert_destination_index,
+              query_handle_->query_id()));
   insert_destination_proto->set_relational_op_index(scan_operator_index);
 
   const QueryPlan::DAGNodeIndex save_blocks_operator_index =
       execution_plan_->addRelationalOperator(
-          new SaveBlocksOperator());
+          new SaveBlocksOperator(query_handle_->query_id()));
   execution_plan_->addDirectDependency(save_blocks_operator_index,
                                        scan_operator_index,
                                        false /* is_pipeline_breaker */);
@@ -1062,6 +1068,7 @@ void ExecutionGenerator::convertDeleteTuples(
     const QueryPlan::DAGNodeIndex drop_table_index =
         execution_plan_->addRelationalOperator(
             new DropTableOperator(*input_relation_info->relation,
+                                  query_handle_->query_id(),
                                   optimizer_context_->catalog_database(),
                                   true /* only_drop_blocks */));
     if (!input_relation_info->isStoredRelation()) {
@@ -1087,7 +1094,7 @@ void ExecutionGenerator::convertDeleteTuples(
 
     const QueryPlan::DAGNodeIndex save_blocks_index =
         execution_plan_->addRelationalOperator(
-            new SaveBlocksOperator());
+            new SaveBlocksOperator(query_handle_->query_id()));
     execution_plan_->addDirectDependency(save_blocks_index,
                                          delete_tuples_index,
                                          false /* is_pipeline_breaker */);
@@ -1105,6 +1112,7 @@ void ExecutionGenerator::convertDropTable(
 
   execution_plan_->addRelationalOperator(
       new DropTableOperator(catalog_relation,
+                            query_handle_->query_id(),
                             optimizer_context_->catalog_database()));
 }
 
@@ -1158,12 +1166,13 @@ void ExecutionGenerator::convertInsertTuple(
       execution_plan_->addRelationalOperator(
           new InsertOperator(input_relation,
                              insert_destination_index,
-                             tuple_index));
+                             tuple_index,
+                             query_handle_->query_id()));
   insert_destination_proto->set_relational_op_index(insert_operator_index);
 
   const QueryPlan::DAGNodeIndex save_blocks_index =
       execution_plan_->addRelationalOperator(
-          new SaveBlocksOperator());
+          new SaveBlocksOperator(query_handle_->query_id()));
   if (!input_relation_info->isStoredRelation()) {
     execution_plan_->addDirectDependency(insert_operator_index,
                                          input_relation_info->producer_operator_index,
@@ -1236,14 +1245,15 @@ void ExecutionGenerator::convertInsertSelection(
                          insert_destination_index,
                          QueryContext::kInvalidPredicateId,
                          move(attributes),
-                         selection_relation_info->isStoredRelation());
+                         selection_relation_info->isStoredRelation(),
+                         query_handle_->query_id());
 
   const QueryPlan::DAGNodeIndex insert_selection_index =
       execution_plan_->addRelationalOperator(insert_selection_op);
   insert_destination_proto->set_relational_op_index(insert_selection_index);
 
   const QueryPlan::DAGNodeIndex save_blocks_index =
-      execution_plan_->addRelationalOperator(new SaveBlocksOperator());
+      execution_plan_->addRelationalOperator(new SaveBlocksOperator(query_handle_->query_id()));
 
   if (!selection_relation_info->isStoredRelation()) {
     execution_plan_->addDirectDependency(insert_selection_index,
@@ -1316,12 +1326,13 @@ void ExecutionGenerator::convertUpdateTable(
               *optimizer_context_->catalog_database()->getRelationById(input_rel_id),
               relocation_destination_index,
               execution_predicate_index,
-              update_group_index));
+              update_group_index,
+              query_handle_->query_id()));
   relocation_destination_proto->set_relational_op_index(update_operator_index);
 
   const QueryPlan::DAGNodeIndex save_blocks_index =
       execution_plan_->addRelationalOperator(
-          new SaveBlocksOperator());
+          new SaveBlocksOperator(query_handle_->query_id()));
   if (!input_relation_info->isStoredRelation()) {
     execution_plan_->addDirectDependency(update_operator_index,
                                          input_relation_info->producer_operator_index,
@@ -1441,7 +1452,8 @@ void ExecutionGenerator::convertAggregate(
       execution_plan_->addRelationalOperator(
           new FinalizeAggregationOperator(aggr_state_index,
                                           *output_relation,
-                                          insert_destination_index));
+                                          insert_destination_index,
+                                          query_handle_->query_id()));
   insert_destination_proto->set_relational_op_index(finalize_aggregation_operator_index);
 
   execution_plan_->addDirectDependency(finalize_aggregation_operator_index,
@@ -1492,7 +1504,8 @@ void ExecutionGenerator::convertSort(const P::SortPtr &physical_sort) {
                                         *initial_runs_relation,
                                         initial_runs_destination_id,
                                         sort_run_gen_config_id,
-                                        input_relation_info->isStoredRelation()));
+                                        input_relation_info->isStoredRelation(),
+                                        query_handle_->query_id()));
   if (!input_relation_info->isStoredRelation()) {
     execution_plan_->addDirectDependency(run_generator_index,
                                          input_relation_info->producer_operator_index,
@@ -1549,7 +1562,8 @@ void ExecutionGenerator::convertSort(const P::SortPtr &physical_sort) {
                                    sort_merge_run_config_id,
                                    64 /* merge_factor */,
                                    physical_sort->limit(),
-                                   false /* input_relation_is_stored */));
+                                   false /* input_relation_is_stored */,
+                                   query_handle_->query_id()));
   execution_plan_->addDirectDependency(merge_run_operator_index,
                                        run_generator_index,
                                        false /* is_pipeline_breaker */);
@@ -1563,6 +1577,7 @@ void ExecutionGenerator::convertSort(const P::SortPtr &physical_sort) {
       execution_plan_->addRelationalOperator(
           new DropTableOperator(
               *merged_runs_relation,
+              query_handle_->query_id(),
               optimizer_context_->catalog_database(),
               false /* only_drop_blocks */));
   execution_plan_->addDirectDependency(
@@ -1600,7 +1615,8 @@ void ExecutionGenerator::convertTableGenerator(
   TableGeneratorOperator *op =
       new TableGeneratorOperator(*output_relation,
                                  insert_destination_index,
-                                 generator_function_index);
+                                 generator_function_index,
+                                 query_handle_->query_id());
 
   const QueryPlan::DAGNodeIndex tablegen_index =
       execution_plan_->addRelationalOperator(op);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1290dd70/relational_operators/DropTableOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/DropTableOperator.cpp b/relational_operators/DropTableOperator.cpp
index f3a3a2c..256f6a1 100644
--- a/relational_operators/DropTableOperator.cpp
+++ b/relational_operators/DropTableOperator.cpp
@@ -45,7 +45,8 @@ bool DropTableOperator::getAllWorkOrders(
 
     // DropTableWorkOrder only drops blocks, if any.
     container->addNormalWorkOrder(
-        new DropTableWorkOrder(std::move(relation_blocks), storage_manager),
+        new DropTableWorkOrder(
+            query_id_, std::move(relation_blocks), storage_manager),
         op_index_);
 
     database_->setStatus(CatalogDatabase::Status::kPendingBlockDeletions);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1290dd70/relational_operators/DropTableOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/DropTableOperator.hpp b/relational_operators/DropTableOperator.hpp
index bf9b1b1..e5fc989 100644
--- a/relational_operators/DropTableOperator.hpp
+++ b/relational_operators/DropTableOperator.hpp
@@ -18,6 +18,7 @@
 #ifndef QUICKSTEP_RELATIONAL_OPERATORS_DROP_TABLE_OPERATOR_HPP_
 #define QUICKSTEP_RELATIONAL_OPERATORS_DROP_TABLE_OPERATOR_HPP_
 
+#include <cstddef>
 #include <utility>
 #include <vector>
 
@@ -55,14 +56,17 @@ class DropTableOperator : public RelationalOperator {
    * @brief Constructor.
    *
    * @param relation The relation to drop.
+   * @param The ID of the query to which this operator belongs.
    * @param database The databse where to drop \c relation.
    * @param only_drop_blocks If true, only drop the blocks belonging to \c
    *        relation, but leave \c relation in \c database.
    **/
   DropTableOperator(const CatalogRelation &relation,
+                    const std::size_t query_id,
                     CatalogDatabase *database,
                     const bool only_drop_blocks = false)
-      : relation_(relation),
+      : RelationalOperator(query_id),
+        relation_(relation),
         database_(database),
         only_drop_blocks_(only_drop_blocks),
         work_generated_(false) {}
@@ -95,17 +99,20 @@ class DropTableWorkOrder : public WorkOrder {
   /**
    * @brief Constructor.
    *
+   * @param The ID of the query to which this operator belongs.
    * @param blocks The blocks to drop.
    * @param storage_manager The StorageManager to use.
    * @param rel_id The relation id to drop.
    * @param catalog_database_cache The CatalogDatabaseCache in the distributed
    *        version.
    **/
-  DropTableWorkOrder(std::vector<block_id> &&blocks,
+  DropTableWorkOrder(const std::size_t query_id,
+                     std::vector<block_id> &&blocks,
                      StorageManager *storage_manager,
                      const relation_id rel_id = kInvalidCatalogId,
                      CatalogDatabaseLite *catalog_database_cache = nullptr)
-      : blocks_(std::move(blocks)),
+      : WorkOrder(query_id),
+        blocks_(std::move(blocks)),
         storage_manager_(DCHECK_NOTNULL(storage_manager)),
         rel_id_(rel_id),
         catalog_database_cache_(catalog_database_cache) {}

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1290dd70/relational_operators/FinalizeAggregationOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/FinalizeAggregationOperator.cpp b/relational_operators/FinalizeAggregationOperator.cpp
index 410ec69..1dc4188 100644
--- a/relational_operators/FinalizeAggregationOperator.cpp
+++ b/relational_operators/FinalizeAggregationOperator.cpp
@@ -38,8 +38,10 @@ bool FinalizeAggregationOperator::getAllWorkOrders(
   if (blocking_dependencies_met_ && !started_) {
     started_ = true;
     container->addNormalWorkOrder(
-        new FinalizeAggregationWorkOrder(query_context->releaseAggregationState(aggr_state_index_),
-                                         query_context->getInsertDestination(output_destination_index_)),
+        new FinalizeAggregationWorkOrder(
+            query_id_,
+            query_context->releaseAggregationState(aggr_state_index_),
+            query_context->getInsertDestination(output_destination_index_)),
         op_index_);
   }
   return started_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1290dd70/relational_operators/FinalizeAggregationOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/FinalizeAggregationOperator.hpp b/relational_operators/FinalizeAggregationOperator.hpp
index fb9608a..da93e34 100644
--- a/relational_operators/FinalizeAggregationOperator.hpp
+++ b/relational_operators/FinalizeAggregationOperator.hpp
@@ -18,6 +18,7 @@
 #ifndef QUICKSTEP_RELATIONAL_OPERATORS_FINALIZE_AGGREGATION_OPERATOR_HPP_
 #define QUICKSTEP_RELATIONAL_OPERATORS_FINALIZE_AGGREGATION_OPERATOR_HPP_
 
+#include <cstddef>
 #include <memory>
 
 #include "catalog/CatalogRelation.hpp"
@@ -57,11 +58,14 @@ class FinalizeAggregationOperator : public RelationalOperator {
    * @param output_relation The output relation.
    * @param output_destination_index The index of the InsertDestination in the
    *        QueryContext to insert aggregation results.
+   * @param The ID of the query to which this operator belongs.
    */
   FinalizeAggregationOperator(const QueryContext::aggregation_state_id aggr_state_index,
                               const CatalogRelation &output_relation,
-                              const QueryContext::insert_destination_id output_destination_index)
-      : aggr_state_index_(aggr_state_index),
+                              const QueryContext::insert_destination_id output_destination_index,
+                              const std::size_t query_id)
+      : RelationalOperator(query_id),
+        aggr_state_index_(aggr_state_index),
         output_relation_(output_relation),
         output_destination_index_(output_destination_index),
         started_(false) {}
@@ -101,13 +105,16 @@ class FinalizeAggregationWorkOrder : public WorkOrder {
    *
    * @note InsertWorkOrder takes ownership of \c state.
    *
+   * @param The ID of the query to which this operator belongs.
    * @param state The AggregationState to use.
    * @param output_destination The InsertDestination to insert aggregation
    *        results.
    */
-  FinalizeAggregationWorkOrder(AggregationOperationState *state,
+  FinalizeAggregationWorkOrder(const std::size_t query_id,
+                               AggregationOperationState *state,
                                InsertDestination *output_destination)
-      : state_(DCHECK_NOTNULL(state)),
+      : WorkOrder(query_id),
+        state_(DCHECK_NOTNULL(state)),
         output_destination_(DCHECK_NOTNULL(output_destination)) {}
 
   ~FinalizeAggregationWorkOrder() override {}

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1290dd70/relational_operators/InsertOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/InsertOperator.cpp b/relational_operators/InsertOperator.cpp
index 8d083e5..3ec9933 100644
--- a/relational_operators/InsertOperator.cpp
+++ b/relational_operators/InsertOperator.cpp
@@ -40,8 +40,10 @@ bool InsertOperator::getAllWorkOrders(
 
     work_generated_ = true;
     container->addNormalWorkOrder(
-        new InsertWorkOrder(query_context->getInsertDestination(output_destination_index_),
-                            query_context->releaseTuple(tuple_index_)),
+        new InsertWorkOrder(
+            query_id_,
+            query_context->getInsertDestination(output_destination_index_),
+            query_context->releaseTuple(tuple_index_)),
         op_index_);
   }
   return work_generated_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1290dd70/relational_operators/InsertOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/InsertOperator.hpp b/relational_operators/InsertOperator.hpp
index 8a06c94..5ac0051 100644
--- a/relational_operators/InsertOperator.hpp
+++ b/relational_operators/InsertOperator.hpp
@@ -18,6 +18,7 @@
 #ifndef QUICKSTEP_RELATIONAL_OPERATORS_INSERT_OPERATOR_HPP_
 #define QUICKSTEP_RELATIONAL_OPERATORS_INSERT_OPERATOR_HPP_
 
+#include <cstddef>
 #include <memory>
 
 #include "catalog/CatalogRelation.hpp"
@@ -56,11 +57,14 @@ class InsertOperator : public RelationalOperator {
    * @param output_destination_index The index of the InsertDestination in the
    *        QueryContext to insert the tuple.
    * @param tuple_index The index of the tuple to insert in the QueryContext.
+   * @param The ID of the query to which this operator belongs.
    **/
   InsertOperator(const CatalogRelation &output_relation,
                  const QueryContext::insert_destination_id output_destination_index,
-                 const QueryContext::tuple_id tuple_index)
-      : output_relation_(output_relation),
+                 const QueryContext::tuple_id tuple_index,
+                 const std::size_t query_id)
+      : RelationalOperator(query_id),
+        output_relation_(output_relation),
         output_destination_index_(output_destination_index),
         tuple_index_(tuple_index),
         work_generated_(false) {}
@@ -100,12 +104,15 @@ class InsertWorkOrder : public WorkOrder {
    *
    * @note InsertWorkOrder takes ownership of \c tuple.
    *
+   * @param The ID of the query to which this operator belongs.
    * @param output_destination The InsertDestination to insert the tuple.
    * @param tuple The tuple to insert.
    **/
-  InsertWorkOrder(InsertDestination *output_destination,
+  InsertWorkOrder(const std::size_t query_id,
+                  InsertDestination *output_destination,
                   Tuple *tuple)
-      : output_destination_(DCHECK_NOTNULL(output_destination)),
+      : WorkOrder(query_id),
+        output_destination_(DCHECK_NOTNULL(output_destination)),
         tuple_(DCHECK_NOTNULL(tuple)) {}
 
   ~InsertWorkOrder() override {}

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1290dd70/relational_operators/NestedLoopsJoinOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/NestedLoopsJoinOperator.cpp b/relational_operators/NestedLoopsJoinOperator.cpp
index 5cc498b..317cc5d 100644
--- a/relational_operators/NestedLoopsJoinOperator.cpp
+++ b/relational_operators/NestedLoopsJoinOperator.cpp
@@ -82,6 +82,7 @@ bool NestedLoopsJoinOperator::getAllWorkOrders(
                                            right_block_id,
                                            query_context->getPredicate(join_predicate_index_),
                                            query_context->getScalarGroup(selection_index_),
+                                           query_id_,
                                            query_context->getInsertDestination(output_destination_index_),
                                            storage_manager),
               op_index_);
@@ -171,6 +172,7 @@ std::size_t NestedLoopsJoinOperator::getAllWorkOrdersHelperBothNotStored(WorkOrd
                                        right_relation_block_ids_[right_index],
                                        query_context->getPredicate(join_predicate_index_),
                                        query_context->getScalarGroup(selection_index_),
+                                       query_id_,
                                        query_context->getInsertDestination(output_destination_index_),
                                        storage_manager),
           op_index_);
@@ -205,6 +207,7 @@ bool NestedLoopsJoinOperator::getAllWorkOrdersHelperOneStored(WorkOrdersContaine
                 right_relation_block_ids_[right_index],
                 join_predicate,
                 selection,
+                query_id_,
                 output_destination,
                 storage_manager),
             op_index_);
@@ -224,6 +227,7 @@ bool NestedLoopsJoinOperator::getAllWorkOrdersHelperOneStored(WorkOrdersContaine
                                          right_block_id,
                                          join_predicate,
                                          selection,
+                                         query_id_,
                                          output_destination,
                                          storage_manager),
             op_index_);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1290dd70/relational_operators/NestedLoopsJoinOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/NestedLoopsJoinOperator.hpp b/relational_operators/NestedLoopsJoinOperator.hpp
index a52ca25..9d81d10 100644
--- a/relational_operators/NestedLoopsJoinOperator.hpp
+++ b/relational_operators/NestedLoopsJoinOperator.hpp
@@ -76,6 +76,7 @@ class NestedLoopsJoinOperator : public RelationalOperator {
    * @param left_relation_is_stored If left_input_relation is a stored relation.
    * @param right_relation_is_stored If right_input_relation is a stored
    *                                 relation.
+   * @param The ID of the query to which this operator belongs.
    **/
   NestedLoopsJoinOperator(const CatalogRelation &left_input_relation,
                           const CatalogRelation &right_input_relation,
@@ -84,8 +85,10 @@ class NestedLoopsJoinOperator : public RelationalOperator {
                           const QueryContext::predicate_id join_predicate_index,
                           const QueryContext::scalar_group_id selection_index,
                           bool left_relation_is_stored,
-                          bool right_relation_is_stored)
-      : left_input_relation_(left_input_relation),
+                          bool right_relation_is_stored,
+                          const std::size_t query_id)
+      : RelationalOperator(query_id),
+        left_input_relation_(left_input_relation),
         right_input_relation_(right_input_relation),
         output_relation_(output_relation),
         output_destination_index_(output_destination_index),
@@ -230,6 +233,7 @@ class NestedLoopsJoinWorkOrder : public WorkOrder {
    * @param selection A list of Scalars corresponding to the relation attributes
    *        in \c output_destination. Each Scalar is evaluated for the joined
    *        tuples, and the resulting value is inserted into the join result.
+   * @param The ID of the query to which this operator belongs.
    * @param output_destination The InsertDestination to insert the join results.
    * @param storage_manager The StorageManager to use.
    **/
@@ -239,9 +243,11 @@ class NestedLoopsJoinWorkOrder : public WorkOrder {
                            const block_id right_block_id,
                            const Predicate *join_predicate,
                            const std::vector<std::unique_ptr<const Scalar>> &selection,
+                           const std::size_t query_id,
                            InsertDestination *output_destination,
                            StorageManager *storage_manager)
-      : left_input_relation_(left_input_relation),
+      : WorkOrder(query_id),
+        left_input_relation_(left_input_relation),
         right_input_relation_(right_input_relation),
         left_block_id_(left_block_id),
         right_block_id_(right_block_id),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1290dd70/relational_operators/RebuildWorkOrder.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/RebuildWorkOrder.hpp b/relational_operators/RebuildWorkOrder.hpp
index 5443d48..88e5248 100644
--- a/relational_operators/RebuildWorkOrder.hpp
+++ b/relational_operators/RebuildWorkOrder.hpp
@@ -55,12 +55,14 @@ class RebuildWorkOrder : public WorkOrder {
    * @param input_relation_id The ID of the CatalogRelation to which the given
    *        storage block belongs to.
    * @param scheduler_client_id The TMB client ID of the scheduler thread.
+   * @param The ID of the query to which this operator belongs.
    * @param bus A pointer to the TMB.
    **/
   RebuildWorkOrder(MutableBlockReference &&block_ref,
                    const std::size_t input_operator_index,
                    const relation_id input_relation_id,
                    const client_id scheduler_client_id,
+                   const std::size_t query_id,
                    MessageBus *bus)
       : block_ref_(std::move(block_ref)),
         input_operator_index_(input_operator_index),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1290dd70/relational_operators/SampleOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/SampleOperator.cpp b/relational_operators/SampleOperator.cpp
index 6842b28..b318ce4 100644
--- a/relational_operators/SampleOperator.cpp
+++ b/relational_operators/SampleOperator.cpp
@@ -58,6 +58,7 @@ bool SampleOperator::getAllWorkOrders(
                                     input_block_id,
                                     is_block_sample_,
                                     percentage_,
+                                    query_id_,
                                     output_destination,
                                     storage_manager),
                 op_index_);
@@ -72,6 +73,7 @@ bool SampleOperator::getAllWorkOrders(
                                   input_block_id,
                                   is_block_sample_,
                                   percentage_,
+                                  query_id_,
                                   output_destination,
                                   storage_manager),
               op_index_);
@@ -89,6 +91,7 @@ bool SampleOperator::getAllWorkOrders(
                                       input_relation_block_ids_[num_workorders_generated_],
                                       is_block_sample_,
                                       percentage_,
+                                      query_id_,
                                       output_destination,
                                       storage_manager),
                   op_index_);
@@ -101,7 +104,9 @@ bool SampleOperator::getAllWorkOrders(
               new SampleWorkOrder(input_relation_,
                                   input_relation_block_ids_[num_workorders_generated_],
                                   is_block_sample_,
-                                  percentage_, output_destination,
+                                  percentage_,
+                                  query_id_,
+                                  output_destination,
                                   storage_manager),
               op_index_);
           ++num_workorders_generated_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1290dd70/relational_operators/SampleOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SampleOperator.hpp b/relational_operators/SampleOperator.hpp
index 305de34..7f88eea 100644
--- a/relational_operators/SampleOperator.hpp
+++ b/relational_operators/SampleOperator.hpp
@@ -18,6 +18,7 @@
 #ifndef QUICKSTEP_RELATIONAL_OPERATORS_SAMPLE_OPERATOR_HPP_
 #define QUICKSTEP_RELATIONAL_OPERATORS_SAMPLE_OPERATOR_HPP_
 
+#include <cstddef>
 #include <memory>
 #include <vector>
 
@@ -64,15 +65,17 @@ class SampleOperator : public RelationalOperator {
    *        workorders.
    * @param is_block_sample Flag indicating whether the sample type is block or tuple.
    * @param percentage The percentage of data to be sampled.
-   *
+   * @param The ID of the query to which this operator belongs.
    **/
   SampleOperator(const CatalogRelation &input_relation,
                  const CatalogRelationSchema &output_relation,
                  const QueryContext::insert_destination_id output_destination_index,
                  const bool input_relation_is_stored,
                  const bool is_block_sample,
-                 const int percentage)
-      : input_relation_(input_relation),
+                 const int percentage,
+                 const std::size_t query_id)
+      : RelationalOperator(query_id),
+        input_relation_(input_relation),
         output_relation_(output_relation),
         output_destination_index_(output_destination_index),
         input_relation_is_stored_(input_relation_is_stored),
@@ -134,9 +137,11 @@ class SampleWorkOrder : public WorkOrder {
                   const block_id input_block_id,
                   const bool is_block_sample,
                   const int percentage,
+                  const std::size_t query_id,
                   InsertDestination *output_destination,
                   StorageManager *storage_manager)
-      : input_relation_(input_relation),
+      : WorkOrder(query_id),
+        input_relation_(input_relation),
         input_block_id_(input_block_id),
         is_block_sample_(is_block_sample),
         percentage_(percentage),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1290dd70/relational_operators/SaveBlocksOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/SaveBlocksOperator.cpp b/relational_operators/SaveBlocksOperator.cpp
index ac61407..3581090 100644
--- a/relational_operators/SaveBlocksOperator.cpp
+++ b/relational_operators/SaveBlocksOperator.cpp
@@ -38,6 +38,7 @@ bool SaveBlocksOperator::getAllWorkOrders(
         new SaveBlocksWorkOrder(
             destination_block_ids_[num_workorders_generated_],
             force_,
+            query_id_,
             storage_manager),
         op_index_);
     ++num_workorders_generated_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1290dd70/relational_operators/SaveBlocksOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SaveBlocksOperator.hpp b/relational_operators/SaveBlocksOperator.hpp
index 49195ea..3bcc6a7 100644
--- a/relational_operators/SaveBlocksOperator.hpp
+++ b/relational_operators/SaveBlocksOperator.hpp
@@ -18,6 +18,7 @@
 #ifndef QUICKSTEP_RELATIONAL_OPERATORS_SAVE_BLOCKS_OPERATOR_HPP_
 #define QUICKSTEP_RELATIONAL_OPERATORS_SAVE_BLOCKS_OPERATOR_HPP_
 
+#include <cstddef>
 #include <vector>
 
 #include "catalog/CatalogTypedefs.hpp"
@@ -50,11 +51,13 @@ 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 force If true, force writing of all blocks to disk, otherwise only
    *        write dirty blocks.
    **/
-  explicit SaveBlocksOperator(bool force = false)
-      : force_(force),
+  explicit SaveBlocksOperator(const std::size_t query_id, bool force = false)
+      : RelationalOperator(query_id),
+        force_(force),
         num_workorders_generated_(0) {}
 
   ~SaveBlocksOperator() override {}
@@ -96,10 +99,12 @@ class SaveBlocksWorkOrder : public WorkOrder {
    * @param save_block_id The id of the block to save.
    * @param force If true, force writing of all blocks to disk, otherwise only
    *        write dirty blocks.
+   * @param query_id The ID of the query to which this operator belongs.
    * @param storage_manager The StorageManager to use.
    **/
   SaveBlocksWorkOrder(const block_id save_block_id,
                       const bool force,
+                      const std::size_t query_id,
                       StorageManager *storage_manager)
       : save_block_id_(save_block_id),
         force_(force),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1290dd70/relational_operators/SelectOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/SelectOperator.cpp b/relational_operators/SelectOperator.cpp
index 69bb434..350890d 100644
--- a/relational_operators/SelectOperator.cpp
+++ b/relational_operators/SelectOperator.cpp
@@ -49,6 +49,7 @@ void SelectOperator::addWorkOrders(WorkOrdersContainer *container,
                               simple_projection_,
                               simple_selection_,
                               selection,
+                              query_id_,
                               output_destination,
                               storage_manager),
           op_index_);
@@ -63,6 +64,7 @@ void SelectOperator::addWorkOrders(WorkOrdersContainer *container,
               simple_projection_,
               simple_selection_,
               selection,
+              query_id_,
               output_destination,
               storage_manager),
           op_index_);
@@ -91,6 +93,7 @@ void SelectOperator::addPartitionAwareWorkOrders(WorkOrdersContainer *container,
                 simple_projection_,
                 simple_selection_,
                 selection,
+                query_id_,
                 output_destination,
                 storage_manager,
                 placement_scheme_->getNUMANodeForBlock(input_block_id)),
@@ -111,6 +114,7 @@ void SelectOperator::addPartitionAwareWorkOrders(WorkOrdersContainer *container,
                 simple_projection_,
                 simple_selection_,
                 selection,
+                query_id_,
                 output_destination,
                 storage_manager,
                 placement_scheme_->getNUMANodeForBlock(block_in_partition)),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1290dd70/relational_operators/SelectOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SelectOperator.hpp b/relational_operators/SelectOperator.hpp
index 76f4cb6..4f5b8ca 100644
--- a/relational_operators/SelectOperator.hpp
+++ b/relational_operators/SelectOperator.hpp
@@ -76,14 +76,17 @@ class SelectOperator : public RelationalOperator {
    * @param input_relation_is_stored If input_relation is a stored relation and
    *        is fully available to the operator before it can start generating
    *        workorders.
+   * @param query_id The ID of the query to which this operator belongs.
    **/
   SelectOperator(const CatalogRelation &input_relation,
                  const CatalogRelation &output_relation,
                  const QueryContext::insert_destination_id output_destination_index,
                  const QueryContext::predicate_id predicate_index,
                  const QueryContext::scalar_group_id selection_index,
-                 const bool input_relation_is_stored)
-      : input_relation_(input_relation),
+                 const bool input_relation_is_stored,
+                 const std::size_t query_id)
+      : RelationalOperator(query_id),
+        input_relation_(input_relation),
         output_relation_(output_relation),
         output_destination_index_(output_destination_index),
         predicate_index_(predicate_index),
@@ -133,14 +136,17 @@ class SelectOperator : public RelationalOperator {
    * @param input_relation_is_stored If input_relation is a stored relation and
    *        is fully available to the operator before it can start generating
    *        workorders.
+   * @param query_id The ID of the query to which this operator belongs.
    **/
   SelectOperator(const CatalogRelation &input_relation,
                  const CatalogRelation &output_relation,
                  const QueryContext::insert_destination_id output_destination_index,
                  const QueryContext::predicate_id predicate_index,
                  std::vector<attribute_id> &&selection,
-                 const bool input_relation_is_stored)
-      : input_relation_(input_relation),
+                 const bool input_relation_is_stored,
+                 const std::size_t query_id)
+      : RelationalOperator(query_id),
+        input_relation_(input_relation),
         output_relation_(output_relation),
         output_destination_index_(output_destination_index),
         predicate_index_(predicate_index),
@@ -281,6 +287,7 @@ class SelectWorkOrder : public WorkOrder {
    *        simple_projection is true.
    * @param selection A list of Scalars which will be evaluated to project
    *        input tuples, used if \c simple_projection is false.
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to insert the selection
    *        results.
    * @param storage_manager The StorageManager to use.
@@ -291,10 +298,12 @@ class SelectWorkOrder : public WorkOrder {
                   const bool simple_projection,
                   const std::vector<attribute_id> &simple_selection,
                   const std::vector<std::unique_ptr<const Scalar>> *selection,
+                  const std::size_t query_id,
                   InsertDestination *output_destination,
                   StorageManager *storage_manager,
                   const numa_node_id numa_node = 0)
-      : input_relation_(input_relation),
+      : WorkOrder(query_id),
+        input_relation_(input_relation),
         input_block_id_(input_block_id),
         predicate_(predicate),
         simple_projection_(simple_projection),
@@ -320,6 +329,7 @@ class SelectWorkOrder : public WorkOrder {
    *        simple_projection is true.
    * @param selection A list of Scalars which will be evaluated to project
    *        input tuples, used if \c simple_projection is false.
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to insert the selection
    *        results.
    * @param storage_manager The StorageManager to use.
@@ -330,10 +340,12 @@ class SelectWorkOrder : public WorkOrder {
                   const bool simple_projection,
                   std::vector<attribute_id> &&simple_selection,
                   const std::vector<std::unique_ptr<const Scalar>> *selection,
+                  const std::size_t query_id,
                   InsertDestination *output_destination,
                   StorageManager *storage_manager,
                   const numa_node_id numa_node = 0)
-      : input_relation_(input_relation),
+      : WorkOrder(query_id),
+        input_relation_(input_relation),
         input_block_id_(input_block_id),
         predicate_(predicate),
         simple_projection_(simple_projection),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1290dd70/relational_operators/SortMergeRunOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/SortMergeRunOperator.cpp b/relational_operators/SortMergeRunOperator.cpp
index 7427d44..9db8de1 100644
--- a/relational_operators/SortMergeRunOperator.cpp
+++ b/relational_operators/SortMergeRunOperator.cpp
@@ -90,6 +90,7 @@ WorkOrder *SortMergeRunOperator::createWorkOrder(
       std::move(job->runs),
       top_k_,
       job->level,
+      query_id_,
       output_destination,
       storage_manager,
       op_index_,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1290dd70/relational_operators/SortMergeRunOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SortMergeRunOperator.hpp b/relational_operators/SortMergeRunOperator.hpp
index f92affe..f54e925 100644
--- a/relational_operators/SortMergeRunOperator.hpp
+++ b/relational_operators/SortMergeRunOperator.hpp
@@ -88,6 +88,7 @@ class SortMergeRunOperator : public RelationalOperator {
    *              \c top_k is 0.
    * @param input_relation_is_stored Boolean to indicate is input relation is
    *                                 stored or streamed.
+   * @param query_id The ID of the query to which this operator belongs.
    **/
   SortMergeRunOperator(const CatalogRelation &input_relation,
                        const CatalogRelation &output_relation,
@@ -97,8 +98,10 @@ class SortMergeRunOperator : public RelationalOperator {
                        const QueryContext::sort_config_id sort_config_index,
                        const std::size_t merge_factor,
                        const std::size_t top_k,
-                       const bool input_relation_is_stored)
-      : input_relation_(input_relation),
+                       const bool input_relation_is_stored,
+                       const std::size_t query_id)
+      : RelationalOperator(query_id),
+        input_relation_(input_relation),
         output_relation_(output_relation),
         output_destination_index_(output_destination_index),
         sort_config_index_(sort_config_index),
@@ -216,6 +219,7 @@ class SortMergeRunWorkOrder : public WorkOrder {
    * @param input_runs Input runs to merge.
    * @param top_k If non-zero will merge only \c top_k tuples.
    * @param merge_level Merge level in the merge tree.
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to create new blocks.
    * @param storage_manager The StorageManager to use.
    * @param operator_index Merge-run operator index to send feedback messages
@@ -229,12 +233,14 @@ class SortMergeRunWorkOrder : public WorkOrder {
       std::vector<merge_run_operator::Run> &&input_runs,
       const std::size_t top_k,
       const std::size_t merge_level,
+      const std::size_t query_id,
       InsertDestination *output_destination,
       StorageManager *storage_manager,
       const std::size_t operator_index,
       const tmb::client_id scheduler_client_id,
       MessageBus *bus)
-      : sort_config_(sort_config),
+      : WorkOrder(query_id),
+        sort_config_(sort_config),
         run_relation_(run_relation),
         input_runs_(std::move(input_runs)),
         top_k_(top_k),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1290dd70/relational_operators/SortRunGenerationOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/SortRunGenerationOperator.cpp b/relational_operators/SortRunGenerationOperator.cpp
index 9bb3f51..e352f9e 100644
--- a/relational_operators/SortRunGenerationOperator.cpp
+++ b/relational_operators/SortRunGenerationOperator.cpp
@@ -54,6 +54,7 @@ bool SortRunGenerationOperator::getAllWorkOrders(
             new SortRunGenerationWorkOrder(input_relation_,
                                            input_block_id,
                                            sort_config,
+                                           query_id_,
                                            output_destination,
                                            storage_manager),
             op_index_);
@@ -69,6 +70,7 @@ bool SortRunGenerationOperator::getAllWorkOrders(
               input_relation_,
               input_relation_block_ids_[num_workorders_generated_],
               sort_config,
+              query_id_,
               output_destination,
               storage_manager),
           op_index_);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1290dd70/relational_operators/SortRunGenerationOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SortRunGenerationOperator.hpp b/relational_operators/SortRunGenerationOperator.hpp
index 04290a9..7618790 100644
--- a/relational_operators/SortRunGenerationOperator.hpp
+++ b/relational_operators/SortRunGenerationOperator.hpp
@@ -83,13 +83,16 @@ class SortRunGenerationOperator : public RelationalOperator {
    * @param input_relation_is_stored Does the input relation contain the blocks
    *                                 to sort. If \c false, the blocks are
    *                                 streamed.
+   * @param query_id The ID of the query to which this operator belongs.
    **/
   SortRunGenerationOperator(const CatalogRelation &input_relation,
                             const CatalogRelation &output_relation,
                             const QueryContext::insert_destination_id output_destination_index,
                             const QueryContext::sort_config_id sort_config_index,
-                            bool input_relation_is_stored)
-      : input_relation_(input_relation),
+                            bool input_relation_is_stored,
+                            const std::size_t query_id)
+      : RelationalOperator(query_id),
+        input_relation_(input_relation),
         output_relation_(output_relation),
         output_destination_index_(output_destination_index),
         sort_config_index_(sort_config_index),
@@ -152,6 +155,7 @@ class SortRunGenerationWorkOrder : public WorkOrder {
    * @param input_block_id The block id.
    * @param sort_config The Sort configuration specifying ORDER BY, ordering,
    *        and null ordering.
+   * @param query_id The ID of the query to which this operator belongs.
    * @param output_destination The InsertDestination to store the sorted blocks
    *        of runs.
    * @param storage_manager The StorageManager to use.
@@ -159,9 +163,11 @@ class SortRunGenerationWorkOrder : public WorkOrder {
   SortRunGenerationWorkOrder(const CatalogRelationSchema &input_relation,
                              const block_id input_block_id,
                              const SortConfiguration &sort_config,
+                             const std::size_t query_id,
                              InsertDestination *output_destination,
                              StorageManager *storage_manager)
-      : input_relation_(input_relation),
+      : WorkOrder(query_id),
+        input_relation_(input_relation),
         input_block_id_(input_block_id),
         sort_config_(sort_config),
         output_destination_(DCHECK_NOTNULL(output_destination)),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1290dd70/relational_operators/TableGeneratorOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/TableGeneratorOperator.cpp b/relational_operators/TableGeneratorOperator.cpp
index 886d05f..fb1f743 100644
--- a/relational_operators/TableGeneratorOperator.cpp
+++ b/relational_operators/TableGeneratorOperator.cpp
@@ -42,8 +42,11 @@ bool TableGeneratorOperator::getAllWorkOrders(
     // Currently the generator function is not abstracted to be parallelizable,
     // so just produce one work order.
     container->addNormalWorkOrder(
-        new TableGeneratorWorkOrder(query_context->getGeneratorFunctionHandle(generator_function_index_),
-                                    query_context->getInsertDestination(output_destination_index_)),
+        new TableGeneratorWorkOrder(
+            query_context->getGeneratorFunctionHandle(
+                generator_function_index_),
+            query_id_,
+            query_context->getInsertDestination(output_destination_index_)),
         op_index_);
     started_ = true;
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1290dd70/relational_operators/TableGeneratorOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/TableGeneratorOperator.hpp b/relational_operators/TableGeneratorOperator.hpp
index a26b227..27efcad 100644
--- a/relational_operators/TableGeneratorOperator.hpp
+++ b/relational_operators/TableGeneratorOperator.hpp
@@ -60,12 +60,14 @@ class TableGeneratorOperator : public RelationalOperator {
    *        QueryContext to insert the generated output.
    * @param generator_function_index The index of the GeneratorFunctionHandle in
    *        the QueryContext.
-   *
+   * @param query_id The ID of the query to which this operator belongs.
    **/
   TableGeneratorOperator(const CatalogRelation &output_relation,
                          const QueryContext::insert_destination_id output_destination_index,
-                         const QueryContext::generator_function_id generator_function_index)
-      : output_relation_(output_relation),
+                         const QueryContext::generator_function_id generator_function_index,
+                         const std::size_t query_id)
+      : RelationalOperator(query_id),
+        output_relation_(output_relation),
         output_destination_index_(output_destination_index),
         generator_function_index_(generator_function_index),
         started_(false) {
@@ -112,12 +114,15 @@ class TableGeneratorWorkOrder : public WorkOrder {
    * @brief Constructor.
    *
    * @param generator_function The GeneratorFunctionHandle to use.
+   * @param query_id The ID of the query to which this operator belongs.
    * @param output_destination The InsertDestination to insert the generated
    *        output.
    **/
   TableGeneratorWorkOrder(const GeneratorFunctionHandle &function_handle,
+                          const std::size_t query_id,
                           InsertDestination *output_destination)
-      : function_handle_(function_handle),
+      : WorkOrder(query_id),
+        function_handle_(function_handle),
         output_destination_(DCHECK_NOTNULL(output_destination)) {}
 
   ~TableGeneratorWorkOrder() override {}

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1290dd70/relational_operators/TextScanOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/TextScanOperator.cpp b/relational_operators/TextScanOperator.cpp
index 5ede6f7..8db5ef1 100644
--- a/relational_operators/TextScanOperator.cpp
+++ b/relational_operators/TextScanOperator.cpp
@@ -167,6 +167,7 @@ bool TextScanOperator::getAllWorkOrders(
           container->addNormalWorkOrder(
               new TextSplitWorkOrder(file,
                                      process_escape_sequences_,
+                                     query_id_,
                                      storage_manager,
                                      op_index_,
                                      scheduler_client_id,
@@ -185,6 +186,7 @@ bool TextScanOperator::getAllWorkOrders(
                                     blob_work.size,
                                     field_terminator_,
                                     process_escape_sequences_,
+                                    query_id_,
                                     output_destination,
                                     storage_manager),
               op_index_);
@@ -204,6 +206,7 @@ bool TextScanOperator::getAllWorkOrders(
             new TextScanWorkOrder(file,
                                   field_terminator_,
                                   process_escape_sequences_,
+                                  query_id_,
                                   output_destination,
                                   storage_manager),
             op_index_);
@@ -235,9 +238,11 @@ void TextScanOperator::receiveFeedbackMessage(const WorkOrder::FeedbackMessage &
 TextScanWorkOrder::TextScanWorkOrder(const std::string &filename,
                                      const char field_terminator,
                                      const bool process_escape_sequences,
+                                     const std::size_t query_id,
                                      InsertDestination *output_destination,
                                      StorageManager *storage_manager)
-    : is_file_(true),
+    : WorkOrder(query_id),
+      is_file_(true),
       filename_(filename),
       field_terminator_(field_terminator),
       text_blob_(0),
@@ -253,9 +258,11 @@ TextScanWorkOrder::TextScanWorkOrder(const block_id text_blob,
                                      const std::size_t text_size,
                                      const char field_terminator,
                                      const bool process_escape_sequences,
+                                     const std::size_t query_id,
                                      InsertDestination *output_destination,
                                      StorageManager *storage_manager)
-    : is_file_(false),
+    : WorkOrder(query_id),
+      is_file_(false),
       field_terminator_(field_terminator),
       text_blob_(text_blob),
       text_size_(text_size),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1290dd70/relational_operators/TextScanOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/TextScanOperator.hpp b/relational_operators/TextScanOperator.hpp
index a2d4ced..ad03f6a 100644
--- a/relational_operators/TextScanOperator.hpp
+++ b/relational_operators/TextScanOperator.hpp
@@ -134,14 +134,17 @@ class TextScanOperator : public RelationalOperator {
    * @param output_relation The output relation.
    * @param output_destination_index The index of the InsertDestination in the
    *        QueryContext to insert tuples.
+   * @param query_id The ID of the query to which this operator belongs.
    **/
   TextScanOperator(const std::string &file_pattern,
                    const char field_terminator,
                    const bool process_escape_sequences,
                    const bool parallelize_load,
                    const CatalogRelation &output_relation,
-                   const QueryContext::insert_destination_id output_destination_index)
-      : file_pattern_(file_pattern),
+                   const QueryContext::insert_destination_id output_destination_index,
+                   const std::size_t query_id)
+      : RelationalOperator(query_id),
+        file_pattern_(file_pattern),
         field_terminator_(field_terminator),
         process_escape_sequences_(process_escape_sequences),
         parallelize_load_(parallelize_load),
@@ -202,6 +205,7 @@ class TextScanWorkOrder : public WorkOrder {
    *        the text file.
    * @param process_escape_sequences Whether to decode escape sequences in the
    *        text file.
+   * @param query_id The ID of the query to which this operator belongs.
    * @param output_destination The InsertDestination to insert tuples.
    * @param storage_manager The StorageManager to use.
    **/
@@ -209,6 +213,7 @@ class TextScanWorkOrder : public WorkOrder {
       const std::string &filename,
       const char field_terminator,
       const bool process_escape_sequences,
+      const std::size_t query_id,
       InsertDestination *output_destination,
       StorageManager *storage_manager);
 
@@ -221,6 +226,7 @@ class TextScanWorkOrder : public WorkOrder {
    *        the text file.
    * @param process_escape_sequences Whether to decode escape sequences in the
    *        text file.
+   * @param query_id The ID of the query to which this operator belongs.
    * @param output_destination The InsertDestination to write the read tuples.
    * @param storage_manager The StorageManager to use.
    */
@@ -229,6 +235,7 @@ class TextScanWorkOrder : public WorkOrder {
       const std::size_t text_size,
       const char field_terminator,
       const bool process_escape_sequences,
+      const std::size_t query_id,
       InsertDestination *output_destination,
       StorageManager *storage_manager);
 
@@ -318,6 +325,7 @@ class TextSplitWorkOrder : public WorkOrder {
    * @param filename File to split into row-aligned blobs.
    * @param process_escape_sequences Whether to decode escape sequences in the
    *        text file.
+   * @param query_id The ID of the query to which this operator belongs.
    * @param storage_manager The StorageManager to use.
    * @param operator_index Operator index of the current operator. This is used
    *                       to send new-work available message to Foreman.
@@ -326,11 +334,13 @@ class TextSplitWorkOrder : public WorkOrder {
    */
   TextSplitWorkOrder(const std::string &filename,
                      const bool process_escape_sequences,
+                     const std::size_t query_id,
                      StorageManager *storage_manager,
                      const std::size_t operator_index,
                      const tmb::client_id scheduler_client_id,
                      MessageBus *bus)
-      : filename_(filename),
+      : WorkOrder(query_id),
+        filename_(filename),
         process_escape_sequences_(process_escape_sequences),
         storage_manager_(DCHECK_NOTNULL(storage_manager)),
         operator_index_(operator_index),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1290dd70/relational_operators/UpdateOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/UpdateOperator.cpp b/relational_operators/UpdateOperator.cpp
index 7585db1..b331a9c 100644
--- a/relational_operators/UpdateOperator.cpp
+++ b/relational_operators/UpdateOperator.cpp
@@ -57,6 +57,7 @@ bool UpdateOperator::getAllWorkOrders(
                               input_block_id,
                               query_context->getPredicate(predicate_index_),
                               query_context->getUpdateGroup(update_group_index_),
+                              query_id_,
                               query_context->getInsertDestination(relocation_destination_index_),
                               storage_manager,
                               op_index_,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1290dd70/relational_operators/UpdateOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/UpdateOperator.hpp b/relational_operators/UpdateOperator.hpp
index 78f8fe0..f1de0eb 100644
--- a/relational_operators/UpdateOperator.hpp
+++ b/relational_operators/UpdateOperator.hpp
@@ -72,6 +72,7 @@ class UpdateOperator : public RelationalOperator {
    * @param update_group_index The index of a update group (the map of
    *        attribute_ids to Scalars) which should be evaluated to get the new
    *        value for the corresponding attribute.
+   * @param query_id The ID of the query to which this operator belongs.
    *
    * @warning The constructed InsertDestination should belong to relation, but
    *          must NOT contain any pre-existing blocks.
@@ -79,8 +80,10 @@ class UpdateOperator : public RelationalOperator {
   UpdateOperator(const CatalogRelation &relation,
                  const QueryContext::insert_destination_id relocation_destination_index,
                  const QueryContext::predicate_id predicate_index,
-                 const QueryContext::update_group_id update_group_index)
-      : relation_(relation),
+                 const QueryContext::update_group_id update_group_index,
+                 const std::size_t query_id)
+      : RelationalOperator(query_id),
+        relation_(relation),
         relocation_destination_index_(relocation_destination_index),
         predicate_index_(predicate_index),
         update_group_index_(update_group_index),
@@ -130,6 +133,7 @@ class UpdateWorkOrder : public WorkOrder {
    * @param assignments The assignments (the map of attribute_ids to Scalars)
    *        which should be evaluated to get the new value for the corresponding
    *        attribute.
+   * @param query_id The ID of the query to which this operator belongs.
    * @param input_block_id The block id.
    * @param relocation_destination The InsertDestination to relocate tuples
    *        which can not be updated in-place.
@@ -143,12 +147,14 @@ class UpdateWorkOrder : public WorkOrder {
                   const block_id input_block_id,
                   const Predicate *predicate,
                   const std::unordered_map<attribute_id, std::unique_ptr<const Scalar>> &assignments,
+                  const std::size_t query_id,
                   InsertDestination *relocation_destination,
                   StorageManager *storage_manager,
                   const std::size_t update_operator_index,
                   const tmb::client_id scheduler_client_id,
                   MessageBus *bus)
-      : relation_(relation),
+      : WorkOrder(query_id),
+        relation_(relation),
         input_block_id_(input_block_id),
         predicate_(predicate),
         assignments_(assignments),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1290dd70/relational_operators/WorkOrderFactory.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrderFactory.cpp b/relational_operators/WorkOrderFactory.cpp
index 9828ab9..fdd694f 100644
--- a/relational_operators/WorkOrderFactory.cpp
+++ b/relational_operators/WorkOrderFactory.cpp
@@ -130,6 +130,7 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
       }
 
       return new DropTableWorkOrder(
+          proto.query_id(),
           move(blocks),
           storage_manager,
           proto.HasExtension(serialization::DropTableWorkOrder::relation_id)
@@ -140,10 +141,12 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
     case serialization::FINALIZE_AGGREGATION: {
       LOG(INFO) << "Creating FinalizeAggregationWorkOrder";
       return new FinalizeAggregationWorkOrder(
-          query_context->releaseAggregationState(
-              proto.GetExtension(serialization::FinalizeAggregationWorkOrder::aggr_state_index)),
+          proto.query_id(),
+          query_context->releaseAggregationState(proto.GetExtension(
+              serialization::FinalizeAggregationWorkOrder::aggr_state_index)),
           query_context->getInsertDestination(
-              proto.GetExtension(serialization::FinalizeAggregationWorkOrder::insert_destination_index)));
+              proto.GetExtension(serialization::FinalizeAggregationWorkOrder::
+                                     insert_destination_index)));
     }
     case serialization::HASH_JOIN: {
       const auto hash_join_work_order_type =
@@ -262,6 +265,7 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
     case serialization::INSERT: {
       LOG(INFO) << "Creating InsertWorkOrder";
       return new InsertWorkOrder(
+          proto.query_id(),
           query_context->getInsertDestination(
               proto.GetExtension(serialization::InsertWorkOrder::insert_destination_index)),
           query_context->releaseTuple(
@@ -280,6 +284,7 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
               proto.GetExtension(serialization::NestedLoopsJoinWorkOrder::join_predicate_index)),
           query_context->getScalarGroup(
               proto.GetExtension(serialization::NestedLoopsJoinWorkOrder::selection_index)),
+          proto.query_id(),
           query_context->getInsertDestination(
               proto.GetExtension(serialization::NestedLoopsJoinWorkOrder::insert_destination_index)),
           storage_manager);
@@ -292,6 +297,7 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
           proto.GetExtension(serialization::SampleWorkOrder::block_id),
           proto.GetExtension(serialization::SampleWorkOrder::is_block_sample),
           proto.GetExtension(serialization::SampleWorkOrder::percentage),
+          proto.query_id(),
           query_context->getInsertDestination(
               proto.GetExtension(serialization::SampleWorkOrder::insert_destination_index)),
           storage_manager);
@@ -301,6 +307,7 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
       return new SaveBlocksWorkOrder(
           proto.GetExtension(serialization::SaveBlocksWorkOrder::block_id),
           proto.GetExtension(serialization::SaveBlocksWorkOrder::force),
+          proto.query_id(),
           storage_manager);
     }
     case serialization::SELECT: {
@@ -324,6 +331,7 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
           simple_projection ? nullptr
                             : &query_context->getScalarGroup(
                                   proto.GetExtension(serialization::SelectWorkOrder::selection_index)),
+          proto.query_id(),
           query_context->getInsertDestination(
               proto.GetExtension(serialization::SelectWorkOrder::insert_destination_index)),
           storage_manager);
@@ -349,6 +357,7 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
           move(runs),
           proto.GetExtension(serialization::SortMergeRunWorkOrder::top_k),
           proto.GetExtension(serialization::SortMergeRunWorkOrder::merge_level),
+          proto.query_id(),
           query_context->getInsertDestination(
               proto.GetExtension(serialization::SortMergeRunWorkOrder::insert_destination_index)),
           storage_manager,
@@ -364,6 +373,7 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
           proto.GetExtension(serialization::SortRunGenerationWorkOrder::block_id),
           query_context->getSortConfig(
               proto.GetExtension(serialization::SortRunGenerationWorkOrder::sort_config_index)),
+          proto.query_id(),
           query_context->getInsertDestination(
               proto.GetExtension(serialization::SortRunGenerationWorkOrder::insert_destination_index)),
           storage_manager);
@@ -373,6 +383,7 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
       return new TableGeneratorWorkOrder(
           query_context->getGeneratorFunctionHandle(
               proto.GetExtension(serialization::TableGeneratorWorkOrder::generator_function_index)),
+          proto.query_id(),
           query_context->getInsertDestination(
               proto.GetExtension(serialization::TableGeneratorWorkOrder::insert_destination_index)));
     }
@@ -383,6 +394,7 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
             proto.GetExtension(serialization::TextScanWorkOrder::filename),
             proto.GetExtension(serialization::TextScanWorkOrder::field_terminator),
             proto.GetExtension(serialization::TextScanWorkOrder::process_escape_sequences),
+            proto.query_id(),
             query_context->getInsertDestination(
                 proto.GetExtension(serialization::TextScanWorkOrder::insert_destination_index)),
             storage_manager);
@@ -395,6 +407,7 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
             text_blob_proto.size(),
             proto.GetExtension(serialization::TextScanWorkOrder::field_terminator),
             proto.GetExtension(serialization::TextScanWorkOrder::process_escape_sequences),
+            proto.query_id(),
             query_context->getInsertDestination(
                 proto.GetExtension(serialization::TextScanWorkOrder::insert_destination_index)),
             storage_manager);
@@ -405,6 +418,7 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
       return new TextSplitWorkOrder(
           proto.GetExtension(serialization::TextSplitWorkOrder::filename),
           proto.GetExtension(serialization::TextSplitWorkOrder::process_escape_sequences),
+          proto.query_id(),
           storage_manager,
           proto.GetExtension(serialization::TextSplitWorkOrder::operator_index),
           shiftboss_client_id,
@@ -420,6 +434,7 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
               proto.GetExtension(serialization::UpdateWorkOrder::predicate_index)),
           query_context->getUpdateGroup(
               proto.GetExtension(serialization::UpdateWorkOrder::update_group_index)),
+          proto.query_id(),
           query_context->getInsertDestination(
               proto.GetExtension(serialization::UpdateWorkOrder::insert_destination_index)),
           storage_manager,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1290dd70/relational_operators/tests/AggregationOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/AggregationOperator_unittest.cpp b/relational_operators/tests/AggregationOperator_unittest.cpp
index 2c408df..ace7951 100644
--- a/relational_operators/tests/AggregationOperator_unittest.cpp
+++ b/relational_operators/tests/AggregationOperator_unittest.cpp
@@ -281,8 +281,10 @@ class AggregationOperatorTest : public ::testing::Test {
     insert_destination_proto->set_relation_id(result_table_->getID());
     insert_destination_proto->set_relational_op_index(kOpIndex);
 
-    finalize_op_.reset(
-        new FinalizeAggregationOperator(aggr_state_index, *result_table_, insert_destination_index));
+    finalize_op_.reset(new FinalizeAggregationOperator(aggr_state_index,
+                                                       *result_table_,
+                                                       insert_destination_index,
+                                                       0 /* dummy query ID */));
 
     // Set up the QueryContext.
     query_context_.reset(new QueryContext(query_context_proto,
@@ -363,8 +365,10 @@ class AggregationOperatorTest : public ::testing::Test {
     insert_destination_proto->set_relation_id(result_table_->getID());
     insert_destination_proto->set_relational_op_index(kOpIndex);
 
-    finalize_op_.reset(
-        new FinalizeAggregationOperator(aggr_state_index, *result_table_, insert_destination_index));
+    finalize_op_.reset(new FinalizeAggregationOperator(aggr_state_index,
+                                                       *result_table_,
+                                                       insert_destination_index,
+                                                       0 /* dummy query ID */));
 
     // Set up the QueryContext.
     query_context_.reset(new QueryContext(query_context_proto,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1290dd70/relational_operators/tests/SortMergeRunOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/SortMergeRunOperator_unittest.cpp b/relational_operators/tests/SortMergeRunOperator_unittest.cpp
index 50c508d..244091f 100644
--- a/relational_operators/tests/SortMergeRunOperator_unittest.cpp
+++ b/relational_operators/tests/SortMergeRunOperator_unittest.cpp
@@ -1573,7 +1573,8 @@ class SortMergeRunOperatorTest : public ::testing::Test {
                                              sort_config_index,
                                              merge_factor,
                                              top_k,
-                                             true));
+                                             true,
+                                             0  /* dummy query ID */));
     merge_op_->setOperatorIndex(kOpIndex);
 
     // Set up the QueryContext.
@@ -1616,7 +1617,8 @@ class SortMergeRunOperatorTest : public ::testing::Test {
                                              sort_config_index,
                                              merge_factor,
                                              top_k,
-                                             false));
+                                             false,
+                                             0  /* dummy query ID */));
     merge_op_->setOperatorIndex(kOpIndex);
 
     // Set up the QueryContext.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1290dd70/relational_operators/tests/SortRunGenerationOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/SortRunGenerationOperator_unittest.cpp b/relational_operators/tests/SortRunGenerationOperator_unittest.cpp
index 7491778..6f24b92 100644
--- a/relational_operators/tests/SortRunGenerationOperator_unittest.cpp
+++ b/relational_operators/tests/SortRunGenerationOperator_unittest.cpp
@@ -358,7 +358,8 @@ class SortRunGenerationOperatorTest : public ::testing::Test {
                                       *result_table_,
                                       insert_destination_index,
                                       sort_config_index,
-                                      true /* is_stored */));
+                                      true /* is_stored */,
+                                      0  /* dummy query ID */));
     run_gen->setOperatorIndex(kOpIndex);
 
     // Set up the QueryContext.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1290dd70/relational_operators/tests/TextScanOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/TextScanOperator_unittest.cpp b/relational_operators/tests/TextScanOperator_unittest.cpp
index 1dfad7b..7626686 100644
--- a/relational_operators/tests/TextScanOperator_unittest.cpp
+++ b/relational_operators/tests/TextScanOperator_unittest.cpp
@@ -193,7 +193,8 @@ TEST_F(TextScanOperatorTest, ScanTest) {
                            true,
                            false,
                            *relation_,
-                           output_destination_index));
+                           output_destination_index,
+                           0  /* dummy query ID */));
 
   // Setup query_context_.
   query_context_.reset(new QueryContext(query_context_proto,


[14/16] incubator-quickstep git commit: Reordered query ID in operators and work orders.

Posted by hb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d67f61e1/relational_operators/SampleOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SampleOperator.hpp b/relational_operators/SampleOperator.hpp
index f65f28a..3372da9 100644
--- a/relational_operators/SampleOperator.hpp
+++ b/relational_operators/SampleOperator.hpp
@@ -56,6 +56,7 @@ class SampleOperator : public RelationalOperator {
   /**
    * @brief Constructor for SampleOperator  with the sampling percentage and type of sampling.
    *
+   * @param query_id The ID of the query to which this operator belongs.
    * @param input_relation The relation to perform sampling over.
    * @param output_relation The output relation.
    * @param output_destination_index The index of the InsertDestination in the
@@ -65,15 +66,15 @@ class SampleOperator : public RelationalOperator {
    *        workorders.
    * @param is_block_sample Flag indicating whether the sample type is block or tuple.
    * @param percentage The percentage of data to be sampled.
-   * @param query_id The ID of the query to which this operator belongs.
    **/
-  SampleOperator(const CatalogRelation &input_relation,
-                 const CatalogRelationSchema &output_relation,
-                 const QueryContext::insert_destination_id output_destination_index,
-                 const bool input_relation_is_stored,
-                 const bool is_block_sample,
-                 const int percentage,
-                 const std::size_t query_id)
+  SampleOperator(
+      const std::size_t query_id,
+      const CatalogRelation &input_relation,
+      const CatalogRelationSchema &output_relation,
+      const QueryContext::insert_destination_id output_destination_index,
+      const bool input_relation_is_stored,
+      const bool is_block_sample,
+      const int percentage)
       : RelationalOperator(query_id),
         input_relation_(input_relation),
         output_relation_(output_relation),
@@ -81,8 +82,9 @@ class SampleOperator : public RelationalOperator {
         input_relation_is_stored_(input_relation_is_stored),
         is_block_sample_(is_block_sample),
         percentage_(percentage),
-        input_relation_block_ids_(input_relation_is_stored ? input_relation.getBlocksSnapshot()
-                                                           : std::vector<block_id>()),
+        input_relation_block_ids_(input_relation_is_stored
+                                      ? input_relation.getBlocksSnapshot()
+                                      : std::vector<block_id>()),
         num_workorders_generated_(0),
         started_(false) {}
 
@@ -133,11 +135,11 @@ class SampleOperator : public RelationalOperator {
  **/
 class SampleWorkOrder : public WorkOrder {
  public:
-  SampleWorkOrder(const CatalogRelationSchema &input_relation,
+  SampleWorkOrder(const std::size_t query_id,
+                  const CatalogRelationSchema &input_relation,
                   const block_id input_block_id,
                   const bool is_block_sample,
                   const int percentage,
-                  const std::size_t query_id,
                   InsertDestination *output_destination,
                   StorageManager *storage_manager)
       : WorkOrder(query_id),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d67f61e1/relational_operators/SaveBlocksOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/SaveBlocksOperator.cpp b/relational_operators/SaveBlocksOperator.cpp
index 3581090..8127d88 100644
--- a/relational_operators/SaveBlocksOperator.cpp
+++ b/relational_operators/SaveBlocksOperator.cpp
@@ -36,9 +36,9 @@ bool SaveBlocksOperator::getAllWorkOrders(
   while (num_workorders_generated_ < destination_block_ids_.size()) {
     container->addNormalWorkOrder(
         new SaveBlocksWorkOrder(
+            query_id_,
             destination_block_ids_[num_workorders_generated_],
             force_,
-            query_id_,
             storage_manager),
         op_index_);
     ++num_workorders_generated_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d67f61e1/relational_operators/SaveBlocksOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SaveBlocksOperator.hpp b/relational_operators/SaveBlocksOperator.hpp
index f4650bb..21cce2e 100644
--- a/relational_operators/SaveBlocksOperator.hpp
+++ b/relational_operators/SaveBlocksOperator.hpp
@@ -96,15 +96,15 @@ class SaveBlocksWorkOrder : public WorkOrder {
   /**
    * @brief Constructor.
    *
+   * @param query_id The ID of the query to which this operator belongs.
    * @param save_block_id The id of the block to save.
    * @param force If true, force writing of all blocks to disk, otherwise only
    *        write dirty blocks.
-   * @param query_id The ID of the query to which this operator belongs.
    * @param storage_manager The StorageManager to use.
    **/
-  SaveBlocksWorkOrder(const block_id save_block_id,
+  SaveBlocksWorkOrder(const std::size_t query_id,
+                      const block_id save_block_id,
                       const bool force,
-                      const std::size_t query_id,
                       StorageManager *storage_manager)
       : WorkOrder(query_id),
         save_block_id_(save_block_id),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d67f61e1/relational_operators/SelectOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/SelectOperator.cpp b/relational_operators/SelectOperator.cpp
index 350890d..e9a96f3 100644
--- a/relational_operators/SelectOperator.cpp
+++ b/relational_operators/SelectOperator.cpp
@@ -42,29 +42,28 @@ void SelectOperator::addWorkOrders(WorkOrdersContainer *container,
                                    InsertDestination *output_destination) {
   if (input_relation_is_stored_) {
     for (const block_id input_block_id : input_relation_block_ids_) {
-      container->addNormalWorkOrder(
-          new SelectWorkOrder(input_relation_,
-                              input_block_id,
-                              predicate,
-                              simple_projection_,
-                              simple_selection_,
-                              selection,
-                              query_id_,
-                              output_destination,
-                              storage_manager),
-          op_index_);
+      container->addNormalWorkOrder(new SelectWorkOrder(query_id_,
+                                                        input_relation_,
+                                                        input_block_id,
+                                                        predicate,
+                                                        simple_projection_,
+                                                        simple_selection_,
+                                                        selection,
+                                                        output_destination,
+                                                        storage_manager),
+                                    op_index_);
     }
   } else {
     while (num_workorders_generated_ < input_relation_block_ids_.size()) {
       container->addNormalWorkOrder(
           new SelectWorkOrder(
+              query_id_,
               input_relation_,
               input_relation_block_ids_[num_workorders_generated_],
               predicate,
               simple_projection_,
               simple_selection_,
               selection,
-              query_id_,
               output_destination,
               storage_manager),
           op_index_);
@@ -87,13 +86,13 @@ void SelectOperator::addPartitionAwareWorkOrders(WorkOrdersContainer *container,
            input_relation_block_ids_in_partition_[part_id]) {
         container->addNormalWorkOrder(
             new SelectWorkOrder(
+                query_id_,
                 input_relation_,
                 input_block_id,
                 predicate,
                 simple_projection_,
                 simple_selection_,
                 selection,
-                query_id_,
                 output_destination,
                 storage_manager,
                 placement_scheme_->getNUMANodeForBlock(input_block_id)),
@@ -108,13 +107,13 @@ void SelectOperator::addPartitionAwareWorkOrders(WorkOrdersContainer *container,
             = input_relation_block_ids_in_partition_[part_id][num_workorders_generated_in_partition_[part_id]];
         container->addNormalWorkOrder(
             new SelectWorkOrder(
+                query_id_,
                 input_relation_,
                 block_in_partition,
                 predicate,
                 simple_projection_,
                 simple_selection_,
                 selection,
-                query_id_,
                 output_destination,
                 storage_manager,
                 placement_scheme_->getNUMANodeForBlock(block_in_partition)),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d67f61e1/relational_operators/SelectOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SelectOperator.hpp b/relational_operators/SelectOperator.hpp
index 4f5b8ca..ac7b038 100644
--- a/relational_operators/SelectOperator.hpp
+++ b/relational_operators/SelectOperator.hpp
@@ -64,6 +64,7 @@ class SelectOperator : public RelationalOperator {
    * @brief Constructor for selection with arbitrary expressions in projection
    *        list.
    *
+   * @param query_id The ID of the query to which this operator belongs.
    * @param input_relation The relation to perform selection over.
    * @param output_relation The output relation.
    * @param output_destination_index The index of the InsertDestination in the
@@ -76,23 +77,24 @@ class SelectOperator : public RelationalOperator {
    * @param input_relation_is_stored If input_relation is a stored relation and
    *        is fully available to the operator before it can start generating
    *        workorders.
-   * @param query_id The ID of the query to which this operator belongs.
    **/
-  SelectOperator(const CatalogRelation &input_relation,
-                 const CatalogRelation &output_relation,
-                 const QueryContext::insert_destination_id output_destination_index,
-                 const QueryContext::predicate_id predicate_index,
-                 const QueryContext::scalar_group_id selection_index,
-                 const bool input_relation_is_stored,
-                 const std::size_t query_id)
+  SelectOperator(
+      const std::size_t query_id,
+      const CatalogRelation &input_relation,
+      const CatalogRelation &output_relation,
+      const QueryContext::insert_destination_id output_destination_index,
+      const QueryContext::predicate_id predicate_index,
+      const QueryContext::scalar_group_id selection_index,
+      const bool input_relation_is_stored)
       : RelationalOperator(query_id),
         input_relation_(input_relation),
         output_relation_(output_relation),
         output_destination_index_(output_destination_index),
         predicate_index_(predicate_index),
         selection_index_(selection_index),
-        input_relation_block_ids_(input_relation_is_stored ? input_relation.getBlocksSnapshot()
-                                                           : std::vector<block_id>()),
+        input_relation_block_ids_(input_relation_is_stored
+                                      ? input_relation.getBlocksSnapshot()
+                                      : std::vector<block_id>()),
         num_workorders_generated_(0),
         simple_projection_(false),
         input_relation_is_stored_(input_relation_is_stored),
@@ -124,6 +126,7 @@ class SelectOperator : public RelationalOperator {
    *
    * @note selection_index_ is invalid, and will not be used for projection.
    *
+   * @param query_id The ID of the query to which this operator belongs.
    * @param input_relation The relation to perform selection over.
    * @param output_relation The output relation.
    * @param output_destination_index The index of the InsertDestination in the
@@ -136,15 +139,15 @@ class SelectOperator : public RelationalOperator {
    * @param input_relation_is_stored If input_relation is a stored relation and
    *        is fully available to the operator before it can start generating
    *        workorders.
-   * @param query_id The ID of the query to which this operator belongs.
    **/
-  SelectOperator(const CatalogRelation &input_relation,
-                 const CatalogRelation &output_relation,
-                 const QueryContext::insert_destination_id output_destination_index,
-                 const QueryContext::predicate_id predicate_index,
-                 std::vector<attribute_id> &&selection,
-                 const bool input_relation_is_stored,
-                 const std::size_t query_id)
+  SelectOperator(
+      const std::size_t query_id,
+      const CatalogRelation &input_relation,
+      const CatalogRelation &output_relation,
+      const QueryContext::insert_destination_id output_destination_index,
+      const QueryContext::predicate_id predicate_index,
+      std::vector<attribute_id> &&selection,
+      const bool input_relation_is_stored)
       : RelationalOperator(query_id),
         input_relation_(input_relation),
         output_relation_(output_relation),
@@ -152,8 +155,9 @@ class SelectOperator : public RelationalOperator {
         predicate_index_(predicate_index),
         selection_index_(QueryContext::kInvalidScalarGroupId),
         simple_selection_(std::move(selection)),
-        input_relation_block_ids_(input_relation_is_stored ? input_relation.getBlocksSnapshot()
-                                                           : std::vector<block_id>()),
+        input_relation_block_ids_(input_relation_is_stored
+                                      ? input_relation.getBlocksSnapshot()
+                                      : std::vector<block_id>()),
         num_workorders_generated_(0),
         simple_projection_(true),
         input_relation_is_stored_(input_relation_is_stored),
@@ -278,6 +282,7 @@ class SelectWorkOrder : public WorkOrder {
    * @note Reference parameter selection is NOT owned by this class and must
    *       remain valid until after execute() is called.
    *
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param input_relation The relation to perform selection over.
    * @param input_block_id The block id.
    * @param predicate All tuples matching \c predicate will be selected (or NULL
@@ -287,18 +292,17 @@ class SelectWorkOrder : public WorkOrder {
    *        simple_projection is true.
    * @param selection A list of Scalars which will be evaluated to project
    *        input tuples, used if \c simple_projection is false.
-   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to insert the selection
    *        results.
    * @param storage_manager The StorageManager to use.
    **/
-  SelectWorkOrder(const CatalogRelationSchema &input_relation,
+  SelectWorkOrder(const std::size_t query_id,
+                  const CatalogRelationSchema &input_relation,
                   const block_id input_block_id,
                   const Predicate *predicate,
                   const bool simple_projection,
                   const std::vector<attribute_id> &simple_selection,
                   const std::vector<std::unique_ptr<const Scalar>> *selection,
-                  const std::size_t query_id,
                   InsertDestination *output_destination,
                   StorageManager *storage_manager,
                   const numa_node_id numa_node = 0)
@@ -320,6 +324,7 @@ class SelectWorkOrder : public WorkOrder {
    * @note Reference parameter selection is NOT owned by this class and must
    *       remain valid until after execute() is called.
    *
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param input_relation The relation to perform selection over.
    * @param input_block_id The block id.
    * @param predicate All tuples matching \c predicate will be selected (or NULL
@@ -329,18 +334,17 @@ class SelectWorkOrder : public WorkOrder {
    *        simple_projection is true.
    * @param selection A list of Scalars which will be evaluated to project
    *        input tuples, used if \c simple_projection is false.
-   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to insert the selection
    *        results.
    * @param storage_manager The StorageManager to use.
    **/
-  SelectWorkOrder(const CatalogRelationSchema &input_relation,
+  SelectWorkOrder(const std::size_t query_id,
+                  const CatalogRelationSchema &input_relation,
                   const block_id input_block_id,
                   const Predicate *predicate,
                   const bool simple_projection,
                   std::vector<attribute_id> &&simple_selection,
                   const std::vector<std::unique_ptr<const Scalar>> *selection,
-                  const std::size_t query_id,
                   InsertDestination *output_destination,
                   StorageManager *storage_manager,
                   const numa_node_id numa_node = 0)

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d67f61e1/relational_operators/SortMergeRunOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/SortMergeRunOperator.cpp b/relational_operators/SortMergeRunOperator.cpp
index 9db8de1..6bf5719 100644
--- a/relational_operators/SortMergeRunOperator.cpp
+++ b/relational_operators/SortMergeRunOperator.cpp
@@ -85,12 +85,12 @@ WorkOrder *SortMergeRunOperator::createWorkOrder(
 
   // Create a work order from the merge job from merge tree.
   return new SortMergeRunWorkOrder(
+      query_id_,
       query_context->getSortConfig(sort_config_index_),
       job->level > 0 ? run_relation_ : input_relation_,
       std::move(job->runs),
       top_k_,
       job->level,
-      query_id_,
       output_destination,
       storage_manager,
       op_index_,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d67f61e1/relational_operators/SortMergeRunOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SortMergeRunOperator.hpp b/relational_operators/SortMergeRunOperator.hpp
index f54e925..cfff8b9 100644
--- a/relational_operators/SortMergeRunOperator.hpp
+++ b/relational_operators/SortMergeRunOperator.hpp
@@ -72,6 +72,7 @@ class SortMergeRunOperator : public RelationalOperator {
   /**
    * @brief Constructor for merging sorted runs to generate a sorted relation.
    *
+   * @param query_id The ID of the query to which this operator belongs.
    * @param input_relation The relation to merge sorted blocks.
    * @param output_relation The output relation.
    * @param output_destination_index The index of the InsertDestination in the
@@ -88,18 +89,18 @@ class SortMergeRunOperator : public RelationalOperator {
    *              \c top_k is 0.
    * @param input_relation_is_stored Boolean to indicate is input relation is
    *                                 stored or streamed.
-   * @param query_id The ID of the query to which this operator belongs.
    **/
-  SortMergeRunOperator(const CatalogRelation &input_relation,
-                       const CatalogRelation &output_relation,
-                       const QueryContext::insert_destination_id output_destination_index,
-                       const CatalogRelation &run_relation,
-                       const QueryContext::insert_destination_id run_block_destination_index,
-                       const QueryContext::sort_config_id sort_config_index,
-                       const std::size_t merge_factor,
-                       const std::size_t top_k,
-                       const bool input_relation_is_stored,
-                       const std::size_t query_id)
+  SortMergeRunOperator(
+      const std::size_t query_id,
+      const CatalogRelation &input_relation,
+      const CatalogRelation &output_relation,
+      const QueryContext::insert_destination_id output_destination_index,
+      const CatalogRelation &run_relation,
+      const QueryContext::insert_destination_id run_block_destination_index,
+      const QueryContext::sort_config_id sort_config_index,
+      const std::size_t merge_factor,
+      const std::size_t top_k,
+      const bool input_relation_is_stored)
       : RelationalOperator(query_id),
         input_relation_(input_relation),
         output_relation_(output_relation),
@@ -214,12 +215,12 @@ class SortMergeRunWorkOrder : public WorkOrder {
   /**
    * @brief Constructor.
    *
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param sort_config The Sort configuration.
    * @param run_relation The relation to which the run blocks belong to.
    * @param input_runs Input runs to merge.
    * @param top_k If non-zero will merge only \c top_k tuples.
    * @param merge_level Merge level in the merge tree.
-   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to create new blocks.
    * @param storage_manager The StorageManager to use.
    * @param operator_index Merge-run operator index to send feedback messages
@@ -228,12 +229,12 @@ class SortMergeRunWorkOrder : public WorkOrder {
    * @param bus TMB to send the feedback message on.
    **/
   SortMergeRunWorkOrder(
+      const std::size_t query_id,
       const SortConfiguration &sort_config,
       const CatalogRelationSchema &run_relation,
       std::vector<merge_run_operator::Run> &&input_runs,
       const std::size_t top_k,
       const std::size_t merge_level,
-      const std::size_t query_id,
       InsertDestination *output_destination,
       StorageManager *storage_manager,
       const std::size_t operator_index,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d67f61e1/relational_operators/SortRunGenerationOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/SortRunGenerationOperator.cpp b/relational_operators/SortRunGenerationOperator.cpp
index e352f9e..37b8fb8 100644
--- a/relational_operators/SortRunGenerationOperator.cpp
+++ b/relational_operators/SortRunGenerationOperator.cpp
@@ -51,10 +51,10 @@ bool SortRunGenerationOperator::getAllWorkOrders(
     if (!started_) {
       for (const block_id input_block_id : input_relation_block_ids_) {
         container->addNormalWorkOrder(
-            new SortRunGenerationWorkOrder(input_relation_,
+            new SortRunGenerationWorkOrder(query_id_,
+                                           input_relation_,
                                            input_block_id,
                                            sort_config,
-                                           query_id_,
                                            output_destination,
                                            storage_manager),
             op_index_);
@@ -67,10 +67,10 @@ bool SortRunGenerationOperator::getAllWorkOrders(
     while (num_workorders_generated_ < input_relation_block_ids_.size()) {
       container->addNormalWorkOrder(
           new SortRunGenerationWorkOrder(
+              query_id_,
               input_relation_,
               input_relation_block_ids_[num_workorders_generated_],
               sort_config,
-              query_id_,
               output_destination,
               storage_manager),
           op_index_);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d67f61e1/relational_operators/SortRunGenerationOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SortRunGenerationOperator.hpp b/relational_operators/SortRunGenerationOperator.hpp
index 3da9813..f96e6a6 100644
--- a/relational_operators/SortRunGenerationOperator.hpp
+++ b/relational_operators/SortRunGenerationOperator.hpp
@@ -73,6 +73,7 @@ class SortRunGenerationOperator : public RelationalOperator {
    * @brief Constructor for sorting tuples in blocks based on the sort
    * configuration and writing to output destination.
    *
+   * @param query_id The ID of the query to which this operator belongs.
    * @param input_relation The relation to generate sorted runs of.
    * @param output_relation The output relation.
    * @param output_destination_index The index of the InsertDestination in the
@@ -83,21 +84,22 @@ class SortRunGenerationOperator : public RelationalOperator {
    * @param input_relation_is_stored Does the input relation contain the blocks
    *                                 to sort. If \c false, the blocks are
    *                                 streamed.
-   * @param query_id The ID of the query to which this operator belongs.
    **/
-  SortRunGenerationOperator(const CatalogRelation &input_relation,
-                            const CatalogRelation &output_relation,
-                            const QueryContext::insert_destination_id output_destination_index,
-                            const QueryContext::sort_config_id sort_config_index,
-                            bool input_relation_is_stored,
-                            const std::size_t query_id)
+  SortRunGenerationOperator(
+      const std::size_t query_id,
+      const CatalogRelation &input_relation,
+      const CatalogRelation &output_relation,
+      const QueryContext::insert_destination_id output_destination_index,
+      const QueryContext::sort_config_id sort_config_index,
+      bool input_relation_is_stored)
       : RelationalOperator(query_id),
         input_relation_(input_relation),
         output_relation_(output_relation),
         output_destination_index_(output_destination_index),
         sort_config_index_(sort_config_index),
-        input_relation_block_ids_(input_relation_is_stored ? input_relation.getBlocksSnapshot()
-                                                           : std::vector<block_id>()),
+        input_relation_block_ids_(input_relation_is_stored
+                                      ? input_relation.getBlocksSnapshot()
+                                      : std::vector<block_id>()),
         num_workorders_generated_(0),
         started_(false),
         input_relation_is_stored_(input_relation_is_stored) {}
@@ -151,19 +153,19 @@ class SortRunGenerationWorkOrder : public WorkOrder {
   /**
    * @brief Constructor.
    *
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param input_relation The relation to generate sorted runs of.
    * @param input_block_id The block id.
    * @param sort_config The Sort configuration specifying ORDER BY, ordering,
    *        and null ordering.
-   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to store the sorted blocks
    *        of runs.
    * @param storage_manager The StorageManager to use.
    **/
-  SortRunGenerationWorkOrder(const CatalogRelationSchema &input_relation,
+  SortRunGenerationWorkOrder(const std::size_t query_id,
+                             const CatalogRelationSchema &input_relation,
                              const block_id input_block_id,
                              const SortConfiguration &sort_config,
-                             const std::size_t query_id,
                              InsertDestination *output_destination,
                              StorageManager *storage_manager)
       : WorkOrder(query_id),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d67f61e1/relational_operators/TableGeneratorOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/TableGeneratorOperator.cpp b/relational_operators/TableGeneratorOperator.cpp
index fb1f743..a3f9340 100644
--- a/relational_operators/TableGeneratorOperator.cpp
+++ b/relational_operators/TableGeneratorOperator.cpp
@@ -43,9 +43,9 @@ bool TableGeneratorOperator::getAllWorkOrders(
     // so just produce one work order.
     container->addNormalWorkOrder(
         new TableGeneratorWorkOrder(
+            query_id_,
             query_context->getGeneratorFunctionHandle(
                 generator_function_index_),
-            query_id_,
             query_context->getInsertDestination(output_destination_index_)),
         op_index_);
     started_ = true;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d67f61e1/relational_operators/TableGeneratorOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/TableGeneratorOperator.hpp b/relational_operators/TableGeneratorOperator.hpp
index bfc70c5..6a6af4b 100644
--- a/relational_operators/TableGeneratorOperator.hpp
+++ b/relational_operators/TableGeneratorOperator.hpp
@@ -55,23 +55,23 @@ class TableGeneratorOperator : public RelationalOperator {
   /**
    * @brief Constructor
    *
+   * @param query_id The ID of the query to which this operator belongs.
    * @param output_relation The output relation.
    * @param output_destination_index The index of the InsertDestination in the
    *        QueryContext to insert the generated output.
    * @param generator_function_index The index of the GeneratorFunctionHandle in
    *        the QueryContext.
-   * @param query_id The ID of the query to which this operator belongs.
    **/
-  TableGeneratorOperator(const CatalogRelation &output_relation,
-                         const QueryContext::insert_destination_id output_destination_index,
-                         const QueryContext::generator_function_id generator_function_index,
-                         const std::size_t query_id)
+  TableGeneratorOperator(
+      const std::size_t query_id,
+      const CatalogRelation &output_relation,
+      const QueryContext::insert_destination_id output_destination_index,
+      const QueryContext::generator_function_id generator_function_index)
       : RelationalOperator(query_id),
         output_relation_(output_relation),
         output_destination_index_(output_destination_index),
         generator_function_index_(generator_function_index),
-        started_(false) {
-  }
+        started_(false) {}
 
   ~TableGeneratorOperator() override {}
 
@@ -113,13 +113,13 @@ class TableGeneratorWorkOrder : public WorkOrder {
   /**
    * @brief Constructor.
    *
-   * @param generator_function The GeneratorFunctionHandle to use.
    * @param query_id The ID of the query to which this WorkOrder belongs.
+   * @param generator_function The GeneratorFunctionHandle to use.
    * @param output_destination The InsertDestination to insert the generated
    *        output.
    **/
-  TableGeneratorWorkOrder(const GeneratorFunctionHandle &function_handle,
-                          const std::size_t query_id,
+  TableGeneratorWorkOrder(const std::size_t query_id,
+                          const GeneratorFunctionHandle &function_handle,
                           InsertDestination *output_destination)
       : WorkOrder(query_id),
         function_handle_(function_handle),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d67f61e1/relational_operators/TextScanOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/TextScanOperator.cpp b/relational_operators/TextScanOperator.cpp
index 8db5ef1..5acecbf 100644
--- a/relational_operators/TextScanOperator.cpp
+++ b/relational_operators/TextScanOperator.cpp
@@ -165,9 +165,9 @@ bool TextScanOperator::getAllWorkOrders(
         // First, generate text-split work orders.
         for (const auto &file : files) {
           container->addNormalWorkOrder(
-              new TextSplitWorkOrder(file,
+              new TextSplitWorkOrder(query_id_,
+                                     file,
                                      process_escape_sequences_,
-                                     query_id_,
                                      storage_manager,
                                      op_index_,
                                      scheduler_client_id,
@@ -182,11 +182,11 @@ bool TextScanOperator::getAllWorkOrders(
         while (!text_blob_queue_.empty()) {
           const TextBlob blob_work = text_blob_queue_.popOne();
           container->addNormalWorkOrder(
-              new TextScanWorkOrder(blob_work.blob_id,
+              new TextScanWorkOrder(query_id_,
+                                    blob_work.blob_id,
                                     blob_work.size,
                                     field_terminator_,
                                     process_escape_sequences_,
-                                    query_id_,
                                     output_destination,
                                     storage_manager),
               op_index_);
@@ -203,10 +203,10 @@ bool TextScanOperator::getAllWorkOrders(
     if (blocking_dependencies_met_ && !work_generated_) {
       for (const auto &file : files) {
         container->addNormalWorkOrder(
-            new TextScanWorkOrder(file,
+            new TextScanWorkOrder(query_id_,
+                                  file,
                                   field_terminator_,
                                   process_escape_sequences_,
-                                  query_id_,
                                   output_destination,
                                   storage_manager),
             op_index_);
@@ -234,11 +234,10 @@ void TextScanOperator::receiveFeedbackMessage(const WorkOrder::FeedbackMessage &
   }
 }
 
-
-TextScanWorkOrder::TextScanWorkOrder(const std::string &filename,
+TextScanWorkOrder::TextScanWorkOrder(const std::size_t query_id,
+                                     const std::string &filename,
                                      const char field_terminator,
                                      const bool process_escape_sequences,
-                                     const std::size_t query_id,
                                      InsertDestination *output_destination,
                                      StorageManager *storage_manager)
     : WorkOrder(query_id),
@@ -254,11 +253,11 @@ TextScanWorkOrder::TextScanWorkOrder(const std::string &filename,
   DCHECK(storage_manager_ != nullptr);
 }
 
-TextScanWorkOrder::TextScanWorkOrder(const block_id text_blob,
+TextScanWorkOrder::TextScanWorkOrder(const std::size_t query_id,
+                                     const block_id text_blob,
                                      const std::size_t text_size,
                                      const char field_terminator,
                                      const bool process_escape_sequences,
-                                     const std::size_t query_id,
                                      InsertDestination *output_destination,
                                      StorageManager *storage_manager)
     : WorkOrder(query_id),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d67f61e1/relational_operators/TextScanOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/TextScanOperator.hpp b/relational_operators/TextScanOperator.hpp
index 1d0c04f..3cda65b 100644
--- a/relational_operators/TextScanOperator.hpp
+++ b/relational_operators/TextScanOperator.hpp
@@ -121,6 +121,7 @@ class TextScanOperator : public RelationalOperator {
   /**
    * @brief Constructor
    *
+   * @param query_id The ID of the query to which this operator belongs.
    * @param file_pattern The glob-like file pattern of the sources to load. The
    *        pattern could include * (wildcard for multiple chars) and ?
    *        (wildcard for single char). It defaults to single file load, if a
@@ -134,15 +135,15 @@ class TextScanOperator : public RelationalOperator {
    * @param output_relation The output relation.
    * @param output_destination_index The index of the InsertDestination in the
    *        QueryContext to insert tuples.
-   * @param query_id The ID of the query to which this operator belongs.
    **/
-  TextScanOperator(const std::string &file_pattern,
-                   const char field_terminator,
-                   const bool process_escape_sequences,
-                   const bool parallelize_load,
-                   const CatalogRelation &output_relation,
-                   const QueryContext::insert_destination_id output_destination_index,
-                   const std::size_t query_id)
+  TextScanOperator(
+      const std::size_t query_id,
+      const std::string &file_pattern,
+      const char field_terminator,
+      const bool process_escape_sequences,
+      const bool parallelize_load,
+      const CatalogRelation &output_relation,
+      const QueryContext::insert_destination_id output_destination_index)
       : RelationalOperator(query_id),
         file_pattern_(file_pattern),
         field_terminator_(field_terminator),
@@ -200,42 +201,42 @@ class TextScanWorkOrder : public WorkOrder {
   /**
    * @brief Constructor
    *
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param filename The name of the text file to bulk insert.
    * @param field_terminator The string which separates attribute values in
    *        the text file.
    * @param process_escape_sequences Whether to decode escape sequences in the
    *        text file.
-   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to insert tuples.
    * @param storage_manager The StorageManager to use.
    **/
   TextScanWorkOrder(
+      const std::size_t query_id,
       const std::string &filename,
       const char field_terminator,
       const bool process_escape_sequences,
-      const std::size_t query_id,
       InsertDestination *output_destination,
       StorageManager *storage_manager);
 
   /**
    * @brief Constructor.
    *
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param text_blob Blob ID containing the data to be scanned.
    * @param text_size Size of the data in the blob.
    * @param field_terminator The character which separates attribute values in
    *        the text file.
    * @param process_escape_sequences Whether to decode escape sequences in the
    *        text file.
-   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to write the read tuples.
    * @param storage_manager The StorageManager to use.
    */
   TextScanWorkOrder(
+      const std::size_t query_id,
       const block_id text_blob,
       const std::size_t text_size,
       const char field_terminator,
       const bool process_escape_sequences,
-      const std::size_t query_id,
       InsertDestination *output_destination,
       StorageManager *storage_manager);
 
@@ -322,19 +323,20 @@ class TextSplitWorkOrder : public WorkOrder {
  public:
   /**
    * @brief Constructor.
+   *
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param filename File to split into row-aligned blobs.
    * @param process_escape_sequences Whether to decode escape sequences in the
    *        text file.
-   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param storage_manager The StorageManager to use.
    * @param operator_index Operator index of the current operator. This is used
    *                       to send new-work available message to Foreman.
    * @param scheduler_client_id The TMB client ID of the scheduler thread.
    * @param bus A pointer to the TMB.
    */
-  TextSplitWorkOrder(const std::string &filename,
+  TextSplitWorkOrder(const std::size_t query_id,
+                     const std::string &filename,
                      const bool process_escape_sequences,
-                     const std::size_t query_id,
                      StorageManager *storage_manager,
                      const std::size_t operator_index,
                      const tmb::client_id scheduler_client_id,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d67f61e1/relational_operators/UpdateOperator.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/UpdateOperator.cpp b/relational_operators/UpdateOperator.cpp
index b331a9c..2130563 100644
--- a/relational_operators/UpdateOperator.cpp
+++ b/relational_operators/UpdateOperator.cpp
@@ -53,16 +53,18 @@ bool UpdateOperator::getAllWorkOrders(
 
     for (const block_id input_block_id : input_blocks_) {
       container->addNormalWorkOrder(
-          new UpdateWorkOrder(relation_,
-                              input_block_id,
-                              query_context->getPredicate(predicate_index_),
-                              query_context->getUpdateGroup(update_group_index_),
-                              query_id_,
-                              query_context->getInsertDestination(relocation_destination_index_),
-                              storage_manager,
-                              op_index_,
-                              scheduler_client_id,
-                              bus),
+          new UpdateWorkOrder(
+              query_id_,
+              relation_,
+              input_block_id,
+              query_context->getPredicate(predicate_index_),
+              query_context->getUpdateGroup(update_group_index_),
+              query_context->getInsertDestination(
+                  relocation_destination_index_),
+              storage_manager,
+              op_index_,
+              scheduler_client_id,
+              bus),
           op_index_);
     }
     started_ = true;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d67f61e1/relational_operators/UpdateOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/UpdateOperator.hpp b/relational_operators/UpdateOperator.hpp
index ba2d6cf..cebb9b5 100644
--- a/relational_operators/UpdateOperator.hpp
+++ b/relational_operators/UpdateOperator.hpp
@@ -62,6 +62,7 @@ class UpdateOperator : public RelationalOperator {
   /**
    * @brief Constructor
    *
+   * @param query_id The ID of the query to which this operator belongs.
    * @param relation The relation to perform the UPDATE over.
    * @param relocation_destination_index The index of the InsertDestination in
    *        the QueryContext to relocate tuples which can not be updated
@@ -72,16 +73,16 @@ class UpdateOperator : public RelationalOperator {
    * @param update_group_index The index of a update group (the map of
    *        attribute_ids to Scalars) which should be evaluated to get the new
    *        value for the corresponding attribute.
-   * @param query_id The ID of the query to which this operator belongs.
    *
    * @warning The constructed InsertDestination should belong to relation, but
    *          must NOT contain any pre-existing blocks.
    **/
-  UpdateOperator(const CatalogRelation &relation,
-                 const QueryContext::insert_destination_id relocation_destination_index,
-                 const QueryContext::predicate_id predicate_index,
-                 const QueryContext::update_group_id update_group_index,
-                 const std::size_t query_id)
+  UpdateOperator(
+      const std::size_t query_id,
+      const CatalogRelation &relation,
+      const QueryContext::insert_destination_id relocation_destination_index,
+      const QueryContext::predicate_id predicate_index,
+      const QueryContext::update_group_id update_group_index)
       : RelationalOperator(query_id),
         relation_(relation),
         relocation_destination_index_(relocation_destination_index),
@@ -127,13 +128,13 @@ class UpdateWorkOrder : public WorkOrder {
   /**
    * @brief Constructor
    *
+   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param relation The relation to perform the UPDATE over.
    * @param predicate All tuples matching \c predicate will be updated (or NULL
    *        to update all tuples).
    * @param assignments The assignments (the map of attribute_ids to Scalars)
    *        which should be evaluated to get the new value for the corresponding
    *        attribute.
-   * @param query_id The ID of the query to which this WorkOrder belongs.
    * @param input_block_id The block id.
    * @param relocation_destination The InsertDestination to relocate tuples
    *        which can not be updated in-place.
@@ -143,16 +144,18 @@ class UpdateWorkOrder : public WorkOrder {
    * @param scheduler_client_id The TMB client ID of the scheduler thread.
    * @param bus A pointer to the TMB.
    **/
-  UpdateWorkOrder(const CatalogRelationSchema &relation,
-                  const block_id input_block_id,
-                  const Predicate *predicate,
-                  const std::unordered_map<attribute_id, std::unique_ptr<const Scalar>> &assignments,
-                  const std::size_t query_id,
-                  InsertDestination *relocation_destination,
-                  StorageManager *storage_manager,
-                  const std::size_t update_operator_index,
-                  const tmb::client_id scheduler_client_id,
-                  MessageBus *bus)
+  UpdateWorkOrder(
+      const std::size_t query_id,
+      const CatalogRelationSchema &relation,
+      const block_id input_block_id,
+      const Predicate *predicate,
+      const std::unordered_map<attribute_id, std::unique_ptr<const Scalar>>
+          &assignments,
+      InsertDestination *relocation_destination,
+      StorageManager *storage_manager,
+      const std::size_t update_operator_index,
+      const tmb::client_id scheduler_client_id,
+      MessageBus *bus)
       : WorkOrder(query_id),
         relation_(relation),
         input_block_id_(input_block_id),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d67f61e1/relational_operators/WorkOrderFactory.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/WorkOrderFactory.cpp b/relational_operators/WorkOrderFactory.cpp
index fdd694f..489b666 100644
--- a/relational_operators/WorkOrderFactory.cpp
+++ b/relational_operators/WorkOrderFactory.cpp
@@ -75,8 +75,8 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
     case serialization::AGGREGATION: {
       LOG(INFO) << "Creating AggregationWorkOrder";
       return new AggregationWorkOrder(
-          proto.GetExtension(serialization::AggregationWorkOrder::block_id),
           proto.query_id(),
+          proto.GetExtension(serialization::AggregationWorkOrder::block_id),
           query_context->getAggregationState(
               proto.GetExtension(serialization::AggregationWorkOrder::aggr_state_index)));
     }
@@ -89,12 +89,12 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
       }
 
       return new BuildHashWorkOrder(
+          proto.query_id(),
           catalog_database->getRelationSchemaById(
               proto.GetExtension(serialization::BuildHashWorkOrder::relation_id)),
           move(join_key_attributes),
           proto.GetExtension(serialization::BuildHashWorkOrder::any_join_key_attributes_nullable),
           proto.GetExtension(serialization::BuildHashWorkOrder::block_id),
-          proto.query_id(),
           query_context->getJoinHashTable(
               proto.GetExtension(serialization::BuildHashWorkOrder::join_hash_table_index)),
           storage_manager);
@@ -102,6 +102,7 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
     case serialization::DELETE: {
       LOG(INFO) << "Creating DeleteWorkOrder";
       return new DeleteWorkOrder(
+          proto.query_id(),
           catalog_database->getRelationSchemaById(
               proto.GetExtension(serialization::DeleteWorkOrder::relation_id)),
           proto.GetExtension(serialization::DeleteWorkOrder::block_id),
@@ -110,15 +111,14 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
           storage_manager,
           proto.GetExtension(serialization::DeleteWorkOrder::operator_index),
           shiftboss_client_id,
-          proto.query_id(),
           bus);
     }
     case serialization::DESTROY_HASH: {
       LOG(INFO) << "Creating DestroyHashWorkOrder";
       return new DestroyHashWorkOrder(
+          proto.query_id(),
           proto.GetExtension(
               serialization::DestroyHashWorkOrder::join_hash_table_index),
-          proto.query_id(),
           query_context);
     }
     case serialization::DROP_TABLE: {
@@ -193,6 +193,7 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
         case serialization::HashJoinWorkOrder::HASH_ANTI_JOIN: {
           LOG(INFO) << "Creating HashAntiJoinWorkOrder";
           return new HashAntiJoinWorkOrder(
+              proto.query_id(),
               build_relation,
               probe_relation,
               move(join_key_attributes),
@@ -201,13 +202,13 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
               residual_predicate,
               selection,
               hash_table,
-              proto.query_id(),
               output_destination,
               storage_manager);
         }
         case serialization::HashJoinWorkOrder::HASH_INNER_JOIN: {
           LOG(INFO) << "Creating HashInnerJoinWorkOrder";
           return new HashInnerJoinWorkOrder(
+              proto.query_id(),
               build_relation,
               probe_relation,
               move(join_key_attributes),
@@ -216,7 +217,6 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
               residual_predicate,
               selection,
               hash_table,
-              proto.query_id(),
               output_destination,
               storage_manager);
         }
@@ -231,6 +231,7 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
 
           LOG(INFO) << "Creating HashOuterJoinWorkOrder";
           return new HashOuterJoinWorkOrder(
+              proto.query_id(),
               build_relation,
               probe_relation,
               move(join_key_attributes),
@@ -239,13 +240,13 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
               selection,
               move(is_selection_on_build),
               hash_table,
-              proto.query_id(),
               output_destination,
               storage_manager);
         }
         case serialization::HashJoinWorkOrder::HASH_SEMI_JOIN: {
           LOG(INFO) << "Creating HashSemiJoinWorkOrder";
           return new HashSemiJoinWorkOrder(
+              proto.query_id(),
               build_relation,
               probe_relation,
               move(join_key_attributes),
@@ -254,7 +255,6 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
               residual_predicate,
               selection,
               hash_table,
-              proto.query_id(),
               output_destination,
               storage_manager);
         }
@@ -274,6 +274,7 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
     case serialization::NESTED_LOOP_JOIN: {
       LOG(INFO) << "Creating NestedLoopsJoinWorkOrder";
       return new NestedLoopsJoinWorkOrder(
+          proto.query_id(),
           catalog_database->getRelationSchemaById(
               proto.GetExtension(serialization::NestedLoopsJoinWorkOrder::left_relation_id)),
           catalog_database->getRelationSchemaById(
@@ -284,7 +285,6 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
               proto.GetExtension(serialization::NestedLoopsJoinWorkOrder::join_predicate_index)),
           query_context->getScalarGroup(
               proto.GetExtension(serialization::NestedLoopsJoinWorkOrder::selection_index)),
-          proto.query_id(),
           query_context->getInsertDestination(
               proto.GetExtension(serialization::NestedLoopsJoinWorkOrder::insert_destination_index)),
           storage_manager);
@@ -292,12 +292,12 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
     case serialization::SAMPLE: {
       LOG(INFO) << "Creating SampleWorkOrder";
       return new SampleWorkOrder(
+          proto.query_id(),
           catalog_database->getRelationSchemaById(
               proto.GetExtension(serialization::SampleWorkOrder::relation_id)),
           proto.GetExtension(serialization::SampleWorkOrder::block_id),
           proto.GetExtension(serialization::SampleWorkOrder::is_block_sample),
           proto.GetExtension(serialization::SampleWorkOrder::percentage),
-          proto.query_id(),
           query_context->getInsertDestination(
               proto.GetExtension(serialization::SampleWorkOrder::insert_destination_index)),
           storage_manager);
@@ -305,9 +305,9 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
     case serialization::SAVE_BLOCKS: {
       LOG(INFO) << "Creating SaveBlocksWorkOrder";
       return new SaveBlocksWorkOrder(
+          proto.query_id(),
           proto.GetExtension(serialization::SaveBlocksWorkOrder::block_id),
           proto.GetExtension(serialization::SaveBlocksWorkOrder::force),
-          proto.query_id(),
           storage_manager);
     }
     case serialization::SELECT: {
@@ -321,6 +321,7 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
       }
 
       return new SelectWorkOrder(
+          proto.query_id(),
           catalog_database->getRelationSchemaById(
               proto.GetExtension(serialization::SelectWorkOrder::relation_id)),
           proto.GetExtension(serialization::SelectWorkOrder::block_id),
@@ -331,7 +332,6 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
           simple_projection ? nullptr
                             : &query_context->getScalarGroup(
                                   proto.GetExtension(serialization::SelectWorkOrder::selection_index)),
-          proto.query_id(),
           query_context->getInsertDestination(
               proto.GetExtension(serialization::SelectWorkOrder::insert_destination_index)),
           storage_manager);
@@ -350,6 +350,7 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
       }
 
       return new SortMergeRunWorkOrder(
+          proto.query_id(),
           query_context->getSortConfig(
               proto.GetExtension(serialization::SortMergeRunWorkOrder::sort_config_index)),
           catalog_database->getRelationSchemaById(
@@ -357,7 +358,6 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
           move(runs),
           proto.GetExtension(serialization::SortMergeRunWorkOrder::top_k),
           proto.GetExtension(serialization::SortMergeRunWorkOrder::merge_level),
-          proto.query_id(),
           query_context->getInsertDestination(
               proto.GetExtension(serialization::SortMergeRunWorkOrder::insert_destination_index)),
           storage_manager,
@@ -368,12 +368,12 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
     case serialization::SORT_RUN_GENERATION: {
       LOG(INFO) << "Creating SortRunGenerationWorkOrder";
       return new SortRunGenerationWorkOrder(
+          proto.query_id(),
           catalog_database->getRelationSchemaById(
               proto.GetExtension(serialization::SortRunGenerationWorkOrder::relation_id)),
           proto.GetExtension(serialization::SortRunGenerationWorkOrder::block_id),
           query_context->getSortConfig(
               proto.GetExtension(serialization::SortRunGenerationWorkOrder::sort_config_index)),
-          proto.query_id(),
           query_context->getInsertDestination(
               proto.GetExtension(serialization::SortRunGenerationWorkOrder::insert_destination_index)),
           storage_manager);
@@ -381,9 +381,9 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
     case serialization::TABLE_GENERATOR: {
       LOG(INFO) << "Creating SortRunGenerationWorkOrder";
       return new TableGeneratorWorkOrder(
+          proto.query_id(),
           query_context->getGeneratorFunctionHandle(
               proto.GetExtension(serialization::TableGeneratorWorkOrder::generator_function_index)),
-          proto.query_id(),
           query_context->getInsertDestination(
               proto.GetExtension(serialization::TableGeneratorWorkOrder::insert_destination_index)));
     }
@@ -391,10 +391,10 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
       LOG(INFO) << "Creating TextScanWorkOrder";
       if (proto.HasExtension(serialization::TextScanWorkOrder::filename)) {
         return new TextScanWorkOrder(
+            proto.query_id(),
             proto.GetExtension(serialization::TextScanWorkOrder::filename),
             proto.GetExtension(serialization::TextScanWorkOrder::field_terminator),
             proto.GetExtension(serialization::TextScanWorkOrder::process_escape_sequences),
-            proto.query_id(),
             query_context->getInsertDestination(
                 proto.GetExtension(serialization::TextScanWorkOrder::insert_destination_index)),
             storage_manager);
@@ -403,11 +403,11 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
             proto.GetExtension(serialization::TextScanWorkOrder::text_blob);
 
         return new TextScanWorkOrder(
+            proto.query_id(),
             text_blob_proto.blob_id(),
             text_blob_proto.size(),
             proto.GetExtension(serialization::TextScanWorkOrder::field_terminator),
             proto.GetExtension(serialization::TextScanWorkOrder::process_escape_sequences),
-            proto.query_id(),
             query_context->getInsertDestination(
                 proto.GetExtension(serialization::TextScanWorkOrder::insert_destination_index)),
             storage_manager);
@@ -416,9 +416,9 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
     case serialization::TEXT_SPLIT: {
       LOG(INFO) << "Creating TextSplitWorkOrder";
       return new TextSplitWorkOrder(
+          proto.query_id(),
           proto.GetExtension(serialization::TextSplitWorkOrder::filename),
           proto.GetExtension(serialization::TextSplitWorkOrder::process_escape_sequences),
-          proto.query_id(),
           storage_manager,
           proto.GetExtension(serialization::TextSplitWorkOrder::operator_index),
           shiftboss_client_id,
@@ -427,6 +427,7 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
     case serialization::UPDATE: {
       LOG(INFO) << "Creating UpdateWorkOrder";
       return new UpdateWorkOrder(
+          proto.query_id(),
           catalog_database->getRelationSchemaById(
               proto.GetExtension(serialization::UpdateWorkOrder::relation_id)),
           proto.GetExtension(serialization::UpdateWorkOrder::block_id),
@@ -434,7 +435,6 @@ WorkOrder* WorkOrderFactory::ReconstructFromProto(const serialization::WorkOrder
               proto.GetExtension(serialization::UpdateWorkOrder::predicate_index)),
           query_context->getUpdateGroup(
               proto.GetExtension(serialization::UpdateWorkOrder::update_group_index)),
-          proto.query_id(),
           query_context->getInsertDestination(
               proto.GetExtension(serialization::UpdateWorkOrder::insert_destination_index)),
           storage_manager,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d67f61e1/relational_operators/tests/AggregationOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/AggregationOperator_unittest.cpp b/relational_operators/tests/AggregationOperator_unittest.cpp
index 5f3a434..dc15c56 100644
--- a/relational_operators/tests/AggregationOperator_unittest.cpp
+++ b/relational_operators/tests/AggregationOperator_unittest.cpp
@@ -281,10 +281,11 @@ class AggregationOperatorTest : public ::testing::Test {
     insert_destination_proto->set_relation_id(result_table_->getID());
     insert_destination_proto->set_relational_op_index(kOpIndex);
 
-    finalize_op_.reset(new FinalizeAggregationOperator(aggr_state_index,
-                                                       *result_table_,
-                                                       insert_destination_index,
-                                                       0 /* dummy query ID */));
+    finalize_op_.reset(
+        new FinalizeAggregationOperator(0 /* dummy query ID */,
+                                        aggr_state_index,
+                                        *result_table_,
+                                        insert_destination_index));
 
     // Set up the QueryContext.
     query_context_.reset(new QueryContext(query_context_proto,
@@ -365,10 +366,11 @@ class AggregationOperatorTest : public ::testing::Test {
     insert_destination_proto->set_relation_id(result_table_->getID());
     insert_destination_proto->set_relational_op_index(kOpIndex);
 
-    finalize_op_.reset(new FinalizeAggregationOperator(aggr_state_index,
-                                                       *result_table_,
-                                                       insert_destination_index,
-                                                       0 /* dummy query ID */));
+    finalize_op_.reset(
+        new FinalizeAggregationOperator(0 /* dummy query ID */,
+                                        aggr_state_index,
+                                        *result_table_,
+                                        insert_destination_index));
 
     // Set up the QueryContext.
     query_context_.reset(new QueryContext(query_context_proto,

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d67f61e1/relational_operators/tests/HashJoinOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/HashJoinOperator_unittest.cpp b/relational_operators/tests/HashJoinOperator_unittest.cpp
index 4ef5a5c..75445fb 100644
--- a/relational_operators/tests/HashJoinOperator_unittest.cpp
+++ b/relational_operators/tests/HashJoinOperator_unittest.cpp
@@ -332,12 +332,12 @@ TEST_P(HashJoinOperatorTest, LongKeyHashJoinTest) {
 
   // Create the builder operator.
   unique_ptr<BuildHashOperator> builder(
-      new BuildHashOperator(*dim_table_,
+      new BuildHashOperator(0,  /* dummy query ID */
+                            *dim_table_,
                             true /* is_stored */,
                             std::vector<attribute_id>(1, dim_col_long.getID()),
                             dim_col_long.getType().isNullable(),
-                            join_hash_table_index,
-                            0));  // dummy query ID
+                            join_hash_table_index));
 
   // Create the prober operator with one selection attribute.
   const QueryContext::scalar_group_id selection_index = query_context_proto.scalar_groups_size();
@@ -359,18 +359,18 @@ TEST_P(HashJoinOperatorTest, LongKeyHashJoinTest) {
   insert_destination_proto->set_relation_id(output_relation_id);
   insert_destination_proto->set_relational_op_index(kOpIndex);
 
-  unique_ptr<HashJoinOperator> prober(
-      new HashJoinOperator(*dim_table_,
-                           *fact_table_,
-                           true /* is_stored */,
-                           std::vector<attribute_id>(1, fact_col_long.getID()),
-                           fact_col_long.getType().isNullable(),
-                           *result_table,
-                           output_destination_index,
-                           join_hash_table_index,
-                           QueryContext::kInvalidPredicateId /* residual_predicate_index */,
-                           selection_index,
-                           0  /* dummy query ID */));
+  unique_ptr<HashJoinOperator> prober(new HashJoinOperator(
+      0 /* dummy query ID */,
+      *dim_table_,
+      *fact_table_,
+      true /* is_stored */,
+      std::vector<attribute_id>(1, fact_col_long.getID()),
+      fact_col_long.getType().isNullable(),
+      *result_table,
+      output_destination_index,
+      join_hash_table_index,
+      QueryContext::kInvalidPredicateId /* residual_predicate_index */,
+      selection_index));
 
   // Set up the QueryContext.
   query_context_.reset(new QueryContext(query_context_proto,
@@ -423,7 +423,7 @@ TEST_P(HashJoinOperatorTest, LongKeyHashJoinTest) {
   }
 
   // Create cleaner operator.
-  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(join_hash_table_index, 0  /* dummy query ID */));
+  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(0  /* dummy query ID */, join_hash_table_index));
   cleaner->informAllBlockingDependenciesMet();
   fetchAndExecuteWorkOrders(cleaner.get());
 
@@ -475,12 +475,12 @@ TEST_P(HashJoinOperatorTest, IntDuplicateKeyHashJoinTest) {
 
   // Create the builder operator.
   unique_ptr<BuildHashOperator> builder(
-      new BuildHashOperator(*dim_table_,
+      new BuildHashOperator(0,  /* dummy query ID */
+                            *dim_table_,
                             true /* is_stored */,
                             std::vector<attribute_id>(1, dim_col_int.getID()),
                             dim_col_int.getType().isNullable(),
-                            join_hash_table_index,
-                            0));  // dummy query ID
+                            join_hash_table_index));
 
   // Create the prober operator with two selection attributes.
   const QueryContext::scalar_group_id selection_index = query_context_proto.scalar_groups_size();
@@ -507,18 +507,19 @@ TEST_P(HashJoinOperatorTest, IntDuplicateKeyHashJoinTest) {
   insert_destination_proto->set_relation_id(output_relation_id);
   insert_destination_proto->set_relational_op_index(kOpIndex);
 
-  unique_ptr<HashJoinOperator> prober(
-      new HashJoinOperator(*dim_table_,
-                           *fact_table_,
-                           true /* is_stored */,
-                           std::vector<attribute_id>(1, fact_col_int.getID()),
-                           fact_col_int.getType().isNullable(),
-                           *result_table,
-                           output_destination_index,
-                           join_hash_table_index,
-                           QueryContext::kInvalidPredicateId /* residual_predicate_index */,
-                           selection_index,
-                           0  /* dummy query ID */));
+  unique_ptr<HashJoinOperator> prober(new HashJoinOperator(
+      0 /* dummy query ID */,
+      *dim_table_,
+      *fact_table_,
+      true /* is_stored */,
+      std::vector<attribute_id>(1, fact_col_int.getID()),
+      fact_col_int.getType().isNullable(),
+      *result_table,
+      output_destination_index,
+      join_hash_table_index,
+      QueryContext::kInvalidPredicateId /* residual_predicate_index */,
+      selection_index));
+
 
   // Set up the QueryContext.
   query_context_.reset(new QueryContext(query_context_proto,
@@ -592,7 +593,7 @@ TEST_P(HashJoinOperatorTest, IntDuplicateKeyHashJoinTest) {
   }
 
   // Create cleaner operator.
-  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(join_hash_table_index, 0  /* dummy query ID */));
+  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(0  /* dummy query ID */, join_hash_table_index));
   cleaner->informAllBlockingDependenciesMet();
   fetchAndExecuteWorkOrders(cleaner.get());
 
@@ -636,12 +637,12 @@ TEST_P(HashJoinOperatorTest, CharKeyCartesianProductHashJoinTest) {
 
   // Create builder operator.
   unique_ptr<BuildHashOperator> builder(
-      new BuildHashOperator(*dim_table_,
+      new BuildHashOperator(0,  /* dummy query ID */
+                            *dim_table_,
                             true /* is_stored */,
                             std::vector<attribute_id>(1, dim_col_char.getID()),
                             dim_col_char.getType().isNullable(),
-                            join_hash_table_index,
-                            0));  // dummy query ID.
+                            join_hash_table_index));
 
   // Create prober operator with one selection attribute.
   const QueryContext::scalar_group_id selection_index = query_context_proto.scalar_groups_size();
@@ -663,18 +664,18 @@ TEST_P(HashJoinOperatorTest, CharKeyCartesianProductHashJoinTest) {
   insert_destination_proto->set_relation_id(output_relation_id);
   insert_destination_proto->set_relational_op_index(kOpIndex);
 
-  unique_ptr<HashJoinOperator> prober(
-      new HashJoinOperator(*dim_table_,
-                           *fact_table_,
-                           true /* is_stored */,
-                           std::vector<attribute_id>(1, fact_col_char.getID()),
-                           fact_col_char.getType().isNullable(),
-                           *result_table,
-                           output_destination_index,
-                           join_hash_table_index,
-                           QueryContext::kInvalidPredicateId /* residual_predicate_index */,
-                           selection_index,
-                           0  /* dummy query ID */));
+  unique_ptr<HashJoinOperator> prober(new HashJoinOperator(
+      0 /* dummy query ID */,
+      *dim_table_,
+      *fact_table_,
+      true /* is_stored */,
+      std::vector<attribute_id>(1, fact_col_char.getID()),
+      fact_col_char.getType().isNullable(),
+      *result_table,
+      output_destination_index,
+      join_hash_table_index,
+      QueryContext::kInvalidPredicateId /* residual_predicate_index */,
+      selection_index));
 
   // Set up the QueryContext.
   query_context_.reset(new QueryContext(query_context_proto,
@@ -727,7 +728,7 @@ TEST_P(HashJoinOperatorTest, CharKeyCartesianProductHashJoinTest) {
   }
 
   // Create cleaner operator.
-  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(join_hash_table_index, 0  /* dummy query ID */));
+  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(0  /* dummy query ID */, join_hash_table_index));
   cleaner->informAllBlockingDependenciesMet();
   fetchAndExecuteWorkOrders(cleaner.get());
 
@@ -772,12 +773,12 @@ TEST_P(HashJoinOperatorTest, VarCharDuplicateKeyHashJoinTest) {
 
   // Create builder operator.
   unique_ptr<BuildHashOperator> builder(
-      new BuildHashOperator(*dim_table_,
+      new BuildHashOperator(0,  /* dummy query ID */
+                            *dim_table_,
                             true /* is_stored */,
                             std::vector<attribute_id>(1, dim_col_varchar.getID()),
                             dim_col_varchar.getType().isNullable(),
-                            join_hash_table_index,
-                            0));  // dummy query ID.
+                            join_hash_table_index));
 
   // Create prober operator with two selection attributes.
   const QueryContext::scalar_group_id selection_index = query_context_proto.scalar_groups_size();
@@ -804,18 +805,19 @@ TEST_P(HashJoinOperatorTest, VarCharDuplicateKeyHashJoinTest) {
   insert_destination_proto->set_relation_id(output_relation_id);
   insert_destination_proto->set_relational_op_index(kOpIndex);
 
-  unique_ptr<HashJoinOperator> prober(
-      new HashJoinOperator(*dim_table_,
-                           *fact_table_,
-                           true /* is_stored */,
-                           std::vector<attribute_id>(1, fact_col_varchar.getID()),
-                           fact_col_varchar.getType().isNullable(),
-                           *result_table,
-                           output_destination_index,
-                           join_hash_table_index,
-                           QueryContext::kInvalidPredicateId /* residual_predicate_index */,
-                           selection_index,
-                           0  /* dummy query ID */));
+  unique_ptr<HashJoinOperator> prober(new HashJoinOperator(
+      0 /* dummy query ID */,
+      *dim_table_,
+      *fact_table_,
+      true /* is_stored */,
+      std::vector<attribute_id>(1, fact_col_varchar.getID()),
+      fact_col_varchar.getType().isNullable(),
+      *result_table,
+      output_destination_index,
+      join_hash_table_index,
+      QueryContext::kInvalidPredicateId /* residual_predicate_index */,
+      selection_index));
+
 
   // Set up the QueryContext.
   query_context_.reset(new QueryContext(query_context_proto,
@@ -893,7 +895,7 @@ TEST_P(HashJoinOperatorTest, VarCharDuplicateKeyHashJoinTest) {
   }
 
   // Create the cleaner operator.
-  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(join_hash_table_index, 0  /* dummy query ID */));
+  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(0  /* dummy query ID */, join_hash_table_index));
   cleaner->informAllBlockingDependenciesMet();
   fetchAndExecuteWorkOrders(cleaner.get());
 
@@ -943,12 +945,12 @@ TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinTest) {
   dim_key_attrs.push_back(dim_col_varchar.getID());
 
   unique_ptr<BuildHashOperator> builder(
-      new BuildHashOperator(*dim_table_,
+      new BuildHashOperator(0,  /* dummy query ID */
+                            *dim_table_,
                             true /* is_stored */,
                             dim_key_attrs,
                             dim_col_long.getType().isNullable() || dim_col_varchar.getType().isNullable(),
-                            join_hash_table_index,
-                            0));  // dummy query ID.
+                            join_hash_table_index));
 
   // Create the prober operator with two selection attributes.
   const QueryContext::scalar_group_id selection_index = query_context_proto.scalar_groups_size();
@@ -979,18 +981,19 @@ TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinTest) {
   fact_key_attrs.push_back(fact_col_long.getID());
   fact_key_attrs.push_back(fact_col_varchar.getID());
 
-  unique_ptr<HashJoinOperator> prober(
-      new HashJoinOperator(*dim_table_,
-                           *fact_table_,
-                           true /* is_stored */,
-                           fact_key_attrs,
-                           fact_col_long.getType().isNullable() || fact_col_varchar.getType().isNullable(),
-                           *result_table,
-                           output_destination_index,
-                           join_hash_table_index,
-                           QueryContext::kInvalidPredicateId /* residual_predicate_index */,
-                           selection_index,
-                           0  /* dummy query ID */));
+  unique_ptr<HashJoinOperator> prober(new HashJoinOperator(
+      0 /* dummy query ID */,
+      *dim_table_,
+      *fact_table_,
+      true /* is_stored */,
+      fact_key_attrs,
+      fact_col_long.getType().isNullable() ||
+          fact_col_varchar.getType().isNullable(),
+      *result_table,
+      output_destination_index,
+      join_hash_table_index,
+      QueryContext::kInvalidPredicateId /* residual_predicate_index */,
+      selection_index));
 
   // Set up the QueryContext.
   query_context_.reset(new QueryContext(query_context_proto,
@@ -1068,7 +1071,7 @@ TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinTest) {
   }
 
   // Create cleaner operator.
-  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(join_hash_table_index, 0  /* dummy query ID */));
+  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(0  /* dummy query ID */, join_hash_table_index));
   cleaner->informAllBlockingDependenciesMet();
   fetchAndExecuteWorkOrders(cleaner.get());
 
@@ -1119,12 +1122,12 @@ TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinWithResidualPredicateTest) {
   dim_key_attrs.push_back(dim_col_varchar.getID());
 
   unique_ptr<BuildHashOperator> builder(
-      new BuildHashOperator(*dim_table_,
+      new BuildHashOperator(0,  /* dummy query ID */
+                            *dim_table_,
                             true /* is_stored */,
                             dim_key_attrs,
                             dim_col_long.getType().isNullable() || dim_col_varchar.getType().isNullable(),
-                            join_hash_table_index,
-                            0));  // dummy query ID.
+                            join_hash_table_index));
 
   // Create prober operator with two selection attributes.
   const QueryContext::scalar_group_id selection_index = query_context_proto.scalar_groups_size();
@@ -1166,17 +1169,18 @@ TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinWithResidualPredicateTest) {
   query_context_proto.add_predicates()->CopyFrom(residual_pred->getProto());
 
   unique_ptr<HashJoinOperator> prober(
-      new HashJoinOperator(*dim_table_,
+      new HashJoinOperator(0 /* dummy query ID */,
+                           *dim_table_,
                            *fact_table_,
                            true /* is_stored */,
                            fact_key_attrs,
-                           fact_col_long.getType().isNullable() || fact_col_varchar.getType().isNullable(),
+                           fact_col_long.getType().isNullable() ||
+                               fact_col_varchar.getType().isNullable(),
                            *result_table,
                            output_destination_index,
                            join_hash_table_index,
                            residual_pred_index,
-                           selection_index,
-                           0  /* dummy query ID */));
+                           selection_index));
 
   // Set up the QueryContext.
   query_context_.reset(new QueryContext(query_context_proto,
@@ -1254,7 +1258,7 @@ TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinWithResidualPredicateTest) {
   }
 
   // Create cleaner operator.
-  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(join_hash_table_index, 0  /* dummy query ID */));
+  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(0  /* dummy query ID */, join_hash_table_index));
   cleaner->informAllBlockingDependenciesMet();
   fetchAndExecuteWorkOrders(cleaner.get());
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d67f61e1/relational_operators/tests/SortMergeRunOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/SortMergeRunOperator_unittest.cpp b/relational_operators/tests/SortMergeRunOperator_unittest.cpp
index 244091f..37f155c 100644
--- a/relational_operators/tests/SortMergeRunOperator_unittest.cpp
+++ b/relational_operators/tests/SortMergeRunOperator_unittest.cpp
@@ -1565,7 +1565,8 @@ class SortMergeRunOperatorTest : public ::testing::Test {
                            const std::size_t top_k = 0) {
     const QueryContext::sort_config_id sort_config_index = createSortConfigProto(attrs, ordering, null_ordering);
 
-    merge_op_.reset(new SortMergeRunOperator(*input_table_,
+    merge_op_.reset(new SortMergeRunOperator(0  /* dummy query ID */,
+                                             *input_table_,
                                              *result_table_,
                                              insert_destination_index_,
                                              *run_table_,
@@ -1573,8 +1574,8 @@ class SortMergeRunOperatorTest : public ::testing::Test {
                                              sort_config_index,
                                              merge_factor,
                                              top_k,
-                                             true,
-                                             0  /* dummy query ID */));
+                                             true));
+
     merge_op_->setOperatorIndex(kOpIndex);
 
     // Set up the QueryContext.
@@ -1609,7 +1610,8 @@ class SortMergeRunOperatorTest : public ::testing::Test {
                         const std::size_t top_k = 0) {
     const QueryContext::sort_config_id sort_config_index = createSortConfigProto(attrs, ordering, null_ordering);
 
-    merge_op_.reset(new SortMergeRunOperator(*input_table_,
+    merge_op_.reset(new SortMergeRunOperator(0  /* dummy query ID */,
+                                             *input_table_,
                                              *result_table_,
                                              insert_destination_index_,
                                              *run_table_,
@@ -1617,8 +1619,7 @@ class SortMergeRunOperatorTest : public ::testing::Test {
                                              sort_config_index,
                                              merge_factor,
                                              top_k,
-                                             false,
-                                             0  /* dummy query ID */));
+                                             false));
     merge_op_->setOperatorIndex(kOpIndex);
 
     // Set up the QueryContext.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d67f61e1/relational_operators/tests/SortRunGenerationOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/SortRunGenerationOperator_unittest.cpp b/relational_operators/tests/SortRunGenerationOperator_unittest.cpp
index 6f24b92..846c472 100644
--- a/relational_operators/tests/SortRunGenerationOperator_unittest.cpp
+++ b/relational_operators/tests/SortRunGenerationOperator_unittest.cpp
@@ -353,13 +353,14 @@ class SortRunGenerationOperatorTest : public ::testing::Test {
       order_by_proto->set_null_first(null_ordering[i]);
     }
 
-    std::unique_ptr<RelationalOperator> run_gen(
-        new SortRunGenerationOperator(*input_table_,
-                                      *result_table_,
-                                      insert_destination_index,
-                                      sort_config_index,
-                                      true /* is_stored */,
-                                      0  /* dummy query ID */));
+    std::unique_ptr<RelationalOperator> run_gen(new SortRunGenerationOperator(
+        0 /* dummy query ID */,
+        *input_table_,
+        *result_table_,
+        insert_destination_index,
+        sort_config_index,
+        true /* is_stored */));
+
     run_gen->setOperatorIndex(kOpIndex);
 
     // Set up the QueryContext.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/d67f61e1/relational_operators/tests/TextScanOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/TextScanOperator_unittest.cpp b/relational_operators/tests/TextScanOperator_unittest.cpp
index 7626686..15697f8 100644
--- a/relational_operators/tests/TextScanOperator_unittest.cpp
+++ b/relational_operators/tests/TextScanOperator_unittest.cpp
@@ -188,13 +188,13 @@ TEST_F(TextScanOperatorTest, ScanTest) {
   output_destination_proto->set_relational_op_index(kOpIndex);
 
   std::unique_ptr<TextScanOperator> text_scan_op(
-      new TextScanOperator(input_filename,
+      new TextScanOperator(0  /* dummy query ID */,
+                           input_filename,
                            '\t',
                            true,
                            false,
                            *relation_,
-                           output_destination_index,
-                           0  /* dummy query ID */));
+                           output_destination_index));
 
   // Setup query_context_.
   query_context_.reset(new QueryContext(query_context_proto,