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

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

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1be47dcb/relational_operators/tests/HashJoinOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/HashJoinOperator_unittest.cpp b/relational_operators/tests/HashJoinOperator_unittest.cpp
index 4ef5a5c..074b603 100644
--- a/relational_operators/tests/HashJoinOperator_unittest.cpp
+++ b/relational_operators/tests/HashJoinOperator_unittest.cpp
@@ -92,6 +92,7 @@ constexpr tuple_id kNumDimTuples = 200;
 constexpr tuple_id kNumFactTuples = 300;
 constexpr tuple_id kBlockSize = 10;
 
+constexpr std::size_t kQueryId = 0;
 constexpr int kOpIndex = 0;
 
 }  // namespace
@@ -332,12 +333,12 @@ TEST_P(HashJoinOperatorTest, LongKeyHashJoinTest) {
 
   // Create the builder operator.
   unique_ptr<BuildHashOperator> builder(
-      new BuildHashOperator(*dim_table_,
+      new BuildHashOperator(kQueryId,
+                            *dim_table_,
                             true /* is_stored */,
                             std::vector<attribute_id>(1, dim_col_long.getID()),
                             dim_col_long.getType().isNullable(),
-                            join_hash_table_index,
-                            0));  // dummy query ID
+                            join_hash_table_index));
 
   // Create the prober operator with one selection attribute.
   const QueryContext::scalar_group_id selection_index = query_context_proto.scalar_groups_size();
