Commit 72af9221 authored by Dominik Charousset's avatar Dominik Charousset

Refactor the BASP broker as a class-based actor

Our new design must make sure that the BASP instance inside the BASP
broker outlives the scheduler. Otherwise, currently running BASP workers
might access released memory during shutdown. Hence, we move the BASP
instance to a member of the actor itself to make sure that object won't
get destroyed when the BASP broker quits but when it gets actually
destroyed instead.
parent 6f1fb436
......@@ -40,114 +40,121 @@
namespace caf {
namespace io {
struct basp_broker_state : proxy_registry::backend, basp::instance::callee {
basp_broker_state(broker* selfptr);
/// A broker implementation for the Binary Actor System Protocol (BASP).
class basp_broker : public broker,
public proxy_registry::backend,
public basp::instance::callee {
public:
// -- member types -----------------------------------------------------------
using super = broker;
using ctx_map = std::unordered_map<connection_handle, basp::endpoint_context>;
using monitored_actor_map = std::unordered_map<actor_addr,
std::unordered_set<node_id>>;
// -- constructors, destructors, and assignment operators --------------------
~basp_broker_state() override;
explicit basp_broker(actor_config& cfg);
~basp_broker() override;
// -- implementation of broker -----------------------------------------------
void on_exit() override;
const char* name() const override;
behavior make_behavior() override;
proxy_registry* proxy_registry_ptr() override;
resume_result resume(execution_unit*, size_t) override;
// -- implementation of proxy_registry::backend ------------------------------
// inherited from proxy_registry::backend
strong_actor_ptr make_proxy(node_id nid, actor_id aid) override;
// inherited from proxy_registry::backend
void set_last_hop(node_id* ptr) override;
// inherited from basp::instance::callee
// -- implementation of basp::instance::callee -------------------------------
void finalize_handshake(const node_id& nid, actor_id aid,
std::set<std::string>& sigs) override;
// inherited from basp::instance::callee
void purge_state(const node_id& nid) override;
// inherited from basp::instance::callee
void proxy_announced(const node_id& nid, actor_id aid) override;
// performs bookkeeping such as managing `spawn_servers`
void learned_new_node(const node_id& nid);
// inherited from basp::instance::callee
void learned_new_node_directly(const node_id& nid,
bool was_indirectly_before) override;
// inherited from basp::instance::callee
void learned_new_node_indirectly(const node_id& nid) override;
// inherited from basp::instance::callee
buffer_type& get_buffer(connection_handle hdl) override;
// inherited from basp::instance::callee
void flush(connection_handle hdl) override;
// inherited from basp::instance::callee
void handle_heartbeat() override;
// inherited from basp::instance::callee
execution_unit* current_execution_unit() override;
// -- utility functions ------------------------------------------------------
/// Performs bookkeeping such as managing `spawn_servers`.
void learned_new_node(const node_id& nid);
/// Sets `this_context` by either creating or accessing state for `hdl`.
void set_context(connection_handle hdl);
/// Cleans up any state for `hdl`.
void cleanup(connection_handle hdl);
void connection_cleanup(connection_handle hdl);
// pointer to ourselves
broker* self;
/// Sends a basp::down_message message to a remote node.
void send_basp_down_message(const node_id& nid, actor_id aid, error err);
// protocol instance of BASP
basp::instance instance;
// Sends basp::down_message to all nodes monitoring the terminated actor.
void handle_down_msg(down_msg&);
using ctx_map = std::unordered_map<connection_handle, basp::endpoint_context>;
// -- disambiguation for functions found in multiple base classes ------------
actor_system& system() {
return super::system();
}
const actor_system_config& config() {
return system().config();
}
// -- member variables -------------------------------------------------------
// keeps context information for all open connections
/// Protocol instance of BASP.
basp::instance instance;
/// Keeps context information for all open connections.
ctx_map ctx;
// points to the current context for callbacks such as `make_proxy`
basp::endpoint_context* this_context = nullptr;
/// points to the current context for callbacks.
basp::endpoint_context* this_context;
// stores handles to spawn servers for other nodes; these servers
// are spawned whenever the broker learns a new node ID and try to
// get a 'SpawnServ' instance on the remote side
/// Stores handles to spawn servers for other nodes. These servers are
/// spawned whenever the broker learns a new node ID and tries to get a
/// 'SpawnServ' instance on the remote side.
std::unordered_map<node_id, actor> spawn_servers;
/// Configures whether BASP automatically open new connections to optimize
/// routing paths by forming a mesh between all nodes.
bool automatic_connections = false;
// timeout for delivery of pending messages of endpoints with ordering
const std::chrono::milliseconds pending_to = std::chrono::milliseconds(100);
// returns the node identifier of the underlying BASP instance
/// Returns the node identifier of the underlying BASP instance.
const node_id& this_node() const {
return instance.this_node();
}
using monitored_actor_map =
std::unordered_map<actor_addr, std::unordered_set<node_id>>;
// keeps a list of nodes that monitor a particular local actor
/// Keeps track of nodes that monitor local actors.
monitored_actor_map monitored_actors;
// sends a basp::down_message message to a remote node
void send_basp_down_message(const node_id& nid, actor_id aid, error err);
// sends basp::down_message to all nodes monitoring the terminated
// actor
void handle_down_msg(down_msg&);
static const char* name;
};
/// A broker implementation for the Binary Actor System Protocol (BASP).
class basp_broker : public stateful_actor<basp_broker_state, broker> {
public:
using super = stateful_actor<basp_broker_state, broker>;
explicit basp_broker(actor_config& cfg);
behavior make_behavior() override;
proxy_registry* proxy_registry_ptr() override;
resume_result resume(execution_unit*, size_t) override;
};
} // namespace io
} // namespace caf
......@@ -32,6 +32,7 @@
#include "caf/io/connection_helper.hpp"
#include "caf/io/middleman.hpp"
#include "caf/io/network/interfaces.hpp"
#include "caf/logger.hpp"
#include "caf/make_counted.hpp"
#include "caf/sec.hpp"
#include "caf/send.hpp"
......@@ -54,27 +55,249 @@ THREAD_LOCAL caf::node_id* t_last_hop = nullptr;
namespace caf {
namespace io {
const char* basp_broker_state::name = "basp_broker";
// -- constructors, destructors, and assignment operators ----------------------
/******************************************************************************
* basp_broker_state *
******************************************************************************/
basp_broker_state::basp_broker_state(broker* selfptr)
: basp::instance::callee(selfptr->system(),
basp_broker::basp_broker(actor_config& cfg)
: super(cfg),
basp::instance::callee(super::system(),
static_cast<proxy_registry::backend&>(*this)),
self(selfptr),
instance(selfptr, *this) {
instance(this, *this),
this_context(nullptr) {
CAF_ASSERT(this_node() != none);
}
basp_broker_state::~basp_broker_state() {
// make sure all spawn servers are down
basp_broker::~basp_broker() {
// nop
}
// -- implementation of local_actor/broker -------------------------------------
void basp_broker::on_exit() {
// Release any obsolete state.
ctx.clear();
// Make sure all spawn servers are down before clearing the container.
for (auto& kvp : spawn_servers)
anon_send_exit(kvp.second, exit_reason::kill);
spawn_servers.clear();
}
const char* basp_broker::name() const {
return "basp-broker";
}
behavior basp_broker::make_behavior() {
CAF_LOG_TRACE(CAF_ARG(system().node()));
set_down_handler([](local_actor* ptr, down_msg& x) {
static_cast<basp_broker*>(ptr)->handle_down_msg(x);
});
if (get_or(config(), "middleman.enable-automatic-connections", false)) {
CAF_LOG_DEBUG("enable automatic connections");
// open a random port and store a record for our peers how to
// connect to this broker directly in the configuration server
auto res = add_tcp_doorman(uint16_t{0});
if (res) {
auto port = res->second;
auto addrs = network::interfaces::list_addresses(false);
auto config_server = system().registry().get(atom("ConfigServ"));
send(actor_cast<actor>(config_server), put_atom::value,
"basp.default-connectivity-tcp",
make_message(port, std::move(addrs)));
}
automatic_connections = true;
}
auto heartbeat_interval = get_or(config(), "middleman.heartbeat-interval",
defaults::middleman::heartbeat_interval);
if (heartbeat_interval > 0) {
CAF_LOG_DEBUG("enable heartbeat" << CAF_ARG(heartbeat_interval));
send(this, tick_atom::value, heartbeat_interval);
}
return {
// received from underlying broker implementation
[=](new_data_msg& msg) {
CAF_LOG_TRACE(CAF_ARG(msg.handle));
set_context(msg.handle);
auto& ctx = *this_context;
auto next = instance.handle(context(), msg, ctx.hdr,
ctx.cstate == basp::await_payload);
if (next == basp::close_connection) {
connection_cleanup(msg.handle);
close(msg.handle);
return;
}
if (next != ctx.cstate) {
auto rd_size = next == basp::await_payload ? ctx.hdr.payload_len
: basp::header_size;
configure_read(msg.handle, receive_policy::exactly(rd_size));
ctx.cstate = next;
}
},
// received from proxy instances
[=](forward_atom, strong_actor_ptr& src,
const std::vector<strong_actor_ptr>& fwd_stack,
strong_actor_ptr& dest, message_id mid, const message& msg) {
CAF_LOG_TRACE(CAF_ARG(src) << CAF_ARG(dest)
<< CAF_ARG(mid) << CAF_ARG(msg));
if (!dest || system().node() == dest->node()) {
CAF_LOG_WARNING("cannot forward to invalid or local actor:"
<< CAF_ARG(dest));
return;
}
if (src && system().node() == src->node())
system().registry().put(src->id(), src);
if (!instance.dispatch(context(), src, fwd_stack, dest->node(),
dest->id(), 0, mid, msg)
&& mid.is_request()) {
detail::sync_request_bouncer srb{exit_reason::remote_link_unreachable};
srb(src, mid);
}
},
// received from some system calls like whereis
[=](forward_atom, const node_id& dest_node, atom_value dest_name,
const message& msg) -> result<message> {
auto cme = current_mailbox_element();
if (cme == nullptr || cme->sender == nullptr)
return sec::invalid_argument;
CAF_LOG_TRACE(CAF_ARG2("sender", cme->sender)
<< ", " << CAF_ARG(dest_node)
<< ", " << CAF_ARG(dest_name)
<< ", " << CAF_ARG(msg));
auto& sender = cme->sender;
if (system().node() == sender->node())
system().registry().put(sender->id(), sender);
if (!instance.dispatch(context(), sender, cme->stages,
dest_node, static_cast<uint64_t>(dest_name),
basp::header::named_receiver_flag, cme->mid,
msg)) {
detail::sync_request_bouncer srb{exit_reason::remote_link_unreachable};
srb(sender, cme->mid);
}
return delegated<message>();
},
// received from proxy instances to signal that we need to send a BASP
// monitor_message to the origin node
[=](monitor_atom, const strong_actor_ptr& proxy) {
if (proxy == nullptr) {
CAF_LOG_WARNING("received a monitor message from an invalid proxy");
return;
}
auto route = instance.tbl().lookup(proxy->node());
if (route == none) {
CAF_LOG_DEBUG("connection to origin already lost, kill proxy");
instance.proxies().erase(proxy->node(), proxy->id());
return;
}
CAF_LOG_DEBUG("write monitor_message:" << CAF_ARG(proxy));
// tell remote side we are monitoring this actor now
auto hdl = route->hdl;
instance.write_monitor_message(context(), get_buffer(hdl),
proxy->node(), proxy->id());
flush(hdl);
},
// received from underlying broker implementation
[=](const new_connection_msg& msg) {
CAF_LOG_TRACE(CAF_ARG(msg.handle));
auto& bi = instance;
bi.write_server_handshake(context(), get_buffer(msg.handle),
local_port(msg.source));
flush(msg.handle);
configure_read(msg.handle, receive_policy::exactly(basp::header_size));
},
// received from underlying broker implementation
[=](const connection_closed_msg& msg) {
CAF_LOG_TRACE(CAF_ARG(msg.handle));
connection_cleanup(msg.handle);
},
// received from underlying broker implementation
[=](const acceptor_closed_msg& msg) {
CAF_LOG_TRACE("");
auto port = local_port(msg.handle);
instance.remove_published_actor(port);
},
// received from middleman actor
[=](publish_atom, doorman_ptr& ptr, uint16_t port,
const strong_actor_ptr& whom, std::set<std::string>& sigs) {
CAF_LOG_TRACE(CAF_ARG(ptr) << CAF_ARG(port)
<< CAF_ARG(whom) << CAF_ARG(sigs));
CAF_ASSERT(ptr != nullptr);
add_doorman(std::move(ptr));
if (whom)
system().registry().put(whom->id(), whom);
instance.add_published_actor(port, whom, std::move(sigs));
},
// received from middleman actor (delegated)
[=](connect_atom, scribe_ptr& ptr, uint16_t port) {
CAF_LOG_TRACE(CAF_ARG(ptr) << CAF_ARG(port));
CAF_ASSERT(ptr != nullptr);
auto rp = make_response_promise();
auto hdl = ptr->hdl();
add_scribe(std::move(ptr));
auto& ctx = this->ctx[hdl];
ctx.hdl = hdl;
ctx.remote_port = port;
ctx.cstate = basp::await_header;
ctx.callback = rp;
// await server handshake
configure_read(hdl, receive_policy::exactly(basp::header_size));
},
[=](delete_atom, const node_id& nid, actor_id aid) {
CAF_LOG_TRACE(CAF_ARG(nid) << ", " << CAF_ARG(aid));
proxies().erase(nid, aid);
},
[=](unpublish_atom, const actor_addr& whom, uint16_t port) -> result<void> {
CAF_LOG_TRACE(CAF_ARG(whom) << CAF_ARG(port));
auto cb = make_callback(
[&](const strong_actor_ptr&, uint16_t x) -> error {
close(hdl_by_port(x));
return none;
}
);
if (instance.remove_published_actor(whom, port, &cb) == 0)
return sec::no_actor_published_at_port;
return unit;
},
[=](close_atom, uint16_t port) -> result<void> {
if (port == 0)
return sec::cannot_close_invalid_port;
// It is well-defined behavior to not have an actor published here, hence
// the result can be ignored safely.
instance.remove_published_actor(port, nullptr);
auto res = close(hdl_by_port(port));
if (res)
return unit;
return sec::cannot_close_invalid_port;
},
[=](get_atom, const node_id& x)
-> std::tuple<node_id, std::string, uint16_t> {
std::string addr;
uint16_t port = 0;
auto hdl = instance.tbl().lookup_direct(x);
if (hdl) {
addr = remote_addr(*hdl);
port = remote_port(*hdl);
}
return std::make_tuple(x, std::move(addr), port);
},
[=](tick_atom, size_t interval) {
instance.handle_heartbeat(context());
delayed_send(this, std::chrono::milliseconds{interval},
tick_atom::value, interval);
}
};
}
proxy_registry* basp_broker::proxy_registry_ptr() {
return &instance.proxies();
}
resumable::resume_result basp_broker::resume(execution_unit* ctx, size_t mt) {
ctx->proxy_registry_ptr(&instance.proxies());
auto guard = detail::make_scope_guard([=] {
ctx->proxy_registry_ptr(nullptr);
});
return super::resume(ctx, mt);
}
strong_actor_ptr basp_broker_state::make_proxy(node_id nid, actor_id aid) {
strong_actor_ptr basp_broker::make_proxy(node_id nid, actor_id aid) {
CAF_LOG_TRACE(CAF_ARG(nid) << CAF_ARG(aid));
CAF_ASSERT(nid != this_node());
if (nid == none || aid == invalid_actor_id)
......@@ -92,8 +315,8 @@ strong_actor_ptr basp_broker_state::make_proxy(node_id nid, actor_id aid) {
// receive a basp::down_message
actor_config cfg;
auto res = make_actor<forwarding_actor_proxy, strong_actor_ptr>(
aid, nid, &(self->home_system()), cfg, self);
strong_actor_ptr selfptr{self->ctrl()};
aid, nid, &(system()), cfg, this);
strong_actor_ptr selfptr{ctrl()};
res->get()->attach_functor([=](const error& rsn) {
mm->backend().post([=] {
// using res->id() instead of aid keeps this actor instance alive
......@@ -101,17 +324,17 @@ strong_actor_ptr basp_broker_state::make_proxy(node_id nid, actor_id aid) {
// bugs with attachables
auto bptr = static_cast<basp_broker*>(selfptr->get());
if (!bptr->getf(abstract_actor::is_terminated_flag))
bptr->state.proxies().erase(nid, res->id(), rsn);
bptr->proxies().erase(nid, res->id(), rsn);
});
});
return res;
}
void basp_broker_state::set_last_hop(node_id* ptr) {
void basp_broker::set_last_hop(node_id* ptr) {
t_last_hop = ptr;
}
void basp_broker_state::finalize_handshake(const node_id& nid, actor_id aid,
void basp_broker::finalize_handshake(const node_id& nid, actor_id aid,
std::set<std::string>& sigs) {
CAF_LOG_TRACE(CAF_ARG(nid) << CAF_ARG(aid) << CAF_ARG(sigs));
CAF_ASSERT(this_context != nullptr);
......@@ -135,7 +358,7 @@ void basp_broker_state::finalize_handshake(const node_id& nid, actor_id aid,
cb = none;
}
void basp_broker_state::purge_state(const node_id& nid) {
void basp_broker::purge_state(const node_id& nid) {
CAF_LOG_TRACE(CAF_ARG(nid));
// Destroy all proxies of the lost node.
namespace_.erase(nid);
......@@ -144,7 +367,7 @@ void basp_broker_state::purge_state(const node_id& nid) {
kvp.second.erase(nid);
}
void basp_broker_state::send_basp_down_message(const node_id& nid, actor_id aid,
void basp_broker::send_basp_down_message(const node_id& nid, actor_id aid,
error rsn) {
CAF_LOG_TRACE(CAF_ARG(nid) << CAF_ARG(aid) << CAF_ARG(rsn));
auto path = instance.tbl().lookup(nid);
......@@ -153,12 +376,11 @@ void basp_broker_state::send_basp_down_message(const node_id& nid, actor_id aid,
<< CAF_ARG(nid));
return;
}
instance.write_down_message(self->context(), get_buffer(path->hdl), nid, aid,
rsn);
instance.write_down_message(context(), get_buffer(path->hdl), nid, aid, rsn);
instance.flush(*path);
}
void basp_broker_state::proxy_announced(const node_id& nid, actor_id aid) {
void basp_broker::proxy_announced(const node_id& nid, actor_id aid) {
CAF_LOG_TRACE(CAF_ARG(nid) << CAF_ARG(aid));
// source node has created a proxy for one of our actors
auto ptr = system().registry().get(aid);
......@@ -170,7 +392,7 @@ void basp_broker_state::proxy_announced(const node_id& nid, actor_id aid) {
auto entry = ptr->address();
auto i = monitored_actors.find(entry);
if (i == monitored_actors.end()) {
self->monitor(ptr);
monitor(ptr);
std::unordered_set<node_id> tmp{nid};
monitored_actors.emplace(entry, std::move(tmp));
} else {
......@@ -179,7 +401,7 @@ void basp_broker_state::proxy_announced(const node_id& nid, actor_id aid) {
}
}
void basp_broker_state::handle_down_msg(down_msg& dm) {
void basp_broker::handle_down_msg(down_msg& dm) {
auto i = monitored_actors.find(dm.source);
if (i == monitored_actors.end())
return;
......@@ -188,7 +410,7 @@ void basp_broker_state::handle_down_msg(down_msg& dm) {
monitored_actors.erase(i);
}
void basp_broker_state::learned_new_node(const node_id& nid) {
void basp_broker::learned_new_node(const node_id& nid) {
CAF_LOG_TRACE(CAF_ARG(nid));
if (spawn_servers.count(nid) > 0) {
CAF_LOG_ERROR("learned_new_node called for known node " << CAF_ARG(nid));
......@@ -234,7 +456,7 @@ void basp_broker_state::learned_new_node(const node_id& nid) {
auto tmp_ptr = actor_cast<strong_actor_ptr>(tmp);
system().registry().put(tmp.id(), tmp_ptr);
std::vector<strong_actor_ptr> stages;
if (!instance.dispatch(self->context(), tmp_ptr, stages, nid,
if (!instance.dispatch(context(), tmp_ptr, stages, nid,
static_cast<uint64_t>(atom("SpawnServ")),
basp::header::named_receiver_flag, make_message_id(),
make_message(sys_atom::value, get_atom::value,
......@@ -244,14 +466,14 @@ void basp_broker_state::learned_new_node(const node_id& nid) {
}
}
void basp_broker_state::learned_new_node_directly(const node_id& nid,
void basp_broker::learned_new_node_directly(const node_id& nid,
bool was_indirectly_before) {
CAF_LOG_TRACE(CAF_ARG(nid));
if (!was_indirectly_before)
learned_new_node(nid);
}
void basp_broker_state::learned_new_node_indirectly(const node_id& nid) {
void basp_broker::learned_new_node_indirectly(const node_id& nid) {
CAF_LOG_TRACE(CAF_ARG(nid));
learned_new_node(nid);
if (!automatic_connections)
......@@ -261,12 +483,12 @@ void basp_broker_state::learned_new_node_indirectly(const node_id& nid) {
// once and there is no need to track in-flight connection requests.
using namespace detail;
auto tmp = get_or(config(), "middleman.attach-utility-actors", false)
? system().spawn<hidden>(connection_helper, self)
: system().spawn<detached + hidden>(connection_helper, self);
? system().spawn<hidden>(connection_helper, this)
: system().spawn<detached + hidden>(connection_helper, this);
auto sender = actor_cast<strong_actor_ptr>(tmp);
system().registry().put(sender->id(), sender);
std::vector<strong_actor_ptr> fwd_stack;
if (!instance.dispatch(self->context(), sender, fwd_stack, nid,
if (!instance.dispatch(context(), sender, fwd_stack, nid,
static_cast<uint64_t>(atom("ConfigServ")),
basp::header::named_receiver_flag, make_message_id(),
make_message(get_atom::value,
......@@ -275,7 +497,7 @@ void basp_broker_state::learned_new_node_indirectly(const node_id& nid) {
}
}
void basp_broker_state::set_context(connection_handle hdl) {
void basp_broker::set_context(connection_handle hdl) {
CAF_LOG_TRACE(CAF_ARG(hdl));
auto i = ctx.find(hdl);
if (i == ctx.end()) {
......@@ -291,7 +513,7 @@ void basp_broker_state::set_context(connection_handle hdl) {
t_last_hop = &i->second.id;
}
void basp_broker_state::cleanup(connection_handle hdl) {
void basp_broker::connection_cleanup(connection_handle hdl) {
CAF_LOG_TRACE(CAF_ARG(hdl));
// Remove handle from the routing table and clean up any node-specific state
// we might still have.
......@@ -311,242 +533,20 @@ void basp_broker_state::cleanup(connection_handle hdl) {
}
}
basp_broker_state::buffer_type&
basp_broker_state::get_buffer(connection_handle hdl) {
return self->wr_buf(hdl);
basp_broker::buffer_type& basp_broker::get_buffer(connection_handle hdl) {
return wr_buf(hdl);
}
void basp_broker_state::flush(connection_handle hdl) {
self->flush(hdl);
void basp_broker::flush(connection_handle hdl) {
super::flush(hdl);
}
void basp_broker_state::handle_heartbeat() {
void basp_broker::handle_heartbeat() {
// nop
}
execution_unit* basp_broker_state::current_execution_unit() {
return self->context();
}
/******************************************************************************
* basp_broker *
******************************************************************************/
basp_broker::basp_broker(actor_config& cfg)
: stateful_actor<basp_broker_state, broker>(cfg) {
set_down_handler([](local_actor* ptr, down_msg& x) {
static_cast<basp_broker*>(ptr)->state.handle_down_msg(x);
});
}
behavior basp_broker::make_behavior() {
CAF_LOG_TRACE(CAF_ARG(system().node()));
if (get_or(config(), "middleman.enable-automatic-connections", false)) {
CAF_LOG_DEBUG("enable automatic connections");
// open a random port and store a record for our peers how to
// connect to this broker directly in the configuration server
auto res = add_tcp_doorman(uint16_t{0});
if (res) {
auto port = res->second;
auto addrs = network::interfaces::list_addresses(false);
auto config_server = system().registry().get(atom("ConfigServ"));
send(actor_cast<actor>(config_server), put_atom::value,
"basp.default-connectivity-tcp",
make_message(port, std::move(addrs)));
}
state.automatic_connections = true;
}
auto heartbeat_interval = get_or(config(), "middleman.heartbeat-interval",
defaults::middleman::heartbeat_interval);
if (heartbeat_interval > 0) {
CAF_LOG_DEBUG("enable heartbeat" << CAF_ARG(heartbeat_interval));
send(this, tick_atom::value, heartbeat_interval);
}
return {
// received from underlying broker implementation
[=](new_data_msg& msg) {
CAF_LOG_TRACE(CAF_ARG(msg.handle));
state.set_context(msg.handle);
auto& ctx = *state.this_context;
auto next = state.instance.handle(context(), msg, ctx.hdr,
ctx.cstate == basp::await_payload);
if (next == basp::close_connection) {
state.cleanup(msg.handle);
close(msg.handle);
return;
}
if (next != ctx.cstate) {
auto rd_size = next == basp::await_payload
? ctx.hdr.payload_len
: basp::header_size;
configure_read(msg.handle, receive_policy::exactly(rd_size));
ctx.cstate = next;
}
},
// received from proxy instances
[=](forward_atom, strong_actor_ptr& src,
const std::vector<strong_actor_ptr>& fwd_stack,
strong_actor_ptr& dest, message_id mid, const message& msg) {
CAF_LOG_TRACE(CAF_ARG(src) << CAF_ARG(dest)
<< CAF_ARG(mid) << CAF_ARG(msg));
if (!dest || system().node() == dest->node()) {
CAF_LOG_WARNING("cannot forward to invalid or local actor:"
<< CAF_ARG(dest));
return;
}
if (src && system().node() == src->node())
system().registry().put(src->id(), src);
if (!state.instance.dispatch(context(), src, fwd_stack, dest->node(),
dest->id(), 0, mid, msg)
&& mid.is_request()) {
detail::sync_request_bouncer srb{exit_reason::remote_link_unreachable};
srb(src, mid);
}
},
// received from some system calls like whereis
[=](forward_atom, const node_id& dest_node, atom_value dest_name,
const message& msg) -> result<message> {
auto cme = current_mailbox_element();
if (cme == nullptr || cme->sender == nullptr)
return sec::invalid_argument;
CAF_LOG_TRACE(CAF_ARG2("sender", cme->sender)
<< ", " << CAF_ARG(dest_node)
<< ", " << CAF_ARG(dest_name)
<< ", " << CAF_ARG(msg));
auto& sender = cme->sender;
if (system().node() == sender->node())
system().registry().put(sender->id(), sender);
if (!state.instance.dispatch(context(), sender, cme->stages,
dest_node, static_cast<uint64_t>(dest_name),
basp::header::named_receiver_flag, cme->mid,
msg)) {
detail::sync_request_bouncer srb{exit_reason::remote_link_unreachable};
srb(sender, cme->mid);
}
return delegated<message>();
},
// received from proxy instances to signal that we need to send a BASP
// monitor_message to the origin node
[=](monitor_atom, const strong_actor_ptr& proxy) {
if (proxy == nullptr) {
CAF_LOG_WARNING("received a monitor message from an invalid proxy");
return;
}
auto route = state.instance.tbl().lookup(proxy->node());
if (route == none) {
CAF_LOG_DEBUG("connection to origin already lost, kill proxy");
state.instance.proxies().erase(proxy->node(), proxy->id());
return;
}
CAF_LOG_DEBUG("write monitor_message:" << CAF_ARG(proxy));
// tell remote side we are monitoring this actor now
auto hdl = route->hdl;
state.instance.write_monitor_message(context(), state.get_buffer(hdl),
proxy->node(), proxy->id());
flush(hdl);
},
// received from underlying broker implementation
[=](const new_connection_msg& msg) {
CAF_LOG_TRACE(CAF_ARG(msg.handle));
auto& bi = state.instance;
bi.write_server_handshake(context(), state.get_buffer(msg.handle),
local_port(msg.source));
state.flush(msg.handle);
configure_read(msg.handle, receive_policy::exactly(basp::header_size));
},
// received from underlying broker implementation
[=](const connection_closed_msg& msg) {
CAF_LOG_TRACE(CAF_ARG(msg.handle));
state.cleanup(msg.handle);
},
// received from underlying broker implementation
[=](const acceptor_closed_msg& msg) {
CAF_LOG_TRACE("");
auto port = local_port(msg.handle);
state.instance.remove_published_actor(port);
},
// received from middleman actor
[=](publish_atom, doorman_ptr& ptr, uint16_t port,
const strong_actor_ptr& whom, std::set<std::string>& sigs) {
CAF_LOG_TRACE(CAF_ARG(ptr) << CAF_ARG(port)
<< CAF_ARG(whom) << CAF_ARG(sigs));
CAF_ASSERT(ptr != nullptr);
add_doorman(std::move(ptr));
if (whom)
system().registry().put(whom->id(), whom);
state.instance.add_published_actor(port, whom, std::move(sigs));
},
// received from middleman actor (delegated)
[=](connect_atom, scribe_ptr& ptr, uint16_t port) {
CAF_LOG_TRACE(CAF_ARG(ptr) << CAF_ARG(port));
CAF_ASSERT(ptr != nullptr);
auto rp = make_response_promise();
auto hdl = ptr->hdl();
add_scribe(std::move(ptr));
auto& ctx = state.ctx[hdl];
ctx.hdl = hdl;
ctx.remote_port = port;
ctx.cstate = basp::await_header;
ctx.callback = rp;
// await server handshake
configure_read(hdl, receive_policy::exactly(basp::header_size));
},
[=](delete_atom, const node_id& nid, actor_id aid) {
CAF_LOG_TRACE(CAF_ARG(nid) << ", " << CAF_ARG(aid));
state.proxies().erase(nid, aid);
},
[=](unpublish_atom, const actor_addr& whom, uint16_t port) -> result<void> {
CAF_LOG_TRACE(CAF_ARG(whom) << CAF_ARG(port));
auto cb = make_callback(
[&](const strong_actor_ptr&, uint16_t x) -> error {
close(hdl_by_port(x));
return none;
}
);
if (state.instance.remove_published_actor(whom, port, &cb) == 0)
return sec::no_actor_published_at_port;
return unit;
},
[=](close_atom, uint16_t port) -> result<void> {
if (port == 0)
return sec::cannot_close_invalid_port;
// it is well-defined behavior to not have an actor published here,
// hence the result can be ignored safely
state.instance.remove_published_actor(port, nullptr);
auto res = close(hdl_by_port(port));
if (res)
return unit;
return sec::cannot_close_invalid_port;
},
[=](get_atom, const node_id& x)
-> std::tuple<node_id, std::string, uint16_t> {
std::string addr;
uint16_t port = 0;
auto hdl = state.instance.tbl().lookup_direct(x);
if (hdl) {
addr = remote_addr(*hdl);
port = remote_port(*hdl);
}
return std::make_tuple(x, std::move(addr), port);
},
[=](tick_atom, size_t interval) {
state.instance.handle_heartbeat(context());
delayed_send(this, std::chrono::milliseconds{interval},
tick_atom::value, interval);
}
};
}
resumable::resume_result basp_broker::resume(execution_unit* ctx, size_t mt) {
ctx->proxy_registry_ptr(&state.instance.proxies());
auto guard = detail::make_scope_guard([=] {
ctx->proxy_registry_ptr(nullptr);
});
return super::resume(ctx, mt);
}
proxy_registry* basp_broker::proxy_registry_ptr() {
return &state.instance.proxies();
execution_unit* basp_broker::current_execution_unit() {
return context();
}
} // namespace io
......
......@@ -200,17 +200,17 @@ public:
// implementation of the Binary Actor System Protocol
basp::instance& instance() {
return aut()->state.instance;
return aut()->instance;
}
// our routing table (filled by BASP)
basp::routing_table& tbl() {
return aut()->state.instance.tbl();
return aut()->instance.tbl();
}
// access to proxy instances
proxy_registry& proxies() {
return aut()->state.proxies();
return aut()->proxies();
}
// stores the singleton pointer for convenience
......
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