You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@quickstep.apache.org by ji...@apache.org on 2016/06/07 17:25:38 UTC

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

Query ID added in CreateTable, Index, Delete and DestroyHash


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

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

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


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

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

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

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

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

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

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

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

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

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