@@ -359,18 +360,18 @@ TEST_P(HashJoinOperatorTest, LongKeyHashJoinTest) {
   insert_destination_proto->set_relation_id(output_relation_id);
   insert_destination_proto->set_relational_op_index(kOpIndex);
 
-  unique_ptr<HashJoinOperator> prober(
-      new HashJoinOperator(*dim_table_,
-                           *fact_table_,
-                           true /* is_stored */,
-                           std::vector<attribute_id>(1, fact_col_long.getID()),
-                           fact_col_long.getType().isNullable(),
-                           *result_table,
-                           output_destination_index,
-                           join_hash_table_index,
-                           QueryContext::kInvalidPredicateId /* residual_predicate_index */,
-                           selection_index,
-                           0  /* dummy query ID */));
+  unique_ptr<HashJoinOperator> prober(new HashJoinOperator(
+      kQueryId,
+      *dim_table_,
+      *fact_table_,
+      true /* is_stored */,
+      std::vector<attribute_id>(1, fact_col_long.getID()),
+      fact_col_long.getType().isNullable(),
+      *result_table,
+      output_destination_index,
+      join_hash_table_index,
+      QueryContext::kInvalidPredicateId /* residual_predicate_index */,
+      selection_index));
 
   // Set up the QueryContext.
   query_context_.reset(new QueryContext(query_context_proto,
@@ -423,7 +424,7 @@ TEST_P(HashJoinOperatorTest, LongKeyHashJoinTest) {
   }
 
   // Create cleaner operator.
-  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(join_hash_table_index, 0  /* dummy query ID */));
+  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(kQueryId, join_hash_table_index));
   cleaner->informAllBlockingDependenciesMet();
   fetchAndExecuteWorkOrders(cleaner.get());
 
@@ -475,12 +476,12 @@ TEST_P(HashJoinOperatorTest, IntDuplicateKeyHashJoinTest) {
 
   // Create the builder operator.
   unique_ptr<BuildHashOperator> builder(
-      new BuildHashOperator(*dim_table_,
+      new BuildHashOperator(kQueryId,
+                            *dim_table_,
                             true /* is_stored */,
                             std::vector<attribute_id>(1, dim_col_int.getID()),
                             dim_col_int.getType().isNullable(),
-                            join_hash_table_index,
-                            0));  // dummy query ID
+                            join_hash_table_index));
 
   // Create the prober operator with two selection attributes.
   const QueryContext::scalar_group_id selection_index = query_context_proto.scalar_groups_size();
@@ -507,18 +508,19 @@ TEST_P(HashJoinOperatorTest, IntDuplicateKeyHashJoinTest) {
   insert_destination_proto->set_relation_id(output_relation_id);
   insert_destination_proto->set_relational_op_index(kOpIndex);
 
-  unique_ptr<HashJoinOperator> prober(
-      new HashJoinOperator(*dim_table_,
-                           *fact_table_,
-                           true /* is_stored */,
-                           std::vector<attribute_id>(1, fact_col_int.getID()),
-                           fact_col_int.getType().isNullable(),
-                           *result_table,
-                           output_destination_index,
-                           join_hash_table_index,
-                           QueryContext::kInvalidPredicateId /* residual_predicate_index */,
-                           selection_index,
-                           0  /* dummy query ID */));
+  unique_ptr<HashJoinOperator> prober(new HashJoinOperator(
+      kQueryId,
+      *dim_table_,
+      *fact_table_,
+      true /* is_stored */,
+      std::vector<attribute_id>(1, fact_col_int.getID()),
+      fact_col_int.getType().isNullable(),
+      *result_table,
+      output_destination_index,
+      join_hash_table_index,
+      QueryContext::kInvalidPredicateId /* residual_predicate_index */,
+      selection_index));
+
 
   // Set up the QueryContext.
   query_context_.reset(new QueryContext(query_context_proto,
@@ -592,7 +594,7 @@ TEST_P(HashJoinOperatorTest, IntDuplicateKeyHashJoinTest) {
   }
 
   // Create cleaner operator.
-  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(join_hash_table_index, 0  /* dummy query ID */));
+  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(kQueryId, join_hash_table_index));
   cleaner->informAllBlockingDependenciesMet();
   fetchAndExecuteWorkOrders(cleaner.get());
 
@@ -636,12 +638,12 @@ TEST_P(HashJoinOperatorTest, CharKeyCartesianProductHashJoinTest) {
 
   // Create builder operator.
   unique_ptr<BuildHashOperator> builder(
-      new BuildHashOperator(*dim_table_,
+      new BuildHashOperator(kQueryId,
+                            *dim_table_,
                             true /* is_stored */,
                             std::vector<attribute_id>(1, dim_col_char.getID()),
                             dim_col_char.getType().isNullable(),
-                            join_hash_table_index,
-                            0));  // dummy query ID.
+                            join_hash_table_index));
 
   // Create prober operator with one selection attribute.
   const QueryContext::scalar_group_id selection_index = query_context_proto.scalar_groups_size();
@@ -663,18 +665,18 @@ TEST_P(HashJoinOperatorTest, CharKeyCartesianProductHashJoinTest) {
   insert_destination_proto->set_relation_id(output_relation_id);
   insert_destination_proto->set_relational_op_index(kOpIndex);
 
-  unique_ptr<HashJoinOperator> prober(
-      new HashJoinOperator(*dim_table_,
-                           *fact_table_,
-                           true /* is_stored */,
-                           std::vector<attribute_id>(1, fact_col_char.getID()),
-                           fact_col_char.getType().isNullable(),
-                           *result_table,
-                           output_destination_index,
-                           join_hash_table_index,
-                           QueryContext::kInvalidPredicateId /* residual_predicate_index */,
-                           selection_index,
-                           0  /* dummy query ID */));
+  unique_ptr<HashJoinOperator> prober(new HashJoinOperator(
+      kQueryId,
+      *dim_table_,
+      *fact_table_,
+      true /* is_stored */,
+      std::vector<attribute_id>(1, fact_col_char.getID()),
+      fact_col_char.getType().isNullable(),
+      *result_table,
+      output_destination_index,
+      join_hash_table_index,
+      QueryContext::kInvalidPredicateId /* residual_predicate_index */,
+      selection_index));
 
   // Set up the QueryContext.
   query_context_.reset(new QueryContext(query_context_proto,
@@ -727,7 +729,7 @@ TEST_P(HashJoinOperatorTest, CharKeyCartesianProductHashJoinTest) {
   }
 
   // Create cleaner operator.
-  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(join_hash_table_index, 0  /* dummy query ID */));
+  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(kQueryId, join_hash_table_index));
   cleaner->informAllBlockingDependenciesMet();
   fetchAndExecuteWorkOrders(cleaner.get());
 
@@ -772,12 +774,12 @@ TEST_P(HashJoinOperatorTest, VarCharDuplicateKeyHashJoinTest) {
 
   // Create builder operator.
   unique_ptr<BuildHashOperator> builder(
-      new BuildHashOperator(*dim_table_,
+      new BuildHashOperator(kQueryId,
+                            *dim_table_,
                             true /* is_stored */,
                             std::vector<attribute_id>(1, dim_col_varchar.getID()),
                             dim_col_varchar.getType().isNullable(),
-                            join_hash_table_index,
-                            0));  // dummy query ID.
+                            join_hash_table_index));
 
   // Create prober operator with two selection attributes.
   const QueryContext::scalar_group_id selection_index = query_context_proto.scalar_groups_size();
@@ -804,18 +806,19 @@ TEST_P(HashJoinOperatorTest, VarCharDuplicateKeyHashJoinTest) {
   insert_destination_proto->set_relation_id(output_relation_id);
   insert_destination_proto->set_relational_op_index(kOpIndex);
 
-  unique_ptr<HashJoinOperator> prober(
-      new HashJoinOperator(*dim_table_,
-                           *fact_table_,
-                           true /* is_stored */,
-                           std::vector<attribute_id>(1, fact_col_varchar.getID()),
-                           fact_col_varchar.getType().isNullable(),
-                           *result_table,
-                           output_destination_index,
-                           join_hash_table_index,
-                           QueryContext::kInvalidPredicateId /* residual_predicate_index */,
-                           selection_index,
-                           0  /* dummy query ID */));
+  unique_ptr<HashJoinOperator> prober(new HashJoinOperator(
+      kQueryId,
+      *dim_table_,
+      *fact_table_,
+      true /* is_stored */,
+      std::vector<attribute_id>(1, fact_col_varchar.getID()),
+      fact_col_varchar.getType().isNullable(),
+      *result_table,
+      output_destination_index,
+      join_hash_table_index,
+      QueryContext::kInvalidPredicateId /* residual_predicate_index */,
+      selection_index));
+
 
   // Set up the QueryContext.
   query_context_.reset(new QueryContext(query_context_proto,
@@ -893,7 +896,7 @@ TEST_P(HashJoinOperatorTest, VarCharDuplicateKeyHashJoinTest) {
   }
 
   // Create the cleaner operator.
-  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(join_hash_table_index, 0  /* dummy query ID */));
+  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(kQueryId, join_hash_table_index));
   cleaner->informAllBlockingDependenciesMet();
   fetchAndExecuteWorkOrders(cleaner.get());
 
@@ -943,12 +946,12 @@ TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinTest) {
   dim_key_attrs.push_back(dim_col_varchar.getID());
 
   unique_ptr<BuildHashOperator> builder(
-      new BuildHashOperator(*dim_table_,
+      new BuildHashOperator(kQueryId,
+                            *dim_table_,
                             true /* is_stored */,
                             dim_key_attrs,
                             dim_col_long.getType().isNullable() || dim_col_varchar.getType().isNullable(),
-                            join_hash_table_index,
-                            0));  // dummy query ID.
+                            join_hash_table_index));
 
   // Create the prober operator with two selection attributes.
   const QueryContext::scalar_group_id selection_index = query_context_proto.scalar_groups_size();
@@ -979,18 +982,19 @@ TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinTest) {
   fact_key_attrs.push_back(fact_col_long.getID());
   fact_key_attrs.push_back(fact_col_varchar.getID());
 
-  unique_ptr<HashJoinOperator> prober(
-      new HashJoinOperator(*dim_table_,
-                           *fact_table_,
-                           true /* is_stored */,
-                           fact_key_attrs,
-                           fact_col_long.getType().isNullable() || fact_col_varchar.getType().isNullable(),
-                           *result_table,
-                           output_destination_index,
-                           join_hash_table_index,
-                           QueryContext::kInvalidPredicateId /* residual_predicate_index */,
-                           selection_index,
-                           0  /* dummy query ID */));
+  unique_ptr<HashJoinOperator> prober(new HashJoinOperator(
+      kQueryId,
+      *dim_table_,
+      *fact_table_,
+      true /* is_stored */,
+      fact_key_attrs,
+      fact_col_long.getType().isNullable() ||
+          fact_col_varchar.getType().isNullable(),
+      *result_table,
+      output_destination_index,
+      join_hash_table_index,
+      QueryContext::kInvalidPredicateId /* residual_predicate_index */,
+      selection_index));
 
   // Set up the QueryContext.
   query_context_.reset(new QueryContext(query_context_proto,
@@ -1068,7 +1072,7 @@ TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinTest) {
   }
 
   // Create cleaner operator.
-  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(join_hash_table_index, 0  /* dummy query ID */));
+  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(kQueryId, join_hash_table_index));
   cleaner->informAllBlockingDependenciesMet();
   fetchAndExecuteWorkOrders(cleaner.get());
 
@@ -1119,12 +1123,12 @@ TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinWithResidualPredicateTest) {
   dim_key_attrs.push_back(dim_col_varchar.getID());
 
   unique_ptr<BuildHashOperator> builder(
-      new BuildHashOperator(*dim_table_,
+      new BuildHashOperator(kQueryId,
+                            *dim_table_,
                             true /* is_stored */,
                             dim_key_attrs,
                             dim_col_long.getType().isNullable() || dim_col_varchar.getType().isNullable(),
-                            join_hash_table_index,
-                            0));  // dummy query ID.
+                            join_hash_table_index));
 
   // Create prober operator with two selection attributes.
   const QueryContext::scalar_group_id selection_index = query_context_proto.scalar_groups_size();
@@ -1166,17 +1170,18 @@ TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinWithResidualPredicateTest) {
   query_context_proto.add_predicates()->CopyFrom(residual_pred->getProto());
 
   unique_ptr<HashJoinOperator> prober(
-      new HashJoinOperator(*dim_table_,
+      new HashJoinOperator(kQueryId,
+                           *dim_table_,
                            *fact_table_,
                            true /* is_stored */,
                            fact_key_attrs,
-                           fact_col_long.getType().isNullable() || fact_col_varchar.getType().isNullable(),
+                           fact_col_long.getType().isNullable() ||
+                               fact_col_varchar.getType().isNullable(),
                            *result_table,
                            output_destination_index,
                            join_hash_table_index,
                            residual_pred_index,
-                           selection_index,
-                           0  /* dummy query ID */));
+                           selection_index));
 
   // Set up the QueryContext.
   query_context_.reset(new QueryContext(query_context_proto,
@@ -1254,7 +1259,7 @@ TEST_P(HashJoinOperatorTest, CompositeKeyHashJoinWithResidualPredicateTest) {
   }
 
   // Create cleaner operator.
-  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(join_hash_table_index, 0  /* dummy query ID */));
+  unique_ptr<DestroyHashOperator> cleaner(new DestroyHashOperator(kQueryId, join_hash_table_index));
   cleaner->informAllBlockingDependenciesMet();
   fetchAndExecuteWorkOrders(cleaner.get());
 

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1be47dcb/relational_operators/tests/SortMergeRunOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/SortMergeRunOperator_unittest.cpp b/relational_operators/tests/SortMergeRunOperator_unittest.cpp
index 244091f..fc10671 100644
--- a/relational_operators/tests/SortMergeRunOperator_unittest.cpp
+++ b/relational_operators/tests/SortMergeRunOperator_unittest.cpp
@@ -84,6 +84,7 @@ namespace quickstep {
 
 namespace {
 
+constexpr std::size_t kQueryId = 0;
 constexpr const std::size_t kOpIndex = 0;
 
 // Helper struct for test tuple that will that will be inserted and sorted.
@@ -1565,7 +1566,8 @@ class SortMergeRunOperatorTest : public ::testing::Test {
                            const std::size_t top_k = 0) {
     const QueryContext::sort_config_id sort_config_index = createSortConfigProto(attrs, ordering, null_ordering);
 
-    merge_op_.reset(new SortMergeRunOperator(*input_table_,
+    merge_op_.reset(new SortMergeRunOperator(kQueryId,
+                                             *input_table_,
                                              *result_table_,
                                              insert_destination_index_,
                                              *run_table_,
@@ -1573,8 +1575,8 @@ class SortMergeRunOperatorTest : public ::testing::Test {
                                              sort_config_index,
                                              merge_factor,
                                              top_k,
-                                             true,
-                                             0  /* dummy query ID */));
+                                             true));
+
     merge_op_->setOperatorIndex(kOpIndex);
 
     // Set up the QueryContext.
@@ -1609,7 +1611,8 @@ class SortMergeRunOperatorTest : public ::testing::Test {
                         const std::size_t top_k = 0) {
     const QueryContext::sort_config_id sort_config_index = createSortConfigProto(attrs, ordering, null_ordering);
 
-    merge_op_.reset(new SortMergeRunOperator(*input_table_,
+    merge_op_.reset(new SortMergeRunOperator(kQueryId,
+                                             *input_table_,
                                              *result_table_,
                                              insert_destination_index_,
                                              *run_table_,
@@ -1617,8 +1620,7 @@ class SortMergeRunOperatorTest : public ::testing::Test {
                                              sort_config_index,
                                              merge_factor,
                                              top_k,
-                                             false,
-                                             0  /* dummy query ID */));
+                                             false));
     merge_op_->setOperatorIndex(kOpIndex);
 
     // Set up the QueryContext.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1be47dcb/relational_operators/tests/SortRunGenerationOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/SortRunGenerationOperator_unittest.cpp b/relational_operators/tests/SortRunGenerationOperator_unittest.cpp
index 6f24b92..71a80e4 100644
--- a/relational_operators/tests/SortRunGenerationOperator_unittest.cpp
+++ b/relational_operators/tests/SortRunGenerationOperator_unittest.cpp
@@ -77,6 +77,7 @@ namespace quickstep {
 
 namespace {
 
+constexpr std::size_t kQueryId = 0;
 constexpr int kOpIndex = 0;
 
 // Helper struct for test tuple that will that will be inserted and sorted.
@@ -353,13 +354,14 @@ class SortRunGenerationOperatorTest : public ::testing::Test {
       order_by_proto->set_null_first(null_ordering[i]);
     }
 
-    std::unique_ptr<RelationalOperator> run_gen(
-        new SortRunGenerationOperator(*input_table_,
-                                      *result_table_,
-                                      insert_destination_index,
-                                      sort_config_index,
-                                      true /* is_stored */,
-                                      0  /* dummy query ID */));
+    std::unique_ptr<RelationalOperator> run_gen(new SortRunGenerationOperator(
+        kQueryId,
+        *input_table_,
+        *result_table_,
+        insert_destination_index,
+        sort_config_index,
+        true /* is_stored */));
+
     run_gen->setOperatorIndex(kOpIndex);
 
     // Set up the QueryContext.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/1be47dcb/relational_operators/tests/TextScanOperator_unittest.cpp
----------------------------------------------------------------------
diff --git a/relational_operators/tests/TextScanOperator_unittest.cpp b/relational_operators/tests/TextScanOperator_unittest.cpp
index 7626686..ef6fc2d 100644
--- a/relational_operators/tests/TextScanOperator_unittest.cpp
+++ b/relational_operators/tests/TextScanOperator_unittest.cpp
@@ -55,6 +55,7 @@ const char *failure_output_filename;
 namespace quickstep {
 
 namespace {
+constexpr std::size_t kQueryId = 0;
 constexpr int kOpIndex = 0;
 }  // namespace
 
@@ -188,13 +189,13 @@ TEST_F(TextScanOperatorTest, ScanTest) {
   output_destination_proto->set_relational_op_index(kOpIndex);
 
   std::unique_ptr<TextScanOperator> text_scan_op(
-      new TextScanOperator(input_filename,
+      new TextScanOperator(kQueryId,
+                           input_filename,
                            '\t',
                            true,
                            false,
                            *relation_,
-                           output_destination_index,
-                           0  /* dummy query ID */));
+                           output_destination_index));
 
   // Setup query_context_.
   query_context_.reset(new QueryContext(query_context_proto,