Commit 808dc849 authored by Dominik Charousset's avatar Dominik Charousset

Use policy-based design for scheduler

This patch separates interface and implementation of scheduling coordinator and
worker. The implementation is now outsourced into two policies: a steal policy
and a (de)queue policy. The previously hardwired implementation is split into
the two policy classes `iterative_stealing` and `fork_join`.
parent 1f9aa431
...@@ -22,24 +22,11 @@ ...@@ -22,24 +22,11 @@
#include <atomic> #include <atomic>
#include <cstddef> // size_t #include <cstddef> // size_t
namespace cppa { #include "cppa/fwd.hpp"
class node_id;
namespace scheduler {
class coordinator;
}
} // namespace cppa
namespace cppa { namespace cppa {
namespace detail { namespace detail {
class logging;
class message_data;
class group_manager;
class actor_registry;
class uniform_type_info_map;
class abstract_singleton { class abstract_singleton {
public: public:
...@@ -72,7 +59,7 @@ class singletons { ...@@ -72,7 +59,7 @@ class singletons {
static node_id get_node_id(); static node_id get_node_id();
static scheduler::coordinator* get_scheduling_coordinator(); static scheduler::abstract_coordinator* get_scheduling_coordinator();
static group_manager* get_group_manager(); static group_manager* get_group_manager();
......
...@@ -23,7 +23,7 @@ namespace cppa { ...@@ -23,7 +23,7 @@ namespace cppa {
class resumable; class resumable;
/* /**
* @brief Identifies an execution unit, e.g., a worker thread of the scheduler. * @brief Identifies an execution unit, e.g., a worker thread of the scheduler.
*/ */
class execution_unit { class execution_unit {
...@@ -32,7 +32,7 @@ class execution_unit { ...@@ -32,7 +32,7 @@ class execution_unit {
virtual ~execution_unit(); virtual ~execution_unit();
/* /**
* @brief Enqueues @p ptr to the job list of the execution unit. * @brief Enqueues @p ptr to the job list of the execution unit.
* @warning Must only be called from a {@link resumable} currently * @warning Must only be called from a {@link resumable} currently
* executed by this execution unit. * executed by this execution unit.
......
...@@ -29,11 +29,11 @@ class intrusive_ptr; ...@@ -29,11 +29,11 @@ class intrusive_ptr;
// classes // classes
class actor; class actor;
class group; class group;
class message;
class channel; class channel;
class node_id; class node_id;
class behavior; class behavior;
class resumable; class resumable;
class message;
class actor_addr; class actor_addr;
class local_actor; class local_actor;
class actor_proxy; class actor_proxy;
...@@ -64,10 +64,23 @@ template<typename T, typename U> ...@@ -64,10 +64,23 @@ template<typename T, typename U>
T actor_cast(const U&); T actor_cast(const U&);
namespace io { namespace io {
class broker;
class middleman;
} // namespace io
class broker; namespace scheduler {
class abstract_worker;
class abstract_coordinator;
} // namespace scheduler
} // namespace io namespace detail {
class logging;
class singletons;
class message_data;
class group_manager;
class actor_registry;
class uniform_type_info_map;
} // namespace detail
} // namespace cppa } // namespace cppa
......
/******************************************************************************\
* ___ __ *
* /\_ \ __/\ \ *
* \//\ \ /\_\ \ \____ ___ _____ _____ __ *
* \ \ \ \/\ \ \ '__`\ /'___\/\ '__`\/\ '__`\ /'__`\ *
* \_\ \_\ \ \ \ \L\ \/\ \__/\ \ \L\ \ \ \L\ \/\ \L\.\_ *
* /\____\\ \_\ \_,__/\ \____\\ \ ,__/\ \ ,__/\ \__/.\_\ *
* \/____/ \/_/\/___/ \/____/ \ \ \/ \ \ \/ \/__/\/_/ *
* \ \_\ \ \_\ *
* \/_/ \/_/ *
* *
* Copyright (C) 2011 - 2014 *
* Dominik Charousset <dominik.charousset (at) haw-hamburg.de> *
* *
* Distributed under the Boost Software License, Version 1.0. See *
* accompanying file LICENSE or copy at http://www.boost.org/LICENSE_1_0.txt *
\******************************************************************************/
#ifndef CPPA_POLICY_FORK_JOIN_HPP
#define CPPA_POLICY_FORK_JOIN_HPP
#include <chrono>
#include <vector>
#include <thread>
#include "cppa/resumable.hpp"
#include "cppa/detail/producer_consumer_list.hpp"
namespace cppa {
namespace policy {
/**
* @brief An implementation of the {@link job_queue_policy} concept for
* fork-join like processing of actors.
*
* This work-stealing fork-join implementation uses two queues: a
* synchronized queue accessible by other threads and an internal queue.
* Access to the synchronized queue is minimized.
* The reasoning behind this design decision is that it
* has been shown that stealing actually is very rare for most workloads [1].
* Hence, implementations should focus on the performance in
* the non-stealing case. For this reason, each worker has an exposed
* job queue that can be accessed by the central scheduler instance as
* well as other workers, but it also has a private job list it is
* currently working on. To account for the load balancing aspect, each
* worker makes sure that at least one job is left in its exposed queue
* to allow other workers to steal it.
*
* @relates job_queue_policy
*/
class fork_join {
public:
fork_join() = default;
fork_join(fork_join&& other) {
// delegate to move assignment operator
*this = std::move(other);
}
fork_join& operator=(fork_join&& other) {
m_private_queue = std::move(other.m_private_queue);
auto next = [&] { return other.m_exposed_queue.try_pop(); };
for (auto j = next(); j != nullptr; j = next()) {
m_exposed_queue.push_back(j);
}
return *this;
}
/**
* @brief A thead-safe queue implementation.
*/
using sync_queue = detail::producer_consumer_list<resumable>;
/**
* @brief A queue implementation supporting fast push and pop
* operations. Note that we do dequeue from the back of the
* queue.
*/
using priv_queue = std::vector<resumable*>;
template<class Worker>
inline void external_enqueue(Worker*, resumable* job) {
m_exposed_queue.push_back(job);
}
template<class Worker>
inline void internal_enqueue(Worker*, resumable* job) {
// give others the opportunity to steal from us
if (m_exposed_queue.empty()) {
if (m_private_queue.empty()) {
m_exposed_queue.push_back(job);
} else {
m_exposed_queue.push_back(m_private_queue.front());
m_private_queue.erase(m_private_queue.begin());
m_private_queue.push_back(job);
}
} else {
m_private_queue.push_back(job);
}
}
template<class Worker>
inline resumable* try_external_dequeue(Worker*) {
return m_exposed_queue.try_pop();
}
template<class Worker>
inline resumable* internal_dequeue(Worker* self) {
resumable* job;
auto local_poll = [&]() -> bool {
if (!m_private_queue.empty()) {
job = m_private_queue.back();
m_private_queue.pop_back();
return true;
}
return false;
};
auto aggressive_poll = [&]() -> bool {
for (int i = 1; i < 101; ++i) {
job = m_exposed_queue.try_pop();
if (job) {
return true;
}
// try to steal every 10 poll attempts
if ((i % 10) == 0) {
job = self->raid();
if (job) {
return true;
}
}
std::this_thread::yield();
}
return false;
};
auto moderate_poll = [&]() -> bool {
for (int i = 1; i < 550; ++i) {
job = m_exposed_queue.try_pop();
if (job) {
return true;
}
// try to steal every 5 poll attempts
if ((i % 5) == 0) {
job = self->raid();
if (job) {
return true;
}
}
std::this_thread::sleep_for(std::chrono::microseconds(50));
}
return false;
};
auto relaxed_poll = [&]() -> bool {
for (;;) {
job = m_exposed_queue.try_pop();
if (job) {
return true;
}
// always try to steal at this stage
job = self->raid();
if (job) {
return true;
}
std::this_thread::sleep_for(std::chrono::milliseconds(10));
}
};
local_poll() || aggressive_poll() || moderate_poll() || relaxed_poll();
return job;
}
template<class Worker>
void clear_internal_queue(Worker*) {
// give others the opportunity to steal unfinished jobs
for (auto ptr : m_private_queue) {
m_exposed_queue.push_back(ptr);
}
m_private_queue.clear();
}
template<class Worker>
void assert_stealable(Worker*) {
// give others the opportunity to steal from us
if (m_private_queue.size() > 1 && m_exposed_queue.empty()) {
m_exposed_queue.push_back(m_private_queue.front());
m_private_queue.erase(m_private_queue.begin());
}
}
template<class Worker, typename UnaryFunction>
void consume_all(Worker*, UnaryFunction f) {
for (auto job : m_private_queue) {
f(job);
}
m_private_queue.clear();
auto next = [&] { return m_exposed_queue.try_pop(); };
for (auto job = next(); job != nullptr; job = next()) {
f(job);
}
}
private:
// this queue is exposed to others, i.e., other workers
// may attempt to steal jobs from it and the central scheduling
// unit can push new jobs to the queue
sync_queue m_exposed_queue;
// internal job queue
priv_queue m_private_queue;
};
} // namespace policy
} // namespace cppa
#endif // CPPA_POLICY_FORK_JOIN_HPP
/******************************************************************************\
* ___ __ *
* /\_ \ __/\ \ *
* \//\ \ /\_\ \ \____ ___ _____ _____ __ *
* \ \ \ \/\ \ \ '__`\ /'___\/\ '__`\/\ '__`\ /'__`\ *
* \_\ \_\ \ \ \ \L\ \/\ \__/\ \ \L\ \ \ \L\ \/\ \L\.\_ *
* /\____\\ \_\ \_,__/\ \____\\ \ ,__/\ \ ,__/\ \__/.\_\ *
* \/____/ \/_/\/___/ \/____/ \ \ \/ \ \ \/ \/__/\/_/ *
* \ \_\ \ \_\ *
* \/_/ \/_/ *
* *
* Copyright (C) 2011 - 2014 *
* Dominik Charousset <dominik.charousset (at) haw-hamburg.de> *
* *
* Distributed under the Boost Software License, Version 1.0. See *
* accompanying file LICENSE or copy at http://www.boost.org/LICENSE_1_0.txt *
\******************************************************************************/
#ifndef CPPA_POLICY_ITERATIVE_STEALING_HPP
#define CPPA_POLICY_ITERATIVE_STEALING_HPP
#include <cstddef>
#include "cppa/fwd.hpp"
namespace cppa {
namespace policy {
/**
* @brief An implementation of the {@link steal_policy} concept
* that iterates over all other workers when stealing.
*
* @relates steal_policy
*/
class iterative_stealing {
public:
constexpr iterative_stealing() : m_victim(0) { }
template<class Worker>
resumable* raid(Worker* self) {
// try once to steal from anyone
auto inc = [](size_t arg) -> size_t {
return arg + 1;
};
auto dec = [](size_t arg) -> size_t {
return arg - 1;
};
// reduce probability of 'steal collisions' by letting
// half the workers pick victims by increasing IDs and
// the other half by decreasing IDs
size_t (*next)(size_t) = (self->id() % 2) == 0 ? inc : dec;
auto n = self->parent()->num_workers();
for (size_t i = 0; i < n; ++i) {
m_victim = next(m_victim) % n;
if (m_victim != self->id()) {
auto job = self->parent()->worker_by_id(m_victim)->try_steal();
if (job) {
return job;
}
}
}
return nullptr;
}
private:
size_t m_victim;
};
} // namespace policy
} // namespace cppa
#endif // CPPA_POLICY_ITERATIVE_STEALING_HPP
/******************************************************************************\
* ___ __ *
* /\_ \ __/\ \ *
* \//\ \ /\_\ \ \____ ___ _____ _____ __ *
* \ \ \ \/\ \ \ '__`\ /'___\/\ '__`\/\ '__`\ /'__`\ *
* \_\ \_\ \ \ \ \L\ \/\ \__/\ \ \L\ \ \ \L\ \/\ \L\.\_ *
* /\____\\ \_\ \_,__/\ \____\\ \ ,__/\ \ ,__/\ \__/.\_\ *
* \/____/ \/_/\/___/ \/____/ \ \ \/ \ \ \/ \/__/\/_/ *
* \ \_\ \ \_\ *
* \/_/ \/_/ *
* *
* Copyright (C) 2011 - 2014 *
* Dominik Charousset <dominik.charousset (at) haw-hamburg.de> *
* *
* Distributed under the Boost Software License, Version 1.0. See *
* accompanying file LICENSE or copy at http://www.boost.org/LICENSE_1_0.txt *
\******************************************************************************/
#ifndef CPPA_POLICY_JOB_QUEUE_POLICY_HPP
#define CPPA_POLICY_JOB_QUEUE_POLICY_HPP
#include "cppa/fwd.hpp"
namespace cppa {
namespace policy {
/**
* @brief This concept class describes the interface of a policy class
* for managing the queue(s) of a scheduler worker.
*/
class job_queue_policy {
public:
/**
* @brief Enqueues a new job to the worker's queue from an
* external source, i.e., from any other thread.
*/
template<class Worker>
void external_enqueue(Worker* self, resumable* job);
/**
* @brief Enqueues a new job to the worker's queue from an
* internal source, i.e., from the same thread.
*/
template<class Worker>
void internal_enqueue(Worker* self, resumable* job);
/**
* @brief Called by external sources to try to dequeue an element.
* Returns @p nullptr if no element could be dequeued immediately.
*/
template<class Worker>
resumable* try_external_dequeue(Worker* self);
/**
* @brief Called by the worker itself to acquire a new job.
* Blocks until a job could be dequeued.
*/
template<class Worker>
resumable* internal_dequeue(Worker* self);
/**
* @brief Moves all elements form the internal queue to the external queue.
*/
template<class Worker>
void clear_internal_queue(Worker* self);
/**
* @brief Tries to move at least one element from the internal queue ot
* the external queue if possible to allow others to steal from us.
*/
template<class Worker>
void assert_stealable(Worker* self);
/**
* @brief Applies given functor to all elements in all queues and
* clears all queues afterwards.
*/
template<class Worker, typename UnaryFunction>
void consume_all(Worker* self, UnaryFunction f);
};
} // namespace policy
} // namespace cppa
#endif // CPPA_POLICY_JOB_QUEUE_POLICY_HPP
/******************************************************************************\
* ___ __ *
* /\_ \ __/\ \ *
* \//\ \ /\_\ \ \____ ___ _____ _____ __ *
* \ \ \ \/\ \ \ '__`\ /'___\/\ '__`\/\ '__`\ /'__`\ *
* \_\ \_\ \ \ \ \L\ \/\ \__/\ \ \L\ \ \ \L\ \/\ \L\.\_ *
* /\____\\ \_\ \_,__/\ \____\\ \ ,__/\ \ ,__/\ \__/.\_\ *
* \/____/ \/_/\/___/ \/____/ \ \ \/ \ \ \/ \/__/\/_/ *
* \ \_\ \ \_\ *
* \/_/ \/_/ *
* *
* Copyright (C) 2011 - 2014 *
* Dominik Charousset <dominik.charousset (at) haw-hamburg.de> *
* *
* Distributed under the Boost Software License, Version 1.0. See *
* accompanying file LICENSE or copy at http://www.boost.org/LICENSE_1_0.txt *
\******************************************************************************/
#ifndef CPPA_POLICY_STEAL_POLICY_HPP
#define CPPA_POLICY_STEAL_POLICY_HPP
#include <cstddef>
#include "cppa/fwd.hpp"
namespace cppa {
namespace policy {
/**
* @brief This concept class describes the interface of a policy class
* for stealing jobs from other workers.
*/
class steal_policy {
public:
/**
* @brief Go on a raid in quest for a shiny new job. Returns @p nullptr
* if no other worker provided any work to steal.
*/
template<class Worker>
resumable* raid(Worker* self);
};
} // namespace policy
} // namespace cppa
#endif // CPPA_POLICY_STEAL_POLICY_HPP
...@@ -26,57 +26,143 @@ ...@@ -26,57 +26,143 @@
#include <functional> #include <functional>
#include <type_traits> #include <type_traits>
#include "cppa/fwd.hpp"
#include "cppa/atom.hpp" #include "cppa/atom.hpp"
#include "cppa/actor.hpp" #include "cppa/actor.hpp"
#include "cppa/channel.hpp" #include "cppa/channel.hpp"
#include "cppa/message.hpp" #include "cppa/message.hpp"
#include "cppa/duration.hpp" #include "cppa/duration.hpp"
#include "cppa/resumable.hpp"
#include "cppa/attachable.hpp" #include "cppa/attachable.hpp"
#include "cppa/scoped_actor.hpp" #include "cppa/scoped_actor.hpp"
#include "cppa/spawn_options.hpp" #include "cppa/spawn_options.hpp"
#include "cppa/execution_unit.hpp" #include "cppa/execution_unit.hpp"
#include "cppa/detail/logging.hpp"
#include "cppa/detail/producer_consumer_list.hpp" #include "cppa/detail/producer_consumer_list.hpp"
namespace cppa { namespace cppa {
namespace scheduler {
class resumable; class abstract_coordinator;
namespace detail { class abstract_worker : public execution_unit {
class singletons;
}
namespace scheduler { friend class abstract_coordinator;
public:
/**
* @brief Attempt to steal an element from this worker.
*/
virtual resumable* try_steal() = 0;
/**
* @brief Enqueues a new job to the worker's queue from an external
* source, i.e., from any other thread.
*/
virtual void external_enqueue(resumable*) = 0;
/**
* @brief Starts the thread of this worker.
*/
virtual void start(size_t id, abstract_coordinator* parent) = 0;
};
class abstract_coordinator {
friend class detail::singletons;
public:
explicit abstract_coordinator(size_t num_worker_threads);
virtual ~abstract_coordinator();
/**
* @brief Returns a handle to the central printing actor.
*/
actor printer() const;
/**
* @brief Puts @p what into the queue of a randomly chosen worker.
*/
void enqueue(resumable* what);
template<typename Duration, typename... Data>
void delayed_send(Duration rel_time, actor_addr from, channel to,
message_id mid, message data) {
m_timer->enqueue(invalid_actor_addr, message_id::invalid,
make_message(atom("_Send"), duration{rel_time},
std::move(from), std::move(to), mid,
std::move(data)),
nullptr);
}
inline size_t num_workers() const {
return m_num_workers;
}
virtual abstract_worker* worker_by_id(size_t id) = 0;
protected:
abstract_coordinator();
virtual void initialize();
virtual void stop();
private:
// Creates a default instance.
static abstract_coordinator* create_singleton();
inline void dispose() {
delete this;
}
intrusive_ptr<blocking_actor> m_timer;
scoped_actor m_printer;
// ID of the worker receiving the next enqueue
std::atomic<size_t> m_next_worker;
size_t m_num_workers;
std::thread m_timer_thread;
std::thread m_printer_thread;
class coordinator; };
/** /**
* @brief A work-stealing scheduling worker. * @brief A work-stealing scheduling worker.
*
* The work-stealing implementation of libcppa minimizes access to the
* synchronized queue. The reasoning behind this design decision is that it
* has been shown that stealing actually is very rare for most workloads [1].
* Hence, implementations should focus on the performance in
* the non-stealing case. For this reason, each worker has an exposed
* job queue that can be accessed by the central scheduler instance as
* well as other workers, but it also has a private job list it is
* currently working on. To account for the load balancing aspect, each
* worker makes sure that at least one job is left in its exposed queue
* to allow other workers to steal it.
*
* [1] http://dl.acm.org/citation.cfm?doid=2398857.2384639
*/ */
class worker : public execution_unit { template<class StealPolicy, class JobQueuePolicy>
class worker : public abstract_worker {
friend class coordinator;
public: public:
worker() = default; worker() = default;
worker(worker&&); worker(worker&& other) {
*this = std::move(other); // delegate to move assignment operator
}
worker& operator=(worker&&); worker& operator=(worker&& other) {
// cannot be moved once m_this_thread is up and running
auto running = [](std::thread& t) {
return t.get_id() != std::thread::id{};
};
if (running(m_this_thread) || running(other.m_this_thread)) {
throw std::runtime_error("running workers cannot be moved");
}
m_queue_policy = std::move(other.m_queue_policy);
m_steal_policy = std::move(other.m_steal_policy);
return *this;
}
worker(const worker&) = delete; worker(const worker&) = delete;
...@@ -89,13 +175,17 @@ class worker : public execution_unit { ...@@ -89,13 +175,17 @@ class worker : public execution_unit {
/** /**
* @brief Attempt to steal an element from the exposed job queue. * @brief Attempt to steal an element from the exposed job queue.
*/ */
job_ptr try_steal(); job_ptr try_steal() override {
return m_queue_policy.try_external_dequeue(this);
}
/** /**
* @brief Enqueues a new job to the worker's queue from an external * @brief Enqueues a new job to the worker's queue from an external
* source, i.e., from any other thread. * source, i.e., from any other thread.
*/ */
void external_enqueue(job_ptr); void external_enqueue(job_ptr job) override {
m_queue_policy.external_enqueue(this, job);
}
/** /**
* @brief Enqueues a new job to the worker's queue from an internal * @brief Enqueues a new job to the worker's queue from an internal
...@@ -103,23 +193,64 @@ class worker : public execution_unit { ...@@ -103,23 +193,64 @@ class worker : public execution_unit {
* this worker. * this worker.
* @warning Must not be called from other threads. * @warning Must not be called from other threads.
*/ */
void exec_later(job_ptr) override; void exec_later(job_ptr job) override {
m_queue_policy.internal_enqueue(this, job);
}
private: // go on a raid in quest for a shiny new job
job_ptr raid() {
return m_steal_policy.raid(this);
}
void start(size_t id, coordinator* parent); // called from coordinator inline abstract_coordinator* parent() {
return m_parent;
}
void run(); // work loop inline size_t id() const {
return m_id;
}
inline std::thread& get_thread() {
return m_this_thread;
}
void detach_all() {
m_queue_policy.consume_all(this, [](resumable* job) {
job->detach_from_scheduler();
});
}
job_ptr raid(); // go on a raid in quest for a shiny new job void start(size_t id, abstract_coordinator* parent) override {
m_id = id;
m_parent = parent;
auto this_worker = this;
m_this_thread = std::thread{[this_worker] { this_worker->run(); }};
}
// this queue is exposed to others, i.e., other workers private:
// may attempt to steal jobs from it and the central scheduling
// unit can push new jobs to the queue
job_queue m_exposed_queue;
// internal job stack void run() {
std::vector<job_ptr> m_job_list; CPPA_LOG_TRACE("worker with ID " << m_id);
// scheduling loop
for (;;) {
auto job = m_queue_policy.internal_dequeue(this);
CPPA_PUSH_AID_FROM_PTR(dynamic_cast<abstract_actor*>(job));
switch (job->resume(this)) {
case resumable::done: {
job->detach_from_scheduler();
break;
}
case resumable::resume_later: {
break;
}
case resumable::shutdown_execution_unit: {
m_queue_policy.clear_internal_queue(this);
return;
}
}
m_queue_policy.assert_stealable(this);
}
}
// the worker's thread // the worker's thread
std::thread m_this_thread; std::thread m_this_thread;
...@@ -127,73 +258,57 @@ class worker : public execution_unit { ...@@ -127,73 +258,57 @@ class worker : public execution_unit {
// the worker's ID received from scheduler // the worker's ID received from scheduler
size_t m_id; size_t m_id;
// the ID of the last victim we stole from abstract_coordinator* m_parent;
size_t m_last_victim;
coordinator* m_parent; JobQueuePolicy m_queue_policy;
StealPolicy m_steal_policy;
}; };
/** /**
* @brief Central scheduling interface. * @brief Central scheduling interface.
*/ */
class coordinator { template<class StealPolicy, class JobQueuePolicy>
class coordinator : public abstract_coordinator {
friend class detail::singletons; using super = abstract_coordinator;
public: public:
class shutdown_helper; coordinator(size_t nw = std::thread::hardware_concurrency()) : super(nw) {
// nop
/** }
* @brief Returns a handle to the central printing actor.
*/
actor printer() const;
/** using worker_type = worker<StealPolicy, JobQueuePolicy>;
* @brief Puts @p what into the queue of a randomly chosen worker.
*/
void enqueue(resumable* what);
template<typename Duration, typename... Data> abstract_worker* worker_by_id(size_t id) override {
void delayed_send(Duration rel_time, actor_addr from, channel to, return &m_workers[id];
message_id mid, message data) {
m_timer->enqueue(invalid_actor_addr, message_id::invalid,
make_message(atom("_Send"), duration{rel_time},
std::move(from), std::move(to), mid,
std::move(data)),
nullptr);
} }
inline size_t num_workers() const { protected:
return static_cast<unsigned>(m_workers.size());
void initialize() override {
super::initialize();
// create & start workers
m_workers.resize(num_workers());
for (size_t i = 0; i < num_workers(); ++i) {
m_workers[i].start(i, this);
}
} }
inline worker& worker_by_id(size_t id) { return m_workers[id]; } void stop() override {
super::stop();
// wait until all actors are done
for (auto& w : m_workers) w.get_thread().join();
// clear all queues
for (auto& w : m_workers) w.detach_all();
}
private: private:
static coordinator* create_singleton();
coordinator();
inline void dispose() { delete this; }
void initialize();
void stop();
intrusive_ptr<blocking_actor> m_timer;
scoped_actor m_printer;
std::thread m_timer_thread;
std::thread m_printer_thread;
// ID of the worker receiving the next enqueue
std::atomic<size_t> m_next_worker;
// vector of size std::thread::hardware_concurrency() // vector of size std::thread::hardware_concurrency()
std::vector<worker> m_workers; std::vector<worker_type> m_workers;
}; };
......
...@@ -23,6 +23,7 @@ ...@@ -23,6 +23,7 @@
#include <condition_variable> #include <condition_variable>
#include "cppa/on.hpp" #include "cppa/on.hpp"
#include "cppa/send.hpp"
#include "cppa/anything.hpp" #include "cppa/anything.hpp"
#include "cppa/to_string.hpp" #include "cppa/to_string.hpp"
#include "cppa/scheduler.hpp" #include "cppa/scheduler.hpp"
...@@ -30,11 +31,13 @@ ...@@ -30,11 +31,13 @@
#include "cppa/scoped_actor.hpp" #include "cppa/scoped_actor.hpp"
#include "cppa/system_messages.hpp" #include "cppa/system_messages.hpp"
#include "cppa/policy/fork_join.hpp"
#include "cppa/policy/no_resume.hpp" #include "cppa/policy/no_resume.hpp"
#include "cppa/policy/no_scheduling.hpp" #include "cppa/policy/no_scheduling.hpp"
#include "cppa/policy/actor_policies.hpp" #include "cppa/policy/actor_policies.hpp"
#include "cppa/policy/nestable_invoke.hpp" #include "cppa/policy/nestable_invoke.hpp"
#include "cppa/policy/not_prioritizing.hpp" #include "cppa/policy/not_prioritizing.hpp"
#include "cppa/policy/iterative_stealing.hpp"
#include "cppa/detail/logging.hpp" #include "cppa/detail/logging.hpp"
#include "cppa/detail/proper_actor.hpp" #include "cppa/detail/proper_actor.hpp"
...@@ -198,17 +201,17 @@ void printer_loop(blocking_actor* self) { ...@@ -198,17 +201,17 @@ void printer_loop(blocking_actor* self) {
* implementation of coordinator * * implementation of coordinator *
******************************************************************************/ ******************************************************************************/
class coordinator::shutdown_helper : public resumable { class shutdown_helper : public resumable {
public: public:
void attach_to_scheduler() override {} void attach_to_scheduler() override { }
void detach_from_scheduler() override {} void detach_from_scheduler() override { }
resumable::resume_result resume(execution_unit* ptr) { resumable::resume_result resume(execution_unit* ptr) {
CPPA_LOG_DEBUG("shutdown_helper::resume => shutdown worker"); CPPA_LOG_DEBUG("shutdown_helper::resume => shutdown worker");
auto wptr = dynamic_cast<worker*>(ptr); auto wptr = dynamic_cast<abstract_worker*>(ptr);
CPPA_REQUIRE(wptr != nullptr); CPPA_REQUIRE(wptr != nullptr);
std::unique_lock<std::mutex> guard(mtx); std::unique_lock<std::mutex> guard(mtx);
last_worker = wptr; last_worker = wptr;
...@@ -216,38 +219,45 @@ class coordinator::shutdown_helper : public resumable { ...@@ -216,38 +219,45 @@ class coordinator::shutdown_helper : public resumable {
return resumable::shutdown_execution_unit; return resumable::shutdown_execution_unit;
} }
shutdown_helper() : last_worker(nullptr) {} shutdown_helper() : last_worker(nullptr) { }
~shutdown_helper(); ~shutdown_helper();
std::mutex mtx; std::mutex mtx;
std::condition_variable cv; std::condition_variable cv;
worker* last_worker; abstract_worker* last_worker;
}; };
// avoid weak-vtables warning by providing dtor out-of-line shutdown_helper::~shutdown_helper() {
coordinator::shutdown_helper::~shutdown_helper() {} // nop
}
abstract_coordinator::~abstract_coordinator() {
// nop
}
// creates a default instance
abstract_coordinator* abstract_coordinator::create_singleton() {
return new coordinator<policy::iterative_stealing, policy::fork_join>;
}
void coordinator::initialize() { void abstract_coordinator::initialize() {
// launch threads of utility actors // launch threads of utility actors
auto ptr = m_timer.get(); auto ptr = m_timer.get();
m_timer_thread = std::thread{[ptr] { ptr->act(); }}; m_timer_thread = std::thread{[ptr] { ptr->act(); }};
m_printer_thread = std::thread{printer_loop, m_printer.get()}; m_printer_thread = std::thread{printer_loop, m_printer.get()};
// create & start workers
auto hwc = static_cast<size_t>(std::thread::hardware_concurrency());
m_workers.resize(hwc);
for (size_t i = 0; i < hwc; ++i) {
m_workers[i].start(i, this);
}
} }
void coordinator::stop() { void abstract_coordinator::stop() {
CPPA_LOG_TRACE(""); CPPA_LOG_TRACE("");
// shutdown workers // shutdown workers
shutdown_helper sh; shutdown_helper sh;
std::vector<worker*> alive_workers; std::vector<abstract_worker*> alive_workers;
for (auto& w : m_workers) alive_workers.push_back(&w); auto num = num_workers();
for (size_t i = 0; i < num; ++i) {
alive_workers.push_back(worker_by_id(i));
}
CPPA_LOG_DEBUG("enqueue shutdown_helper into each worker"); CPPA_LOG_DEBUG("enqueue shutdown_helper into each worker");
while (!alive_workers.empty()) { while (!alive_workers.empty()) {
alive_workers.back()->external_enqueue(&sh); alive_workers.back()->external_enqueue(&sh);
...@@ -257,224 +267,33 @@ void coordinator::stop() { ...@@ -257,224 +267,33 @@ void coordinator::stop() {
std::unique_lock<std::mutex> guard(sh.mtx); std::unique_lock<std::mutex> guard(sh.mtx);
sh.cv.wait(guard, [&] { return sh.last_worker != nullptr; }); sh.cv.wait(guard, [&] { return sh.last_worker != nullptr; });
} }
auto first = alive_workers.begin();
auto last = alive_workers.end(); auto last = alive_workers.end();
auto i = std::find(first, last, sh.last_worker); auto i = std::find(alive_workers.begin(), last, sh.last_worker);
sh.last_worker = nullptr; sh.last_worker = nullptr;
alive_workers.erase(i); alive_workers.erase(i);
} }
// shutdown utility actors // shutdown utility actors
CPPA_LOG_DEBUG("send 'DIE' messages to timer & printer"); CPPA_LOG_DEBUG("send 'DIE' messages to timer & printer");
auto msg = make_message(atom("DIE")); anon_send_exit(m_timer->address(), exit_reason::user_shutdown);
m_timer->enqueue(invalid_actor_addr, message_id::invalid, msg, nullptr); anon_send_exit(m_printer->address(), exit_reason::user_shutdown);
m_printer->enqueue(invalid_actor_addr, message_id::invalid, msg, nullptr);
CPPA_LOG_DEBUG("join threads of utility actors"); CPPA_LOG_DEBUG("join threads of utility actors");
m_timer_thread.join(); m_timer_thread.join();
m_printer_thread.join(); m_printer_thread.join();
// join each worker thread for good manners // join each worker thread for good manners
CPPA_LOG_DEBUG("join threads of workers");
for (auto& w : m_workers) w.m_this_thread.join();
CPPA_LOG_DEBUG("detach all resumables from all workers");
for (auto& w : m_workers) {
auto next = [&] { return w.m_exposed_queue.try_pop(); };
for (auto job = next(); job != nullptr; job = next()) {
job->detach_from_scheduler();
}
}
}
coordinator::coordinator()
: m_timer(new timer_actor), m_printer(true), m_next_worker(0) {}
coordinator* coordinator::create_singleton() { return new coordinator; }
actor coordinator::printer() const { return m_printer.get(); }
void coordinator::enqueue(resumable* what) {
size_t nw = m_next_worker++;
m_workers[nw % m_workers.size()].external_enqueue(what);
} }
/****************************************************************************** abstract_coordinator::abstract_coordinator(size_t nw)
* implementation of worker * : m_timer(new timer_actor), m_printer(true)
******************************************************************************/ , m_next_worker(0), m_num_workers(nw) {
// nop
#define CPPA_LOG_DEBUG_WORKER(msg) \
CPPA_LOG_DEBUG("worker " << m_id << ": " << msg)
worker::worker(worker&& other) {
*this = std::move(other); // delegate to move assignment operator
}
worker& worker::operator=(worker&& other) {
// cannot be moved once m_this_thread is up and running
auto running = [](std::thread& t) {
return t.get_id() != std::thread::id{};
};
if (running(m_this_thread) || running(other.m_this_thread)) {
throw std::runtime_error("running workers cannot be moved");
}
m_job_list = std::move(other.m_job_list);
auto next = [&] { return other.m_exposed_queue.try_pop(); };
for (auto j = next(); j != nullptr; j = next()) {
m_exposed_queue.push_back(j);
}
return *this;
}
void worker::start(size_t id, coordinator* parent) {
m_id = id;
m_last_victim = id;
m_parent = parent;
auto this_worker = this;
m_this_thread = std::thread{[this_worker] { this_worker->run(); }};
}
void worker::run() {
CPPA_LOG_TRACE(CPPA_ARG(m_id));
// local variables
job_ptr job = nullptr;
// some utility functions
auto local_poll = [&]()->bool {
if (!m_job_list.empty()) {
job = m_job_list.back();
m_job_list.pop_back();
CPPA_LOG_DEBUG_WORKER("got job from m_job_list");
return true;
}
return false;
};
auto aggressive_poll = [&]()->bool {
for (int i = 1; i < 101; ++i) {
job = m_exposed_queue.try_pop();
if (job) {
CPPA_LOG_DEBUG_WORKER("got job with aggressive polling");
return true;
}
// try to steal every 10 poll attempts
if ((i % 10) == 0) {
job = raid();
if (job) {
CPPA_LOG_DEBUG_WORKER("got job with aggressive polling");
return true;
}
}
std::this_thread::yield();
}
return false;
};
auto moderate_poll = [&]()->bool {
for (int i = 1; i < 550; ++i) {
job = m_exposed_queue.try_pop();
if (job) {
CPPA_LOG_DEBUG_WORKER("got job with moderate polling");
return true;
}
// try to steal every 5 poll attempts
if ((i % 5) == 0) {
job = raid();
if (job) {
CPPA_LOG_DEBUG_WORKER("got job with moderate polling");
return true;
}
}
std::this_thread::sleep_for(std::chrono::microseconds(50));
}
return false;
};
auto relaxed_poll = [&]()->bool {
for (;;) {
job = m_exposed_queue.try_pop();
if (job) {
CPPA_LOG_DEBUG_WORKER("got job with relaxed polling");
return true;
}
// always try to steal at this stage
job = raid();
if (job) {
CPPA_LOG_DEBUG_WORKER("got job with relaxed polling");
return true;
}
std::this_thread::sleep_for(std::chrono::milliseconds(10));
}
};
// scheduling loop
for (;;) {
local_poll() || aggressive_poll() || moderate_poll() || relaxed_poll();
CPPA_PUSH_AID_FROM_PTR(dynamic_cast<abstract_actor*>(job));
switch (job->resume(this)) {
case resumable::done: {
job->detach_from_scheduler();
break;
}
case resumable::resume_later: { break; }
case resumable::shutdown_execution_unit: {
// give others the opportunity to steal unfinished jobs
for (auto ptr : m_job_list) m_exposed_queue.push_back(ptr);
m_job_list.clear();
return;
}
}
job = nullptr;
// give others the opportunity to steal from us
if (m_job_list.size() > 1 && m_exposed_queue.empty()) {
m_exposed_queue.push_back(m_job_list.front());
m_job_list.erase(m_job_list.begin());
}
}
} }
worker::job_ptr worker::try_steal() { return m_exposed_queue.try_pop(); } actor abstract_coordinator::printer() const {
return m_printer.get();
worker::job_ptr worker::raid() {
// try once to steal from anyone
auto inc = [](size_t arg)->size_t {
return arg + 1;
};
auto dec = [](size_t arg)->size_t {
return arg - 1;
};
// reduce probability of 'steal collisions' by letting
// half the workers pick victims by increasing IDs and
// the other half by decreasing IDs
size_t (*next)(size_t) = (m_id % 2) == 0 ? inc : dec;
auto n = m_parent->num_workers();
for (size_t i = 0; i < n; ++i) {
m_last_victim = next(m_last_victim) % n;
if (m_last_victim != m_id) {
auto job = m_parent->worker_by_id(m_last_victim).try_steal();
if (job) {
CPPA_LOG_DEBUG_WORKER("successfully stolen a job from "
<< m_last_victim);
return job;
}
}
}
return nullptr;
} }
void worker::external_enqueue(job_ptr ptr) { m_exposed_queue.push_back(ptr); } void abstract_coordinator::enqueue(resumable* what) {
worker_by_id(m_next_worker++ % m_num_workers)->external_enqueue(what);
void worker::exec_later(job_ptr ptr) {
CPPA_REQUIRE(std::this_thread::get_id() == m_this_thread.get_id());
// give others the opportunity to steal from us
if (m_exposed_queue.empty()) {
if (m_job_list.empty()) {
m_exposed_queue.push_back(ptr);
} else {
m_exposed_queue.push_back(m_job_list.front());
m_job_list.erase(m_job_list.begin());
m_job_list.push_back(ptr);
}
} else
m_job_list.push_back(ptr);
} }
} // namespace scheduler } // namespace scheduler
......
...@@ -36,7 +36,7 @@ namespace detail { ...@@ -36,7 +36,7 @@ namespace detail {
namespace { namespace {
std::atomic<abstract_singleton*> s_plugins[singletons::max_plugin_singletons]; std::atomic<abstract_singleton*> s_plugins[singletons::max_plugin_singletons];
std::atomic<scheduler::coordinator*> s_scheduling_coordinator; std::atomic<scheduler::abstract_coordinator*> s_scheduling_coordinator;
std::atomic<uniform_type_info_map*> s_uniform_type_info_map; std::atomic<uniform_type_info_map*> s_uniform_type_info_map;
std::atomic<actor_registry*> s_actor_registry; std::atomic<actor_registry*> s_actor_registry;
std::atomic<group_manager*> s_group_manager; std::atomic<group_manager*> s_group_manager;
...@@ -84,7 +84,7 @@ group_manager* singletons::get_group_manager() { ...@@ -84,7 +84,7 @@ group_manager* singletons::get_group_manager() {
return lazy_get(s_group_manager); return lazy_get(s_group_manager);
} }
scheduler::coordinator* singletons::get_scheduling_coordinator() { scheduler::abstract_coordinator* singletons::get_scheduling_coordinator() {
return lazy_get(s_scheduling_coordinator); return lazy_get(s_scheduling_coordinator);
} }
......
...@@ -304,6 +304,7 @@ struct master : event_based_actor { ...@@ -304,6 +304,7 @@ struct master : event_based_actor {
behavior make_behavior() override { behavior make_behavior() override {
return ( return (
on(atom("done")) >> [=] { on(atom("done")) >> [=] {
CPPA_PRINT("master: received done");
quit(exit_reason::user_shutdown); quit(exit_reason::user_shutdown);
} }
); );
...@@ -316,7 +317,12 @@ struct slave : event_based_actor { ...@@ -316,7 +317,12 @@ struct slave : event_based_actor {
behavior make_behavior() override { behavior make_behavior() override {
link_to(master); link_to(master);
trap_exit(true);
return ( return (
[=](const exit_msg& msg) {
CPPA_PRINT("slave: received exit message");
quit(msg.reason);
},
others() >> CPPA_UNEXPECTED_MSG_CB(this) others() >> CPPA_UNEXPECTED_MSG_CB(this)
); );
} }
......
Markdown is supported
0%
or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment