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:16 UTC

[47/48] incubator-quickstep git commit: Reordered Query ID in operators and work orders.

Reordered Query ID in 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/1be47dcb
Tree: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/tree/1be47dcb
Diff: http://git-wip-us.apache.org/repos/asf/incubator-quickstep/diff/1be47dcb

Branch: refs/heads/query-manager-used-in-foreman
Commit: 1be47dcbd79e6e2ee01f237d0dbaa8a97e562519
Parents: 4054268
Author: Harshad Deshmukh <hb...@apache.org>
Authored: Fri Jun 3 15:40:34 2016 -0500
Committer: Zuyu Zhang <zz...@pivotal.io>
Committed: Wed Jun 8 14:00:13 2016 -0700

----------------------------------------------------------------------
 query_execution/Foreman.cpp                     |  12 +-
 query_execution/QueryManager.cpp                |  12 +-
 query_execution/tests/Foreman_unittest.cpp      |   3 +-
 query_execution/tests/QueryManager_unittest.cpp |   3 +-
 query_optimizer/ExecutionGenerator.cpp          | 194 ++++++++--------
 .../tests/ExecutionHeuristics_unittest.cpp      |  34 +--
 relational_operators/AggregationOperator.cpp    |   4 +-
 relational_operators/AggregationOperator.hpp    |  14 +-
 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       | 229 ++++++++++---------
 relational_operators/InsertOperator.hpp         |  11 +-
 .../NestedLoopsJoinOperator.cpp                 |  45 ++--
 .../NestedLoopsJoinOperator.hpp                 |  52 +++--
 relational_operators/RebuildWorkOrder.hpp       |  15 +-
 relational_operators/RelationalOperator.hpp     |  11 +-
 relational_operators/SampleOperator.cpp         |  52 +++--
 relational_operators/SampleOperator.hpp         |  39 ++--
 relational_operators/SaveBlocksOperator.cpp     |   2 +-
 relational_operators/SaveBlocksOperator.hpp     |   9 +-
 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      |  23 +-
 .../tests/HashJoinOperator_unittest.cpp         | 181 ++++++++-------
 .../tests/SortMergeRunOperator_unittest.cpp     |  14 +-
 .../SortRunGenerationOperator_unittest.cpp      |  16 +-
 .../tests/TextScanOperator_unittest.cpp         |   7 +-
 46 files changed, 745 insertions(+), 669 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1be47dcb/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/1be47dcb/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/1be47dcb/query_execution/tests/Foreman_unittest.cpp
----------------------------------------------------------------------
diff --git a/query_execution/tests/Foreman_unittest.cpp b/query_execution/tests/Foreman_unittest.cpp
index d2f43a4..79f8f4a 100644
--- a/query_execution/tests/Foreman_unittest.cpp
+++ b/query_execution/tests/Foreman_unittest.cpp
@@ -90,7 +90,8 @@ class MockOperator: public RelationalOperator {
                const bool has_streaming_input,
                const int max_getworkorder_iters = 1,
                const int max_workorders = INT_MAX)
-      : produce_workorders_(produce_workorders),
+      : RelationalOperator(0 /* Query Id */),
+        produce_workorders_(produce_workorders),
         has_streaming_input_(has_streaming_input),
         max_workorders_(max_workorders),
         max_getworkorder_iters_(max_getworkorder_iters),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1be47dcb/query_execution/tests/QueryManager_unittest.cpp
----------------------------------------------------------------------
diff --git a/query_execution/tests/QueryManager_unittest.cpp b/query_execution/tests/QueryManager_unittest.cpp
index 80876f2..308d5ca 100644
--- a/query_execution/tests/QueryManager_unittest.cpp
+++ b/query_execution/tests/QueryManager_unittest.cpp
@@ -91,7 +91,8 @@ class MockOperator: public RelationalOperator {
                const bool has_streaming_input,
                const int max_getworkorder_iters = 1,
                const int max_workorders = INT_MAX)
-      : produce_workorders_(produce_workorders),
+      : RelationalOperator(0 /* Query Id */),
+        produce_workorders_(produce_workorders),
         has_streaming_input_(has_streaming_input),
         max_workorders_(max_workorders),
         max_getworkorder_iters_(max_getworkorder_iters),

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1be47dcb/query_optimizer/ExecutionGenerator.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/ExecutionGenerator.cpp b/query_optimizer/ExecutionGenerator.cpp
index 30dfa8e..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 =
@@ -1429,10 +1432,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,
@@ -1450,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,
@@ -1499,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,
@@ -1553,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 */);
@@ -1576,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(
@@ -1613,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/1be47dcb/query_optimizer/tests/ExecutionHeuristics_unittest.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/ExecutionHeuristics_unittest.cpp b/query_optimizer/tests/ExecutionHeuristics_unittest.cpp
index a08a476..815c13e 100644
--- a/query_optimizer/tests/ExecutionHeuristics_unittest.cpp
+++ b/query_optimizer/tests/ExecutionHeuristics_unittest.cpp
@@ -15,6 +15,7 @@
  *   limitations under the License.
  **/
 
+#include <cstddef>
 #include <memory>
 #include <string>
 #include <vector>
@@ -36,6 +37,10 @@
 namespace quickstep {
 namespace optimizer {
 
+namespace {
+constexpr std::size_t kQueryId = 0;
+}
+
 class ExecutionHeuristicsTest : public ::testing::Test {
  protected:
   virtual void SetUp() {
@@ -75,12 +80,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(kQueryId,
+                                                                *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 +97,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(kQueryId,
+                                 *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/1be47dcb/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/1be47dcb/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/1be47dcb/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/1be47dcb/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/1be47dcb/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/1be47dcb/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/1be47dcb/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/1be47dcb/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/1be47dcb/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/1be47dcb/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/1be47dcb/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/1be47dcb/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/1be47dcb/relational_operators/HashJoinOperator.hpp
----------------------------------------------------------------------
diff --git a/relational_operators/HashJoinOperator.hpp b/relational_operators/HashJoinOperator.hpp
index 825f360..1d5d4e3 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),
@@ -608,6 +615,7 @@ class HashOuterJoinWorkOrder : 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
@@ -623,21 +631,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 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.
    **/
-  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/1be47dcb/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/1be47dcb/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_);