Commit 1608c0b8 authored by Dominik Charousset's avatar Dominik Charousset

Merge branch 'topic/broker-mailbox' into develop

parents 4feb5410 44bf62ad
......@@ -142,7 +142,7 @@ protected:
/// Called by the runtime system to perform cleanup actions for this actor.
/// Subtypes should always call this member function when overriding it.
void cleanup(uint32_t reason);
virtual void cleanup(uint32_t reason);
/// Returns `exit_reason() != exit_reason::not_exited`.
inline bool exited() const {
......
......@@ -145,7 +145,7 @@ public:
template <class F>
void close(const F& f) {
clear_cached_elements(f);
if (fetch_new_data(nullptr)) {
if (! blocked() && fetch_new_data(nullptr)) {
clear_cached_elements(f);
}
cache_.clear(f);
......
......@@ -430,6 +430,12 @@ public:
/// @cond PRIVATE
// handle `ptr` in an event-based actor
resumable::resume_result exec_event(mailbox_element_ptr& ptr);
// handle `ptr` in an event-based actor, not suitable to be called in a loop
void exec_single_event(mailbox_element_ptr& ptr);
local_actor();
template <class ActorHandle>
......@@ -563,7 +569,11 @@ public:
virtual void initialize() = 0;
void cleanup(uint32_t reason);
// clear behavior stack and call cleanup if actor either has no
// valid behavior left or has set a planned exit reason
bool finalize();
void cleanup(uint32_t reason) override;
// an actor can have multiple pending timeouts, but only
// the latest one is active (i.e. the pending_timeouts_.back())
......
......@@ -43,7 +43,8 @@ class abstract_coordinator {
public:
friend class detail::singletons;
explicit abstract_coordinator(size_t num_worker_threads);
explicit abstract_coordinator(size_t num_worker_threads,
size_t max_throughput_param);
virtual ~abstract_coordinator();
......@@ -64,6 +65,10 @@ public:
nullptr);
}
inline size_t max_throughput() const {
return max_throughput_;
}
inline size_t num_workers() const {
return num_workers_;
}
......@@ -84,13 +89,16 @@ protected:
delete this;
}
actor timer_;
actor printer_;
// ID of the worker receiving the next enqueue
std::atomic<size_t> next_worker_;
// number of messages each actor is allowed to consume per resume
size_t max_throughput_;
size_t num_workers_;
actor timer_;
actor printer_;
};
} // namespace scheduler
......
......@@ -43,8 +43,7 @@ public:
coordinator(size_t nw = std::max(std::thread::hardware_concurrency(), 4u),
size_t mt = std::numeric_limits<size_t>::max())
: super(nw),
max_throughput_(mt) {
: super(nw, mt) {
// nop
}
......@@ -145,8 +144,6 @@ private:
policy_data data_;
// instance of our policy object
Policy policy_;
// number of messages each actor is allowed to consume per resume
size_t max_throughput_;
};
} // namespace scheduler
......
......@@ -24,6 +24,8 @@
#include <type_traits>
#include "caf/fwd.hpp"
#include "caf/detail/logging.hpp"
#include "caf/detail/type_traits.hpp"
namespace caf {
......@@ -60,6 +62,7 @@ public:
/// Destroys the state of this actor (no further overriding allowed).
void on_exit() override final {
CAF_LOG_TRACE("");
state_.~State();
}
......
......@@ -345,8 +345,9 @@ void abstract_coordinator::stop_actors() {
);
}
abstract_coordinator::abstract_coordinator(size_t nw)
abstract_coordinator::abstract_coordinator(size_t nw, size_t mt)
: next_worker_(0),
max_throughput_(mt),
num_workers_(nw) {
// nop
}
......
......@@ -682,33 +682,21 @@ resumable::resume_result local_actor::resume(execution_unit* eu,
}
// actor is cooperatively scheduled
host(eu);
auto actor_done = [&]() -> bool {
if (! has_behavior() || planned_exit_reason() != exit_reason::not_exited) {
CAF_LOG_DEBUG("actor either has no behavior or has set an exit reason");
on_exit();
bhvr_stack().clear();
bhvr_stack().cleanup();
auto rsn = planned_exit_reason();
if (rsn == exit_reason::not_exited) {
rsn = exit_reason::normal;
planned_exit_reason(rsn);
}
cleanup(rsn);
return true;
}
return false;
};
// actors without behavior or that have already defined
// an exit reason must not be resumed
CAF_ASSERT(! is_initialized()
|| (has_behavior()
&& planned_exit_reason() == exit_reason::not_exited));
if (is_initialized()
&& (! has_behavior()
|| planned_exit_reason() != exit_reason::not_exited)) {
CAF_LOG_DEBUG_IF(! has_behavior(),
"resume called on an actor without behavior");
CAF_LOG_DEBUG_IF(planned_exit_reason() != exit_reason::not_exited,
"resume called on an actor with exit reason");
return resumable::done;
}
std::exception_ptr eptr = nullptr;
try {
if (! is_initialized()) {
CAF_LOG_DEBUG("initialize actor");
initialize();
if (actor_done()) {
if (finalize()) {
CAF_LOG_DEBUG("actor_done() returned true right "
<< "after make_behavior()");
return resumable::resume_result::done;
......@@ -723,36 +711,9 @@ resumable::resume_result local_actor::resume(execution_unit* eu,
for (size_t i = 0; i < max_throughput; ++i) {
auto ptr = next_message();
if (ptr) {
auto& bhvr = awaits_response()
? awaited_response_handler()
: bhvr_stack().back();
auto mid = awaited_response_id();
switch (invoke_message(ptr, bhvr, mid)) {
case im_success:
bhvr_stack().cleanup();
++handled_msgs;
if (actor_done()) {
CAF_LOG_DEBUG("actor exited");
return resumable::resume_result::done;
}
// continue from cache if current message was
// handled, because the actor might have changed
// its behavior to match 'old' messages now
while (invoke_from_cache()) {
if (actor_done()) {
CAF_LOG_DEBUG("actor exited");
return resumable::resume_result::done;
}
}
break;
case im_skipped:
CAF_ASSERT(ptr != nullptr);
push_to_cache(std::move(ptr));
break;
case im_dropped:
// destroy msg
break;
}
if (exec_event(ptr) == resumable::resume_result::done)
return resumable::resume_result::done;
++handled_msgs;
} else {
CAF_LOG_DEBUG("no more element in mailbox; going to block");
reset_timeout_if_needed();
......@@ -797,13 +758,75 @@ resumable::resume_result local_actor::resume(execution_unit* eu,
planned_exit_reason(*opt_reason);
}
}
if (! actor_done()) {
if (! finalize()) {
// actor has been "revived", try running it again later
return resumable::resume_later;
}
return resumable::done;
}
resumable::resume_result local_actor::exec_event(mailbox_element_ptr& ptr) {
auto& bhvr = awaits_response() ? awaited_response_handler()
: bhvr_stack().back();
auto mid = awaited_response_id();
switch (invoke_message(ptr, bhvr, mid)) {
case im_success:
bhvr_stack().cleanup();
if (finalize()) {
CAF_LOG_DEBUG("actor exited");
return resumable::resume_result::done;
}
// continue from cache if current message was
// handled, because the actor might have changed
// its behavior to match 'old' messages now
while (invoke_from_cache()) {
if (finalize()) {
CAF_LOG_DEBUG("actor exited");
return resumable::resume_result::done;
}
}
break;
case im_skipped:
CAF_ASSERT(ptr != nullptr);
push_to_cache(std::move(ptr));
break;
case im_dropped:
// destroy msg
break;
}
return resumable::resume_result::resume_later;
}
void local_actor::exec_single_event(mailbox_element_ptr& ptr) {
if (! is_initialized()) {
CAF_LOG_DEBUG("initialize actor");
initialize();
if (finalize()) {
CAF_LOG_DEBUG("actor_done() returned true right "
<< "after make_behavior()");
return;
}
}
if (! has_behavior() || planned_exit_reason() != exit_reason::not_exited) {
CAF_LOG_DEBUG_IF(! has_behavior(),
"resume called on an actor without behavior");
CAF_LOG_DEBUG_IF(planned_exit_reason() != exit_reason::not_exited,
"resume called on an actor with exit reason");
return;
}
try {
exec_event(ptr);
}
catch (...) {
CAF_LOG_INFO("broker died because of an exception");
auto eptr = std::current_exception();
auto opt_reason = this->handle(eptr);
if (opt_reason)
planned_exit_reason(*opt_reason);
finalize();
}
}
mailbox_element_ptr local_actor::next_message() {
if (! is_priority_aware()) {
return mailbox_element_ptr{mailbox().try_pop()};
......@@ -934,6 +957,22 @@ behavior& local_actor::get_behavior() {
: pending_responses_.front().second;
}
bool local_actor::finalize() {
if (has_behavior() && planned_exit_reason() == exit_reason::not_exited)
return false;
CAF_LOG_DEBUG("actor either has no behavior or has set an exit reason");
on_exit();
bhvr_stack().clear();
bhvr_stack().cleanup();
auto rsn = planned_exit_reason();
if (rsn == exit_reason::not_exited) {
rsn = exit_reason::normal;
planned_exit_reason(rsn);
}
cleanup(rsn);
return true;
}
void local_actor::cleanup(uint32_t reason) {
CAF_LOG_TRACE(CAF_ARG(reason));
current_mailbox_element().reset();
......
......@@ -27,8 +27,6 @@
#include "caf/detail/intrusive_partitioned_list.hpp"
#include "caf/io/fwd.hpp"
#include "caf/io/scribe.hpp"
#include "caf/io/doorman.hpp"
#include "caf/io/accept_handle.hpp"
#include "caf/io/receive_policy.hpp"
#include "caf/io/system_messages.hpp"
......@@ -92,7 +90,7 @@ public:
void enqueue(mailbox_element_ptr, execution_unit*) override;
/// Called after this broker has finished execution.
void cleanup(uint32_t reason);
void cleanup(uint32_t reason) override;
/// Starts running this broker in the `middleman`.
void launch(execution_unit* eu, bool lazy, bool hide);
......@@ -177,12 +175,6 @@ public:
/// Returns the handle associated to given local `port` or `none`.
optional<accept_handle> hdl_by_port(uint16_t port);
/// Invokes `msg` on this broker.
void invoke_message(mailbox_element_ptr& msg);
/// Creates a mailbox element from given data and invoke it.
void invoke_message(const actor_addr& sender, message_id mid, message& msg);
/// Closes all connections and acceptors.
void close_all();
......@@ -199,7 +191,22 @@ public:
return get_map(hdl).count(hdl) > 0;
}
/// @cond PRIVATE
template <class Handle>
void erase(Handle hdl) {
auto& elements = get_map(hdl);
auto i = elements.find(hdl);
if (i != elements.end())
elements.erase(i);
}
/// @endcond
const char* name() const override;
protected:
void init_broker();
abstract_broker();
abstract_broker(middleman& parent_ref);
......@@ -257,9 +264,6 @@ protected:
return result;
}
/// Tries to invoke a message from the cache.
bool invoke_message_from_cache();
private:
scribe_map scribes_;
doorman_map doormen_;
......
......@@ -27,6 +27,8 @@
#include "caf/extend.hpp"
#include "caf/local_actor.hpp"
#include "caf/io/scribe.hpp"
#include "caf/io/doorman.hpp"
#include "caf/io/abstract_broker.hpp"
namespace caf {
......
/******************************************************************************
* ____ _ _____ *
* / ___| / \ | ___| C++ *
* | | / _ \ | |_ Actor *
* | |___ / ___ \| _| Framework *
* \____/_/ \_|_| *
* *
* Copyright (C) 2011 - 2015 *
* Dominik Charousset <dominik.charousset (at) haw-hamburg.de> *
* *
* Distributed under the terms and conditions of the BSD 3-Clause License or *
* (at your option) under the terms and conditions of the Boost Software *
* License 1.0. See accompanying files LICENSE and LICENSE_ALTERNATIVE. *
* *
* If you did not receive a copy of the license files, see *
* http://opensource.org/licenses/BSD-3-Clause and *
* http://www.boost.org/LICENSE_1_0.txt. *
******************************************************************************/
#ifndef CAF_IO_BROKER_SERVANT_HPP
#define CAF_IO_BROKER_SERVANT_HPP
#include "caf/mailbox_element.hpp"
#include "caf/io/abstract_broker.hpp"
#include "caf/io/system_messages.hpp"
namespace caf {
namespace io {
/// Base class for `scribe` and `doorman`.
/// @ingroup Broker
template <class Base, class Handle, class SysMsgType>
class broker_servant : public Base {
public:
broker_servant(abstract_broker* ptr, Handle hdl) : Base(ptr), hdl_(hdl) {
// nop
}
Handle hdl() const {
return hdl_;
}
protected:
void detach_from_parent() override {
auto ptr = this->parent();
this->set_parent(nullptr);
ptr->erase(hdl_);
}
void invoke_mailbox_element() {
this->parent()->exec_single_event(mailbox_elem_ptr_);
}
SysMsgType& msg() {
if (! mailbox_elem_ptr_)
reset_mailbox_element();
return mailbox_elem_ptr_->msg.get_as_mutable<SysMsgType>(0);
}
static void set_hdl(new_connection_msg& lhs, Handle& hdl) {
lhs.source = hdl;
}
static void set_hdl(new_data_msg& lhs, Handle& hdl) {
lhs.handle = hdl;
}
void reset_mailbox_element() {
SysMsgType tmp;
set_hdl(tmp, hdl_);
mailbox_elem_ptr_ = mailbox_element::make_joint(invalid_actor_addr,
invalid_message_id, tmp);
}
Handle hdl_;
mailbox_element_ptr mailbox_elem_ptr_;
};
} // namespace io {
} // namespace caf {
#endif // CAF_IO_BROKER_SERVANT_HPP
......@@ -23,46 +23,34 @@
#include <cstddef>
#include "caf/message.hpp"
#include "caf/mailbox_element.hpp"
#include "caf/io/accept_handle.hpp"
#include "caf/io/broker_servant.hpp"
#include "caf/io/system_messages.hpp"
#include "caf/io/network/acceptor_manager.hpp"
namespace caf {
namespace io {
using doorman_base = broker_servant<network::acceptor_manager, accept_handle,
new_connection_msg>;
/// Manages incoming connections.
/// @ingroup Broker
class doorman : public network::acceptor_manager {
class doorman : public doorman_base {
public:
doorman(abstract_broker* parent, accept_handle hdl);
~doorman();
inline accept_handle hdl() const {
return hdl_;
}
void io_failure(network::operation op) override;
// needs to be launched explicitly
virtual void launch() = 0;
protected:
void detach_from_parent() override;
message detach_message() override;
inline new_connection_msg& accept_msg() {
return accept_msg_.get_as_mutable<new_connection_msg>(0);
}
inline const new_connection_msg& accept_msg() const {
return accept_msg_.get_as<new_connection_msg>(0);
}
accept_handle hdl_;
message accept_msg_;
};
} // namespace io
......
......@@ -153,7 +153,8 @@ public:
}
void initialize() override {
this->is_initialized(true);
CAF_LOG_TRACE("");
this->init_broker();
auto bhvr = make_behavior();
CAF_LOG_DEBUG_IF(! bhvr, "make_behavior() did not return a behavior, "
<< "has_behavior() = "
......
......@@ -23,7 +23,9 @@
namespace caf {
namespace io {
class scribe;
class broker;
class doorman;
class middleman;
class basp_broker;
class receive_policy;
......
......@@ -137,6 +137,10 @@ public:
middleman(const backend_factory&);
inline size_t max_throughput() const {
return max_throughput_;
}
/// @endcond
private:
......@@ -154,6 +158,8 @@ private:
hook_uptr hooks_;
// actor offering asyncronous IO by managing this singleton instance
middleman_actor manager_;
// stores the max_throughput parameter from the scheduler coordinator
size_t max_throughput_;
};
} // namespace io
......
......@@ -227,10 +227,9 @@ asio_multiplexer::add_tcp_doorman(abstract_broker* self,
void new_connection() override {
CAF_LOG_TRACE("");
auto& am = acceptor_.backend();
accept_msg().handle
msg().handle
= am.add_tcp_scribe(parent(), std::move(acceptor_.accepted_socket()));
parent()->invoke_message(invalid_actor_addr, invalid_message_id,
accept_msg_);
invoke_mailbox_element();
}
void stop_reading() override {
CAF_LOG_TRACE("");
......
......@@ -24,6 +24,7 @@
#include "caf/message.hpp"
#include "caf/io/broker_servant.hpp"
#include "caf/io/receive_policy.hpp"
#include "caf/io/system_messages.hpp"
#include "caf/io/network/stream_manager.hpp"
......@@ -31,9 +32,12 @@
namespace caf {
namespace io {
using scribe_base = broker_servant<network::stream_manager, connection_handle,
new_data_msg>;
/// Manages a stream.
/// @ingroup Broker
class scribe : public network::stream_manager {
class scribe : public scribe_base {
public:
scribe(abstract_broker* parent, connection_handle hdl);
......@@ -42,39 +46,22 @@ public:
/// Implicitly starts the read loop on first call.
virtual void configure_read(receive_policy::config config) = 0;
/// Grants access to the output buffer.
/// Returns the current output buffer.
virtual std::vector<char>& wr_buf() = 0;
/// Flushes the output buffer, i.e., sends the content of
/// the buffer via the network.
virtual void flush() = 0;
/// Returns the current input buffer.
virtual std::vector<char>& rd_buf() = 0;
inline connection_handle hdl() const {
return hdl_;
}
/// Flushes the output buffer, i.e., sends the
/// content of the buffer via the network.
virtual void flush() = 0;
void io_failure(network::operation op) override;
void consume(const void* data, size_t num_bytes) override;
protected:
virtual std::vector<char>& rd_buf() = 0;
inline new_data_msg& read_msg() {
return read_msg_.get_as_mutable<new_data_msg>(0);
}
inline const new_data_msg& read_msg() const {
return read_msg_.get_as<new_data_msg>(0);
}
void detach_from_parent() override;
message detach_message() override;
private:
connection_handle hdl_;
message read_msg_;
};
} // namespace io
......
......@@ -29,14 +29,14 @@
#include "caf/detail/scope_guard.hpp"
#include "caf/detail/sync_request_bouncer.hpp"
#include "caf/event_based_actor.hpp"
namespace caf {
namespace io {
class abstract_broker::continuation {
public:
continuation(intrusive_ptr<abstract_broker> bptr, mailbox_element_ptr mptr)
: self_(std::move(bptr)),
ptr_(std::move(mptr)) {
continuation(intrusive_ptr<abstract_broker> bptr) : self_(std::move(bptr)) {
// nop
}
......@@ -45,16 +45,41 @@ public:
inline void operator()() {
CAF_PUSH_AID(self_->id());
CAF_LOG_TRACE("");
self_->invoke_message(ptr_);
auto mt = self_->parent().max_throughput();
// re-schedule broker if it reached its maximum message throughput
if (self_->resume(nullptr, mt) == resumable::resume_later)
self_->backend().post(std::move(*this));
}
private:
intrusive_ptr<abstract_broker> self_;
mailbox_element_ptr ptr_;
};
void abstract_broker::enqueue(mailbox_element_ptr ptr, execution_unit*) {
backend().post(continuation{this, std::move(ptr)});
CAF_PUSH_AID(id());
CAF_LOG_TRACE("enqueue " << CAF_TSARG(ptr->msg));
auto mid = ptr->mid;
auto sender = ptr->sender;
switch (mailbox().enqueue(ptr.release())) {
case detail::enqueue_result::unblocked_reader: {
// re-schedule broker
CAF_LOG_DEBUG("unblocked_reader");
backend().post(continuation{this});
break;
}
case detail::enqueue_result::queue_closed: {
CAF_LOG_DEBUG("queue_closed");
if (mid.is_request()) {
detail::sync_request_bouncer f{exit_reason()};
f(sender, mid);
}
break;
}
case detail::enqueue_result::success:
// enqueued to a running actors' mailbox; nothing to do
CAF_LOG_DEBUG("success");
break;
}
}
void abstract_broker::enqueue(const actor_addr& sender, message_id mid,
......@@ -62,33 +87,19 @@ void abstract_broker::enqueue(const actor_addr& sender, message_id mid,
enqueue(mailbox_element::make(sender, mid, std::move(msg)), eu);
}
void abstract_broker::launch(execution_unit*, bool, bool is_hidden) {
void abstract_broker::launch(execution_unit*, bool is_lazy, bool is_hidden) {
// add implicit reference count held by the middleman
ref();
is_registered(! is_hidden);
CAF_PUSH_AID(id());
CAF_LOGF_TRACE("init and launch broker with ID " << id());
// we want to make sure initialization is executed in MM context
do_become(
[=](sys_atom) {
CAF_LOGF_TRACE("ID " << id());
bhvr_stack_.pop_back();
// launch backends now, because user-defined initialization
// might call functions like add_connection
for (auto& kvp : doormen_) {
kvp.second->launch();
}
initialize();
},
true);
enqueue(invalid_actor_addr, invalid_message_id,
make_message(sys_atom::value), nullptr);
if (is_lazy && mailbox().try_block())
return;
backend().post(continuation{this});
}
void abstract_broker::cleanup(uint32_t reason) {
CAF_LOG_TRACE(CAF_ARG(reason));
planned_exit_reason(reason);
on_exit();
close_all();
CAF_ASSERT(doormen_.empty());
CAF_ASSERT(scribes_.empty());
......@@ -203,79 +214,6 @@ optional<accept_handle> abstract_broker::hdl_by_port(uint16_t port) {
return none;
}
void abstract_broker::invoke_message(mailbox_element_ptr& ptr) {
CAF_LOG_TRACE(CAF_TARG(ptr->msg, to_string));
if (exit_reason() != exit_reason::not_exited || ! has_behavior()) {
CAF_LOG_DEBUG("actor already finished execution"
<< ", planned_exit_reason = " << planned_exit_reason()
<< ", has_behavior() = " << has_behavior());
if (ptr->mid.valid()) {
detail::sync_request_bouncer srb{exit_reason()};
srb(ptr->sender, ptr->mid);
}
return;
}
// prepare actor for invocation of message handler
try {
auto& bhvr = this->awaits_response()
? this->awaited_response_handler()
: this->bhvr_stack().back();
auto bid = this->awaited_response_id();
switch (local_actor::invoke_message(ptr, bhvr, bid)) {
case im_success: {
CAF_LOG_DEBUG("handle_message returned hm_msg_handled");
while (has_behavior()
&& planned_exit_reason() == exit_reason::not_exited
&& invoke_message_from_cache()) {
// rinse and repeat
}
break;
}
case im_dropped:
CAF_LOG_DEBUG("handle_message returned hm_drop_msg");
break;
case im_skipped: {
CAF_LOG_DEBUG("handle_message returned hm_skip_msg or hm_cache_msg");
if (ptr) {
cache_.push_second_back(ptr.release());
}
break;
}
}
}
catch (std::exception& e) {
CAF_LOG_INFO("broker killed due to an unhandled exception: "
<< to_verbose_string(e));
// keep compiler happy in non-debug mode
static_cast<void>(e);
quit(exit_reason::unhandled_exception);
}
catch (...) {
CAF_LOG_ERROR("broker killed due to an unknown exception");
quit(exit_reason::unhandled_exception);
}
// safe to actually release behaviors now
bhvr_stack().cleanup();
// cleanup actor if needed
if (planned_exit_reason() != exit_reason::not_exited) {
cleanup(planned_exit_reason());
} else if (! has_behavior()) {
CAF_LOG_DEBUG("no behavior set, quit for normal exit reason");
quit(exit_reason::normal);
cleanup(planned_exit_reason());
}
}
void abstract_broker::invoke_message(const actor_addr& sender,
message_id mid,
message& msg) {
auto ptr = mailbox_element::make(sender, mid, message{});
ptr->msg.swap(msg);
invoke_message(ptr);
if (ptr)
ptr->msg.swap(msg);
}
void abstract_broker::close_all() {
CAF_LOG_TRACE("");
while (! doormen_.empty()) {
......@@ -288,6 +226,20 @@ void abstract_broker::close_all() {
}
}
const char* abstract_broker::name() const {
return "broker";
}
void abstract_broker::init_broker() {
CAF_LOG_TRACE("");
is_initialized(true);
// launch backends now, because user-defined initialization
// might call functions like add_connection
for (auto& kvp : doormen_)
kvp.second->launch();
}
abstract_broker::abstract_broker() : mm_(*middleman::instance()) {
// nop
}
......@@ -300,17 +252,5 @@ network::multiplexer& abstract_broker::backend() {
return mm_.backend();
}
bool abstract_broker::invoke_message_from_cache() {
CAF_LOG_TRACE("");
auto& bhvr = this->awaits_response()
? this->awaited_response_handler()
: this->bhvr_stack().back();
auto bid = this->awaited_response_id();
auto i = cache_.second_begin();
auto e = cache_.second_end();
CAF_LOG_DEBUG(std::distance(i, e) << " elements in cache");
return cache_.invoke(static_cast<local_actor*>(this), i, e, bhvr, bid);
}
} // namespace io
} // namespace caf
......@@ -35,7 +35,8 @@ namespace caf {
namespace io {
void broker::initialize() {
is_initialized(true);
CAF_LOG_TRACE("");
init_broker();
auto bhvr = make_behavior();
CAF_LOG_DEBUG_IF(! bhvr, "make_behavior() did not return a behavior, "
<< "has_behavior() = "
......
......@@ -815,10 +815,9 @@ default_multiplexer::add_tcp_doorman(abstract_broker* self,
void new_connection() override {
CAF_LOG_TRACE("");
auto& dm = acceptor_.backend();
accept_msg().handle
msg().handle
= dm.add_tcp_scribe(parent(), std::move(acceptor_.accepted_socket()));
parent()->invoke_message(invalid_actor_addr, invalid_message_id,
accept_msg_);
invoke_mailbox_element();
}
void stop_reading() override {
CAF_LOG_TRACE("");
......
......@@ -27,9 +27,7 @@ namespace caf {
namespace io {
doorman::doorman(abstract_broker* ptr, accept_handle acc_hdl)
: network::acceptor_manager(ptr),
hdl_(acc_hdl),
accept_msg_(make_message(new_connection_msg{hdl_, connection_handle{}})) {
: doorman_base(ptr, acc_hdl) {
// nop
}
......@@ -37,11 +35,6 @@ doorman::~doorman() {
// nop
}
void doorman::detach_from_parent() {
CAF_LOG_TRACE("hdl = " << hdl().id());
parent()->doormen_.erase(hdl());
}
message doorman::detach_message() {
return make_message(acceptor_closed_msg{hdl()});
}
......
......@@ -44,12 +44,12 @@ void manager::detach(bool invoke_disconnect_message) {
CAF_LOG_TRACE("");
if (! detached()) {
CAF_LOG_DEBUG("disconnect servant from broker");
detach_from_parent();
if (invoke_disconnect_message) {
auto msg = detach_message();
parent_->invoke_message(parent_->address(),invalid_message_id, msg);
auto ptr = mailbox_element::make(invalid_actor_addr, invalid_message_id,
detach_message());
parent_->exec_single_event(ptr);
}
parent_ = nullptr;
detach_from_parent();
}
}
......
......@@ -39,9 +39,10 @@
#include "caf/io/middleman.hpp"
#include "caf/io/basp_broker.hpp"
#include "caf/io/system_messages.hpp"
#include "caf/io/network/interfaces.hpp"
#include "caf/scheduler/abstract_coordinator.hpp"
#include "caf/detail/logging.hpp"
#include "caf/detail/ripemd_160.hpp"
#include "caf/detail/safe_equal.hpp"
......@@ -140,6 +141,10 @@ public:
broker_ = invalid_actor;
}
const char* name() const override {
return "middleman_actor";
}
using put_res = either<ok_atom, uint16_t>::or_else<error_atom, std::string>;
using get_res = delegated<either<ok_atom, node_id, actor_addr,
......@@ -244,6 +249,8 @@ void middleman::add_broker(broker_ptr bptr) {
void middleman::initialize() {
CAF_LOG_TRACE("");
auto sc = detail::singletons::get_scheduling_coordinator();
max_throughput_ = sc->max_throughput();
backend_supervisor_ = backend_->make_supervisor();
if (backend_supervisor_ == nullptr) {
// the only backend that returns a `nullptr` is the `test_multiplexer`
......@@ -281,8 +288,10 @@ void middleman::stop() {
// managers_ will be modified while we are stopping each manager,
// because each manager will call remove(...)
for (auto& kvp : named_brokers_) {
if (kvp.second->exit_reason() == exit_reason::not_exited) {
kvp.second->cleanup(exit_reason::normal);
auto& ptr = kvp.second;
if (ptr->exit_reason() == exit_reason::not_exited) {
ptr->planned_exit_reason(exit_reason::normal);
ptr->finalize();
}
}
});
......
......@@ -21,21 +21,12 @@
#include "caf/detail/logging.hpp"
#include "caf/io/abstract_broker.hpp"
namespace caf {
namespace io {
scribe::scribe(abstract_broker* ptr, connection_handle conn_hdl)
: network::stream_manager(ptr),
hdl_(conn_hdl) {
std::vector<char> tmp;
read_msg_ = make_message(new_data_msg{hdl_, std::move(tmp)});
}
void scribe::detach_from_parent() {
CAF_LOG_TRACE("hdl = " << hdl().id());
parent()->scribes_.erase(hdl());
: scribe_base(ptr, conn_hdl) {
// nop
}
scribe::~scribe() {
......@@ -59,11 +50,13 @@ void scribe::consume(const void*, size_t num_bytes) {
CAF_ASSERT(buf.size() >= num_bytes);
// make sure size is correct, swap into message, and then call client
buf.resize(num_bytes);
read_msg().buf.swap(buf);
parent()->invoke_message(invalid_actor_addr, invalid_message_id, read_msg_);
if (! read_msg_.empty()) {
auto& msg_buf = msg().buf;
msg_buf.swap(buf);
invoke_mailbox_element();
// `mailbox_elem_ptr_ == nullptr` if the broker moved it to the cache
if (mailbox_elem_ptr_) {
// swap buffer back to stream and implicitly flush wr_buf()
read_msg().buf.swap(buf);
msg_buf.swap(buf);
flush();
}
}
......
......@@ -19,6 +19,9 @@
#include "caf/io/network/test_multiplexer.hpp"
#include "caf/io/scribe.hpp"
#include "caf/io/doorman.hpp"
namespace caf {
namespace io {
namespace network {
......@@ -126,9 +129,8 @@ void test_multiplexer::assign_tcp_doorman(abstract_broker* ptr,
auto& mm = mpx_->pending_connects();
auto i = mm.find(hdl());
if (i != mm.end()) {
accept_msg().handle = i->second;
parent()->invoke_message(invalid_actor_addr, invalid_message_id,
accept_msg_);
msg().handle = i->second;
invoke_mailbox_element();
mm.erase(i);
}
}
......
......@@ -480,6 +480,7 @@ void test_remote_actor(const char* path, bool run_remote, bool use_asio) {
CAF_CHECK(serv == serv2);
thread child;
if (run_remote) {
CAF_MESSAGE("start child process");
child = detail::run_sub_unit_test(self,
path,
test::engine::max_runtime(),
......@@ -567,8 +568,6 @@ CAF_TEST(remote_actors) {
);
grp->stop();
} else {
for (int i = 0; i < 100; ++i) spawn([]{});
await_all_actors_done();
test_remote_actor(test::engine::path(), true, use_asio);
}
await_all_actors_done();
......
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