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/09 01:10:13 UTC

[44/48] incubator-quickstep git commit: Added Query ID to Relational operators and WorkOrders.

Added Query ID to Relational operators 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/40542682
Tree: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/tree/40542682
Diff: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/diff/40542682

Branch: refs/heads/query-manager-used-in-foreman
Commit: 405426823d0e12c75cef447c130fb2c92cacb90f
Parents: 83935e7
Author: Harshad Deshmukh <ha...@cs.wisc.edu>
Authored: Sun May 1 23:01:39 2016 -0500
Committer: Zuyu Zhang <zz...@pivotal.io>
Committed: Wed Jun 8 11:57:49 2016 -0700

----------------------------------------------------------------------
 query_execution/Foreman.cpp                     |  3 +
 query_execution/QueryManager.cpp                |  1 +
 query_execution/tests/Foreman_unittest.cpp      |  2 +-
 query_execution/tests/QueryManager_unittest.cpp |  2 +-
 .../tests/WorkOrdersContainer_unittest.cpp      |  2 +-
 query_optimizer/ExecutionGenerator.cpp          | 68 +++++++++++++-------
 .../tests/ExecutionHeuristics_unittest.cpp      |  6 +-
 relational_operators/AggregationOperator.cpp    | 12 ++--
 relational_operators/AggregationOperator.hpp    | 12 +++-
 relational_operators/BuildHashOperator.cpp      |  2 +
 relational_operators/BuildHashOperator.hpp      | 17 +++--
 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    | 12 +++-
 relational_operators/DropTableOperator.cpp      |  3 +-
 relational_operators/DropTableOperator.hpp      | 13 +++-
 .../FinalizeAggregationOperator.cpp             |  6 +-
 .../FinalizeAggregationOperator.hpp             | 15 +++--
 relational_operators/HashJoinOperator.cpp       |  4 ++
 relational_operators/HashJoinOperator.hpp       | 49 ++++++++++----
 relational_operators/InsertOperator.cpp         |  6 +-
 relational_operators/InsertOperator.hpp         | 15 +++--
 .../NestedLoopsJoinOperator.cpp                 |  4 ++
 .../NestedLoopsJoinOperator.hpp                 | 12 +++-
 relational_operators/RebuildWorkOrder.hpp       |  5 +-
 relational_operators/RelationalOperator.hpp     |  8 ++-
 relational_operators/SampleOperator.cpp         |  7 +-
 relational_operators/SampleOperator.hpp         | 13 ++--
 relational_operators/SaveBlocksOperator.cpp     |  1 +
 relational_operators/SaveBlocksOperator.hpp     | 12 +++-
 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/WorkOrder.hpp              |  9 ++-
 relational_operators/WorkOrder.proto            |  1 +
 relational_operators/WorkOrderFactory.cpp       | 32 +++++++--
 .../tests/AggregationOperator_unittest.cpp      | 16 +++--
 .../tests/HashJoinOperator_unittest.cpp         | 48 ++++++++------
 .../tests/SortMergeRunOperator_unittest.cpp     |  6 +-
 .../SortRunGenerationOperator_unittest.cpp      |  3 +-
 .../tests/TextScanOperator_unittest.cpp         |  3 +-
 53 files changed, 430 insertions(+), 152 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/40542682/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/40542682/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/40542682/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/40542682/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/40542682/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/40542682/query_optimizer/ExecutionGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.cpp b/query_optimizer/ExecutionGenerator.cpp
index c590b6e..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);
@@ -741,7 +745,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;
@@ -787,13 +792,14 @@ 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));
+      execution_plan_->addRelationalOperator(new DestroyHashOperator(
+          join_hash_table_index, query_handle_->query_id()));
 
   if (!build_relation_info->isStoredRelation()) {
     execution_plan_->addDirectDependency(build_operator_index,
@@ -887,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()) {
@@ -938,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 */);
@@ -991,6 +999,7 @@ void ExecutionGenerator::convertCreateIndex(
   }
   execution_plan_->addRelationalOperator(new CreateIndexOperator(input_relation,
                                                                  physical_plan->index_name(),
+                                                                 query_handle_->query_id(),
                                                                  std::move(index_description)));
 }
 
@@ -1031,7 +1040,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()));
 }
 
@@ -1058,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()) {
@@ -1073,7 +1084,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,
@@ -1082,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 */);
@@ -1100,6 +1112,7 @@ void ExecutionGenerator::convertDropTable(
 
   execution_plan_->addRelationalOperator(
       new DropTableOperator(catalog_relation,
+                            query_handle_->query_id(),
                             optimizer_context_->catalog_database()));
 }
 
