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

[07/50] [abbrv] incubator-quickstep git commit: Added TMB Client Id in QueryHandle.

Added TMB Client Id in QueryHandle.


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

Branch: refs/heads/travis-grpc
Commit: 5e5ec9e3bbe12015fdac950f552672562f1312bf
Parents: 1b07eaa
Author: Zuyu Zhang <zu...@twitter.com>
Authored: Wed Aug 3 12:19:48 2016 -0700
Committer: Zuyu Zhang <zu...@twitter.com>
Committed: Thu Aug 4 09:44:23 2016 -0700

----------------------------------------------------------------------
 cli/CommandExecutor.cpp                           |  7 ++++++-
 cli/QuickstepCli.cpp                              | 16 ++++++++++------
 cli/tests/CommandExecutorTestRunner.cpp           |  2 +-
 .../tests/QueryManagerSingleNode_unittest.cpp     |  2 +-
 query_optimizer/CMakeLists.txt                    |  5 ++---
 query_optimizer/QueryHandle.hpp                   | 18 +++++++++++++++++-
 query_optimizer/QueryProcessor.cpp                | 12 +++---------
 query_optimizer/QueryProcessor.hpp                | 14 +++++++++++---
 .../tests/ExecutionGeneratorTestRunner.cpp        |  2 +-
 9 files changed, 52 insertions(+), 26 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5e5ec9e3/cli/CommandExecutor.cpp
----------------------------------------------------------------------
diff --git a/cli/CommandExecutor.cpp b/cli/CommandExecutor.cpp
index 8acfae8..82c1ad9 100644
--- a/cli/CommandExecutor.cpp
+++ b/cli/CommandExecutor.cpp
@@ -210,9 +210,14 @@ inline TypedValue executeQueryForSingleResult(
   ParseResult result = parser_wrapper->getNextStatement();
   DCHECK(result.condition == ParseResult::kSuccess);
 
+  const ParseStatement &statement = *result.parsed_statement;
+
   // Generate the query plan.
   std::unique_ptr<QueryHandle> query_handle(
-      query_processor->generateQueryHandle(*result.parsed_statement));
+      std::make_unique<QueryHandle>(query_processor->query_id(),
+                                    main_thread_client_id,
+                                    statement.getPriority()));
+  query_processor->generateQueryHandle(statement, query_handle.get());
   DCHECK(query_handle->getQueryPlanMutable() != nullptr);
 
   // Use foreman to execute the query plan.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5e5ec9e3/cli/QuickstepCli.cpp
