Commit 645d595e authored by Dominik Charousset's avatar Dominik Charousset

Implement run_delayed and re-implement timeouts

With the new actions and disposables, we no longer need special-purpose
message types such as `timeout_msg` and can instead use the new
abstraction consistently.

The new actor clock interface also reflects this by replacing the
various timeout types with a generic, action-based interface.
parent 98cfdbfa
......@@ -67,6 +67,7 @@ caf_add_component(
src/abstract_actor.cpp
src/abstract_channel.cpp
src/abstract_group.cpp
src/action.cpp
src/actor.cpp
src/actor_addr.cpp
src/actor_clock.cpp
......@@ -97,7 +98,6 @@ caf_add_component(
src/deserializer.cpp
src/detail/abstract_worker.cpp
src/detail/abstract_worker_hub.cpp
src/detail/action.cpp
src/detail/append_percent_encoded.cpp
src/detail/base64.cpp
src/detail/behavior_impl.cpp
......@@ -127,7 +127,6 @@ caf_add_component(
src/detail/serialized_size.cpp
src/detail/set_thread_name.cpp
src/detail/shared_spinlock.cpp
src/detail/simple_actor_clock.cpp
src/detail/size_based_credit_controller.cpp
src/detail/stringification_inspector.cpp
src/detail/sync_request_bouncer.cpp
......@@ -193,12 +192,12 @@ caf_add_component(
src/string_algorithms.cpp
src/string_view.cpp
src/telemetry/collector/prometheus.cpp
src/telemetry/importer/process.cpp
src/telemetry/label.cpp
src/telemetry/label_view.cpp
src/telemetry/metric.cpp
src/telemetry/metric_family.cpp
src/telemetry/metric_registry.cpp
src/telemetry/importer/process.cpp
src/term.cpp
src/thread_hook.cpp
src/timestamp.cpp
......@@ -213,6 +212,7 @@ caf_add_component(
test/core-test.cpp
test/nasty.cpp
TEST_SUITES
action
actor_clock
actor_factory
actor_lifetime
......@@ -241,7 +241,6 @@ caf_add_component(
decorator.sequencer
deep_to_string
detached_actors
detail.action
detail.base64
detail.bounds_checker
detail.config_consumer
......
......@@ -52,12 +52,12 @@ public:
/// implementation is required to call `super::destroy()` at the end.
virtual void on_destroy();
void enqueue(strong_actor_ptr sender, message_id mid, message msg,
bool enqueue(strong_actor_ptr sender, message_id mid, message msg,
execution_unit* host) override;
/// Enqueues a new message wrapped in a `mailbox_element` to the actor.
/// This `enqueue` variant allows to define forwarding chains.
virtual void enqueue(mailbox_element_ptr what, execution_unit* host) = 0;
virtual bool enqueue(mailbox_element_ptr what, execution_unit* host) = 0;
/// Attaches `ptr` to this actor. The actor will call `ptr->detach(...)` on
/// exit, or immediately if it already finished execution.
......@@ -103,15 +103,14 @@ public:
virtual mailbox_element* peek_at_next_mailbox_element();
template <class... Ts>
void eq_impl(message_id mid, strong_actor_ptr sender, execution_unit* ctx,
bool eq_impl(message_id mid, strong_actor_ptr sender, execution_unit* ctx,
Ts&&... xs) {
enqueue(make_mailbox_element(std::move(sender), mid, {},
std::forward<Ts>(xs)...),
ctx);
return enqueue(make_mailbox_element(std::move(sender), mid, {},
std::forward<Ts>(xs)...),
ctx);
}
// flags storing runtime information used by ...
static constexpr int has_timeout_flag = 0x0004; // single_timeout
static constexpr int is_registered_flag = 0x0008; // (several actors)
static constexpr int is_initialized_flag = 0x0010; // event-based actors
static constexpr int is_blocking_flag = 0x0020; // blocking_actor
......
......@@ -23,7 +23,7 @@ public:
virtual ~abstract_channel();
/// Enqueues a new message without forwarding stack to the channel.
virtual void enqueue(strong_actor_ptr sender, message_id mid, message content,
virtual bool enqueue(strong_actor_ptr sender, message_id mid, message content,
execution_unit* host = nullptr)
= 0;
......
......@@ -13,53 +13,51 @@
#include <atomic>
namespace caf::detail {
namespace caf {
/// A functional interface similar to `std::function<void()>` with dispose
/// semantics.
class CAF_CORE_EXPORT action {
public:
// -- member types -----------------------------------------------------------
/// Describes the current state of an `action`.
enum class state {
disposed,
scheduled,
invoked,
disposed, /// The action may no longer run.
scheduled, /// The action is scheduled for execution.
invoked, /// The action fired and needs rescheduling before running again.
waiting, /// The action waits for reschedule but didn't run yet.
};
class impl : public ref_counted, public disposable::impl {
public:
friend class action;
impl();
void dispose() override;
bool disposed() const noexcept override;
void ref_disposable() const noexcept override;
/// Describes the result of an attempted state transition.
enum class transition {
success, /// Transition completed as expected.
disposed, /// No transition since the action has been disposed.
failure, /// No transition since preconditions did not hold.
};
void deref_disposable() const noexcept override;
/// Internal interface of `action`.
class impl : public disposable::impl {
public:
virtual transition reschedule() = 0;
/// Tries setting the state from `invoked` back to `scheduled`.
/// @return the new state.
state reschedule();
virtual transition run() = 0;
/// Runs the action if the state is `scheduled`.
virtual void run() = 0;
virtual state current_state() const noexcept = 0;
friend void intrusive_ptr_add_ref(const impl* ptr) noexcept {
ptr->ref();
ptr->ref_disposable();
}
friend void intrusive_ptr_release(const impl* ptr) noexcept {
ptr->deref();
ptr->deref_disposable();
}
protected:
std::atomic<state> state_;
};
using impl_ptr = intrusive_ptr<impl>;
// -- constructors, destructors, and assignment operators --------------------
action(impl_ptr ptr) noexcept : pimpl_(std::move(ptr)) {
// nop
}
......@@ -74,77 +72,165 @@ public:
action& operator=(const action&) noexcept = default;
/// Runs the action if it is still scheduled for execution, does nothing
/// otherwise.
void run();
/// Cancel the action if it has not been invoked yet.
void dispose() {
pimpl_->dispose();
}
// -- observers --------------------------------------------------------------
[[nodiscard]] bool disposed() const {
return pimpl_->state_ == state::disposed;
return pimpl_->current_state() == state::disposed;
}
[[nodiscard]] bool scheduled() const {
return pimpl_->state_ == state::scheduled;
return pimpl_->current_state() == state::scheduled;
}
[[nodiscard]] bool invoked() const {
return pimpl_->state_ == state::invoked;
return pimpl_->current_state() == state::invoked;
}
// -- mutators ---------------------------------------------------------------
/// Tries to transition from `scheduled` to `invoked`, running the body of the
/// internal function object as a side effect on success.
/// @return whether the transition took place.
transition run();
/// Cancel the action if it has not been invoked yet.
void dispose() {
pimpl_->dispose();
}
/// Tries setting the state from `invoked` back to `scheduled`.
/// @return the new state.
state reschedule() {
/// @return whether the transition took place.
transition reschedule() {
return pimpl_->reschedule();
}
/// Returns a pointer to the implementation.
[[nodiscard]] impl* ptr() noexcept {
return pimpl_.get();
// -- conversion -------------------------------------------------------------
/// Returns a smart pointer to the implementation.
[[nodiscard]] disposable as_disposable() && noexcept {
return disposable{std::move(pimpl_)};
}
/// Returns a smart pointer to the implementation.
[[nodiscard]] disposable as_disposable() const& noexcept {
return disposable{pimpl_};
}
/// Returns a pointer to the implementation.
[[nodiscard]] const impl* ptr() const noexcept {
[[nodiscard]] impl* ptr() const noexcept {
return pimpl_.get();
}
/// Returns a smart pointer to the implementation.
[[nodiscard]] disposable as_disposable() && noexcept {
return disposable{std::move(pimpl_)};
[[nodiscard]] impl_ptr&& as_intrusive_ptr() && noexcept {
return std::move(pimpl_);
}
/// Returns a smart pointer to the implementation.
[[nodiscard]] disposable as_disposable() const& noexcept {
return disposable{pimpl_};
[[nodiscard]] impl_ptr as_intrusive_ptr() const& noexcept {
return pimpl_;
}
private:
impl_ptr pimpl_;
};
} // namespace caf
namespace caf::detail {
template <class F>
action make_action(F f) {
struct impl : action::impl {
F f_;
explicit impl(F fn) : f_(std::move(fn)) {
// nop
struct default_action_impl : ref_counted, action::impl {
std::atomic<action::state> state_;
F f_;
default_action_impl(F fn, action::state init_state)
: state_(init_state), f_(std::move(fn)) {
// nop
}
void dispose() override {
state_ = action::state::disposed;
}
bool disposed() const noexcept override {
return state_.load() == action::state::disposed;
}
action::state current_state() const noexcept override {
return state_.load();
}
action::transition reschedule() override {
auto st = action::state::invoked;
for (;;) {
if (state_.compare_exchange_strong(st, action::state::scheduled))
return action::transition::success;
switch (st) {
case action::state::invoked:
case action::state::waiting:
break; // Try again.
case action::state::disposed:
return action::transition::disposed;
default:
return action::transition::failure;
}
}
void run() override {
if (state_ == action::state::scheduled) {
}
action::transition run() override {
auto st = state_.load();
switch (st) {
case action::state::scheduled:
f_();
auto expected = action::state::scheduled;
// No retry. If this action has been disposed while running, we stay in
// the state 'disposed'.
state_.compare_exchange_strong(expected, action::state::invoked);
}
// No retry. If this action has been disposed while running, we stay
// in the state 'disposed'. We assume that only one thread may try to
// transition from scheduled to invoked, while other threads may only
// dispose the action.
if (state_.compare_exchange_strong(st, action::state::invoked)) {
return action::transition::success;
} else {
CAF_ASSERT(st == action::state::disposed);
return action::transition::disposed;
}
case action::state::disposed:
return action::transition::disposed;
default:
return action::transition::failure;
}
};
return action{make_counted<impl>(std::move(f))};
}
}
void ref_disposable() const noexcept override {
ref();
}
void deref_disposable() const noexcept override {
deref();
}
friend void intrusive_ptr_add_ref(const default_action_impl* ptr) noexcept {
ptr->ref();
}
friend void intrusive_ptr_release(const default_action_impl* ptr) noexcept {
ptr->deref();
}
};
} // namespace caf::detail
CAF_ALLOW_UNSAFE_MESSAGE_TYPE(caf::detail::action)
namespace caf {
/// Convenience function for creating @ref action objects from a function
/// object.
/// @param f The body for the action.
/// @param init_state either `action::state::scheduled` or
/// `action::state::waiting`.
template <class F>
action make_action(F f, action::state init_state = action::state::scheduled) {
using impl_t = detail::default_action_impl<F>;
return action{make_counted<impl_t>(std::move(f), init_state)};
}
} // namespace caf
CAF_ALLOW_UNSAFE_MESSAGE_TYPE(caf::action)
......@@ -26,53 +26,93 @@ public:
/// Time interval.
using duration_type = typename clock_type::duration;
/// Configures how the clock responds to a stalling actor when trying to
/// schedule a periodic action.
enum class stall_policy {
fail, /// Causes the clock to dispose an action send an error to the actor.
skip, /// Causes the clock to skip scheduled runs without emitting errors.
};
// -- constructors, destructors, and assignment operators --------------------
virtual ~actor_clock();
// -- observers --------------------------------------------------------------
// -- scheduling -------------------------------------------------------------
/// Returns the current wall-clock time.
virtual time_point now() const noexcept;
/// Schedules a `timeout_msg` for `self` at time point `t`, overriding any
/// previous receive timeout.
virtual void set_ordinary_timeout(time_point t, abstract_actor* self,
std::string type, uint64_t id)
= 0;
/// Schedules a `timeout_msg` for `self` at time point `t`.
virtual void set_multi_timeout(time_point t, abstract_actor* self,
std::string type, uint64_t id)
/// Schedules an action for execution.
/// @param f The action to schedule.
/// @note The action runs on the thread of the clock worker and thus must
/// complete within a very short time in order to not delay other work.
disposable schedule(action f);
/// Schedules an action for execution at a later time.
/// @param t The local time at which the action should run.
/// @param f The action to schedule.
/// @note The action runs on the thread of the clock worker and thus must
/// complete within a very short time in order to not delay other work.
disposable schedule(time_point t, action f);
/// Schedules an action for periodic execution.
/// @param first_run The local time at which the action should run initially.
/// @param f The action to schedule.
/// @param period The time to wait between two runs. A non-positive period
/// disables periodic execution.
/// @note The action runs on the thread of the clock worker and thus must
/// complete within a very short time in order to not delay other work.
virtual disposable
schedule_periodically(time_point first_run, action f, duration_type period)
= 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_ordinary_timeout(abstract_actor* self, std::string type)
= 0;
/// Schedules an action for execution by an actor at a later time.
/// @param t The local time at which the action gets enqueued to the mailbox
/// of the worker.
/// @param f The action to schedule.
/// @param worker The actor that should run the action.
disposable schedule(time_point t, action f, strong_actor_ptr worker);
/// Schedules an action for periodic execution by an actor.
/// @param first_run The local time at which the action gets enqueued to the
/// mailbox of the worker for the first time.
/// @param f The action to schedule.
/// @param worker The actor that should run the action.
/// @param period The time to wait between two messages to the actor.
/// @param policy The strategy for dealing with a stalling worker.
disposable schedule_periodically(time_point first_run, action f,
strong_actor_ptr worker,
duration_type period, stall_policy policy);
/// Schedules an action for execution by an actor at a later time.
/// @param worker The actor that should run the action.
/// @param f The action to schedule.
/// @param t The local time at which the action gets enqueued to the mailbox
/// of the worker.
disposable schedule(time_point t, action f, weak_actor_ptr worker);
/// Schedules an action for periodic execution by an actor.
/// @param worker The actor that should run the action.
/// @param policy The strategy for dealing with a stalling worker.
/// @param f The action to schedule.
/// @param first_run The local time at which the action gets enqueued to the
/// mailbox of the worker for the first time.
/// @param period The time to wait between two messages to the actor.
disposable schedule_periodically(time_point first_run, action f,
weak_actor_ptr worker, duration_type period,
stall_policy policy);
/// 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`.
disposable schedule_message(time_point t, strong_actor_ptr receiver,
mailbox_element_ptr content);
/// 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;
disposable schedule_message(time_point t, weak_actor_ptr receiver,
mailbox_element_ptr content);
/// 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;
/// Cancels all timeouts and scheduled messages.
virtual void cancel_all() = 0;
disposable schedule_message(time_point t, group target,
strong_actor_ptr sender, message content);
};
} // namespace caf
......@@ -62,9 +62,9 @@ public:
// -- overridden functions ---------------------------------------------------
void enqueue(mailbox_element_ptr ptr, execution_unit* host) override;
bool enqueue(mailbox_element_ptr ptr, execution_unit* host) override;
void enqueue(strong_actor_ptr src, message_id mid, message content,
bool enqueue(strong_actor_ptr src, message_id mid, message content,
execution_unit* eu) override;
void launch(execution_unit* eu, bool lazy, bool hide) override;
......
......@@ -115,10 +115,10 @@ public:
return nid;
}
void enqueue(strong_actor_ptr sender, message_id mid, message content,
bool enqueue(strong_actor_ptr sender, message_id mid, message content,
execution_unit* host);
void enqueue(mailbox_element_ptr what, execution_unit* host);
bool enqueue(mailbox_element_ptr what, execution_unit* host);
/// @endcond
};
......
......@@ -85,7 +85,7 @@ public:
static actor
make(execution_unit* eu, size_t num_workers, const factory& fac, policy pol);
void enqueue(mailbox_element_ptr what, execution_unit* eu) override;
bool enqueue(mailbox_element_ptr what, execution_unit* eu) override;
actor_pool(actor_config& cfg);
......
......@@ -220,7 +220,7 @@ public:
// -- overridden functions of abstract_actor ---------------------------------
void enqueue(mailbox_element_ptr, execution_unit*) override;
bool enqueue(mailbox_element_ptr, execution_unit*) override;
mailbox_element* peek_at_next_mailbox_element() override;
......
......@@ -28,7 +28,7 @@ public:
// non-system messages are processed and then forwarded;
// system messages are handled and consumed on the spot;
// in either case, the processing is done synchronously
void enqueue(mailbox_element_ptr what, execution_unit* context) override;
bool enqueue(mailbox_element_ptr what, execution_unit* context) override;
message_types_set message_types() const override;
......
......@@ -29,7 +29,7 @@ public:
// non-system messages are processed and then forwarded;
// system messages are handled and consumed on the spot;
// in either case, the processing is done synchronously
void enqueue(mailbox_element_ptr what, execution_unit* context) override;
bool enqueue(mailbox_element_ptr what, execution_unit* context) override;
message_types_set message_types() const override;
......
// This file is part of CAF, the C++ Actor Framework. See the file LICENSE in
// the main distribution directory for license terms and copyright or visit
// https://github.com/actor-framework/actor-framework/blob/master/LICENSE.
#pragma once
#include "caf/detail/type_traits.hpp"
#include "caf/disposable.hpp"
namespace caf::detail {
template <class Signature>
struct dispose_on_call_t;
template <class R, class... Ts>
struct dispose_on_call_t<R(Ts...)> {
template <class F>
auto operator()(disposable resource, F f) {
return [resource{std::move(resource)}, f{std::move(f)}](Ts... xs) mutable {
resource.dispose();
return f(xs...);
};
}
};
/// Returns a decorator for the function object `f` that calls
/// `resource.dispose()` before invoking `f`.
template <class F>
auto dispose_on_call(disposable resource, F f) {
using sig = typename get_callable_trait_t<F>::fun_sig;
dispose_on_call_t<sig> factory;
return factory(std::move(resource), std::move(f));
}
} // namespace caf::detail
......@@ -40,7 +40,7 @@ public:
void unsubscribe(const actor_control_block* who) override;
// Locally enqueued message, forwarded via worker_.
void enqueue(strong_actor_ptr sender, message_id mid, message content,
bool enqueue(strong_actor_ptr sender, message_id mid, message content,
execution_unit* host) override;
void stop() override;
......
......@@ -54,7 +54,7 @@ public:
~impl() override;
void enqueue(strong_actor_ptr sender, message_id mid, message content,
bool enqueue(strong_actor_ptr sender, message_id mid, message content,
execution_unit* host) override;
bool subscribe(strong_actor_ptr who) override;
......
......@@ -94,6 +94,11 @@ public:
cv_empty_.notify_all();
}
template <class... Us>
void emplace_back(Us&&... xs) {
push_back(T{std::forward<Us>(xs)...});
}
bool empty() const noexcept {
return rd_pos_ == wr_pos_;
}
......
This diff is collapsed.
......@@ -4,22 +4,45 @@
#pragma once
#include <algorithm>
#include <map>
#include "caf/action.hpp"
#include "caf/actor_clock.hpp"
#include "caf/detail/core_export.hpp"
#include "caf/detail/simple_actor_clock.hpp"
namespace caf::detail {
class CAF_CORE_EXPORT test_actor_clock : public simple_actor_clock {
class CAF_CORE_EXPORT test_actor_clock : public actor_clock {
public:
time_point current_time;
// -- member types -----------------------------------------------------------
struct schedule_entry {
action f;
duration_type period;
};
using schedule_map = std::multimap<time_point, schedule_entry>;
// -- constructors, destructors, and assignment operators --------------------
test_actor_clock();
// -- overrides --------------------------------------------------------------
time_point now() const noexcept override;
disposable schedule_periodically(time_point first_run, action f,
duration_type period) override;
// -- testing DSL API --------------------------------------------------------
/// Returns whether the actor clock has at least one pending timeout.
bool has_pending_timeout() const {
return !schedule_.empty();
auto not_disposed = [](const auto& kvp) {
return !kvp.second.f.disposed();
};
return std::any_of(schedule.begin(), schedule.end(), not_disposed);
}
/// Triggers the next pending timeout regardless of its timestamp. Sets
......@@ -37,6 +60,15 @@ public:
/// Advances the time by `x` and dispatches timeouts and delayed messages.
/// @returns The number of triggered timeouts.
size_t advance_time(duration_type x);
// -- member variables -------------------------------------------------------
time_point current_time;
schedule_map schedule;
private:
bool try_trigger_once();
};
} // namespace caf::detail
......@@ -4,21 +4,20 @@
#pragma once
#include <array>
#include <atomic>
#include <condition_variable>
#include <memory>
#include <mutex>
#include <string>
#include <thread>
#include <vector>
#include "caf/abstract_actor.hpp"
#include "caf/action.hpp"
#include "caf/actor_clock.hpp"
#include "caf/actor_control_block.hpp"
#include "caf/detail/core_export.hpp"
#include "caf/detail/ringbuffer.hpp"
#include "caf/detail/simple_actor_clock.hpp"
#include "caf/fwd.hpp"
namespace caf::detail {
class CAF_CORE_EXPORT thread_safe_actor_clock : public simple_actor_clock {
class CAF_CORE_EXPORT thread_safe_actor_clock : public actor_clock {
public:
// -- constants --------------------------------------------------------------
......@@ -26,45 +25,47 @@ public:
// -- member types -----------------------------------------------------------
using super = simple_actor_clock;
/// Stores actions along with their scheduling period.
struct schedule_entry {
time_point t;
action f;
duration_type period;
};
// -- member functions -------------------------------------------------------
/// @relates schedule_entry
using schedule_entry_ptr = std::unique_ptr<schedule_entry>;
void set_ordinary_timeout(time_point t, abstract_actor* self,
std::string type, uint64_t id) override;
// -- constructors, destructors, and assignment operators --------------------
void set_request_timeout(time_point t, abstract_actor* self,
message_id id) override;
thread_safe_actor_clock();
void set_multi_timeout(time_point t, abstract_actor* self, std::string type,
uint64_t id) override;
// -- overrides --------------------------------------------------------------
void cancel_ordinary_timeout(abstract_actor* self, std::string type) override;
disposable schedule_periodically(time_point first_run, action f,
duration_type period) override;
void cancel_request_timeout(abstract_actor* self, message_id id) override;
// -- thread management ------------------------------------------------------
void cancel_timeouts(abstract_actor* self) override;
void start_dispatch_loop(caf::actor_system& sys);
void schedule_message(time_point t, strong_actor_ptr receiver,
mailbox_element_ptr content) override;
void stop_dispatch_loop();
void schedule_message(time_point t, group target, strong_actor_ptr sender,
message content) override;
void cancel_all() override;
private:
void run();
void run_dispatch_loop();
// -- member variables -------------------------------------------------------
void cancel_dispatch_loop();
/// Communication to the dispatcher thread.
detail::ringbuffer<schedule_entry_ptr, buffer_size> queue_;
private:
void push(event* ptr);
/// Handle to the dispatcher thread.
std::thread dispatcher_;
/// Receives timer events from other threads.
detail::ringbuffer<unique_event_ptr, buffer_size> queue_;
/// Internal data of the dispatcher.
bool running_ = true;
/// Locally caches events for processing.
std::array<unique_event_ptr, buffer_size> events_;
/// Internal data of the dispatcher.
std::vector<schedule_entry_ptr> tbl_;
};
} // namespace caf::detail
......@@ -4,6 +4,8 @@
#pragma once
#include <vector>
#include "caf/detail/core_export.hpp"
#include "caf/intrusive_ptr.hpp"
#include "caf/ref_counted.hpp"
......@@ -48,6 +50,11 @@ public:
disposable& operator=(disposable&&) noexcept = default;
disposable& operator=(const disposable&) noexcept = default;
/// Combines multiple disposable into a single disposable. The new disposable
/// is disposed if all of its elements are disposed. Disposing the composite
/// disposes all elements individually.
static disposable make_composite(std::vector<disposable> entries);
/// Disposes the resource. Calling `dispose()` on a disposed resource is a
/// no-op.
void dispose() {
......@@ -78,12 +85,7 @@ public:
}
/// Returns a pointer to the implementation.
[[nodiscard]] impl* ptr() noexcept {
return pimpl_.get();
}
/// Returns a pointer to the implementation.
[[nodiscard]] const impl* ptr() const noexcept {
[[nodiscard]] impl* ptr() const noexcept {
return pimpl_.get();
}
......
......@@ -20,7 +20,7 @@ public:
~forwarding_actor_proxy() override;
void enqueue(mailbox_element_ptr what, execution_unit* context) override;
bool enqueue(mailbox_element_ptr what, execution_unit* context) override;
bool add_backlink(abstract_actor* x) override;
......@@ -29,7 +29,7 @@ public:
void kill_proxy(execution_unit* ctx, error rsn) override;
private:
void forward_msg(strong_actor_ptr sender, message_id mid, message msg,
bool forward_msg(strong_actor_ptr sender, message_id mid, message msg,
const forwarding_stack* fwd = nullptr);
mutable detail::shared_spinlock broker_mtx_;
......
......@@ -75,6 +75,7 @@ template <class, class...> class outbound_stream_slot;
class [[nodiscard]] error;
class abstract_actor;
class abstract_group;
class action;
class actor;
class actor_addr;
class actor_clock;
......@@ -96,6 +97,7 @@ class config_option_adder;
class config_option_set;
class config_value;
class deserializer;
class disposable;
class downstream_manager;
class downstream_manager_base;
class event_based_actor;
......@@ -159,7 +161,6 @@ struct none_t;
struct open_stream_msg;
struct prohibit_top_level_spawn_marker;
struct stream_slots;
struct timeout_msg;
struct unit_t;
struct upstream_msg;
struct upstream_msg_ack_batch;
......@@ -347,7 +348,6 @@ class stream_distribution_tree;
class abstract_worker;
class abstract_worker_hub;
class action;
class disposer;
class dynamic_message_data;
class group_manager;
......
......@@ -110,7 +110,7 @@ public:
/// Requests a new timeout for `mid`.
/// @pre `mid.is_request()`
void request_response_timeout(timespan d, message_id mid);
disposable request_response_timeout(timespan d, message_id mid);
// -- spawn functions --------------------------------------------------------
......
......@@ -11,6 +11,7 @@
#include "caf/actor.hpp"
#include "caf/check_typed_input.hpp"
#include "caf/detail/profiled_send.hpp"
#include "caf/disposable.hpp"
#include "caf/fwd.hpp"
#include "caf/message.hpp"
#include "caf/message_id.hpp"
......@@ -54,10 +55,11 @@ public:
"receiver does not accept given message");
auto self = static_cast<Subtype*>(this);
auto req_id = self->new_request_id(P);
auto pending_msg = disposable{};
if (dest) {
detail::profiled_send(self, self->ctrl(), dest, req_id, {},
self->context(), std::forward<Ts>(xs)...);
self->request_response_timeout(timeout, req_id);
pending_msg = self->request_response_timeout(timeout, req_id);
} else {
self->eq_impl(req_id.response_id(), self->ctrl(), self->context(),
make_error(sec::invalid_argument));
......@@ -68,7 +70,7 @@ public:
detail::implicit_conversions_t<detail::decay_t<Ts>>...>;
using handle_type
= response_handle<Subtype, policy::single_response<response_type>>;
return handle_type{self, req_id.response_id()};
return handle_type{self, req_id.response_id(), std::move(pending_msg)};
}
/// Sends `{xs...}` to each actor in the range `destinations` as a synchronous
......@@ -106,13 +108,16 @@ public:
auto dptr = static_cast<Subtype*>(this);
std::vector<message_id> ids;
ids.reserve(destinations.size());
std::vector<disposable> pending_msgs;
pending_msgs.reserve(destinations.size());
for (const auto& dest : destinations) {
if (!dest)
continue;
auto req_id = dptr->new_request_id(Prio);
dest->eq_impl(req_id, dptr->ctrl(), dptr->context(),
std::forward<Ts>(xs)...);
dptr->request_response_timeout(timeout, req_id);
pending_msgs.emplace_back(
dptr->request_response_timeout(timeout, req_id));
ids.emplace_back(req_id.response_id());
}
if (ids.empty()) {
......@@ -125,7 +130,8 @@ public:
= response_type_t<typename handle_type::signatures,
detail::implicit_conversions_t<detail::decay_t<Ts>>...>;
using result_type = response_handle<Subtype, MergePolicy<response_type>>;
return result_type{dptr, std::move(ids)};
return result_type{dptr, std::move(ids),
disposable::make_composite(std::move(pending_msgs))};
}
};
......
......@@ -15,60 +15,54 @@
#include "caf/detail/type_list.hpp"
#include "caf/detail/type_traits.hpp"
#include "caf/detail/typed_actor_util.hpp"
#include "caf/disposable.hpp"
#include "caf/error.hpp"
#include "caf/logger.hpp"
#include "caf/message_id.hpp"
namespace caf::detail {
template <class F, class T>
struct select_all_helper {
std::vector<T> results;
std::shared_ptr<size_t> pending;
F f;
void operator()(T& x) {
CAF_LOG_TRACE(CAF_ARG2("pending", *pending));
if (*pending > 0) {
results.emplace_back(std::move(x));
if (--*pending == 0)
f(std::move(results));
}
}
template <class Fun>
select_all_helper(size_t pending, Fun&& f)
: pending(std::make_shared<size_t>(pending)), f(std::forward<Fun>(f)) {
results.reserve(pending);
}
template <class... Ts>
struct select_all_helper_value_oracle {
using type = std::tuple<Ts...>;
};
auto wrap() {
return [this](T& x) { (*this)(x); };
}
template <class T>
struct select_all_helper_value_oracle<T> {
using type = T;
};
template <class... Ts>
using select_all_helper_value_t =
typename select_all_helper_value_oracle<Ts...>::type;
template <class F, class... Ts>
struct select_all_tuple_helper {
using value_type = std::tuple<Ts...>;
struct select_all_helper {
using value_type = select_all_helper_value_t<Ts...>;
std::vector<value_type> results;
std::shared_ptr<size_t> pending;
disposable timeouts;
F f;
template <class Fun>
select_all_helper(size_t pending, disposable timeouts, Fun&& f)
: pending(std::make_shared<size_t>(pending)),
timeouts(std::move(timeouts)),
f(std::forward<Fun>(f)) {
results.reserve(pending);
}
void operator()(Ts&... xs) {
CAF_LOG_TRACE(CAF_ARG2("pending", *pending));
if (*pending > 0) {
results.emplace_back(std::move(xs)...);
if (--*pending == 0)
if (--*pending == 0) {
timeouts.dispose();
f(std::move(results));
}
}
}
template <class Fun>
select_all_tuple_helper(size_t pending, Fun&& f)
: pending(std::make_shared<size_t>(pending)), f(std::forward<Fun>(f)) {
results.reserve(pending);
}
auto wrap() {
return [this](Ts&... xs) { (*this)(xs...); };
}
......@@ -80,7 +74,7 @@ struct select_select_all_helper;
template <class F, class... Ts>
struct select_select_all_helper<
F, detail::type_list<std::vector<std::tuple<Ts...>>>> {
using type = select_all_tuple_helper<F, Ts...>;
using type = select_all_helper<F, Ts...>;
};
template <class F, class T>
......@@ -113,7 +107,8 @@ public:
= detail::type_checker<response_type,
detail::select_all_helper_t<detail::decay_t<Fun>>>;
explicit select_all(message_id_list ids) : ids_(std::move(ids)) {
explicit select_all(message_id_list ids, disposable pending_timeouts)
: ids_(std::move(ids)), pending_timeouts_(std::move(pending_timeouts)) {
CAF_ASSERT(ids_.size()
<= static_cast<size_t>(std::numeric_limits<int>::max()));
}
......@@ -123,7 +118,7 @@ public:
select_all& operator=(select_all&&) noexcept = default;
template <class Self, class F, class OnError>
void await(Self* self, F&& f, OnError&& g) const {
void await(Self* self, F&& f, OnError&& g) {
CAF_LOG_TRACE(CAF_ARG(ids_));
auto bhvr = make_behavior(std::forward<F>(f), std::forward<OnError>(g));
for (auto id : ids_)
......@@ -131,7 +126,7 @@ public:
}
template <class Self, class F, class OnError>
void then(Self* self, F&& f, OnError&& g) const {
void then(Self* self, F&& f, OnError&& g) {
CAF_LOG_TRACE(CAF_ARG(ids_));
auto bhvr = make_behavior(std::forward<F>(f), std::forward<OnError>(g));
for (auto id : ids_)
......@@ -139,12 +134,13 @@ public:
}
template <class Self, class F, class G>
void receive(Self* self, F&& f, G&& g) const {
void receive(Self* self, F&& f, G&& g) {
CAF_LOG_TRACE(CAF_ARG(ids_));
using helper_type = detail::select_all_helper_t<detail::decay_t<F>>;
helper_type helper{ids_.size(), std::forward<F>(f)};
auto error_handler = [&](error& err) {
helper_type helper{ids_.size(), pending_timeouts_, std::forward<F>(f)};
auto error_handler = [&](error& err) mutable {
if (*helper.pending > 0) {
pending_timeouts_.dispose();
*helper.pending = 0;
helper.results.clear();
g(err);
......@@ -161,28 +157,35 @@ public:
return ids_;
}
disposable pending_timeouts() {
return pending_timeouts_;
}
private:
template <class F, class OnError>
behavior make_behavior(F&& f, OnError&& g) const {
behavior make_behavior(F&& f, OnError&& g) {
using namespace detail;
using helper_type = select_all_helper_t<decay_t<F>>;
helper_type helper{ids_.size(), std::move(f)};
helper_type helper{ids_.size(), pending_timeouts_, std::move(f)};
auto pending = helper.pending;
auto error_handler = [pending{std::move(pending)},
timeouts{pending_timeouts_},
g{std::forward<OnError>(g)}](error& err) mutable {
CAF_LOG_TRACE(CAF_ARG2("pending", *pending));
if (*pending > 0) {
timeouts.dispose();
*pending = 0;
g(err);
}
};
return {
std::move(helper),
std::move(error_handler),
std::move(error_handler)
};
}
message_id_list ids_;
disposable pending_timeouts_;
};
} // namespace caf::policy
......@@ -12,6 +12,7 @@
#include "caf/detail/type_list.hpp"
#include "caf/detail/type_traits.hpp"
#include "caf/detail/typed_actor_util.hpp"
#include "caf/disposable.hpp"
#include "caf/logger.hpp"
#include "caf/sec.hpp"
......@@ -23,10 +24,14 @@ struct select_any_factory;
template <class F, class... Ts>
struct select_any_factory<F, type_list<Ts...>> {
template <class Fun>
static auto make(std::shared_ptr<size_t> pending, Fun&& fun) {
return [pending, f{std::forward<Fun>(fun)}](Ts... xs) mutable {
static auto
make(std::shared_ptr<size_t> pending, disposable timeouts, Fun f) {
using std::move;
return [pending{move(pending)}, timeouts{move(timeouts)},
f{move(f)}](Ts... xs) mutable {
CAF_LOG_TRACE(CAF_ARG2("pending", *pending));
if (*pending > 0) {
timeouts.dispose();
f(xs...);
*pending = 0;
}
......@@ -54,13 +59,14 @@ public:
using type_checker
= detail::type_checker<response_type, detail::decay_t<Fun>>;
explicit select_any(message_id_list ids) : ids_(std::move(ids)) {
explicit select_any(message_id_list ids, disposable pending_timeouts)
: ids_(std::move(ids)), pending_timeouts_(std::move(pending_timeouts)) {
CAF_ASSERT(ids_.size()
<= static_cast<size_t>(std::numeric_limits<int>::max()));
}
template <class Self, class F, class OnError>
void await(Self* self, F&& f, OnError&& g) const {
void await(Self* self, F&& f, OnError&& g) {
CAF_LOG_TRACE(CAF_ARG(ids_));
auto bhvr = make_behavior(std::forward<F>(f), std::forward<OnError>(g));
for (auto id : ids_)
......@@ -68,7 +74,7 @@ public:
}
template <class Self, class F, class OnError>
void then(Self* self, F&& f, OnError&& g) const {
void then(Self* self, F&& f, OnError&& g) {
CAF_LOG_TRACE(CAF_ARG(ids_));
auto bhvr = make_behavior(std::forward<F>(f), std::forward<OnError>(g));
for (auto id : ids_)
......@@ -76,11 +82,11 @@ public:
}
template <class Self, class F, class G>
void receive(Self* self, F&& f, G&& g) const {
void receive(Self* self, F&& f, G&& g) {
CAF_LOG_TRACE(CAF_ARG(ids_));
using factory = detail::select_any_factory<std::decay_t<F>>;
auto pending = std::make_shared<size_t>(ids_.size());
auto fw = factory::make(pending, std::forward<F>(f));
auto fw = factory::make(pending, pending_timeouts_, std::forward<F>(f));
auto gw = make_error_handler(std::move(pending), std::forward<G>(g));
for (auto id : ids_) {
typename Self::accept_one_cond rc;
......@@ -94,13 +100,19 @@ public:
return ids_;
}
disposable pending_timeouts() {
return pending_timeouts_;
}
private:
template <class OnError>
auto make_error_handler(std::shared_ptr<size_t> p, OnError&& g) const {
return [p{std::move(p)}, g{std::forward<OnError>(g)}](error&) mutable {
auto make_error_handler(std::shared_ptr<size_t> p, OnError&& g) {
return [p{std::move(p)}, timeouts{pending_timeouts_},
g{std::forward<OnError>(g)}](error&) mutable {
if (*p == 0) {
// nop
} else if (*p == 1) {
timeouts.dispose();
auto err = make_error(sec::all_requests_failed);
g(err);
} else {
......@@ -110,17 +122,17 @@ private:
}
template <class F, class OnError>
behavior make_behavior(F&& f, OnError&& g) const {
behavior make_behavior(F&& f, OnError&& g) {
using factory = detail::select_any_factory<std::decay_t<F>>;
auto pending = std::make_shared<size_t>(ids_.size());
auto result_handler = factory::make(pending, std::forward<F>(f));
return {
std::move(result_handler),
make_error_handler(std::move(pending), std::forward<OnError>(g)),
};
auto result_handler = factory::make(pending, pending_timeouts_,
std::forward<F>(f));
return {std::move(result_handler),
make_error_handler(std::move(pending), std::forward<OnError>(g))};
}
message_id_list ids_;
disposable pending_timeouts_;
};
} // namespace caf::policy
......@@ -6,9 +6,11 @@
#include "caf/behavior.hpp"
#include "caf/config.hpp"
#include "caf/detail/dispose_on_call.hpp"
#include "caf/detail/type_list.hpp"
#include "caf/detail/type_traits.hpp"
#include "caf/detail/typed_actor_util.hpp"
#include "caf/disposable.hpp"
#include "caf/error.hpp"
#include "caf/message_id.hpp"
......@@ -26,7 +28,8 @@ public:
template <class Fun>
using type_checker = detail::type_checker<response_type, Fun>;
explicit single_response(message_id mid) noexcept : mid_(mid) {
explicit single_response(message_id mid, disposable pending_timeout) noexcept
: mid_(mid), pending_timeout_(std::move(pending_timeout)) {
// nop
}
......@@ -35,30 +38,41 @@ public:
single_response& operator=(single_response&&) noexcept = default;
template <class Self, class F, class OnError>
void await(Self* self, F&& f, OnError&& g) const {
behavior bhvr{std::forward<F>(f), std::forward<OnError>(g)};
void await(Self* self, F&& f, OnError&& g) {
using detail::dispose_on_call;
behavior bhvr{dispose_on_call(pending_timeout_, std::forward<F>(f)),
dispose_on_call(pending_timeout_, std::forward<OnError>(g))};
self->add_awaited_response_handler(mid_, std::move(bhvr));
}
template <class Self, class F, class OnError>
void then(Self* self, F&& f, OnError&& g) const {
behavior bhvr{std::forward<F>(f), std::forward<OnError>(g)};
void then(Self* self, F&& f, OnError&& g) {
using detail::dispose_on_call;
behavior bhvr{dispose_on_call(pending_timeout_, std::forward<F>(f)),
dispose_on_call(pending_timeout_, std::forward<OnError>(g))};
self->add_multiplexed_response_handler(mid_, std::move(bhvr));
}
template <class Self, class F, class OnError>
void receive(Self* self, F&& f, OnError&& g) const {
void receive(Self* self, F&& f, OnError&& g) {
using detail::dispose_on_call;
typename Self::accept_one_cond rc;
self->varargs_receive(rc, mid_, std::forward<F>(f),
std::forward<OnError>(g));
self->varargs_receive(
rc, mid_, dispose_on_call(pending_timeout_, std::forward<F>(f)),
dispose_on_call(pending_timeout_, std::forward<OnError>(g)));
}
message_id id() const noexcept {
return mid_;
}
disposable pending_timeouts() {
return pending_timeout_;
}
private:
message_id mid_;
disposable pending_timeout_;
};
} // namespace caf::policy
......@@ -41,12 +41,13 @@ public:
};
template <class Coordinator>
void enqueue(Coordinator* self, resumable* job) {
bool enqueue(Coordinator* self, resumable* job) {
queue_type l;
l.push_back(job);
std::unique_lock<std::mutex> guard(d(self).lock);
d(self).queue.splice(d(self).queue.end(), l);
d(self).cv.notify_one();
return true;
}
template <class Coordinator>
......
......@@ -55,7 +55,7 @@ public:
// -- non-blocking API -------------------------------------------------------
template <class T = traits, class F, class OnError>
detail::enable_if_t<T::is_non_blocking> await(F f, OnError g) const {
detail::enable_if_t<T::is_non_blocking> await(F f, OnError g) {
static_assert(detail::has_add_awaited_response_handler_v<ActorType>,
"this actor type does not support awaiting responses, "
"try using .then instead");
......@@ -74,13 +74,13 @@ public:
template <class T = traits, class F>
detail::enable_if_t<detail::has_call_error_handler_v<ActorType> //
&& T::is_non_blocking>
await(F f) const {
await(F f) {
auto self = self_;
await(std::move(f), [self](error& err) { self->call_error_handler(err); });
}
template <class T = traits, class F, class OnError>
detail::enable_if_t<T::is_non_blocking> then(F f, OnError g) const {
detail::enable_if_t<T::is_non_blocking> then(F f, OnError g) {
static_assert(detail::has_add_multiplexed_response_handler_v<ActorType>,
"this actor type does not support multiplexed responses, "
"try using .await instead");
......@@ -99,7 +99,7 @@ public:
template <class T = traits, class F>
detail::enable_if_t<detail::has_call_error_handler_v<ActorType> //
&& T::is_non_blocking>
then(F f) const {
then(F f) {
auto self = self_;
then(std::move(f), [self](error& err) { self->call_error_handler(err); });
}
......@@ -152,6 +152,10 @@ public:
return self_;
}
policy_type& policy() noexcept {
return policy_;
}
private:
/// Points to the parent actor.
actor_type* self_;
......
......@@ -15,6 +15,7 @@
#include <type_traits>
#include <unordered_map>
#include "caf/action.hpp"
#include "caf/actor_traits.hpp"
#include "caf/detail/behavior_stack.hpp"
#include "caf/detail/core_export.hpp"
......@@ -217,7 +218,7 @@ public:
using abstract_actor::enqueue;
void enqueue(mailbox_element_ptr ptr, execution_unit* eu) override;
bool enqueue(mailbox_element_ptr ptr, execution_unit* eu) override;
mailbox_element* peek_at_next_mailbox_element() override;
......@@ -396,20 +397,11 @@ public:
// -- timeout management -----------------------------------------------------
/// Requests a new timeout and returns its ID.
uint64_t set_receive_timeout(actor_clock::time_point x);
/// Requests a new timeout for the current behavior and returns its ID.
uint64_t set_receive_timeout();
/// Resets the timeout if `timeout_id` is the active timeout.
void reset_receive_timeout(uint64_t timeout_id);
/// Returns whether `timeout_id` is currently active.
bool is_active_receive_timeout(uint64_t tid) const;
void set_receive_timeout();
/// Requests a new timeout and returns its ID.
uint64_t set_stream_timeout(actor_clock::time_point x);
void set_stream_timeout(actor_clock::time_point x);
// -- message processing -----------------------------------------------------
......@@ -574,10 +566,46 @@ public:
call_handler(error_handler_, this, err);
}
// -- timeout management -----------------------------------------------------
// -- scheduling actions -----------------------------------------------------
/// Runs `what` asynchronously at some point after `when`.
/// @param when The local time until the actor waits before invoking the
/// action. Due to scheduling delays, there will always be some
/// additional wait time. Passing the current time or a past times
/// immediately schedules the action for execution.
/// @param what The action to invoke after waiting on the timeout.
/// @returns A @ref disposable that allows the actor to cancel the action.
template <class Duration, class F>
disposable run_scheduled(
std::chrono::time_point<std::chrono::system_clock, Duration> when, F what) {
using std::chrono::time_point_cast;
return run_scheduled(time_point_cast<timespan>(when),
make_action(what, action::state::waiting));
}
/// Requests a new timeout and returns its ID.
uint64_t set_timeout(std::string type, actor_clock::time_point x);
/// @copydoc run_scheduled
template <class Duration, class F>
disposable
run_scheduled(std::chrono::time_point<actor_clock::clock_type, Duration> when,
F what) {
using std::chrono::time_point_cast;
using duration_t = actor_clock::duration_type;
return run_scheduled(time_point_cast<duration_t>(when),
make_action(what, action::state::waiting));
}
/// Runs `what` asynchronously after the `delay`.
/// @param delay Minimum amount of time that actor waits before invoking the
/// action. Due to scheduling delays, there will always be some
/// additional wait time.
/// @param what The action to invoke after the delay.
/// @returns A @ref disposable that allows the actor to cancel the action.
template <class Rep, class Period, class F>
disposable run_delayed(std::chrono::duration<Rep, Period> delay, F what) {
using std::chrono::duration_cast;
return run_delayed(duration_cast<timespan>(delay),
make_action(what, action::state::waiting));
}
// -- stream processing ------------------------------------------------------
......@@ -654,8 +682,8 @@ protected:
/// Stores user-defined callbacks for message handling.
detail::behavior_stack bhvr_stack_;
/// Identifies the timeout messages we are currently waiting for.
uint64_t timeout_id_;
/// Allows us to cancel our current in-flight timeout.
disposable pending_timeout_;
/// Stores callbacks for awaited responses.
std::forward_list<pending_response> awaited_responses_;
......@@ -723,6 +751,10 @@ private:
return body();
}
}
disposable run_scheduled(timestamp when, action what);
disposable run_scheduled(actor_clock::time_point when, action what);
disposable run_delayed(timespan delay, action what);
};
} // namespace caf
......@@ -58,16 +58,13 @@ protected:
// Start all workers.
for (auto& w : workers_)
w->start();
// Launch an additional background thread for dispatching timeouts and
// delayed messages.
timer_ = system().launch_thread("caf.clock",
[this] { clock_.run_dispatch_loop(); });
// Run remaining startup code.
clock_.start_dispatch_loop(system());
super::start();
}
void stop() override {
// shutdown workers
// Shutdown workers.
class shutdown_helper : public resumable, public ref_counted {
public:
resumable::resume_result resume(execution_unit* ptr, size_t) override {
......@@ -91,18 +88,18 @@ protected:
std::condition_variable cv;
execution_unit* last_worker;
};
// use a set to keep track of remaining workers
// Use a set to keep track of remaining workers.
shutdown_helper sh;
std::set<worker_type*> alive_workers;
auto num = num_workers();
for (size_t i = 0; i < num; ++i) {
alive_workers.insert(worker_by_id(i));
sh.ref(); // make sure reference count is high enough
sh.ref(); // Make sure reference count is high enough.
}
while (!alive_workers.empty()) {
(*alive_workers.begin())->external_enqueue(&sh);
// since jobs can be stolen, we cannot assume that we have
// actually shut down the worker we've enqueued sh to
// Since jobs can be stolen, we cannot assume that we have actually shut
// down the worker we've enqueued sh to.
{ // lifetime scope of guard
std::unique_lock<std::mutex> guard(sh.mtx);
sh.cv.wait(guard, [&] { return sh.last_worker != nullptr; });
......@@ -110,20 +107,19 @@ protected:
alive_workers.erase(static_cast<worker_type*>(sh.last_worker));
sh.last_worker = nullptr;
}
// shutdown utility actors
// Shutdown utility actors.
stop_actors();
// wait until all workers are done
// Wait until all workers are done.
for (auto& w : workers_) {
w->get_thread().join();
}
// run cleanup code for each resumable
// Run cleanup code for each resumable.
auto f = &abstract_coordinator::cleanup_and_release;
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();
// Stop timer thread.
clock_.stop_dispatch_loop();
}
void enqueue(resumable* ptr) override {
......
......@@ -163,6 +163,9 @@ enum class sec : uint8_t {
broken_promise,
/// Disconnected from a BASP node after reaching the connection timeout.
connection_timeout,
/// Signals that an actor fell behind a periodic action trigger. After raising
/// this error, an @ref actor_clock stops scheduling the action.
action_reschedule_failed,
};
// --(rst-sec-end)--
......
......@@ -106,22 +106,6 @@ bool inspect(Inspector& f, node_down_msg& x) {
f.field("reason", x.reason));
}
/// Signalizes a timeout event.
/// @note This message is handled implicitly by the runtime system.
struct timeout_msg {
/// Type of the timeout (usually either "receive" or "cycle").
std::string type;
/// Actor-specific timeout ID.
uint64_t timeout_id;
};
/// @relates timeout_msg
template <class Inspector>
bool inspect(Inspector& f, timeout_msg& x) {
return f.object(x).fields(f.field("type", x.type),
f.field("timeout_id", x.timeout_id));
}
/// Demands the receiver to open a new stream from the sender to the receiver.
struct open_stream_msg {
/// Reserved slot on the source.
......
......@@ -381,11 +381,11 @@ CAF_BEGIN_TYPE_ID_BLOCK(core_module, 0)
// -- CAF types
CAF_ADD_TYPE_ID(core_module, (caf::action))
CAF_ADD_TYPE_ID(core_module, (caf::actor))
CAF_ADD_TYPE_ID(core_module, (caf::actor_addr))
CAF_ADD_TYPE_ID(core_module, (caf::byte_buffer))
CAF_ADD_TYPE_ID(core_module, (caf::config_value))
CAF_ADD_TYPE_ID(core_module, (caf::detail::action))
CAF_ADD_TYPE_ID(core_module, (caf::dictionary<caf::config_value>) )
CAF_ADD_TYPE_ID(core_module, (caf::down_msg))
CAF_ADD_TYPE_ID(core_module, (caf::downstream_msg))
......@@ -414,7 +414,6 @@ CAF_BEGIN_TYPE_ID_BLOCK(core_module, 0)
CAF_ADD_TYPE_ID(core_module, (caf::sec))
CAF_ADD_TYPE_ID(core_module, (caf::stream_slots))
CAF_ADD_TYPE_ID(core_module, (caf::strong_actor_ptr))
CAF_ADD_TYPE_ID(core_module, (caf::timeout_msg))
CAF_ADD_TYPE_ID(core_module, (caf::timespan))
CAF_ADD_TYPE_ID(core_module, (caf::timestamp))
CAF_ADD_TYPE_ID(core_module, (caf::unit_t))
......
......@@ -230,7 +230,7 @@ public:
return self_->new_request_id(mp);
}
void request_response_timeout(timespan d, message_id mid) {
disposable request_response_timeout(timespan d, message_id mid) {
return self_->request_response_timeout(d, mid);
}
......
......@@ -41,9 +41,9 @@ void abstract_actor::on_destroy() {
// nop
}
void abstract_actor::enqueue(strong_actor_ptr sender, message_id mid,
bool abstract_actor::enqueue(strong_actor_ptr sender, message_id mid,
message msg, execution_unit* host) {
enqueue(make_mailbox_element(sender, mid, {}, std::move(msg)), host);
return enqueue(make_mailbox_element(sender, mid, {}, std::move(msg)), host);
}
abstract_actor::abstract_actor(actor_config& cfg)
......
......@@ -2,44 +2,16 @@
// the main distribution directory for license terms and copyright or visit
// https://github.com/actor-framework/actor-framework/blob/master/LICENSE.
#include "caf/detail/action.hpp"
#include "caf/action.hpp"
#include "caf/logger.hpp"
namespace caf::detail {
namespace caf {
action::impl::impl() : state_(action::state::scheduled) {
// nop
}
void action::impl::dispose() {
state_ = state::disposed;
}
bool action::impl::disposed() const noexcept {
return state_.load() == state::disposed;
}
void action::impl::ref_disposable() const noexcept {
ref();
}
void action::impl::deref_disposable() const noexcept {
deref();
}
action::state action::impl::reschedule() {
auto expected = state::invoked;
if (state_.compare_exchange_strong(expected, state::scheduled))
return state::scheduled;
else
return expected;
}
void action::run() {
action::transition action::run() {
CAF_LOG_TRACE("");
CAF_ASSERT(pimpl_ != nullptr);
pimpl_->run();
return pimpl_->run();
}
} // namespace caf::detail
} // namespace caf
......@@ -4,18 +4,219 @@
#include "caf/actor_clock.hpp"
#include "caf/action.hpp"
#include "caf/actor_cast.hpp"
#include "caf/actor_control_block.hpp"
#include "caf/disposable.hpp"
#include "caf/group.hpp"
#include "caf/sec.hpp"
namespace caf {
// -- private utility ----------------------------------------------------------
namespace {
// Unlike the regular action implementation, this one is *not* thread-safe!
// Only the clock itself may access this.
template <class WorkerPtr>
class action_decorator : public ref_counted, public action::impl {
public:
using state = action::state;
using transition = action::transition;
action_decorator(action::impl_ptr decorated, WorkerPtr worker,
actor_clock::stall_policy policy)
: decorated_(std::move(decorated)),
worker_(std::move(worker)),
policy_(policy) {
CAF_ASSERT(decorated_ != nullptr);
CAF_ASSERT(worker_ != nullptr);
}
void dispose() override {
if (decorated_) {
decorated_->dispose();
decorated_ = nullptr;
}
if (worker_)
worker_ = nullptr;
}
bool disposed() const noexcept override {
return decorated_ ? decorated_->disposed() : true;
}
void ref_disposable() const noexcept override {
ref();
}
void deref_disposable() const noexcept override {
deref();
}
transition reschedule() override {
// Always succeeds since we implicitly reschedule in do_run.
return transition::success;
}
transition run() override {
CAF_ASSERT(decorated_ != nullptr);
CAF_ASSERT(worker_ != nullptr);
if constexpr (std::is_same_v<WorkerPtr, weak_actor_ptr>) {
if (auto ptr = actor_cast<strong_actor_ptr>(worker_)) {
return do_run(ptr);
} else {
dispose();
return transition::disposed;
}
} else {
return do_run(worker_);
}
}
state current_state() const noexcept override {
return decorated_ ? decorated_->current_state() : action::state::disposed;
}
friend void intrusive_ptr_add_ref(const action_decorator* ptr) noexcept {
ptr->ref();
}
friend void intrusive_ptr_release(const action_decorator* ptr) noexcept {
ptr->deref();
}
private:
transition do_run(strong_actor_ptr& ptr) {
switch (decorated_->reschedule()) {
case transition::disposed:
decorated_ = nullptr;
worker_ = nullptr;
return transition::disposed;
case transition::success:
if (ptr->enqueue(nullptr, make_message_id(),
make_message(action{decorated_}), nullptr)) {
return transition::success;
} else {
dispose();
return transition::disposed;
}
default:
if (policy_ == actor_clock::stall_policy::fail) {
ptr->enqueue(nullptr, make_message_id(),
make_message(make_error(sec::action_reschedule_failed)),
nullptr);
dispose();
return transition::failure;
} else {
return transition::success;
}
}
}
action::impl_ptr decorated_;
WorkerPtr worker_;
actor_clock::stall_policy policy_;
};
template <class WorkerPtr>
action decorate(action f, WorkerPtr worker, actor_clock::stall_policy policy) {
CAF_ASSERT(f.ptr() != nullptr);
using impl_t = action_decorator<WorkerPtr>;
auto ptr = make_counted<impl_t>(std::move(f).as_intrusive_ptr(),
std::move(worker), policy);
return action{std::move(ptr)};
}
} // namespace
// -- constructors, destructors, and assignment operators ----------------------
actor_clock::~actor_clock() {
// nop
}
// -- observers ----------------------------------------------------------------
// -- scheduling ---------------------------------------------------------------
actor_clock::time_point actor_clock::now() const noexcept {
return clock_type::now();
}
disposable actor_clock::schedule(action f) {
return schedule_periodically(time_point{duration_type{0}}, std::move(f),
duration_type{0});
}
disposable actor_clock::schedule(time_point t, action f) {
return schedule_periodically(t, std::move(f), duration_type{0});
}
disposable actor_clock::schedule(time_point t, action f,
strong_actor_ptr worker) {
return schedule_periodically(t, std::move(f), std::move(worker),
duration_type{0}, stall_policy::skip);
}
disposable actor_clock::schedule_periodically(time_point first_run, action f,
strong_actor_ptr worker,
duration_type period,
stall_policy policy) {
auto res = f.as_disposable();
auto g = decorate(std::move(f), std::move(worker), policy);
schedule_periodically(first_run, std::move(g), period);
return res;
}
disposable actor_clock::schedule(time_point t, action f,
weak_actor_ptr worker) {
return schedule_periodically(t, std::move(f), std::move(worker),
duration_type{0}, stall_policy::skip);
}
disposable actor_clock::schedule_periodically(time_point first_run, action f,
weak_actor_ptr worker,
duration_type period,
stall_policy policy) {
auto res = f.as_disposable();
auto g = decorate(std::move(f), std::move(worker), policy);
schedule_periodically(first_run, std::move(g), period);
return res;
}
disposable actor_clock::schedule_message(time_point t,
strong_actor_ptr receiver,
mailbox_element_ptr content) {
auto f = make_action(
[rptr{std::move(receiver)}, cptr{std::move(content)}]() mutable {
rptr->enqueue(std::move(cptr), nullptr);
});
schedule(t, f);
return std::move(f).as_disposable();
}
disposable actor_clock::schedule_message(time_point t, weak_actor_ptr receiver,
mailbox_element_ptr content) {
auto f = make_action(
[rptr{std::move(receiver)}, cptr{std::move(content)}]() mutable {
if (auto ptr = actor_cast<strong_actor_ptr>(rptr))
ptr->enqueue(std::move(cptr), nullptr);
});
schedule(t, f);
return std::move(f).as_disposable();
}
disposable actor_clock::schedule_message(time_point t, group target,
strong_actor_ptr sender,
message content) {
auto f = make_action([=]() mutable {
if (auto dst = target->get())
dst->enqueue(std::move(sender), make_message_id(), std::move(content),
nullptr);
});
schedule(t, f);
return std::move(f).as_disposable();
}
} // namespace caf
......@@ -24,17 +24,21 @@ void actor_companion::on_exit(on_exit_handler handler) {
on_exit_ = std::move(handler);
}
void actor_companion::enqueue(mailbox_element_ptr ptr, execution_unit*) {
bool actor_companion::enqueue(mailbox_element_ptr ptr, execution_unit*) {
CAF_ASSERT(ptr);
shared_lock<lock_type> guard(lock_);
if (on_enqueue_)
if (on_enqueue_) {
on_enqueue_(std::move(ptr));
return true;
} else {
return false;
}
}
void actor_companion::enqueue(strong_actor_ptr src, message_id mid,
bool actor_companion::enqueue(strong_actor_ptr src, message_id mid,
message content, execution_unit* eu) {
auto ptr = make_mailbox_element(std::move(src), mid, {}, std::move(content));
enqueue(std::move(ptr), eu);
return enqueue(std::move(ptr), eu);
}
void actor_companion::launch(execution_unit*, bool, bool hide) {
......
......@@ -17,14 +17,14 @@ actor_addr actor_control_block::address() {
return {this, true};
}
void actor_control_block::enqueue(strong_actor_ptr sender, message_id mid,
bool actor_control_block::enqueue(strong_actor_ptr sender, message_id mid,
message content, execution_unit* host) {
get()->enqueue(std::move(sender), mid, std::move(content), host);
return get()->enqueue(std::move(sender), mid, std::move(content), host);
}
void actor_control_block::enqueue(mailbox_element_ptr what,
bool actor_control_block::enqueue(mailbox_element_ptr what,
execution_unit* host) {
get()->enqueue(std::move(what), host);
return get()->enqueue(std::move(what), host);
}
bool intrusive_ptr_upgrade_weak(actor_control_block* x) {
......
......@@ -101,11 +101,12 @@ actor actor_pool::make(execution_unit* eu, size_t num_workers,
return res;
}
void actor_pool::enqueue(mailbox_element_ptr what, execution_unit* eu) {
bool actor_pool::enqueue(mailbox_element_ptr what, execution_unit* eu) {
upgrade_lock<detail::shared_spinlock> guard{workers_mtx_};
if (filter(guard, what->sender, what->mid, what->payload, eu))
return;
return false;
policy_(home_system(), guard, workers_, what, eu);
return true;
}
actor_pool::actor_pool(actor_config& cfg)
......
......@@ -50,7 +50,7 @@ blocking_actor::~blocking_actor() {
// avoid weak-vtables warning
}
void blocking_actor::enqueue(mailbox_element_ptr ptr, execution_unit*) {
bool blocking_actor::enqueue(mailbox_element_ptr ptr, execution_unit*) {
CAF_ASSERT(ptr != nullptr);
CAF_ASSERT(getf(is_blocking_flag));
CAF_LOG_TRACE(CAF_ARG(*ptr));
......@@ -72,8 +72,10 @@ void blocking_actor::enqueue(mailbox_element_ptr ptr, execution_unit*) {
detail::sync_request_bouncer srb{exit_reason()};
srb(src, mid);
}
return false;
} else {
CAF_LOG_ACCEPT_EVENT(false);
return true;
}
}
......
......@@ -31,13 +31,13 @@ sequencer::sequencer(strong_actor_ptr f, strong_actor_ptr g,
}
}
void sequencer::enqueue(mailbox_element_ptr what, execution_unit* context) {
bool sequencer::enqueue(mailbox_element_ptr what, execution_unit* context) {
auto down_msg_handler = [&](down_msg& dm) {
// quit if either `f` or `g` are no longer available
cleanup(std::move(dm.reason), context);
};
if (handle_system_message(*what, context, false, down_msg_handler))
return;
return true;
strong_actor_ptr f;
strong_actor_ptr g;
error err;
......@@ -49,13 +49,13 @@ void sequencer::enqueue(mailbox_element_ptr what, execution_unit* context) {
if (!f) {
// f and g are invalid only after the sequencer terminated
bounce(what, err);
return;
return false;
}
// process and forward the non-system message;
// store `f` as the next stage in the forwarding chain
what->stages.push_back(std::move(f));
// forward modified message to `g`
g->enqueue(std::move(what), context);
return g->enqueue(std::move(what), context);
}
sequencer::message_types_set sequencer::message_types() const {
......
......@@ -89,7 +89,7 @@ void group_tunnel::unsubscribe(const actor_control_block* who) {
});
}
void group_tunnel::enqueue(strong_actor_ptr sender, message_id mid,
bool group_tunnel::enqueue(strong_actor_ptr sender, message_id mid,
message content, execution_unit* host) {
CAF_LOG_TRACE(CAF_ARG(sender) << CAF_ARG(content));
std::unique_lock<std::mutex> guard{mtx_};
......@@ -100,6 +100,7 @@ void group_tunnel::enqueue(strong_actor_ptr sender, message_id mid,
auto wrapped = make_message(sys_atom_v, forward_atom_v, std::move(content));
cached_messages_.emplace_back(std::move(sender), mid, std::move(wrapped));
}
return true;
}
void group_tunnel::stop() {
......
......@@ -66,11 +66,12 @@ local_group_module::impl::~impl() {
// nop
}
void local_group_module::impl::enqueue(strong_actor_ptr sender, message_id mid,
bool local_group_module::impl::enqueue(strong_actor_ptr sender, message_id mid,
message content, execution_unit* host) {
std::unique_lock<std::mutex> guard{mtx_};
for (auto subscriber : subscribers_)
subscriber->enqueue(sender, mid, content, host);
return true;
}
bool local_group_module::impl::subscribe(strong_actor_ptr who) {
......
// This file is part of CAF, the C++ Actor Framework. See the file LICENSE in
// the main distribution directory for license terms and copyright or visit
// https://github.com/actor-framework/actor-framework/blob/master/LICENSE.
#include "caf/detail/simple_actor_clock.hpp"
#include "caf/actor_cast.hpp"
#include "caf/sec.hpp"
#include "caf/system_messages.hpp"
namespace caf::detail {
simple_actor_clock::event::~event() {
// nop
}
void simple_actor_clock::set_ordinary_timeout(time_point t,
abstract_actor* self,
std::string type, uint64_t id) {
new_schedule_entry<ordinary_timeout>(t, self->ctrl(), type, id);
}
void simple_actor_clock::set_multi_timeout(time_point t, abstract_actor* self,
std::string type, uint64_t id) {
new_schedule_entry<multi_timeout>(t, self->ctrl(), type, id);
}
void simple_actor_clock::set_request_timeout(time_point t, abstract_actor* self,
message_id id) {
new_schedule_entry<request_timeout>(t, self->ctrl(), id);
}
void simple_actor_clock::cancel_ordinary_timeout(abstract_actor* self,
std::string type) {
ordinary_timeout_cancellation tmp{self->id(), std::move(type)};
handle(tmp);
}
void simple_actor_clock::cancel_request_timeout(abstract_actor* self,
message_id id) {
request_timeout_cancellation tmp{self->id(), id};
handle(tmp);
}
void simple_actor_clock::cancel_timeouts(abstract_actor* self) {
auto range = actor_lookup_.equal_range(self->id());
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) {
new_schedule_entry<actor_msg>(t, std::move(receiver), std::move(content));
}
void simple_actor_clock::schedule_message(time_point t, group target,
strong_actor_ptr sender,
message content) {
new_schedule_entry<group_msg>(t, std::move(target), std::move(sender),
std::move(content));
}
void simple_actor_clock::cancel_all() {
actor_lookup_.clear();
schedule_.clear();
}
void simple_actor_clock::ship(delayed_event& x) {
switch (x.subtype) {
case ordinary_timeout_type: {
auto& dref = static_cast<ordinary_timeout&>(x);
auto& self = dref.self;
self->get()->eq_impl(make_message_id(), self, nullptr,
timeout_msg{dref.type, dref.id});
break;
}
case multi_timeout_type: {
auto& dref = static_cast<multi_timeout&>(x);
auto& self = dref.self;
self->get()->eq_impl(make_message_id(), self, nullptr,
timeout_msg{dref.type, dref.id});
break;
}
case request_timeout_type: {
auto& dref = static_cast<request_timeout&>(x);
auto& self = dref.self;
self->get()->eq_impl(dref.id, self, nullptr, sec::request_timeout);
break;
}
case actor_msg_type: {
auto& dref = static_cast<actor_msg&>(x);
dref.receiver->enqueue(std::move(dref.content), nullptr);
break;
}
case group_msg_type: {
auto& dref = static_cast<group_msg&>(x);
auto dst = dref.target->get();
if (dst)
dst->enqueue(std::move(dref.sender), make_message_id(),
std::move(dref.content), nullptr);
break;
}
default:
break;
}
}
void simple_actor_clock::handle(const ordinary_timeout_cancellation& x) {
auto pred = [&](const actor_lookup_map::value_type& kvp) {
auto& y = *kvp.second->second;
return y.subtype == ordinary_timeout_type
&& x.type == static_cast<const ordinary_timeout&>(y).type;
};
cancel(x.aid, pred);
}
void simple_actor_clock::handle(const multi_timeout_cancellation& x) {
auto pred = [&](const actor_lookup_map::value_type& kvp) {
auto& y = *kvp.second->second;
if (y.subtype != multi_timeout_type)
return false;
auto& dref = static_cast<const multi_timeout&>(y);
return x.type == dref.type && x.id == dref.id;
};
cancel(x.aid, pred);
}
void simple_actor_clock::handle(const request_timeout_cancellation& x) {
auto pred = [&](const actor_lookup_map::value_type& kvp) {
auto& y = *kvp.second->second;
return y.subtype == request_timeout_type
&& x.id == static_cast<const request_timeout&>(y).id;
};
cancel(x.aid, pred);
}
void simple_actor_clock::handle(const timeouts_cancellation& x) {
auto range = actor_lookup_.equal_range(x.aid);
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);
}
size_t simple_actor_clock::trigger_expired_timeouts() {
size_t result = 0;
auto t = now();
auto i = schedule_.begin();
auto e = schedule_.end();
while (i != e && i->first <= t) {
auto ptr = std::move(i->second);
auto backlink = ptr->backlink;
if (backlink != actor_lookup_.end())
actor_lookup_.erase(backlink);
i = schedule_.erase(i);
ship(*ptr);
++result;
}
return result;
}
void simple_actor_clock::add_schedule_entry(
time_point t, std::unique_ptr<ordinary_timeout> x) {
auto aid = x->self->id();
auto type = x->type;
auto pred = [&](const actor_lookup_map::value_type& kvp) {
auto& y = *kvp.second->second;
return y.subtype == ordinary_timeout_type
&& static_cast<const ordinary_timeout&>(y).type == type;
};
auto i = lookup(aid, pred);
if (i != actor_lookup_.end()) {
schedule_.erase(i->second);
i->second = schedule_.emplace(t, std::move(x));
} else {
auto j = schedule_.emplace(t, std::move(x));
i = actor_lookup_.emplace(aid, j);
}
i->second->second->backlink = i;
}
} // namespace caf::detail
......@@ -10,8 +10,15 @@ namespace caf::detail {
test_actor_clock::test_actor_clock() : current_time(duration_type{1}) {
// This ctor makes sure that the clock isn't at the default-constructed
// time_point, because that value has special meaning (for the tick_emitter,
// for example).
// time_point, because begin-of-epoch may have special meaning.
}
disposable test_actor_clock::schedule_periodically(time_point first_run,
action f,
duration_type period) {
CAF_ASSERT(f.ptr() != nullptr);
schedule.emplace(first_run, schedule_entry{f, period});
return std::move(f).as_disposable();
}
test_actor_clock::time_point test_actor_clock::now() const noexcept {
......@@ -19,25 +26,22 @@ test_actor_clock::time_point test_actor_clock::now() const noexcept {
}
bool test_actor_clock::trigger_timeout() {
CAF_LOG_TRACE(CAF_ARG2("schedule.size", schedule_.size()));
if (schedule_.empty())
return false;
auto i = schedule_.begin();
auto tout = i->first;
if (tout > current_time)
current_time = tout;
auto ptr = std::move(i->second);
schedule_.erase(i);
auto backlink = ptr->backlink;
if (backlink != actor_lookup_.end())
actor_lookup_.erase(backlink);
ship(*ptr);
return true;
CAF_LOG_TRACE(CAF_ARG2("schedule.size", schedule.size()));
for (;;) {
if (schedule.empty())
return false;
auto i = schedule.begin();
auto t = i->first;
if (t > current_time)
current_time = t;
if (try_trigger_once())
return true;
}
}
size_t test_actor_clock::trigger_timeouts() {
CAF_LOG_TRACE(CAF_ARG2("schedule.size", schedule_.size()));
if (schedule_.empty())
CAF_LOG_TRACE(CAF_ARG2("schedule.size", schedule.size()));
if (schedule.empty())
return 0u;
size_t result = 0;
while (trigger_timeout())
......@@ -46,10 +50,34 @@ size_t test_actor_clock::trigger_timeouts() {
}
size_t test_actor_clock::advance_time(duration_type x) {
CAF_LOG_TRACE(CAF_ARG(x) << CAF_ARG2("schedule.size", schedule_.size()));
CAF_LOG_TRACE(CAF_ARG(x) << CAF_ARG2("schedule.size", schedule.size()));
CAF_ASSERT(x.count() >= 0);
current_time += x;
return trigger_expired_timeouts();
auto result = size_t{0};
while (!schedule.empty() && schedule.begin()->first <= current_time)
if (try_trigger_once())
++result;
return result;
}
bool test_actor_clock::try_trigger_once() {
auto i = schedule.begin();
auto t = i->first;
if (t > current_time)
return false;
auto [f, period] = i->second;
schedule.erase(i);
if (f.run() == action::transition::success) {
if (period.count() > 0) {
auto next = t + period;
while (next <= current_time)
next += period;
schedule.emplace(next, schedule_entry{std::move(f), period});
}
return true;
} else {
return false;
}
}
} // namespace caf::detail
......@@ -5,147 +5,69 @@
#include "caf/detail/thread_safe_actor_clock.hpp"
#include "caf/actor_control_block.hpp"
#include "caf/actor_system.hpp"
#include "caf/logger.hpp"
#include "caf/sec.hpp"
#include "caf/system_messages.hpp"
namespace caf::detail {
void thread_safe_actor_clock::set_ordinary_timeout(time_point t,
abstract_actor* self,
std::string type,
uint64_t id) {
push(new ordinary_timeout(t, self->ctrl(), type, id));
thread_safe_actor_clock::thread_safe_actor_clock() {
tbl_.reserve(buffer_size * 2);
}
void thread_safe_actor_clock::set_request_timeout(time_point t,
abstract_actor* self,
message_id id) {
push(new request_timeout(t, self->ctrl(), id));
disposable
thread_safe_actor_clock::schedule_periodically(time_point first_run, action f,
duration_type period) {
queue_.emplace_back(new schedule_entry{first_run, f, period});
return std::move(f).as_disposable();
}
void thread_safe_actor_clock::set_multi_timeout(time_point t,
abstract_actor* self,
std::string type, uint64_t id) {
push(new multi_timeout(t, self->ctrl(), type, id));
}
void thread_safe_actor_clock::cancel_ordinary_timeout(abstract_actor* self,
std::string type) {
push(new ordinary_timeout_cancellation(self->id(), type));
}
void thread_safe_actor_clock::cancel_request_timeout(abstract_actor* self,
message_id id) {
push(new request_timeout_cancellation(self->id(), id));
}
void thread_safe_actor_clock::cancel_timeouts(abstract_actor* self) {
push(new timeouts_cancellation(self->id()));
}
void thread_safe_actor_clock::schedule_message(time_point t,
strong_actor_ptr receiver,
mailbox_element_ptr content) {
push(new actor_msg(t, std::move(receiver), std::move(content)));
}
void thread_safe_actor_clock::schedule_message(time_point t, group target,
strong_actor_ptr sender,
message content) {
auto ptr = new group_msg(t, std::move(target), std::move(sender),
std::move(content));
push(ptr);
}
void thread_safe_actor_clock::cancel_all() {
push(new drop_all);
}
void thread_safe_actor_clock::run_dispatch_loop() {
for (;;) {
// Wait until queue is non-empty.
if (schedule_.empty()) {
void thread_safe_actor_clock::run() {
CAF_LOG_TRACE("");
auto is_disposed = [](auto& x) { return !x || x->f.disposed(); };
auto by_timeout = [](auto& x, auto& y) { return x->t < y->t; };
while (running_) {
if (tbl_.empty()) {
queue_.wait_nonempty();
queue_.get_all(std::back_inserter(tbl_));
std::sort(tbl_.begin(), tbl_.end(), by_timeout);
} else {
auto t = schedule_.begin()->second->due;
if (!queue_.wait_nonempty(t)) {
// Handle timeout by shipping timed-out events and starting anew.
trigger_expired_timeouts();
continue;
auto next_timeout = (*tbl_.begin())->t;
if (queue_.wait_nonempty(next_timeout)) {
queue_.get_all(std::back_inserter(tbl_));
std::sort(tbl_.begin(), tbl_.end(), by_timeout);
}
}
// Push all elements from the queue to the events buffer.
auto i = events_.begin();
auto e = queue_.get_all(i);
for (; i != e; ++i) {
auto& x = *i;
CAF_ASSERT(x != nullptr);
switch (x->subtype) {
case ordinary_timeout_cancellation_type: {
handle(static_cast<ordinary_timeout_cancellation&>(*x));
break;
}
case request_timeout_cancellation_type: {
handle(static_cast<request_timeout_cancellation&>(*x));
break;
}
case timeouts_cancellation_type: {
handle(static_cast<timeouts_cancellation&>(*x));
break;
}
case drop_all_type: {
schedule_.clear();
actor_lookup_.clear();
break;
}
case shutdown_type: {
schedule_.clear();
actor_lookup_.clear();
// Call it a day.
return;
}
case ordinary_timeout_type: {
auto dptr = static_cast<ordinary_timeout*>(x.release());
add_schedule_entry(std::unique_ptr<ordinary_timeout>{dptr});
break;
}
case multi_timeout_type: {
auto dptr = static_cast<multi_timeout*>(x.release());
add_schedule_entry(std::unique_ptr<multi_timeout>{dptr});
break;
}
case request_timeout_type: {
auto dptr = static_cast<request_timeout*>(x.release());
add_schedule_entry(std::unique_ptr<request_timeout>{dptr});
break;
}
case actor_msg_type: {
auto dptr = static_cast<actor_msg*>(x.release());
add_schedule_entry(std::unique_ptr<actor_msg>{dptr});
break;
}
case group_msg_type: {
auto dptr = static_cast<group_msg*>(x.release());
add_schedule_entry(std::unique_ptr<group_msg>{dptr});
break;
}
default: {
CAF_LOG_ERROR("unexpected event type");
break;
auto n = now();
for (auto i = tbl_.begin(); i != tbl_.end() && (*i)->t <= n; ++i) {
auto& entry = **i;
if (entry.f.run() == action::transition::success) {
if (entry.period.count() > 0) {
auto next = entry.t + entry.period;
while (next <= n) {
CAF_LOG_WARNING("clock lagging behind, skipping a tick!");
next += entry.period;
}
} else {
i->reset(); // Remove from tbl_ after the for-loop body.
}
} else {
i->reset(); // Remove from tbl_ after the for-loop body.
}
x.reset();
}
tbl_.erase(std::remove_if(tbl_.begin(), tbl_.end(), is_disposed),
tbl_.end());
}
}
void thread_safe_actor_clock::cancel_dispatch_loop() {
push(new shutdown);
void thread_safe_actor_clock::start_dispatch_loop(caf::actor_system& sys) {
dispatcher_ = sys.launch_thread("caf.clock", [this] { run(); });
}
void thread_safe_actor_clock::push(event* ptr) {
queue_.push_back(unique_event_ptr{ptr});
void thread_safe_actor_clock::stop_dispatch_loop() {
schedule(make_action([this] { running_ = false; }));
dispatcher_.join();
}
} // namespace caf::detail
......@@ -4,14 +4,67 @@
#include "caf/disposable.hpp"
#include <algorithm>
#include "caf/make_counted.hpp"
namespace caf {
namespace {
class composite_impl : public ref_counted, public disposable::impl {
public:
using disposable_list = std::vector<disposable>;
composite_impl(disposable_list entries) : entries_(std::move(entries)) {
// nop
}
void dispose() {
for (auto& entry : entries_)
entry.dispose();
}
bool disposed() const noexcept {
auto is_disposed = [](const disposable& entry) { return entry.disposed(); };
return std::all_of(entries_.begin(), entries_.end(), is_disposed);
}
void ref_disposable() const noexcept {
ref();
}
void deref_disposable() const noexcept {
deref();
}
friend void intrusive_ptr_add_ref(const composite_impl* ptr) noexcept {
ptr->ref();
}
friend void intrusive_ptr_release(const composite_impl* ptr) noexcept {
ptr->deref();
}
private:
std::vector<disposable> entries_;
};
} // namespace
disposable::impl::~impl() {
// nop
}
disposable disposable::impl::as_disposable() noexcept {
return disposable{intrusive_ptr<disposable::impl>{this}};
return disposable{intrusive_ptr<impl>{this}};
}
disposable disposable::make_composite(std::vector<disposable> entries) {
if (entries.empty())
return {};
else
return disposable{make_counted<composite_impl>(std::move(entries))};
}
} // namespace caf
......@@ -22,7 +22,7 @@ forwarding_actor_proxy::~forwarding_actor_proxy() {
anon_send(broker_, make_message(delete_atom_v, node(), id()));
}
void forwarding_actor_proxy::forward_msg(strong_actor_ptr sender,
bool forwarding_actor_proxy::forward_msg(strong_actor_ptr sender,
message_id mid, message msg,
const forwarding_stack* fwd) {
CAF_LOG_TRACE(CAF_ARG(id())
......@@ -32,20 +32,22 @@ void forwarding_actor_proxy::forward_msg(strong_actor_ptr sender,
forwarding_stack tmp;
shared_lock<detail::shared_spinlock> guard(broker_mtx_);
if (broker_)
broker_->enqueue(nullptr, make_message_id(),
make_message(forward_atom_v, std::move(sender),
fwd != nullptr ? *fwd : tmp,
strong_actor_ptr{ctrl()}, mid,
std::move(msg)),
nullptr);
return broker_->enqueue(nullptr, make_message_id(),
make_message(forward_atom_v, std::move(sender),
fwd != nullptr ? *fwd : tmp,
strong_actor_ptr{ctrl()}, mid,
std::move(msg)),
nullptr);
else
return false;
}
void forwarding_actor_proxy::enqueue(mailbox_element_ptr what,
bool forwarding_actor_proxy::enqueue(mailbox_element_ptr what,
execution_unit*) {
CAF_PUSH_AID(0);
CAF_ASSERT(what);
forward_msg(std::move(what->sender), what->mid, std::move(what->payload),
&what->stages);
return forward_msg(std::move(what->sender), what->mid,
std::move(what->payload), &what->stages);
}
bool forwarding_actor_proxy::add_backlink(abstract_actor* x) {
......
......@@ -4,13 +4,13 @@
#include "caf/init_global_meta_objects.hpp"
#include "caf/action.hpp"
#include "caf/actor.hpp"
#include "caf/actor_addr.hpp"
#include "caf/actor_control_block.hpp"
#include "caf/actor_system.hpp"
#include "caf/byte_buffer.hpp"
#include "caf/config_value.hpp"
#include "caf/detail/action.hpp"
#include "caf/downstream_msg.hpp"
#include "caf/error.hpp"
#include "caf/group.hpp"
......
......@@ -14,6 +14,7 @@
#include "caf/binary_serializer.hpp"
#include "caf/default_attachable.hpp"
#include "caf/detail/glob_match.hpp"
#include "caf/disposable.hpp"
#include "caf/exit_reason.hpp"
#include "caf/logger.hpp"
#include "caf/resumable.hpp"
......@@ -84,13 +85,16 @@ auto local_actor::now() const noexcept -> clock_type::time_point {
return clock().now();
}
void local_actor::request_response_timeout(timespan timeout, message_id mid) {
disposable local_actor::request_response_timeout(timespan timeout,
message_id mid) {
CAF_LOG_TRACE(CAF_ARG(timeout) << CAF_ARG(mid));
if (timeout == infinite)
return;
auto t = clock().now();
t += timeout;
clock().set_request_timeout(t, this, mid.response_id());
return {};
auto t = clock().now() + timeout;
return clock().schedule_message(
t, strong_actor_ptr{ctrl()},
make_mailbox_element(nullptr, mid.response_id(), {},
make_error(sec::request_timeout)));
}
void local_actor::monitor(abstract_actor* ptr, message_priority priority) {
......@@ -163,7 +167,6 @@ bool local_actor::cleanup(error&& fail_state, execution_unit* host) {
unregister_from_system();
CAF_LOG_TERMINATE_EVENT(this, fail_state);
monitorable_actor::cleanup(std::move(fail_state), host);
clock().cancel_timeouts(this);
return true;
}
......
......@@ -4,11 +4,11 @@
#include "caf/scheduled_actor.hpp"
#include "caf/action.hpp"
#include "caf/actor_ostream.hpp"
#include "caf/actor_system_config.hpp"
#include "caf/config.hpp"
#include "caf/defaults.hpp"
#include "caf/detail/action.hpp"
#include "caf/detail/default_invoke_result_visitor.hpp"
#include "caf/detail/meta_object.hpp"
#include "caf/detail/private_thread.hpp"
......@@ -112,7 +112,6 @@ error scheduled_actor::default_exception_handler(local_actor* ptr,
scheduled_actor::scheduled_actor(actor_config& cfg)
: super(cfg),
mailbox_(unit, unit, unit, unit, unit),
timeout_id_(0),
default_handler_(print_and_drop),
error_handler_(default_error_handler),
down_handler_(default_down_handler),
......@@ -135,7 +134,7 @@ scheduled_actor::~scheduled_actor() {
// -- overridden functions of abstract_actor -----------------------------------
void scheduled_actor::enqueue(mailbox_element_ptr ptr, execution_unit* eu) {
bool scheduled_actor::enqueue(mailbox_element_ptr ptr, execution_unit* eu) {
CAF_ASSERT(ptr != nullptr);
CAF_ASSERT(!getf(is_blocking_flag));
CAF_LOG_TRACE(CAF_ARG(*ptr));
......@@ -157,7 +156,7 @@ void scheduled_actor::enqueue(mailbox_element_ptr ptr, execution_unit* eu) {
eu->exec_later(this);
else
home_system().scheduler().enqueue(this);
break;
return true;
}
case intrusive::inbox_result::queue_closed: {
CAF_LOG_REJECT_EVENT();
......@@ -168,12 +167,12 @@ void scheduled_actor::enqueue(mailbox_element_ptr ptr, execution_unit* eu) {
detail::sync_request_bouncer f{exit_reason()};
f(sender, mid);
}
break;
return false;
}
case intrusive::inbox_result::success:
// enqueued to a running actors' mailbox; nothing to do
CAF_LOG_ACCEPT_EVENT(false);
break;
return true;
}
}
......@@ -217,6 +216,7 @@ void scheduled_actor::launch(execution_unit* ctx, bool lazy, bool hide) {
bool scheduled_actor::cleanup(error&& fail_state, execution_unit* host) {
CAF_LOG_TRACE(CAF_ARG(fail_state));
pending_timeout_.dispose();
// Shutdown hosting thread when running detached.
if (private_thread_)
home_system().release_private_thread(private_thread_);
......@@ -529,48 +529,27 @@ auto scheduled_actor::outbound_stream_metrics(type_id_t type)
// -- timeout management -------------------------------------------------------
uint64_t scheduled_actor::set_receive_timeout(actor_clock::time_point x) {
CAF_LOG_TRACE(x);
setf(has_timeout_flag);
return set_timeout("receive", x);
}
uint64_t scheduled_actor::set_receive_timeout() {
void scheduled_actor::set_receive_timeout() {
CAF_LOG_TRACE("");
if (bhvr_stack_.empty())
return 0;
auto timeout = bhvr_stack_.back().timeout();
if (timeout == infinite) {
unsetf(has_timeout_flag);
return 0;
}
if (timeout == timespan{0}) {
// immediately enqueue timeout message if duration == 0s
auto id = ++timeout_id_;
auto type = "receive"s;
eq_impl(make_message_id(), nullptr, context(), timeout_msg{type, id});
return id;
pending_timeout_.dispose();
if (bhvr_stack_.empty()) {
// nop
} else if (auto delay = bhvr_stack_.back().timeout(); delay == infinite) {
// nop
} else {
pending_timeout_ = run_delayed(delay, [this] {
if (!bhvr_stack_.empty())
bhvr_stack_.back().handle_timeout();
});
}
auto t = clock().now();
t += timeout;
return set_receive_timeout(t);
}
void scheduled_actor::reset_receive_timeout(uint64_t timeout_id) {
if (is_active_receive_timeout(timeout_id))
unsetf(has_timeout_flag);
}
bool scheduled_actor::is_active_receive_timeout(uint64_t tid) const {
return getf(has_timeout_flag) && timeout_id_ == tid;
}
uint64_t scheduled_actor::set_stream_timeout(actor_clock::time_point x) {
void scheduled_actor::set_stream_timeout(actor_clock::time_point x) {
CAF_LOG_TRACE(x);
// Do not request 'infinite' timeouts.
if (x == actor_clock::time_point::max()) {
CAF_LOG_DEBUG("drop infinite timeout");
return 0;
return;
}
// Do not request a timeout if all streams are idle.
std::vector<stream_manager_ptr> mgrs;
......@@ -580,11 +559,14 @@ uint64_t scheduled_actor::set_stream_timeout(actor_clock::time_point x) {
auto e = std::unique(mgrs.begin(), mgrs.end());
auto idle = [=](const stream_manager_ptr& y) { return y->idle(); };
if (std::all_of(mgrs.begin(), e, idle)) {
CAF_LOG_DEBUG("suppress stream timeout");
return 0;
CAF_LOG_DEBUG("suppress stream timeout: all managers are idle");
return;
}
// Delegate call.
return set_timeout("stream", x);
run_scheduled(x, [this] {
auto next_timeout = advance_streams(clock().now());
set_stream_timeout(next_timeout);
});
}
// -- message processing -------------------------------------------------------
......@@ -622,22 +604,6 @@ scheduled_actor::categorize(mailbox_element& x) {
}
return message_category::internal;
}
if (content.match_elements<timeout_msg>()) {
CAF_ASSERT(x.mid.is_async());
auto& tm = content.get_as<timeout_msg>(0);
auto tid = tm.timeout_id;
if (tm.type == "receive") {
CAF_LOG_DEBUG("handle ordinary timeout message");
if (is_active_receive_timeout(tid) && !bhvr_stack_.empty())
bhvr_stack_.back().handle_timeout();
} else if (tm.type == "stream") {
CAF_LOG_DEBUG("handle stream timeout message");
set_stream_timeout(advance_streams(clock().now()));
} else {
// Drop. Other types not supported yet.
}
return message_category::internal;
}
if (auto view = make_typed_message_view<exit_msg>(content)) {
auto& em = get<0>(view);
// make sure to get rid of attachables if they're no longer needed
......@@ -667,11 +633,11 @@ scheduled_actor::categorize(mailbox_element& x) {
call_handler(down_handler_, this, dm);
return message_category::internal;
}
if (auto view = make_typed_message_view<detail::action>(content)) {
if (auto ptr = get<0>(view).ptr()) {
CAF_LOG_DEBUG("run action");
ptr->run();
}
if (auto view = make_typed_message_view<action>(content)) {
auto ptr = get<0>(view).ptr();
CAF_ASSERT(ptr != nullptr);
CAF_LOG_DEBUG("run action");
ptr->run();
return message_category::internal;
}
if (auto view = make_typed_message_view<node_down_msg>(content)) {
......@@ -749,9 +715,6 @@ invoke_message_result scheduled_actor::consume(mailbox_element& x) {
return invoke_message_result::consumed;
case message_category::ordinary: {
detail::default_invoke_result_visitor<scheduled_actor> visitor{this};
auto had_timeout = getf(has_timeout_flag);
if (had_timeout)
unsetf(has_timeout_flag);
if (!bhvr_stack_.empty()) {
auto& bhvr = bhvr_stack_.back();
if (bhvr(visitor, x.content()))
......@@ -764,8 +727,6 @@ invoke_message_result scheduled_actor::consume(mailbox_element& x) {
return invoke_message_result::consumed;
},
[&](skip_t&) {
if (had_timeout)
setf(has_timeout_flag);
return invoke_message_result::skipped;
});
return visit(f, sres);
......@@ -1027,13 +988,25 @@ void scheduled_actor::handle_upstream_msg(stream_slots slots,
}
}
uint64_t scheduled_actor::set_timeout(std::string type,
actor_clock::time_point x) {
CAF_LOG_TRACE(CAF_ARG(type) << CAF_ARG(x));
auto id = ++timeout_id_;
CAF_LOG_DEBUG("set timeout:" << CAF_ARG(type) << CAF_ARG(x));
clock().set_ordinary_timeout(x, this, std::move(type), id);
return id;
disposable scheduled_actor::run_scheduled(timestamp when, action what) {
CAF_ASSERT(what.ptr() != nullptr);
CAF_LOG_TRACE(CAF_ARG(when));
auto delay = when - make_timestamp();
return run_scheduled(clock().now() + delay, std::move(what));
}
disposable scheduled_actor::run_scheduled(actor_clock::time_point when,
action what) {
CAF_ASSERT(what.ptr() != nullptr);
CAF_LOG_TRACE(CAF_ARG(when));
return clock().schedule(when, std::move(what), strong_actor_ptr{ctrl()});
}
disposable scheduled_actor::run_delayed(timespan delay, action what) {
CAF_ASSERT(what.ptr() != nullptr);
CAF_LOG_TRACE(CAF_ARG(delay));
return clock().schedule(clock().now() + delay, std::move(what),
strong_actor_ptr{ctrl()});
}
stream_slot scheduled_actor::next_slot() {
......
......@@ -37,8 +37,9 @@ public:
[&](add_atom, actor_id, const std::string& str) { std::cout << str; });
}
void enqueue(mailbox_element_ptr what, execution_unit*) override {
bool enqueue(mailbox_element_ptr what, execution_unit*) override {
mh_(what->content());
return true;
}
void setup_metrics() {
......
......@@ -2,9 +2,9 @@
// the main distribution directory for license terms and copyright or visit
// https://github.com/actor-framework/actor-framework/blob/master/LICENSE.
#define CAF_SUITE detail.action
#define CAF_SUITE action
#include "caf/detail/action.hpp"
#include "caf/action.hpp"
#include "core-test.hpp"
......@@ -23,7 +23,7 @@ SCENARIO("actions wrap function calls") {
WHEN("running the action") {
THEN("it calls the lambda and transitions from scheduled to invoked") {
auto called = false;
auto uut = detail::make_action([&called] { called = true; });
auto uut = make_action([&called] { called = true; });
CHECK(uut.scheduled());
uut.run();
CHECK(called);
......@@ -33,7 +33,7 @@ SCENARIO("actions wrap function calls") {
WHEN("disposing the action") {
THEN("it transitions to disposed and run no longer calls the lambda") {
auto called = false;
auto uut = detail::make_action([&called] { called = true; });
auto uut = make_action([&called] { called = true; });
CHECK(uut.scheduled());
uut.dispose();
CHECK(uut.disposed());
......@@ -45,7 +45,7 @@ SCENARIO("actions wrap function calls") {
WHEN("running the action multiple times") {
THEN("any call after the first becomes a no-op") {
auto n = 0;
auto uut = detail::make_action([&n] { ++n; });
auto uut = make_action([&n] { ++n; });
uut.run();
uut.run();
uut.run();
......@@ -56,10 +56,10 @@ SCENARIO("actions wrap function calls") {
WHEN("re-scheduling an action after running it") {
THEN("then the lambda gets invoked twice") {
auto n = 0;
auto uut = detail::make_action([&n] { ++n; });
auto uut = make_action([&n] { ++n; });
uut.run();
uut.run();
CHECK_EQ(uut.reschedule(), detail::action::state::scheduled);
CHECK_EQ(uut.reschedule(), action::transition::success);
uut.run();
uut.run();
CHECK(uut.invoked());
......@@ -68,9 +68,9 @@ SCENARIO("actions wrap function calls") {
}
WHEN("converting an action to a disposable") {
THEN("the disposable and the action point to the same impl object") {
auto uut = detail::make_action([] {});
auto d1 = uut.as_disposable(); // const& overload
auto d2 = detail::action{uut}.as_disposable(); // && overload
auto uut = make_action([] {});
auto d1 = uut.as_disposable(); // const& overload
auto d2 = action{uut}.as_disposable(); // && overload
CHECK_EQ(uut.ptr(), d1.ptr());
CHECK_EQ(uut.ptr(), d2.ptr());
}
......@@ -88,8 +88,7 @@ SCENARIO("actors run actions that they receive") {
};
});
auto n = 0;
inject((detail::action),
to(aut).with(detail::make_action([&n] { ++n; })));
inject((action), to(aut).with(make_action([&n] { ++n; })));
CHECK_EQ(n, 1);
}
}
......
......@@ -21,188 +21,89 @@ using namespace std::chrono_literals;
namespace {
struct testee_state {
uint64_t timeout_id = 41;
event_based_actor* self;
disposable pending;
bool run_delayed_called=false;
testee_state(event_based_actor* self) : self(self) {
// nop
}
behavior make_behavior() {
self->set_exit_handler([this](exit_msg& x) { self->quit(x.reason); });
self->set_error_handler([](scheduled_actor*, error&) {});
return {
[this](ok_atom) {
CAF_LOG_TRACE("" << self->current_mailbox_element()->content());
pending = self->run_delayed(10s, [this] { run_delayed_called = true; });
},
[](const std::string&) { CAF_LOG_TRACE(""); },
[this](group& grp) {
CAF_LOG_TRACE("");
self->join(grp);
},
};
}
};
behavior testee(stateful_actor<testee_state>* self,
detail::test_actor_clock* t) {
self->set_exit_handler([self](exit_msg& x) { self->quit(x.reason); });
self->set_error_handler([](scheduled_actor*, error&) {});
return {
[=](ok_atom) {
CAF_LOG_TRACE("" << self->current_mailbox_element()->content());
auto n = t->now() + 10s;
self->state.timeout_id += 1;
t->set_ordinary_timeout(n, self, "", self->state.timeout_id);
},
[=](add_atom) {
CAF_LOG_TRACE("");
auto n = t->now() + 10s;
self->state.timeout_id += 1;
t->set_multi_timeout(n, self, "", self->state.timeout_id);
},
[=](put_atom) {
CAF_LOG_TRACE("");
auto n = t->now() + 10s;
self->state.timeout_id += 1;
auto mid = make_message_id(self->state.timeout_id).response_id();
t->set_request_timeout(n, self, mid);
},
[](const std::string&) { CAF_LOG_TRACE(""); },
[=](group& grp) {
CAF_LOG_TRACE("");
self->join(grp);
},
};
}
using testee_actor = stateful_actor<testee_state>;
struct fixture : test_coordinator_fixture<> {
detail::test_actor_clock t;
detail::test_actor_clock& t;
actor aut;
fixture() : aut(sys.spawn<lazy_init>(testee, &t)) {
fixture() : t(sched.clock()), aut(sys.spawn<testee_actor, lazy_init>()) {
// nop
}
};
struct tid {
uint32_t value;
auto& state() {
return deref<testee_actor>(aut).state;
}
};
inline bool operator==(const timeout_msg& x, const tid& y) {
return x.timeout_id == y.value;
}
} // namespace
CAF_TEST_FIXTURE_SCOPE(timer_tests, fixture)
CAF_TEST(single_receive_timeout) {
CAF_TEST(run_delayed without dispose) {
// Have AUT call t.set_receive_timeout().
self->send(aut, ok_atom_v);
expect((ok_atom), from(self).to(aut).with(_));
CAF_CHECK_EQUAL(t.schedule().size(), 1u);
CAF_CHECK_EQUAL(t.actor_lookup().size(), 1u);
CAF_CHECK_EQUAL(t.schedule.size(), 1u);
// Advance time to send timeout message.
t.advance_time(10s);
CAF_CHECK_EQUAL(t.schedule().size(), 0u);
CAF_CHECK_EQUAL(t.actor_lookup().size(), 0u);
CAF_CHECK_EQUAL(t.schedule.size(), 0u);
// Have AUT receive the timeout.
expect((timeout_msg), from(aut).to(aut).with(tid{42}));
expect((action), to(aut));
CAF_CHECK(state().run_delayed_called);
}
CAF_TEST(override_receive_timeout) {
CAF_TEST(run_delayed with dispose before expire) {
// Have AUT call t.set_receive_timeout().
self->send(aut, ok_atom_v);
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_v);
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.
state().pending.dispose();
CAF_CHECK_EQUAL(t.schedule.size(), 1u);
// Advance time, but the clock drops the disposed callback.
t.advance_time(10s);
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(multi_timeout) {
// Have AUT call t.set_multi_timeout().
self->send(aut, add_atom_v);
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 just a little bit.
t.advance_time(5s);
// Have AUT call t.set_multi_timeout() again.
self->send(aut, add_atom_v);
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 timeout message.
t.advance_time(5s);
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 second timeout message.
t.advance_time(5s);
CAF_CHECK_EQUAL(t.schedule().size(), 0u);
CAF_CHECK_EQUAL(t.actor_lookup().size(), 0u);
CAF_CHECK_EQUAL(t.schedule.size(), 0u);
// Have AUT receive the timeout.
expect((timeout_msg), from(aut).to(aut).with(tid{43}));
disallow((action), to(aut));
CAF_CHECK(!state().run_delayed_called);
}
CAF_TEST(mixed_receive_and_multi_timeouts) {
CAF_TEST(run_delayed with dispose after expire) {
// Have AUT call t.set_receive_timeout().
self->send(aut, add_atom_v);
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 just a little bit.
t.advance_time(5s);
// Have AUT call t.set_multi_timeout() again.
self->send(aut, ok_atom_v);
expect((ok_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 timeout message.
t.advance_time(5s);
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 second timeout message.
t.advance_time(5s);
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, put_atom_v);
expect((put_atom), from(self).to(aut).with(_));
CAF_CHECK_EQUAL(t.schedule().size(), 1u);
CAF_CHECK_EQUAL(t.actor_lookup().size(), 1u);
CAF_CHECK_EQUAL(t.schedule.size(), 1u);
// Advance time to send timeout message.
t.advance_time(10s);
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_v);
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(5s);
// Have AUT call t.set_request_timeout().
self->send(aut, put_atom_v);
expect((put_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(5s);
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(10s);
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_CHECK_EQUAL(t.schedule.size(), 0u);
// Have AUT receive the timeout but dispose it: turns into a nop.
state().pending.dispose();
expect((action), to(aut));
CAF_CHECK(!state().run_delayed_called);
}
CAF_TEST(delay_actor_message) {
......@@ -212,12 +113,10 @@ CAF_TEST(delay_actor_message) {
t.schedule_message(n, autptr,
make_mailbox_element(autptr, make_message_id(), no_stages,
"foo"));
CAF_CHECK_EQUAL(t.schedule().size(), 1u);
CAF_CHECK_EQUAL(t.actor_lookup().size(), 0u);
CAF_CHECK_EQUAL(t.schedule.size(), 1u);
// Advance time to send the message.
t.advance_time(10s);
CAF_CHECK_EQUAL(t.schedule().size(), 0u);
CAF_CHECK_EQUAL(t.actor_lookup().size(), 0u);
CAF_CHECK_EQUAL(t.schedule.size(), 0u);
// Have AUT receive the message.
expect((std::string), from(aut).to(aut).with("foo"));
}
......@@ -231,12 +130,10 @@ CAF_TEST(delay_group_message) {
auto n = t.now() + 10s;
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);
CAF_CHECK_EQUAL(t.schedule.size(), 1u);
// Advance time to send the message.
t.advance_time(10s);
CAF_CHECK_EQUAL(t.schedule().size(), 0u);
CAF_CHECK_EQUAL(t.actor_lookup().size(), 0u);
CAF_CHECK_EQUAL(t.schedule.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).
......
......@@ -67,8 +67,9 @@ public:
// nop
}
void enqueue(mailbox_element_ptr what, execution_unit*) override {
bool enqueue(mailbox_element_ptr what, execution_unit*) override {
mbox.push_back(std::move(what->payload));
return true;
}
void attach(attachable_ptr) override {
......
......@@ -9,76 +9,109 @@
#include "core-test.hpp"
using namespace caf;
using std::endl;
namespace {
struct fixture {
actor_system_config cfg;
actor_system sys;
scoped_actor self;
fixture() : sys(cfg), self(sys, true) {
// nop
using namespace std::literals;
SCENARIO("an actor system shuts down after the last actor terminates") {
GIVEN("an actor system and a detached actor") {
WHEN("the actor sets no behavior") {
auto ran = std::make_shared<bool>(false);
THEN("the actor terminates immediately and the system shuts down") {
actor_system_config cfg;
actor_system sys{cfg};
sys.spawn<detached>([=] { *ran = true; });
}
CHECK(*ran);
}
WHEN("the actor uses delayed_send but ignores the message") {
auto ran = std::make_shared<bool>(false);
THEN("the actor terminates immediately and the system shuts down") {
actor_system_config cfg;
actor_system sys{cfg};
sys.spawn<detached>([=](event_based_actor* self) {
*ran = true;
self->delayed_send(self, 1h, ok_atom_v);
});
}
CHECK(*ran);
}
WHEN("the actor uses delayed_send and waits for the message") {
auto ran = std::make_shared<bool>(false);
auto message_handled = std::make_shared<bool>(false);
THEN("the system waits for the actor to handle its message") {
actor_system_config cfg;
actor_system sys{cfg};
sys.spawn<detached>([=](event_based_actor* self) -> behavior {
*ran = true;
self->delayed_send(self, 1ns, ok_atom_v);
return {
[=](ok_atom) {
*message_handled = true;
self->quit();
},
};
});
}
CHECK(*ran);
CHECK(*message_handled);
}
WHEN("the actor uses run_delayed() to wait some time") {
auto ran = std::make_shared<bool>(false);
auto timeout_handled = std::make_shared<bool>(false);
THEN("the system waits for the actor to handle the timeout") {
actor_system_config cfg;
actor_system sys{cfg};
sys.spawn<detached>([=](event_based_actor* self) -> behavior {
*ran = true;
self->run_delayed(1ns, [=] {
*timeout_handled = true;
self->quit();
});
return {
[](int) {
// Dummy handler to force the actor to stay alive until we call
// self->quit in the run_delayed lambda.
},
};
});
}
CHECK(*ran);
CHECK(*timeout_handled);
}
WHEN("the actor uses after() to wait some time") {
auto ran = std::make_shared<bool>(false);
auto timeout_handled = std::make_shared<bool>(false);
THEN("the system waits for the actor to handle the timeout") {
actor_system_config cfg;
actor_system sys{cfg};
sys.spawn<detached>([=](event_based_actor* self) -> behavior {
*ran = true;
return {
after(1ns) >>
[=] {
*timeout_handled = true;
self->quit();
},
};
});
}
CHECK(*ran);
CHECK(*timeout_handled);
}
}
};
} // namespace
CAF_TEST_FIXTURE_SCOPE(detached_actors, fixture)
CAF_TEST(shutdown) {
CAF_MESSAGE("does sys shut down after spawning a detached actor?");
sys.spawn<detached>([] {});
}
CAF_TEST(shutdown_with_delayed_send) {
CAF_MESSAGE("does sys shut down after spawning a detached actor that used "
"delayed_send?");
auto f = [](event_based_actor* self) -> behavior {
self->delayed_send(self, std::chrono::nanoseconds(1), ok_atom_v);
return {
[=](ok_atom) { self->quit(); },
};
};
sys.spawn<detached>(f);
}
CAF_TEST(shutdown_with_unhandled_delayed_send) {
CAF_MESSAGE("does sys shut down after spawning a detached actor that used "
"delayed_send but didn't bother waiting for it?");
auto f = [](event_based_actor* self) {
self->delayed_send(self, std::chrono::nanoseconds(1), ok_atom_v);
};
sys.spawn<detached>(f);
}
CAF_TEST(shutdown_with_after) {
CAF_MESSAGE("does sys shut down after spawning a detached actor that used "
"after()?");
auto f = [](event_based_actor* self) -> behavior {
return {
after(std::chrono::nanoseconds(1)) >> [=] { self->quit(); },
};
};
sys.spawn<detached>(f);
}
CAF_TEST(shutdown_delayed_send_loop) {
CAF_MESSAGE("does sys shut down after spawning a detached actor that used "
"a delayed send loop and was interrupted via exit message?");
auto f = [](event_based_actor* self) -> behavior {
self->delayed_send(self, std::chrono::milliseconds(1), ok_atom_v);
return {
[=](ok_atom) {
self->delayed_send(self, std::chrono::milliseconds(1), ok_atom_v);
},
};
};
auto a = sys.spawn<detached>(f);
auto g = detail::make_scope_guard(
[&] { self->send_exit(a, exit_reason::user_shutdown); });
}
CAF_TEST_FIXTURE_SCOPE_END()
// CAF_TEST(shutdown_delayed_send_loop) {
// CAF_MESSAGE("does sys shut down after spawning a detached actor that used "
// "a delayed send loop and was interrupted via exit message?");
// auto f = [](event_based_actor* self) -> behavior {
// self->delayed_send(self, 1ns, ok_atom_v);
// return {
// [=](ok_atom) {
// self->delayed_send(self, 1ns, ok_atom_v);
// },
// };
// };
// auto a = sys.spawn<detached>(f);
// auto g = detail::make_scope_guard(
// [&] { self->send_exit(a, exit_reason::user_shutdown); });
// }
......@@ -152,10 +152,11 @@ public:
CAF_ASSERT(global_time_ != nullptr);
}
void enqueue(mailbox_element_ptr what, execution_unit*) override {
bool enqueue(mailbox_element_ptr what, execution_unit*) override {
auto push_back_result = mbox.push_back(std::move(what));
CAF_CHECK_EQUAL(push_back_result, true);
CAF_ASSERT(push_back_result);
return true;
}
void attach(attachable_ptr) override {
......
......@@ -310,8 +310,8 @@ CAF_TEST(depth_2_pipeline_500_items) {
}
CAF_MESSAGE("trigger timeouts");
tick();
allow((timeout_msg), from(snk).to(snk));
allow((timeout_msg), from(src).to(src));
allow((action), from(snk).to(snk));
allow((action), from(src).to(src));
CAF_MESSAGE("process ack_batch in source");
expect((upstream_msg::ack_batch), from(snk).to(src));
} while (!received<downstream_msg::close>(snk));
......
......@@ -31,13 +31,20 @@ struct fixture : test_coordinator_fixture<> {
return sys.spawn(init);
}
std::function<void(const error&)> make_error_handler() {
auto make_error_handler() {
return [](const error& err) { CAF_FAIL("unexpected error: " << err); };
}
std::function<void(const error&)> make_counting_error_handler(size_t* count) {
auto make_counting_error_handler(size_t* count) {
return [count](const error&) { *count += 1; };
}
template <class... ResponseHandles>
auto fuse(ResponseHandles&... handles) {
return select_all<detail::type_list<int>>{
{handles.id()...},
disposable::make_composite({handles.policy().pending_timeouts()...})};
}
};
} // namespace
......@@ -58,7 +65,7 @@ CAF_TEST(select_all combines two integer results into one vector) {
SUBTEST("vector of int") {
auto r1 = self->request(server1, infinite, 1, 2);
auto r2 = self->request(server2, infinite, 2, 3);
select_all<detail::type_list<int>> merge{{r1.id(), r2.id()}};
auto merge = fuse(r1, r2);
run();
merge.receive(
self.ptr(),
......@@ -72,15 +79,14 @@ CAF_TEST(select_all combines two integer results into one vector) {
using std::make_tuple;
auto r1 = self->request(server1, infinite, 1, 2);
auto r2 = self->request(server2, infinite, 2, 3);
select_all<detail::type_list<int>> merge{{r1.id(), r2.id()}};
auto merge = fuse(r1, r2);
run();
using results_vector = std::vector<std::tuple<int>>;
using results_vector = std::vector<int>;
merge.receive(
self.ptr(),
[](results_vector results) {
std::sort(results.begin(), results.end());
CAF_CHECK_EQUAL(results,
results_vector({make_tuple(3), make_tuple(5)}));
CAF_CHECK_EQUAL(results, results_vector({3, 5}));
},
make_error_handler());
}
......@@ -90,7 +96,7 @@ CAF_TEST(select_all combines two integer results into one vector) {
auto client = sys.spawn([=, &results](event_based_actor* client_ptr) {
auto r1 = client_ptr->request(server1, infinite, 1, 2);
auto r2 = client_ptr->request(server2, infinite, 2, 3);
select_all<detail::type_list<int>> merge{{r1.id(), r2.id()}};
auto merge = fuse(r1, r2);
merge.then(
client_ptr, [&results](int_list xs) { results = std::move(xs); },
make_error_handler());
......@@ -108,7 +114,7 @@ CAF_TEST(select_all combines two integer results into one vector) {
auto client = sys.spawn([=, &results](event_based_actor* client_ptr) {
auto r1 = client_ptr->request(server1, infinite, 1, 2);
auto r2 = client_ptr->request(server2, infinite, 2, 3);
select_all<detail::type_list<int>> merge{{r1.id(), r2.id()}};
auto merge = fuse(r1, r2);
merge.await(
client_ptr, [&results](int_list xs) { results = std::move(xs); },
make_error_handler());
......@@ -133,7 +139,7 @@ CAF_TEST(select_all calls the error handler at most once) {
SUBTEST("request.receive") {
auto r1 = self->request(server1, infinite, 1, 2);
auto r2 = self->request(server2, infinite, 2, 3);
select_all<detail::type_list<int>> merge{{r1.id(), r2.id()}};
auto merge = fuse(r1, r2);
run();
size_t errors = 0;
merge.receive(
......@@ -147,7 +153,7 @@ CAF_TEST(select_all calls the error handler at most once) {
auto client = sys.spawn([=, &errors](event_based_actor* client_ptr) {
auto r1 = client_ptr->request(server1, infinite, 1, 2);
auto r2 = client_ptr->request(server2, infinite, 2, 3);
select_all<detail::type_list<int>> merge{{r1.id(), r2.id()}};
auto merge = fuse(r1, r2);
merge.then(
client_ptr,
[](int_list) { CAF_FAIL("fan-in policy called the result handler"); },
......@@ -165,7 +171,7 @@ CAF_TEST(select_all calls the error handler at most once) {
auto client = sys.spawn([=, &errors](event_based_actor* client_ptr) {
auto r1 = client_ptr->request(server1, infinite, 1, 2);
auto r2 = client_ptr->request(server2, infinite, 2, 3);
select_all<detail::type_list<int>> merge{{r1.id(), r2.id()}};
auto merge = fuse(r1, r2);
merge.await(
client_ptr,
[](int_list) { CAF_FAIL("fan-in policy called the result handler"); },
......
......@@ -36,6 +36,13 @@ struct fixture : test_coordinator_fixture<> {
auto make_counting_error_handler(size_t* count) {
return [count](const error&) { *count += 1; };
}
template <class... ResponseHandles>
auto fuse(ResponseHandles&... handles) {
return select_any<detail::type_list<int>>{
{handles.id()...},
disposable::make_composite({handles.policy().pending_timeouts()...})};
}
};
} // namespace
......@@ -55,7 +62,7 @@ CAF_TEST(select_any picks the first arriving integer) {
SUBTEST("single integer") {
auto r1 = self->request(server1, infinite, 1, 2);
auto r2 = self->request(server2, infinite, 2, 3);
select_any<detail::type_list<int>> choose{{r1.id(), r2.id()}};
auto choose = fuse(r1, r2);
run();
choose.receive(
self.ptr(), [](int result) { CAF_CHECK_EQUAL(result, 3); },
......@@ -67,7 +74,7 @@ CAF_TEST(select_any picks the first arriving integer) {
auto client = sys.spawn([=, &result](event_based_actor* client_ptr) {
auto r1 = client_ptr->request(server1, infinite, 1, 2);
auto r2 = client_ptr->request(server2, infinite, 2, 3);
select_any<detail::type_list<int>> choose{{r1.id(), r2.id()}};
auto choose = fuse(r1, r2);
choose.then(
client_ptr, [&result](int x) { result = x; }, make_error_handler());
});
......@@ -84,7 +91,7 @@ CAF_TEST(select_any picks the first arriving integer) {
auto client = sys.spawn([=, &result](event_based_actor* client_ptr) {
auto r1 = client_ptr->request(server1, infinite, 1, 2);
auto r2 = client_ptr->request(server2, infinite, 2, 3);
select_any<detail::type_list<int>> choose{{r1.id(), r2.id()}};
auto choose = fuse(r1, r2);
choose.await(
client_ptr, [&result](int x) { result = x; }, make_error_handler());
});
......@@ -107,7 +114,7 @@ CAF_TEST(select_any calls the error handler at most once) {
SUBTEST("request.receive") {
auto r1 = self->request(server1, infinite, 1, 2);
auto r2 = self->request(server2, infinite, 2, 3);
select_any<detail::type_list<int>> choose{{r1.id(), r2.id()}};
auto choose = fuse(r1, r2);
run();
size_t errors = 0;
choose.receive(
......@@ -121,7 +128,7 @@ CAF_TEST(select_any calls the error handler at most once) {
auto client = sys.spawn([=, &errors](event_based_actor* client_ptr) {
auto r1 = client_ptr->request(server1, infinite, 1, 2);
auto r2 = client_ptr->request(server2, infinite, 2, 3);
select_any<detail::type_list<int>> choose{{r1.id(), r2.id()}};
auto choose = fuse(r1, r2);
choose.then(
client_ptr,
[](int) { CAF_FAIL("fan-in policy called the result handler"); },
......@@ -139,7 +146,7 @@ CAF_TEST(select_any calls the error handler at most once) {
auto client = sys.spawn([=, &errors](event_based_actor* client_ptr) {
auto r1 = client_ptr->request(server1, infinite, 1, 2);
auto r2 = client_ptr->request(server2, infinite, 2, 3);
select_any<detail::type_list<int>> choose{{r1.id(), r2.id()}};
auto choose = fuse(r1, r2);
choose.await(
client_ptr,
[](int) { CAF_FAIL("fan-in policy called the result handler"); },
......
......@@ -79,9 +79,9 @@ public:
// -- overridden modifiers of abstract_actor ---------------------------------
void enqueue(mailbox_element_ptr, execution_unit*) override;
bool enqueue(mailbox_element_ptr, execution_unit*) override;
void enqueue(strong_actor_ptr, message_id, message, execution_unit*) override;
bool enqueue(strong_actor_ptr, message_id, message, execution_unit*) override;
// -- overridden modifiers of local_actor ------------------------------------
......
......@@ -17,15 +17,15 @@
namespace caf::io {
void abstract_broker::enqueue(strong_actor_ptr src, message_id mid, message msg,
bool abstract_broker::enqueue(strong_actor_ptr src, message_id mid, message msg,
execution_unit*) {
enqueue(make_mailbox_element(std::move(src), mid, {}, std::move(msg)),
backend_);
return enqueue(make_mailbox_element(std::move(src), mid, {}, std::move(msg)),
backend_);
}
void abstract_broker::enqueue(mailbox_element_ptr ptr, execution_unit*) {
bool abstract_broker::enqueue(mailbox_element_ptr ptr, execution_unit*) {
CAF_PUSH_AID(id());
scheduled_actor::enqueue(std::move(ptr), backend_);
return scheduled_actor::enqueue(std::move(ptr), backend_);
}
void abstract_broker::launch(execution_unit* eu, bool lazy, bool hide) {
......
......@@ -42,8 +42,9 @@ public:
// nop
}
void enqueue(mailbox_element_ptr, execution_unit*) override {
bool enqueue(mailbox_element_ptr, execution_unit*) override {
CAF_FAIL("mock_actor_proxy::enqueue called");
return false;
}
void kill_proxy(execution_unit*, error) override {
......
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