@@ -1153,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,
@@ -1231,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,
@@ -1311,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,
@@ -1415,7 +1431,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,
@@ -1435,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,
@@ -1486,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,
@@ -1543,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 */);
@@ -1557,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(
@@ -1594,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/40542682/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/40542682/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/40542682/relational_operators/AggregationOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/AggregationOperator.hpp b/relational_operators/AggregationOperator.hpp
index 0e74dfc..0fbc381 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),
@@ -107,11 +110,14 @@ 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,
+                       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/40542682/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/40542682/relational_operators/BuildHashOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/BuildHashOperator.hpp b/relational_operators/BuildHashOperator.hpp
index f9d830f..5a46d8b 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 query_id 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/40542682/relational_operators/CreateIndexOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/CreateIndexOperator.hpp b/relational_operators/CreateIndexOperator.hpp
index 2bfacc4..ede3f02 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 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,
                       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/40542682/relational_operators/CreateTableOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/CreateTableOperator.hpp b/relational_operators/CreateTableOperator.hpp
index 98f3253..60bcef4 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 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.
    * @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/40542682/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/40542682/relational_operators/DeleteOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/DeleteOperator.hpp b/relational_operators/DeleteOperator.hpp
index 1d44552..ba1f825 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 query_id 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 query_id The ID of the query to which this workorder 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/40542682/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/40542682/relational_operators/DestroyHashOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/DestroyHashOperator.hpp b/relational_operators/DestroyHashOperator.hpp
index 46331ba..086c279 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 query_id 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,11 +79,14 @@ 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 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/40542682/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/40542682/relational_operators/DropTableOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/DropTableOperator.hpp b/relational_operators/DropTableOperator.hpp
index bf9b1b1..0bbb718 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 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.
    **/
   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 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.
    * @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/40542682/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/40542682/relational_operators/FinalizeAggregationOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/FinalizeAggregationOperator.hpp b/relational_operators/FinalizeAggregationOperator.hpp
index fb9608a..51e55dc 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 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)
-      : 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 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.
    */
-  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/40542682/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/40542682/relational_operators/HashJoinOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/HashJoinOperator.hpp b/relational_operators/HashJoinOperator.hpp
index fcc087a..825f360 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),
@@ -243,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.
    **/
@@ -254,9 +258,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),
@@ -286,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.
    **/
@@ -297,9 +304,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),
@@ -366,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.
    **/
@@ -377,9 +387,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),
@@ -409,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.
    **/
@@ -420,9 +433,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),
@@ -482,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.
    **/
@@ -493,9 +509,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),
@@ -525,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.
    **/
@@ -536,9 +555,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 +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 to which this WorkOrder belongs.
    * @param output_destination The InsertDestination to insert the join results.
    * @param storage_manager The StorageManager to use.
    **/
@@ -613,9 +635,11 @@ 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),
+      : 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),
@@ -636,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.
    **/
@@ -655,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/40542682/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/40542682/relational_operators/InsertOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/InsertOperator.hpp b/relational_operators/InsertOperator.hpp
index 8a06c94..ccef444 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 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)
-      : 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 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.
    **/
-  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/40542682/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/40542682/relational_operators/NestedLoopsJoinOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/NestedLoopsJoinOperator.hpp b/relational_operators/NestedLoopsJoinOperator.hpp
index a52ca25..f165442 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 query_id 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 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.
    **/
@@ -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/40542682/relational_operators/RebuildWorkOrder.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/RebuildWorkOrder.hpp b/relational_operators/RebuildWorkOrder.hpp
index 5443d48..ae876ba 100644
--- a/relational_operators/RebuildWorkOrder.hpp
+++ b/relational_operators/RebuildWorkOrder.hpp
@@ -55,14 +55,17 @@ 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)
-      : 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/40542682/relational_operators/RelationalOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/RelationalOperator.hpp b/relational_operators/RelationalOperator.hpp
index 75fde17..aa93018 100644
--- a/relational_operators/RelationalOperator.hpp
+++ b/relational_operators/RelationalOperator.hpp
@@ -208,16 +208,20 @@ 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.
    **/
-  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/40542682/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/40542682/relational_operators/SampleOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SampleOperator.hpp b/relational_operators/SampleOperator.hpp
index 305de34..f65f28a 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 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)
-      : 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/40542682/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/40542682/relational_operators/SaveBlocksOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/SaveBlocksOperator.hpp b/relational_operators/SaveBlocksOperator.hpp
index 49195ea..f4650bb 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,12 +99,15 @@ 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),
+      : 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/40542682/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/40542682/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/40542682/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_,