----------------------------------------------------------------------
diff --git a/cli/QuickstepCli.cpp b/cli/QuickstepCli.cpp
index 154c689..78d7765 100644
--- a/cli/QuickstepCli.cpp
+++ b/cli/QuickstepCli.cpp
@@ -403,16 +403,17 @@ int main(int argc, char* argv[]) {
     bool reset_parser = false;
     for (;;) {
       ParseResult result = parser_wrapper->getNextStatement();
+      const ParseStatement &statement = *result.parsed_statement;
       if (result.condition == ParseResult::kSuccess) {
-        if (result.parsed_statement->getStatementType() == ParseStatement::kQuit) {
+        if (statement.getStatementType() == ParseStatement::kQuit) {
           quitting = true;
           break;
         }
 
-        if (result.parsed_statement->getStatementType() == ParseStatement::kCommand) {
+        if (statement.getStatementType() == ParseStatement::kCommand) {
           try {
             quickstep::cli::executeCommand(
-                *result.parsed_statement,
+                statement,
                 *(query_processor->getDefaultDatabase()),
                 main_thread_client_id,
                 foreman.getBusClientID(),
@@ -426,12 +427,15 @@ int main(int argc, char* argv[]) {
             reset_parser = true;
             break;
           }
-        continue;
+          continue;
         }
 
-        std::unique_ptr<QueryHandle> query_handle;
+        std::unique_ptr<QueryHandle> query_handle(
+            std::make_unique<QueryHandle>(query_processor->query_id(),
+                                          main_thread_client_id,
+                                          statement.getPriority()));
         try {
-          query_handle.reset(query_processor->generateQueryHandle(*result.parsed_statement));
+          query_processor->generateQueryHandle(statement, query_handle.get());
         } catch (const quickstep::SqlError &sql_error) {
           fprintf(stderr, "%s", sql_error.formatMessage(*command_string).c_str());
           reset_parser = true;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5e5ec9e3/cli/tests/CommandExecutorTestRunner.cpp
----------------------------------------------------------------------
diff --git a/cli/tests/CommandExecutorTestRunner.cpp b/cli/tests/CommandExecutorTestRunner.cpp
index dc7e43f..56f4ab9 100644
--- a/cli/tests/CommandExecutorTestRunner.cpp
+++ b/cli/tests/CommandExecutorTestRunner.cpp
@@ -98,7 +98,7 @@ void CommandExecutorTestRunner::runTestCase(
               nullptr,
               output_stream.file());
         } else  {
-          QueryHandle query_handle(0 /* query_id */);
+          QueryHandle query_handle(0 /* query_id */, main_thread_client_id_);
           O::LogicalGenerator logical_generator(&optimizer_context);
           O::PhysicalGenerator physical_generator;
           O::ExecutionGenerator execution_generator(&optimizer_context, &query_handle);

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5e5ec9e3/query_execution/tests/QueryManagerSingleNode_unittest.cpp
----------------------------------------------------------------------
diff --git a/query_execution/tests/QueryManagerSingleNode_unittest.cpp b/query_execution/tests/QueryManagerSingleNode_unittest.cpp
index 09ae6ba..050b315 100644
--- a/query_execution/tests/QueryManagerSingleNode_unittest.cpp
+++ b/query_execution/tests/QueryManagerSingleNode_unittest.cpp
@@ -232,7 +232,7 @@ class QueryManagerTest : public ::testing::Test {
     db_.reset(new CatalogDatabase(nullptr /* catalog */, "database"));
     storage_manager_.reset(new StorageManager("./"));
     bus_.Initialize();
-    query_handle_.reset(new QueryHandle(0));  // dummy query ID.
+    query_handle_.reset(new QueryHandle(0 /* dummy query ID */, tmb::kClientIdNone /* cli_id */));
     query_plan_ = query_handle_->getQueryPlanMutable();
     query_handle_->getQueryContextProtoMutable()->set_query_id(query_handle_->query_id());
   }

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5e5ec9e3/query_optimizer/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/query_optimizer/CMakeLists.txt b/query_optimizer/CMakeLists.txt
index 04e54d3..de63bd0 100644
--- a/query_optimizer/CMakeLists.txt
+++ b/query_optimizer/CMakeLists.txt
@@ -211,7 +211,8 @@ target_link_libraries(quickstep_queryoptimizer_QueryHandle
                       quickstep_catalog_Catalog_proto
                       quickstep_queryexecution_QueryContext_proto
                       quickstep_queryoptimizer_QueryPlan
-                      quickstep_utility_Macros)
+                      quickstep_utility_Macros
+                      tmb)
 target_link_libraries(quickstep_queryoptimizer_QueryPlan
                       quickstep_relationaloperators_RelationalOperator
                       quickstep_utility_DAG
@@ -219,9 +220,7 @@ target_link_libraries(quickstep_queryoptimizer_QueryPlan
 target_link_libraries(quickstep_queryoptimizer_QueryProcessor
                       quickstep_catalog_Catalog
                       quickstep_catalog_Catalog_proto
-                      quickstep_parser_ParseStatement
                       quickstep_queryoptimizer_Optimizer
-                      quickstep_queryoptimizer_QueryHandle
                       quickstep_storage_StorageManager
                       quickstep_utility_Macros)
 target_link_libraries(quickstep_queryoptimizer_Validator

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5e5ec9e3/query_optimizer/QueryHandle.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/QueryHandle.hpp b/query_optimizer/QueryHandle.hpp
index 5f3649a..55427cd 100644
--- a/query_optimizer/QueryHandle.hpp
+++ b/query_optimizer/QueryHandle.hpp
@@ -20,13 +20,14 @@
 #include <cstddef>
 #include <cstdint>
 #include <memory>
-#include <utility>
 
 #include "catalog/Catalog.pb.h"
 #include "query_execution/QueryContext.pb.h"
 #include "query_optimizer/QueryPlan.hpp"
 #include "utility/Macros.hpp"
 
+#include "tmb/id_typedefs.h"
+
 namespace quickstep {
 
 class CatalogRelation;
@@ -44,10 +45,14 @@ class QueryHandle {
    * @brief Constructor.
    *
    * @param query_id The given query id.
+   * @param cli_id The client id of the CLI which submits the query.
+   * @param query_priority The priority of this query.
    */
   explicit QueryHandle(const std::size_t query_id,
+                       const tmb::client_id cli_id,
                        const std::uint64_t query_priority = 1)
       : query_id_(query_id),
+        cli_id_(cli_id),
         query_priority_(query_priority),
         query_plan_(new QueryPlan()),
         query_result_relation_(nullptr) {}
@@ -64,6 +69,13 @@ class QueryHandle {
   }
 
   /**
+   * @brief Get the client id of the CLI which submits the query.
+   */
+  tmb::client_id getClientId() const {
+    return cli_id_;
+  }
+
+  /**
    * @brief Get the query priority.
    **/
   const std::uint64_t query_priority() const {
@@ -121,6 +133,10 @@ class QueryHandle {
 
  private:
   const std::size_t query_id_;
+
+  // The client id of the CLI which submits the query.
+  const tmb::client_id cli_id_;
+
   const std::uint64_t query_priority_;
 
   std::unique_ptr<QueryPlan> query_plan_;

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5e5ec9e3/query_optimizer/QueryProcessor.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/QueryProcessor.cpp b/query_optimizer/QueryProcessor.cpp
index 8af4408..f3693a1 100644
--- a/query_optimizer/QueryProcessor.cpp
+++ b/query_optimizer/QueryProcessor.cpp
@@ -23,28 +23,22 @@
 
 #include "catalog/Catalog.hpp"
 #include "catalog/Catalog.pb.h"
-#include "parser/ParseStatement.hpp"
 #include "query_optimizer/Optimizer.hpp"
-#include "query_optimizer/QueryHandle.hpp"
 
 using std::ifstream;
 using std::ofstream;
 
 namespace quickstep {
 
-QueryHandle* QueryProcessor::generateQueryHandle(const ParseStatement &statement) {
-  std::unique_ptr<QueryHandle> query_handle(
-      new QueryHandle(query_id_, statement.getPriority()));
-
-  optimizer_->generateQueryHandle(statement, query_handle.get());
+void QueryProcessor::generateQueryHandle(const ParseStatement &statement,
+                                         QueryHandle *query_handle) {
+  optimizer_->generateQueryHandle(statement, query_handle);
 
   if (optimizer_->isCatalogChanged() && !catalog_altered_) {
     catalog_altered_ = true;
   }
 
   ++query_id_;
-
-  return query_handle.release();
 }
 
 void QueryProcessor::saveCatalog() {

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5e5ec9e3/query_optimizer/QueryProcessor.hpp
----------------------------------------------------------------------
diff --git a/query_optimizer/QueryProcessor.hpp b/query_optimizer/QueryProcessor.hpp
index f3844a0..43e462f 100644
--- a/query_optimizer/QueryProcessor.hpp
+++ b/query_optimizer/QueryProcessor.hpp
@@ -151,13 +151,21 @@ class QueryProcessor {
   ~QueryProcessor() {}
 
   /**
-   * @brief Create a query handle for the given parsed SQL statement. This
+   * @brief Get the next query id.
+   **/
+  std::size_t query_id() const {
+    return query_id_;
+  }
+
+  /**
+   * @brief Fill a query handle for the given parsed SQL statement. This
    *        includes that the optimizer creates a QueryPlan inside the handle.
    *
    * @param statement The parsed SQL statement to generate a query handle for.
-   * @return A query handle for statement.
+   * @param query_handle The generated query handle to output.
    **/
-  QueryHandle* generateQueryHandle(const ParseStatement &statement);
+  void generateQueryHandle(const ParseStatement &statement,
+                           QueryHandle *query_handle);
 
   /**
    * @brief Save the catalog back to disk.

http://git-wip-us.apache.org/repos/asf/incubator-quickstep/blob/5e5ec9e3/query_optimizer/tests/ExecutionGeneratorTestRunner.cpp
----------------------------------------------------------------------
diff --git a/query_optimizer/tests/ExecutionGeneratorTestRunner.cpp b/query_optimizer/tests/ExecutionGeneratorTestRunner.cpp
index fd1bb86..195ca59 100644
--- a/query_optimizer/tests/ExecutionGeneratorTestRunner.cpp
+++ b/query_optimizer/tests/ExecutionGeneratorTestRunner.cpp
@@ -84,7 +84,7 @@ void ExecutionGeneratorTestRunner::runTestCase(
     } else {
       std::printf("%s\n", result.parsed_statement->toString().c_str());
       try {
-        QueryHandle query_handle(0 /* query_id */);
+        QueryHandle query_handle(0 /* query_id */, main_thread_client_id_);
         LogicalGenerator logical_generator(&optimizer_context);
         PhysicalGenerator physical_generator;
         ExecutionGenerator execution_generator(&optimizer_context,