Commit 6241921e authored by Dominik Charousset's avatar Dominik Charousset

Revise scheduler, make max throughput configurable

The scheduler now has only a single policy that is currently only implemented
for work stealing. When defining a scheduler, users can now define a max
throughput parameter that denotes how many messages an actor is allowed to
consume before it must return control to the scheduler.
parent 9ddc8b01
......@@ -39,22 +39,26 @@ namespace caf {
namespace policy {
class event_based_resume {
public:
// Base must be a mailbox-based actor
template <class Base, class Derived>
struct mixin : Base, resumable {
template <class... Ts>
mixin(Ts&&... args)
: Base(std::forward<Ts>(args)...) {}
mixin(Ts&&... args) : Base(std::forward<Ts>(args)...) {
// nop
}
void attach_to_scheduler() override { this->ref(); }
void attach_to_scheduler() override {
this->ref();
}
void detach_from_scheduler() override { this->deref(); }
void detach_from_scheduler() override {
this->deref();
}
resumable::resume_result resume(execution_unit* new_host) override {
resumable::resume_result resume(execution_unit* new_host,
size_t max_throughput) override {
auto d = static_cast<Derived*>(this);
d->host(new_host);
CAF_LOG_TRACE("id = " << d->id());
......@@ -78,13 +82,13 @@ class event_based_resume {
};
auto actor_done = [&] {
return d->bhvr_stack().empty() ||
d->planned_exit_reason() != exit_reason::not_exited;
return d->bhvr_stack().empty()
|| d->planned_exit_reason() != exit_reason::not_exited;
};
// actors without behavior or that have already defined
// an exit reason must not be resumed
CAF_REQUIRE(!d->m_initialized || !actor_done());
try {
if (!d->m_initialized) {
d->m_initialized = true;
auto bhvr = d->make_behavior();
......@@ -93,8 +97,12 @@ class event_based_resume {
if (actor_done() && done_cb()) return resume_result::done;
// else: enter resume loop
}
try {
for (;;) {
// max_throughput = 0 means infinite
size_t increment = max_throughput == 0 ? 0 : 1;
if (max_throughput == 0) {
max_throughput = 1;
}
for (size_t i = 0; i < max_throughput; i += increment) {
auto ptr = d->next_message();
if (ptr) {
if (d->invoke_message(ptr)) {
......@@ -121,11 +129,16 @@ class event_based_resume {
} else {
CAF_LOG_DEBUG("no more element in mailbox; going to block");
if (d->mailbox().try_block()) {
return resumable::resume_later;
return resumable::awaiting_message;
}
CAF_LOG_DEBUG("try_block() interrupted by new message");
}
}
if (!d->has_next_message() && d->mailbox().try_block()) {
return resumable::awaiting_message;
}
// time's up
return resumable::resume_later;
}
catch (actor_exited& what) {
CAF_LOG_INFO("actor died because of exception: actor_exited, "
......@@ -151,7 +164,6 @@ class event_based_resume {
done_cb();
return resumable::done;
}
};
template <class Actor>
......@@ -168,7 +180,6 @@ class event_based_resume {
"to implement blocking actors");
return false;
}
};
} // namespace policy
......
/******************************************************************************
* ____ _ _____ *
* / ___| / \ | ___| C++ *
* | | / _ \ | |_ Actor *
* | |___ / ___ \| _| Framework *
* \____/_/ \_|_| *
* *
* Copyright (C) 2011 - 2014 *
* Dominik Charousset <dominik.charousset (at) haw-hamburg.de> *
* *
* Distributed under the terms and conditions of the BSD 3-Clause License or *
* (at your option) under the terms and conditions of the Boost Software *
* License 1.0. See accompanying files LICENSE and LICENCE_ALTERNATIVE. *
* *
* If you did not receive a copy of the license files, see *
* http://opensource.org/licenses/BSD-3-Clause and *
* http://www.boost.org/LICENSE_1_0.txt. *
******************************************************************************/
#ifndef CAF_POLICY_ITERATIVE_STEALING_HPP
#define CAF_POLICY_ITERATIVE_STEALING_HPP
#include <cstddef>
#include "caf/fwd.hpp"
namespace caf {
namespace policy {
/**
* An implementation of the {@link steal_policy} concept
* that simply iterates over 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 caf
#endif // CAF_POLICY_ITERATIVE_STEALING_HPP
......@@ -28,7 +28,7 @@ class no_resume {
this->deref();
}
resumable::resume_result resume(execution_unit*) {
resumable::resume_result resume(execution_unit*, size_t) {
auto done_cb = [=](uint32_t reason) {
this->planned_exit_reason(reason);
this->on_exit();
......
......@@ -74,7 +74,7 @@ class no_scheduling {
CAF_PUSH_AID(mself->id());
CAF_LOG_TRACE("");
for (;;) {
if (mself->resume(nullptr) == resumable::done) {
if (mself->resume(nullptr, 0) == resumable::done) {
return;
}
// await new data before resuming actor
......
......@@ -17,8 +17,8 @@
* http://www.boost.org/LICENSE_1_0.txt. *
******************************************************************************/
#ifndef CAF_POLICY_JOB_QUEUE_POLICY_HPP
#define CAF_POLICY_JOB_QUEUE_POLICY_HPP
#ifndef CAF_POLICY_SCHEDULER_POLICY_HPP
#define CAF_POLICY_SCHEDULER_POLICY_HPP
#include "caf/fwd.hpp"
......@@ -26,12 +26,26 @@ namespace caf {
namespace policy {
/**
* This concept class describes the interface of a policy class
* for managing the queue(s) of a scheduler worker.
* This concept class describes a policy for worker
* and coordinator of the scheduler.
*/
class job_queue_policy {
class scheduler_policy {
public:
/**
* Policy-specific data fields for the coordinator.
*/
struct coordinator_data { };
/**
* Policy-specific data fields for the worker.
*/
struct worker_data { };
/**
* Enqueues a new job to coordinator.
*/
template <class Coordinator>
void central_enqueue(Coordinator* self, resumable* job);
/**
* Enqueues a new job to the worker's queue from an
......@@ -47,6 +61,12 @@ class job_queue_policy {
template <class Worker>
void internal_enqueue(Worker* self, resumable* job);
/**
* Called whenever resumable returned for reason `resumable::resume_later`.
*/
template <class Worker>
void resume_job_later(Worker* self, resumable* job);
/**
* Returns `nullptr` if no element could be dequeued immediately.
* Called by external sources to try to dequeue an element.
......@@ -62,28 +82,33 @@ class job_queue_policy {
resumable* internal_dequeue(Worker* self);
/**
* Moves all elements form the internal queue to the external queue.
* Performs cleanup action before a shutdown takes place.
*/
template <class Worker>
void clear_internal_queue(Worker* self);
void before_shutdown(Worker* self);
/**
* Tries to move at least one element from the internal queue to
* the external queue if possible to allow others to steal from us.
* Called whenever an actor has been resumed. This function can
* prepare some fields before the next resume operation takes place
* or perform cleanup actions between to actor runs.
*/
template <class Worker>
void assert_stealable(Worker* self);
void after_resume(Worker* self);
/**
* Applies given functor to all elements in all queues and
* clears all queues afterwards.
* Applies given functor to all resumables attached to a worker.
*/
template <class Worker, typename UnaryFunction>
void consume_all(Worker* self, UnaryFunction f);
void foreach_resumable(Worker* self, UnaryFunction f);
/**
* Applies given functor to all resumables attached to the coordinator.
*/
template <class Coordinator, typename UnaryFunction>
void foreach_central_resumable(Coordinator* self, UnaryFunction f);
};
} // namespace policy
} // namespace caf
#endif // CAF_POLICY_JOB_QUEUE_POLICY_HPP
#endif // CAF_POLICY_SCHEDULER_POLICY_HPP
/******************************************************************************
* ____ _ _____ *
* / ___| / \ | ___| C++ *
* | | / _ \ | |_ Actor *
* | |___ / ___ \| _| Framework *
* \____/_/ \_|_| *
* *
* Copyright (C) 2011 - 2014 *
* Dominik Charousset <dominik.charousset (at) haw-hamburg.de> *
* *
* Distributed under the terms and conditions of the BSD 3-Clause License or *
* (at your option) under the terms and conditions of the Boost Software *
* License 1.0. See accompanying files LICENSE and LICENCE_ALTERNATIVE. *
* *
* If you did not receive a copy of the license files, see *
* http://opensource.org/licenses/BSD-3-Clause and *
* http://www.boost.org/LICENSE_1_0.txt. *
******************************************************************************/
#ifndef CAF_POLICY_STEAL_POLICY_HPP
#define CAF_POLICY_STEAL_POLICY_HPP
#include <cstddef>
#include "caf/fwd.hpp"
namespace caf {
namespace policy {
/**
* This concept class describes the interface of a policy class
* for stealing jobs from other workers.
*/
class steal_policy {
public:
/**
* Go on a raid in quest for a shiny new job. Returns `nullptr`
* if no other worker provided any work to steal.
*/
template <class Worker>
resumable* raid(Worker* self);
};
} // namespace policy
} // namespace caf
#endif // CAF_POLICY_STEAL_POLICY_HPP
......@@ -17,12 +17,13 @@
* http://www.boost.org/LICENSE_1_0.txt. *
******************************************************************************/
#ifndef CAF_POLICY_FORK_JOIN_HPP
#define CAF_POLICY_FORK_JOIN_HPP
#ifndef CAF_POLICY_WORK_STEALING_HPP
#define CAF_POLICY_WORK_STEALING_HPP
#include <deque>
#include <chrono>
#include <thread>
#include <random>
#include <cstddef>
#include "caf/resumable.hpp"
......@@ -33,12 +34,9 @@ namespace caf {
namespace policy {
/**
* An implementation of the `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.
* Implements scheduling of actors via work stealing. This 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
......@@ -51,54 +49,90 @@ namespace policy {
*
* [1] http://dl.acm.org/citation.cfm?doid=2398857.2384639
*
* @relates job_queue_policy
* @extends scheduler_policy
*/
class fork_join {
class work_stealing {
public:
// A thead-safe queue implementation.
using sync_queue = detail::producer_consumer_list<resumable>;
fork_join() = default;
// A queue implementation supporting fast push and pop
// operations on both ends of the queue.
using priv_queue = std::deque<resumable*>;
fork_join(fork_join&& other) {
// delegate to move assignment operator
*this = std::move(other);
// The coordinator has no data since our scheduling is decentralized.
struct coordinator_data {
size_t next_worker;
inline coordinator_data() : next_worker(0) {
// nop
}
};
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);
// Holds the job queues of a worker.
struct worker_data {
// This queue is exposed to other workers that may attempt to steal jobs
// from it and the central scheduling unit can push new jobs to the queue.
sync_queue exposed_queue;
// Internal job queue of a worker (not exposed to others).
priv_queue private_queue;
// needed by our engine
std::random_device rdevice;
// needed to generate pseudo random numbers
std::default_random_engine rengine;
inline worker_data() : rdevice(), rengine(rdevice()) {
// nop
}
return *this;
};
// convenience function to access the data field
template <class WorkerOrCoordinator>
auto d(WorkerOrCoordinator* self) -> decltype(self->data()) {
return self->data();
}
/**
* A thead-safe queue implementation.
*/
using sync_queue = detail::producer_consumer_list<resumable>;
// go on a raid in quest for a shiny new job
template <class Worker>
resumable* try_steal(Worker* self) {
auto p = self->parent();
auto victim = d(self).rengine() % p->num_workers();
return try_external_dequeue(p->worker_by_id(victim));
}
/**
* A queue implementation supporting fast push and pop
* operations on both ends of the queue.
*/
using priv_queue = std::deque<resumable*>;
template <class Coordinator>
void central_enqueue(Coordinator* self, resumable* job) {
auto w = self->worker_by_id(d(self).next_worker++ % self->num_workers());
w->external_enqueue(job);
}
template <class Worker>
void external_enqueue(Worker*, resumable* job) {
m_exposed_queue.push_back(job);
void external_enqueue(Worker* self, resumable* job) {
d(self).exposed_queue.push_back(job);
}
template <class Worker>
void internal_enqueue(Worker* ptr, resumable* job) {
m_exposed_queue.push_back(job);
void internal_enqueue(Worker* self, resumable* job) {
d(self).private_queue.push_back(job);
// give others the opportunity to steal from us
assert_stealable(ptr);
after_resume(self);
}
template <class Worker>
resumable* try_external_dequeue(Worker*) {
return m_exposed_queue.try_pop();
void resume_job_later(Worker* self, resumable* job) {
// job has voluntarily released the CPU to let others run instead
// this means we are going to put this job to the very end of our queue
// by moving everything from the exposed to private queue first and
// then enqueue job to the exposed queue
resumable* ptr = d(self).exposed_queue.try_pop();
while (ptr) {
d(self).private_queue.push_front(ptr);
ptr = d(self).exposed_queue.try_pop();
}
d(self).exposed_queue.push_back(job);
}
template <class Worker>
resumable* try_external_dequeue(Worker* self) {
return d(self).exposed_queue.try_pop();
}
template <class Worker>
......@@ -113,7 +147,7 @@ class fork_join {
struct poll_strategy {
size_t attempts;
size_t step_size;
size_t raid_interval;
size_t steal_interval;
std::chrono::microseconds sleep_duration;
};
constexpr poll_strategy strategies[3] = {
......@@ -126,20 +160,20 @@ class fork_join {
};
resumable* job = nullptr;
// local poll
if (!m_private_queue.empty()) {
job = m_private_queue.back();
m_private_queue.pop_back();
if (!d(self).private_queue.empty()) {
job = d(self).private_queue.back();
d(self).private_queue.pop_back();
return job;
}
for (auto& strat : strategies) {
for (size_t i = 0; i < strat.attempts; i += strat.step_size) {
job = m_exposed_queue.try_pop();
job = d(self).exposed_queue.try_pop();
if (job) {
return job;
}
// try to steal every X poll attempts
if ((i % strat.raid_interval) == 0) {
job = self->raid();
if ((i % strat.steal_interval) == 0) {
job = try_steal(self);
if (job) {
return job;
}
......@@ -153,47 +187,42 @@ class fork_join {
}
template <class Worker>
void clear_internal_queue(Worker*) {
void before_shutdown(Worker* self) {
// give others the opportunity to steal unfinished jobs
for (auto ptr : m_private_queue) {
m_exposed_queue.push_back(ptr);
for (auto ptr : d(self).private_queue) {
d(self).exposed_queue.push_back(ptr);
}
m_private_queue.clear();
d(self).private_queue.clear();
}
template <class Worker>
void assert_stealable(Worker*) {
void after_resume(Worker* self) {
// 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.pop_front();
if (d(self).private_queue.size() > 1 && d(self).exposed_queue.empty()) {
d(self).exposed_queue.push_back(d(self).private_queue.front());
d(self).private_queue.pop_front();
}
}
template <class Worker, typename UnaryFunction>
void consume_all(Worker*, UnaryFunction f) {
for (auto job : m_private_queue) {
template <class Worker, class UnaryFunction>
void foreach_resumable(Worker* self, UnaryFunction f) {
for (auto job : d(self).private_queue) {
f(job);
}
m_private_queue.clear();
auto next = [&] { return this->m_exposed_queue.try_pop(); };
d(self).private_queue.clear();
auto next = [&] { return this->d(self).exposed_queue.try_pop(); };
for (auto job = next(); job != nullptr; job = next()) {
f(job);
}
}
private:
// this queue is exposed to other workers that 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;
template <class Coordinator, class UnaryFunction>
void foreach_central_resumable(Coordinator*, UnaryFunction) {
// nop
}
};
} // namespace policy
} // namespace caf
#endif // CAF_POLICY_FORK_JOIN_HPP
#endif // CAF_POLICY_WORK_STEALING_HPP
......@@ -20,6 +20,8 @@
#ifndef CAF_RESUMABLE_HPP
#define CAF_RESUMABLE_HPP
#include <cstddef> // size_t
namespace caf {
class execution_unit;
......@@ -32,9 +34,9 @@ class resumable {
public:
enum resume_result {
resume_later,
awaiting_message,
done,
shutdown_execution_unit
};
resumable();
......@@ -55,7 +57,7 @@ class resumable {
* Resume any pending computation until it is either finished
* or needs to be re-scheduled later.
*/
virtual resume_result resume(execution_unit*) = 0;
virtual resume_result resume(execution_unit*, size_t max_throughput) = 0;
protected:
bool m_hidden;
......
......@@ -70,7 +70,7 @@ class abstract_worker : public execution_unit {
/**
* Starts the thread of this worker.
*/
virtual void start(size_t id, abstract_coordinator* parent) = 0;
//virtual void start(size_t id, abstract_coordinator* parent) = 0;
};
......@@ -96,7 +96,7 @@ class abstract_coordinator {
/**
* Puts `what` into the queue of a randomly chosen worker.
*/
void enqueue(resumable* what);
virtual void enqueue(resumable* what) = 0;
template <class Duration, class... Data>
void delayed_send(Duration rel_time, actor_addr from, channel to,
......@@ -142,10 +142,13 @@ class abstract_coordinator {
std::thread m_printer_thread;
};
template <class Policy>
class coordinator;
/**
* Policy-based implementation of the abstract worker base class.
*/
template <class StealPolicy, class JobQueuePolicy>
template <class Policy>
class worker : public abstract_worker {
public:
worker(const worker&) = delete;
......@@ -165,20 +168,24 @@ class worker : public abstract_worker {
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);
m_policy = std::move(other.m_policy);
m_policy = std::move(other.m_policy);
return *this;
}
using coordinator_type = coordinator<Policy>;
using job_ptr = resumable*;
using job_queue = detail::producer_consumer_list<resumable>;
using policy_data = typename Policy::worker_data;
/**
* Attempt to steal an element from the exposed job queue.
*/
job_ptr try_steal() override {
auto result = m_queue_policy.try_external_dequeue(this);
auto result = m_policy.try_external_dequeue(this);
CAF_LOG_DEBUG_IF(result, "stole actor with id " << id_of(result));
return result;
}
......@@ -190,7 +197,7 @@ class worker : public abstract_worker {
void external_enqueue(job_ptr job) override {
CAF_REQUIRE(job != nullptr);
CAF_LOG_TRACE("id = " << id() << " actor id " << id_of(job));
m_queue_policy.external_enqueue(this, job);
m_policy.external_enqueue(this, job);
}
/**
......@@ -201,42 +208,42 @@ class worker : public abstract_worker {
void exec_later(job_ptr job) override {
CAF_REQUIRE(job != nullptr);
CAF_LOG_TRACE("id = " << id() << " actor id " << id_of(job));
m_queue_policy.internal_enqueue(this, job);
m_policy.internal_enqueue(this, job);
}
// go on a raid in quest for a shiny new job
job_ptr raid() {
auto result = m_steal_policy.raid(this);
auto result = m_policy.raid(this);
CAF_LOG_DEBUG_IF(result, "got actor with id " << id_of(result));
return result;
}
inline abstract_coordinator* parent() {
coordinator_type* parent() {
return m_parent;
}
inline size_t id() const {
size_t id() const {
return m_id;
}
inline std::thread& get_thread() {
std::thread& get_thread() {
return m_this_thread;
}
void detach_all() {
CAF_LOG_TRACE("");
m_queue_policy.consume_all(this, [](resumable* job) {
m_policy.foreach_resumable(this, [](resumable* job) {
job->detach_from_scheduler();
});
}
void start(size_t id, abstract_coordinator* parent) override {
void start(size_t id, coordinator_type* parent, size_t max_throughput) {
m_max_throughput = max_throughput;
m_id = id;
m_parent = parent;
auto this_worker = this;
m_this_thread = std::thread{[this_worker] {
CAF_LOGC_TRACE("caf::scheduler::worker",
"start$lambda",
CAF_LOGC_TRACE("caf::scheduler::worker", "start$lambda",
"id = " << this_worker->id());
this_worker->run();
}};
......@@ -248,83 +255,122 @@ class worker : public abstract_worker {
return aptr ? aptr->id() : 0;
}
policy_data& data() {
return m_data;
}
size_t max_throughput() {
return m_max_throughput;
}
private:
void run() {
CAF_LOG_TRACE("worker with ID " << m_id);
// scheduling loop
for (;;) {
auto job = m_queue_policy.internal_dequeue(this);
auto job = m_policy.internal_dequeue(this);
CAF_REQUIRE(job != nullptr);
CAF_LOG_DEBUG("resume actor " << id_of(job));
CAF_PUSH_AID_FROM_PTR(dynamic_cast<abstract_actor*>(job));
switch (job->resume(this)) {
switch (job->resume(this, m_max_throughput)) {
case resumable::resume_later: {
m_policy.resume_job_later(this, job);
break;
}
case resumable::done: {
job->detach_from_scheduler();
break;
}
case resumable::resume_later: {
case resumable::awaiting_message: {
// resumable will be enqueued again later
break;
}
case resumable::shutdown_execution_unit: {
m_queue_policy.clear_internal_queue(this);
m_policy.before_shutdown(this);
return;
}
}
m_queue_policy.assert_stealable(this);
m_policy.after_resume(this);
}
}
// number of messages each actor is allowed to consume per resume
size_t m_max_throughput;
// the worker's thread
std::thread m_this_thread;
// the worker's ID received from scheduler
size_t m_id;
// pointer to central coordinator
abstract_coordinator* m_parent;
// policy managing queues
JobQueuePolicy m_queue_policy;
// policy managing steal operations
StealPolicy m_steal_policy;
coordinator_type* m_parent;
// policy-specific data
policy_data m_data;
// instance of our policy object
Policy m_policy;
};
/**
* Policy-based implementation of the abstract coordinator base class.
*/
template <class StealPolicy, class JobQueuePolicy>
template <class Policy>
class coordinator : public abstract_coordinator {
public:
using super = abstract_coordinator;
using policy_data = typename Policy::coordinator_data;
coordinator(size_t nw = std::thread::hardware_concurrency()) : super(nw) {
// nop
}
using worker_type = worker<StealPolicy, JobQueuePolicy>;
using worker_type = worker<Policy>;
abstract_worker* worker_by_id(size_t id) override {
worker_type* worker_by_id(size_t id) override {
return &m_workers[id];
}
policy_data& data() {
return m_data;
}
protected:
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);
m_workers[i].start(i, this, m_max_throughput);
}
}
void stop() override {
// perform cleanup code of base classe
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();
// wait until all workers are done
for (auto& w : m_workers) {
w.get_thread().join();
}
// run cleanup code for each resumable
auto f = [](resumable* job) {
job->detach_from_scheduler();
};
for (auto& w : m_workers) {
m_policy.foreach_resumable(&w, f);
}
m_policy.foreach_central_resumable(this, f);
}
void enqueue(resumable* ptr) {
m_policy.central_enqueue(this, ptr);
}
private:
// usually of size std::thread::hardware_concurrency()
std::vector<worker_type> m_workers;
// policy-specific data
policy_data m_data;
// instance of our policy object
Policy m_policy;
// number of messages each actor is allowed to consume per resume
size_t m_max_throughput;
};
} // namespace scheduler
......@@ -339,14 +385,16 @@ void set_scheduler(scheduler::abstract_coordinator* ptr);
/**
* Sets a user-defined scheduler using given policies. The scheduler
* is instantiated with `nw` number of workers.
* is instantiated with `nw` number of workers and allows each actor
* to consume up to `max_throughput` per resume (0 means infinite).
* @note This function must be used before actor is spawned. Dynamically
* changing the scheduler at runtime is not supported.
* @throws std::logic_error if a scheduler is already defined
*/
template <class StealPolicy, class JobQueuePolicy>
void set_scheduler(size_t nw = std::thread::hardware_concurrency()) {
set_scheduler(new scheduler::coordinator<StealPolicy, JobQueuePolicy>(nw));
template <class Policy>
void set_scheduler(size_t nw = std::thread::hardware_concurrency(),
size_t max_throughput = 0) {
set_scheduler(new scheduler::coordinator<Policy>(nw, max_throughput));
}
} // namespace caf
......
......@@ -34,13 +34,12 @@
#include "caf/actor_ostream.hpp"
#include "caf/policy/fork_join.hpp"
#include "caf/policy/work_stealing.hpp"
#include "caf/policy/no_resume.hpp"
#include "caf/policy/no_scheduling.hpp"
#include "caf/policy/actor_policies.hpp"
#include "caf/policy/nestable_invoke.hpp"
#include "caf/policy/not_prioritizing.hpp"
#include "caf/policy/iterative_stealing.hpp"
#include "caf/detail/logging.hpp"
#include "caf/detail/proper_actor.hpp"
......@@ -219,7 +218,7 @@ class shutdown_helper : public resumable {
void detach_from_scheduler() override {
// nop
}
resumable::resume_result resume(execution_unit* ptr) {
resumable::resume_result resume(execution_unit* ptr, size_t) override {
CAF_LOG_DEBUG("shutdown_helper::resume => shutdown worker");
auto wptr = dynamic_cast<abstract_worker*>(ptr);
CAF_REQUIRE(wptr != nullptr);
......@@ -247,7 +246,7 @@ abstract_coordinator::~abstract_coordinator() {
// creates a default instance
abstract_coordinator* abstract_coordinator::create_singleton() {
return new coordinator<policy::iterative_stealing, policy::fork_join>;
return new coordinator<policy::work_stealing>;
}
void abstract_coordinator::initialize() {
......@@ -302,10 +301,6 @@ actor abstract_coordinator::printer() const {
return m_printer.get();
}
void abstract_coordinator::enqueue(resumable* what) {
worker_by_id(m_next_worker++ % m_num_workers)->external_enqueue(what);
}
} // namespace scheduler
void set_scheduler(scheduler::abstract_coordinator* impl) {
......
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