Commit b8b76c36 authored by Dominik Charousset's avatar Dominik Charousset

Merge branch 'topic/timer'

Relates #647
parents a50563f2 42f463c9
......@@ -18,6 +18,7 @@ set (LIBCAF_CORE_SRCS
src/abstract_group.cpp
src/actor.cpp
src/actor_addr.cpp
src/actor_clock.cpp
src/actor_companion.cpp
src/actor_config.cpp
src/actor_control_block.cpp
......@@ -89,6 +90,7 @@ set (LIBCAF_CORE_SRCS
src/sequencer.cpp
src/serializer.cpp
src/shared_spinlock.cpp
src/simple_actor_clock.cpp
src/skip.cpp
src/splitter.cpp
src/stream.cpp
......@@ -106,7 +108,9 @@ set (LIBCAF_CORE_SRCS
src/sync_request_bouncer.cpp
src/term.cpp
src/terminal_stream_scatterer.cpp
src/test_actor_clock.cpp
src/test_coordinator.cpp
src/thread_safe_actor_clock.cpp
src/timestamp.cpp
src/try_match.cpp
src/type_erased_tuple.cpp
......
/******************************************************************************
* ____ _ _____ *
* / ___| / \ | ___| C++ *
* | | / _ \ | |_ Actor *
* | |___ / ___ \| _| Framework *
* \____/_/ \_|_| *
* *
* Copyright 2011-2018 Dominik Charousset *
* *
* 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 LICENSE_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_ACTOR_CLOCK
#define CAF_ACTOR_CLOCK
#include <chrono>
#include "caf/fwd.hpp"
namespace caf {
/// A monotonic clock for scheduling timeouts and delayed messages.
class actor_clock {
public:
// -- member types -----------------------------------------------------------
/// Underlying clock type.
using clock_type = std::chrono::steady_clock;
/// Discrete point in time.
using time_point = typename clock_type::time_point;
/// Difference between two points in time.
using duration_type = typename clock_type::duration;
// -- constructors, destructors, and assignment operators --------------------
virtual ~actor_clock();
// -- observers --------------------------------------------------------------
/// Returns the current wall-clock time.
virtual time_point now() const noexcept;
/// Returns the difference between `t0` and `t1`, allowing the clock to
/// return any arbitrary value depending on the measurement that took place.
virtual duration_type difference(atom_value measurement, time_point t0,
time_point t1) const noexcept;
/// Schedules a `timeout_msg` for `self` at time point `t`, overriding any
/// previous receive timeout.
virtual void set_receive_timeout(time_point t, abstract_actor* self,
uint32_t id) = 0;
/// Schedules a `sec::request_timeout` for `self` at time point `t`.
virtual void set_request_timeout(time_point t, abstract_actor* self,
message_id id) = 0;
/// Cancels a pending receive timeout.
virtual void cancel_receive_timeout(abstract_actor* self) = 0;
/// Cancels the pending request timeout for `id`.
virtual void cancel_request_timeout(abstract_actor* self, message_id id) = 0;
/// Cancels all timeouts for `self`.
virtual void cancel_timeouts(abstract_actor* self) = 0;
/// Schedules an arbitrary message to `receiver` for time point `t`.
virtual void schedule_message(time_point t, strong_actor_ptr receiver,
mailbox_element_ptr content) = 0;
/// Schedules an arbitrary message to `target` for time point `t`.
virtual void schedule_message(time_point t, group target,
strong_actor_ptr sender, message content) = 0;
};
} // namespace caf
#endif // CAF_ACTOR_CLOCK
......@@ -32,6 +32,7 @@
#include "caf/logger.hpp"
#include "caf/actor_cast.hpp"
#include "caf/make_actor.hpp"
#include "caf/actor_clock.hpp"
#include "caf/infer_handle.hpp"
#include "caf/actor_config.hpp"
#include "caf/spawn_options.hpp"
......@@ -475,10 +476,13 @@ public:
}
/// Returns the configuration of this actor system.
const actor_system_config& config() const {
inline const actor_system_config& config() const {
return cfg_;
}
/// Returns the system-wide clock.
actor_clock& clock() noexcept;
/// @cond PRIVATE
/// Increases running-detached-threads-count by one.
......
......@@ -51,6 +51,7 @@
#include "caf/message_id.hpp"
#include "caf/replies_to.hpp"
#include "caf/serializer.hpp"
#include "caf/actor_clock.hpp"
#include "caf/actor_proxy.hpp"
#include "caf/exit_reason.hpp"
#include "caf/local_actor.hpp"
......
/******************************************************************************
* ____ _ _____ *
* / ___| / \ | ___| C++ *
* | | / _ \ | |_ Actor *
* | |___ / ___ \| _| Framework *
* \____/_/ \_|_| *
* *
* Copyright 2011-2018 Dominik Charousset *
* *
* 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 LICENSE_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_DETAIL_SIMPLE_ACTOR_CLOCK_HPP
#define CAF_DETAIL_SIMPLE_ACTOR_CLOCK_HPP
#include <map>
#include "caf/actor_clock.hpp"
#include "caf/actor_control_block.hpp"
#include "caf/group.hpp"
#include "caf/mailbox_element.hpp"
#include "caf/message.hpp"
#include "caf/message_id.hpp"
#include "caf/variant.hpp"
namespace caf {
namespace detail {
class simple_actor_clock : public actor_clock {
public:
// -- member types -----------------------------------------------------------
/// Request for a `timeout_msg`.
struct receive_timeout {
strong_actor_ptr self;
uint32_t id;
};
/// Request for a `sec::request_timeout` error.
struct request_timeout {
strong_actor_ptr self;
message_id id;
};
/// Request for sending a message to an actor at a later time.
struct actor_msg {
strong_actor_ptr receiver;
mailbox_element_ptr content;
};
/// Request for sending a message to a group at a later time.
struct group_msg {
group target;
strong_actor_ptr sender;
message content;
};
using value_type = variant<receive_timeout, request_timeout,
actor_msg, group_msg>;
using map_type = std::multimap<time_point, value_type>;
using secondary_map = std::multimap<abstract_actor*, map_type::iterator>;
struct receive_predicate {
bool operator()(const secondary_map::value_type& x) const noexcept;
};
struct request_predicate {
message_id id;
bool operator()(const secondary_map::value_type& x) const noexcept;
};
struct visitor {
simple_actor_clock* thisptr;
void operator()(receive_timeout& x);
void operator()(request_timeout& x);
void operator()(actor_msg& x);
void operator()(group_msg& x);
};
void set_receive_timeout(time_point t, abstract_actor* self,
uint32_t id) override;
void set_request_timeout(time_point t, abstract_actor* self,
message_id id) override;
void cancel_receive_timeout(abstract_actor* self) override;
void cancel_request_timeout(abstract_actor* self, message_id id) override;
void cancel_timeouts(abstract_actor* self) override;
void schedule_message(time_point t, strong_actor_ptr receiver,
mailbox_element_ptr content) override;
void schedule_message(time_point t, group target, strong_actor_ptr sender,
message content) override;
inline const map_type& schedule() const {
return schedule_;
}
inline const secondary_map& actor_lookup() const {
return actor_lookup_;
}
protected:
template <class Predicate>
secondary_map::iterator lookup(abstract_actor* self, Predicate pred) {
auto e = actor_lookup_.end();
auto range = actor_lookup_.equal_range(self);
if (range.first == range.second)
return e;
auto i = std::find_if(range.first, range.second, pred);
return i != range.second ? i : e;
}
template <class Predicate>
void cancel(abstract_actor* self, Predicate pred) {
auto i = lookup(self, pred);
if (i != actor_lookup_.end()) {
schedule_.erase(i->second);
actor_lookup_.erase(i);
}
}
template <class Predicate>
void drop_lookup(abstract_actor* self, Predicate pred) {
auto i = lookup(self, pred);
if (i != actor_lookup_.end())
actor_lookup_.erase(i);
}
map_type schedule_;
secondary_map actor_lookup_;
};
} // namespace detail
} // namespace caf
#endif // CAF_DETAIL_SIMPLE_ACTOR_CLOCK_HPP
/******************************************************************************
* ____ _ _____ *
* / ___| / \ | ___| C++ *
* | | / _ \ | |_ Actor *
* | |___ / ___ \| _| Framework *
* \____/_/ \_|_| *
* *
* Copyright 2011-2018 Dominik Charousset *
* *
* 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 LICENSE_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_DETAIL_TEST_ACTOR_CLOCK_HPP
#define CAF_DETAIL_TEST_ACTOR_CLOCK_HPP
#include "caf/detail/simple_actor_clock.hpp"
namespace caf {
namespace detail {
class test_actor_clock : public simple_actor_clock {
public:
time_point current_time;
time_point now() const noexcept override;
/// Tries to dispatch the next timeout or delayed message regardless of its
/// timestamp. Returns `false` if `schedule().empty()`, otherwise `true`.
bool dispatch_once();
/// Dispatches all timeouts and delayed messages regardless of their
/// timestamp. Returns the number of dispatched events.
size_t dispatch();
/// Advances the time by `x` and dispatches timeouts and delayed messages.
void advance_time(duration_type x);
};
} // namespace detail
} // namespace caf
#endif // CAF_DETAIL_TEST_ACTOR_CLOCK_HPP
/******************************************************************************
* ____ _ _____ *
* / ___| / \ | ___| C++ *
* | | / _ \ | |_ Actor *
* | |___ / ___ \| _| Framework *
* \____/_/ \_|_| *
* *
* Copyright 2011-2018 Dominik Charousset *
* *
* 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 LICENSE_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_DETAIL_THREAD_SAFE_ACTOR_CLOCK_HPP
#define CAF_DETAIL_THREAD_SAFE_ACTOR_CLOCK_HPP
#include <mutex>
#include <atomic>
#include <condition_variable>
#include "caf/detail/simple_actor_clock.hpp"
namespace caf {
namespace detail {
class thread_safe_actor_clock : public simple_actor_clock {
public:
using super = simple_actor_clock;
thread_safe_actor_clock();
void set_receive_timeout(time_point t, abstract_actor* self,
uint32_t id) override;
void set_request_timeout(time_point t, abstract_actor* self,
message_id id) override;
void cancel_receive_timeout(abstract_actor* self) override;
void cancel_request_timeout(abstract_actor* self, message_id id) override;
void cancel_timeouts(abstract_actor* self) override;
void schedule_message(time_point t, strong_actor_ptr receiver,
mailbox_element_ptr content) override;
void schedule_message(time_point t, group target, strong_actor_ptr sender,
message content) override;
void run_dispatch_loop();
void cancel_dispatch_loop();
private:
std::mutex mx_;
std::condition_variable cv_;
std::atomic<bool> done_;
};
} // namespace detail
} // namespace caf
#endif // CAF_DETAIL_THREAD_SAFE_ACTOR_CLOCK_HPP
......@@ -74,6 +74,7 @@ class actor_addr;
class actor_pool;
class message_id;
class serializer;
class actor_clock;
class actor_proxy;
class local_actor;
class ref_counted;
......
......@@ -206,6 +206,11 @@ public:
return context_->system();
}
/// Returns the clock of the actor system.
inline actor_clock& clock() const {
return home_system().clock();
}
/// @cond PRIVATE
void monitor(abstract_actor* ptr);
......
......@@ -26,6 +26,7 @@
#include "caf/actor.hpp"
#include "caf/message.hpp"
#include "caf/duration.hpp"
#include "caf/no_stages.hpp"
#include "caf/response_type.hpp"
#include "caf/response_handle.hpp"
#include "caf/message_priority.hpp"
......@@ -100,9 +101,10 @@ public:
dptr()->context(), std::forward<Ts>(xs)...);
}
template <message_priority P = message_priority::normal,
class Dest = actor, class... Ts>
void delayed_send(const Dest& dest, const duration& rtime, Ts&&... xs) {
template <message_priority P = message_priority::normal, class Rep = int,
class Period = std::ratio<1>, class Dest = actor, class... Ts>
void delayed_send(const Dest& dest, std::chrono::duration<Rep, Period> rtime,
Ts&&... xs) {
using token =
detail::type_list<
typename detail::implicit_conversions<
......@@ -136,23 +138,21 @@ public:
>::type
>::valid,
"this actor does not accept the response message");
if (dest)
dptr()->system().scheduler().delayed_send(
rtime, dptr()->ctrl(), actor_cast<strong_actor_ptr>(dest),
message_id::make(P), make_message(std::forward<Ts>(xs)...));
if (dest) {
auto& clock = dptr()->system().clock();
auto t = clock.now() + rtime;
auto me = make_mailbox_element(dptr()->ctrl(), message_id::make(P),
no_stages, std::forward<Ts>(xs)...);
clock.schedule_message(t, actor_cast<strong_actor_ptr>(dest),
std::move(me));
}
}
template <message_priority P = message_priority::normal,
class Rep = int, class Period = std::ratio<1>,
template <message_priority P = message_priority::normal, class Rep = int,
class Period = std::ratio<1>, class Source = actor,
class Dest = actor, class... Ts>
void delayed_send(const Dest& dest, std::chrono::duration<Rep, Period> rtime,
Ts&&... xs) {
delayed_send(dest, duration{rtime}, std::forward<Ts>(xs)...);
}
template <message_priority P = message_priority::normal,
class Source = actor, class Dest = actor, class... Ts>
void delayed_anon_send(const Dest& dest, const duration& rtime, Ts&&... xs) {
void delayed_anon_send(const Dest& dest,
std::chrono::duration<Rep, Period> rtime, Ts&&... xs) {
static_assert(sizeof...(Ts) > 0, "no message to send");
using token =
detail::type_list<
......@@ -164,19 +164,14 @@ public:
token
>::valid,
"receiver does not accept given message");
if (dest)
dptr()->system().scheduler().delayed_send(
rtime, nullptr, actor_cast<strong_actor_ptr>(dest), message_id::make(P),
make_message(std::forward<Ts>(xs)...));
}
template <message_priority P = message_priority::normal,
class Rep = int, class Period = std::ratio<1>,
class Source = actor, class Dest = actor, class... Ts>
void delayed_anon_send(const Dest& dest,
std::chrono::duration<Rep, Period> rtime,
Ts&&... xs) {
delayed_anon_send(dest, duration{rtime}, std::forward<Ts>(xs)...);
if (dest) {
auto& clock = dptr()->system().clock();
auto t = clock.now() + rtime;
auto me = make_mailbox_element(nullptr, message_id::make(P), no_stages,
std::forward<Ts>(xs)...);
clock.schedule_message(t, actor_cast<strong_actor_ptr>(dest),
std::move(me));
}
}
private:
......
......@@ -30,6 +30,7 @@
#include "caf/duration.hpp"
#include "caf/actor_addr.hpp"
#include "caf/actor_cast.hpp"
#include "caf/actor_clock.hpp"
#include "caf/actor_system.hpp"
namespace caf {
......@@ -43,7 +44,6 @@ class abstract_coordinator : public actor_system::module {
public:
enum utility_actor_id : size_t {
printer_id,
timer_id,
max_id
};
......@@ -54,11 +54,6 @@ public:
return actor_cast<actor>(utility_actors_[printer_id]);
}
/// Returns a handle to the central timer actor.
inline actor timer() const {
return actor_cast<actor>(utility_actors_[timer_id]);
}
/// Returns the number of utility actors.
inline size_t num_utility_actors() const {
return utility_actors_.size();
......@@ -67,16 +62,6 @@ public:
/// Puts `what` into the queue of a randomly chosen worker.
virtual void enqueue(resumable* what) = 0;
template <class Duration, class... Data>
void delayed_send(Duration rel_time, strong_actor_ptr from,
strong_actor_ptr to, message_id mid, message data) {
auto& dest = utility_actors_[timer_id];
dest->enqueue(nullptr, invalid_message_id,
make_message(duration{rel_time}, std::move(from),
std::move(to), mid, std::move(data)),
nullptr);
}
inline actor_system& system() {
return system_;
}
......@@ -102,21 +87,26 @@ public:
static void cleanup_and_release(resumable*);
virtual actor_clock& clock() noexcept = 0;
protected:
void stop_actors();
// ID of the worker receiving the next enqueue
/// ID of the worker receiving the next enqueue (round-robin dispatch).
std::atomic<size_t> next_worker_;
// number of messages each actor is allowed to consume per resume
/// Number of messages each actor is allowed to consume per resume.
size_t max_throughput_;
// configured number of workers
/// Configured number of workers.
size_t num_workers_;
/// Background workers, e.g., printer.
std::array<actor, max_id> utility_actors_;
/// Reference to the host system.
actor_system& system_;
};
} // namespace scheduler
......
......@@ -29,6 +29,8 @@
#include "caf/scheduler/worker.hpp"
#include "caf/scheduler/abstract_coordinator.hpp"
#include "caf/detail/thread_safe_actor_clock.hpp"
namespace caf {
namespace scheduler {
......@@ -68,6 +70,10 @@ protected:
// start all workers now that all workers have been initialized
for (auto& w : workers_)
w->start();
// launch thread for dispatching timeouts and delayed messages
timer_ = std::thread{[&] {
clock_.run_dispatch_loop();
}};
// run remaining startup code
super::start();
}
......@@ -127,19 +133,34 @@ protected:
for (auto& w : workers_)
policy_.foreach_resumable(w.get(), f);
policy_.foreach_central_resumable(this, f);
// stop timer thread
clock_.cancel_dispatch_loop();
timer_.join();
}
void enqueue(resumable* ptr) override {
policy_.central_enqueue(this, ptr);
}
detail::thread_safe_actor_clock& clock() noexcept override {
return clock_;
}
private:
// usually of size std::thread::hardware_concurrency()
/// System-wide clock.
detail::thread_safe_actor_clock clock_;
/// Set of workers.
std::vector<std::unique_ptr<worker_type>> workers_;
// policy-specific data
/// Policy-specific data.
policy_data data_;
// instance of our policy object
/// The policy object.
Policy policy_;
/// Thread for managing timeouts and delayed messages.
std::thread timer_;
};
} // namespace scheduler
......
......@@ -30,6 +30,8 @@
#include "caf/scheduled_actor.hpp"
#include "caf/scheduler/abstract_coordinator.hpp"
#include "caf/detail/test_actor_clock.hpp"
namespace caf {
namespace scheduler {
......@@ -43,20 +45,9 @@ public:
/// A double-ended queue representing our current job queue.
std::deque<resumable*> jobs;
/// A scheduled message or timeout.
struct delayed_msg {
strong_actor_ptr from;
strong_actor_ptr to;
message_id mid;
message msg;
};
/// A clock type using the highest available precision.
using hrc = std::chrono::high_resolution_clock;
/// A map type for storing scheduled messages and timeouts.
std::multimap<hrc::time_point, delayed_msg> delayed_messages;
/// Returns whether at least one job is in the queue.
inline bool has_job() const {
return !jobs.empty();
......@@ -142,6 +133,8 @@ public:
bool detaches_utility_actors() const override;
detail::test_actor_clock& clock() noexcept override;
protected:
void start() override;
......@@ -152,6 +145,9 @@ protected:
private:
void inline_all_enqueues_helper();
/// Allows users to fake time at will.
detail::test_actor_clock clock_;
/// User-provided callback for triggering custom code in `enqueue`.
std::function<void()> after_next_enqueue_;
};
......
......@@ -53,85 +53,6 @@ namespace scheduler {
namespace {
using hrc = std::chrono::high_resolution_clock;
class timer_actor : public blocking_actor {
public:
explicit timer_actor(actor_config& cfg) : blocking_actor(cfg) {
// nop
}
struct delayed_msg {
strong_actor_ptr from;
strong_actor_ptr to;
message_id mid;
message msg;
};
void deliver(delayed_msg& dm) {
dm.to->enqueue(dm.from, dm.mid, std::move(dm.msg), nullptr);
}
template <class Map, class... Ts>
void insert_dmsg(Map& storage, const duration& d, Ts&&... xs) {
auto tout = hrc::now();
tout += d;
delayed_msg dmsg{std::forward<Ts>(xs)...};
storage.emplace(std::move(tout), std::move(dmsg));
}
void act() override {
// local state
accept_one_cond rc;
bool running = true;
std::multimap<hrc::time_point, delayed_msg> messages;
// our message handler
behavior nested{
[&](const duration& d, strong_actor_ptr& from,
strong_actor_ptr& to, message_id mid, message& msg) {
insert_dmsg(messages, d, std::move(from),
std::move(to), mid, std::move(msg));
},
[&](const exit_msg& dm) {
if (dm.reason) {
fail_state(dm.reason);
running = false;
}
}
};
auto bhvr = detail::make_blocking_behavior(
&nested,
others >> [&](message_view& x) -> result<message> {
std::cerr << "*** unexpected message in timer_actor: "
<< to_string(x.content()) << std::endl;
return sec::unexpected_message;
}
);
// loop until receiving an exit message
while (running) {
if (messages.empty()) {
// use regular receive as long as we don't have a pending timeout
receive_impl(rc, message_id::make(), bhvr);
} else {
auto tout = messages.begin()->first;
if (await_data(tout)) {
receive_impl(rc, message_id::make(), bhvr);
} else {
auto it = messages.begin();
while (it != messages.end() && (it->first) <= tout) {
deliver(it->second);
it = messages.erase(it);
}
}
}
}
}
const char* name() const override {
return "timer_actor";
}
};
using string_sink = std::function<void (std::string&&)>;
// the first value is the use count, the last ostream_handle that
......@@ -322,7 +243,6 @@ void abstract_coordinator::start() {
CAF_LOG_TRACE("");
// launch utility actors
static constexpr auto fs = hidden + detached;
utility_actors_[timer_id] = system_.spawn<timer_actor, fs>();
utility_actors_[printer_id] = system_.spawn<printer_actor, fs>();
}
......
/******************************************************************************
* ____ _ _____ *
* / ___| / \ | ___| C++ *
* | | / _ \ | |_ Actor *
* | |___ / ___ \| _| Framework *
* \____/_/ \_|_| *
* *
* Copyright 2011-2018 Dominik Charousset *
* *
* 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 LICENSE_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. *
******************************************************************************/
#include "caf/actor_clock.hpp"
namespace caf {
// -- constructors, destructors, and assignment operators ----------------------
actor_clock::~actor_clock() {
// nop
}
// -- observers ----------------------------------------------------------------
actor_clock::time_point actor_clock::now() const noexcept {
return clock_type::now();
}
actor_clock::duration_type
actor_clock::difference(atom_value, time_point t0,
time_point t1) const noexcept {
return t1 - t0;
}
} // namespace caf
......@@ -221,11 +221,13 @@ actor_system::actor_system(actor_system_config& cfg)
modules_[mod_ptr->id()].reset(mod_ptr);
}
auto& sched = modules_[module::scheduler];
using test = scheduler::test_coordinator;
using share = scheduler::coordinator<policy::work_sharing>;
using steal = scheduler::coordinator<policy::work_stealing>;
using profiled_share = scheduler::profiled_coordinator<policy::profiled<policy::work_sharing>>;
using profiled_steal = scheduler::profiled_coordinator<policy::profiled<policy::work_stealing>>;
using namespace scheduler;
using policy::work_sharing;
using policy::work_stealing;
using share = coordinator<work_sharing>;
using steal = coordinator<work_stealing>;
using profiled_share = profiled_coordinator<policy::profiled<work_sharing>>;
using profiled_steal = profiled_coordinator<policy::profiled<work_stealing>>;
// set scheduler only if not explicitly loaded by user
if (!sched) {
enum sched_conf {
......@@ -262,7 +264,7 @@ actor_system::actor_system(actor_system_config& cfg)
sched.reset(new profiled_share(*this));
break;
case testing:
sched.reset(new test(*this));
sched.reset(new test_coordinator(*this));
}
}
// initialize state for each module and give each module the opportunity
......@@ -402,6 +404,11 @@ void actor_system::await_all_actors_done() const {
registry_.await_running_count_equal(0);
}
actor_clock& actor_system::clock() noexcept {
return scheduler().clock();
}
void actor_system::inc_detached_threads() {
++detached;
}
......
......@@ -65,8 +65,9 @@ void local_actor::request_response_timeout(const duration& d, message_id mid) {
CAF_LOG_TRACE(CAF_ARG(d) << CAF_ARG(mid));
if (!d.valid())
return;
system().scheduler().delayed_send(d, ctrl(), ctrl(), mid.response_id(),
make_message(sec::request_timeout));
auto t = clock().now();
t += d;
clock().set_request_timeout(t, this, mid.response_id());
}
void local_actor::monitor(abstract_actor* ptr) {
......@@ -185,6 +186,7 @@ bool local_actor::cleanup(error&& fail_state, execution_unit* host) {
// tell registry we're done
unregister_from_system();
monitorable_actor::cleanup(std::move(fail_state), host);
clock().cancel_timeouts(this);
CAF_LOG_TERMINATE_EVENT(this, fail_state);
return true;
}
......
......@@ -297,14 +297,16 @@ uint32_t scheduled_actor::request_timeout(const duration& d) {
}
setf(has_timeout_flag);
auto result = ++timeout_id_;
auto msg = make_message(timeout_msg{++timeout_id_});
auto msg = make_message(timeout_msg{result});
CAF_LOG_TRACE("send new timeout_msg, " << CAF_ARG(timeout_id_));
if (d.is_zero())
if (d.is_zero()) {
// immediately enqueue timeout message if duration == 0s
enqueue(ctrl(), invalid_message_id, std::move(msg), context());
else
system().scheduler().delayed_send(d, ctrl(), strong_actor_ptr(ctrl()),
message_id::make(), std::move(msg));
} else {
auto t = clock().now();
t += d;
clock().set_receive_timeout(t, this, result);
}
return result;
}
......
/******************************************************************************
* ____ _ _____ *
* / ___| / \ | ___| C++ *
* | | / _ \ | |_ Actor *
* | |___ / ___ \| _| Framework *
* \____/_/ \_|_| *
* *
* Copyright 2011-2018 Dominik Charousset *
* *
* 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 LICENSE_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. *
******************************************************************************/
#include "caf/detail/simple_actor_clock.hpp"
#include "caf/actor_cast.hpp"
#include "caf/sec.hpp"
#include "caf/system_messages.hpp"
namespace caf {
namespace detail {
bool simple_actor_clock::receive_predicate::
operator()(const secondary_map::value_type& x) const noexcept {
return holds_alternative<receive_timeout>(x.second->second);
}
bool simple_actor_clock::request_predicate::
operator()(const secondary_map::value_type& x) const noexcept {
if (holds_alternative<request_timeout>(x.second->second)) {
auto& rt = get<request_timeout>(x.second->second);
return rt.id == id;
}
return false;
}
void simple_actor_clock::visitor::operator()(receive_timeout& x) {
CAF_ASSERT(x.self != nullptr);
x.self->get()->eq_impl(message_id::make(), x.self, nullptr,
timeout_msg{x.id});
receive_predicate pred;
thisptr->drop_lookup(x.self->get(), pred);
}
void simple_actor_clock::visitor::operator()(request_timeout& x) {
CAF_ASSERT(x.self != nullptr);
x.self->get()->eq_impl(x.id, x.self, nullptr, sec::request_timeout);
request_predicate pred{x.id};
thisptr->drop_lookup(x.self->get(), pred);
}
void simple_actor_clock::visitor::operator()(actor_msg& x) {
x.receiver->enqueue(std::move(x.content), nullptr);
}
void simple_actor_clock::visitor::operator()(group_msg& x) {
x.target->eq_impl(message_id::make(), std::move(x.sender), nullptr,
std::move(x.content));
}
void simple_actor_clock::set_receive_timeout(time_point t, abstract_actor* self,
uint32_t id) {
receive_predicate pred;
auto i = lookup(self, pred);
auto sptr = actor_cast<strong_actor_ptr>(self);
if (i != actor_lookup_.end()) {
schedule_.erase(i->second);
i->second = schedule_.emplace(t, receive_timeout{std::move(sptr), id});
} else {
auto j = schedule_.emplace(t, receive_timeout{std::move(sptr), id});
actor_lookup_.emplace(self, j);
}
}
void simple_actor_clock::set_request_timeout(time_point t, abstract_actor* self,
message_id id) {
request_predicate pred{id};
auto i = lookup(self, pred);
auto sptr = actor_cast<strong_actor_ptr>(self);
if (i != actor_lookup_.end()) {
schedule_.erase(i->second);
i->second = schedule_.emplace(t, request_timeout{std::move(sptr), id});
} else {
auto j = schedule_.emplace(t, request_timeout{std::move(sptr), id});
actor_lookup_.emplace(self, j);
}
}
void simple_actor_clock::cancel_receive_timeout(abstract_actor* self) {
receive_predicate pred;
cancel(self, pred);
}
void simple_actor_clock::cancel_request_timeout(abstract_actor* self,
message_id id) {
request_predicate pred{id};
cancel(self, pred);
}
void simple_actor_clock::cancel_timeouts(abstract_actor* self) {
auto range = actor_lookup_.equal_range(self);
if (range.first == range.second)
return;
for (auto i = range.first; i != range.second; ++i)
schedule_.erase(i->second);
actor_lookup_.erase(range.first, range.second);
}
void simple_actor_clock::schedule_message(time_point t,
strong_actor_ptr receiver,
mailbox_element_ptr content) {
schedule_.emplace(t, actor_msg{std::move(receiver), std::move(content)});
}
void simple_actor_clock::schedule_message(time_point t, group target,
strong_actor_ptr sender,
message content) {
schedule_.emplace(
t, group_msg{std::move(target), std::move(sender), std::move(content)});
}
} // namespace detail
} // namespace caf
/******************************************************************************
* ____ _ _____ *
* / ___| / \ | ___| C++ *
* | | / _ \ | |_ Actor *
* | |___ / ___ \| _| Framework *
* \____/_/ \_|_| *
* *
* Copyright 2011-2018 Dominik Charousset *
* *
* 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 LICENSE_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. *
******************************************************************************/
#include "caf/detail/test_actor_clock.hpp"
namespace caf {
namespace detail {
test_actor_clock::time_point test_actor_clock::now() const noexcept {
return current_time;
}
bool test_actor_clock::dispatch_once() {
if (schedule_.empty())
return false;
visitor f{this};
auto i = schedule_.begin();
visit(f, i->second);
schedule_.erase(i);
return true;
}
size_t test_actor_clock::dispatch() {
if (schedule_.empty())
return 0u;
visitor f{this};
auto result = schedule_.size();
for (auto& kvp : schedule_)
visit(f, kvp.second);
schedule_.clear();
return result;
}
void test_actor_clock::advance_time(duration_type x) {
visitor f{this};
current_time += x;
auto i = schedule_.begin();
while (i != schedule_.end() && i->first <= current_time) {
visit(f, i->second);
i = schedule_.erase(i);
}
}
} // namespace detail
} // namespace caf
......@@ -62,33 +62,6 @@ private:
message_handler mh_;
};
class dummy_timer : public monitorable_actor {
public:
dummy_timer(actor_config& cfg, test_coordinator* parent)
: monitorable_actor(cfg),
parent_(parent) {
mh_.assign(
[&](const duration& d, strong_actor_ptr& from,
strong_actor_ptr& to, message_id mid, message& msg) {
auto tout = test_coordinator::hrc::now();
tout += d;
using delayed_msg = test_coordinator::delayed_msg;
parent_->delayed_messages.emplace(tout, delayed_msg{std::move(from),
std::move(to), mid,
std::move(msg)});
}
);
}
void enqueue(mailbox_element_ptr what, execution_unit*) override {
mh_(what->content());
}
private:
test_coordinator* parent_;
message_handler mh_;
};
} // namespace <anonymous>
test_coordinator::test_coordinator(actor_system& sys) : super(sys) {
......@@ -99,12 +72,14 @@ bool test_coordinator::detaches_utility_actors() const {
return false;
}
detail::test_actor_clock& test_coordinator::clock() noexcept {
return clock_;
}
void test_coordinator::start() {
dummy_worker worker{this};
actor_config cfg{&worker};
auto& sys = system();
utility_actors_[timer_id] = make_actor<dummy_timer, actor>(
sys.next_actor_id(), sys.node(), &sys, cfg, this);
utility_actors_[printer_id] = make_actor<dummy_printer, actor>(
sys.next_actor_id(), sys.node(), &sys, cfg);
}
......@@ -171,20 +146,11 @@ size_t test_coordinator::run(size_t max_count) {
}
bool test_coordinator::dispatch_once() {
auto i = delayed_messages.begin();
if (i == delayed_messages.end())
return false;
auto& dm = i->second;
dm.to->enqueue(dm.from, dm.mid, std::move(dm.msg), nullptr);
delayed_messages.erase(i);
return true;
return clock().dispatch_once();
}
size_t test_coordinator::dispatch() {
size_t res = 0;
while (dispatch_once())
++res;
return res;
return clock().dispatch();
}
std::pair<size_t, size_t> test_coordinator::run_dispatch_loop() {
......
/******************************************************************************
* ____ _ _____ *
* / ___| / \ | ___| C++ *
* | | / _ \ | |_ Actor *
* | |___ / ___ \| _| Framework *
* \____/_/ \_|_| *
* *
* Copyright 2011-2018 Dominik Charousset *
* *
* 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 LICENSE_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. *
******************************************************************************/
#include "caf/detail/thread_safe_actor_clock.hpp"
namespace caf {
namespace detail {
namespace {
using guard_type = std::unique_lock<std::mutex>;
} // namespace <anonymous>
thread_safe_actor_clock::thread_safe_actor_clock() : done_(false) {
// nop
}
void thread_safe_actor_clock::set_receive_timeout(time_point t,
abstract_actor* self,
uint32_t id) {
guard_type guard{mx_};
super::set_receive_timeout(t, self, id);
cv_.notify_all();
}
void thread_safe_actor_clock::set_request_timeout(time_point t,
abstract_actor* self,
message_id id) {
guard_type guard{mx_};
super::set_request_timeout(t, self, id);
cv_.notify_all();
}
void thread_safe_actor_clock::cancel_receive_timeout(abstract_actor* self) {
guard_type guard{mx_};
super::cancel_receive_timeout(self);
cv_.notify_all();
}
void thread_safe_actor_clock::cancel_request_timeout(abstract_actor* self,
message_id id) {
guard_type guard{mx_};
super::cancel_request_timeout(self, id);
cv_.notify_all();
}
void thread_safe_actor_clock::cancel_timeouts(abstract_actor* self) {
guard_type guard{mx_};
super::cancel_timeouts(self);
cv_.notify_all();
}
void thread_safe_actor_clock::schedule_message(time_point t,
strong_actor_ptr receiver,
mailbox_element_ptr content) {
guard_type guard{mx_};
super::schedule_message(t, std::move(receiver), std::move(content));
cv_.notify_all();
}
void thread_safe_actor_clock::schedule_message(time_point t, group target,
strong_actor_ptr sender,
message content) {
guard_type guard{mx_};
super::schedule_message(t, std::move(target), std::move(sender),
std::move(content));
cv_.notify_all();
}
void thread_safe_actor_clock::run_dispatch_loop() {
visitor f{this};
guard_type guard{mx_};
while (done_ == false) {
// Wait for non-empty schedule.
if (schedule_.empty()) {
cv_.wait(guard);
} else {
auto tout = schedule_.begin()->first;
cv_.wait_until(guard, tout);
}
// Double-check whether schedule is non-empty and execute it.
if (!schedule_.empty()) {
auto t = now();
auto i = schedule_.begin();
while (i != schedule_.end() && i->first <= t) {
visit(f, i->second);
i = schedule_.erase(i);
}
}
}
}
void thread_safe_actor_clock::cancel_dispatch_loop() {
guard_type guard{mx_};
done_ = true;
cv_.notify_all();
}
} // namespace detail
} // namespace caf
/******************************************************************************
* ____ _ _____ *
* / ___| / \ | ___| C++ *
* | | / _ \ | |_ Actor *
* | |___ / ___ \| _| Framework *
* \____/_/ \_|_| *
* *
* Copyright 2011-2018 Dominik Charousset *
* *
* 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 LICENSE_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. *
******************************************************************************/
#include "caf/config.hpp"
#define CAF_SUITE actor_clock
#include "caf/test/dsl.hpp"
#include <chrono>
#include <memory>
#include "caf/all.hpp"
#include "caf/detail/test_actor_clock.hpp"
#include "caf/raw_event_based_actor.hpp"
using namespace caf;
namespace {
using std::chrono::seconds;
struct testee_state {
uint32_t timeout_id = 41;
};
behavior testee(stateful_actor<testee_state, raw_event_based_actor>* self,
detail::test_actor_clock* t) {
return {
[=](ok_atom) {
auto n = t->now() + seconds(10);
self->state.timeout_id += 1;
t->set_receive_timeout(n, self, self->state.timeout_id);
},
[=](add_atom) {
auto n = t->now() + seconds(10);
self->state.timeout_id += 1;
auto mid = message_id::make(self->state.timeout_id).response_id();
t->set_request_timeout(n, self, mid);
},
[](const timeout_msg&) {
// nop
},
[](const error&) {
// nop
},
[](const std::string&) {
// nop
},
[=](group& grp) {
self->join(grp);
},
[=](exit_msg& x) {
self->quit(x.reason);
}
};
}
struct fixture : test_coordinator_fixture<> {
detail::test_actor_clock t;
actor aut;
fixture() : aut(sys.spawn<lazy_init>(testee, &t)) {
// nop
}
};
struct tid {
uint32_t value;
};
inline bool operator==(const timeout_msg& x, const tid& y) {
return x.timeout_id == y.value;
}
} // namespace <anonymous>
CAF_TEST_FIXTURE_SCOPE(timer_tests, fixture)
CAF_TEST(single_receive_timeout) {
// Have AUT call t.set_receive_timeout().
self->send(aut, ok_atom::value);
expect((ok_atom), from(self).to(aut).with(_));
CAF_CHECK_EQUAL(t.schedule().size(), 1u);
CAF_CHECK_EQUAL(t.actor_lookup().size(), 1u);
// Advance time to send timeout message.
t.advance_time(seconds(10));
CAF_CHECK_EQUAL(t.schedule().size(), 0u);
CAF_CHECK_EQUAL(t.actor_lookup().size(), 0u);
// Have AUT receive the timeout.
expect((timeout_msg), from(aut).to(aut).with(tid{42}));
}
CAF_TEST(override_receive_timeout) {
// Have AUT call t.set_receive_timeout().
self->send(aut, ok_atom::value);
expect((ok_atom), from(self).to(aut).with(_));
CAF_CHECK_EQUAL(t.schedule().size(), 1u);
CAF_CHECK_EQUAL(t.actor_lookup().size(), 1u);
// Have AUT call t.set_timeout() again.
self->send(aut, ok_atom::value);
expect((ok_atom), from(self).to(aut).with(_));
CAF_CHECK_EQUAL(t.schedule().size(), 1u);
CAF_CHECK_EQUAL(t.actor_lookup().size(), 1u);
// Advance time to send timeout message.
t.advance_time(seconds(10));
CAF_CHECK_EQUAL(t.schedule().size(), 0u);
CAF_CHECK_EQUAL(t.actor_lookup().size(), 0u);
// Have AUT receive the timeout.
expect((timeout_msg), from(aut).to(aut).with(tid{43}));
}
CAF_TEST(single_request_timeout) {
// Have AUT call t.set_request_timeout().
self->send(aut, add_atom::value);
expect((add_atom), from(self).to(aut).with(_));
CAF_CHECK_EQUAL(t.schedule().size(), 1u);
CAF_CHECK_EQUAL(t.actor_lookup().size(), 1u);
// Advance time to send timeout message.
t.advance_time(seconds(10));
CAF_CHECK_EQUAL(t.schedule().size(), 0u);
CAF_CHECK_EQUAL(t.actor_lookup().size(), 0u);
// Have AUT receive the timeout.
expect((error), from(aut).to(aut).with(sec::request_timeout));
}
CAF_TEST(mixed_receive_and_request_timeouts) {
// Have AUT call t.set_receive_timeout().
self->send(aut, ok_atom::value);
expect((ok_atom), from(self).to(aut).with(_));
CAF_CHECK_EQUAL(t.schedule().size(), 1u);
CAF_CHECK_EQUAL(t.actor_lookup().size(), 1u);
// Cause the request timeout to arrive later.
t.advance_time(seconds(5));
// Have AUT call t.set_request_timeout().
self->send(aut, add_atom::value);
expect((add_atom), from(self).to(aut).with(_));
CAF_CHECK_EQUAL(t.schedule().size(), 2u);
CAF_CHECK_EQUAL(t.actor_lookup().size(), 2u);
// Advance time to send receive timeout message.
t.advance_time(seconds(5));
CAF_CHECK_EQUAL(t.schedule().size(), 1u);
CAF_CHECK_EQUAL(t.actor_lookup().size(), 1u);
// Have AUT receive the timeout.
expect((timeout_msg), from(aut).to(aut).with(tid{42}));
// Advance time to send request timeout message.
t.advance_time(seconds(10));
CAF_CHECK_EQUAL(t.schedule().size(), 0u);
CAF_CHECK_EQUAL(t.actor_lookup().size(), 0u);
// Have AUT receive the timeout.
expect((error), from(aut).to(aut).with(sec::request_timeout));
}
CAF_TEST(delay_actor_message) {
// Schedule a message for now + 10s.
auto n = t.now() + seconds(10);
auto autptr = actor_cast<strong_actor_ptr>(aut);
t.schedule_message(n, autptr,
make_mailbox_element(autptr, message_id::make(),
no_stages, "foo"));
CAF_CHECK_EQUAL(t.schedule().size(), 1u);
CAF_CHECK_EQUAL(t.actor_lookup().size(), 0u);
// Advance time to send the message.
t.advance_time(seconds(10));
CAF_CHECK_EQUAL(t.schedule().size(), 0u);
CAF_CHECK_EQUAL(t.actor_lookup().size(), 0u);
// Have AUT receive the message.
expect((std::string), from(aut).to(aut).with("foo"));
}
CAF_TEST(delay_group_message) {
// Have AUT join the group.
auto grp = sys.groups().anonymous();
self->send(aut, grp);
expect((group), from(self).to(aut).with(_));
// Schedule a message for now + 10s.
auto n = t.now() + seconds(10);
auto autptr = actor_cast<strong_actor_ptr>(aut);
t.schedule_message(n, std::move(grp), autptr, make_message("foo"));
CAF_CHECK_EQUAL(t.schedule().size(), 1u);
CAF_CHECK_EQUAL(t.actor_lookup().size(), 0u);
// Advance time to send the message.
t.advance_time(seconds(10));
CAF_CHECK_EQUAL(t.schedule().size(), 0u);
CAF_CHECK_EQUAL(t.actor_lookup().size(), 0u);
// Have AUT receive the message.
expect((std::string), from(aut).to(aut).with("foo"));
// Kill AUT (necessary because the group keeps a reference around).
self->send_exit(aut, exit_reason::kill);
expect((exit_msg), from(self).to(aut).with(_));
}
CAF_TEST_FIXTURE_SCOPE_END()
......@@ -280,7 +280,6 @@ struct fixture {
self(system),
sched(dynamic_cast<scheduler::test_coordinator&>(system.scheduler())) {
CAF_REQUIRE(sched.jobs.empty());
CAF_REQUIRE(sched.delayed_messages.empty());
}
~fixture() {
......@@ -335,8 +334,7 @@ CAF_TEST(nested_timeout) {
// not respond to the message yet, i.e., timeout arrives before response
sched.run();
// dispatch second timeout
CAF_REQUIRE(!sched.delayed_messages.empty());
sched.dispatch();
CAF_REQUIRE_EQUAL(sched.dispatch(), true);
CAF_REQUIRE_EQUAL(sched.next_job<local_actor>().name(), string{"ping"});
CAF_CHECK(!had_timeout);
CAF_CHECK(sched.next_job<ping_actor>().state.had_first_timeout);
......
......@@ -87,7 +87,6 @@ struct fixture {
self(system),
sched(dynamic_cast<scheduler::test_coordinator&>(system.scheduler())) {
CAF_REQUIRE(sched.jobs.empty());
CAF_REQUIRE(sched.delayed_messages.empty());
}
~fixture() {
......
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