You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@quickstep.apache.org by zuyu <gi...@git.apache.org> on 2016/07/22 20:32:01 UTC

[GitHub] incubator-quickstep pull request #64: Introduced Shiftboss for the distribut...

GitHub user zuyu opened a pull request:

    https://github.com/apache/incubator-quickstep/pull/64

    Introduced Shiftboss for the distributed version.

    

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/apache/incubator-quickstep shiftboss

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/incubator-quickstep/pull/64.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #64
    
----
commit ffc987a90aea79897c21df2f66f243004cc70839
Author: Zuyu Zhang <zu...@twitter.com>
Date:   2016-07-22T20:29:03Z

    Introduced Shiftboss for the distributed version.

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #64: Introduced Shiftboss for the distribut...

Posted by hbdeshmukh <gi...@git.apache.org>.
Github user hbdeshmukh commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/64#discussion_r72499189
  
    --- Diff: query_execution/QueryExecutionTypedefs.hpp ---
    @@ -73,9 +73,17 @@ enum QueryExecutionMessageType : message_type_id {
       kPoisonMessage,  // From the main thread to Foreman and Workers.
     
     #ifdef QUICKSTEP_DISTRIBUTED
    +  kShiftbossRegistrationMessage,  // From Shiftboss to Foreman.
    +  kShiftbossRegistrationResponseMessage,  // From Foreman to Shiftboss.
    +  kQueryInitiateMessage,  // From Foreman to Shiftboss.
    +  kQueryInitiateResponseMessage,  // From Shiftboss to Foreman.
    +
       kInitiateRebuildMessage,  // From Foreman to Shiftboss.
       kInitiateRebuildResponseMessage,  // From Shiftboss to Foreman.
     
    +  kQueryResultRelationMessage,  // From Foreman to Shiftboss.
    +  kQueryResultRelationResponseMessage,  // From Shiftboss to Foreman.
    --- End diff --
    
    Adding to the above comment: Because there are several types of messages, we can write a function which tests if all the sender and receivers are configured to send and receiver appropriate types of messages. That will give us an assurance that everything is correct in terms of communication. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #64: Introduced Shiftboss for the distribut...

Posted by zuyu <gi...@git.apache.org>.
Github user zuyu commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/64#discussion_r72338431
  
    --- Diff: query_execution/Shiftboss.cpp ---
    @@ -0,0 +1,357 @@
    +/**
    + *   Licensed under the Apache License, Version 2.0 (the "License");
    + *   you may not use this file except in compliance with the License.
    + *   You may obtain a copy of the License at
    + *
    + *       http://www.apache.org/licenses/LICENSE-2.0
    + *
    + *   Unless required by applicable law or agreed to in writing, software
    + *   distributed under the License is distributed on an "AS IS" BASIS,
    + *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + *   See the License for the specific language governing permissions and
    + *   limitations under the License.
    + **/
    +
    +#include "query_execution/Shiftboss.hpp"
    +
    +#include <cstddef>
    +#include <cstdlib>
    +#include <memory>
    +#include <string>
    +#include <unordered_map>
    +#include <utility>
    +#include <vector>
    +
    +#include "catalog/CatalogTypedefs.hpp"
    +#include "query_execution/QueryContext.hpp"
    +#include "query_execution/QueryExecutionMessages.pb.h"
    +#include "query_execution/QueryExecutionTypedefs.hpp"
    +#include "query_execution/QueryExecutionUtil.hpp"
    +#include "query_execution/WorkerMessage.hpp"
    +#include "relational_operators/RebuildWorkOrder.hpp"
    +#include "relational_operators/WorkOrderFactory.hpp"
    +#include "storage/InsertDestination.hpp"
    +#include "storage/StorageBlock.hpp"
    +#include "storage/StorageBlockInfo.hpp"
    +#include "storage/StorageManager.hpp"
    +#include "threading/ThreadUtil.hpp"
    +
    +#include "glog/logging.h"
    +
    +#include "tmb/address.h"
    +#include "tmb/id_typedefs.h"
    +#include "tmb/message_bus.h"
    +#include "tmb/message_style.h"
    +#include "tmb/tagged_message.h"
    +
    +using std::free;
    +using std::malloc;
    +using std::move;
    +using std::size_t;
    +using std::string;
    +using std::unique_ptr;
    +using std::vector;
    +
    +using tmb::TaggedMessage;
    +
    +namespace quickstep {
    +
    +class WorkOrder;
    +
    +void Shiftboss::run() {
    +  if (cpu_id_ >= 0) {
    +    // We can pin the shiftboss thread to a CPU if specified.
    +    ThreadUtil::BindToCPU(cpu_id_);
    +  }
    +
    +  for (;;) {
    +    // Receive() is a blocking call, causing this thread to sleep until next
    +    // message is received.
    +    AnnotatedMessage annotated_message(bus_->Receive(shiftboss_client_id_, 0, true));
    +    LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +              << "') received the typed '" << annotated_message.tagged_message.message_type()
    +              << "' message from client " << annotated_message.sender;
    +    switch (annotated_message.tagged_message.message_type()) {
    +      case kShiftbossRegistrationResponseMessage: {
    +        foreman_client_id_ = annotated_message.sender;
    +        break;
    +      }
    +      case kShiftbossInitiateMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::ShiftbossInitiateMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processShiftbossInitiateMessage(proto.query_id(), proto.catalog_database_cache(), proto.query_context());
    +        break;
    +      }
    +      case kWorkOrderMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::WorkOrderMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        const std::size_t query_id = proto.query_id();
    +        DCHECK_EQ(1u, query_contexts_.count(query_id));
    +
    +        WorkOrder *work_order = WorkOrderFactory::ReconstructFromProto(proto.work_order(),
    +                                                                       &database_cache_,
    +                                                                       query_contexts_[query_id].get(),
    +                                                                       storage_manager_,
    +                                                                       shiftboss_client_id_,
    +                                                                       bus_);
    +
    +        unique_ptr<WorkerMessage> worker_message(
    +            WorkerMessage::WorkOrderMessage(work_order, proto.operator_index()));
    +
    +        TaggedMessage worker_tagged_message(worker_message.get(),
    +                                            sizeof(*worker_message),
    +                                            kWorkOrderMessage);
    +
    +        const size_t worker_index = getSchedulableWorker();
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded WorkOrderMessage (typed '" << kWorkOrderMessage
    +                  << "') from Foreman to worker " << worker_index;
    +
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           workers_->getClientID(worker_index),
    +                                           move(worker_tagged_message));
    +        break;
    +      }
    +      case kInitiateRebuildMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::InitiateRebuildMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processInitiateRebuildMessage(proto.query_id(),
    +                                      proto.operator_index(),
    +                                      proto.insert_destination_index(),
    +                                      proto.relation_id());
    +        break;
    +      }
    +      case kWorkOrderCompleteMessage:  // Fall through.
    +      case kRebuildWorkOrderCompleteMessage:
    +      case kDataPipelineMessage:
    +      case kWorkOrdersAvailableMessage:
    +      case kWorkOrderFeedbackMessage: {
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded typed '" << annotated_message.tagged_message.message_type()
    +                  << "' message from worker (client " << annotated_message.sender << ") to Foreman";
    +
    +        DCHECK_NE(foreman_client_id_, tmb::kClientIdNone);
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           foreman_client_id_,
    +                                           move(annotated_message.tagged_message));
    +        break;
    +      }
    +      case kQueryResultRelationMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::QueryResultRelationMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        for (int i = 0; i < proto.blocks_size(); ++i) {
    +          const block_id block = proto.blocks(i);
    +          storage_manager_->saveBlockOrBlob(block);
    +          if (storage_manager_->blockOrBlobIsLoaded(block)) {
    +            // NOTE(zuyu): eviction is required to avoid accesses to the query
    +            // result relation schema in CatalogDatabaseCache, for all query
    +            // optimizer execution generator unit tests and the single-process
    +            // Quickstep CLI.
    +            storage_manager_->evictBlockOrBlob(block);
    --- End diff --
    
    `saveBlockOrBlob` does not evict a block from memory.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #64: Introduced Shiftboss for the distribut...

Posted by zuyu <gi...@git.apache.org>.
Github user zuyu commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/64#discussion_r72498665
  
    --- Diff: query_execution/Shiftboss.hpp ---
    @@ -0,0 +1,241 @@
    +/**
    + *   Licensed under the Apache License, Version 2.0 (the "License");
    + *   you may not use this file except in compliance with the License.
    + *   You may obtain a copy of the License at
    + *
    + *       http://www.apache.org/licenses/LICENSE-2.0
    + *
    + *   Unless required by applicable law or agreed to in writing, software
    + *   distributed under the License is distributed on an "AS IS" BASIS,
    + *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + *   See the License for the specific language governing permissions and
    + *   limitations under the License.
    + **/
    +
    +#ifndef QUICKSTEP_QUERY_EXECUTION_SHIFTBOSS_HPP_
    +#define QUICKSTEP_QUERY_EXECUTION_SHIFTBOSS_HPP_
    +
    +#include <cstddef>
    +#include <memory>
    +#include <unordered_map>
    +
    +#include "catalog/CatalogDatabaseCache.hpp"
    +#include "catalog/CatalogTypedefs.hpp"
    +#include "query_execution/QueryContext.hpp"
    +#include "query_execution/QueryExecutionTypedefs.hpp"
    +#include "query_execution/WorkerDirectory.hpp"
    +#include "threading/Thread.hpp"
    +#include "utility/Macros.hpp"
    +
    +#include "glog/logging.h"
    +
    +#include "tmb/address.h"
    +#include "tmb/id_typedefs.h"
    +#include "tmb/message_bus.h"
    +
    +namespace quickstep {
    +
    +class StorageManager;
    +
    +namespace serialization {
    +class CatalogDatabase;
    +class QueryContext;
    +}  // namespace serialization
    +
    +/** \addtogroup QueryExecution
    + *  @{
    + */
    +
    +/**
    + * @brief The Shiftboss accepts workorder protos from shiftboss, and assigns
    + *        the workorders to workers.
    + **/
    +class Shiftboss : public Thread {
    + public:
    +  /**
    +   * @brief Constructor.
    +   *
    +   * @param bus A pointer to the TMB.
    +   * @param storage_manager The StorageManager to use.
    +   * @param workers A pointer to the WorkerDirectory.
    --- End diff --
    
    `ForemanSingleNode` does not describe which creates `WorkerDirectory`. And it is obviously the `QuicksteCli` that does.
    
    For the distributed case, the `Worker` executable does the similar task.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #64: Introduced Shiftboss for the distribut...

Posted by zuyu <gi...@git.apache.org>.
Github user zuyu commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/64#discussion_r72125717
  
    --- Diff: query_execution/Shiftboss.hpp ---
    @@ -0,0 +1,238 @@
    +/**
    + *   Licensed under the Apache License, Version 2.0 (the "License");
    + *   you may not use this file except in compliance with the License.
    + *   You may obtain a copy of the License at
    + *
    + *       http://www.apache.org/licenses/LICENSE-2.0
    + *
    + *   Unless required by applicable law or agreed to in writing, software
    + *   distributed under the License is distributed on an "AS IS" BASIS,
    + *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + *   See the License for the specific language governing permissions and
    + *   limitations under the License.
    + **/
    +
    +#ifndef QUICKSTEP_QUERY_EXECUTION_SHIFTBOSS_HPP_
    +#define QUICKSTEP_QUERY_EXECUTION_SHIFTBOSS_HPP_
    +
    +#include <cstddef>
    +#include <memory>
    +
    +#include "catalog/CatalogDatabaseCache.hpp"
    +#include "catalog/CatalogTypedefs.hpp"
    +#include "query_execution/QueryContext.hpp"
    +#include "query_execution/QueryExecutionTypedefs.hpp"
    +#include "query_execution/WorkerDirectory.hpp"
    +#include "threading/Thread.hpp"
    +#include "utility/Macros.hpp"
    +
    +#include "glog/logging.h"
    +
    +#include "tmb/address.h"
    +#include "tmb/id_typedefs.h"
    +#include "tmb/message_bus.h"
    +
    +namespace quickstep {
    +
    +class StorageManager;
    +
    +namespace serialization {
    +class CatalogDatabase;
    +class QueryContext;
    +}  // namespace serialization
    +
    +/** \addtogroup QueryExecution
    + *  @{
    + */
    +
    +/**
    + * @brief The Shiftboss accepts workorder protos from shiftboss, and assigns
    + *        the workorders to workers.
    + **/
    +class Shiftboss : public Thread {
    + public:
    +  /**
    +   * @brief Constructor.
    +   *
    +   * @param bus A pointer to the TMB.
    +   * @param storage_manager The StorageManager to use.
    +   * @param workers A pointer to the WorkerDirectory.
    +   * @param cpu_id The ID of the CPU to which the Shiftboss thread can be pinned.
    +   *
    +   * @note If cpu_id is not specified, Shiftboss thread can be possibly moved
    +   *       around on different CPUs by the OS.
    +  **/
    +  Shiftboss(tmb::MessageBus *bus,
    +            StorageManager *storage_manager,
    +            WorkerDirectory *workers,
    +            const int cpu_id = -1)
    +      : bus_(DCHECK_NOTNULL(bus)),
    +        storage_manager_(DCHECK_NOTNULL(storage_manager)),
    +        workers_(DCHECK_NOTNULL(workers)),
    +        cpu_id_(cpu_id),
    +        shiftboss_client_id_(tmb::kClientIdNone),
    +        foreman_client_id_(tmb::kClientIdNone),
    +        max_msgs_per_worker_(1),
    +        start_worker_index_(0u) {
    +    // Check to have at least one Worker.
    +    DCHECK_GT(workers->getNumWorkers(), 0u);
    +
    +    shiftboss_client_id_ = bus_->Connect();
    +    LOG(INFO) << "Shiftboss TMB client ID: " << shiftboss_client_id_;
    +    DCHECK_NE(shiftboss_client_id_, tmb::kClientIdNone);
    +
    +    // Messages between Foreman and Shiftboss.
    +    bus_->RegisterClientAsSender(shiftboss_client_id_, kShiftbossRegistrationMessage);
    --- End diff --
    
    That's a future work. For now, we use one TMB.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #64: Introduced Shiftboss for the distribut...

Posted by hbdeshmukh <gi...@git.apache.org>.
Github user hbdeshmukh commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/64#discussion_r72338677
  
    --- Diff: query_execution/Shiftboss.hpp ---
    @@ -0,0 +1,241 @@
    +/**
    + *   Licensed under the Apache License, Version 2.0 (the "License");
    + *   you may not use this file except in compliance with the License.
    + *   You may obtain a copy of the License at
    + *
    + *       http://www.apache.org/licenses/LICENSE-2.0
    + *
    + *   Unless required by applicable law or agreed to in writing, software
    + *   distributed under the License is distributed on an "AS IS" BASIS,
    + *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + *   See the License for the specific language governing permissions and
    + *   limitations under the License.
    + **/
    +
    +#ifndef QUICKSTEP_QUERY_EXECUTION_SHIFTBOSS_HPP_
    +#define QUICKSTEP_QUERY_EXECUTION_SHIFTBOSS_HPP_
    +
    +#include <cstddef>
    +#include <memory>
    +#include <unordered_map>
    +
    +#include "catalog/CatalogDatabaseCache.hpp"
    +#include "catalog/CatalogTypedefs.hpp"
    +#include "query_execution/QueryContext.hpp"
    +#include "query_execution/QueryExecutionTypedefs.hpp"
    +#include "query_execution/WorkerDirectory.hpp"
    +#include "threading/Thread.hpp"
    +#include "utility/Macros.hpp"
    +
    +#include "glog/logging.h"
    +
    +#include "tmb/address.h"
    +#include "tmb/id_typedefs.h"
    +#include "tmb/message_bus.h"
    +
    +namespace quickstep {
    +
    +class StorageManager;
    +
    +namespace serialization {
    +class CatalogDatabase;
    +class QueryContext;
    +}  // namespace serialization
    +
    +/** \addtogroup QueryExecution
    + *  @{
    + */
    +
    +/**
    + * @brief The Shiftboss accepts workorder protos from shiftboss, and assigns
    + *        the workorders to workers.
    + **/
    +class Shiftboss : public Thread {
    + public:
    +  /**
    +   * @brief Constructor.
    +   *
    +   * @param bus A pointer to the TMB.
    +   * @param storage_manager The StorageManager to use.
    +   * @param workers A pointer to the WorkerDirectory.
    --- End diff --
    
    Who creates the ``WorkerDirectory`` which is being passed to the ShiftBoss? I believe that ShiftBoss itself should create it, as it controls the workers on that node. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #64: Introduced Shiftboss for the distribut...

Posted by hbdeshmukh <gi...@git.apache.org>.
Github user hbdeshmukh commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/64#discussion_r72334473
  
    --- Diff: query_execution/Shiftboss.cpp ---
    @@ -0,0 +1,357 @@
    +/**
    + *   Licensed under the Apache License, Version 2.0 (the "License");
    + *   you may not use this file except in compliance with the License.
    + *   You may obtain a copy of the License at
    + *
    + *       http://www.apache.org/licenses/LICENSE-2.0
    + *
    + *   Unless required by applicable law or agreed to in writing, software
    + *   distributed under the License is distributed on an "AS IS" BASIS,
    + *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + *   See the License for the specific language governing permissions and
    + *   limitations under the License.
    + **/
    +
    +#include "query_execution/Shiftboss.hpp"
    +
    +#include <cstddef>
    +#include <cstdlib>
    +#include <memory>
    +#include <string>
    +#include <unordered_map>
    +#include <utility>
    +#include <vector>
    +
    +#include "catalog/CatalogTypedefs.hpp"
    +#include "query_execution/QueryContext.hpp"
    +#include "query_execution/QueryExecutionMessages.pb.h"
    +#include "query_execution/QueryExecutionTypedefs.hpp"
    +#include "query_execution/QueryExecutionUtil.hpp"
    +#include "query_execution/WorkerMessage.hpp"
    +#include "relational_operators/RebuildWorkOrder.hpp"
    +#include "relational_operators/WorkOrderFactory.hpp"
    +#include "storage/InsertDestination.hpp"
    +#include "storage/StorageBlock.hpp"
    +#include "storage/StorageBlockInfo.hpp"
    +#include "storage/StorageManager.hpp"
    +#include "threading/ThreadUtil.hpp"
    +
    +#include "glog/logging.h"
    +
    +#include "tmb/address.h"
    +#include "tmb/id_typedefs.h"
    +#include "tmb/message_bus.h"
    +#include "tmb/message_style.h"
    +#include "tmb/tagged_message.h"
    +
    +using std::free;
    +using std::malloc;
    +using std::move;
    +using std::size_t;
    +using std::string;
    +using std::unique_ptr;
    +using std::vector;
    +
    +using tmb::TaggedMessage;
    +
    +namespace quickstep {
    +
    +class WorkOrder;
    +
    +void Shiftboss::run() {
    +  if (cpu_id_ >= 0) {
    +    // We can pin the shiftboss thread to a CPU if specified.
    +    ThreadUtil::BindToCPU(cpu_id_);
    +  }
    +
    +  for (;;) {
    +    // Receive() is a blocking call, causing this thread to sleep until next
    +    // message is received.
    +    AnnotatedMessage annotated_message(bus_->Receive(shiftboss_client_id_, 0, true));
    +    LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +              << "') received the typed '" << annotated_message.tagged_message.message_type()
    +              << "' message from client " << annotated_message.sender;
    +    switch (annotated_message.tagged_message.message_type()) {
    +      case kShiftbossRegistrationResponseMessage: {
    +        foreman_client_id_ = annotated_message.sender;
    +        break;
    +      }
    +      case kShiftbossInitiateMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::ShiftbossInitiateMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processShiftbossInitiateMessage(proto.query_id(), proto.catalog_database_cache(), proto.query_context());
    +        break;
    +      }
    +      case kWorkOrderMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::WorkOrderMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        const std::size_t query_id = proto.query_id();
    +        DCHECK_EQ(1u, query_contexts_.count(query_id));
    +
    +        WorkOrder *work_order = WorkOrderFactory::ReconstructFromProto(proto.work_order(),
    +                                                                       &database_cache_,
    +                                                                       query_contexts_[query_id].get(),
    +                                                                       storage_manager_,
    +                                                                       shiftboss_client_id_,
    +                                                                       bus_);
    +
    +        unique_ptr<WorkerMessage> worker_message(
    +            WorkerMessage::WorkOrderMessage(work_order, proto.operator_index()));
    +
    +        TaggedMessage worker_tagged_message(worker_message.get(),
    +                                            sizeof(*worker_message),
    +                                            kWorkOrderMessage);
    +
    +        const size_t worker_index = getSchedulableWorker();
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded WorkOrderMessage (typed '" << kWorkOrderMessage
    +                  << "') from Foreman to worker " << worker_index;
    +
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           workers_->getClientID(worker_index),
    +                                           move(worker_tagged_message));
    +        break;
    +      }
    +      case kInitiateRebuildMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::InitiateRebuildMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processInitiateRebuildMessage(proto.query_id(),
    +                                      proto.operator_index(),
    +                                      proto.insert_destination_index(),
    +                                      proto.relation_id());
    +        break;
    +      }
    +      case kWorkOrderCompleteMessage:  // Fall through.
    +      case kRebuildWorkOrderCompleteMessage:
    --- End diff --
    
    Do you assume the following?
    1. All the rebuild work orders are generated by the Foreman.
    2. For every rebuild work order completion, a message is sent from Worker to Shiftboss, which is relayed to Foreman.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #64: Introduced Shiftboss for the distribut...

Posted by hbdeshmukh <gi...@git.apache.org>.
Github user hbdeshmukh commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/64#discussion_r72338372
  
    --- Diff: query_execution/Shiftboss.cpp ---
    @@ -0,0 +1,357 @@
    +/**
    + *   Licensed under the Apache License, Version 2.0 (the "License");
    + *   you may not use this file except in compliance with the License.
    + *   You may obtain a copy of the License at
    + *
    + *       http://www.apache.org/licenses/LICENSE-2.0
    + *
    + *   Unless required by applicable law or agreed to in writing, software
    + *   distributed under the License is distributed on an "AS IS" BASIS,
    + *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + *   See the License for the specific language governing permissions and
    + *   limitations under the License.
    + **/
    +
    +#include "query_execution/Shiftboss.hpp"
    +
    +#include <cstddef>
    +#include <cstdlib>
    +#include <memory>
    +#include <string>
    +#include <unordered_map>
    +#include <utility>
    +#include <vector>
    +
    +#include "catalog/CatalogTypedefs.hpp"
    +#include "query_execution/QueryContext.hpp"
    +#include "query_execution/QueryExecutionMessages.pb.h"
    +#include "query_execution/QueryExecutionTypedefs.hpp"
    +#include "query_execution/QueryExecutionUtil.hpp"
    +#include "query_execution/WorkerMessage.hpp"
    +#include "relational_operators/RebuildWorkOrder.hpp"
    +#include "relational_operators/WorkOrderFactory.hpp"
    +#include "storage/InsertDestination.hpp"
    +#include "storage/StorageBlock.hpp"
    +#include "storage/StorageBlockInfo.hpp"
    +#include "storage/StorageManager.hpp"
    +#include "threading/ThreadUtil.hpp"
    +
    +#include "glog/logging.h"
    +
    +#include "tmb/address.h"
    +#include "tmb/id_typedefs.h"
    +#include "tmb/message_bus.h"
    +#include "tmb/message_style.h"
    +#include "tmb/tagged_message.h"
    +
    +using std::free;
    +using std::malloc;
    +using std::move;
    +using std::size_t;
    +using std::string;
    +using std::unique_ptr;
    +using std::vector;
    +
    +using tmb::TaggedMessage;
    +
    +namespace quickstep {
    +
    +class WorkOrder;
    +
    +void Shiftboss::run() {
    +  if (cpu_id_ >= 0) {
    +    // We can pin the shiftboss thread to a CPU if specified.
    +    ThreadUtil::BindToCPU(cpu_id_);
    +  }
    +
    +  for (;;) {
    +    // Receive() is a blocking call, causing this thread to sleep until next
    +    // message is received.
    +    AnnotatedMessage annotated_message(bus_->Receive(shiftboss_client_id_, 0, true));
    +    LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +              << "') received the typed '" << annotated_message.tagged_message.message_type()
    +              << "' message from client " << annotated_message.sender;
    +    switch (annotated_message.tagged_message.message_type()) {
    +      case kShiftbossRegistrationResponseMessage: {
    +        foreman_client_id_ = annotated_message.sender;
    +        break;
    +      }
    +      case kShiftbossInitiateMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::ShiftbossInitiateMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processShiftbossInitiateMessage(proto.query_id(), proto.catalog_database_cache(), proto.query_context());
    +        break;
    +      }
    +      case kWorkOrderMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::WorkOrderMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        const std::size_t query_id = proto.query_id();
    +        DCHECK_EQ(1u, query_contexts_.count(query_id));
    +
    +        WorkOrder *work_order = WorkOrderFactory::ReconstructFromProto(proto.work_order(),
    +                                                                       &database_cache_,
    +                                                                       query_contexts_[query_id].get(),
    +                                                                       storage_manager_,
    +                                                                       shiftboss_client_id_,
    +                                                                       bus_);
    +
    +        unique_ptr<WorkerMessage> worker_message(
    +            WorkerMessage::WorkOrderMessage(work_order, proto.operator_index()));
    +
    +        TaggedMessage worker_tagged_message(worker_message.get(),
    +                                            sizeof(*worker_message),
    +                                            kWorkOrderMessage);
    +
    +        const size_t worker_index = getSchedulableWorker();
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded WorkOrderMessage (typed '" << kWorkOrderMessage
    +                  << "') from Foreman to worker " << worker_index;
    +
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           workers_->getClientID(worker_index),
    +                                           move(worker_tagged_message));
    +        break;
    +      }
    +      case kInitiateRebuildMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::InitiateRebuildMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processInitiateRebuildMessage(proto.query_id(),
    +                                      proto.operator_index(),
    +                                      proto.insert_destination_index(),
    +                                      proto.relation_id());
    +        break;
    +      }
    +      case kWorkOrderCompleteMessage:  // Fall through.
    +      case kRebuildWorkOrderCompleteMessage:
    +      case kDataPipelineMessage:
    +      case kWorkOrdersAvailableMessage:
    +      case kWorkOrderFeedbackMessage: {
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded typed '" << annotated_message.tagged_message.message_type()
    +                  << "' message from worker (client " << annotated_message.sender << ") to Foreman";
    +
    +        DCHECK_NE(foreman_client_id_, tmb::kClientIdNone);
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           foreman_client_id_,
    +                                           move(annotated_message.tagged_message));
    +        break;
    +      }
    +      case kQueryResultRelationMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::QueryResultRelationMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        for (int i = 0; i < proto.blocks_size(); ++i) {
    +          const block_id block = proto.blocks(i);
    +          storage_manager_->saveBlockOrBlob(block);
    +          if (storage_manager_->blockOrBlobIsLoaded(block)) {
    +            // NOTE(zuyu): eviction is required to avoid accesses to the query
    +            // result relation schema in CatalogDatabaseCache, for all query
    +            // optimizer execution generator unit tests and the single-process
    +            // Quickstep CLI.
    +            storage_manager_->evictBlockOrBlob(block);
    +          }
    +        }
    +
    +        serialization::QueryResultRelationResponseMessage ack_proto;
    +        ack_proto.set_relation_id(proto.relation_id());
    +
    +        const size_t ack_proto_length = ack_proto.ByteSize();
    +        char *ack_proto_bytes = static_cast<char*>(malloc(ack_proto_length));
    +        CHECK(ack_proto.SerializeToArray(ack_proto_bytes, ack_proto_length));
    +
    +        TaggedMessage ack_message(static_cast<const void*>(ack_proto_bytes),
    +                                  ack_proto_length,
    +                                  kQueryResultRelationResponseMessage);
    +        free(ack_proto_bytes);
    +
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') sent QueryResultRelationResponseMessage (typed '" << kQueryResultRelationResponseMessage
    +                  << ") to Foreman";
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           foreman_client_id_,
    +                                           move(ack_message));
    +        break;
    +      }
    +      case kPoisonMessage: {
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded PoisonMessage (typed '" << kPoisonMessage
    +                  << "') from Foreman to all workers";
    +
    +        tmb::MessageStyle broadcast_style;
    +        broadcast_style.Broadcast(true);
    +
    +        tmb::MessageBus::SendStatus send_status =
    +            bus_->Send(shiftboss_client_id_,
    +                       worker_addresses_,
    +                       broadcast_style,
    +                       move(annotated_message.tagged_message));
    +        DCHECK(send_status == tmb::MessageBus::SendStatus::kOK);
    +        return;
    +      }
    +      default: {
    +        LOG(FATAL) << "Unknown TMB message type";
    +      }
    +    }
    +  }
    +}
    +
    +size_t Shiftboss::getSchedulableWorker() {
    +  const size_t num_workers = workers_->getNumWorkers();
    +
    +  size_t curr_worker = start_worker_index_;
    +  for (;;) {
    +    if (workers_->getNumQueuedWorkOrders(curr_worker) < max_msgs_per_worker_) {
    +      start_worker_index_ = (curr_worker + 1) % num_workers;
    +      // TODO(zuyu): workers_->incrementNumQueuedWorkOrders(curr_worker);
    +      // But we need a WorkOrder queue first.
    +      return curr_worker;
    +    }
    +
    +    curr_worker = (curr_worker + 1) % num_workers;
    +  }
    +}
    +
    +void Shiftboss::registerWithForeman() {
    +  LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +            << "') sent ShiftbossRegistrationMessage (typed '" << kShiftbossRegistrationMessage
    +            << "') to all";
    +
    +  tmb::Address all_addresses;
    +  all_addresses.All(true);
    +
    +  tmb::MessageStyle style;
    +
    +  serialization::ShiftbossRegistrationMessage proto;
    +  proto.set_work_order_capacity(getWorkOrderCapacity());
    +
    +  const size_t proto_length = proto.ByteSize();
    +  char *proto_bytes = static_cast<char*>(malloc(proto_length));
    +  CHECK(proto.SerializeToArray(proto_bytes, proto_length));
    +
    +  TaggedMessage message(static_cast<const void*>(proto_bytes),
    +                        proto_length,
    +                        kShiftbossRegistrationMessage);
    +  free(proto_bytes);
    +
    +  tmb::MessageBus::SendStatus send_status =
    +      bus_->Send(shiftboss_client_id_, all_addresses, style, move(message));
    +  DCHECK(send_status == tmb::MessageBus::SendStatus::kOK);
    +}
    +
    +void Shiftboss::processShiftbossInitiateMessage(
    +    const std::size_t query_id,
    +    const serialization::CatalogDatabase &catalog_database_cache_proto,
    +    const serialization::QueryContext &query_context_proto) {
    +  database_cache_.update(catalog_database_cache_proto);
    +
    +  unique_ptr<QueryContext> query_context(
    +      new QueryContext(query_context_proto,
    +                       database_cache_,
    +                       storage_manager_,
    +                       shiftboss_client_id_,
    +                       bus_));
    +  query_contexts_.emplace(query_id, move(query_context));
    +
    +  LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +            << "') sent ShiftbossInitiateResponseMessage (typed '"
    +            << kShiftbossInitiateResponseMessage << "') to Foreman";
    +
    +  serialization::ShiftbossInitiateResponseMessage proto;
    +  proto.set_query_id(query_id);
    +
    +  const size_t proto_length = proto.ByteSize();
    +  char *proto_bytes = static_cast<char*>(malloc(proto_length));
    +  CHECK(proto.SerializeToArray(proto_bytes, proto_length));
    +
    +  TaggedMessage ack_message(static_cast<const void*>(proto_bytes),
    +                            proto_length,
    +                            kShiftbossInitiateResponseMessage);
    +  free(proto_bytes);
    +
    +  QueryExecutionUtil::SendTMBMessage(bus_,
    +                                     shiftboss_client_id_,
    +                                     foreman_client_id_,
    +                                     move(ack_message));
    +}
    +
    +void Shiftboss::processInitiateRebuildMessage(const std::size_t query_id,
    +                                              const std::size_t op_index,
    +                                              const QueryContext::insert_destination_id dest_index,
    +                                              const relation_id rel_id) {
    +  DCHECK_NE(foreman_client_id_, tmb::kClientIdNone);
    +
    +  DCHECK_EQ(1u, query_contexts_.count(query_id));
    +  InsertDestination *insert_destination = query_contexts_[query_id]->getInsertDestination(dest_index);
    +  DCHECK(insert_destination != nullptr);
    +
    +  vector<MutableBlockReference> partially_filled_block_refs;
    +  insert_destination->getPartiallyFilledBlocks(&partially_filled_block_refs);
    +
    +  LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +            << "') sent InitiateRebuildResponseMessage (typed '" << kInitiateRebuildResponseMessage
    +            << "') to Foreman";
    +
    +  serialization::InitiateRebuildResponseMessage proto;
    +  proto.set_query_id(query_id);
    +  proto.set_operator_index(op_index);
    +  proto.set_num_rebuild_work_orders(partially_filled_block_refs.size());
    +
    +  const size_t proto_length = proto.ByteSize();
    +  char *proto_bytes = static_cast<char*>(malloc(proto_length));
    +  CHECK(proto.SerializeToArray(proto_bytes, proto_length));
    +
    +  TaggedMessage ack_message(static_cast<const void*>(proto_bytes),
    +                            proto_length,
    +                            kInitiateRebuildResponseMessage);
    +  free(proto_bytes);
    +
    +  QueryExecutionUtil::SendTMBMessage(bus_,
    +                                     shiftboss_client_id_,
    +                                     foreman_client_id_,
    +                                     move(ack_message));
    +
    +  for (size_t i = 0; i < partially_filled_block_refs.size(); ++i) {
    +    // NOTE(zuyu): Worker releases the memory after the execution of
    +    // RebuildWorkOrder on the Worker.
    +    WorkOrder *rebuild_work_order =
    +        new RebuildWorkOrder(query_id,
    +                             move(partially_filled_block_refs[0]),
    --- End diff --
    
    Do you mean ``partially_filled_block_refs[i]``?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #64: Introduced Shiftboss for the distribut...

Posted by hbdeshmukh <gi...@git.apache.org>.
Github user hbdeshmukh commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/64#discussion_r72523040
  
    --- Diff: query_execution/Shiftboss.cpp ---
    @@ -0,0 +1,357 @@
    +/**
    + *   Licensed under the Apache License, Version 2.0 (the "License");
    + *   you may not use this file except in compliance with the License.
    + *   You may obtain a copy of the License at
    + *
    + *       http://www.apache.org/licenses/LICENSE-2.0
    + *
    + *   Unless required by applicable law or agreed to in writing, software
    + *   distributed under the License is distributed on an "AS IS" BASIS,
    + *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + *   See the License for the specific language governing permissions and
    + *   limitations under the License.
    + **/
    +
    +#include "query_execution/Shiftboss.hpp"
    +
    +#include <cstddef>
    +#include <cstdlib>
    +#include <memory>
    +#include <string>
    +#include <unordered_map>
    +#include <utility>
    +#include <vector>
    +
    +#include "catalog/CatalogTypedefs.hpp"
    +#include "query_execution/QueryContext.hpp"
    +#include "query_execution/QueryExecutionMessages.pb.h"
    +#include "query_execution/QueryExecutionTypedefs.hpp"
    +#include "query_execution/QueryExecutionUtil.hpp"
    +#include "query_execution/WorkerMessage.hpp"
    +#include "relational_operators/RebuildWorkOrder.hpp"
    +#include "relational_operators/WorkOrderFactory.hpp"
    +#include "storage/InsertDestination.hpp"
    +#include "storage/StorageBlock.hpp"
    +#include "storage/StorageBlockInfo.hpp"
    +#include "storage/StorageManager.hpp"
    +#include "threading/ThreadUtil.hpp"
    +
    +#include "glog/logging.h"
    +
    +#include "tmb/address.h"
    +#include "tmb/id_typedefs.h"
    +#include "tmb/message_bus.h"
    +#include "tmb/message_style.h"
    +#include "tmb/tagged_message.h"
    +
    +using std::free;
    +using std::malloc;
    +using std::move;
    +using std::size_t;
    +using std::string;
    +using std::unique_ptr;
    +using std::vector;
    +
    +using tmb::TaggedMessage;
    +
    +namespace quickstep {
    +
    +class WorkOrder;
    +
    +void Shiftboss::run() {
    +  if (cpu_id_ >= 0) {
    +    // We can pin the shiftboss thread to a CPU if specified.
    +    ThreadUtil::BindToCPU(cpu_id_);
    +  }
    +
    +  for (;;) {
    +    // Receive() is a blocking call, causing this thread to sleep until next
    +    // message is received.
    +    AnnotatedMessage annotated_message(bus_->Receive(shiftboss_client_id_, 0, true));
    +    LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +              << "') received the typed '" << annotated_message.tagged_message.message_type()
    +              << "' message from client " << annotated_message.sender;
    +    switch (annotated_message.tagged_message.message_type()) {
    +      case kShiftbossRegistrationResponseMessage: {
    +        foreman_client_id_ = annotated_message.sender;
    +        break;
    +      }
    +      case kShiftbossInitiateMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::ShiftbossInitiateMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processShiftbossInitiateMessage(proto.query_id(), proto.catalog_database_cache(), proto.query_context());
    +        break;
    +      }
    +      case kWorkOrderMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::WorkOrderMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        const std::size_t query_id = proto.query_id();
    +        DCHECK_EQ(1u, query_contexts_.count(query_id));
    +
    +        WorkOrder *work_order = WorkOrderFactory::ReconstructFromProto(proto.work_order(),
    +                                                                       &database_cache_,
    +                                                                       query_contexts_[query_id].get(),
    +                                                                       storage_manager_,
    +                                                                       shiftboss_client_id_,
    +                                                                       bus_);
    +
    +        unique_ptr<WorkerMessage> worker_message(
    +            WorkerMessage::WorkOrderMessage(work_order, proto.operator_index()));
    +
    +        TaggedMessage worker_tagged_message(worker_message.get(),
    +                                            sizeof(*worker_message),
    +                                            kWorkOrderMessage);
    +
    +        const size_t worker_index = getSchedulableWorker();
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded WorkOrderMessage (typed '" << kWorkOrderMessage
    +                  << "') from Foreman to worker " << worker_index;
    +
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           workers_->getClientID(worker_index),
    +                                           move(worker_tagged_message));
    +        break;
    +      }
    +      case kInitiateRebuildMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::InitiateRebuildMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processInitiateRebuildMessage(proto.query_id(),
    +                                      proto.operator_index(),
    +                                      proto.insert_destination_index(),
    +                                      proto.relation_id());
    +        break;
    +      }
    +      case kWorkOrderCompleteMessage:  // Fall through.
    +      case kRebuildWorkOrderCompleteMessage:
    +      case kDataPipelineMessage:
    +      case kWorkOrdersAvailableMessage:
    +      case kWorkOrderFeedbackMessage: {
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded typed '" << annotated_message.tagged_message.message_type()
    +                  << "' message from worker (client " << annotated_message.sender << ") to Foreman";
    +
    +        DCHECK_NE(foreman_client_id_, tmb::kClientIdNone);
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           foreman_client_id_,
    +                                           move(annotated_message.tagged_message));
    +        break;
    +      }
    +      case kQueryResultRelationMessage: {
    --- End diff --
    
    Are you planning to add a TODO in this PR then? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #64: Introduced Shiftboss for the distribut...

Posted by zuyu <gi...@git.apache.org>.
Github user zuyu commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/64#discussion_r72339134
  
    --- Diff: query_execution/Shiftboss.cpp ---
    @@ -0,0 +1,357 @@
    +/**
    + *   Licensed under the Apache License, Version 2.0 (the "License");
    + *   you may not use this file except in compliance with the License.
    + *   You may obtain a copy of the License at
    + *
    + *       http://www.apache.org/licenses/LICENSE-2.0
    + *
    + *   Unless required by applicable law or agreed to in writing, software
    + *   distributed under the License is distributed on an "AS IS" BASIS,
    + *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + *   See the License for the specific language governing permissions and
    + *   limitations under the License.
    + **/
    +
    +#include "query_execution/Shiftboss.hpp"
    +
    +#include <cstddef>
    +#include <cstdlib>
    +#include <memory>
    +#include <string>
    +#include <unordered_map>
    +#include <utility>
    +#include <vector>
    +
    +#include "catalog/CatalogTypedefs.hpp"
    +#include "query_execution/QueryContext.hpp"
    +#include "query_execution/QueryExecutionMessages.pb.h"
    +#include "query_execution/QueryExecutionTypedefs.hpp"
    +#include "query_execution/QueryExecutionUtil.hpp"
    +#include "query_execution/WorkerMessage.hpp"
    +#include "relational_operators/RebuildWorkOrder.hpp"
    +#include "relational_operators/WorkOrderFactory.hpp"
    +#include "storage/InsertDestination.hpp"
    +#include "storage/StorageBlock.hpp"
    +#include "storage/StorageBlockInfo.hpp"
    +#include "storage/StorageManager.hpp"
    +#include "threading/ThreadUtil.hpp"
    +
    +#include "glog/logging.h"
    +
    +#include "tmb/address.h"
    +#include "tmb/id_typedefs.h"
    +#include "tmb/message_bus.h"
    +#include "tmb/message_style.h"
    +#include "tmb/tagged_message.h"
    +
    +using std::free;
    +using std::malloc;
    +using std::move;
    +using std::size_t;
    +using std::string;
    +using std::unique_ptr;
    +using std::vector;
    +
    +using tmb::TaggedMessage;
    +
    +namespace quickstep {
    +
    +class WorkOrder;
    +
    +void Shiftboss::run() {
    +  if (cpu_id_ >= 0) {
    +    // We can pin the shiftboss thread to a CPU if specified.
    +    ThreadUtil::BindToCPU(cpu_id_);
    +  }
    +
    +  for (;;) {
    +    // Receive() is a blocking call, causing this thread to sleep until next
    +    // message is received.
    +    AnnotatedMessage annotated_message(bus_->Receive(shiftboss_client_id_, 0, true));
    +    LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +              << "') received the typed '" << annotated_message.tagged_message.message_type()
    +              << "' message from client " << annotated_message.sender;
    +    switch (annotated_message.tagged_message.message_type()) {
    +      case kShiftbossRegistrationResponseMessage: {
    +        foreman_client_id_ = annotated_message.sender;
    +        break;
    +      }
    +      case kShiftbossInitiateMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::ShiftbossInitiateMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processShiftbossInitiateMessage(proto.query_id(), proto.catalog_database_cache(), proto.query_context());
    +        break;
    +      }
    +      case kWorkOrderMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::WorkOrderMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        const std::size_t query_id = proto.query_id();
    +        DCHECK_EQ(1u, query_contexts_.count(query_id));
    +
    +        WorkOrder *work_order = WorkOrderFactory::ReconstructFromProto(proto.work_order(),
    +                                                                       &database_cache_,
    +                                                                       query_contexts_[query_id].get(),
    +                                                                       storage_manager_,
    +                                                                       shiftboss_client_id_,
    +                                                                       bus_);
    +
    +        unique_ptr<WorkerMessage> worker_message(
    +            WorkerMessage::WorkOrderMessage(work_order, proto.operator_index()));
    +
    +        TaggedMessage worker_tagged_message(worker_message.get(),
    +                                            sizeof(*worker_message),
    +                                            kWorkOrderMessage);
    +
    +        const size_t worker_index = getSchedulableWorker();
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded WorkOrderMessage (typed '" << kWorkOrderMessage
    +                  << "') from Foreman to worker " << worker_index;
    +
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           workers_->getClientID(worker_index),
    +                                           move(worker_tagged_message));
    +        break;
    +      }
    +      case kInitiateRebuildMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::InitiateRebuildMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processInitiateRebuildMessage(proto.query_id(),
    +                                      proto.operator_index(),
    +                                      proto.insert_destination_index(),
    +                                      proto.relation_id());
    +        break;
    +      }
    +      case kWorkOrderCompleteMessage:  // Fall through.
    +      case kRebuildWorkOrderCompleteMessage:
    +      case kDataPipelineMessage:
    +      case kWorkOrdersAvailableMessage:
    +      case kWorkOrderFeedbackMessage: {
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded typed '" << annotated_message.tagged_message.message_type()
    +                  << "' message from worker (client " << annotated_message.sender << ") to Foreman";
    +
    +        DCHECK_NE(foreman_client_id_, tmb::kClientIdNone);
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           foreman_client_id_,
    +                                           move(annotated_message.tagged_message));
    +        break;
    +      }
    +      case kQueryResultRelationMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::QueryResultRelationMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        for (int i = 0; i < proto.blocks_size(); ++i) {
    +          const block_id block = proto.blocks(i);
    +          storage_manager_->saveBlockOrBlob(block);
    +          if (storage_manager_->blockOrBlobIsLoaded(block)) {
    +            // NOTE(zuyu): eviction is required to avoid accesses to the query
    +            // result relation schema in CatalogDatabaseCache, for all query
    +            // optimizer execution generator unit tests and the single-process
    +            // Quickstep CLI.
    +            storage_manager_->evictBlockOrBlob(block);
    +          }
    +        }
    +
    +        serialization::QueryResultRelationResponseMessage ack_proto;
    +        ack_proto.set_relation_id(proto.relation_id());
    +
    +        const size_t ack_proto_length = ack_proto.ByteSize();
    +        char *ack_proto_bytes = static_cast<char*>(malloc(ack_proto_length));
    +        CHECK(ack_proto.SerializeToArray(ack_proto_bytes, ack_proto_length));
    +
    +        TaggedMessage ack_message(static_cast<const void*>(ack_proto_bytes),
    +                                  ack_proto_length,
    +                                  kQueryResultRelationResponseMessage);
    +        free(ack_proto_bytes);
    +
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') sent QueryResultRelationResponseMessage (typed '" << kQueryResultRelationResponseMessage
    +                  << ") to Foreman";
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           foreman_client_id_,
    +                                           move(ack_message));
    +        break;
    +      }
    +      case kPoisonMessage: {
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded PoisonMessage (typed '" << kPoisonMessage
    +                  << "') from Foreman to all workers";
    +
    +        tmb::MessageStyle broadcast_style;
    +        broadcast_style.Broadcast(true);
    +
    +        tmb::MessageBus::SendStatus send_status =
    +            bus_->Send(shiftboss_client_id_,
    +                       worker_addresses_,
    +                       broadcast_style,
    +                       move(annotated_message.tagged_message));
    +        DCHECK(send_status == tmb::MessageBus::SendStatus::kOK);
    +        return;
    +      }
    +      default: {
    +        LOG(FATAL) << "Unknown TMB message type";
    +      }
    +    }
    +  }
    +}
    +
    +size_t Shiftboss::getSchedulableWorker() {
    +  const size_t num_workers = workers_->getNumWorkers();
    +
    +  size_t curr_worker = start_worker_index_;
    +  for (;;) {
    +    if (workers_->getNumQueuedWorkOrders(curr_worker) < max_msgs_per_worker_) {
    +      start_worker_index_ = (curr_worker + 1) % num_workers;
    +      // TODO(zuyu): workers_->incrementNumQueuedWorkOrders(curr_worker);
    +      // But we need a WorkOrder queue first.
    +      return curr_worker;
    +    }
    +
    +    curr_worker = (curr_worker + 1) % num_workers;
    +  }
    +}
    +
    +void Shiftboss::registerWithForeman() {
    +  LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +            << "') sent ShiftbossRegistrationMessage (typed '" << kShiftbossRegistrationMessage
    +            << "') to all";
    --- End diff --
    
    No, but this is a way to allow `Shiftboss` to do a service discovery of `Foreman`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #64: Introduced Shiftboss for the distribut...

Posted by zuyu <gi...@git.apache.org>.
Github user zuyu commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/64#discussion_r72500396
  
    --- Diff: query_execution/QueryExecutionTypedefs.hpp ---
    @@ -73,9 +73,17 @@ enum QueryExecutionMessageType : message_type_id {
       kPoisonMessage,  // From the main thread to Foreman and Workers.
     
     #ifdef QUICKSTEP_DISTRIBUTED
    +  kShiftbossRegistrationMessage,  // From Shiftboss to Foreman.
    +  kShiftbossRegistrationResponseMessage,  // From Foreman to Shiftboss.
    +  kQueryInitiateMessage,  // From Foreman to Shiftboss.
    +  kQueryInitiateResponseMessage,  // From Shiftboss to Foreman.
    +
       kInitiateRebuildMessage,  // From Foreman to Shiftboss.
       kInitiateRebuildResponseMessage,  // From Shiftboss to Foreman.
     
    +  kQueryResultRelationMessage,  // From Foreman to Shiftboss.
    +  kQueryResultRelationResponseMessage,  // From Shiftboss to Foreman.
    --- End diff --
    
    We will add a new unit test for the distributed query execution engine.
    
    And I have tested it end-to-end in [a old repo](https://github.com/pivotalsoftware/quickstep/pull/117), w/o concurrent query support.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #64: Introduced Shiftboss for the distribut...

Posted by zuyu <gi...@git.apache.org>.
Github user zuyu commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/64#discussion_r72350508
  
    --- Diff: query_execution/Shiftboss.cpp ---
    @@ -0,0 +1,357 @@
    +/**
    + *   Licensed under the Apache License, Version 2.0 (the "License");
    + *   you may not use this file except in compliance with the License.
    + *   You may obtain a copy of the License at
    + *
    + *       http://www.apache.org/licenses/LICENSE-2.0
    + *
    + *   Unless required by applicable law or agreed to in writing, software
    + *   distributed under the License is distributed on an "AS IS" BASIS,
    + *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + *   See the License for the specific language governing permissions and
    + *   limitations under the License.
    + **/
    +
    +#include "query_execution/Shiftboss.hpp"
    +
    +#include <cstddef>
    +#include <cstdlib>
    +#include <memory>
    +#include <string>
    +#include <unordered_map>
    +#include <utility>
    +#include <vector>
    +
    +#include "catalog/CatalogTypedefs.hpp"
    +#include "query_execution/QueryContext.hpp"
    +#include "query_execution/QueryExecutionMessages.pb.h"
    +#include "query_execution/QueryExecutionTypedefs.hpp"
    +#include "query_execution/QueryExecutionUtil.hpp"
    +#include "query_execution/WorkerMessage.hpp"
    +#include "relational_operators/RebuildWorkOrder.hpp"
    +#include "relational_operators/WorkOrderFactory.hpp"
    +#include "storage/InsertDestination.hpp"
    +#include "storage/StorageBlock.hpp"
    +#include "storage/StorageBlockInfo.hpp"
    +#include "storage/StorageManager.hpp"
    +#include "threading/ThreadUtil.hpp"
    +
    +#include "glog/logging.h"
    +
    +#include "tmb/address.h"
    +#include "tmb/id_typedefs.h"
    +#include "tmb/message_bus.h"
    +#include "tmb/message_style.h"
    +#include "tmb/tagged_message.h"
    +
    +using std::free;
    +using std::malloc;
    +using std::move;
    +using std::size_t;
    +using std::string;
    +using std::unique_ptr;
    +using std::vector;
    +
    +using tmb::TaggedMessage;
    +
    +namespace quickstep {
    +
    +class WorkOrder;
    +
    +void Shiftboss::run() {
    +  if (cpu_id_ >= 0) {
    +    // We can pin the shiftboss thread to a CPU if specified.
    +    ThreadUtil::BindToCPU(cpu_id_);
    +  }
    +
    +  for (;;) {
    +    // Receive() is a blocking call, causing this thread to sleep until next
    +    // message is received.
    +    AnnotatedMessage annotated_message(bus_->Receive(shiftboss_client_id_, 0, true));
    +    LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +              << "') received the typed '" << annotated_message.tagged_message.message_type()
    +              << "' message from client " << annotated_message.sender;
    +    switch (annotated_message.tagged_message.message_type()) {
    +      case kShiftbossRegistrationResponseMessage: {
    +        foreman_client_id_ = annotated_message.sender;
    +        break;
    +      }
    +      case kShiftbossInitiateMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::ShiftbossInitiateMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processShiftbossInitiateMessage(proto.query_id(), proto.catalog_database_cache(), proto.query_context());
    +        break;
    +      }
    +      case kWorkOrderMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::WorkOrderMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        const std::size_t query_id = proto.query_id();
    +        DCHECK_EQ(1u, query_contexts_.count(query_id));
    +
    +        WorkOrder *work_order = WorkOrderFactory::ReconstructFromProto(proto.work_order(),
    +                                                                       &database_cache_,
    +                                                                       query_contexts_[query_id].get(),
    +                                                                       storage_manager_,
    +                                                                       shiftboss_client_id_,
    +                                                                       bus_);
    +
    +        unique_ptr<WorkerMessage> worker_message(
    +            WorkerMessage::WorkOrderMessage(work_order, proto.operator_index()));
    +
    +        TaggedMessage worker_tagged_message(worker_message.get(),
    +                                            sizeof(*worker_message),
    +                                            kWorkOrderMessage);
    +
    +        const size_t worker_index = getSchedulableWorker();
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded WorkOrderMessage (typed '" << kWorkOrderMessage
    +                  << "') from Foreman to worker " << worker_index;
    +
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           workers_->getClientID(worker_index),
    +                                           move(worker_tagged_message));
    +        break;
    +      }
    +      case kInitiateRebuildMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::InitiateRebuildMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processInitiateRebuildMessage(proto.query_id(),
    +                                      proto.operator_index(),
    +                                      proto.insert_destination_index(),
    +                                      proto.relation_id());
    +        break;
    +      }
    +      case kWorkOrderCompleteMessage:  // Fall through.
    +      case kRebuildWorkOrderCompleteMessage:
    +      case kDataPipelineMessage:
    +      case kWorkOrdersAvailableMessage:
    +      case kWorkOrderFeedbackMessage: {
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded typed '" << annotated_message.tagged_message.message_type()
    +                  << "' message from worker (client " << annotated_message.sender << ") to Foreman";
    +
    +        DCHECK_NE(foreman_client_id_, tmb::kClientIdNone);
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           foreman_client_id_,
    +                                           move(annotated_message.tagged_message));
    +        break;
    +      }
    +      case kQueryResultRelationMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::QueryResultRelationMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        for (int i = 0; i < proto.blocks_size(); ++i) {
    +          const block_id block = proto.blocks(i);
    +          storage_manager_->saveBlockOrBlob(block);
    +          if (storage_manager_->blockOrBlobIsLoaded(block)) {
    +            // NOTE(zuyu): eviction is required to avoid accesses to the query
    +            // result relation schema in CatalogDatabaseCache, for all query
    +            // optimizer execution generator unit tests and the single-process
    +            // Quickstep CLI.
    +            storage_manager_->evictBlockOrBlob(block);
    +          }
    +        }
    +
    +        serialization::QueryResultRelationResponseMessage ack_proto;
    +        ack_proto.set_relation_id(proto.relation_id());
    +
    +        const size_t ack_proto_length = ack_proto.ByteSize();
    +        char *ack_proto_bytes = static_cast<char*>(malloc(ack_proto_length));
    +        CHECK(ack_proto.SerializeToArray(ack_proto_bytes, ack_proto_length));
    +
    +        TaggedMessage ack_message(static_cast<const void*>(ack_proto_bytes),
    +                                  ack_proto_length,
    +                                  kQueryResultRelationResponseMessage);
    +        free(ack_proto_bytes);
    +
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') sent QueryResultRelationResponseMessage (typed '" << kQueryResultRelationResponseMessage
    +                  << ") to Foreman";
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           foreman_client_id_,
    +                                           move(ack_message));
    +        break;
    +      }
    +      case kPoisonMessage: {
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded PoisonMessage (typed '" << kPoisonMessage
    +                  << "') from Foreman to all workers";
    +
    +        tmb::MessageStyle broadcast_style;
    +        broadcast_style.Broadcast(true);
    +
    +        tmb::MessageBus::SendStatus send_status =
    +            bus_->Send(shiftboss_client_id_,
    +                       worker_addresses_,
    +                       broadcast_style,
    +                       move(annotated_message.tagged_message));
    +        DCHECK(send_status == tmb::MessageBus::SendStatus::kOK);
    +        return;
    +      }
    +      default: {
    +        LOG(FATAL) << "Unknown TMB message type";
    +      }
    +    }
    +  }
    +}
    +
    +size_t Shiftboss::getSchedulableWorker() {
    +  const size_t num_workers = workers_->getNumWorkers();
    +
    +  size_t curr_worker = start_worker_index_;
    +  for (;;) {
    +    if (workers_->getNumQueuedWorkOrders(curr_worker) < max_msgs_per_worker_) {
    +      start_worker_index_ = (curr_worker + 1) % num_workers;
    +      // TODO(zuyu): workers_->incrementNumQueuedWorkOrders(curr_worker);
    +      // But we need a WorkOrder queue first.
    +      return curr_worker;
    +    }
    +
    +    curr_worker = (curr_worker + 1) % num_workers;
    +  }
    +}
    +
    +void Shiftboss::registerWithForeman() {
    +  LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +            << "') sent ShiftbossRegistrationMessage (typed '" << kShiftbossRegistrationMessage
    +            << "') to all";
    +
    +  tmb::Address all_addresses;
    +  all_addresses.All(true);
    +
    +  tmb::MessageStyle style;
    +
    +  serialization::ShiftbossRegistrationMessage proto;
    +  proto.set_work_order_capacity(getWorkOrderCapacity());
    +
    +  const size_t proto_length = proto.ByteSize();
    +  char *proto_bytes = static_cast<char*>(malloc(proto_length));
    +  CHECK(proto.SerializeToArray(proto_bytes, proto_length));
    +
    +  TaggedMessage message(static_cast<const void*>(proto_bytes),
    +                        proto_length,
    +                        kShiftbossRegistrationMessage);
    +  free(proto_bytes);
    +
    +  tmb::MessageBus::SendStatus send_status =
    +      bus_->Send(shiftboss_client_id_, all_addresses, style, move(message));
    +  DCHECK(send_status == tmb::MessageBus::SendStatus::kOK);
    +}
    +
    +void Shiftboss::processShiftbossInitiateMessage(
    +    const std::size_t query_id,
    +    const serialization::CatalogDatabase &catalog_database_cache_proto,
    +    const serialization::QueryContext &query_context_proto) {
    +  database_cache_.update(catalog_database_cache_proto);
    +
    +  unique_ptr<QueryContext> query_context(
    +      new QueryContext(query_context_proto,
    --- End diff --
    
    Have changed to `QueryInitiate`. Thanks!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #64: Introduced Shiftboss for the distribut...

Posted by hbdeshmukh <gi...@git.apache.org>.
Github user hbdeshmukh commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/64#discussion_r72118495
  
    --- Diff: query_execution/Shiftboss.hpp ---
    @@ -0,0 +1,238 @@
    +/**
    + *   Licensed under the Apache License, Version 2.0 (the "License");
    + *   you may not use this file except in compliance with the License.
    + *   You may obtain a copy of the License at
    + *
    + *       http://www.apache.org/licenses/LICENSE-2.0
    + *
    + *   Unless required by applicable law or agreed to in writing, software
    + *   distributed under the License is distributed on an "AS IS" BASIS,
    + *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + *   See the License for the specific language governing permissions and
    + *   limitations under the License.
    + **/
    +
    +#ifndef QUICKSTEP_QUERY_EXECUTION_SHIFTBOSS_HPP_
    +#define QUICKSTEP_QUERY_EXECUTION_SHIFTBOSS_HPP_
    +
    +#include <cstddef>
    +#include <memory>
    +
    +#include "catalog/CatalogDatabaseCache.hpp"
    +#include "catalog/CatalogTypedefs.hpp"
    +#include "query_execution/QueryContext.hpp"
    +#include "query_execution/QueryExecutionTypedefs.hpp"
    +#include "query_execution/WorkerDirectory.hpp"
    +#include "threading/Thread.hpp"
    +#include "utility/Macros.hpp"
    +
    +#include "glog/logging.h"
    +
    +#include "tmb/address.h"
    +#include "tmb/id_typedefs.h"
    +#include "tmb/message_bus.h"
    +
    +namespace quickstep {
    +
    +class StorageManager;
    +
    +namespace serialization {
    +class CatalogDatabase;
    +class QueryContext;
    +}  // namespace serialization
    +
    +/** \addtogroup QueryExecution
    + *  @{
    + */
    +
    +/**
    + * @brief The Shiftboss accepts workorder protos from shiftboss, and assigns
    + *        the workorders to workers.
    + **/
    +class Shiftboss : public Thread {
    + public:
    +  /**
    +   * @brief Constructor.
    +   *
    +   * @param bus A pointer to the TMB.
    +   * @param storage_manager The StorageManager to use.
    +   * @param workers A pointer to the WorkerDirectory.
    +   * @param cpu_id The ID of the CPU to which the Shiftboss thread can be pinned.
    +   *
    +   * @note If cpu_id is not specified, Shiftboss thread can be possibly moved
    +   *       around on different CPUs by the OS.
    +  **/
    +  Shiftboss(tmb::MessageBus *bus,
    +            StorageManager *storage_manager,
    +            WorkerDirectory *workers,
    +            const int cpu_id = -1)
    +      : bus_(DCHECK_NOTNULL(bus)),
    +        storage_manager_(DCHECK_NOTNULL(storage_manager)),
    +        workers_(DCHECK_NOTNULL(workers)),
    +        cpu_id_(cpu_id),
    +        shiftboss_client_id_(tmb::kClientIdNone),
    +        foreman_client_id_(tmb::kClientIdNone),
    +        max_msgs_per_worker_(1),
    +        start_worker_index_(0u) {
    +    // Check to have at least one Worker.
    +    DCHECK_GT(workers->getNumWorkers(), 0u);
    +
    +    shiftboss_client_id_ = bus_->Connect();
    +    LOG(INFO) << "Shiftboss TMB client ID: " << shiftboss_client_id_;
    +    DCHECK_NE(shiftboss_client_id_, tmb::kClientIdNone);
    +
    +    // Messages between Foreman and Shiftboss.
    +    bus_->RegisterClientAsSender(shiftboss_client_id_, kShiftbossRegistrationMessage);
    --- End diff --
    
    I am unable to follow this - shouldn't we require two different types of buses? One for communication between foreman and shiftboss and another for communication between workers and shiftboss? I see only one bus instance being used here. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #64: Introduced Shiftboss for the distribut...

Posted by hbdeshmukh <gi...@git.apache.org>.
Github user hbdeshmukh commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/64#discussion_r72335984
  
    --- Diff: query_execution/Shiftboss.cpp ---
    @@ -0,0 +1,357 @@
    +/**
    + *   Licensed under the Apache License, Version 2.0 (the "License");
    + *   you may not use this file except in compliance with the License.
    + *   You may obtain a copy of the License at
    + *
    + *       http://www.apache.org/licenses/LICENSE-2.0
    + *
    + *   Unless required by applicable law or agreed to in writing, software
    + *   distributed under the License is distributed on an "AS IS" BASIS,
    + *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + *   See the License for the specific language governing permissions and
    + *   limitations under the License.
    + **/
    +
    +#include "query_execution/Shiftboss.hpp"
    +
    +#include <cstddef>
    +#include <cstdlib>
    +#include <memory>
    +#include <string>
    +#include <unordered_map>
    +#include <utility>
    +#include <vector>
    +
    +#include "catalog/CatalogTypedefs.hpp"
    +#include "query_execution/QueryContext.hpp"
    +#include "query_execution/QueryExecutionMessages.pb.h"
    +#include "query_execution/QueryExecutionTypedefs.hpp"
    +#include "query_execution/QueryExecutionUtil.hpp"
    +#include "query_execution/WorkerMessage.hpp"
    +#include "relational_operators/RebuildWorkOrder.hpp"
    +#include "relational_operators/WorkOrderFactory.hpp"
    +#include "storage/InsertDestination.hpp"
    +#include "storage/StorageBlock.hpp"
    +#include "storage/StorageBlockInfo.hpp"
    +#include "storage/StorageManager.hpp"
    +#include "threading/ThreadUtil.hpp"
    +
    +#include "glog/logging.h"
    +
    +#include "tmb/address.h"
    +#include "tmb/id_typedefs.h"
    +#include "tmb/message_bus.h"
    +#include "tmb/message_style.h"
    +#include "tmb/tagged_message.h"
    +
    +using std::free;
    +using std::malloc;
    +using std::move;
    +using std::size_t;
    +using std::string;
    +using std::unique_ptr;
    +using std::vector;
    +
    +using tmb::TaggedMessage;
    +
    +namespace quickstep {
    +
    +class WorkOrder;
    +
    +void Shiftboss::run() {
    +  if (cpu_id_ >= 0) {
    +    // We can pin the shiftboss thread to a CPU if specified.
    +    ThreadUtil::BindToCPU(cpu_id_);
    +  }
    +
    +  for (;;) {
    +    // Receive() is a blocking call, causing this thread to sleep until next
    +    // message is received.
    +    AnnotatedMessage annotated_message(bus_->Receive(shiftboss_client_id_, 0, true));
    +    LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +              << "') received the typed '" << annotated_message.tagged_message.message_type()
    +              << "' message from client " << annotated_message.sender;
    +    switch (annotated_message.tagged_message.message_type()) {
    +      case kShiftbossRegistrationResponseMessage: {
    +        foreman_client_id_ = annotated_message.sender;
    +        break;
    +      }
    +      case kShiftbossInitiateMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::ShiftbossInitiateMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processShiftbossInitiateMessage(proto.query_id(), proto.catalog_database_cache(), proto.query_context());
    +        break;
    +      }
    +      case kWorkOrderMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::WorkOrderMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        const std::size_t query_id = proto.query_id();
    +        DCHECK_EQ(1u, query_contexts_.count(query_id));
    +
    +        WorkOrder *work_order = WorkOrderFactory::ReconstructFromProto(proto.work_order(),
    +                                                                       &database_cache_,
    +                                                                       query_contexts_[query_id].get(),
    +                                                                       storage_manager_,
    +                                                                       shiftboss_client_id_,
    +                                                                       bus_);
    +
    +        unique_ptr<WorkerMessage> worker_message(
    +            WorkerMessage::WorkOrderMessage(work_order, proto.operator_index()));
    +
    +        TaggedMessage worker_tagged_message(worker_message.get(),
    +                                            sizeof(*worker_message),
    +                                            kWorkOrderMessage);
    +
    +        const size_t worker_index = getSchedulableWorker();
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded WorkOrderMessage (typed '" << kWorkOrderMessage
    +                  << "') from Foreman to worker " << worker_index;
    +
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           workers_->getClientID(worker_index),
    +                                           move(worker_tagged_message));
    +        break;
    +      }
    +      case kInitiateRebuildMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::InitiateRebuildMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processInitiateRebuildMessage(proto.query_id(),
    +                                      proto.operator_index(),
    +                                      proto.insert_destination_index(),
    +                                      proto.relation_id());
    +        break;
    +      }
    +      case kWorkOrderCompleteMessage:  // Fall through.
    +      case kRebuildWorkOrderCompleteMessage:
    +      case kDataPipelineMessage:
    +      case kWorkOrdersAvailableMessage:
    +      case kWorkOrderFeedbackMessage: {
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded typed '" << annotated_message.tagged_message.message_type()
    +                  << "' message from worker (client " << annotated_message.sender << ") to Foreman";
    +
    +        DCHECK_NE(foreman_client_id_, tmb::kClientIdNone);
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           foreman_client_id_,
    +                                           move(annotated_message.tagged_message));
    +        break;
    +      }
    +      case kQueryResultRelationMessage: {
    --- End diff --
    
    What is the purpose of this message? From the description in QueryExecutionTypedefs, it appears that the message gets sent from the Foreman to the Shiftboss. Why would a Shiftboss want to receive a result relation from the Foreman? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #64: Introduced Shiftboss for the distribut...

Posted by hbdeshmukh <gi...@git.apache.org>.
Github user hbdeshmukh commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/64#discussion_r72335211
  
    --- Diff: query_execution/Shiftboss.cpp ---
    @@ -0,0 +1,357 @@
    +/**
    + *   Licensed under the Apache License, Version 2.0 (the "License");
    + *   you may not use this file except in compliance with the License.
    + *   You may obtain a copy of the License at
    + *
    + *       http://www.apache.org/licenses/LICENSE-2.0
    + *
    + *   Unless required by applicable law or agreed to in writing, software
    + *   distributed under the License is distributed on an "AS IS" BASIS,
    + *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + *   See the License for the specific language governing permissions and
    + *   limitations under the License.
    + **/
    +
    +#include "query_execution/Shiftboss.hpp"
    +
    +#include <cstddef>
    +#include <cstdlib>
    +#include <memory>
    +#include <string>
    +#include <unordered_map>
    +#include <utility>
    +#include <vector>
    +
    +#include "catalog/CatalogTypedefs.hpp"
    +#include "query_execution/QueryContext.hpp"
    +#include "query_execution/QueryExecutionMessages.pb.h"
    +#include "query_execution/QueryExecutionTypedefs.hpp"
    +#include "query_execution/QueryExecutionUtil.hpp"
    +#include "query_execution/WorkerMessage.hpp"
    +#include "relational_operators/RebuildWorkOrder.hpp"
    +#include "relational_operators/WorkOrderFactory.hpp"
    +#include "storage/InsertDestination.hpp"
    +#include "storage/StorageBlock.hpp"
    +#include "storage/StorageBlockInfo.hpp"
    +#include "storage/StorageManager.hpp"
    +#include "threading/ThreadUtil.hpp"
    +
    +#include "glog/logging.h"
    +
    +#include "tmb/address.h"
    +#include "tmb/id_typedefs.h"
    +#include "tmb/message_bus.h"
    +#include "tmb/message_style.h"
    +#include "tmb/tagged_message.h"
    +
    +using std::free;
    +using std::malloc;
    +using std::move;
    +using std::size_t;
    +using std::string;
    +using std::unique_ptr;
    +using std::vector;
    +
    +using tmb::TaggedMessage;
    +
    +namespace quickstep {
    +
    +class WorkOrder;
    +
    +void Shiftboss::run() {
    +  if (cpu_id_ >= 0) {
    +    // We can pin the shiftboss thread to a CPU if specified.
    +    ThreadUtil::BindToCPU(cpu_id_);
    +  }
    +
    +  for (;;) {
    +    // Receive() is a blocking call, causing this thread to sleep until next
    +    // message is received.
    +    AnnotatedMessage annotated_message(bus_->Receive(shiftboss_client_id_, 0, true));
    +    LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +              << "') received the typed '" << annotated_message.tagged_message.message_type()
    +              << "' message from client " << annotated_message.sender;
    +    switch (annotated_message.tagged_message.message_type()) {
    +      case kShiftbossRegistrationResponseMessage: {
    +        foreman_client_id_ = annotated_message.sender;
    +        break;
    +      }
    +      case kShiftbossInitiateMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::ShiftbossInitiateMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processShiftbossInitiateMessage(proto.query_id(), proto.catalog_database_cache(), proto.query_context());
    +        break;
    +      }
    +      case kWorkOrderMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::WorkOrderMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        const std::size_t query_id = proto.query_id();
    +        DCHECK_EQ(1u, query_contexts_.count(query_id));
    +
    +        WorkOrder *work_order = WorkOrderFactory::ReconstructFromProto(proto.work_order(),
    +                                                                       &database_cache_,
    +                                                                       query_contexts_[query_id].get(),
    +                                                                       storage_manager_,
    +                                                                       shiftboss_client_id_,
    +                                                                       bus_);
    +
    +        unique_ptr<WorkerMessage> worker_message(
    +            WorkerMessage::WorkOrderMessage(work_order, proto.operator_index()));
    +
    +        TaggedMessage worker_tagged_message(worker_message.get(),
    +                                            sizeof(*worker_message),
    +                                            kWorkOrderMessage);
    +
    +        const size_t worker_index = getSchedulableWorker();
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded WorkOrderMessage (typed '" << kWorkOrderMessage
    +                  << "') from Foreman to worker " << worker_index;
    +
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           workers_->getClientID(worker_index),
    +                                           move(worker_tagged_message));
    +        break;
    +      }
    +      case kInitiateRebuildMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::InitiateRebuildMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processInitiateRebuildMessage(proto.query_id(),
    +                                      proto.operator_index(),
    +                                      proto.insert_destination_index(),
    +                                      proto.relation_id());
    +        break;
    +      }
    +      case kWorkOrderCompleteMessage:  // Fall through.
    +      case kRebuildWorkOrderCompleteMessage:
    +      case kDataPipelineMessage:
    +      case kWorkOrdersAvailableMessage:
    +      case kWorkOrderFeedbackMessage: {
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded typed '" << annotated_message.tagged_message.message_type()
    +                  << "' message from worker (client " << annotated_message.sender << ") to Foreman";
    +
    +        DCHECK_NE(foreman_client_id_, tmb::kClientIdNone);
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           foreman_client_id_,
    +                                           move(annotated_message.tagged_message));
    +        break;
    +      }
    +      case kQueryResultRelationMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::QueryResultRelationMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        for (int i = 0; i < proto.blocks_size(); ++i) {
    +          const block_id block = proto.blocks(i);
    +          storage_manager_->saveBlockOrBlob(block);
    +          if (storage_manager_->blockOrBlobIsLoaded(block)) {
    +            // NOTE(zuyu): eviction is required to avoid accesses to the query
    +            // result relation schema in CatalogDatabaseCache, for all query
    +            // optimizer execution generator unit tests and the single-process
    +            // Quickstep CLI.
    +            storage_manager_->evictBlockOrBlob(block);
    --- End diff --
    
    I am not following this code and the corresponding NOTE - Why do you need to explicitly evict a block after calling ``saveBlockOrBlob`` on it?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #64: Introduced Shiftboss for the distribut...

Posted by hbdeshmukh <gi...@git.apache.org>.
Github user hbdeshmukh commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/64#discussion_r72509054
  
    --- Diff: query_execution/Shiftboss.cpp ---
    @@ -0,0 +1,357 @@
    +/**
    + *   Licensed under the Apache License, Version 2.0 (the "License");
    + *   you may not use this file except in compliance with the License.
    + *   You may obtain a copy of the License at
    + *
    + *       http://www.apache.org/licenses/LICENSE-2.0
    + *
    + *   Unless required by applicable law or agreed to in writing, software
    + *   distributed under the License is distributed on an "AS IS" BASIS,
    + *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + *   See the License for the specific language governing permissions and
    + *   limitations under the License.
    + **/
    +
    +#include "query_execution/Shiftboss.hpp"
    +
    +#include <cstddef>
    +#include <cstdlib>
    +#include <memory>
    +#include <string>
    +#include <unordered_map>
    +#include <utility>
    +#include <vector>
    +
    +#include "catalog/CatalogTypedefs.hpp"
    +#include "query_execution/QueryContext.hpp"
    +#include "query_execution/QueryExecutionMessages.pb.h"
    +#include "query_execution/QueryExecutionTypedefs.hpp"
    +#include "query_execution/QueryExecutionUtil.hpp"
    +#include "query_execution/WorkerMessage.hpp"
    +#include "relational_operators/RebuildWorkOrder.hpp"
    +#include "relational_operators/WorkOrderFactory.hpp"
    +#include "storage/InsertDestination.hpp"
    +#include "storage/StorageBlock.hpp"
    +#include "storage/StorageBlockInfo.hpp"
    +#include "storage/StorageManager.hpp"
    +#include "threading/ThreadUtil.hpp"
    +
    +#include "glog/logging.h"
    +
    +#include "tmb/address.h"
    +#include "tmb/id_typedefs.h"
    +#include "tmb/message_bus.h"
    +#include "tmb/message_style.h"
    +#include "tmb/tagged_message.h"
    +
    +using std::free;
    +using std::malloc;
    +using std::move;
    +using std::size_t;
    +using std::string;
    +using std::unique_ptr;
    +using std::vector;
    +
    +using tmb::TaggedMessage;
    +
    +namespace quickstep {
    +
    +class WorkOrder;
    +
    +void Shiftboss::run() {
    +  if (cpu_id_ >= 0) {
    +    // We can pin the shiftboss thread to a CPU if specified.
    +    ThreadUtil::BindToCPU(cpu_id_);
    +  }
    +
    +  for (;;) {
    +    // Receive() is a blocking call, causing this thread to sleep until next
    +    // message is received.
    +    AnnotatedMessage annotated_message(bus_->Receive(shiftboss_client_id_, 0, true));
    +    LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +              << "') received the typed '" << annotated_message.tagged_message.message_type()
    +              << "' message from client " << annotated_message.sender;
    +    switch (annotated_message.tagged_message.message_type()) {
    +      case kShiftbossRegistrationResponseMessage: {
    +        foreman_client_id_ = annotated_message.sender;
    +        break;
    +      }
    +      case kQueryInitiateMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::QueryInitiateMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processQueryInitiateMessage(proto.query_id(), proto.catalog_database_cache(), proto.query_context());
    +        break;
    +      }
    +      case kWorkOrderMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::WorkOrderMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        const std::size_t query_id = proto.query_id();
    +        DCHECK_EQ(1u, query_contexts_.count(query_id));
    +
    +        WorkOrder *work_order = WorkOrderFactory::ReconstructFromProto(proto.work_order(),
    +                                                                       &database_cache_,
    +                                                                       query_contexts_[query_id].get(),
    +                                                                       storage_manager_,
    +                                                                       shiftboss_client_id_,
    +                                                                       bus_);
    +
    +        unique_ptr<WorkerMessage> worker_message(
    +            WorkerMessage::WorkOrderMessage(work_order, proto.operator_index()));
    +
    +        TaggedMessage worker_tagged_message(worker_message.get(),
    +                                            sizeof(*worker_message),
    +                                            kWorkOrderMessage);
    +
    +        const size_t worker_index = getSchedulableWorker();
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded WorkOrderMessage (typed '" << kWorkOrderMessage
    +                  << "') from Foreman to worker " << worker_index;
    +
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           workers_->getClientID(worker_index),
    +                                           move(worker_tagged_message));
    +        break;
    +      }
    +      case kInitiateRebuildMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::InitiateRebuildMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processInitiateRebuildMessage(proto.query_id(),
    +                                      proto.operator_index(),
    +                                      proto.insert_destination_index(),
    +                                      proto.relation_id());
    +        break;
    +      }
    +      case kWorkOrderCompleteMessage:  // Fall through.
    +      case kRebuildWorkOrderCompleteMessage:
    +      case kDataPipelineMessage:
    +      case kWorkOrdersAvailableMessage:
    +      case kWorkOrderFeedbackMessage: {
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded typed '" << annotated_message.tagged_message.message_type()
    +                  << "' message from worker (client " << annotated_message.sender << ") to Foreman";
    +
    +        DCHECK_NE(foreman_client_id_, tmb::kClientIdNone);
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           foreman_client_id_,
    +                                           move(annotated_message.tagged_message));
    +        break;
    +      }
    +      case kQueryResultRelationMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::QueryResultRelationMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        for (int i = 0; i < proto.blocks_size(); ++i) {
    +          const block_id block = proto.blocks(i);
    --- End diff --
    
    > Note that the code works in the single Worker
    
    Do you mean single shiftboss? 
    
    > ForemanDistributed knows the query result relation from QueryHandle\uff0c which contains the blocks info.
    
    Does the ``ForemanDistributed`` get to know about blocks in the result relation via messages of type ``kCatalogRelationNewBlockMessage``? If so, who sends these messages to the ``ForemanDistributed``? I don't see this message type getting used in the ShiftBoss code. Then perhaps the workers? 
    
    > For now, no. And we should refactor later.
    
    This may cause issues when you extend the design to multiple shiftboss. Adding a TODO is highly recommended. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #64: Introduced Shiftboss for the distribut...

Posted by hbdeshmukh <gi...@git.apache.org>.
Github user hbdeshmukh commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/64#discussion_r72338107
  
    --- Diff: query_execution/Shiftboss.cpp ---
    @@ -0,0 +1,357 @@
    +/**
    + *   Licensed under the Apache License, Version 2.0 (the "License");
    + *   you may not use this file except in compliance with the License.
    + *   You may obtain a copy of the License at
    + *
    + *       http://www.apache.org/licenses/LICENSE-2.0
    + *
    + *   Unless required by applicable law or agreed to in writing, software
    + *   distributed under the License is distributed on an "AS IS" BASIS,
    + *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + *   See the License for the specific language governing permissions and
    + *   limitations under the License.
    + **/
    +
    +#include "query_execution/Shiftboss.hpp"
    +
    +#include <cstddef>
    +#include <cstdlib>
    +#include <memory>
    +#include <string>
    +#include <unordered_map>
    +#include <utility>
    +#include <vector>
    +
    +#include "catalog/CatalogTypedefs.hpp"
    +#include "query_execution/QueryContext.hpp"
    +#include "query_execution/QueryExecutionMessages.pb.h"
    +#include "query_execution/QueryExecutionTypedefs.hpp"
    +#include "query_execution/QueryExecutionUtil.hpp"
    +#include "query_execution/WorkerMessage.hpp"
    +#include "relational_operators/RebuildWorkOrder.hpp"
    +#include "relational_operators/WorkOrderFactory.hpp"
    +#include "storage/InsertDestination.hpp"
    +#include "storage/StorageBlock.hpp"
    +#include "storage/StorageBlockInfo.hpp"
    +#include "storage/StorageManager.hpp"
    +#include "threading/ThreadUtil.hpp"
    +
    +#include "glog/logging.h"
    +
    +#include "tmb/address.h"
    +#include "tmb/id_typedefs.h"
    +#include "tmb/message_bus.h"
    +#include "tmb/message_style.h"
    +#include "tmb/tagged_message.h"
    +
    +using std::free;
    +using std::malloc;
    +using std::move;
    +using std::size_t;
    +using std::string;
    +using std::unique_ptr;
    +using std::vector;
    +
    +using tmb::TaggedMessage;
    +
    +namespace quickstep {
    +
    +class WorkOrder;
    +
    +void Shiftboss::run() {
    +  if (cpu_id_ >= 0) {
    +    // We can pin the shiftboss thread to a CPU if specified.
    +    ThreadUtil::BindToCPU(cpu_id_);
    +  }
    +
    +  for (;;) {
    +    // Receive() is a blocking call, causing this thread to sleep until next
    +    // message is received.
    +    AnnotatedMessage annotated_message(bus_->Receive(shiftboss_client_id_, 0, true));
    +    LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +              << "') received the typed '" << annotated_message.tagged_message.message_type()
    +              << "' message from client " << annotated_message.sender;
    +    switch (annotated_message.tagged_message.message_type()) {
    +      case kShiftbossRegistrationResponseMessage: {
    +        foreman_client_id_ = annotated_message.sender;
    +        break;
    +      }
    +      case kShiftbossInitiateMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::ShiftbossInitiateMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processShiftbossInitiateMessage(proto.query_id(), proto.catalog_database_cache(), proto.query_context());
    +        break;
    +      }
    +      case kWorkOrderMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::WorkOrderMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        const std::size_t query_id = proto.query_id();
    +        DCHECK_EQ(1u, query_contexts_.count(query_id));
    +
    +        WorkOrder *work_order = WorkOrderFactory::ReconstructFromProto(proto.work_order(),
    +                                                                       &database_cache_,
    +                                                                       query_contexts_[query_id].get(),
    +                                                                       storage_manager_,
    +                                                                       shiftboss_client_id_,
    +                                                                       bus_);
    +
    +        unique_ptr<WorkerMessage> worker_message(
    +            WorkerMessage::WorkOrderMessage(work_order, proto.operator_index()));
    +
    +        TaggedMessage worker_tagged_message(worker_message.get(),
    +                                            sizeof(*worker_message),
    +                                            kWorkOrderMessage);
    +
    +        const size_t worker_index = getSchedulableWorker();
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded WorkOrderMessage (typed '" << kWorkOrderMessage
    +                  << "') from Foreman to worker " << worker_index;
    +
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           workers_->getClientID(worker_index),
    +                                           move(worker_tagged_message));
    +        break;
    +      }
    +      case kInitiateRebuildMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::InitiateRebuildMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processInitiateRebuildMessage(proto.query_id(),
    +                                      proto.operator_index(),
    +                                      proto.insert_destination_index(),
    +                                      proto.relation_id());
    +        break;
    +      }
    +      case kWorkOrderCompleteMessage:  // Fall through.
    +      case kRebuildWorkOrderCompleteMessage:
    +      case kDataPipelineMessage:
    +      case kWorkOrdersAvailableMessage:
    +      case kWorkOrderFeedbackMessage: {
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded typed '" << annotated_message.tagged_message.message_type()
    +                  << "' message from worker (client " << annotated_message.sender << ") to Foreman";
    +
    +        DCHECK_NE(foreman_client_id_, tmb::kClientIdNone);
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           foreman_client_id_,
    +                                           move(annotated_message.tagged_message));
    +        break;
    +      }
    +      case kQueryResultRelationMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::QueryResultRelationMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        for (int i = 0; i < proto.blocks_size(); ++i) {
    +          const block_id block = proto.blocks(i);
    +          storage_manager_->saveBlockOrBlob(block);
    +          if (storage_manager_->blockOrBlobIsLoaded(block)) {
    +            // NOTE(zuyu): eviction is required to avoid accesses to the query
    +            // result relation schema in CatalogDatabaseCache, for all query
    +            // optimizer execution generator unit tests and the single-process
    +            // Quickstep CLI.
    +            storage_manager_->evictBlockOrBlob(block);
    +          }
    +        }
    +
    +        serialization::QueryResultRelationResponseMessage ack_proto;
    +        ack_proto.set_relation_id(proto.relation_id());
    +
    +        const size_t ack_proto_length = ack_proto.ByteSize();
    +        char *ack_proto_bytes = static_cast<char*>(malloc(ack_proto_length));
    +        CHECK(ack_proto.SerializeToArray(ack_proto_bytes, ack_proto_length));
    +
    +        TaggedMessage ack_message(static_cast<const void*>(ack_proto_bytes),
    +                                  ack_proto_length,
    +                                  kQueryResultRelationResponseMessage);
    +        free(ack_proto_bytes);
    +
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') sent QueryResultRelationResponseMessage (typed '" << kQueryResultRelationResponseMessage
    +                  << ") to Foreman";
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           foreman_client_id_,
    +                                           move(ack_message));
    +        break;
    +      }
    +      case kPoisonMessage: {
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded PoisonMessage (typed '" << kPoisonMessage
    +                  << "') from Foreman to all workers";
    +
    +        tmb::MessageStyle broadcast_style;
    +        broadcast_style.Broadcast(true);
    +
    +        tmb::MessageBus::SendStatus send_status =
    +            bus_->Send(shiftboss_client_id_,
    +                       worker_addresses_,
    +                       broadcast_style,
    +                       move(annotated_message.tagged_message));
    +        DCHECK(send_status == tmb::MessageBus::SendStatus::kOK);
    +        return;
    +      }
    +      default: {
    +        LOG(FATAL) << "Unknown TMB message type";
    +      }
    +    }
    +  }
    +}
    +
    +size_t Shiftboss::getSchedulableWorker() {
    +  const size_t num_workers = workers_->getNumWorkers();
    +
    +  size_t curr_worker = start_worker_index_;
    +  for (;;) {
    +    if (workers_->getNumQueuedWorkOrders(curr_worker) < max_msgs_per_worker_) {
    +      start_worker_index_ = (curr_worker + 1) % num_workers;
    +      // TODO(zuyu): workers_->incrementNumQueuedWorkOrders(curr_worker);
    +      // But we need a WorkOrder queue first.
    +      return curr_worker;
    +    }
    +
    +    curr_worker = (curr_worker + 1) % num_workers;
    +  }
    +}
    +
    +void Shiftboss::registerWithForeman() {
    +  LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +            << "') sent ShiftbossRegistrationMessage (typed '" << kShiftbossRegistrationMessage
    +            << "') to all";
    +
    +  tmb::Address all_addresses;
    +  all_addresses.All(true);
    +
    +  tmb::MessageStyle style;
    +
    +  serialization::ShiftbossRegistrationMessage proto;
    +  proto.set_work_order_capacity(getWorkOrderCapacity());
    +
    +  const size_t proto_length = proto.ByteSize();
    +  char *proto_bytes = static_cast<char*>(malloc(proto_length));
    +  CHECK(proto.SerializeToArray(proto_bytes, proto_length));
    +
    +  TaggedMessage message(static_cast<const void*>(proto_bytes),
    +                        proto_length,
    +                        kShiftbossRegistrationMessage);
    +  free(proto_bytes);
    +
    +  tmb::MessageBus::SendStatus send_status =
    +      bus_->Send(shiftboss_client_id_, all_addresses, style, move(message));
    +  DCHECK(send_status == tmb::MessageBus::SendStatus::kOK);
    +}
    +
    +void Shiftboss::processShiftbossInitiateMessage(
    +    const std::size_t query_id,
    +    const serialization::CatalogDatabase &catalog_database_cache_proto,
    +    const serialization::QueryContext &query_context_proto) {
    +  database_cache_.update(catalog_database_cache_proto);
    +
    +  unique_ptr<QueryContext> query_context(
    +      new QueryContext(query_context_proto,
    --- End diff --
    
    By shift boss initiation, do you mean creation of a query context for a given query? Then the function and the message type names are misleading. Initiation of shiftboss could be interpreted as initialization of the shiftboss thread. Once the shiftboss is initialized, the Foreman can ask it to initialize the query context for a given query. I feel we should clarify this distinction. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #64: Introduced Shiftboss for the distribut...

Posted by zuyu <gi...@git.apache.org>.
Github user zuyu commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/64#discussion_r72526641
  
    --- Diff: query_execution/Shiftboss.cpp ---
    @@ -0,0 +1,357 @@
    +/**
    + *   Licensed under the Apache License, Version 2.0 (the "License");
    + *   you may not use this file except in compliance with the License.
    + *   You may obtain a copy of the License at
    + *
    + *       http://www.apache.org/licenses/LICENSE-2.0
    + *
    + *   Unless required by applicable law or agreed to in writing, software
    + *   distributed under the License is distributed on an "AS IS" BASIS,
    + *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + *   See the License for the specific language governing permissions and
    + *   limitations under the License.
    + **/
    +
    +#include "query_execution/Shiftboss.hpp"
    +
    +#include <cstddef>
    +#include <cstdlib>
    +#include <memory>
    +#include <string>
    +#include <unordered_map>
    +#include <utility>
    +#include <vector>
    +
    +#include "catalog/CatalogTypedefs.hpp"
    +#include "query_execution/QueryContext.hpp"
    +#include "query_execution/QueryExecutionMessages.pb.h"
    +#include "query_execution/QueryExecutionTypedefs.hpp"
    +#include "query_execution/QueryExecutionUtil.hpp"
    +#include "query_execution/WorkerMessage.hpp"
    +#include "relational_operators/RebuildWorkOrder.hpp"
    +#include "relational_operators/WorkOrderFactory.hpp"
    +#include "storage/InsertDestination.hpp"
    +#include "storage/StorageBlock.hpp"
    +#include "storage/StorageBlockInfo.hpp"
    +#include "storage/StorageManager.hpp"
    +#include "threading/ThreadUtil.hpp"
    +
    +#include "glog/logging.h"
    +
    +#include "tmb/address.h"
    +#include "tmb/id_typedefs.h"
    +#include "tmb/message_bus.h"
    +#include "tmb/message_style.h"
    +#include "tmb/tagged_message.h"
    +
    +using std::free;
    +using std::malloc;
    +using std::move;
    +using std::size_t;
    +using std::string;
    +using std::unique_ptr;
    +using std::vector;
    +
    +using tmb::TaggedMessage;
    +
    +namespace quickstep {
    +
    +class WorkOrder;
    +
    +void Shiftboss::run() {
    +  if (cpu_id_ >= 0) {
    +    // We can pin the shiftboss thread to a CPU if specified.
    +    ThreadUtil::BindToCPU(cpu_id_);
    +  }
    +
    +  for (;;) {
    +    // Receive() is a blocking call, causing this thread to sleep until next
    +    // message is received.
    +    AnnotatedMessage annotated_message(bus_->Receive(shiftboss_client_id_, 0, true));
    +    LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +              << "') received the typed '" << annotated_message.tagged_message.message_type()
    +              << "' message from client " << annotated_message.sender;
    +    switch (annotated_message.tagged_message.message_type()) {
    +      case kShiftbossRegistrationResponseMessage: {
    +        foreman_client_id_ = annotated_message.sender;
    +        break;
    +      }
    +      case kShiftbossInitiateMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::ShiftbossInitiateMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processShiftbossInitiateMessage(proto.query_id(), proto.catalog_database_cache(), proto.query_context());
    +        break;
    +      }
    +      case kWorkOrderMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::WorkOrderMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        const std::size_t query_id = proto.query_id();
    +        DCHECK_EQ(1u, query_contexts_.count(query_id));
    +
    +        WorkOrder *work_order = WorkOrderFactory::ReconstructFromProto(proto.work_order(),
    +                                                                       &database_cache_,
    +                                                                       query_contexts_[query_id].get(),
    +                                                                       storage_manager_,
    +                                                                       shiftboss_client_id_,
    +                                                                       bus_);
    +
    +        unique_ptr<WorkerMessage> worker_message(
    +            WorkerMessage::WorkOrderMessage(work_order, proto.operator_index()));
    +
    +        TaggedMessage worker_tagged_message(worker_message.get(),
    +                                            sizeof(*worker_message),
    +                                            kWorkOrderMessage);
    +
    +        const size_t worker_index = getSchedulableWorker();
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded WorkOrderMessage (typed '" << kWorkOrderMessage
    +                  << "') from Foreman to worker " << worker_index;
    +
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           workers_->getClientID(worker_index),
    +                                           move(worker_tagged_message));
    +        break;
    +      }
    +      case kInitiateRebuildMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::InitiateRebuildMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processInitiateRebuildMessage(proto.query_id(),
    +                                      proto.operator_index(),
    +                                      proto.insert_destination_index(),
    +                                      proto.relation_id());
    +        break;
    +      }
    +      case kWorkOrderCompleteMessage:  // Fall through.
    +      case kRebuildWorkOrderCompleteMessage:
    +      case kDataPipelineMessage:
    +      case kWorkOrdersAvailableMessage:
    +      case kWorkOrderFeedbackMessage: {
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded typed '" << annotated_message.tagged_message.message_type()
    +                  << "' message from worker (client " << annotated_message.sender << ") to Foreman";
    +
    +        DCHECK_NE(foreman_client_id_, tmb::kClientIdNone);
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           foreman_client_id_,
    +                                           move(annotated_message.tagged_message));
    +        break;
    +      }
    +      case kQueryResultRelationMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::QueryResultRelationMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        for (int i = 0; i < proto.blocks_size(); ++i) {
    +          const block_id block = proto.blocks(i);
    +          storage_manager_->saveBlockOrBlob(block);
    +          if (storage_manager_->blockOrBlobIsLoaded(block)) {
    +            // NOTE(zuyu): eviction is required to avoid accesses to the query
    +            // result relation schema in CatalogDatabaseCache, for all query
    +            // optimizer execution generator unit tests and the single-process
    +            // Quickstep CLI.
    +            storage_manager_->evictBlockOrBlob(block);
    --- End diff --
    
    Note that the new distributed query execution engine unit test is running in a process, w/ `Foreman`, `Shiftboss`, and `worker thread` running in different threads. It also shares `StorageManager`.
    
    So evicting the query result blocks out from the buffer pool are required to avoid the reference to the schema in `CatalogDatabaseCache`. Using blocks in `cli` would result in invalid reference to the schema inaccessible to `cli`.
    
    On the other hand, `cli` would load the blocks while using its own copy of `CatalogRelation`.
    



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #64: Introduced Shiftboss for the distribut...

Posted by hbdeshmukh <gi...@git.apache.org>.
Github user hbdeshmukh commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/64#discussion_r72500919
  
    --- Diff: query_execution/Shiftboss.cpp ---
    @@ -0,0 +1,357 @@
    +/**
    + *   Licensed under the Apache License, Version 2.0 (the "License");
    + *   you may not use this file except in compliance with the License.
    + *   You may obtain a copy of the License at
    + *
    + *       http://www.apache.org/licenses/LICENSE-2.0
    + *
    + *   Unless required by applicable law or agreed to in writing, software
    + *   distributed under the License is distributed on an "AS IS" BASIS,
    + *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + *   See the License for the specific language governing permissions and
    + *   limitations under the License.
    + **/
    +
    +#include "query_execution/Shiftboss.hpp"
    +
    +#include <cstddef>
    +#include <cstdlib>
    +#include <memory>
    +#include <string>
    +#include <unordered_map>
    +#include <utility>
    +#include <vector>
    +
    +#include "catalog/CatalogTypedefs.hpp"
    +#include "query_execution/QueryContext.hpp"
    +#include "query_execution/QueryExecutionMessages.pb.h"
    +#include "query_execution/QueryExecutionTypedefs.hpp"
    +#include "query_execution/QueryExecutionUtil.hpp"
    +#include "query_execution/WorkerMessage.hpp"
    +#include "relational_operators/RebuildWorkOrder.hpp"
    +#include "relational_operators/WorkOrderFactory.hpp"
    +#include "storage/InsertDestination.hpp"
    +#include "storage/StorageBlock.hpp"
    +#include "storage/StorageBlockInfo.hpp"
    +#include "storage/StorageManager.hpp"
    +#include "threading/ThreadUtil.hpp"
    +
    +#include "glog/logging.h"
    +
    +#include "tmb/address.h"
    +#include "tmb/id_typedefs.h"
    +#include "tmb/message_bus.h"
    +#include "tmb/message_style.h"
    +#include "tmb/tagged_message.h"
    +
    +using std::free;
    +using std::malloc;
    +using std::move;
    +using std::size_t;
    +using std::string;
    +using std::unique_ptr;
    +using std::vector;
    +
    +using tmb::TaggedMessage;
    +
    +namespace quickstep {
    +
    +class WorkOrder;
    +
    +void Shiftboss::run() {
    +  if (cpu_id_ >= 0) {
    +    // We can pin the shiftboss thread to a CPU if specified.
    +    ThreadUtil::BindToCPU(cpu_id_);
    +  }
    +
    +  for (;;) {
    +    // Receive() is a blocking call, causing this thread to sleep until next
    +    // message is received.
    +    AnnotatedMessage annotated_message(bus_->Receive(shiftboss_client_id_, 0, true));
    +    LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +              << "') received the typed '" << annotated_message.tagged_message.message_type()
    +              << "' message from client " << annotated_message.sender;
    +    switch (annotated_message.tagged_message.message_type()) {
    +      case kShiftbossRegistrationResponseMessage: {
    +        foreman_client_id_ = annotated_message.sender;
    +        break;
    +      }
    +      case kShiftbossInitiateMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::ShiftbossInitiateMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processShiftbossInitiateMessage(proto.query_id(), proto.catalog_database_cache(), proto.query_context());
    +        break;
    +      }
    +      case kWorkOrderMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::WorkOrderMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        const std::size_t query_id = proto.query_id();
    +        DCHECK_EQ(1u, query_contexts_.count(query_id));
    +
    +        WorkOrder *work_order = WorkOrderFactory::ReconstructFromProto(proto.work_order(),
    +                                                                       &database_cache_,
    +                                                                       query_contexts_[query_id].get(),
    +                                                                       storage_manager_,
    +                                                                       shiftboss_client_id_,
    +                                                                       bus_);
    +
    +        unique_ptr<WorkerMessage> worker_message(
    +            WorkerMessage::WorkOrderMessage(work_order, proto.operator_index()));
    +
    +        TaggedMessage worker_tagged_message(worker_message.get(),
    +                                            sizeof(*worker_message),
    +                                            kWorkOrderMessage);
    +
    +        const size_t worker_index = getSchedulableWorker();
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded WorkOrderMessage (typed '" << kWorkOrderMessage
    +                  << "') from Foreman to worker " << worker_index;
    +
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           workers_->getClientID(worker_index),
    +                                           move(worker_tagged_message));
    +        break;
    +      }
    +      case kInitiateRebuildMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::InitiateRebuildMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processInitiateRebuildMessage(proto.query_id(),
    +                                      proto.operator_index(),
    +                                      proto.insert_destination_index(),
    +                                      proto.relation_id());
    +        break;
    +      }
    +      case kWorkOrderCompleteMessage:  // Fall through.
    +      case kRebuildWorkOrderCompleteMessage:
    +      case kDataPipelineMessage:
    +      case kWorkOrdersAvailableMessage:
    +      case kWorkOrderFeedbackMessage: {
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded typed '" << annotated_message.tagged_message.message_type()
    +                  << "' message from worker (client " << annotated_message.sender << ") to Foreman";
    +
    +        DCHECK_NE(foreman_client_id_, tmb::kClientIdNone);
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           foreman_client_id_,
    +                                           move(annotated_message.tagged_message));
    +        break;
    +      }
    +      case kQueryResultRelationMessage: {
    --- End diff --
    
    If the purpose of this message is to force the shiftboss to flush its query results, we should have a better name for the message type. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #64: Introduced Shiftboss for the distribut...

Posted by hbdeshmukh <gi...@git.apache.org>.
Github user hbdeshmukh commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/64#discussion_r72119766
  
    --- Diff: query_execution/Shiftboss.hpp ---
    @@ -0,0 +1,238 @@
    +/**
    + *   Licensed under the Apache License, Version 2.0 (the "License");
    + *   you may not use this file except in compliance with the License.
    + *   You may obtain a copy of the License at
    + *
    + *       http://www.apache.org/licenses/LICENSE-2.0
    + *
    + *   Unless required by applicable law or agreed to in writing, software
    + *   distributed under the License is distributed on an "AS IS" BASIS,
    + *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + *   See the License for the specific language governing permissions and
    + *   limitations under the License.
    + **/
    +
    +#ifndef QUICKSTEP_QUERY_EXECUTION_SHIFTBOSS_HPP_
    +#define QUICKSTEP_QUERY_EXECUTION_SHIFTBOSS_HPP_
    +
    +#include <cstddef>
    +#include <memory>
    +
    +#include "catalog/CatalogDatabaseCache.hpp"
    +#include "catalog/CatalogTypedefs.hpp"
    +#include "query_execution/QueryContext.hpp"
    +#include "query_execution/QueryExecutionTypedefs.hpp"
    +#include "query_execution/WorkerDirectory.hpp"
    +#include "threading/Thread.hpp"
    +#include "utility/Macros.hpp"
    +
    +#include "glog/logging.h"
    +
    +#include "tmb/address.h"
    +#include "tmb/id_typedefs.h"
    +#include "tmb/message_bus.h"
    +
    +namespace quickstep {
    +
    +class StorageManager;
    +
    +namespace serialization {
    +class CatalogDatabase;
    +class QueryContext;
    +}  // namespace serialization
    +
    +/** \addtogroup QueryExecution
    + *  @{
    + */
    +
    +/**
    + * @brief The Shiftboss accepts workorder protos from shiftboss, and assigns
    + *        the workorders to workers.
    + **/
    +class Shiftboss : public Thread {
    + public:
    +  /**
    +   * @brief Constructor.
    +   *
    +   * @param bus A pointer to the TMB.
    +   * @param storage_manager The StorageManager to use.
    +   * @param workers A pointer to the WorkerDirectory.
    +   * @param cpu_id The ID of the CPU to which the Shiftboss thread can be pinned.
    +   *
    +   * @note If cpu_id is not specified, Shiftboss thread can be possibly moved
    +   *       around on different CPUs by the OS.
    +  **/
    +  Shiftboss(tmb::MessageBus *bus,
    +            StorageManager *storage_manager,
    +            WorkerDirectory *workers,
    +            const int cpu_id = -1)
    +      : bus_(DCHECK_NOTNULL(bus)),
    +        storage_manager_(DCHECK_NOTNULL(storage_manager)),
    +        workers_(DCHECK_NOTNULL(workers)),
    +        cpu_id_(cpu_id),
    +        shiftboss_client_id_(tmb::kClientIdNone),
    +        foreman_client_id_(tmb::kClientIdNone),
    +        max_msgs_per_worker_(1),
    +        start_worker_index_(0u) {
    +    // Check to have at least one Worker.
    +    DCHECK_GT(workers->getNumWorkers(), 0u);
    +
    +    shiftboss_client_id_ = bus_->Connect();
    +    LOG(INFO) << "Shiftboss TMB client ID: " << shiftboss_client_id_;
    +    DCHECK_NE(shiftboss_client_id_, tmb::kClientIdNone);
    +
    +    // Messages between Foreman and Shiftboss.
    +    bus_->RegisterClientAsSender(shiftboss_client_id_, kShiftbossRegistrationMessage);
    +    bus_->RegisterClientAsReceiver(shiftboss_client_id_, kShiftbossRegistrationResponseMessage);
    +
    +    bus_->RegisterClientAsReceiver(shiftboss_client_id_, kShiftbossInitiateMessage);
    +    bus_->RegisterClientAsSender(shiftboss_client_id_, kShiftbossInitiateResponseMessage);
    +
    +    bus_->RegisterClientAsReceiver(shiftboss_client_id_, kInitiateRebuildMessage);
    +    bus_->RegisterClientAsSender(shiftboss_client_id_, kInitiateRebuildResponseMessage);
    +
    +    // Message sent to Worker.
    +    bus_->RegisterClientAsSender(shiftboss_client_id_, kRebuildWorkOrderMessage);
    +
    +    // Message sent to Foreman.
    +    bus_->RegisterClientAsSender(shiftboss_client_id_, kCatalogRelationNewBlockMessage);
    +    bus_->RegisterClientAsSender(shiftboss_client_id_, kDataPipelineMessage);
    +    bus_->RegisterClientAsSender(shiftboss_client_id_, kWorkOrdersAvailableMessage);
    +    bus_->RegisterClientAsSender(shiftboss_client_id_, kWorkOrderFeedbackMessage);
    +
    +    // Forward the following message types from Foreman to Workers.
    +    bus_->RegisterClientAsReceiver(shiftboss_client_id_, kWorkOrderMessage);
    +    bus_->RegisterClientAsSender(shiftboss_client_id_, kWorkOrderMessage);
    +
    +    // Forward the following message types from Workers to Foreman.
    +    bus_->RegisterClientAsReceiver(shiftboss_client_id_, kWorkOrderCompleteMessage);
    +    bus_->RegisterClientAsSender(shiftboss_client_id_, kWorkOrderCompleteMessage);
    +
    +    bus_->RegisterClientAsReceiver(shiftboss_client_id_, kRebuildWorkOrderCompleteMessage);
    +    bus_->RegisterClientAsSender(shiftboss_client_id_, kRebuildWorkOrderCompleteMessage);
    +
    +    bus_->RegisterClientAsReceiver(shiftboss_client_id_, kQueryResultRelationMessage);
    +    bus_->RegisterClientAsSender(shiftboss_client_id_, kQueryResultRelationResponseMessage);
    +
    +    // Stop itself.
    +    bus_->RegisterClientAsReceiver(shiftboss_client_id_, kPoisonMessage);
    +    // Stop all workers.
    +    bus_->RegisterClientAsSender(shiftboss_client_id_, kPoisonMessage);
    +
    +    for (std::size_t i = 0; i < workers_->getNumWorkers(); ++i) {
    +      worker_addresses_.AddRecipient(workers_->getClientID(i));
    +    }
    +
    +    registerWithForeman();
    +  }
    +
    +  ~Shiftboss() override {
    +  }
    +
    +  /**
    +   * @brief Get the TMB client ID of Shiftboss thread.
    +   *
    +   * @return TMB client ID of shiftboss thread.
    +   **/
    +  inline tmb::client_id getBusClientID() const {
    +    return shiftboss_client_id_;
    +  }
    +
    +  /**
    +   * @brief Get the Work Order processing capacity of all Workers managed by
    +   *        Shiftboss during a single round of WorkOrder dispatch.
    +   **/
    +  inline std::size_t getWorkOrderCapacity() const {
    +    DCHECK_NE(max_msgs_per_worker_, 0u);
    +    return max_msgs_per_worker_ * workers_->getNumWorkers();
    --- End diff --
    
    If I understand this right, you are assuming a node wide limit for number of work orders which can be dispatched (let's call it N). There's another parameter which is the upper limit on the maximum on how many work orders can be dispatched to an individual worker (let's call it n). Assuming w workers, ``N = n * w``. Wouldn't it be better if we just configure a ``N`` and let the shiftboss decide how it needs to set ``n``? We may want to tackle it in the future, but adding a note here should be helpful to remind ourselves. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #64: Introduced Shiftboss for the distribut...

Posted by pateljm <gi...@git.apache.org>.
Github user pateljm commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/64#discussion_r72268755
  
    --- Diff: query_execution/Shiftboss.hpp ---
    @@ -0,0 +1,238 @@
    +/**
    + *   Licensed under the Apache License, Version 2.0 (the "License");
    + *   you may not use this file except in compliance with the License.
    + *   You may obtain a copy of the License at
    + *
    + *       http://www.apache.org/licenses/LICENSE-2.0
    + *
    + *   Unless required by applicable law or agreed to in writing, software
    + *   distributed under the License is distributed on an "AS IS" BASIS,
    + *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + *   See the License for the specific language governing permissions and
    + *   limitations under the License.
    + **/
    +
    +#ifndef QUICKSTEP_QUERY_EXECUTION_SHIFTBOSS_HPP_
    +#define QUICKSTEP_QUERY_EXECUTION_SHIFTBOSS_HPP_
    +
    +#include <cstddef>
    +#include <memory>
    +
    +#include "catalog/CatalogDatabaseCache.hpp"
    +#include "catalog/CatalogTypedefs.hpp"
    +#include "query_execution/QueryContext.hpp"
    +#include "query_execution/QueryExecutionTypedefs.hpp"
    +#include "query_execution/WorkerDirectory.hpp"
    +#include "threading/Thread.hpp"
    +#include "utility/Macros.hpp"
    +
    +#include "glog/logging.h"
    +
    +#include "tmb/address.h"
    +#include "tmb/id_typedefs.h"
    +#include "tmb/message_bus.h"
    +
    +namespace quickstep {
    +
    +class StorageManager;
    +
    +namespace serialization {
    +class CatalogDatabase;
    +class QueryContext;
    +}  // namespace serialization
    +
    +/** \addtogroup QueryExecution
    + *  @{
    + */
    +
    +/**
    + * @brief The Shiftboss accepts workorder protos from shiftboss, and assigns
    + *        the workorders to workers.
    + **/
    +class Shiftboss : public Thread {
    + public:
    +  /**
    +   * @brief Constructor.
    +   *
    +   * @param bus A pointer to the TMB.
    +   * @param storage_manager The StorageManager to use.
    +   * @param workers A pointer to the WorkerDirectory.
    +   * @param cpu_id The ID of the CPU to which the Shiftboss thread can be pinned.
    +   *
    +   * @note If cpu_id is not specified, Shiftboss thread can be possibly moved
    +   *       around on different CPUs by the OS.
    +  **/
    +  Shiftboss(tmb::MessageBus *bus,
    +            StorageManager *storage_manager,
    +            WorkerDirectory *workers,
    +            const int cpu_id = -1)
    +      : bus_(DCHECK_NOTNULL(bus)),
    +        storage_manager_(DCHECK_NOTNULL(storage_manager)),
    +        workers_(DCHECK_NOTNULL(workers)),
    +        cpu_id_(cpu_id),
    +        shiftboss_client_id_(tmb::kClientIdNone),
    +        foreman_client_id_(tmb::kClientIdNone),
    +        max_msgs_per_worker_(1),
    +        start_worker_index_(0u) {
    +    // Check to have at least one Worker.
    +    DCHECK_GT(workers->getNumWorkers(), 0u);
    +
    +    shiftboss_client_id_ = bus_->Connect();
    +    LOG(INFO) << "Shiftboss TMB client ID: " << shiftboss_client_id_;
    +    DCHECK_NE(shiftboss_client_id_, tmb::kClientIdNone);
    +
    +    // Messages between Foreman and Shiftboss.
    +    bus_->RegisterClientAsSender(shiftboss_client_id_, kShiftbossRegistrationMessage);
    --- End diff --
    
    I'd suggest we put a note here in this PR to outline what the design is. That can be augmented in the next PR. Thanks!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #64: Introduced Shiftboss for the distribut...

Posted by zuyu <gi...@git.apache.org>.
Github user zuyu commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/64#discussion_r72499579
  
    --- Diff: query_execution/QueryExecutionTypedefs.hpp ---
    @@ -73,9 +73,17 @@ enum QueryExecutionMessageType : message_type_id {
       kPoisonMessage,  // From the main thread to Foreman and Workers.
     
     #ifdef QUICKSTEP_DISTRIBUTED
    +  kShiftbossRegistrationMessage,  // From Shiftboss to Foreman.
    +  kShiftbossRegistrationResponseMessage,  // From Foreman to Shiftboss.
    +  kQueryInitiateMessage,  // From Foreman to Shiftboss.
    +  kQueryInitiateResponseMessage,  // From Shiftboss to Foreman.
    +
       kInitiateRebuildMessage,  // From Foreman to Shiftboss.
       kInitiateRebuildResponseMessage,  // From Shiftboss to Foreman.
     
    +  kQueryResultRelationMessage,  // From Foreman to Shiftboss.
    +  kQueryResultRelationResponseMessage,  // From Shiftboss to Foreman.
    --- End diff --
    
    I think it would be addressed once `ForemanDistributed` is done.
    
    Note that `Shiftboss` is working with the old query execution part w/o concurrent queries support. And now I am in the process of merging the working code w/ concurrent queries support.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #64: Introduced Shiftboss for the distribut...

Posted by zuyu <gi...@git.apache.org>.
Github user zuyu commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/64#discussion_r72338203
  
    --- Diff: query_execution/Shiftboss.cpp ---
    @@ -0,0 +1,357 @@
    +/**
    + *   Licensed under the Apache License, Version 2.0 (the "License");
    + *   you may not use this file except in compliance with the License.
    + *   You may obtain a copy of the License at
    + *
    + *       http://www.apache.org/licenses/LICENSE-2.0
    + *
    + *   Unless required by applicable law or agreed to in writing, software
    + *   distributed under the License is distributed on an "AS IS" BASIS,
    + *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + *   See the License for the specific language governing permissions and
    + *   limitations under the License.
    + **/
    +
    +#include "query_execution/Shiftboss.hpp"
    +
    +#include <cstddef>
    +#include <cstdlib>
    +#include <memory>
    +#include <string>
    +#include <unordered_map>
    +#include <utility>
    +#include <vector>
    +
    +#include "catalog/CatalogTypedefs.hpp"
    +#include "query_execution/QueryContext.hpp"
    +#include "query_execution/QueryExecutionMessages.pb.h"
    +#include "query_execution/QueryExecutionTypedefs.hpp"
    +#include "query_execution/QueryExecutionUtil.hpp"
    +#include "query_execution/WorkerMessage.hpp"
    +#include "relational_operators/RebuildWorkOrder.hpp"
    +#include "relational_operators/WorkOrderFactory.hpp"
    +#include "storage/InsertDestination.hpp"
    +#include "storage/StorageBlock.hpp"
    +#include "storage/StorageBlockInfo.hpp"
    +#include "storage/StorageManager.hpp"
    +#include "threading/ThreadUtil.hpp"
    +
    +#include "glog/logging.h"
    +
    +#include "tmb/address.h"
    +#include "tmb/id_typedefs.h"
    +#include "tmb/message_bus.h"
    +#include "tmb/message_style.h"
    +#include "tmb/tagged_message.h"
    +
    +using std::free;
    +using std::malloc;
    +using std::move;
    +using std::size_t;
    +using std::string;
    +using std::unique_ptr;
    +using std::vector;
    +
    +using tmb::TaggedMessage;
    +
    +namespace quickstep {
    +
    +class WorkOrder;
    +
    +void Shiftboss::run() {
    +  if (cpu_id_ >= 0) {
    +    // We can pin the shiftboss thread to a CPU if specified.
    +    ThreadUtil::BindToCPU(cpu_id_);
    +  }
    +
    +  for (;;) {
    +    // Receive() is a blocking call, causing this thread to sleep until next
    +    // message is received.
    +    AnnotatedMessage annotated_message(bus_->Receive(shiftboss_client_id_, 0, true));
    +    LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +              << "') received the typed '" << annotated_message.tagged_message.message_type()
    +              << "' message from client " << annotated_message.sender;
    +    switch (annotated_message.tagged_message.message_type()) {
    +      case kShiftbossRegistrationResponseMessage: {
    +        foreman_client_id_ = annotated_message.sender;
    +        break;
    +      }
    +      case kShiftbossInitiateMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::ShiftbossInitiateMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processShiftbossInitiateMessage(proto.query_id(), proto.catalog_database_cache(), proto.query_context());
    +        break;
    +      }
    +      case kWorkOrderMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::WorkOrderMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        const std::size_t query_id = proto.query_id();
    +        DCHECK_EQ(1u, query_contexts_.count(query_id));
    +
    +        WorkOrder *work_order = WorkOrderFactory::ReconstructFromProto(proto.work_order(),
    +                                                                       &database_cache_,
    +                                                                       query_contexts_[query_id].get(),
    +                                                                       storage_manager_,
    +                                                                       shiftboss_client_id_,
    +                                                                       bus_);
    +
    +        unique_ptr<WorkerMessage> worker_message(
    +            WorkerMessage::WorkOrderMessage(work_order, proto.operator_index()));
    +
    +        TaggedMessage worker_tagged_message(worker_message.get(),
    +                                            sizeof(*worker_message),
    +                                            kWorkOrderMessage);
    +
    +        const size_t worker_index = getSchedulableWorker();
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded WorkOrderMessage (typed '" << kWorkOrderMessage
    +                  << "') from Foreman to worker " << worker_index;
    +
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           workers_->getClientID(worker_index),
    +                                           move(worker_tagged_message));
    +        break;
    +      }
    +      case kInitiateRebuildMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::InitiateRebuildMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processInitiateRebuildMessage(proto.query_id(),
    +                                      proto.operator_index(),
    +                                      proto.insert_destination_index(),
    +                                      proto.relation_id());
    +        break;
    +      }
    +      case kWorkOrderCompleteMessage:  // Fall through.
    +      case kRebuildWorkOrderCompleteMessage:
    --- End diff --
    
    1. No, but generated by each `Shiftboss`.
    2. Yes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #64: Introduced Shiftboss for the distribut...

Posted by zuyu <gi...@git.apache.org>.
Github user zuyu commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/64#discussion_r72532437
  
    --- Diff: query_execution/Shiftboss.cpp ---
    @@ -0,0 +1,357 @@
    +/**
    + *   Licensed under the Apache License, Version 2.0 (the "License");
    + *   you may not use this file except in compliance with the License.
    + *   You may obtain a copy of the License at
    + *
    + *       http://www.apache.org/licenses/LICENSE-2.0
    + *
    + *   Unless required by applicable law or agreed to in writing, software
    + *   distributed under the License is distributed on an "AS IS" BASIS,
    + *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + *   See the License for the specific language governing permissions and
    + *   limitations under the License.
    + **/
    +
    +#include "query_execution/Shiftboss.hpp"
    +
    +#include <cstddef>
    +#include <cstdlib>
    +#include <memory>
    +#include <string>
    +#include <unordered_map>
    +#include <utility>
    +#include <vector>
    +
    +#include "catalog/CatalogTypedefs.hpp"
    +#include "query_execution/QueryContext.hpp"
    +#include "query_execution/QueryExecutionMessages.pb.h"
    +#include "query_execution/QueryExecutionTypedefs.hpp"
    +#include "query_execution/QueryExecutionUtil.hpp"
    +#include "query_execution/WorkerMessage.hpp"
    +#include "relational_operators/RebuildWorkOrder.hpp"
    +#include "relational_operators/WorkOrderFactory.hpp"
    +#include "storage/InsertDestination.hpp"
    +#include "storage/StorageBlock.hpp"
    +#include "storage/StorageBlockInfo.hpp"
    +#include "storage/StorageManager.hpp"
    +#include "threading/ThreadUtil.hpp"
    +
    +#include "glog/logging.h"
    +
    +#include "tmb/address.h"
    +#include "tmb/id_typedefs.h"
    +#include "tmb/message_bus.h"
    +#include "tmb/message_style.h"
    +#include "tmb/tagged_message.h"
    +
    +using std::free;
    +using std::malloc;
    +using std::move;
    +using std::size_t;
    +using std::string;
    +using std::unique_ptr;
    +using std::vector;
    +
    +using tmb::TaggedMessage;
    +
    +namespace quickstep {
    +
    +class WorkOrder;
    +
    +void Shiftboss::run() {
    +  if (cpu_id_ >= 0) {
    +    // We can pin the shiftboss thread to a CPU if specified.
    +    ThreadUtil::BindToCPU(cpu_id_);
    +  }
    +
    +  for (;;) {
    +    // Receive() is a blocking call, causing this thread to sleep until next
    +    // message is received.
    +    AnnotatedMessage annotated_message(bus_->Receive(shiftboss_client_id_, 0, true));
    +    LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +              << "') received the typed '" << annotated_message.tagged_message.message_type()
    +              << "' message from client " << annotated_message.sender;
    +    switch (annotated_message.tagged_message.message_type()) {
    +      case kShiftbossRegistrationResponseMessage: {
    +        foreman_client_id_ = annotated_message.sender;
    +        break;
    +      }
    +      case kQueryInitiateMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::QueryInitiateMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processQueryInitiateMessage(proto.query_id(), proto.catalog_database_cache(), proto.query_context());
    +        break;
    +      }
    +      case kWorkOrderMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::WorkOrderMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        const std::size_t query_id = proto.query_id();
    +        DCHECK_EQ(1u, query_contexts_.count(query_id));
    +
    +        WorkOrder *work_order = WorkOrderFactory::ReconstructFromProto(proto.work_order(),
    +                                                                       &database_cache_,
    +                                                                       query_contexts_[query_id].get(),
    +                                                                       storage_manager_,
    +                                                                       shiftboss_client_id_,
    +                                                                       bus_);
    +
    +        unique_ptr<WorkerMessage> worker_message(
    +            WorkerMessage::WorkOrderMessage(work_order, proto.operator_index()));
    +
    +        TaggedMessage worker_tagged_message(worker_message.get(),
    +                                            sizeof(*worker_message),
    +                                            kWorkOrderMessage);
    +
    +        const size_t worker_index = getSchedulableWorker();
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded WorkOrderMessage (typed '" << kWorkOrderMessage
    +                  << "') from Foreman to worker " << worker_index;
    +
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           workers_->getClientID(worker_index),
    +                                           move(worker_tagged_message));
    +        break;
    +      }
    +      case kInitiateRebuildMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::InitiateRebuildMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processInitiateRebuildMessage(proto.query_id(),
    +                                      proto.operator_index(),
    +                                      proto.insert_destination_index(),
    +                                      proto.relation_id());
    +        break;
    +      }
    +      case kWorkOrderCompleteMessage:  // Fall through.
    +      case kRebuildWorkOrderCompleteMessage:
    +      case kDataPipelineMessage:
    +      case kWorkOrdersAvailableMessage:
    +      case kWorkOrderFeedbackMessage: {
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded typed '" << annotated_message.tagged_message.message_type()
    +                  << "' message from worker (client " << annotated_message.sender << ") to Foreman";
    +
    +        DCHECK_NE(foreman_client_id_, tmb::kClientIdNone);
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           foreman_client_id_,
    +                                           move(annotated_message.tagged_message));
    +        break;
    +      }
    +      case kQueryResultRelationMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::QueryResultRelationMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        for (int i = 0; i < proto.blocks_size(); ++i) {
    +          const block_id block = proto.blocks(i);
    --- End diff --
    
    FYI, there is no need for `TODO` in the `Shiftboss` side.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #64: Introduced Shiftboss for the distribut...

Posted by zuyu <gi...@git.apache.org>.
Github user zuyu commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/64#discussion_r72506593
  
    --- Diff: query_execution/Shiftboss.cpp ---
    @@ -0,0 +1,357 @@
    +/**
    + *   Licensed under the Apache License, Version 2.0 (the "License");
    + *   you may not use this file except in compliance with the License.
    + *   You may obtain a copy of the License at
    + *
    + *       http://www.apache.org/licenses/LICENSE-2.0
    + *
    + *   Unless required by applicable law or agreed to in writing, software
    + *   distributed under the License is distributed on an "AS IS" BASIS,
    + *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + *   See the License for the specific language governing permissions and
    + *   limitations under the License.
    + **/
    +
    +#include "query_execution/Shiftboss.hpp"
    +
    +#include <cstddef>
    +#include <cstdlib>
    +#include <memory>
    +#include <string>
    +#include <unordered_map>
    +#include <utility>
    +#include <vector>
    +
    +#include "catalog/CatalogTypedefs.hpp"
    +#include "query_execution/QueryContext.hpp"
    +#include "query_execution/QueryExecutionMessages.pb.h"
    +#include "query_execution/QueryExecutionTypedefs.hpp"
    +#include "query_execution/QueryExecutionUtil.hpp"
    +#include "query_execution/WorkerMessage.hpp"
    +#include "relational_operators/RebuildWorkOrder.hpp"
    +#include "relational_operators/WorkOrderFactory.hpp"
    +#include "storage/InsertDestination.hpp"
    +#include "storage/StorageBlock.hpp"
    +#include "storage/StorageBlockInfo.hpp"
    +#include "storage/StorageManager.hpp"
    +#include "threading/ThreadUtil.hpp"
    +
    +#include "glog/logging.h"
    +
    +#include "tmb/address.h"
    +#include "tmb/id_typedefs.h"
    +#include "tmb/message_bus.h"
    +#include "tmb/message_style.h"
    +#include "tmb/tagged_message.h"
    +
    +using std::free;
    +using std::malloc;
    +using std::move;
    +using std::size_t;
    +using std::string;
    +using std::unique_ptr;
    +using std::vector;
    +
    +using tmb::TaggedMessage;
    +
    +namespace quickstep {
    +
    +class WorkOrder;
    +
    +void Shiftboss::run() {
    +  if (cpu_id_ >= 0) {
    +    // We can pin the shiftboss thread to a CPU if specified.
    +    ThreadUtil::BindToCPU(cpu_id_);
    +  }
    +
    +  for (;;) {
    +    // Receive() is a blocking call, causing this thread to sleep until next
    +    // message is received.
    +    AnnotatedMessage annotated_message(bus_->Receive(shiftboss_client_id_, 0, true));
    +    LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +              << "') received the typed '" << annotated_message.tagged_message.message_type()
    +              << "' message from client " << annotated_message.sender;
    +    switch (annotated_message.tagged_message.message_type()) {
    +      case kShiftbossRegistrationResponseMessage: {
    +        foreman_client_id_ = annotated_message.sender;
    +        break;
    +      }
    +      case kQueryInitiateMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::QueryInitiateMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processQueryInitiateMessage(proto.query_id(), proto.catalog_database_cache(), proto.query_context());
    +        break;
    +      }
    +      case kWorkOrderMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::WorkOrderMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        const std::size_t query_id = proto.query_id();
    +        DCHECK_EQ(1u, query_contexts_.count(query_id));
    +
    +        WorkOrder *work_order = WorkOrderFactory::ReconstructFromProto(proto.work_order(),
    +                                                                       &database_cache_,
    +                                                                       query_contexts_[query_id].get(),
    +                                                                       storage_manager_,
    +                                                                       shiftboss_client_id_,
    +                                                                       bus_);
    +
    +        unique_ptr<WorkerMessage> worker_message(
    +            WorkerMessage::WorkOrderMessage(work_order, proto.operator_index()));
    +
    +        TaggedMessage worker_tagged_message(worker_message.get(),
    +                                            sizeof(*worker_message),
    +                                            kWorkOrderMessage);
    +
    +        const size_t worker_index = getSchedulableWorker();
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded WorkOrderMessage (typed '" << kWorkOrderMessage
    +                  << "') from Foreman to worker " << worker_index;
    +
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           workers_->getClientID(worker_index),
    +                                           move(worker_tagged_message));
    +        break;
    +      }
    +      case kInitiateRebuildMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::InitiateRebuildMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processInitiateRebuildMessage(proto.query_id(),
    +                                      proto.operator_index(),
    +                                      proto.insert_destination_index(),
    +                                      proto.relation_id());
    +        break;
    +      }
    +      case kWorkOrderCompleteMessage:  // Fall through.
    +      case kRebuildWorkOrderCompleteMessage:
    +      case kDataPipelineMessage:
    +      case kWorkOrdersAvailableMessage:
    +      case kWorkOrderFeedbackMessage: {
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded typed '" << annotated_message.tagged_message.message_type()
    +                  << "' message from worker (client " << annotated_message.sender << ") to Foreman";
    +
    +        DCHECK_NE(foreman_client_id_, tmb::kClientIdNone);
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           foreman_client_id_,
    +                                           move(annotated_message.tagged_message));
    +        break;
    +      }
    +      case kQueryResultRelationMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::QueryResultRelationMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        for (int i = 0; i < proto.blocks_size(); ++i) {
    +          const block_id block = proto.blocks(i);
    --- End diff --
    
    Note that the code works in the single `Worker` w/o concurrent query support, so things may change.
    
    1. [`ForemanDistributed`](https://github.com/apache/incubator-quickstep/blob/distributed-exe-gen-test/query_execution/ForemanDistributed.cpp#L368) knows the query result relation from `QueryHandle`\uff0c which contains the blocks info.
    
    2. For now, no. And we should refactor later.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #64: Introduced Shiftboss for the distribut...

Posted by zuyu <gi...@git.apache.org>.
Github user zuyu commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/64#discussion_r72501365
  
    --- Diff: query_execution/Shiftboss.cpp ---
    @@ -0,0 +1,357 @@
    +/**
    + *   Licensed under the Apache License, Version 2.0 (the "License");
    + *   you may not use this file except in compliance with the License.
    + *   You may obtain a copy of the License at
    + *
    + *       http://www.apache.org/licenses/LICENSE-2.0
    + *
    + *   Unless required by applicable law or agreed to in writing, software
    + *   distributed under the License is distributed on an "AS IS" BASIS,
    + *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + *   See the License for the specific language governing permissions and
    + *   limitations under the License.
    + **/
    +
    +#include "query_execution/Shiftboss.hpp"
    +
    +#include <cstddef>
    +#include <cstdlib>
    +#include <memory>
    +#include <string>
    +#include <unordered_map>
    +#include <utility>
    +#include <vector>
    +
    +#include "catalog/CatalogTypedefs.hpp"
    +#include "query_execution/QueryContext.hpp"
    +#include "query_execution/QueryExecutionMessages.pb.h"
    +#include "query_execution/QueryExecutionTypedefs.hpp"
    +#include "query_execution/QueryExecutionUtil.hpp"
    +#include "query_execution/WorkerMessage.hpp"
    +#include "relational_operators/RebuildWorkOrder.hpp"
    +#include "relational_operators/WorkOrderFactory.hpp"
    +#include "storage/InsertDestination.hpp"
    +#include "storage/StorageBlock.hpp"
    +#include "storage/StorageBlockInfo.hpp"
    +#include "storage/StorageManager.hpp"
    +#include "threading/ThreadUtil.hpp"
    +
    +#include "glog/logging.h"
    +
    +#include "tmb/address.h"
    +#include "tmb/id_typedefs.h"
    +#include "tmb/message_bus.h"
    +#include "tmb/message_style.h"
    +#include "tmb/tagged_message.h"
    +
    +using std::free;
    +using std::malloc;
    +using std::move;
    +using std::size_t;
    +using std::string;
    +using std::unique_ptr;
    +using std::vector;
    +
    +using tmb::TaggedMessage;
    +
    +namespace quickstep {
    +
    +class WorkOrder;
    +
    +void Shiftboss::run() {
    +  if (cpu_id_ >= 0) {
    +    // We can pin the shiftboss thread to a CPU if specified.
    +    ThreadUtil::BindToCPU(cpu_id_);
    +  }
    +
    +  for (;;) {
    +    // Receive() is a blocking call, causing this thread to sleep until next
    +    // message is received.
    +    AnnotatedMessage annotated_message(bus_->Receive(shiftboss_client_id_, 0, true));
    +    LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +              << "') received the typed '" << annotated_message.tagged_message.message_type()
    +              << "' message from client " << annotated_message.sender;
    +    switch (annotated_message.tagged_message.message_type()) {
    +      case kShiftbossRegistrationResponseMessage: {
    +        foreman_client_id_ = annotated_message.sender;
    +        break;
    +      }
    +      case kShiftbossInitiateMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::ShiftbossInitiateMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processShiftbossInitiateMessage(proto.query_id(), proto.catalog_database_cache(), proto.query_context());
    +        break;
    +      }
    +      case kWorkOrderMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::WorkOrderMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        const std::size_t query_id = proto.query_id();
    +        DCHECK_EQ(1u, query_contexts_.count(query_id));
    +
    +        WorkOrder *work_order = WorkOrderFactory::ReconstructFromProto(proto.work_order(),
    +                                                                       &database_cache_,
    +                                                                       query_contexts_[query_id].get(),
    +                                                                       storage_manager_,
    +                                                                       shiftboss_client_id_,
    +                                                                       bus_);
    +
    +        unique_ptr<WorkerMessage> worker_message(
    +            WorkerMessage::WorkOrderMessage(work_order, proto.operator_index()));
    +
    +        TaggedMessage worker_tagged_message(worker_message.get(),
    +                                            sizeof(*worker_message),
    +                                            kWorkOrderMessage);
    +
    +        const size_t worker_index = getSchedulableWorker();
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded WorkOrderMessage (typed '" << kWorkOrderMessage
    +                  << "') from Foreman to worker " << worker_index;
    +
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           workers_->getClientID(worker_index),
    +                                           move(worker_tagged_message));
    +        break;
    +      }
    +      case kInitiateRebuildMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::InitiateRebuildMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processInitiateRebuildMessage(proto.query_id(),
    +                                      proto.operator_index(),
    +                                      proto.insert_destination_index(),
    +                                      proto.relation_id());
    +        break;
    +      }
    +      case kWorkOrderCompleteMessage:  // Fall through.
    +      case kRebuildWorkOrderCompleteMessage:
    +      case kDataPipelineMessage:
    +      case kWorkOrdersAvailableMessage:
    +      case kWorkOrderFeedbackMessage: {
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded typed '" << annotated_message.tagged_message.message_type()
    +                  << "' message from worker (client " << annotated_message.sender << ") to Foreman";
    +
    +        DCHECK_NE(foreman_client_id_, tmb::kClientIdNone);
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           foreman_client_id_,
    +                                           move(annotated_message.tagged_message));
    +        break;
    +      }
    +      case kQueryResultRelationMessage: {
    --- End diff --
    
    Yes, I admitted that we would have a better name, and I'll fix it when adding `ForemanDistributed` in the next PR. Thanks!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #64: Introduced Shiftboss for the distribut...

Posted by zuyu <gi...@git.apache.org>.
Github user zuyu commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/64#discussion_r72500784
  
    --- Diff: query_execution/Shiftboss.cpp ---
    @@ -0,0 +1,357 @@
    +/**
    + *   Licensed under the Apache License, Version 2.0 (the "License");
    + *   you may not use this file except in compliance with the License.
    + *   You may obtain a copy of the License at
    + *
    + *       http://www.apache.org/licenses/LICENSE-2.0
    + *
    + *   Unless required by applicable law or agreed to in writing, software
    + *   distributed under the License is distributed on an "AS IS" BASIS,
    + *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + *   See the License for the specific language governing permissions and
    + *   limitations under the License.
    + **/
    +
    +#include "query_execution/Shiftboss.hpp"
    +
    +#include <cstddef>
    +#include <cstdlib>
    +#include <memory>
    +#include <string>
    +#include <unordered_map>
    +#include <utility>
    +#include <vector>
    +
    +#include "catalog/CatalogTypedefs.hpp"
    +#include "query_execution/QueryContext.hpp"
    +#include "query_execution/QueryExecutionMessages.pb.h"
    +#include "query_execution/QueryExecutionTypedefs.hpp"
    +#include "query_execution/QueryExecutionUtil.hpp"
    +#include "query_execution/WorkerMessage.hpp"
    +#include "relational_operators/RebuildWorkOrder.hpp"
    +#include "relational_operators/WorkOrderFactory.hpp"
    +#include "storage/InsertDestination.hpp"
    +#include "storage/StorageBlock.hpp"
    +#include "storage/StorageBlockInfo.hpp"
    +#include "storage/StorageManager.hpp"
    +#include "threading/ThreadUtil.hpp"
    +
    +#include "glog/logging.h"
    +
    +#include "tmb/address.h"
    +#include "tmb/id_typedefs.h"
    +#include "tmb/message_bus.h"
    +#include "tmb/message_style.h"
    +#include "tmb/tagged_message.h"
    +
    +using std::free;
    +using std::malloc;
    +using std::move;
    +using std::size_t;
    +using std::string;
    +using std::unique_ptr;
    +using std::vector;
    +
    +using tmb::TaggedMessage;
    +
    +namespace quickstep {
    +
    +class WorkOrder;
    +
    +void Shiftboss::run() {
    +  if (cpu_id_ >= 0) {
    +    // We can pin the shiftboss thread to a CPU if specified.
    +    ThreadUtil::BindToCPU(cpu_id_);
    +  }
    +
    +  for (;;) {
    +    // Receive() is a blocking call, causing this thread to sleep until next
    +    // message is received.
    +    AnnotatedMessage annotated_message(bus_->Receive(shiftboss_client_id_, 0, true));
    +    LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +              << "') received the typed '" << annotated_message.tagged_message.message_type()
    +              << "' message from client " << annotated_message.sender;
    +    switch (annotated_message.tagged_message.message_type()) {
    +      case kShiftbossRegistrationResponseMessage: {
    +        foreman_client_id_ = annotated_message.sender;
    +        break;
    +      }
    +      case kShiftbossInitiateMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::ShiftbossInitiateMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processShiftbossInitiateMessage(proto.query_id(), proto.catalog_database_cache(), proto.query_context());
    +        break;
    +      }
    +      case kWorkOrderMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::WorkOrderMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        const std::size_t query_id = proto.query_id();
    +        DCHECK_EQ(1u, query_contexts_.count(query_id));
    +
    +        WorkOrder *work_order = WorkOrderFactory::ReconstructFromProto(proto.work_order(),
    +                                                                       &database_cache_,
    +                                                                       query_contexts_[query_id].get(),
    +                                                                       storage_manager_,
    +                                                                       shiftboss_client_id_,
    +                                                                       bus_);
    +
    +        unique_ptr<WorkerMessage> worker_message(
    +            WorkerMessage::WorkOrderMessage(work_order, proto.operator_index()));
    +
    +        TaggedMessage worker_tagged_message(worker_message.get(),
    +                                            sizeof(*worker_message),
    +                                            kWorkOrderMessage);
    +
    +        const size_t worker_index = getSchedulableWorker();
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded WorkOrderMessage (typed '" << kWorkOrderMessage
    +                  << "') from Foreman to worker " << worker_index;
    +
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           workers_->getClientID(worker_index),
    +                                           move(worker_tagged_message));
    +        break;
    +      }
    +      case kInitiateRebuildMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::InitiateRebuildMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processInitiateRebuildMessage(proto.query_id(),
    +                                      proto.operator_index(),
    +                                      proto.insert_destination_index(),
    +                                      proto.relation_id());
    +        break;
    +      }
    +      case kWorkOrderCompleteMessage:  // Fall through.
    +      case kRebuildWorkOrderCompleteMessage:
    +      case kDataPipelineMessage:
    +      case kWorkOrdersAvailableMessage:
    +      case kWorkOrderFeedbackMessage: {
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded typed '" << annotated_message.tagged_message.message_type()
    +                  << "' message from worker (client " << annotated_message.sender << ") to Foreman";
    +
    +        DCHECK_NE(foreman_client_id_, tmb::kClientIdNone);
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           foreman_client_id_,
    +                                           move(annotated_message.tagged_message));
    +        break;
    +      }
    +      case kQueryResultRelationMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::QueryResultRelationMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        for (int i = 0; i < proto.blocks_size(); ++i) {
    +          const block_id block = proto.blocks(i);
    +          storage_manager_->saveBlockOrBlob(block);
    +          if (storage_manager_->blockOrBlobIsLoaded(block)) {
    +            // NOTE(zuyu): eviction is required to avoid accesses to the query
    +            // result relation schema in CatalogDatabaseCache, for all query
    +            // optimizer execution generator unit tests and the single-process
    +            // Quickstep CLI.
    +            storage_manager_->evictBlockOrBlob(block);
    --- End diff --
    
    What's your main comments again? There are many comments above.
    
    The goal here is to remove the query result blocks from the buffer pool in the Worker site. which is remote to the distributed CLI.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #64: Introduced Shiftboss for the distribut...

Posted by hbdeshmukh <gi...@git.apache.org>.
Github user hbdeshmukh commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/64#discussion_r72505141
  
    --- Diff: query_execution/Shiftboss.cpp ---
    @@ -0,0 +1,357 @@
    +/**
    + *   Licensed under the Apache License, Version 2.0 (the "License");
    + *   you may not use this file except in compliance with the License.
    + *   You may obtain a copy of the License at
    + *
    + *       http://www.apache.org/licenses/LICENSE-2.0
    + *
    + *   Unless required by applicable law or agreed to in writing, software
    + *   distributed under the License is distributed on an "AS IS" BASIS,
    + *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + *   See the License for the specific language governing permissions and
    + *   limitations under the License.
    + **/
    +
    +#include "query_execution/Shiftboss.hpp"
    +
    +#include <cstddef>
    +#include <cstdlib>
    +#include <memory>
    +#include <string>
    +#include <unordered_map>
    +#include <utility>
    +#include <vector>
    +
    +#include "catalog/CatalogTypedefs.hpp"
    +#include "query_execution/QueryContext.hpp"
    +#include "query_execution/QueryExecutionMessages.pb.h"
    +#include "query_execution/QueryExecutionTypedefs.hpp"
    +#include "query_execution/QueryExecutionUtil.hpp"
    +#include "query_execution/WorkerMessage.hpp"
    +#include "relational_operators/RebuildWorkOrder.hpp"
    +#include "relational_operators/WorkOrderFactory.hpp"
    +#include "storage/InsertDestination.hpp"
    +#include "storage/StorageBlock.hpp"
    +#include "storage/StorageBlockInfo.hpp"
    +#include "storage/StorageManager.hpp"
    +#include "threading/ThreadUtil.hpp"
    +
    +#include "glog/logging.h"
    +
    +#include "tmb/address.h"
    +#include "tmb/id_typedefs.h"
    +#include "tmb/message_bus.h"
    +#include "tmb/message_style.h"
    +#include "tmb/tagged_message.h"
    +
    +using std::free;
    +using std::malloc;
    +using std::move;
    +using std::size_t;
    +using std::string;
    +using std::unique_ptr;
    +using std::vector;
    +
    +using tmb::TaggedMessage;
    +
    +namespace quickstep {
    +
    +class WorkOrder;
    +
    +void Shiftboss::run() {
    +  if (cpu_id_ >= 0) {
    +    // We can pin the shiftboss thread to a CPU if specified.
    +    ThreadUtil::BindToCPU(cpu_id_);
    +  }
    +
    +  for (;;) {
    +    // Receive() is a blocking call, causing this thread to sleep until next
    +    // message is received.
    +    AnnotatedMessage annotated_message(bus_->Receive(shiftboss_client_id_, 0, true));
    +    LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +              << "') received the typed '" << annotated_message.tagged_message.message_type()
    +              << "' message from client " << annotated_message.sender;
    +    switch (annotated_message.tagged_message.message_type()) {
    +      case kShiftbossRegistrationResponseMessage: {
    +        foreman_client_id_ = annotated_message.sender;
    +        break;
    +      }
    +      case kQueryInitiateMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::QueryInitiateMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processQueryInitiateMessage(proto.query_id(), proto.catalog_database_cache(), proto.query_context());
    +        break;
    +      }
    +      case kWorkOrderMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::WorkOrderMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        const std::size_t query_id = proto.query_id();
    +        DCHECK_EQ(1u, query_contexts_.count(query_id));
    +
    +        WorkOrder *work_order = WorkOrderFactory::ReconstructFromProto(proto.work_order(),
    +                                                                       &database_cache_,
    +                                                                       query_contexts_[query_id].get(),
    +                                                                       storage_manager_,
    +                                                                       shiftboss_client_id_,
    +                                                                       bus_);
    +
    +        unique_ptr<WorkerMessage> worker_message(
    +            WorkerMessage::WorkOrderMessage(work_order, proto.operator_index()));
    +
    +        TaggedMessage worker_tagged_message(worker_message.get(),
    +                                            sizeof(*worker_message),
    +                                            kWorkOrderMessage);
    +
    +        const size_t worker_index = getSchedulableWorker();
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded WorkOrderMessage (typed '" << kWorkOrderMessage
    +                  << "') from Foreman to worker " << worker_index;
    +
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           workers_->getClientID(worker_index),
    +                                           move(worker_tagged_message));
    +        break;
    +      }
    +      case kInitiateRebuildMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::InitiateRebuildMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processInitiateRebuildMessage(proto.query_id(),
    +                                      proto.operator_index(),
    +                                      proto.insert_destination_index(),
    +                                      proto.relation_id());
    +        break;
    +      }
    +      case kWorkOrderCompleteMessage:  // Fall through.
    +      case kRebuildWorkOrderCompleteMessage:
    +      case kDataPipelineMessage:
    +      case kWorkOrdersAvailableMessage:
    +      case kWorkOrderFeedbackMessage: {
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded typed '" << annotated_message.tagged_message.message_type()
    +                  << "' message from worker (client " << annotated_message.sender << ") to Foreman";
    +
    +        DCHECK_NE(foreman_client_id_, tmb::kClientIdNone);
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           foreman_client_id_,
    +                                           move(annotated_message.tagged_message));
    +        break;
    +      }
    +      case kQueryResultRelationMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::QueryResultRelationMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        for (int i = 0; i < proto.blocks_size(); ++i) {
    +          const block_id block = proto.blocks(i);
    --- End diff --
    
    The Foreman gives the ShiftBoss a list of block IDs belonging to the result relation, which the Shiftboss flushes to the disk, right? Two questions in this regard:
    
    1. How does the Foreman get the list of block IDs?
    2. Does the Foreman make sure that the block IDs sent to Shiftboss 1 has blocks belonging to Node 1 only?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #64: Introduced Shiftboss for the distribut...

Posted by zuyu <gi...@git.apache.org>.
Github user zuyu commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/64#discussion_r72137869
  
    --- Diff: query_execution/Shiftboss.hpp ---
    @@ -0,0 +1,238 @@
    +/**
    + *   Licensed under the Apache License, Version 2.0 (the "License");
    + *   you may not use this file except in compliance with the License.
    + *   You may obtain a copy of the License at
    + *
    + *       http://www.apache.org/licenses/LICENSE-2.0
    + *
    + *   Unless required by applicable law or agreed to in writing, software
    + *   distributed under the License is distributed on an "AS IS" BASIS,
    + *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + *   See the License for the specific language governing permissions and
    + *   limitations under the License.
    + **/
    +
    +#ifndef QUICKSTEP_QUERY_EXECUTION_SHIFTBOSS_HPP_
    +#define QUICKSTEP_QUERY_EXECUTION_SHIFTBOSS_HPP_
    +
    +#include <cstddef>
    +#include <memory>
    +
    +#include "catalog/CatalogDatabaseCache.hpp"
    +#include "catalog/CatalogTypedefs.hpp"
    +#include "query_execution/QueryContext.hpp"
    +#include "query_execution/QueryExecutionTypedefs.hpp"
    +#include "query_execution/WorkerDirectory.hpp"
    +#include "threading/Thread.hpp"
    +#include "utility/Macros.hpp"
    +
    +#include "glog/logging.h"
    +
    +#include "tmb/address.h"
    +#include "tmb/id_typedefs.h"
    +#include "tmb/message_bus.h"
    +
    +namespace quickstep {
    +
    +class StorageManager;
    +
    +namespace serialization {
    +class CatalogDatabase;
    +class QueryContext;
    +}  // namespace serialization
    +
    +/** \addtogroup QueryExecution
    + *  @{
    + */
    +
    +/**
    + * @brief The Shiftboss accepts workorder protos from shiftboss, and assigns
    + *        the workorders to workers.
    + **/
    +class Shiftboss : public Thread {
    + public:
    +  /**
    +   * @brief Constructor.
    +   *
    +   * @param bus A pointer to the TMB.
    +   * @param storage_manager The StorageManager to use.
    +   * @param workers A pointer to the WorkerDirectory.
    +   * @param cpu_id The ID of the CPU to which the Shiftboss thread can be pinned.
    +   *
    +   * @note If cpu_id is not specified, Shiftboss thread can be possibly moved
    +   *       around on different CPUs by the OS.
    +  **/
    +  Shiftboss(tmb::MessageBus *bus,
    +            StorageManager *storage_manager,
    +            WorkerDirectory *workers,
    +            const int cpu_id = -1)
    +      : bus_(DCHECK_NOTNULL(bus)),
    +        storage_manager_(DCHECK_NOTNULL(storage_manager)),
    +        workers_(DCHECK_NOTNULL(workers)),
    +        cpu_id_(cpu_id),
    +        shiftboss_client_id_(tmb::kClientIdNone),
    +        foreman_client_id_(tmb::kClientIdNone),
    +        max_msgs_per_worker_(1),
    +        start_worker_index_(0u) {
    +    // Check to have at least one Worker.
    +    DCHECK_GT(workers->getNumWorkers(), 0u);
    +
    +    shiftboss_client_id_ = bus_->Connect();
    +    LOG(INFO) << "Shiftboss TMB client ID: " << shiftboss_client_id_;
    +    DCHECK_NE(shiftboss_client_id_, tmb::kClientIdNone);
    +
    +    // Messages between Foreman and Shiftboss.
    +    bus_->RegisterClientAsSender(shiftboss_client_id_, kShiftbossRegistrationMessage);
    --- End diff --
    
    Not in this PR. Once we use two buses, the names make sense.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #64: Introduced Shiftboss for the distribut...

Posted by zuyu <gi...@git.apache.org>.
Github user zuyu commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/64#discussion_r72339210
  
    --- Diff: query_execution/Shiftboss.cpp ---
    @@ -0,0 +1,357 @@
    +/**
    + *   Licensed under the Apache License, Version 2.0 (the "License");
    + *   you may not use this file except in compliance with the License.
    + *   You may obtain a copy of the License at
    + *
    + *       http://www.apache.org/licenses/LICENSE-2.0
    + *
    + *   Unless required by applicable law or agreed to in writing, software
    + *   distributed under the License is distributed on an "AS IS" BASIS,
    + *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + *   See the License for the specific language governing permissions and
    + *   limitations under the License.
    + **/
    +
    +#include "query_execution/Shiftboss.hpp"
    +
    +#include <cstddef>
    +#include <cstdlib>
    +#include <memory>
    +#include <string>
    +#include <unordered_map>
    +#include <utility>
    +#include <vector>
    +
    +#include "catalog/CatalogTypedefs.hpp"
    +#include "query_execution/QueryContext.hpp"
    +#include "query_execution/QueryExecutionMessages.pb.h"
    +#include "query_execution/QueryExecutionTypedefs.hpp"
    +#include "query_execution/QueryExecutionUtil.hpp"
    +#include "query_execution/WorkerMessage.hpp"
    +#include "relational_operators/RebuildWorkOrder.hpp"
    +#include "relational_operators/WorkOrderFactory.hpp"
    +#include "storage/InsertDestination.hpp"
    +#include "storage/StorageBlock.hpp"
    +#include "storage/StorageBlockInfo.hpp"
    +#include "storage/StorageManager.hpp"
    +#include "threading/ThreadUtil.hpp"
    +
    +#include "glog/logging.h"
    +
    +#include "tmb/address.h"
    +#include "tmb/id_typedefs.h"
    +#include "tmb/message_bus.h"
    +#include "tmb/message_style.h"
    +#include "tmb/tagged_message.h"
    +
    +using std::free;
    +using std::malloc;
    +using std::move;
    +using std::size_t;
    +using std::string;
    +using std::unique_ptr;
    +using std::vector;
    +
    +using tmb::TaggedMessage;
    +
    +namespace quickstep {
    +
    +class WorkOrder;
    +
    +void Shiftboss::run() {
    +  if (cpu_id_ >= 0) {
    +    // We can pin the shiftboss thread to a CPU if specified.
    +    ThreadUtil::BindToCPU(cpu_id_);
    +  }
    +
    +  for (;;) {
    +    // Receive() is a blocking call, causing this thread to sleep until next
    +    // message is received.
    +    AnnotatedMessage annotated_message(bus_->Receive(shiftboss_client_id_, 0, true));
    +    LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +              << "') received the typed '" << annotated_message.tagged_message.message_type()
    +              << "' message from client " << annotated_message.sender;
    +    switch (annotated_message.tagged_message.message_type()) {
    +      case kShiftbossRegistrationResponseMessage: {
    +        foreman_client_id_ = annotated_message.sender;
    +        break;
    +      }
    +      case kShiftbossInitiateMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::ShiftbossInitiateMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processShiftbossInitiateMessage(proto.query_id(), proto.catalog_database_cache(), proto.query_context());
    +        break;
    +      }
    +      case kWorkOrderMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::WorkOrderMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        const std::size_t query_id = proto.query_id();
    +        DCHECK_EQ(1u, query_contexts_.count(query_id));
    +
    +        WorkOrder *work_order = WorkOrderFactory::ReconstructFromProto(proto.work_order(),
    +                                                                       &database_cache_,
    +                                                                       query_contexts_[query_id].get(),
    +                                                                       storage_manager_,
    +                                                                       shiftboss_client_id_,
    +                                                                       bus_);
    +
    +        unique_ptr<WorkerMessage> worker_message(
    +            WorkerMessage::WorkOrderMessage(work_order, proto.operator_index()));
    +
    +        TaggedMessage worker_tagged_message(worker_message.get(),
    +                                            sizeof(*worker_message),
    +                                            kWorkOrderMessage);
    +
    +        const size_t worker_index = getSchedulableWorker();
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded WorkOrderMessage (typed '" << kWorkOrderMessage
    +                  << "') from Foreman to worker " << worker_index;
    +
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           workers_->getClientID(worker_index),
    +                                           move(worker_tagged_message));
    +        break;
    +      }
    +      case kInitiateRebuildMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::InitiateRebuildMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processInitiateRebuildMessage(proto.query_id(),
    +                                      proto.operator_index(),
    +                                      proto.insert_destination_index(),
    +                                      proto.relation_id());
    +        break;
    +      }
    +      case kWorkOrderCompleteMessage:  // Fall through.
    +      case kRebuildWorkOrderCompleteMessage:
    +      case kDataPipelineMessage:
    +      case kWorkOrdersAvailableMessage:
    +      case kWorkOrderFeedbackMessage: {
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded typed '" << annotated_message.tagged_message.message_type()
    +                  << "' message from worker (client " << annotated_message.sender << ") to Foreman";
    +
    +        DCHECK_NE(foreman_client_id_, tmb::kClientIdNone);
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           foreman_client_id_,
    +                                           move(annotated_message.tagged_message));
    +        break;
    +      }
    +      case kQueryResultRelationMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::QueryResultRelationMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        for (int i = 0; i < proto.blocks_size(); ++i) {
    +          const block_id block = proto.blocks(i);
    +          storage_manager_->saveBlockOrBlob(block);
    +          if (storage_manager_->blockOrBlobIsLoaded(block)) {
    +            // NOTE(zuyu): eviction is required to avoid accesses to the query
    +            // result relation schema in CatalogDatabaseCache, for all query
    +            // optimizer execution generator unit tests and the single-process
    +            // Quickstep CLI.
    +            storage_manager_->evictBlockOrBlob(block);
    +          }
    +        }
    +
    +        serialization::QueryResultRelationResponseMessage ack_proto;
    +        ack_proto.set_relation_id(proto.relation_id());
    +
    +        const size_t ack_proto_length = ack_proto.ByteSize();
    +        char *ack_proto_bytes = static_cast<char*>(malloc(ack_proto_length));
    +        CHECK(ack_proto.SerializeToArray(ack_proto_bytes, ack_proto_length));
    +
    +        TaggedMessage ack_message(static_cast<const void*>(ack_proto_bytes),
    +                                  ack_proto_length,
    +                                  kQueryResultRelationResponseMessage);
    +        free(ack_proto_bytes);
    +
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') sent QueryResultRelationResponseMessage (typed '" << kQueryResultRelationResponseMessage
    +                  << ") to Foreman";
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           foreman_client_id_,
    +                                           move(ack_message));
    +        break;
    +      }
    +      case kPoisonMessage: {
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded PoisonMessage (typed '" << kPoisonMessage
    +                  << "') from Foreman to all workers";
    +
    +        tmb::MessageStyle broadcast_style;
    +        broadcast_style.Broadcast(true);
    +
    +        tmb::MessageBus::SendStatus send_status =
    +            bus_->Send(shiftboss_client_id_,
    +                       worker_addresses_,
    +                       broadcast_style,
    +                       move(annotated_message.tagged_message));
    +        DCHECK(send_status == tmb::MessageBus::SendStatus::kOK);
    +        return;
    +      }
    +      default: {
    +        LOG(FATAL) << "Unknown TMB message type";
    +      }
    +    }
    +  }
    +}
    +
    +size_t Shiftboss::getSchedulableWorker() {
    +  const size_t num_workers = workers_->getNumWorkers();
    +
    +  size_t curr_worker = start_worker_index_;
    +  for (;;) {
    +    if (workers_->getNumQueuedWorkOrders(curr_worker) < max_msgs_per_worker_) {
    +      start_worker_index_ = (curr_worker + 1) % num_workers;
    +      // TODO(zuyu): workers_->incrementNumQueuedWorkOrders(curr_worker);
    +      // But we need a WorkOrder queue first.
    +      return curr_worker;
    +    }
    +
    +    curr_worker = (curr_worker + 1) % num_workers;
    +  }
    +}
    +
    +void Shiftboss::registerWithForeman() {
    +  LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +            << "') sent ShiftbossRegistrationMessage (typed '" << kShiftbossRegistrationMessage
    +            << "') to all";
    --- End diff --
    
    `Shiftboss` does not have a knowledge of where is `Foreman`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #64: Introduced Shiftboss for the distribut...

Posted by hbdeshmukh <gi...@git.apache.org>.
Github user hbdeshmukh commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/64#discussion_r72129156
  
    --- Diff: query_execution/Shiftboss.hpp ---
    @@ -0,0 +1,238 @@
    +/**
    + *   Licensed under the Apache License, Version 2.0 (the "License");
    + *   you may not use this file except in compliance with the License.
    + *   You may obtain a copy of the License at
    + *
    + *       http://www.apache.org/licenses/LICENSE-2.0
    + *
    + *   Unless required by applicable law or agreed to in writing, software
    + *   distributed under the License is distributed on an "AS IS" BASIS,
    + *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + *   See the License for the specific language governing permissions and
    + *   limitations under the License.
    + **/
    +
    +#ifndef QUICKSTEP_QUERY_EXECUTION_SHIFTBOSS_HPP_
    +#define QUICKSTEP_QUERY_EXECUTION_SHIFTBOSS_HPP_
    +
    +#include <cstddef>
    +#include <memory>
    +
    +#include "catalog/CatalogDatabaseCache.hpp"
    +#include "catalog/CatalogTypedefs.hpp"
    +#include "query_execution/QueryContext.hpp"
    +#include "query_execution/QueryExecutionTypedefs.hpp"
    +#include "query_execution/WorkerDirectory.hpp"
    +#include "threading/Thread.hpp"
    +#include "utility/Macros.hpp"
    +
    +#include "glog/logging.h"
    +
    +#include "tmb/address.h"
    +#include "tmb/id_typedefs.h"
    +#include "tmb/message_bus.h"
    +
    +namespace quickstep {
    +
    +class StorageManager;
    +
    +namespace serialization {
    +class CatalogDatabase;
    +class QueryContext;
    +}  // namespace serialization
    +
    +/** \addtogroup QueryExecution
    + *  @{
    + */
    +
    +/**
    + * @brief The Shiftboss accepts workorder protos from shiftboss, and assigns
    + *        the workorders to workers.
    + **/
    +class Shiftboss : public Thread {
    + public:
    +  /**
    +   * @brief Constructor.
    +   *
    +   * @param bus A pointer to the TMB.
    +   * @param storage_manager The StorageManager to use.
    +   * @param workers A pointer to the WorkerDirectory.
    +   * @param cpu_id The ID of the CPU to which the Shiftboss thread can be pinned.
    +   *
    +   * @note If cpu_id is not specified, Shiftboss thread can be possibly moved
    +   *       around on different CPUs by the OS.
    +  **/
    +  Shiftboss(tmb::MessageBus *bus,
    +            StorageManager *storage_manager,
    +            WorkerDirectory *workers,
    +            const int cpu_id = -1)
    +      : bus_(DCHECK_NOTNULL(bus)),
    +        storage_manager_(DCHECK_NOTNULL(storage_manager)),
    +        workers_(DCHECK_NOTNULL(workers)),
    +        cpu_id_(cpu_id),
    +        shiftboss_client_id_(tmb::kClientIdNone),
    +        foreman_client_id_(tmb::kClientIdNone),
    +        max_msgs_per_worker_(1),
    +        start_worker_index_(0u) {
    +    // Check to have at least one Worker.
    +    DCHECK_GT(workers->getNumWorkers(), 0u);
    +
    +    shiftboss_client_id_ = bus_->Connect();
    +    LOG(INFO) << "Shiftboss TMB client ID: " << shiftboss_client_id_;
    +    DCHECK_NE(shiftboss_client_id_, tmb::kClientIdNone);
    +
    +    // Messages between Foreman and Shiftboss.
    +    bus_->RegisterClientAsSender(shiftboss_client_id_, kShiftbossRegistrationMessage);
    --- End diff --
    
    Alright. Can you then rename the variables as ``internal_bus`` and ``external_bus`` appropriately for a better understanding? In the current implementation, they both will be equal to the same ``bus`` pointer. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #64: Introduced Shiftboss for the distribut...

Posted by hbdeshmukh <gi...@git.apache.org>.
Github user hbdeshmukh commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/64#discussion_r72499844
  
    --- Diff: query_execution/Shiftboss.cpp ---
    @@ -0,0 +1,357 @@
    +/**
    + *   Licensed under the Apache License, Version 2.0 (the "License");
    + *   you may not use this file except in compliance with the License.
    + *   You may obtain a copy of the License at
    + *
    + *       http://www.apache.org/licenses/LICENSE-2.0
    + *
    + *   Unless required by applicable law or agreed to in writing, software
    + *   distributed under the License is distributed on an "AS IS" BASIS,
    + *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + *   See the License for the specific language governing permissions and
    + *   limitations under the License.
    + **/
    +
    +#include "query_execution/Shiftboss.hpp"
    +
    +#include <cstddef>
    +#include <cstdlib>
    +#include <memory>
    +#include <string>
    +#include <unordered_map>
    +#include <utility>
    +#include <vector>
    +
    +#include "catalog/CatalogTypedefs.hpp"
    +#include "query_execution/QueryContext.hpp"
    +#include "query_execution/QueryExecutionMessages.pb.h"
    +#include "query_execution/QueryExecutionTypedefs.hpp"
    +#include "query_execution/QueryExecutionUtil.hpp"
    +#include "query_execution/WorkerMessage.hpp"
    +#include "relational_operators/RebuildWorkOrder.hpp"
    +#include "relational_operators/WorkOrderFactory.hpp"
    +#include "storage/InsertDestination.hpp"
    +#include "storage/StorageBlock.hpp"
    +#include "storage/StorageBlockInfo.hpp"
    +#include "storage/StorageManager.hpp"
    +#include "threading/ThreadUtil.hpp"
    +
    +#include "glog/logging.h"
    +
    +#include "tmb/address.h"
    +#include "tmb/id_typedefs.h"
    +#include "tmb/message_bus.h"
    +#include "tmb/message_style.h"
    +#include "tmb/tagged_message.h"
    +
    +using std::free;
    +using std::malloc;
    +using std::move;
    +using std::size_t;
    +using std::string;
    +using std::unique_ptr;
    +using std::vector;
    +
    +using tmb::TaggedMessage;
    +
    +namespace quickstep {
    +
    +class WorkOrder;
    +
    +void Shiftboss::run() {
    +  if (cpu_id_ >= 0) {
    +    // We can pin the shiftboss thread to a CPU if specified.
    +    ThreadUtil::BindToCPU(cpu_id_);
    +  }
    +
    +  for (;;) {
    +    // Receive() is a blocking call, causing this thread to sleep until next
    +    // message is received.
    +    AnnotatedMessage annotated_message(bus_->Receive(shiftboss_client_id_, 0, true));
    +    LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +              << "') received the typed '" << annotated_message.tagged_message.message_type()
    +              << "' message from client " << annotated_message.sender;
    +    switch (annotated_message.tagged_message.message_type()) {
    +      case kShiftbossRegistrationResponseMessage: {
    +        foreman_client_id_ = annotated_message.sender;
    +        break;
    +      }
    +      case kShiftbossInitiateMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::ShiftbossInitiateMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processShiftbossInitiateMessage(proto.query_id(), proto.catalog_database_cache(), proto.query_context());
    +        break;
    +      }
    +      case kWorkOrderMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::WorkOrderMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        const std::size_t query_id = proto.query_id();
    +        DCHECK_EQ(1u, query_contexts_.count(query_id));
    +
    +        WorkOrder *work_order = WorkOrderFactory::ReconstructFromProto(proto.work_order(),
    +                                                                       &database_cache_,
    +                                                                       query_contexts_[query_id].get(),
    +                                                                       storage_manager_,
    +                                                                       shiftboss_client_id_,
    +                                                                       bus_);
    +
    +        unique_ptr<WorkerMessage> worker_message(
    +            WorkerMessage::WorkOrderMessage(work_order, proto.operator_index()));
    +
    +        TaggedMessage worker_tagged_message(worker_message.get(),
    +                                            sizeof(*worker_message),
    +                                            kWorkOrderMessage);
    +
    +        const size_t worker_index = getSchedulableWorker();
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded WorkOrderMessage (typed '" << kWorkOrderMessage
    +                  << "') from Foreman to worker " << worker_index;
    +
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           workers_->getClientID(worker_index),
    +                                           move(worker_tagged_message));
    +        break;
    +      }
    +      case kInitiateRebuildMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::InitiateRebuildMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processInitiateRebuildMessage(proto.query_id(),
    +                                      proto.operator_index(),
    +                                      proto.insert_destination_index(),
    +                                      proto.relation_id());
    +        break;
    +      }
    +      case kWorkOrderCompleteMessage:  // Fall through.
    +      case kRebuildWorkOrderCompleteMessage:
    +      case kDataPipelineMessage:
    +      case kWorkOrdersAvailableMessage:
    +      case kWorkOrderFeedbackMessage: {
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded typed '" << annotated_message.tagged_message.message_type()
    +                  << "' message from worker (client " << annotated_message.sender << ") to Foreman";
    +
    +        DCHECK_NE(foreman_client_id_, tmb::kClientIdNone);
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           foreman_client_id_,
    +                                           move(annotated_message.tagged_message));
    +        break;
    +      }
    +      case kQueryResultRelationMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::QueryResultRelationMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        for (int i = 0; i < proto.blocks_size(); ++i) {
    +          const block_id block = proto.blocks(i);
    +          storage_manager_->saveBlockOrBlob(block);
    +          if (storage_manager_->blockOrBlobIsLoaded(block)) {
    +            // NOTE(zuyu): eviction is required to avoid accesses to the query
    +            // result relation schema in CatalogDatabaseCache, for all query
    +            // optimizer execution generator unit tests and the single-process
    +            // Quickstep CLI.
    +            storage_manager_->evictBlockOrBlob(block);
    --- End diff --
    
    By looking at the code it seems that saveBlockOrBlob indeed writes a block to the disk. [Link](https://github.com/apache/incubator-quickstep/blob/master/storage/StorageManager.cpp#L406). So my previous comment still holds then. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #64: Introduced Shiftboss for the distribut...

Posted by zuyu <gi...@git.apache.org>.
Github user zuyu commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/64#discussion_r72341816
  
    --- Diff: query_execution/Shiftboss.hpp ---
    @@ -0,0 +1,241 @@
    +/**
    + *   Licensed under the Apache License, Version 2.0 (the "License");
    + *   you may not use this file except in compliance with the License.
    + *   You may obtain a copy of the License at
    + *
    + *       http://www.apache.org/licenses/LICENSE-2.0
    + *
    + *   Unless required by applicable law or agreed to in writing, software
    + *   distributed under the License is distributed on an "AS IS" BASIS,
    + *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + *   See the License for the specific language governing permissions and
    + *   limitations under the License.
    + **/
    +
    +#ifndef QUICKSTEP_QUERY_EXECUTION_SHIFTBOSS_HPP_
    +#define QUICKSTEP_QUERY_EXECUTION_SHIFTBOSS_HPP_
    +
    +#include <cstddef>
    +#include <memory>
    +#include <unordered_map>
    +
    +#include "catalog/CatalogDatabaseCache.hpp"
    +#include "catalog/CatalogTypedefs.hpp"
    +#include "query_execution/QueryContext.hpp"
    +#include "query_execution/QueryExecutionTypedefs.hpp"
    +#include "query_execution/WorkerDirectory.hpp"
    +#include "threading/Thread.hpp"
    +#include "utility/Macros.hpp"
    +
    +#include "glog/logging.h"
    +
    +#include "tmb/address.h"
    +#include "tmb/id_typedefs.h"
    +#include "tmb/message_bus.h"
    +
    +namespace quickstep {
    +
    +class StorageManager;
    +
    +namespace serialization {
    +class CatalogDatabase;
    +class QueryContext;
    +}  // namespace serialization
    +
    +/** \addtogroup QueryExecution
    + *  @{
    + */
    +
    +/**
    + * @brief The Shiftboss accepts workorder protos from shiftboss, and assigns
    + *        the workorders to workers.
    + **/
    +class Shiftboss : public Thread {
    + public:
    +  /**
    +   * @brief Constructor.
    +   *
    +   * @param bus A pointer to the TMB.
    +   * @param storage_manager The StorageManager to use.
    +   * @param workers A pointer to the WorkerDirectory.
    --- End diff --
    
    No, there will be [an executable](https://github.com/apache/incubator-quickstep/blob/distributed-prototype/cli/WorkerDistributed.cpp#L190) to create `WorkerDirectory`, and `Shiftboss` is one thread of the process.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #64: Introduced Shiftboss for the distribut...

Posted by zuyu <gi...@git.apache.org>.
Github user zuyu commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/64#discussion_r72338998
  
    --- Diff: query_execution/Shiftboss.cpp ---
    @@ -0,0 +1,357 @@
    +/**
    + *   Licensed under the Apache License, Version 2.0 (the "License");
    + *   you may not use this file except in compliance with the License.
    + *   You may obtain a copy of the License at
    + *
    + *       http://www.apache.org/licenses/LICENSE-2.0
    + *
    + *   Unless required by applicable law or agreed to in writing, software
    + *   distributed under the License is distributed on an "AS IS" BASIS,
    + *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + *   See the License for the specific language governing permissions and
    + *   limitations under the License.
    + **/
    +
    +#include "query_execution/Shiftboss.hpp"
    +
    +#include <cstddef>
    +#include <cstdlib>
    +#include <memory>
    +#include <string>
    +#include <unordered_map>
    +#include <utility>
    +#include <vector>
    +
    +#include "catalog/CatalogTypedefs.hpp"
    +#include "query_execution/QueryContext.hpp"
    +#include "query_execution/QueryExecutionMessages.pb.h"
    +#include "query_execution/QueryExecutionTypedefs.hpp"
    +#include "query_execution/QueryExecutionUtil.hpp"
    +#include "query_execution/WorkerMessage.hpp"
    +#include "relational_operators/RebuildWorkOrder.hpp"
    +#include "relational_operators/WorkOrderFactory.hpp"
    +#include "storage/InsertDestination.hpp"
    +#include "storage/StorageBlock.hpp"
    +#include "storage/StorageBlockInfo.hpp"
    +#include "storage/StorageManager.hpp"
    +#include "threading/ThreadUtil.hpp"
    +
    +#include "glog/logging.h"
    +
    +#include "tmb/address.h"
    +#include "tmb/id_typedefs.h"
    +#include "tmb/message_bus.h"
    +#include "tmb/message_style.h"
    +#include "tmb/tagged_message.h"
    +
    +using std::free;
    +using std::malloc;
    +using std::move;
    +using std::size_t;
    +using std::string;
    +using std::unique_ptr;
    +using std::vector;
    +
    +using tmb::TaggedMessage;
    +
    +namespace quickstep {
    +
    +class WorkOrder;
    +
    +void Shiftboss::run() {
    +  if (cpu_id_ >= 0) {
    +    // We can pin the shiftboss thread to a CPU if specified.
    +    ThreadUtil::BindToCPU(cpu_id_);
    +  }
    +
    +  for (;;) {
    +    // Receive() is a blocking call, causing this thread to sleep until next
    +    // message is received.
    +    AnnotatedMessage annotated_message(bus_->Receive(shiftboss_client_id_, 0, true));
    +    LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +              << "') received the typed '" << annotated_message.tagged_message.message_type()
    +              << "' message from client " << annotated_message.sender;
    +    switch (annotated_message.tagged_message.message_type()) {
    +      case kShiftbossRegistrationResponseMessage: {
    +        foreman_client_id_ = annotated_message.sender;
    +        break;
    +      }
    +      case kShiftbossInitiateMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::ShiftbossInitiateMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processShiftbossInitiateMessage(proto.query_id(), proto.catalog_database_cache(), proto.query_context());
    +        break;
    +      }
    +      case kWorkOrderMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::WorkOrderMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        const std::size_t query_id = proto.query_id();
    +        DCHECK_EQ(1u, query_contexts_.count(query_id));
    +
    +        WorkOrder *work_order = WorkOrderFactory::ReconstructFromProto(proto.work_order(),
    +                                                                       &database_cache_,
    +                                                                       query_contexts_[query_id].get(),
    +                                                                       storage_manager_,
    +                                                                       shiftboss_client_id_,
    +                                                                       bus_);
    +
    +        unique_ptr<WorkerMessage> worker_message(
    +            WorkerMessage::WorkOrderMessage(work_order, proto.operator_index()));
    +
    +        TaggedMessage worker_tagged_message(worker_message.get(),
    +                                            sizeof(*worker_message),
    +                                            kWorkOrderMessage);
    +
    +        const size_t worker_index = getSchedulableWorker();
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded WorkOrderMessage (typed '" << kWorkOrderMessage
    +                  << "') from Foreman to worker " << worker_index;
    +
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           workers_->getClientID(worker_index),
    +                                           move(worker_tagged_message));
    +        break;
    +      }
    +      case kInitiateRebuildMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::InitiateRebuildMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processInitiateRebuildMessage(proto.query_id(),
    +                                      proto.operator_index(),
    +                                      proto.insert_destination_index(),
    +                                      proto.relation_id());
    +        break;
    +      }
    +      case kWorkOrderCompleteMessage:  // Fall through.
    +      case kRebuildWorkOrderCompleteMessage:
    +      case kDataPipelineMessage:
    +      case kWorkOrdersAvailableMessage:
    +      case kWorkOrderFeedbackMessage: {
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded typed '" << annotated_message.tagged_message.message_type()
    +                  << "' message from worker (client " << annotated_message.sender << ") to Foreman";
    +
    +        DCHECK_NE(foreman_client_id_, tmb::kClientIdNone);
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           foreman_client_id_,
    +                                           move(annotated_message.tagged_message));
    +        break;
    +      }
    +      case kQueryResultRelationMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::QueryResultRelationMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        for (int i = 0; i < proto.blocks_size(); ++i) {
    +          const block_id block = proto.blocks(i);
    +          storage_manager_->saveBlockOrBlob(block);
    +          if (storage_manager_->blockOrBlobIsLoaded(block)) {
    +            // NOTE(zuyu): eviction is required to avoid accesses to the query
    +            // result relation schema in CatalogDatabaseCache, for all query
    +            // optimizer execution generator unit tests and the single-process
    +            // Quickstep CLI.
    +            storage_manager_->evictBlockOrBlob(block);
    +          }
    +        }
    +
    +        serialization::QueryResultRelationResponseMessage ack_proto;
    +        ack_proto.set_relation_id(proto.relation_id());
    +
    +        const size_t ack_proto_length = ack_proto.ByteSize();
    +        char *ack_proto_bytes = static_cast<char*>(malloc(ack_proto_length));
    +        CHECK(ack_proto.SerializeToArray(ack_proto_bytes, ack_proto_length));
    +
    +        TaggedMessage ack_message(static_cast<const void*>(ack_proto_bytes),
    +                                  ack_proto_length,
    +                                  kQueryResultRelationResponseMessage);
    +        free(ack_proto_bytes);
    +
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') sent QueryResultRelationResponseMessage (typed '" << kQueryResultRelationResponseMessage
    +                  << ") to Foreman";
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           foreman_client_id_,
    +                                           move(ack_message));
    +        break;
    +      }
    +      case kPoisonMessage: {
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded PoisonMessage (typed '" << kPoisonMessage
    +                  << "') from Foreman to all workers";
    +
    +        tmb::MessageStyle broadcast_style;
    +        broadcast_style.Broadcast(true);
    +
    +        tmb::MessageBus::SendStatus send_status =
    +            bus_->Send(shiftboss_client_id_,
    +                       worker_addresses_,
    +                       broadcast_style,
    +                       move(annotated_message.tagged_message));
    +        DCHECK(send_status == tmb::MessageBus::SendStatus::kOK);
    +        return;
    +      }
    +      default: {
    +        LOG(FATAL) << "Unknown TMB message type";
    +      }
    +    }
    +  }
    +}
    +
    +size_t Shiftboss::getSchedulableWorker() {
    +  const size_t num_workers = workers_->getNumWorkers();
    +
    +  size_t curr_worker = start_worker_index_;
    +  for (;;) {
    +    if (workers_->getNumQueuedWorkOrders(curr_worker) < max_msgs_per_worker_) {
    +      start_worker_index_ = (curr_worker + 1) % num_workers;
    +      // TODO(zuyu): workers_->incrementNumQueuedWorkOrders(curr_worker);
    +      // But we need a WorkOrder queue first.
    --- End diff --
    
    This function needs more refactor in many aspects, but I prefer to leave it as it to have a prototype of the distributed version running.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep issue #64: Introduced Shiftboss for the distributed vers...

Posted by zuyu <gi...@git.apache.org>.
Github user zuyu commented on the issue:

    https://github.com/apache/incubator-quickstep/pull/64
  
    The `RebuildWorkOrder` info would be added in the next PR where `ForemanDistributed` is introduced.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #64: Introduced Shiftboss for the distribut...

Posted by hbdeshmukh <gi...@git.apache.org>.
Github user hbdeshmukh commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/64#discussion_r72337331
  
    --- Diff: query_execution/Shiftboss.cpp ---
    @@ -0,0 +1,357 @@
    +/**
    + *   Licensed under the Apache License, Version 2.0 (the "License");
    + *   you may not use this file except in compliance with the License.
    + *   You may obtain a copy of the License at
    + *
    + *       http://www.apache.org/licenses/LICENSE-2.0
    + *
    + *   Unless required by applicable law or agreed to in writing, software
    + *   distributed under the License is distributed on an "AS IS" BASIS,
    + *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + *   See the License for the specific language governing permissions and
    + *   limitations under the License.
    + **/
    +
    +#include "query_execution/Shiftboss.hpp"
    +
    +#include <cstddef>
    +#include <cstdlib>
    +#include <memory>
    +#include <string>
    +#include <unordered_map>
    +#include <utility>
    +#include <vector>
    +
    +#include "catalog/CatalogTypedefs.hpp"
    +#include "query_execution/QueryContext.hpp"
    +#include "query_execution/QueryExecutionMessages.pb.h"
    +#include "query_execution/QueryExecutionTypedefs.hpp"
    +#include "query_execution/QueryExecutionUtil.hpp"
    +#include "query_execution/WorkerMessage.hpp"
    +#include "relational_operators/RebuildWorkOrder.hpp"
    +#include "relational_operators/WorkOrderFactory.hpp"
    +#include "storage/InsertDestination.hpp"
    +#include "storage/StorageBlock.hpp"
    +#include "storage/StorageBlockInfo.hpp"
    +#include "storage/StorageManager.hpp"
    +#include "threading/ThreadUtil.hpp"
    +
    +#include "glog/logging.h"
    +
    +#include "tmb/address.h"
    +#include "tmb/id_typedefs.h"
    +#include "tmb/message_bus.h"
    +#include "tmb/message_style.h"
    +#include "tmb/tagged_message.h"
    +
    +using std::free;
    +using std::malloc;
    +using std::move;
    +using std::size_t;
    +using std::string;
    +using std::unique_ptr;
    +using std::vector;
    +
    +using tmb::TaggedMessage;
    +
    +namespace quickstep {
    +
    +class WorkOrder;
    +
    +void Shiftboss::run() {
    +  if (cpu_id_ >= 0) {
    +    // We can pin the shiftboss thread to a CPU if specified.
    +    ThreadUtil::BindToCPU(cpu_id_);
    +  }
    +
    +  for (;;) {
    +    // Receive() is a blocking call, causing this thread to sleep until next
    +    // message is received.
    +    AnnotatedMessage annotated_message(bus_->Receive(shiftboss_client_id_, 0, true));
    +    LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +              << "') received the typed '" << annotated_message.tagged_message.message_type()
    +              << "' message from client " << annotated_message.sender;
    +    switch (annotated_message.tagged_message.message_type()) {
    +      case kShiftbossRegistrationResponseMessage: {
    +        foreman_client_id_ = annotated_message.sender;
    +        break;
    +      }
    +      case kShiftbossInitiateMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::ShiftbossInitiateMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processShiftbossInitiateMessage(proto.query_id(), proto.catalog_database_cache(), proto.query_context());
    +        break;
    +      }
    +      case kWorkOrderMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::WorkOrderMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        const std::size_t query_id = proto.query_id();
    +        DCHECK_EQ(1u, query_contexts_.count(query_id));
    +
    +        WorkOrder *work_order = WorkOrderFactory::ReconstructFromProto(proto.work_order(),
    +                                                                       &database_cache_,
    +                                                                       query_contexts_[query_id].get(),
    +                                                                       storage_manager_,
    +                                                                       shiftboss_client_id_,
    +                                                                       bus_);
    +
    +        unique_ptr<WorkerMessage> worker_message(
    +            WorkerMessage::WorkOrderMessage(work_order, proto.operator_index()));
    +
    +        TaggedMessage worker_tagged_message(worker_message.get(),
    +                                            sizeof(*worker_message),
    +                                            kWorkOrderMessage);
    +
    +        const size_t worker_index = getSchedulableWorker();
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded WorkOrderMessage (typed '" << kWorkOrderMessage
    +                  << "') from Foreman to worker " << worker_index;
    +
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           workers_->getClientID(worker_index),
    +                                           move(worker_tagged_message));
    +        break;
    +      }
    +      case kInitiateRebuildMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::InitiateRebuildMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processInitiateRebuildMessage(proto.query_id(),
    +                                      proto.operator_index(),
    +                                      proto.insert_destination_index(),
    +                                      proto.relation_id());
    +        break;
    +      }
    +      case kWorkOrderCompleteMessage:  // Fall through.
    +      case kRebuildWorkOrderCompleteMessage:
    +      case kDataPipelineMessage:
    +      case kWorkOrdersAvailableMessage:
    +      case kWorkOrderFeedbackMessage: {
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded typed '" << annotated_message.tagged_message.message_type()
    +                  << "' message from worker (client " << annotated_message.sender << ") to Foreman";
    +
    +        DCHECK_NE(foreman_client_id_, tmb::kClientIdNone);
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           foreman_client_id_,
    +                                           move(annotated_message.tagged_message));
    +        break;
    +      }
    +      case kQueryResultRelationMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::QueryResultRelationMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        for (int i = 0; i < proto.blocks_size(); ++i) {
    +          const block_id block = proto.blocks(i);
    +          storage_manager_->saveBlockOrBlob(block);
    +          if (storage_manager_->blockOrBlobIsLoaded(block)) {
    +            // NOTE(zuyu): eviction is required to avoid accesses to the query
    +            // result relation schema in CatalogDatabaseCache, for all query
    +            // optimizer execution generator unit tests and the single-process
    +            // Quickstep CLI.
    +            storage_manager_->evictBlockOrBlob(block);
    +          }
    +        }
    +
    +        serialization::QueryResultRelationResponseMessage ack_proto;
    +        ack_proto.set_relation_id(proto.relation_id());
    +
    +        const size_t ack_proto_length = ack_proto.ByteSize();
    +        char *ack_proto_bytes = static_cast<char*>(malloc(ack_proto_length));
    +        CHECK(ack_proto.SerializeToArray(ack_proto_bytes, ack_proto_length));
    +
    +        TaggedMessage ack_message(static_cast<const void*>(ack_proto_bytes),
    +                                  ack_proto_length,
    +                                  kQueryResultRelationResponseMessage);
    +        free(ack_proto_bytes);
    +
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') sent QueryResultRelationResponseMessage (typed '" << kQueryResultRelationResponseMessage
    +                  << ") to Foreman";
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           foreman_client_id_,
    +                                           move(ack_message));
    +        break;
    +      }
    +      case kPoisonMessage: {
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded PoisonMessage (typed '" << kPoisonMessage
    +                  << "') from Foreman to all workers";
    +
    +        tmb::MessageStyle broadcast_style;
    +        broadcast_style.Broadcast(true);
    +
    +        tmb::MessageBus::SendStatus send_status =
    +            bus_->Send(shiftboss_client_id_,
    +                       worker_addresses_,
    +                       broadcast_style,
    +                       move(annotated_message.tagged_message));
    +        DCHECK(send_status == tmb::MessageBus::SendStatus::kOK);
    +        return;
    +      }
    +      default: {
    +        LOG(FATAL) << "Unknown TMB message type";
    +      }
    +    }
    +  }
    +}
    +
    +size_t Shiftboss::getSchedulableWorker() {
    +  const size_t num_workers = workers_->getNumWorkers();
    +
    +  size_t curr_worker = start_worker_index_;
    +  for (;;) {
    +    if (workers_->getNumQueuedWorkOrders(curr_worker) < max_msgs_per_worker_) {
    +      start_worker_index_ = (curr_worker + 1) % num_workers;
    +      // TODO(zuyu): workers_->incrementNumQueuedWorkOrders(curr_worker);
    +      // But we need a WorkOrder queue first.
    +      return curr_worker;
    +    }
    +
    +    curr_worker = (curr_worker + 1) % num_workers;
    +  }
    +}
    +
    +void Shiftboss::registerWithForeman() {
    +  LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +            << "') sent ShiftbossRegistrationMessage (typed '" << kShiftbossRegistrationMessage
    +            << "') to all";
    --- End diff --
    
    Why should the registration message be broadcast? Are you assuming multiple Foremen?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #64: Introduced Shiftboss for the distribut...

Posted by hbdeshmukh <gi...@git.apache.org>.
Github user hbdeshmukh commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/64#discussion_r72497674
  
    --- Diff: query_execution/Shiftboss.hpp ---
    @@ -0,0 +1,241 @@
    +/**
    + *   Licensed under the Apache License, Version 2.0 (the "License");
    + *   you may not use this file except in compliance with the License.
    + *   You may obtain a copy of the License at
    + *
    + *       http://www.apache.org/licenses/LICENSE-2.0
    + *
    + *   Unless required by applicable law or agreed to in writing, software
    + *   distributed under the License is distributed on an "AS IS" BASIS,
    + *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + *   See the License for the specific language governing permissions and
    + *   limitations under the License.
    + **/
    +
    +#ifndef QUICKSTEP_QUERY_EXECUTION_SHIFTBOSS_HPP_
    +#define QUICKSTEP_QUERY_EXECUTION_SHIFTBOSS_HPP_
    +
    +#include <cstddef>
    +#include <memory>
    +#include <unordered_map>
    +
    +#include "catalog/CatalogDatabaseCache.hpp"
    +#include "catalog/CatalogTypedefs.hpp"
    +#include "query_execution/QueryContext.hpp"
    +#include "query_execution/QueryExecutionTypedefs.hpp"
    +#include "query_execution/WorkerDirectory.hpp"
    +#include "threading/Thread.hpp"
    +#include "utility/Macros.hpp"
    +
    +#include "glog/logging.h"
    +
    +#include "tmb/address.h"
    +#include "tmb/id_typedefs.h"
    +#include "tmb/message_bus.h"
    +
    +namespace quickstep {
    +
    +class StorageManager;
    +
    +namespace serialization {
    +class CatalogDatabase;
    +class QueryContext;
    +}  // namespace serialization
    +
    +/** \addtogroup QueryExecution
    + *  @{
    + */
    +
    +/**
    + * @brief The Shiftboss accepts workorder protos from shiftboss, and assigns
    + *        the workorders to workers.
    + **/
    +class Shiftboss : public Thread {
    + public:
    +  /**
    +   * @brief Constructor.
    +   *
    +   * @param bus A pointer to the TMB.
    +   * @param storage_manager The StorageManager to use.
    +   * @param workers A pointer to the WorkerDirectory.
    --- End diff --
    
    Alright. Do you have a documentation describing these details? It's difficult to figure them out by looking at the code. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #64: Introduced Shiftboss for the distribut...

Posted by hbdeshmukh <gi...@git.apache.org>.
Github user hbdeshmukh commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/64#discussion_r72522825
  
    --- Diff: query_execution/Shiftboss.cpp ---
    @@ -0,0 +1,357 @@
    +/**
    + *   Licensed under the Apache License, Version 2.0 (the "License");
    + *   you may not use this file except in compliance with the License.
    + *   You may obtain a copy of the License at
    + *
    + *       http://www.apache.org/licenses/LICENSE-2.0
    + *
    + *   Unless required by applicable law or agreed to in writing, software
    + *   distributed under the License is distributed on an "AS IS" BASIS,
    + *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + *   See the License for the specific language governing permissions and
    + *   limitations under the License.
    + **/
    +
    +#include "query_execution/Shiftboss.hpp"
    +
    +#include <cstddef>
    +#include <cstdlib>
    +#include <memory>
    +#include <string>
    +#include <unordered_map>
    +#include <utility>
    +#include <vector>
    +
    +#include "catalog/CatalogTypedefs.hpp"
    +#include "query_execution/QueryContext.hpp"
    +#include "query_execution/QueryExecutionMessages.pb.h"
    +#include "query_execution/QueryExecutionTypedefs.hpp"
    +#include "query_execution/QueryExecutionUtil.hpp"
    +#include "query_execution/WorkerMessage.hpp"
    +#include "relational_operators/RebuildWorkOrder.hpp"
    +#include "relational_operators/WorkOrderFactory.hpp"
    +#include "storage/InsertDestination.hpp"
    +#include "storage/StorageBlock.hpp"
    +#include "storage/StorageBlockInfo.hpp"
    +#include "storage/StorageManager.hpp"
    +#include "threading/ThreadUtil.hpp"
    +
    +#include "glog/logging.h"
    +
    +#include "tmb/address.h"
    +#include "tmb/id_typedefs.h"
    +#include "tmb/message_bus.h"
    +#include "tmb/message_style.h"
    +#include "tmb/tagged_message.h"
    +
    +using std::free;
    +using std::malloc;
    +using std::move;
    +using std::size_t;
    +using std::string;
    +using std::unique_ptr;
    +using std::vector;
    +
    +using tmb::TaggedMessage;
    +
    +namespace quickstep {
    +
    +class WorkOrder;
    +
    +void Shiftboss::run() {
    +  if (cpu_id_ >= 0) {
    +    // We can pin the shiftboss thread to a CPU if specified.
    +    ThreadUtil::BindToCPU(cpu_id_);
    +  }
    +
    +  for (;;) {
    +    // Receive() is a blocking call, causing this thread to sleep until next
    +    // message is received.
    +    AnnotatedMessage annotated_message(bus_->Receive(shiftboss_client_id_, 0, true));
    +    LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +              << "') received the typed '" << annotated_message.tagged_message.message_type()
    +              << "' message from client " << annotated_message.sender;
    +    switch (annotated_message.tagged_message.message_type()) {
    +      case kShiftbossRegistrationResponseMessage: {
    +        foreman_client_id_ = annotated_message.sender;
    +        break;
    +      }
    +      case kShiftbossInitiateMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::ShiftbossInitiateMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processShiftbossInitiateMessage(proto.query_id(), proto.catalog_database_cache(), proto.query_context());
    +        break;
    +      }
    +      case kWorkOrderMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::WorkOrderMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        const std::size_t query_id = proto.query_id();
    +        DCHECK_EQ(1u, query_contexts_.count(query_id));
    +
    +        WorkOrder *work_order = WorkOrderFactory::ReconstructFromProto(proto.work_order(),
    +                                                                       &database_cache_,
    +                                                                       query_contexts_[query_id].get(),
    +                                                                       storage_manager_,
    +                                                                       shiftboss_client_id_,
    +                                                                       bus_);
    +
    +        unique_ptr<WorkerMessage> worker_message(
    +            WorkerMessage::WorkOrderMessage(work_order, proto.operator_index()));
    +
    +        TaggedMessage worker_tagged_message(worker_message.get(),
    +                                            sizeof(*worker_message),
    +                                            kWorkOrderMessage);
    +
    +        const size_t worker_index = getSchedulableWorker();
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded WorkOrderMessage (typed '" << kWorkOrderMessage
    +                  << "') from Foreman to worker " << worker_index;
    +
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           workers_->getClientID(worker_index),
    +                                           move(worker_tagged_message));
    +        break;
    +      }
    +      case kInitiateRebuildMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::InitiateRebuildMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processInitiateRebuildMessage(proto.query_id(),
    +                                      proto.operator_index(),
    +                                      proto.insert_destination_index(),
    +                                      proto.relation_id());
    +        break;
    +      }
    +      case kWorkOrderCompleteMessage:  // Fall through.
    +      case kRebuildWorkOrderCompleteMessage:
    +      case kDataPipelineMessage:
    +      case kWorkOrdersAvailableMessage:
    +      case kWorkOrderFeedbackMessage: {
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded typed '" << annotated_message.tagged_message.message_type()
    +                  << "' message from worker (client " << annotated_message.sender << ") to Foreman";
    +
    +        DCHECK_NE(foreman_client_id_, tmb::kClientIdNone);
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           foreman_client_id_,
    +                                           move(annotated_message.tagged_message));
    +        break;
    +      }
    +      case kQueryResultRelationMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::QueryResultRelationMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        for (int i = 0; i < proto.blocks_size(); ++i) {
    +          const block_id block = proto.blocks(i);
    +          storage_manager_->saveBlockOrBlob(block);
    +          if (storage_manager_->blockOrBlobIsLoaded(block)) {
    +            // NOTE(zuyu): eviction is required to avoid accesses to the query
    +            // result relation schema in CatalogDatabaseCache, for all query
    +            // optimizer execution generator unit tests and the single-process
    +            // Quickstep CLI.
    +            storage_manager_->evictBlockOrBlob(block);
    --- End diff --
    
    > You could play with my old code in distributed-exe-gen-test to confirm my statement.
    
    Please understand that I am not doubting what you have said. I am just trying to understand why do you need to evict a block from a result relation, which presumably won't be accessed by any other worker thread from that node at the time this line of code is getting executed. Your explanation from the source code reads as follows:
    
    ```
                // NOTE(zuyu): eviction is required to avoid accesses to the query
                // result relation schema in CatalogDatabaseCache, for all query
                // optimizer execution generator unit tests and the single-process
                // Quickstep CLI.
    ```
    
    Why would optimizer execution generator unit tests interfere with the actual invocation of the system? 
    Why does quickstep CLI cause an issue here? Is it something about concurrent accesses to a same block from two different processes? 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #64: Introduced Shiftboss for the distribut...

Posted by zuyu <gi...@git.apache.org>.
Github user zuyu commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/64#discussion_r72134688
  
    --- Diff: query_execution/Shiftboss.hpp ---
    @@ -0,0 +1,238 @@
    +/**
    + *   Licensed under the Apache License, Version 2.0 (the "License");
    + *   you may not use this file except in compliance with the License.
    + *   You may obtain a copy of the License at
    + *
    + *       http://www.apache.org/licenses/LICENSE-2.0
    + *
    + *   Unless required by applicable law or agreed to in writing, software
    + *   distributed under the License is distributed on an "AS IS" BASIS,
    + *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + *   See the License for the specific language governing permissions and
    + *   limitations under the License.
    + **/
    +
    +#ifndef QUICKSTEP_QUERY_EXECUTION_SHIFTBOSS_HPP_
    +#define QUICKSTEP_QUERY_EXECUTION_SHIFTBOSS_HPP_
    +
    +#include <cstddef>
    +#include <memory>
    +
    +#include "catalog/CatalogDatabaseCache.hpp"
    +#include "catalog/CatalogTypedefs.hpp"
    +#include "query_execution/QueryContext.hpp"
    +#include "query_execution/QueryExecutionTypedefs.hpp"
    +#include "query_execution/WorkerDirectory.hpp"
    +#include "threading/Thread.hpp"
    +#include "utility/Macros.hpp"
    +
    +#include "glog/logging.h"
    +
    +#include "tmb/address.h"
    +#include "tmb/id_typedefs.h"
    +#include "tmb/message_bus.h"
    +
    +namespace quickstep {
    +
    +class StorageManager;
    +
    +namespace serialization {
    +class CatalogDatabase;
    +class QueryContext;
    +}  // namespace serialization
    +
    +/** \addtogroup QueryExecution
    + *  @{
    + */
    +
    +/**
    + * @brief The Shiftboss accepts workorder protos from shiftboss, and assigns
    + *        the workorders to workers.
    + **/
    +class Shiftboss : public Thread {
    + public:
    +  /**
    +   * @brief Constructor.
    +   *
    +   * @param bus A pointer to the TMB.
    +   * @param storage_manager The StorageManager to use.
    +   * @param workers A pointer to the WorkerDirectory.
    +   * @param cpu_id The ID of the CPU to which the Shiftboss thread can be pinned.
    +   *
    +   * @note If cpu_id is not specified, Shiftboss thread can be possibly moved
    +   *       around on different CPUs by the OS.
    +  **/
    +  Shiftboss(tmb::MessageBus *bus,
    +            StorageManager *storage_manager,
    +            WorkerDirectory *workers,
    +            const int cpu_id = -1)
    +      : bus_(DCHECK_NOTNULL(bus)),
    +        storage_manager_(DCHECK_NOTNULL(storage_manager)),
    +        workers_(DCHECK_NOTNULL(workers)),
    +        cpu_id_(cpu_id),
    +        shiftboss_client_id_(tmb::kClientIdNone),
    +        foreman_client_id_(tmb::kClientIdNone),
    +        max_msgs_per_worker_(1),
    +        start_worker_index_(0u) {
    +    // Check to have at least one Worker.
    +    DCHECK_GT(workers->getNumWorkers(), 0u);
    +
    +    shiftboss_client_id_ = bus_->Connect();
    +    LOG(INFO) << "Shiftboss TMB client ID: " << shiftboss_client_id_;
    +    DCHECK_NE(shiftboss_client_id_, tmb::kClientIdNone);
    +
    +    // Messages between Foreman and Shiftboss.
    +    bus_->RegisterClientAsSender(shiftboss_client_id_, kShiftbossRegistrationMessage);
    +    bus_->RegisterClientAsReceiver(shiftboss_client_id_, kShiftbossRegistrationResponseMessage);
    +
    +    bus_->RegisterClientAsReceiver(shiftboss_client_id_, kShiftbossInitiateMessage);
    +    bus_->RegisterClientAsSender(shiftboss_client_id_, kShiftbossInitiateResponseMessage);
    +
    +    bus_->RegisterClientAsReceiver(shiftboss_client_id_, kInitiateRebuildMessage);
    +    bus_->RegisterClientAsSender(shiftboss_client_id_, kInitiateRebuildResponseMessage);
    +
    +    // Message sent to Worker.
    +    bus_->RegisterClientAsSender(shiftboss_client_id_, kRebuildWorkOrderMessage);
    +
    +    // Message sent to Foreman.
    +    bus_->RegisterClientAsSender(shiftboss_client_id_, kCatalogRelationNewBlockMessage);
    +    bus_->RegisterClientAsSender(shiftboss_client_id_, kDataPipelineMessage);
    +    bus_->RegisterClientAsSender(shiftboss_client_id_, kWorkOrdersAvailableMessage);
    +    bus_->RegisterClientAsSender(shiftboss_client_id_, kWorkOrderFeedbackMessage);
    +
    +    // Forward the following message types from Foreman to Workers.
    +    bus_->RegisterClientAsReceiver(shiftboss_client_id_, kWorkOrderMessage);
    +    bus_->RegisterClientAsSender(shiftboss_client_id_, kWorkOrderMessage);
    +
    +    // Forward the following message types from Workers to Foreman.
    +    bus_->RegisterClientAsReceiver(shiftboss_client_id_, kWorkOrderCompleteMessage);
    +    bus_->RegisterClientAsSender(shiftboss_client_id_, kWorkOrderCompleteMessage);
    +
    +    bus_->RegisterClientAsReceiver(shiftboss_client_id_, kRebuildWorkOrderCompleteMessage);
    +    bus_->RegisterClientAsSender(shiftboss_client_id_, kRebuildWorkOrderCompleteMessage);
    +
    +    bus_->RegisterClientAsReceiver(shiftboss_client_id_, kQueryResultRelationMessage);
    +    bus_->RegisterClientAsSender(shiftboss_client_id_, kQueryResultRelationResponseMessage);
    +
    +    // Stop itself.
    +    bus_->RegisterClientAsReceiver(shiftboss_client_id_, kPoisonMessage);
    +    // Stop all workers.
    +    bus_->RegisterClientAsSender(shiftboss_client_id_, kPoisonMessage);
    +
    +    for (std::size_t i = 0; i < workers_->getNumWorkers(); ++i) {
    +      worker_addresses_.AddRecipient(workers_->getClientID(i));
    +    }
    +
    +    registerWithForeman();
    +  }
    +
    +  ~Shiftboss() override {
    +  }
    +
    +  /**
    +   * @brief Get the TMB client ID of Shiftboss thread.
    +   *
    +   * @return TMB client ID of shiftboss thread.
    +   **/
    +  inline tmb::client_id getBusClientID() const {
    +    return shiftboss_client_id_;
    +  }
    +
    +  /**
    +   * @brief Get the Work Order processing capacity of all Workers managed by
    +   *        Shiftboss during a single round of WorkOrder dispatch.
    +   **/
    +  inline std::size_t getWorkOrderCapacity() const {
    +    DCHECK_NE(max_msgs_per_worker_, 0u);
    +    return max_msgs_per_worker_ * workers_->getNumWorkers();
    --- End diff --
    
    Since `w` is set, a configurable `n` has a fine-grain control over `N`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #64: Introduced Shiftboss for the distribut...

Posted by hbdeshmukh <gi...@git.apache.org>.
Github user hbdeshmukh commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/64#discussion_r72337106
  
    --- Diff: query_execution/Shiftboss.cpp ---
    @@ -0,0 +1,357 @@
    +/**
    + *   Licensed under the Apache License, Version 2.0 (the "License");
    + *   you may not use this file except in compliance with the License.
    + *   You may obtain a copy of the License at
    + *
    + *       http://www.apache.org/licenses/LICENSE-2.0
    + *
    + *   Unless required by applicable law or agreed to in writing, software
    + *   distributed under the License is distributed on an "AS IS" BASIS,
    + *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + *   See the License for the specific language governing permissions and
    + *   limitations under the License.
    + **/
    +
    +#include "query_execution/Shiftboss.hpp"
    +
    +#include <cstddef>
    +#include <cstdlib>
    +#include <memory>
    +#include <string>
    +#include <unordered_map>
    +#include <utility>
    +#include <vector>
    +
    +#include "catalog/CatalogTypedefs.hpp"
    +#include "query_execution/QueryContext.hpp"
    +#include "query_execution/QueryExecutionMessages.pb.h"
    +#include "query_execution/QueryExecutionTypedefs.hpp"
    +#include "query_execution/QueryExecutionUtil.hpp"
    +#include "query_execution/WorkerMessage.hpp"
    +#include "relational_operators/RebuildWorkOrder.hpp"
    +#include "relational_operators/WorkOrderFactory.hpp"
    +#include "storage/InsertDestination.hpp"
    +#include "storage/StorageBlock.hpp"
    +#include "storage/StorageBlockInfo.hpp"
    +#include "storage/StorageManager.hpp"
    +#include "threading/ThreadUtil.hpp"
    +
    +#include "glog/logging.h"
    +
    +#include "tmb/address.h"
    +#include "tmb/id_typedefs.h"
    +#include "tmb/message_bus.h"
    +#include "tmb/message_style.h"
    +#include "tmb/tagged_message.h"
    +
    +using std::free;
    +using std::malloc;
    +using std::move;
    +using std::size_t;
    +using std::string;
    +using std::unique_ptr;
    +using std::vector;
    +
    +using tmb::TaggedMessage;
    +
    +namespace quickstep {
    +
    +class WorkOrder;
    +
    +void Shiftboss::run() {
    +  if (cpu_id_ >= 0) {
    +    // We can pin the shiftboss thread to a CPU if specified.
    +    ThreadUtil::BindToCPU(cpu_id_);
    +  }
    +
    +  for (;;) {
    +    // Receive() is a blocking call, causing this thread to sleep until next
    +    // message is received.
    +    AnnotatedMessage annotated_message(bus_->Receive(shiftboss_client_id_, 0, true));
    +    LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +              << "') received the typed '" << annotated_message.tagged_message.message_type()
    +              << "' message from client " << annotated_message.sender;
    +    switch (annotated_message.tagged_message.message_type()) {
    +      case kShiftbossRegistrationResponseMessage: {
    +        foreman_client_id_ = annotated_message.sender;
    +        break;
    +      }
    +      case kShiftbossInitiateMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::ShiftbossInitiateMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processShiftbossInitiateMessage(proto.query_id(), proto.catalog_database_cache(), proto.query_context());
    +        break;
    +      }
    +      case kWorkOrderMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::WorkOrderMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        const std::size_t query_id = proto.query_id();
    +        DCHECK_EQ(1u, query_contexts_.count(query_id));
    +
    +        WorkOrder *work_order = WorkOrderFactory::ReconstructFromProto(proto.work_order(),
    +                                                                       &database_cache_,
    +                                                                       query_contexts_[query_id].get(),
    +                                                                       storage_manager_,
    +                                                                       shiftboss_client_id_,
    +                                                                       bus_);
    +
    +        unique_ptr<WorkerMessage> worker_message(
    +            WorkerMessage::WorkOrderMessage(work_order, proto.operator_index()));
    +
    +        TaggedMessage worker_tagged_message(worker_message.get(),
    +                                            sizeof(*worker_message),
    +                                            kWorkOrderMessage);
    +
    +        const size_t worker_index = getSchedulableWorker();
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded WorkOrderMessage (typed '" << kWorkOrderMessage
    +                  << "') from Foreman to worker " << worker_index;
    +
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           workers_->getClientID(worker_index),
    +                                           move(worker_tagged_message));
    +        break;
    +      }
    +      case kInitiateRebuildMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::InitiateRebuildMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processInitiateRebuildMessage(proto.query_id(),
    +                                      proto.operator_index(),
    +                                      proto.insert_destination_index(),
    +                                      proto.relation_id());
    +        break;
    +      }
    +      case kWorkOrderCompleteMessage:  // Fall through.
    +      case kRebuildWorkOrderCompleteMessage:
    +      case kDataPipelineMessage:
    +      case kWorkOrdersAvailableMessage:
    +      case kWorkOrderFeedbackMessage: {
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded typed '" << annotated_message.tagged_message.message_type()
    +                  << "' message from worker (client " << annotated_message.sender << ") to Foreman";
    +
    +        DCHECK_NE(foreman_client_id_, tmb::kClientIdNone);
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           foreman_client_id_,
    +                                           move(annotated_message.tagged_message));
    +        break;
    +      }
    +      case kQueryResultRelationMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::QueryResultRelationMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        for (int i = 0; i < proto.blocks_size(); ++i) {
    +          const block_id block = proto.blocks(i);
    +          storage_manager_->saveBlockOrBlob(block);
    +          if (storage_manager_->blockOrBlobIsLoaded(block)) {
    +            // NOTE(zuyu): eviction is required to avoid accesses to the query
    +            // result relation schema in CatalogDatabaseCache, for all query
    +            // optimizer execution generator unit tests and the single-process
    +            // Quickstep CLI.
    +            storage_manager_->evictBlockOrBlob(block);
    +          }
    +        }
    +
    +        serialization::QueryResultRelationResponseMessage ack_proto;
    +        ack_proto.set_relation_id(proto.relation_id());
    +
    +        const size_t ack_proto_length = ack_proto.ByteSize();
    +        char *ack_proto_bytes = static_cast<char*>(malloc(ack_proto_length));
    +        CHECK(ack_proto.SerializeToArray(ack_proto_bytes, ack_proto_length));
    +
    +        TaggedMessage ack_message(static_cast<const void*>(ack_proto_bytes),
    +                                  ack_proto_length,
    +                                  kQueryResultRelationResponseMessage);
    +        free(ack_proto_bytes);
    +
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') sent QueryResultRelationResponseMessage (typed '" << kQueryResultRelationResponseMessage
    +                  << ") to Foreman";
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           foreman_client_id_,
    +                                           move(ack_message));
    +        break;
    +      }
    +      case kPoisonMessage: {
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded PoisonMessage (typed '" << kPoisonMessage
    +                  << "') from Foreman to all workers";
    +
    +        tmb::MessageStyle broadcast_style;
    +        broadcast_style.Broadcast(true);
    +
    +        tmb::MessageBus::SendStatus send_status =
    +            bus_->Send(shiftboss_client_id_,
    +                       worker_addresses_,
    +                       broadcast_style,
    +                       move(annotated_message.tagged_message));
    +        DCHECK(send_status == tmb::MessageBus::SendStatus::kOK);
    +        return;
    +      }
    +      default: {
    +        LOG(FATAL) << "Unknown TMB message type";
    +      }
    +    }
    +  }
    +}
    +
    +size_t Shiftboss::getSchedulableWorker() {
    +  const size_t num_workers = workers_->getNumWorkers();
    +
    +  size_t curr_worker = start_worker_index_;
    +  for (;;) {
    +    if (workers_->getNumQueuedWorkOrders(curr_worker) < max_msgs_per_worker_) {
    +      start_worker_index_ = (curr_worker + 1) % num_workers;
    +      // TODO(zuyu): workers_->incrementNumQueuedWorkOrders(curr_worker);
    +      // But we need a WorkOrder queue first.
    --- End diff --
    
    It appears that you are doing round-robin style allocation of work to workers. If you do the increment as mentioned in your TODO comment above, you could pick a least loaded worker and assign it the work order. In that case you should also do a ``decrementNumQueuedWorkOrders()`` upon completion of a work order.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #64: Introduced Shiftboss for the distribut...

Posted by hbdeshmukh <gi...@git.apache.org>.
Github user hbdeshmukh commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/64#discussion_r72505327
  
    --- Diff: query_execution/Shiftboss.cpp ---
    @@ -0,0 +1,357 @@
    +/**
    + *   Licensed under the Apache License, Version 2.0 (the "License");
    + *   you may not use this file except in compliance with the License.
    + *   You may obtain a copy of the License at
    + *
    + *       http://www.apache.org/licenses/LICENSE-2.0
    + *
    + *   Unless required by applicable law or agreed to in writing, software
    + *   distributed under the License is distributed on an "AS IS" BASIS,
    + *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + *   See the License for the specific language governing permissions and
    + *   limitations under the License.
    + **/
    +
    +#include "query_execution/Shiftboss.hpp"
    +
    +#include <cstddef>
    +#include <cstdlib>
    +#include <memory>
    +#include <string>
    +#include <unordered_map>
    +#include <utility>
    +#include <vector>
    +
    +#include "catalog/CatalogTypedefs.hpp"
    +#include "query_execution/QueryContext.hpp"
    +#include "query_execution/QueryExecutionMessages.pb.h"
    +#include "query_execution/QueryExecutionTypedefs.hpp"
    +#include "query_execution/QueryExecutionUtil.hpp"
    +#include "query_execution/WorkerMessage.hpp"
    +#include "relational_operators/RebuildWorkOrder.hpp"
    +#include "relational_operators/WorkOrderFactory.hpp"
    +#include "storage/InsertDestination.hpp"
    +#include "storage/StorageBlock.hpp"
    +#include "storage/StorageBlockInfo.hpp"
    +#include "storage/StorageManager.hpp"
    +#include "threading/ThreadUtil.hpp"
    +
    +#include "glog/logging.h"
    +
    +#include "tmb/address.h"
    +#include "tmb/id_typedefs.h"
    +#include "tmb/message_bus.h"
    +#include "tmb/message_style.h"
    +#include "tmb/tagged_message.h"
    +
    +using std::free;
    +using std::malloc;
    +using std::move;
    +using std::size_t;
    +using std::string;
    +using std::unique_ptr;
    +using std::vector;
    +
    +using tmb::TaggedMessage;
    +
    +namespace quickstep {
    +
    +class WorkOrder;
    +
    +void Shiftboss::run() {
    +  if (cpu_id_ >= 0) {
    +    // We can pin the shiftboss thread to a CPU if specified.
    +    ThreadUtil::BindToCPU(cpu_id_);
    +  }
    +
    +  for (;;) {
    +    // Receive() is a blocking call, causing this thread to sleep until next
    +    // message is received.
    +    AnnotatedMessage annotated_message(bus_->Receive(shiftboss_client_id_, 0, true));
    +    LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +              << "') received the typed '" << annotated_message.tagged_message.message_type()
    +              << "' message from client " << annotated_message.sender;
    +    switch (annotated_message.tagged_message.message_type()) {
    +      case kShiftbossRegistrationResponseMessage: {
    +        foreman_client_id_ = annotated_message.sender;
    +        break;
    +      }
    +      case kShiftbossInitiateMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::ShiftbossInitiateMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processShiftbossInitiateMessage(proto.query_id(), proto.catalog_database_cache(), proto.query_context());
    +        break;
    +      }
    +      case kWorkOrderMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::WorkOrderMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        const std::size_t query_id = proto.query_id();
    +        DCHECK_EQ(1u, query_contexts_.count(query_id));
    +
    +        WorkOrder *work_order = WorkOrderFactory::ReconstructFromProto(proto.work_order(),
    +                                                                       &database_cache_,
    +                                                                       query_contexts_[query_id].get(),
    +                                                                       storage_manager_,
    +                                                                       shiftboss_client_id_,
    +                                                                       bus_);
    +
    +        unique_ptr<WorkerMessage> worker_message(
    +            WorkerMessage::WorkOrderMessage(work_order, proto.operator_index()));
    +
    +        TaggedMessage worker_tagged_message(worker_message.get(),
    +                                            sizeof(*worker_message),
    +                                            kWorkOrderMessage);
    +
    +        const size_t worker_index = getSchedulableWorker();
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded WorkOrderMessage (typed '" << kWorkOrderMessage
    +                  << "') from Foreman to worker " << worker_index;
    +
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           workers_->getClientID(worker_index),
    +                                           move(worker_tagged_message));
    +        break;
    +      }
    +      case kInitiateRebuildMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::InitiateRebuildMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processInitiateRebuildMessage(proto.query_id(),
    +                                      proto.operator_index(),
    +                                      proto.insert_destination_index(),
    +                                      proto.relation_id());
    +        break;
    +      }
    +      case kWorkOrderCompleteMessage:  // Fall through.
    +      case kRebuildWorkOrderCompleteMessage:
    +      case kDataPipelineMessage:
    +      case kWorkOrdersAvailableMessage:
    +      case kWorkOrderFeedbackMessage: {
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded typed '" << annotated_message.tagged_message.message_type()
    +                  << "' message from worker (client " << annotated_message.sender << ") to Foreman";
    +
    +        DCHECK_NE(foreman_client_id_, tmb::kClientIdNone);
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           foreman_client_id_,
    +                                           move(annotated_message.tagged_message));
    +        break;
    +      }
    +      case kQueryResultRelationMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::QueryResultRelationMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        for (int i = 0; i < proto.blocks_size(); ++i) {
    +          const block_id block = proto.blocks(i);
    +          storage_manager_->saveBlockOrBlob(block);
    +          if (storage_manager_->blockOrBlobIsLoaded(block)) {
    +            // NOTE(zuyu): eviction is required to avoid accesses to the query
    +            // result relation schema in CatalogDatabaseCache, for all query
    +            // optimizer execution generator unit tests and the single-process
    +            // Quickstep CLI.
    +            storage_manager_->evictBlockOrBlob(block);
    --- End diff --
    
    My point is that you need not evict the block explicitly. Mere ``saveBlockOrBlob`` call is enough. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep issue #64: Introduced Shiftboss for the distributed vers...

Posted by pateljm <gi...@git.apache.org>.
Github user pateljm commented on the issue:

    https://github.com/apache/incubator-quickstep/pull/64
  
    @zuyu Somewhere in the files, perhaps in QueryExecutionTypedefs.hpp, would be good to explain the RebuildWorkOrders highlighting when it is issued and the flow in which this is handled. Looks good otherwise. Thanks! 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #64: Introduced Shiftboss for the distribut...

Posted by hbdeshmukh <gi...@git.apache.org>.
Github user hbdeshmukh commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/64#discussion_r72339330
  
    --- Diff: query_execution/Shiftboss.cpp ---
    @@ -0,0 +1,357 @@
    +/**
    + *   Licensed under the Apache License, Version 2.0 (the "License");
    + *   you may not use this file except in compliance with the License.
    + *   You may obtain a copy of the License at
    + *
    + *       http://www.apache.org/licenses/LICENSE-2.0
    + *
    + *   Unless required by applicable law or agreed to in writing, software
    + *   distributed under the License is distributed on an "AS IS" BASIS,
    + *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + *   See the License for the specific language governing permissions and
    + *   limitations under the License.
    + **/
    +
    +#include "query_execution/Shiftboss.hpp"
    +
    +#include <cstddef>
    +#include <cstdlib>
    +#include <memory>
    +#include <string>
    +#include <unordered_map>
    +#include <utility>
    +#include <vector>
    +
    +#include "catalog/CatalogTypedefs.hpp"
    +#include "query_execution/QueryContext.hpp"
    +#include "query_execution/QueryExecutionMessages.pb.h"
    +#include "query_execution/QueryExecutionTypedefs.hpp"
    +#include "query_execution/QueryExecutionUtil.hpp"
    +#include "query_execution/WorkerMessage.hpp"
    +#include "relational_operators/RebuildWorkOrder.hpp"
    +#include "relational_operators/WorkOrderFactory.hpp"
    +#include "storage/InsertDestination.hpp"
    +#include "storage/StorageBlock.hpp"
    +#include "storage/StorageBlockInfo.hpp"
    +#include "storage/StorageManager.hpp"
    +#include "threading/ThreadUtil.hpp"
    +
    +#include "glog/logging.h"
    +
    +#include "tmb/address.h"
    +#include "tmb/id_typedefs.h"
    +#include "tmb/message_bus.h"
    +#include "tmb/message_style.h"
    +#include "tmb/tagged_message.h"
    +
    +using std::free;
    +using std::malloc;
    +using std::move;
    +using std::size_t;
    +using std::string;
    +using std::unique_ptr;
    +using std::vector;
    +
    +using tmb::TaggedMessage;
    +
    +namespace quickstep {
    +
    +class WorkOrder;
    +
    +void Shiftboss::run() {
    +  if (cpu_id_ >= 0) {
    +    // We can pin the shiftboss thread to a CPU if specified.
    +    ThreadUtil::BindToCPU(cpu_id_);
    +  }
    +
    +  for (;;) {
    +    // Receive() is a blocking call, causing this thread to sleep until next
    +    // message is received.
    +    AnnotatedMessage annotated_message(bus_->Receive(shiftboss_client_id_, 0, true));
    +    LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +              << "') received the typed '" << annotated_message.tagged_message.message_type()
    +              << "' message from client " << annotated_message.sender;
    +    switch (annotated_message.tagged_message.message_type()) {
    +      case kShiftbossRegistrationResponseMessage: {
    +        foreman_client_id_ = annotated_message.sender;
    +        break;
    +      }
    +      case kShiftbossInitiateMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::ShiftbossInitiateMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processShiftbossInitiateMessage(proto.query_id(), proto.catalog_database_cache(), proto.query_context());
    +        break;
    +      }
    +      case kWorkOrderMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::WorkOrderMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        const std::size_t query_id = proto.query_id();
    +        DCHECK_EQ(1u, query_contexts_.count(query_id));
    +
    +        WorkOrder *work_order = WorkOrderFactory::ReconstructFromProto(proto.work_order(),
    +                                                                       &database_cache_,
    +                                                                       query_contexts_[query_id].get(),
    +                                                                       storage_manager_,
    +                                                                       shiftboss_client_id_,
    +                                                                       bus_);
    +
    +        unique_ptr<WorkerMessage> worker_message(
    +            WorkerMessage::WorkOrderMessage(work_order, proto.operator_index()));
    +
    +        TaggedMessage worker_tagged_message(worker_message.get(),
    +                                            sizeof(*worker_message),
    +                                            kWorkOrderMessage);
    +
    +        const size_t worker_index = getSchedulableWorker();
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded WorkOrderMessage (typed '" << kWorkOrderMessage
    +                  << "') from Foreman to worker " << worker_index;
    +
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           workers_->getClientID(worker_index),
    +                                           move(worker_tagged_message));
    +        break;
    +      }
    +      case kInitiateRebuildMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::InitiateRebuildMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processInitiateRebuildMessage(proto.query_id(),
    +                                      proto.operator_index(),
    +                                      proto.insert_destination_index(),
    +                                      proto.relation_id());
    +        break;
    +      }
    +      case kWorkOrderCompleteMessage:  // Fall through.
    +      case kRebuildWorkOrderCompleteMessage:
    +      case kDataPipelineMessage:
    +      case kWorkOrdersAvailableMessage:
    +      case kWorkOrderFeedbackMessage: {
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded typed '" << annotated_message.tagged_message.message_type()
    +                  << "' message from worker (client " << annotated_message.sender << ") to Foreman";
    +
    +        DCHECK_NE(foreman_client_id_, tmb::kClientIdNone);
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           foreman_client_id_,
    +                                           move(annotated_message.tagged_message));
    +        break;
    +      }
    +      case kQueryResultRelationMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::QueryResultRelationMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        for (int i = 0; i < proto.blocks_size(); ++i) {
    +          const block_id block = proto.blocks(i);
    +          storage_manager_->saveBlockOrBlob(block);
    +          if (storage_manager_->blockOrBlobIsLoaded(block)) {
    +            // NOTE(zuyu): eviction is required to avoid accesses to the query
    +            // result relation schema in CatalogDatabaseCache, for all query
    +            // optimizer execution generator unit tests and the single-process
    +            // Quickstep CLI.
    +            storage_manager_->evictBlockOrBlob(block);
    --- End diff --
    
    But why do you want the block to be evicted? What's the harm if it stays in memory, no one else modifies it AND you also have a copy on the disk (because of ``saveBlockOrBlob`` call)?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #64: Introduced Shiftboss for the distribut...

Posted by zuyu <gi...@git.apache.org>.
Github user zuyu commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/64#discussion_r72525288
  
    --- Diff: query_execution/Shiftboss.cpp ---
    @@ -0,0 +1,357 @@
    +/**
    + *   Licensed under the Apache License, Version 2.0 (the "License");
    + *   you may not use this file except in compliance with the License.
    + *   You may obtain a copy of the License at
    + *
    + *       http://www.apache.org/licenses/LICENSE-2.0
    + *
    + *   Unless required by applicable law or agreed to in writing, software
    + *   distributed under the License is distributed on an "AS IS" BASIS,
    + *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + *   See the License for the specific language governing permissions and
    + *   limitations under the License.
    + **/
    +
    +#include "query_execution/Shiftboss.hpp"
    +
    +#include <cstddef>
    +#include <cstdlib>
    +#include <memory>
    +#include <string>
    +#include <unordered_map>
    +#include <utility>
    +#include <vector>
    +
    +#include "catalog/CatalogTypedefs.hpp"
    +#include "query_execution/QueryContext.hpp"
    +#include "query_execution/QueryExecutionMessages.pb.h"
    +#include "query_execution/QueryExecutionTypedefs.hpp"
    +#include "query_execution/QueryExecutionUtil.hpp"
    +#include "query_execution/WorkerMessage.hpp"
    +#include "relational_operators/RebuildWorkOrder.hpp"
    +#include "relational_operators/WorkOrderFactory.hpp"
    +#include "storage/InsertDestination.hpp"
    +#include "storage/StorageBlock.hpp"
    +#include "storage/StorageBlockInfo.hpp"
    +#include "storage/StorageManager.hpp"
    +#include "threading/ThreadUtil.hpp"
    +
    +#include "glog/logging.h"
    +
    +#include "tmb/address.h"
    +#include "tmb/id_typedefs.h"
    +#include "tmb/message_bus.h"
    +#include "tmb/message_style.h"
    +#include "tmb/tagged_message.h"
    +
    +using std::free;
    +using std::malloc;
    +using std::move;
    +using std::size_t;
    +using std::string;
    +using std::unique_ptr;
    +using std::vector;
    +
    +using tmb::TaggedMessage;
    +
    +namespace quickstep {
    +
    +class WorkOrder;
    +
    +void Shiftboss::run() {
    +  if (cpu_id_ >= 0) {
    +    // We can pin the shiftboss thread to a CPU if specified.
    +    ThreadUtil::BindToCPU(cpu_id_);
    +  }
    +
    +  for (;;) {
    +    // Receive() is a blocking call, causing this thread to sleep until next
    +    // message is received.
    +    AnnotatedMessage annotated_message(bus_->Receive(shiftboss_client_id_, 0, true));
    +    LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +              << "') received the typed '" << annotated_message.tagged_message.message_type()
    +              << "' message from client " << annotated_message.sender;
    +    switch (annotated_message.tagged_message.message_type()) {
    +      case kShiftbossRegistrationResponseMessage: {
    +        foreman_client_id_ = annotated_message.sender;
    +        break;
    +      }
    +      case kShiftbossInitiateMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::ShiftbossInitiateMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processShiftbossInitiateMessage(proto.query_id(), proto.catalog_database_cache(), proto.query_context());
    +        break;
    +      }
    +      case kWorkOrderMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::WorkOrderMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        const std::size_t query_id = proto.query_id();
    +        DCHECK_EQ(1u, query_contexts_.count(query_id));
    +
    +        WorkOrder *work_order = WorkOrderFactory::ReconstructFromProto(proto.work_order(),
    +                                                                       &database_cache_,
    +                                                                       query_contexts_[query_id].get(),
    +                                                                       storage_manager_,
    +                                                                       shiftboss_client_id_,
    +                                                                       bus_);
    +
    +        unique_ptr<WorkerMessage> worker_message(
    +            WorkerMessage::WorkOrderMessage(work_order, proto.operator_index()));
    +
    +        TaggedMessage worker_tagged_message(worker_message.get(),
    +                                            sizeof(*worker_message),
    +                                            kWorkOrderMessage);
    +
    +        const size_t worker_index = getSchedulableWorker();
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded WorkOrderMessage (typed '" << kWorkOrderMessage
    +                  << "') from Foreman to worker " << worker_index;
    +
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           workers_->getClientID(worker_index),
    +                                           move(worker_tagged_message));
    +        break;
    +      }
    +      case kInitiateRebuildMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::InitiateRebuildMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processInitiateRebuildMessage(proto.query_id(),
    +                                      proto.operator_index(),
    +                                      proto.insert_destination_index(),
    +                                      proto.relation_id());
    +        break;
    +      }
    +      case kWorkOrderCompleteMessage:  // Fall through.
    +      case kRebuildWorkOrderCompleteMessage:
    +      case kDataPipelineMessage:
    +      case kWorkOrdersAvailableMessage:
    +      case kWorkOrderFeedbackMessage: {
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded typed '" << annotated_message.tagged_message.message_type()
    +                  << "' message from worker (client " << annotated_message.sender << ") to Foreman";
    +
    +        DCHECK_NE(foreman_client_id_, tmb::kClientIdNone);
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           foreman_client_id_,
    +                                           move(annotated_message.tagged_message));
    +        break;
    +      }
    +      case kQueryResultRelationMessage: {
    --- End diff --
    
    No. Instead of adding a comment and waste few hours for the CI, I'll fix it in the next PR.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #64: Introduced Shiftboss for the distribut...

Posted by hbdeshmukh <gi...@git.apache.org>.
Github user hbdeshmukh commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/64#discussion_r72498531
  
    --- Diff: query_execution/QueryExecutionTypedefs.hpp ---
    @@ -73,9 +73,17 @@ enum QueryExecutionMessageType : message_type_id {
       kPoisonMessage,  // From the main thread to Foreman and Workers.
     
     #ifdef QUICKSTEP_DISTRIBUTED
    +  kShiftbossRegistrationMessage,  // From Shiftboss to Foreman.
    +  kShiftbossRegistrationResponseMessage,  // From Foreman to Shiftboss.
    +  kQueryInitiateMessage,  // From Foreman to Shiftboss.
    +  kQueryInitiateResponseMessage,  // From Shiftboss to Foreman.
    +
       kInitiateRebuildMessage,  // From Foreman to Shiftboss.
       kInitiateRebuildResponseMessage,  // From Shiftboss to Foreman.
     
    +  kQueryResultRelationMessage,  // From Foreman to Shiftboss.
    +  kQueryResultRelationResponseMessage,  // From Shiftboss to Foreman.
    --- End diff --
    
    There are two kinds of message patterns: X to Shiftboss and Shiftboss to X. I can see that ShiftBoss constructor has appropriate type of registration related code. To appropriately register X with the bus, have you already written the code or you plan to address it in the next PR?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #64: Introduced Shiftboss for the distribut...

Posted by zuyu <gi...@git.apache.org>.
Github user zuyu commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/64#discussion_r72276608
  
    --- Diff: query_execution/Shiftboss.hpp ---
    @@ -0,0 +1,238 @@
    +/**
    + *   Licensed under the Apache License, Version 2.0 (the "License");
    + *   you may not use this file except in compliance with the License.
    + *   You may obtain a copy of the License at
    + *
    + *       http://www.apache.org/licenses/LICENSE-2.0
    + *
    + *   Unless required by applicable law or agreed to in writing, software
    + *   distributed under the License is distributed on an "AS IS" BASIS,
    + *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + *   See the License for the specific language governing permissions and
    + *   limitations under the License.
    + **/
    +
    +#ifndef QUICKSTEP_QUERY_EXECUTION_SHIFTBOSS_HPP_
    +#define QUICKSTEP_QUERY_EXECUTION_SHIFTBOSS_HPP_
    +
    +#include <cstddef>
    +#include <memory>
    +
    +#include "catalog/CatalogDatabaseCache.hpp"
    +#include "catalog/CatalogTypedefs.hpp"
    +#include "query_execution/QueryContext.hpp"
    +#include "query_execution/QueryExecutionTypedefs.hpp"
    +#include "query_execution/WorkerDirectory.hpp"
    +#include "threading/Thread.hpp"
    +#include "utility/Macros.hpp"
    +
    +#include "glog/logging.h"
    +
    +#include "tmb/address.h"
    +#include "tmb/id_typedefs.h"
    +#include "tmb/message_bus.h"
    +
    +namespace quickstep {
    +
    +class StorageManager;
    +
    +namespace serialization {
    +class CatalogDatabase;
    +class QueryContext;
    +}  // namespace serialization
    +
    +/** \addtogroup QueryExecution
    + *  @{
    + */
    +
    +/**
    + * @brief The Shiftboss accepts workorder protos from shiftboss, and assigns
    + *        the workorders to workers.
    + **/
    +class Shiftboss : public Thread {
    + public:
    +  /**
    +   * @brief Constructor.
    +   *
    +   * @param bus A pointer to the TMB.
    +   * @param storage_manager The StorageManager to use.
    +   * @param workers A pointer to the WorkerDirectory.
    +   * @param cpu_id The ID of the CPU to which the Shiftboss thread can be pinned.
    +   *
    +   * @note If cpu_id is not specified, Shiftboss thread can be possibly moved
    +   *       around on different CPUs by the OS.
    +  **/
    +  Shiftboss(tmb::MessageBus *bus,
    +            StorageManager *storage_manager,
    +            WorkerDirectory *workers,
    +            const int cpu_id = -1)
    +      : bus_(DCHECK_NOTNULL(bus)),
    +        storage_manager_(DCHECK_NOTNULL(storage_manager)),
    +        workers_(DCHECK_NOTNULL(workers)),
    +        cpu_id_(cpu_id),
    +        shiftboss_client_id_(tmb::kClientIdNone),
    +        foreman_client_id_(tmb::kClientIdNone),
    +        max_msgs_per_worker_(1),
    +        start_worker_index_(0u) {
    +    // Check to have at least one Worker.
    +    DCHECK_GT(workers->getNumWorkers(), 0u);
    +
    +    shiftboss_client_id_ = bus_->Connect();
    +    LOG(INFO) << "Shiftboss TMB client ID: " << shiftboss_client_id_;
    +    DCHECK_NE(shiftboss_client_id_, tmb::kClientIdNone);
    +
    +    // Messages between Foreman and Shiftboss.
    +    bus_->RegisterClientAsSender(shiftboss_client_id_, kShiftbossRegistrationMessage);
    --- End diff --
    
    @hbdeshmukh I've added [comments](https://github.com/apache/incubator-quickstep/pull/64/files#diff-d520fea60cec1aee1ead8dd4e6b91124R208) regarding two-level message buses for the communications.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #64: Introduced Shiftboss for the distribut...

Posted by zuyu <gi...@git.apache.org>.
Github user zuyu commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/64#discussion_r72517135
  
    --- Diff: query_execution/Shiftboss.cpp ---
    @@ -0,0 +1,357 @@
    +/**
    + *   Licensed under the Apache License, Version 2.0 (the "License");
    + *   you may not use this file except in compliance with the License.
    + *   You may obtain a copy of the License at
    + *
    + *       http://www.apache.org/licenses/LICENSE-2.0
    + *
    + *   Unless required by applicable law or agreed to in writing, software
    + *   distributed under the License is distributed on an "AS IS" BASIS,
    + *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + *   See the License for the specific language governing permissions and
    + *   limitations under the License.
    + **/
    +
    +#include "query_execution/Shiftboss.hpp"
    +
    +#include <cstddef>
    +#include <cstdlib>
    +#include <memory>
    +#include <string>
    +#include <unordered_map>
    +#include <utility>
    +#include <vector>
    +
    +#include "catalog/CatalogTypedefs.hpp"
    +#include "query_execution/QueryContext.hpp"
    +#include "query_execution/QueryExecutionMessages.pb.h"
    +#include "query_execution/QueryExecutionTypedefs.hpp"
    +#include "query_execution/QueryExecutionUtil.hpp"
    +#include "query_execution/WorkerMessage.hpp"
    +#include "relational_operators/RebuildWorkOrder.hpp"
    +#include "relational_operators/WorkOrderFactory.hpp"
    +#include "storage/InsertDestination.hpp"
    +#include "storage/StorageBlock.hpp"
    +#include "storage/StorageBlockInfo.hpp"
    +#include "storage/StorageManager.hpp"
    +#include "threading/ThreadUtil.hpp"
    +
    +#include "glog/logging.h"
    +
    +#include "tmb/address.h"
    +#include "tmb/id_typedefs.h"
    +#include "tmb/message_bus.h"
    +#include "tmb/message_style.h"
    +#include "tmb/tagged_message.h"
    +
    +using std::free;
    +using std::malloc;
    +using std::move;
    +using std::size_t;
    +using std::string;
    +using std::unique_ptr;
    +using std::vector;
    +
    +using tmb::TaggedMessage;
    +
    +namespace quickstep {
    +
    +class WorkOrder;
    +
    +void Shiftboss::run() {
    +  if (cpu_id_ >= 0) {
    +    // We can pin the shiftboss thread to a CPU if specified.
    +    ThreadUtil::BindToCPU(cpu_id_);
    +  }
    +
    +  for (;;) {
    +    // Receive() is a blocking call, causing this thread to sleep until next
    +    // message is received.
    +    AnnotatedMessage annotated_message(bus_->Receive(shiftboss_client_id_, 0, true));
    +    LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +              << "') received the typed '" << annotated_message.tagged_message.message_type()
    +              << "' message from client " << annotated_message.sender;
    +    switch (annotated_message.tagged_message.message_type()) {
    +      case kShiftbossRegistrationResponseMessage: {
    +        foreman_client_id_ = annotated_message.sender;
    +        break;
    +      }
    +      case kShiftbossInitiateMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::ShiftbossInitiateMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processShiftbossInitiateMessage(proto.query_id(), proto.catalog_database_cache(), proto.query_context());
    +        break;
    +      }
    +      case kWorkOrderMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::WorkOrderMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        const std::size_t query_id = proto.query_id();
    +        DCHECK_EQ(1u, query_contexts_.count(query_id));
    +
    +        WorkOrder *work_order = WorkOrderFactory::ReconstructFromProto(proto.work_order(),
    +                                                                       &database_cache_,
    +                                                                       query_contexts_[query_id].get(),
    +                                                                       storage_manager_,
    +                                                                       shiftboss_client_id_,
    +                                                                       bus_);
    +
    +        unique_ptr<WorkerMessage> worker_message(
    +            WorkerMessage::WorkOrderMessage(work_order, proto.operator_index()));
    +
    +        TaggedMessage worker_tagged_message(worker_message.get(),
    +                                            sizeof(*worker_message),
    +                                            kWorkOrderMessage);
    +
    +        const size_t worker_index = getSchedulableWorker();
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded WorkOrderMessage (typed '" << kWorkOrderMessage
    +                  << "') from Foreman to worker " << worker_index;
    +
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           workers_->getClientID(worker_index),
    +                                           move(worker_tagged_message));
    +        break;
    +      }
    +      case kInitiateRebuildMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::InitiateRebuildMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processInitiateRebuildMessage(proto.query_id(),
    +                                      proto.operator_index(),
    +                                      proto.insert_destination_index(),
    +                                      proto.relation_id());
    +        break;
    +      }
    +      case kWorkOrderCompleteMessage:  // Fall through.
    +      case kRebuildWorkOrderCompleteMessage:
    +      case kDataPipelineMessage:
    +      case kWorkOrdersAvailableMessage:
    +      case kWorkOrderFeedbackMessage: {
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded typed '" << annotated_message.tagged_message.message_type()
    +                  << "' message from worker (client " << annotated_message.sender << ") to Foreman";
    +
    +        DCHECK_NE(foreman_client_id_, tmb::kClientIdNone);
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           foreman_client_id_,
    +                                           move(annotated_message.tagged_message));
    +        break;
    +      }
    +      case kQueryResultRelationMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::QueryResultRelationMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        for (int i = 0; i < proto.blocks_size(); ++i) {
    +          const block_id block = proto.blocks(i);
    +          storage_manager_->saveBlockOrBlob(block);
    +          if (storage_manager_->blockOrBlobIsLoaded(block)) {
    +            // NOTE(zuyu): eviction is required to avoid accesses to the query
    +            // result relation schema in CatalogDatabaseCache, for all query
    +            // optimizer execution generator unit tests and the single-process
    +            // Quickstep CLI.
    +            storage_manager_->evictBlockOrBlob(block);
    --- End diff --
    
    No, system will crash if we don't evict the blocks.
    
    You could play with my old code in `distributed-exe-gen-test` to confirm my statement.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #64: Introduced Shiftboss for the distribut...

Posted by zuyu <gi...@git.apache.org>.
Github user zuyu commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/64#discussion_r72516611
  
    --- Diff: query_execution/Shiftboss.cpp ---
    @@ -0,0 +1,357 @@
    +/**
    + *   Licensed under the Apache License, Version 2.0 (the "License");
    + *   you may not use this file except in compliance with the License.
    + *   You may obtain a copy of the License at
    + *
    + *       http://www.apache.org/licenses/LICENSE-2.0
    + *
    + *   Unless required by applicable law or agreed to in writing, software
    + *   distributed under the License is distributed on an "AS IS" BASIS,
    + *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + *   See the License for the specific language governing permissions and
    + *   limitations under the License.
    + **/
    +
    +#include "query_execution/Shiftboss.hpp"
    +
    +#include <cstddef>
    +#include <cstdlib>
    +#include <memory>
    +#include <string>
    +#include <unordered_map>
    +#include <utility>
    +#include <vector>
    +
    +#include "catalog/CatalogTypedefs.hpp"
    +#include "query_execution/QueryContext.hpp"
    +#include "query_execution/QueryExecutionMessages.pb.h"
    +#include "query_execution/QueryExecutionTypedefs.hpp"
    +#include "query_execution/QueryExecutionUtil.hpp"
    +#include "query_execution/WorkerMessage.hpp"
    +#include "relational_operators/RebuildWorkOrder.hpp"
    +#include "relational_operators/WorkOrderFactory.hpp"
    +#include "storage/InsertDestination.hpp"
    +#include "storage/StorageBlock.hpp"
    +#include "storage/StorageBlockInfo.hpp"
    +#include "storage/StorageManager.hpp"
    +#include "threading/ThreadUtil.hpp"
    +
    +#include "glog/logging.h"
    +
    +#include "tmb/address.h"
    +#include "tmb/id_typedefs.h"
    +#include "tmb/message_bus.h"
    +#include "tmb/message_style.h"
    +#include "tmb/tagged_message.h"
    +
    +using std::free;
    +using std::malloc;
    +using std::move;
    +using std::size_t;
    +using std::string;
    +using std::unique_ptr;
    +using std::vector;
    +
    +using tmb::TaggedMessage;
    +
    +namespace quickstep {
    +
    +class WorkOrder;
    +
    +void Shiftboss::run() {
    +  if (cpu_id_ >= 0) {
    +    // We can pin the shiftboss thread to a CPU if specified.
    +    ThreadUtil::BindToCPU(cpu_id_);
    +  }
    +
    +  for (;;) {
    +    // Receive() is a blocking call, causing this thread to sleep until next
    +    // message is received.
    +    AnnotatedMessage annotated_message(bus_->Receive(shiftboss_client_id_, 0, true));
    +    LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +              << "') received the typed '" << annotated_message.tagged_message.message_type()
    +              << "' message from client " << annotated_message.sender;
    +    switch (annotated_message.tagged_message.message_type()) {
    +      case kShiftbossRegistrationResponseMessage: {
    +        foreman_client_id_ = annotated_message.sender;
    +        break;
    +      }
    +      case kQueryInitiateMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::QueryInitiateMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processQueryInitiateMessage(proto.query_id(), proto.catalog_database_cache(), proto.query_context());
    +        break;
    +      }
    +      case kWorkOrderMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::WorkOrderMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        const std::size_t query_id = proto.query_id();
    +        DCHECK_EQ(1u, query_contexts_.count(query_id));
    +
    +        WorkOrder *work_order = WorkOrderFactory::ReconstructFromProto(proto.work_order(),
    +                                                                       &database_cache_,
    +                                                                       query_contexts_[query_id].get(),
    +                                                                       storage_manager_,
    +                                                                       shiftboss_client_id_,
    +                                                                       bus_);
    +
    +        unique_ptr<WorkerMessage> worker_message(
    +            WorkerMessage::WorkOrderMessage(work_order, proto.operator_index()));
    +
    +        TaggedMessage worker_tagged_message(worker_message.get(),
    +                                            sizeof(*worker_message),
    +                                            kWorkOrderMessage);
    +
    +        const size_t worker_index = getSchedulableWorker();
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded WorkOrderMessage (typed '" << kWorkOrderMessage
    +                  << "') from Foreman to worker " << worker_index;
    +
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           workers_->getClientID(worker_index),
    +                                           move(worker_tagged_message));
    +        break;
    +      }
    +      case kInitiateRebuildMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::InitiateRebuildMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processInitiateRebuildMessage(proto.query_id(),
    +                                      proto.operator_index(),
    +                                      proto.insert_destination_index(),
    +                                      proto.relation_id());
    +        break;
    +      }
    +      case kWorkOrderCompleteMessage:  // Fall through.
    +      case kRebuildWorkOrderCompleteMessage:
    +      case kDataPipelineMessage:
    +      case kWorkOrdersAvailableMessage:
    +      case kWorkOrderFeedbackMessage: {
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded typed '" << annotated_message.tagged_message.message_type()
    +                  << "' message from worker (client " << annotated_message.sender << ") to Foreman";
    +
    +        DCHECK_NE(foreman_client_id_, tmb::kClientIdNone);
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           foreman_client_id_,
    +                                           move(annotated_message.tagged_message));
    +        break;
    +      }
    +      case kQueryResultRelationMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::QueryResultRelationMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        for (int i = 0; i < proto.blocks_size(); ++i) {
    +          const block_id block = proto.blocks(i);
    --- End diff --
    
    Yes, single `Shiftboss`.
    
    `kCatalogRelationNewBlockMessage` is used whenever `InsertDestination` creates a new block, and then notifies `Foreman` to add the new block into `Catalog`. And I believe `Shiftboss` would forward this message to `Foreman`.
    
    `Foreman` knows the blocks from `QueryHandle`. Check out the link which contains the actual code.
    
    This won't be a main issue, but just add some performance overhead for serialization.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #64: Introduced Shiftboss for the distribut...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/incubator-quickstep/pull/64


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #64: Introduced Shiftboss for the distribut...

Posted by zuyu <gi...@git.apache.org>.
Github user zuyu commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/64#discussion_r72341258
  
    --- Diff: query_execution/Shiftboss.cpp ---
    @@ -0,0 +1,357 @@
    +/**
    + *   Licensed under the Apache License, Version 2.0 (the "License");
    + *   you may not use this file except in compliance with the License.
    + *   You may obtain a copy of the License at
    + *
    + *       http://www.apache.org/licenses/LICENSE-2.0
    + *
    + *   Unless required by applicable law or agreed to in writing, software
    + *   distributed under the License is distributed on an "AS IS" BASIS,
    + *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + *   See the License for the specific language governing permissions and
    + *   limitations under the License.
    + **/
    +
    +#include "query_execution/Shiftboss.hpp"
    +
    +#include <cstddef>
    +#include <cstdlib>
    +#include <memory>
    +#include <string>
    +#include <unordered_map>
    +#include <utility>
    +#include <vector>
    +
    +#include "catalog/CatalogTypedefs.hpp"
    +#include "query_execution/QueryContext.hpp"
    +#include "query_execution/QueryExecutionMessages.pb.h"
    +#include "query_execution/QueryExecutionTypedefs.hpp"
    +#include "query_execution/QueryExecutionUtil.hpp"
    +#include "query_execution/WorkerMessage.hpp"
    +#include "relational_operators/RebuildWorkOrder.hpp"
    +#include "relational_operators/WorkOrderFactory.hpp"
    +#include "storage/InsertDestination.hpp"
    +#include "storage/StorageBlock.hpp"
    +#include "storage/StorageBlockInfo.hpp"
    +#include "storage/StorageManager.hpp"
    +#include "threading/ThreadUtil.hpp"
    +
    +#include "glog/logging.h"
    +
    +#include "tmb/address.h"
    +#include "tmb/id_typedefs.h"
    +#include "tmb/message_bus.h"
    +#include "tmb/message_style.h"
    +#include "tmb/tagged_message.h"
    +
    +using std::free;
    +using std::malloc;
    +using std::move;
    +using std::size_t;
    +using std::string;
    +using std::unique_ptr;
    +using std::vector;
    +
    +using tmb::TaggedMessage;
    +
    +namespace quickstep {
    +
    +class WorkOrder;
    +
    +void Shiftboss::run() {
    +  if (cpu_id_ >= 0) {
    +    // We can pin the shiftboss thread to a CPU if specified.
    +    ThreadUtil::BindToCPU(cpu_id_);
    +  }
    +
    +  for (;;) {
    +    // Receive() is a blocking call, causing this thread to sleep until next
    +    // message is received.
    +    AnnotatedMessage annotated_message(bus_->Receive(shiftboss_client_id_, 0, true));
    +    LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +              << "') received the typed '" << annotated_message.tagged_message.message_type()
    +              << "' message from client " << annotated_message.sender;
    +    switch (annotated_message.tagged_message.message_type()) {
    +      case kShiftbossRegistrationResponseMessage: {
    +        foreman_client_id_ = annotated_message.sender;
    +        break;
    +      }
    +      case kShiftbossInitiateMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::ShiftbossInitiateMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processShiftbossInitiateMessage(proto.query_id(), proto.catalog_database_cache(), proto.query_context());
    +        break;
    +      }
    +      case kWorkOrderMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::WorkOrderMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        const std::size_t query_id = proto.query_id();
    +        DCHECK_EQ(1u, query_contexts_.count(query_id));
    +
    +        WorkOrder *work_order = WorkOrderFactory::ReconstructFromProto(proto.work_order(),
    +                                                                       &database_cache_,
    +                                                                       query_contexts_[query_id].get(),
    +                                                                       storage_manager_,
    +                                                                       shiftboss_client_id_,
    +                                                                       bus_);
    +
    +        unique_ptr<WorkerMessage> worker_message(
    +            WorkerMessage::WorkOrderMessage(work_order, proto.operator_index()));
    +
    +        TaggedMessage worker_tagged_message(worker_message.get(),
    +                                            sizeof(*worker_message),
    +                                            kWorkOrderMessage);
    +
    +        const size_t worker_index = getSchedulableWorker();
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded WorkOrderMessage (typed '" << kWorkOrderMessage
    +                  << "') from Foreman to worker " << worker_index;
    +
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           workers_->getClientID(worker_index),
    +                                           move(worker_tagged_message));
    +        break;
    +      }
    +      case kInitiateRebuildMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::InitiateRebuildMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processInitiateRebuildMessage(proto.query_id(),
    +                                      proto.operator_index(),
    +                                      proto.insert_destination_index(),
    +                                      proto.relation_id());
    +        break;
    +      }
    +      case kWorkOrderCompleteMessage:  // Fall through.
    +      case kRebuildWorkOrderCompleteMessage:
    +      case kDataPipelineMessage:
    +      case kWorkOrdersAvailableMessage:
    +      case kWorkOrderFeedbackMessage: {
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded typed '" << annotated_message.tagged_message.message_type()
    +                  << "' message from worker (client " << annotated_message.sender << ") to Foreman";
    +
    +        DCHECK_NE(foreman_client_id_, tmb::kClientIdNone);
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           foreman_client_id_,
    +                                           move(annotated_message.tagged_message));
    +        break;
    +      }
    +      case kQueryResultRelationMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::QueryResultRelationMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        for (int i = 0; i < proto.blocks_size(); ++i) {
    +          const block_id block = proto.blocks(i);
    +          storage_manager_->saveBlockOrBlob(block);
    +          if (storage_manager_->blockOrBlobIsLoaded(block)) {
    +            // NOTE(zuyu): eviction is required to avoid accesses to the query
    +            // result relation schema in CatalogDatabaseCache, for all query
    +            // optimizer execution generator unit tests and the single-process
    +            // Quickstep CLI.
    +            storage_manager_->evictBlockOrBlob(block);
    --- End diff --
    
    See my comments above. It is because of the block is associated with a schema. And w/ eviction, it uses the schema from the catalog cache, which was invalid to the cli.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] incubator-quickstep pull request #64: Introduced Shiftboss for the distribut...

Posted by zuyu <gi...@git.apache.org>.
Github user zuyu commented on a diff in the pull request:

    https://github.com/apache/incubator-quickstep/pull/64#discussion_r72338585
  
    --- Diff: query_execution/Shiftboss.cpp ---
    @@ -0,0 +1,357 @@
    +/**
    + *   Licensed under the Apache License, Version 2.0 (the "License");
    + *   you may not use this file except in compliance with the License.
    + *   You may obtain a copy of the License at
    + *
    + *       http://www.apache.org/licenses/LICENSE-2.0
    + *
    + *   Unless required by applicable law or agreed to in writing, software
    + *   distributed under the License is distributed on an "AS IS" BASIS,
    + *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + *   See the License for the specific language governing permissions and
    + *   limitations under the License.
    + **/
    +
    +#include "query_execution/Shiftboss.hpp"
    +
    +#include <cstddef>
    +#include <cstdlib>
    +#include <memory>
    +#include <string>
    +#include <unordered_map>
    +#include <utility>
    +#include <vector>
    +
    +#include "catalog/CatalogTypedefs.hpp"
    +#include "query_execution/QueryContext.hpp"
    +#include "query_execution/QueryExecutionMessages.pb.h"
    +#include "query_execution/QueryExecutionTypedefs.hpp"
    +#include "query_execution/QueryExecutionUtil.hpp"
    +#include "query_execution/WorkerMessage.hpp"
    +#include "relational_operators/RebuildWorkOrder.hpp"
    +#include "relational_operators/WorkOrderFactory.hpp"
    +#include "storage/InsertDestination.hpp"
    +#include "storage/StorageBlock.hpp"
    +#include "storage/StorageBlockInfo.hpp"
    +#include "storage/StorageManager.hpp"
    +#include "threading/ThreadUtil.hpp"
    +
    +#include "glog/logging.h"
    +
    +#include "tmb/address.h"
    +#include "tmb/id_typedefs.h"
    +#include "tmb/message_bus.h"
    +#include "tmb/message_style.h"
    +#include "tmb/tagged_message.h"
    +
    +using std::free;
    +using std::malloc;
    +using std::move;
    +using std::size_t;
    +using std::string;
    +using std::unique_ptr;
    +using std::vector;
    +
    +using tmb::TaggedMessage;
    +
    +namespace quickstep {
    +
    +class WorkOrder;
    +
    +void Shiftboss::run() {
    +  if (cpu_id_ >= 0) {
    +    // We can pin the shiftboss thread to a CPU if specified.
    +    ThreadUtil::BindToCPU(cpu_id_);
    +  }
    +
    +  for (;;) {
    +    // Receive() is a blocking call, causing this thread to sleep until next
    +    // message is received.
    +    AnnotatedMessage annotated_message(bus_->Receive(shiftboss_client_id_, 0, true));
    +    LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +              << "') received the typed '" << annotated_message.tagged_message.message_type()
    +              << "' message from client " << annotated_message.sender;
    +    switch (annotated_message.tagged_message.message_type()) {
    +      case kShiftbossRegistrationResponseMessage: {
    +        foreman_client_id_ = annotated_message.sender;
    +        break;
    +      }
    +      case kShiftbossInitiateMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::ShiftbossInitiateMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processShiftbossInitiateMessage(proto.query_id(), proto.catalog_database_cache(), proto.query_context());
    +        break;
    +      }
    +      case kWorkOrderMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::WorkOrderMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        const std::size_t query_id = proto.query_id();
    +        DCHECK_EQ(1u, query_contexts_.count(query_id));
    +
    +        WorkOrder *work_order = WorkOrderFactory::ReconstructFromProto(proto.work_order(),
    +                                                                       &database_cache_,
    +                                                                       query_contexts_[query_id].get(),
    +                                                                       storage_manager_,
    +                                                                       shiftboss_client_id_,
    +                                                                       bus_);
    +
    +        unique_ptr<WorkerMessage> worker_message(
    +            WorkerMessage::WorkOrderMessage(work_order, proto.operator_index()));
    +
    +        TaggedMessage worker_tagged_message(worker_message.get(),
    +                                            sizeof(*worker_message),
    +                                            kWorkOrderMessage);
    +
    +        const size_t worker_index = getSchedulableWorker();
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded WorkOrderMessage (typed '" << kWorkOrderMessage
    +                  << "') from Foreman to worker " << worker_index;
    +
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           workers_->getClientID(worker_index),
    +                                           move(worker_tagged_message));
    +        break;
    +      }
    +      case kInitiateRebuildMessage: {
    +        const TaggedMessage &tagged_message = annotated_message.tagged_message;
    +
    +        serialization::InitiateRebuildMessage proto;
    +        CHECK(proto.ParseFromArray(tagged_message.message(), tagged_message.message_bytes()));
    +
    +        processInitiateRebuildMessage(proto.query_id(),
    +                                      proto.operator_index(),
    +                                      proto.insert_destination_index(),
    +                                      proto.relation_id());
    +        break;
    +      }
    +      case kWorkOrderCompleteMessage:  // Fall through.
    +      case kRebuildWorkOrderCompleteMessage:
    +      case kDataPipelineMessage:
    +      case kWorkOrdersAvailableMessage:
    +      case kWorkOrderFeedbackMessage: {
    +        LOG(INFO) << "Shiftboss (id '" << shiftboss_client_id_
    +                  << "') forwarded typed '" << annotated_message.tagged_message.message_type()
    +                  << "' message from worker (client " << annotated_message.sender << ") to Foreman";
    +
    +        DCHECK_NE(foreman_client_id_, tmb::kClientIdNone);
    +        QueryExecutionUtil::SendTMBMessage(bus_,
    +                                           shiftboss_client_id_,
    +                                           foreman_client_id_,
    +                                           move(annotated_message.tagged_message));
    +        break;
    +      }
    +      case kQueryResultRelationMessage: {
    --- End diff --
    
    Used to flush the query result.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---