Unverified Commit f5bfab26 authored by Dominik Charousset's avatar Dominik Charousset Committed by GitHub

Revert "Parallelize deserialization of incoming BASP messages"

parent c95230a4
......@@ -56,9 +56,6 @@ manual-multiplexing=false
disable-tcp=false
; enable communication via UDP
enable-udp=false
; configures how many background workers are spawned for deserialization,
; by default CAF uses 1-4 workers depending on the number of cores
workers=<min(3, number of cores / 4) + 1>
; when compiling with logging enabled
[logger]
......
......@@ -112,13 +112,6 @@ public:
/// @cond PRIVATE
/// Called by the testing DSL to peek at the next element in the mailbox. Do
/// not call this function in production code! The default implementation
/// always returns `nullptr`.
/// @returns A pointer to the next mailbox element or `nullptr` if the
/// mailbox is empty or the actor does not have a mailbox.
virtual mailbox_element* peek_at_next_mailbox_element();
template <class... Ts>
void eq_impl(message_id mid, strong_actor_ptr sender,
execution_unit* ctx, Ts&&... xs) {
......
......@@ -149,12 +149,6 @@ using link_atom = atom_constant<atom("link")>;
/// Used for removing networked links.
using unlink_atom = atom_constant<atom("unlink")>;
/// Used for monitor requests over network.
using monitor_atom = atom_constant<atom("monitor")>;
/// Used for removing networked monitors.
using demonitor_atom = atom_constant<atom("demonitor")>;
/// Used for publishing actors at a given port.
using publish_atom = atom_constant<atom("publish")>;
......
......@@ -244,8 +244,6 @@ public:
void enqueue(mailbox_element_ptr, execution_unit*) override;
mailbox_element* peek_at_next_mailbox_element() override;
// -- overridden functions of local_actor ------------------------------------
const char* name() const override;
......
......@@ -82,7 +82,6 @@ extern const size_t max_consecutive_reads;
extern const size_t heartbeat_interval;
extern const size_t cached_udp_buffers;
extern const size_t max_pending_msgs;
extern const size_t workers;
} // namespace middleman
......
......@@ -266,11 +266,6 @@ public:
return *vals_;
}
/// Serializes the content of `x` as if `x` was an instance of `message`. The
/// resulting output of `sink` can then be used to deserialize a `message`
/// even if the serialized object had a different type.
static error save(serializer& sink, const type_erased_tuple& x);
/// @endcond
private:
......
......@@ -18,7 +18,6 @@
#pragma once
#include <mutex>
#include <utility>
#include <functional>
#include <unordered_map>
......@@ -44,8 +43,7 @@ public:
/// Creates a new proxy instance.
virtual strong_actor_ptr make_proxy(node_id, actor_id) = 0;
/// Sets the thread-local last-hop pointer to detect indirect connections.
virtual void set_last_hop(node_id* ptr) = 0;
virtual execution_unit* registry_context() = 0;
};
proxy_registry(actor_system& sys, backend& be);
......@@ -71,17 +69,17 @@ public:
using proxy_map = std::map<actor_id, strong_actor_ptr>;
/// Returns the number of proxies for `node`.
size_t count_proxies(const node_id& node) const;
size_t count_proxies(const node_id& node);
/// Returns the proxy instance identified by `node` and `aid`.
strong_actor_ptr get(const node_id& node, actor_id aid) const;
strong_actor_ptr get(const node_id& node, actor_id aid);
/// Returns the proxy instance identified by `node` and `aid`
/// or creates a new (default) proxy instance.
strong_actor_ptr get_or_put(const node_id& nid, actor_id aid);
/// Returns all known proxies.
std::vector<strong_actor_ptr> get_all(const node_id& node) const;
std::vector<strong_actor_ptr> get_all(const node_id& node);
/// Deletes all proxies for `node`.
void erase(const node_id& nid);
......@@ -97,28 +95,26 @@ public:
void clear();
/// Returns the hosting actor system.
actor_system& system() {
inline actor_system& system() {
return system_;
}
/// Returns the hosting actor system.
const actor_system& system() const {
inline const actor_system& system() const {
return system_;
}
/// Sets the thread-local last hop variable on the backend.
void set_last_hop(node_id* ptr) {
backend_.set_last_hop(ptr);
inline size_t size() const {
return proxies_.size();
}
private:
/// @pre mtx_ is locked
void kill_proxy(strong_actor_ptr&, error);
actor_system& system_;
backend& backend_;
mutable std::mutex mtx_;
std::unordered_map<node_id, proxy_map> proxies_;
};
} // namespace caf
......@@ -262,8 +262,6 @@ public:
void enqueue(mailbox_element_ptr ptr, execution_unit* eu) override;
mailbox_element* peek_at_next_mailbox_element() override;
// -- overridden functions of local_actor ------------------------------------
const char* name() const override;
......
......@@ -90,10 +90,6 @@ actor_system& abstract_actor::home_system() const noexcept {
return *(actor_control_block::from(this)->home_system);
}
mailbox_element* abstract_actor::peek_at_next_mailbox_element() {
return nullptr;
}
void abstract_actor::register_at_system() {
if (getf(is_registered_flag))
return;
......
......@@ -126,8 +126,7 @@ actor_system_config::actor_system_config()
.add<size_t>("max-pending-messages",
"maximum for reordering of UDP receive buffers (default: 10)")
.add<bool>("disable-tcp", "disables communication via TCP")
.add<bool>("enable-udp", "enable communication via UDP")
.add<size_t>("workers", "number of deserialization workers");
.add<bool>("enable-udp", "enable communication via UDP");
opt_group(custom_options_, "opencl")
.add<std::vector<size_t>>("device-ids", "whitelist for OpenCL devices");
opt_group(custom_options_, "openssl")
......
......@@ -79,10 +79,6 @@ void blocking_actor::enqueue(mailbox_element_ptr ptr, execution_unit*) {
}
}
mailbox_element* blocking_actor::peek_at_next_mailbox_element() {
return mailbox().closed() || mailbox().blocked() ? nullptr : mailbox().peek();
}
const char* blocking_actor::name() const {
return "blocking_actor";
}
......
......@@ -23,9 +23,6 @@
#include <limits>
#include <thread>
using std::max;
using std::min;
namespace {
using us_t = std::chrono::microseconds;
......@@ -57,7 +54,7 @@ namespace scheduler {
const atom_value policy = atom("stealing");
string_view profiling_output_file = "";
const size_t max_threads = max(std::thread::hardware_concurrency(), 4u);
const size_t max_threads = std::max(std::thread::hardware_concurrency(), 4u);
const size_t max_throughput = std::numeric_limits<size_t>::max();
const timespan profiling_resolution = ms(100);
......@@ -95,7 +92,6 @@ const size_t max_consecutive_reads = 50;
const size_t heartbeat_interval = 0;
const size_t cached_udp_buffers = 10;
const size_t max_pending_msgs = 10;
const size_t workers = min(3u, std::thread::hardware_concurrency() / 4u) + 1;
} // namespace middleman
......
......@@ -30,7 +30,7 @@ namespace caf {
forwarding_actor_proxy::forwarding_actor_proxy(actor_config& cfg, actor dest)
: actor_proxy(cfg),
broker_(std::move(dest)) {
anon_send(broker_, monitor_atom::value, ctrl());
// nop
}
forwarding_actor_proxy::~forwarding_actor_proxy() {
......@@ -82,6 +82,7 @@ bool forwarding_actor_proxy::remove_backlink(abstract_actor* x) {
}
void forwarding_actor_proxy::kill_proxy(execution_unit* ctx, error rsn) {
CAF_ASSERT(ctx != nullptr);
actor tmp;
{ // lifetime scope of guard
std::unique_lock<detail::shared_spinlock> guard(broker_mtx_);
......
......@@ -156,19 +156,19 @@ error message::load(deserializer& source) {
return none;
}
error message::save(serializer& sink, const type_erased_tuple& x) {
error message::save(serializer& sink) const {
if (sink.context() == nullptr)
return sec::no_context;
// build type name
uint16_t zero = 0;
std::string tname = "@<>";
if (x.empty())
if (empty())
return error::eval([&] { return sink.begin_object(zero, tname); },
[&] { return sink.end_object(); });
auto& types = sink.context()->system().types();
auto n = x.size();
auto n = size();
for (size_t i = 0; i < n; ++i) {
auto rtti = x.type(i);
auto rtti = cvals()->type(i);
auto ptr = types.portable_name(rtti);
if (ptr == nullptr) {
std::cerr << "[ERROR]: cannot serialize message because a type was "
......@@ -185,7 +185,7 @@ error message::save(serializer& sink, const type_erased_tuple& x) {
}
auto save_loop = [&]() -> error {
for (size_t i = 0; i < n; ++i) {
auto e = x.save(i, sink);
auto e = cvals()->save(i, sink);
if (e)
return e;
}
......@@ -196,10 +196,6 @@ error message::save(serializer& sink, const type_erased_tuple& x) {
[&] { return sink.end_object(); });
}
error message::save(serializer& sink) const {
return save(sink, *this);
}
// -- factories ----------------------------------------------------------------
message message::copy(const type_erased_tuple& xs) {
......
......@@ -45,51 +45,42 @@ proxy_registry::~proxy_registry() {
clear();
}
size_t proxy_registry::count_proxies(const node_id& node) const {
std::unique_lock<std::mutex> guard {mtx_};
size_t proxy_registry::count_proxies(const node_id& node) {
auto i = proxies_.find(node);
return i != proxies_.end() ? i->second.size() : 0;
return (i != proxies_.end()) ? i->second.size() : 0;
}
strong_actor_ptr proxy_registry::get(const node_id& node, actor_id aid) const {
std::unique_lock<std::mutex> guard{mtx_};
auto i = proxies_.find(node);
if (i == proxies_.end())
return nullptr;
auto j = i->second.find(aid);
return j != i->second.end() ? j->second : nullptr;
strong_actor_ptr proxy_registry::get(const node_id& node, actor_id aid) {
auto& submap = proxies_[node];
auto i = submap.find(aid);
if (i != submap.end())
return i->second;
return nullptr;
}
strong_actor_ptr proxy_registry::get_or_put(const node_id& nid, actor_id aid) {
CAF_LOG_TRACE(CAF_ARG(nid) << CAF_ARG(aid));
std::unique_lock<std::mutex> guard{mtx_};
auto& result = proxies_[nid][aid];
if (!result)
result = backend_.make_proxy(nid, aid);
return result;
}
std::vector<strong_actor_ptr>
proxy_registry::get_all(const node_id& node) const {
// Reserve at least some memory outside of the critical section.
std::vector<strong_actor_ptr> proxy_registry::get_all(const node_id& node) {
std::vector<strong_actor_ptr> result;
result.reserve(128);
std::unique_lock<std::mutex> guard{mtx_};
auto i = proxies_.find(node);
if (i != proxies_.end())
for (auto& kvp : i->second)
result.emplace_back(kvp.second);
result.push_back(kvp.second);
return result;
}
bool proxy_registry::empty() const {
std::unique_lock<std::mutex> guard{mtx_};
return proxies_.empty();
}
void proxy_registry::erase(const node_id& nid) {
CAF_LOG_TRACE(CAF_ARG(nid));
std::unique_lock<std::mutex> guard{mtx_};
auto i = proxies_.find(nid);
if (i == proxies_.end())
return;
......@@ -100,7 +91,6 @@ void proxy_registry::erase(const node_id& nid) {
void proxy_registry::erase(const node_id& nid, actor_id aid, error rsn) {
CAF_LOG_TRACE(CAF_ARG(nid) << CAF_ARG(aid));
std::unique_lock<std::mutex> guard{mtx_};
auto i = proxies_.find(nid);
if (i != proxies_.end()) {
auto& submap = i->second;
......@@ -115,7 +105,6 @@ void proxy_registry::erase(const node_id& nid, actor_id aid, error rsn) {
}
void proxy_registry::clear() {
std::unique_lock<std::mutex> guard{mtx_};
for (auto& kvp : proxies_)
for (auto& sub_kvp : kvp.second)
kill_proxy(sub_kvp.second, exit_reason::remote_link_unreachable);
......@@ -126,7 +115,7 @@ void proxy_registry::kill_proxy(strong_actor_ptr& ptr, error rsn) {
if (!ptr)
return;
auto pptr = static_cast<actor_proxy*>(actor_cast<abstract_actor*>(ptr));
pptr->kill_proxy(nullptr, std::move(rsn));
pptr->kill_proxy(backend_.registry_context(), std::move(rsn));
}
} // namespace caf
......@@ -187,9 +187,6 @@ void scheduled_actor::enqueue(mailbox_element_ptr ptr, execution_unit* eu) {
break;
}
}
mailbox_element* scheduled_actor::peek_at_next_mailbox_element() {
return mailbox().closed() || mailbox().blocked() ? nullptr : mailbox().peek();
}
// -- overridden functions of local_actor --------------------------------------
......
......@@ -7,47 +7,45 @@ file(GLOB_RECURSE LIBCAF_IO_HDRS "caf/*.hpp")
# list cpp files excluding platform-dependent files
set(LIBCAF_IO_SRCS
src/abstract_broker.cpp
src/acceptor.cpp
src/acceptor_manager.cpp
src/acceptor_manager.cpp
src/basp_broker.cpp
src/broker.cpp
src/connection_helper.cpp
src/datagram_handler.cpp
src/datagram_manager.cpp
src/datagram_servant.cpp
src/datagram_servant_impl.cpp
src/default_multiplexer.cpp
src/doorman.cpp
src/doorman_impl.cpp
src/event_handler.cpp
src/header.cpp
src/hook.cpp
src/instance.cpp
src/interfaces.cpp
src/ip_endpoint.cpp
src/manager.cpp
src/message_queue.cpp
src/message_type.cpp
src/middleman.cpp
src/middleman_actor.cpp
src/middleman_actor_impl.cpp
src/multiplexer.cpp
src/multiplexer.cpp
src/native_socket.cpp
src/pipe_reader.cpp
src/protocol.cpp
src/receive_buffer.cpp
src/routing_table.cpp
src/scribe.cpp
src/stream_manager.cpp
src/test_multiplexer.cpp
src/acceptor.cpp
src/datagram_handler.cpp
src/datagram_servant_impl.cpp
src/doorman_impl.cpp
src/event_handler.cpp
src/pipe_reader.cpp
src/scribe_impl.cpp
src/socket_guard.cpp
src/stream.cpp
src/stream_manager.cpp
src/tcp.cpp
src/test_multiplexer.cpp
src/udp.cpp
src/worker.cpp
src/worker_hub.cpp
src/native_socket.cpp
src/socket_guard.cpp
)
add_custom_target(libcaf_io)
......
/******************************************************************************
* ____ _ _____ *
* / ___| / \ | ___| C++ *
* | | / _ \ | |_ Actor *
* | |___ / ___ \| _| Framework *
* \____/_/ \_|_| *
* *
* Copyright 2011-2019 Dominik Charousset *
* *
* Distributed under the terms and conditions of the BSD 3-Clause License or *
* (at your option) under the terms and conditions of the Boost Software *
* License 1.0. See accompanying files LICENSE and LICENSE_ALTERNATIVE. *
* *
* If you did not receive a copy of the license files, see *
* http://opensource.org/licenses/BSD-3-Clause and *
* http://www.boost.org/LICENSE_1_0.txt. *
******************************************************************************/
#pragma once
namespace caf {
namespace io {
namespace basp {
struct header;
class worker;
class worker_hub;
class message_queue;
class instance;
class routing_table;
} // namespace basp
} // namespace io
} // namespace caf
......@@ -20,19 +20,19 @@
#include <limits>
#include "caf/error.hpp"
#include "caf/variant.hpp"
#include "caf/actor_system_config.hpp"
#include "caf/binary_deserializer.hpp"
#include "caf/callback.hpp"
#include "caf/error.hpp"
#include "caf/io/basp/buffer_type.hpp"
#include "caf/io/basp/connection_state.hpp"
#include "caf/io/hook.hpp"
#include "caf/io/middleman.hpp"
#include "caf/io/basp/header.hpp"
#include "caf/io/basp/message_queue.hpp"
#include "caf/io/basp/buffer_type.hpp"
#include "caf/io/basp/message_type.hpp"
#include "caf/io/basp/routing_table.hpp"
#include "caf/io/basp/worker_hub.hpp"
#include "caf/io/middleman.hpp"
#include "caf/variant.hpp"
#include "caf/io/basp/connection_state.hpp"
namespace caf {
namespace io {
......@@ -45,19 +45,13 @@ class instance {
public:
/// Provides a callback-based interface for certain BASP events.
class callee {
public:
// -- member types ---------------------------------------------------------
protected:
using buffer_type = std::vector<char>;
// -- constructors, destructors, and assignment operators ------------------
public:
explicit callee(actor_system& sys, proxy_registry::backend& backend);
virtual ~callee();
// -- pure virtual functions -----------------------------------------------
/// Called if a server handshake was received and
/// the connection to `nid` is established.
virtual void finalize_handshake(const node_id& nid, actor_id aid,
......@@ -74,6 +68,18 @@ public:
/// for one of our local actors.
virtual void proxy_announced(const node_id& nid, actor_id aid) = 0;
/// Called for each `dispatch_message` without `named_receiver_flag`.
virtual void deliver(const node_id& source_node, actor_id source_actor,
actor_id dest_actor, message_id mid,
std::vector<strong_actor_ptr>& forwarding_stack,
message& msg) = 0;
/// Called for each `dispatch_message` with `named_receiver_flag`.
virtual void deliver(const node_id& source_node, actor_id source_actor,
atom_value dest_actor, message_id mid,
std::vector<strong_actor_ptr>& forwarding_stack,
message& msg) = 0;
/// Called whenever BASP learns the ID of a remote node
/// to which it does not have a direct connection.
virtual void learned_new_node_directly(const node_id& nid,
......@@ -86,14 +92,21 @@ public:
/// Called if a heartbeat was received from `nid`
virtual void handle_heartbeat() = 0;
/// Returns the current CAF scheduler context.
virtual execution_unit* current_execution_unit() = 0;
/// Returns the actor namespace associated to this BASP protocol instance.
proxy_registry& proxies() {
return namespace_;
}
/// Returns the hosting actor system.
actor_system& system() {
return namespace_.system();
}
/// Returns the system-wide configuration.
const actor_system_config& config() const {
return namespace_.system().config();
}
/// Returns a reference to the sent buffer.
virtual buffer_type& get_buffer(connection_handle hdl) = 0;
......@@ -205,12 +218,14 @@ public:
return this_node_;
}
actor_system& system() {
return callee_.proxies().system();
/// Invokes the callback(s) associated with given event.
template <hook::event_type Event, typename... Ts>
void notify(Ts&&... xs) {
system().middleman().template notify<Event>(std::forward<Ts>(xs)...);
}
const actor_system_config& config() {
return system().config();
actor_system& system() {
return callee_.system();
}
bool handle(execution_unit* ctx, connection_handle hdl, header& hdr,
......@@ -224,8 +239,6 @@ private:
published_actor_map published_actors_;
node_id this_node_;
callee& callee_;
message_queue queue_;
worker_hub hub_;
};
/// @}
......
/******************************************************************************
* ____ _ _____ *
* / ___| / \ | ___| C++ *
* | | / _ \ | |_ Actor *
* | |___ / ___ \| _| Framework *
* \____/_/ \_|_| *
* *
* Copyright 2011-2019 Dominik Charousset *
* *
* Distributed under the terms and conditions of the BSD 3-Clause License or *
* (at your option) under the terms and conditions of the Boost Software *
* License 1.0. See accompanying files LICENSE and LICENSE_ALTERNATIVE. *
* *
* If you did not receive a copy of the license files, see *
* http://opensource.org/licenses/BSD-3-Clause and *
* http://www.boost.org/LICENSE_1_0.txt. *
******************************************************************************/
#pragma once
#include <cstdint>
#include <mutex>
#include <vector>
#include "caf/actor_control_block.hpp"
#include "caf/fwd.hpp"
#include "caf/mailbox_element.hpp"
namespace caf {
namespace io {
namespace basp {
/// Enforces strict order of message delivery, i.e., deliver messages in the
/// same order as if they were deserialized by a single thread.
class message_queue {
public:
// -- member types -----------------------------------------------------------
/// Request for sending a message to an actor at a later time.
struct actor_msg {
uint64_t id;
strong_actor_ptr receiver;
mailbox_element_ptr content;
};
// -- constructors, destructors, and assignment operators --------------------
message_queue();
// -- mutators ---------------------------------------------------------------
/// Adds a new message to the queue or deliver it immediately if possible.
void push(execution_unit* ctx, uint64_t id, strong_actor_ptr receiver,
mailbox_element_ptr content);
/// Marks given ID as dropped, effectively skipping it without effect.
void drop(execution_unit* ctx, uint64_t id);
/// Returns the next ascending ID.
uint64_t new_id();
// -- member variables -------------------------------------------------------
/// Protects all other properties.
std::mutex lock;
/// The next available ascending ID. The counter is large enough to overflow
/// after roughly 600 years if we dispatch a message every microsecond.
uint64_t next_id;
/// The next ID that we can ship.
uint64_t next_undelivered;
/// Keeps messages in sorted order in case a message other than
/// `next_undelivered` gets ready first.
std::vector<actor_msg> pending;
};
} // namespace basp
} // namespace io
} // namespace caf
/******************************************************************************
* ____ _ _____ *
* / ___| / \ | ___| C++ *
* | | / _ \ | |_ Actor *
* | |___ / ___ \| _| Framework *
* \____/_/ \_|_| *
* *
* Copyright 2011-2019 Dominik Charousset *
* *
* Distributed under the terms and conditions of the BSD 3-Clause License or *
* (at your option) under the terms and conditions of the Boost Software *
* License 1.0. See accompanying files LICENSE and LICENSE_ALTERNATIVE. *
* *
* If you did not receive a copy of the license files, see *
* http://opensource.org/licenses/BSD-3-Clause and *
* http://www.boost.org/LICENSE_1_0.txt. *
******************************************************************************/
#pragma once
#include <vector>
#include "caf/actor_control_block.hpp"
#include "caf/actor_proxy.hpp"
#include "caf/binary_deserializer.hpp"
#include "caf/config.hpp"
#include "caf/detail/scope_guard.hpp"
#include "caf/detail/sync_request_bouncer.hpp"
#include "caf/execution_unit.hpp"
#include "caf/io/basp/header.hpp"
#include "caf/logger.hpp"
#include "caf/message.hpp"
#include "caf/message_id.hpp"
#include "caf/node_id.hpp"
namespace caf {
namespace io {
namespace basp {
template <class Subtype>
class remote_message_handler {
public:
void handle_remote_message(execution_unit* ctx) {
CAF_LOG_TRACE("");
// Local variables.
auto& dref = static_cast<Subtype&>(*this);
auto& sys = *dref.system_;
strong_actor_ptr src;
strong_actor_ptr dst;
std::vector<strong_actor_ptr> stages;
message msg;
auto mid = make_message_id(dref.hdr_.operation_data);
binary_deserializer source{ctx, dref.payload_};
// Make sure to drop the message in case we return abnormally.
auto guard = detail::make_scope_guard(
[&] { dref.queue_->drop(ctx, dref.msg_id_); });
// Registry setup.
dref.proxies_->set_last_hop(&dref.last_hop_);
// Get the local receiver.
if (dref.hdr_.has(basp::header::named_receiver_flag))
dst = sys.registry().get(static_cast<atom_value>(dref.hdr_.dest_actor));
else
dst = sys.registry().get(dref.hdr_.dest_actor);
// Short circuit if we already know there's nothing to do.
if (dst == nullptr && !mid.is_request()) {
CAF_LOG_INFO("drop asynchronous remote message: unknown destination");
return;
}
// Deserialize source and destination node for routed messages.
if (dref.hdr_.operation == basp::message_type::routed_message) {
node_id src_node;
node_id dst_node;
if (auto err = source(src_node, dst_node)) {
CAF_LOG_ERROR("cannot read source and destination of remote message");
return;
}
CAF_ASSERT(dst_node == sys.node());
if (dref.hdr_.source_actor != 0) {
src = src_node == sys.node()
? sys.registry().get(dref.hdr_.source_actor)
: dref.proxies_->get_or_put(src_node, dref.hdr_.source_actor);
}
} else {
CAF_ASSERT(dref.hdr_.operation == basp::message_type::direct_message);
src = dref.proxies_->get_or_put(dref.last_hop_, dref.hdr_.source_actor);
}
// Send errors for dropped requests.
if (dst == nullptr) {
CAF_ASSERT(mid.is_request());
CAF_LOG_INFO("drop remote request: unknown destination");
detail::sync_request_bouncer srb{exit_reason::remote_link_unreachable};
srb(src, mid);
return;
}
// Get the remainder of the message.
if (auto err = source(stages, msg)) {
CAF_LOG_ERROR("cannot read stages and content of remote message");
return;
}
// Intercept link messages. Forwarding actor proxies signalize linking
// by sending link_atom/unlink_atom message with src == dest.
if (msg.type_token() == make_type_token<atom_value, strong_actor_ptr>()) {
const auto& ptr = msg.get_as<strong_actor_ptr>(1);
switch (static_cast<uint64_t>(msg.get_as<atom_value>(0))) {
default:
break;
case link_atom::uint_value(): {
if (ptr != nullptr)
static_cast<actor_proxy*>(ptr->get())->add_link(dst->get());
else
CAF_LOG_WARNING("received link message with invalid target");
return;
}
case unlink_atom::uint_value(): {
if (ptr != nullptr)
static_cast<actor_proxy*>(ptr->get())->remove_link(dst->get());
else
CAF_LOG_DEBUG("received unlink message with invalid target");
return;
}
}
}
// Ship the message.
guard.disable();
dref.queue_->push(ctx, dref.msg_id_, std::move(dst),
make_mailbox_element(std::move(src), mid,
std::move(stages), std::move(msg)));
}
};
} // namespace basp
} // namespace io
} // namespace caf
......@@ -18,11 +18,10 @@
#pragma once
#include <mutex>
#include <unordered_map>
#include <unordered_set>
#include <vector>
#include "caf/callback.hpp"
#include "caf/io/abstract_broker.hpp"
#include "caf/io/basp/buffer_type.hpp"
#include "caf/node_id.hpp"
......@@ -48,6 +47,10 @@ public:
connection_handle hdl;
};
/// Describes a function object for erase operations that
/// is called for each indirectly lost connection.
using erase_callback = callback<const node_id&>;
/// Returns a route to `target` or `none` on error.
optional<route> lookup(const node_id& target);
......@@ -70,27 +73,52 @@ public:
/// Adds a new indirect route to the table.
bool add_indirect(const node_id& hop, const node_id& dest);
/// Removes a direct connection and return the node ID that became
/// unreachable as a result of this operation.
node_id erase_direct(const connection_handle& hdl);
/// Blacklist the route to `dest` via `hop`.
void blacklist(const node_id& hop, const node_id& dest);
/// Removes a direct connection and calls `cb` for any node
/// that became unreachable as a result of this operation,
/// including the node that is assigned as direct path for `hdl`.
void erase_direct(const connection_handle& hdl, erase_callback& cb);
/// Removes any entry for indirect connection to `dest` and returns
/// `true` if `dest` had an indirect route, otherwise `false`.
bool erase_indirect(const node_id& dest);
/// Queries whether `dest` is reachable.
bool reachable(const node_id& dest);
/// Removes all direct and indirect routes to `dest` and calls
/// `cb` for any node that became unreachable as a result of this
/// operation, including `dest`.
/// @returns the number of removed routes (direct and indirect)
size_t erase(const node_id& dest, erase_callback& cb);
/// Returns the parent broker.
abstract_broker* parent() {
inline abstract_broker* parent() {
return parent_;
}
public:
template <class Map, class Fallback>
typename Map::mapped_type
get_opt(const Map& m, const typename Map::key_type& k, Fallback&& x) const {
auto i = m.find(k);
if (i != m.end())
return i->second;
return std::forward<Fallback>(x);
}
using node_id_set = std::unordered_set<node_id>;
using indirect_entries = std::unordered_map<node_id, // dest
node_id_set>; // hop
abstract_broker* parent_;
mutable std::mutex mtx_;
std::unordered_map<connection_handle, node_id> direct_by_hdl_;
std::unordered_map<node_id, connection_handle> direct_by_nid_;
std::unordered_map<node_id, node_id_set> indirect_;
indirect_entries indirect_;
indirect_entries blacklist_;
};
/// @}
......@@ -98,3 +126,4 @@ public:
} // namespace basp
} // namespace io
} // namespace caf
/******************************************************************************
* ____ _ _____ *
* / ___| / \ | ___| C++ *
* | | / _ \ | |_ Actor *
* | |___ / ___ \| _| Framework *
* \____/_/ \_|_| *
* *
* Copyright 2011-2019 Dominik Charousset *
* *
* Distributed under the terms and conditions of the BSD 3-Clause License or *
* (at your option) under the terms and conditions of the Boost Software *
* License 1.0. See accompanying files LICENSE and LICENSE_ALTERNATIVE. *
* *
* If you did not receive a copy of the license files, see *
* http://opensource.org/licenses/BSD-3-Clause and *
* http://www.boost.org/LICENSE_1_0.txt. *
******************************************************************************/
#pragma once
#include <atomic>
#include <cstdint>
#include <vector>
#include "caf/config.hpp"
#include "caf/fwd.hpp"
#include "caf/io/basp/fwd.hpp"
#include "caf/io/basp/header.hpp"
#include "caf/io/basp/remote_message_handler.hpp"
#include "caf/node_id.hpp"
#include "caf/resumable.hpp"
namespace caf {
namespace io {
namespace basp {
/// Deserializes payloads for BASP messages asynchronously.
class worker : public resumable, public remote_message_handler<worker> {
public:
// -- friends ----------------------------------------------------------------
friend worker_hub;
friend remote_message_handler<worker>;
// -- member types -----------------------------------------------------------
using atomic_pointer = std::atomic<worker*>;
using scheduler_type = scheduler::abstract_coordinator;
using buffer_type = std::vector<char>;
// -- constructors, destructors, and assignment operators --------------------
~worker() override;
// -- management -------------------------------------------------------------
void launch(const node_id& last_hop, const basp::header& hdr,
const buffer_type& payload);
// -- implementation of resumable --------------------------------------------
subtype_t subtype() const override;
resume_result resume(execution_unit* ctx, size_t) override;
void intrusive_ptr_add_ref_impl() override;
void intrusive_ptr_release_impl() override;
private:
// -- constructors, destructors, and assignment operators --------------------
/// Only the ::worker_hub has access to the construtor.
worker(worker_hub& hub, message_queue& queue, proxy_registry& proxies);
// -- constants and assertions -----------------------------------------------
/// Stores how many bytes the "first half" of this object requires.
static constexpr size_t pointer_members_size = sizeof(atomic_pointer)
+ sizeof(worker_hub*)
+ sizeof(message_queue*)
+ sizeof(proxy_registry*)
+ sizeof(actor_system*);
static_assert(CAF_CACHE_LINE_SIZE > pointer_members_size,
"invalid cache line size");
// -- member variables -------------------------------------------------------
/// Points to the next worker in the hub.
atomic_pointer next_;
/// Points to our home hub.
worker_hub* hub_;
/// Points to the queue for establishing strict ordering.
message_queue* queue_;
/// Points to our proxy registry / factory.
proxy_registry* proxies_;
/// Points to the parent system.
actor_system* system_;
/// Prevents false sharing when writing to `next`.
char pad_[CAF_CACHE_LINE_SIZE - pointer_members_size];
/// ID for local ordering.
uint64_t msg_id_;
/// Identifies the node that sent us `hdr_` and `payload_`.
node_id last_hop_;
/// The header for the next message. Either a direct_message or a
/// routed_message.
header hdr_;
/// Contains whatever this worker deserializes next.
buffer_type payload_;
};
} // namespace basp
} // namespace io
} // namespace caf
/******************************************************************************
* ____ _ _____ *
* / ___| / \ | ___| C++ *
* | | / _ \ | |_ Actor *
* | |___ / ___ \| _| Framework *
* \____/_/ \_|_| *
* *
* Copyright 2011-2019 Dominik Charousset *
* *
* Distributed under the terms and conditions of the BSD 3-Clause License or *
* (at your option) under the terms and conditions of the Boost Software *
* License 1.0. See accompanying files LICENSE and LICENSE_ALTERNATIVE. *
* *
* If you did not receive a copy of the license files, see *
* http://opensource.org/licenses/BSD-3-Clause and *
* http://www.boost.org/LICENSE_1_0.txt. *
******************************************************************************/
#pragma once
#include <atomic>
#include "caf/fwd.hpp"
#include "caf/io/basp/fwd.hpp"
namespace caf {
namespace io {
namespace basp {
/// A central place where BASP workers return to after finishing a task. A hub
/// supports any number of workers that call `push`, but only a single master
/// that calls `pop`. The hub takes ownership of all workers. Workers register
/// at the hub during construction and get destroyed when the hub gets
/// destroyed.
class worker_hub {
public:
// -- member types -----------------------------------------------------------
using pointer = worker*;
// -- constructors, destructors, and assignment operators --------------------
worker_hub();
~worker_hub();
// -- properties -------------------------------------------------------------
/// Creates a new worker and adds it to the hub.
void push_new_worker(message_queue&, proxy_registry&);
/// Add a worker to the hub.
void push(pointer ptr);
/// Get a worker from the hub.
/// @returns the next available worker (in LIFO order) or `nullptr` if the
/// hub is currently empty.
pointer pop();
/// Check which worker would `pop` currently return.
/// @returns the next available worker (in LIFO order) or `nullptr` if the
/// hub is currently empty.
pointer peek();
private:
// -- member variables -------------------------------------------------------
std::atomic<pointer> head_;
};
} // namespace basp
} // namespace io
} // namespace caf
......@@ -40,121 +40,126 @@
namespace caf {
namespace io {
/// 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>>;
struct basp_broker_state : proxy_registry::backend, basp::instance::callee {
basp_broker_state(broker* selfptr);
// -- constructors, destructors, and assignment operators --------------------
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 ------------------------------
~basp_broker_state() override;
// inherited from proxy_registry::backend
strong_actor_ptr make_proxy(node_id nid, actor_id aid) override;
void set_last_hop(node_id* ptr) override;
// -- implementation of basp::instance::callee -------------------------------
// inherited from proxy_registry::backend
execution_unit* registry_context() override;
// inherited from 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;
// inherited from basp::instance::callee
void deliver(const node_id& src_nid, actor_id src_aid,
actor_id dest_aid, message_id mid,
std::vector<strong_actor_ptr>& stages, message& msg) override;
// inherited from basp::instance::callee
void deliver(const node_id& src_nid, actor_id src_aid,
atom_value dest_name, message_id mid,
std::vector<strong_actor_ptr>& stages, message& msg) override;
// called from both overriden functions
void deliver(const node_id& src_nid, actor_id src_aid,
strong_actor_ptr dest, message_id mid,
std::vector<strong_actor_ptr>& stages, message& msg);
// 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;
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 connection_cleanup(connection_handle hdl);
void cleanup(connection_handle hdl);
/// 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&);
// -- disambiguation for functions found in multiple base classes ------------
actor_system& system() {
return super::system();
}
const actor_system_config& config() {
return system().config();
}
// pointer to ourselves
broker* self;
// -- member variables -------------------------------------------------------
/// Protocol instance of BASP.
// protocol instance of BASP
basp::instance instance;
/// Keeps context information for all open connections.
using ctx_map = std::unordered_map<connection_handle, basp::endpoint_context>;
// keeps context information for all open connections
ctx_map ctx;
/// points to the current context for callbacks.
basp::endpoint_context* this_context;
// points to the current context for callbacks such as `make_proxy`
basp::endpoint_context* this_context = nullptr;
/// 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.
// 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
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;
/// Returns the node identifier of the underlying BASP instance.
// 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
const node_id& this_node() const {
return instance.this_node();
}
/// Keeps track of nodes that monitor local actors.
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
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
/******************************************************************************
* ____ _ _____ *
* / ___| / \ | ___| C++ *
* | | / _ \ | |_ Actor *
* | |___ / ___ \| _| Framework *
* \____/_/ \_|_| *
* *
* Copyright 2011-2018 Dominik Charousset *
* *
* Distributed under the terms and conditions of the BSD 3-Clause License or *
* (at your option) under the terms and conditions of the Boost Software *
* License 1.0. See accompanying files LICENSE and LICENSE_ALTERNATIVE. *
* *
* If you did not receive a copy of the license files, see *
* http://opensource.org/licenses/BSD-3-Clause and *
* http://www.boost.org/LICENSE_1_0.txt. *
******************************************************************************/
#pragma once
#include <set>
#include <string>
#include <memory>
#include <vector>
#include "caf/fwd.hpp"
namespace caf {
namespace io {
class hook;
// safes us some typing for the static dispatching
#define CAF_IO_HOOK_DISPATCH(eventname) \
template <typename... Ts> \
void dispatch(event<eventname>, Ts&&... ts) { \
eventname##_cb(std::forward<Ts>(ts)...); \
}
/// @relates hook
using hook_uptr = std::unique_ptr<hook>;
/// Interface to define hooks into the IO layer.
class hook {
public:
explicit hook(actor_system& sys);
virtual ~hook();
/// Called whenever a message has arrived via the network.
virtual void message_received_cb(const node_id& source,
const strong_actor_ptr& from,
const strong_actor_ptr& dest,
message_id mid,
const message& msg);
/// Called whenever a message has been sent to the network.
/// @param from The address of the sending actor.
/// @param hop The node in the network we've sent the message to.
/// @param dest The address of the receiving actor. Note that the node ID
/// of `dest` can differ from `hop` in case we don't
/// have a direct connection to `dest_node`.
/// @param mid The ID of the message.
/// @param payload The message we've sent.
virtual void message_sent_cb(const strong_actor_ptr& from, const node_id& hop,
const strong_actor_ptr& dest, message_id mid,
const message& payload);
/// Called whenever no route for sending a message exists.
virtual void message_sending_failed_cb(const strong_actor_ptr& from,
const strong_actor_ptr& dest,
message_id mid,
const message& payload);
/// Called whenever a message is forwarded to a different node.
virtual void message_forwarded_cb(const basp::header& hdr,
const std::vector<char>* payload);
/// Called whenever no route for a forwarding request exists.
virtual void message_forwarding_failed_cb(const basp::header& hdr,
const std::vector<char>* payload);
/// Called whenever an actor has been published.
virtual void actor_published_cb(const strong_actor_ptr& addr,
const std::set<std::string>& ifs,
uint16_t port);
/// Called whenever a new remote actor appeared.
virtual void new_remote_actor_cb(const strong_actor_ptr& addr);
/// Called whenever a handshake via a direct TCP connection succeeded.
virtual void new_connection_established_cb(const node_id& node);
/// Called whenever a message from or to a yet unknown node was received.
/// @param via The node that has sent us the message.
/// @param node The newly added entry to the routing table.
virtual void new_route_added_cb(const node_id& via, const node_id& node);
/// Called whenever a direct connection was lost.
virtual void connection_lost_cb(const node_id& dest);
/// Called whenever a route became unavailable.
/// @param hop The node that was either disconnected
/// or lost a connection itself.
/// @param dest The node that is no longer reachable via `hop`.
virtual void route_lost_cb(const node_id& hop, const node_id& dest);
/// Called whenever a message was discarded because a remote node
/// tried to send a message to an actor ID that could not be found
/// in the registry.
virtual void invalid_message_received_cb(const node_id& source,
const strong_actor_ptr& sender,
actor_id invalid_dest,
message_id mid, const message& msg);
/// Called before middleman shuts down.
virtual void before_shutdown_cb();
/// All possible events for IO hooks.
enum event_type {
message_received,
message_sent,
message_forwarded,
message_sending_failed,
message_forwarding_failed,
actor_published,
new_remote_actor,
new_connection_established,
new_route_added,
connection_lost,
route_lost,
invalid_message_received,
before_shutdown
};
/// Handles an event by invoking the associated callback.
template <event_type Event, typename... Ts>
void handle(Ts&&... ts) {
dispatch(event<Event>{}, std::forward<Ts>(ts)...);
}
inline actor_system& system() const {
return system_;
}
private:
// ------------ convenience interface based on static dispatching ------------
template <event_type Id>
using event = std::integral_constant<event_type, Id>;
CAF_IO_HOOK_DISPATCH(message_received)
CAF_IO_HOOK_DISPATCH(message_sent)
CAF_IO_HOOK_DISPATCH(message_forwarded)
CAF_IO_HOOK_DISPATCH(message_sending_failed)
CAF_IO_HOOK_DISPATCH(message_forwarding_failed)
CAF_IO_HOOK_DISPATCH(actor_published)
CAF_IO_HOOK_DISPATCH(new_remote_actor)
CAF_IO_HOOK_DISPATCH(new_connection_established)
CAF_IO_HOOK_DISPATCH(new_route_added)
CAF_IO_HOOK_DISPATCH(connection_lost)
CAF_IO_HOOK_DISPATCH(route_lost)
CAF_IO_HOOK_DISPATCH(invalid_message_received)
CAF_IO_HOOK_DISPATCH(before_shutdown)
actor_system& system_;
};
} // namespace io
} // namespace caf
......@@ -31,6 +31,7 @@
#include "caf/proxy_registry.hpp"
#include "caf/send.hpp"
#include "caf/io/hook.hpp"
#include "caf/io/broker.hpp"
#include "caf/io/middleman_actor.hpp"
#include "caf/io/network/multiplexer.hpp"
......@@ -43,6 +44,8 @@ class middleman : public actor_system::module {
public:
friend class ::caf::actor_system;
using hook_vector = std::vector<hook_uptr>;
~middleman() override;
/// Tries to open a port for other CAF instances to connect to.
......@@ -146,6 +149,23 @@ public:
/// Returns the IO backend used by this middleman.
virtual network::multiplexer& backend() = 0;
/// Invokes the callback(s) associated with given event.
template <hook::event_type Event, typename... Ts>
void notify(Ts&&... ts) {
for (auto& hook : hooks_)
hook->handle<Event>(std::forward<Ts>(ts)...);
}
/// Returns whether this middleman has any hooks installed.
inline bool has_hook() const {
return !hooks_.empty();
}
/// Returns all installed hooks.
const hook_vector& hooks() const {
return hooks_;
}
/// Returns the actor associated with `name` at `nid` or
/// `invalid_actor` if `nid` is not connected or has no actor
/// associated to this `name`.
......@@ -328,6 +348,8 @@ private:
std::thread thread_;
// keeps track of "singleton-like" brokers
std::map<atom_value, actor> named_brokers_;
// user-defined hooks
hook_vector hooks_;
// actor offering asyncronous IO by managing this singleton instance
middleman_actor manager_;
};
......
This diff is collapsed.
......@@ -5,7 +5,7 @@
* | |___ / ___ \| _| Framework *
* \____/_/ \_|_| *
* *
* Copyright 2011-2019 Dominik Charousset *
* Copyright 2011-2018 Dominik Charousset *
* *
* Distributed under the terms and conditions of the BSD 3-Clause License or *
* (at your option) under the terms and conditions of the Boost Software *
......@@ -16,60 +16,81 @@
* http://www.boost.org/LICENSE_1_0.txt. *
******************************************************************************/
#include "caf/io/basp/worker_hub.hpp"
#include "caf/io/hook.hpp"
#include "caf/io/basp/worker.hpp"
#include "caf/message_id.hpp"
namespace caf {
namespace io {
namespace basp {
// -- constructors, destructors, and assignment operators ----------------------
hook::hook(actor_system& sys) : system_(sys) {
// nop
}
hook::~hook() {
// nop
}
void hook::message_received_cb(const node_id&, const strong_actor_ptr&,
const strong_actor_ptr&, message_id,
const message&) {
// nop
}
void hook::message_sent_cb(const strong_actor_ptr&, const node_id&,
const strong_actor_ptr&, message_id,
const message&) {
// nop
}
void hook::message_forwarded_cb(const basp::header&, const std::vector<char>*) {
// nop
}
void hook::message_forwarding_failed_cb(const basp::header&,
const std::vector<char>*) {
// nop
}
worker_hub::worker_hub() : head_(nullptr) {
void hook::message_sending_failed_cb(const strong_actor_ptr&,
const strong_actor_ptr&,
message_id, const message&) {
// nop
}
worker_hub::~worker_hub() {
auto head = head_.load();
while (head != nullptr) {
auto next = head->next_.load();
delete head;
head = next;
}
void hook::actor_published_cb(const strong_actor_ptr&,
const std::set<std::string>&, uint16_t) {
// nop
}
// -- properties ---------------------------------------------------------------
void hook::new_remote_actor_cb(const strong_actor_ptr&) {
// nop
}
void worker_hub::push_new_worker(message_queue& queue,
proxy_registry& proxies) {
push(new worker(*this, queue, proxies));
void hook::new_connection_established_cb(const node_id&) {
// nop
}
void worker_hub::push(pointer ptr) {
auto next = head_.load();
for (;;) {
ptr->next_ = next;
if (head_.compare_exchange_strong(next, ptr))
return;
}
void hook::new_route_added_cb(const node_id&, const node_id&) {
// nop
}
worker_hub::pointer worker_hub::pop() {
auto result = head_.load();
if (result == nullptr)
return nullptr;
for (;;) {
auto next = result->next_.load();
if (head_.compare_exchange_strong(result, next))
return result;
}
void hook::connection_lost_cb(const node_id&) {
// nop
}
worker_hub::pointer worker_hub::peek() {
return head_.load();
void hook::route_lost_cb(const node_id&, const node_id&) {
// nop
}
void hook::invalid_message_received_cb(const node_id&, const strong_actor_ptr&,
actor_id, message_id, const message&) {
// nop
}
void hook::before_shutdown_cb() {
// nop
}
} // namespace basp
} // namespace io
} // namespace caf
......@@ -22,10 +22,7 @@
#include "caf/binary_deserializer.hpp"
#include "caf/binary_serializer.hpp"
#include "caf/defaults.hpp"
#include "caf/io/basp/remote_message_handler.hpp"
#include "caf/io/basp/version.hpp"
#include "caf/io/basp/worker.hpp"
#include "caf/settings.hpp"
#include "caf/streambuf.hpp"
namespace caf {
......@@ -46,10 +43,6 @@ instance::instance(abstract_broker* parent, callee& lstnr)
this_node_(parent->system().node()),
callee_(lstnr) {
CAF_ASSERT(this_node_ != none);
auto workers = get_or(config(), "middleman.workers",
defaults::middleman::workers);
for (size_t i = 0; i < workers; ++i)
hub_.push_new_worker(queue_, proxies());
}
connection_state instance::handle(execution_unit* ctx,
......@@ -58,8 +51,11 @@ connection_state instance::handle(execution_unit* ctx,
CAF_LOG_TRACE(CAF_ARG(dm) << CAF_ARG(is_payload));
// function object providing cleanup code on errors
auto err = [&]() -> connection_state {
if (auto nid = tbl_.erase_direct(dm.handle))
auto cb = make_callback([&](const node_id& nid) -> error {
callee_.purge_state(nid);
return none;
});
tbl_.erase_direct(dm.handle, cb);
return close_connection;
};
std::vector<char>* payload = nullptr;
......@@ -122,6 +118,7 @@ void instance::add_published_actor(uint16_t port,
auto& entry = published_actors_[port];
swap(entry.first, published_actor);
swap(entry.second, published_interface);
notify<hook::actor_published>(entry.first, entry.second, port);
}
size_t instance::remove_published_actor(uint16_t port,
......@@ -173,8 +170,10 @@ bool instance::dispatch(execution_unit* ctx, const strong_actor_ptr& sender,
<< CAF_ARG(msg));
CAF_ASSERT(dest_node && this_node_ != dest_node);
auto path = lookup(dest_node);
if (!path)
if (!path) {
//notify<hook::message_sending_failed>(sender, receiver, mid, msg);
return false;
}
auto& source_node = sender ? sender->node() : this_node_;
if (dest_node == path->next_hop && source_node == this_node_) {
header hdr{message_type::direct_message, flags, 0, mid.integer_value(),
......@@ -192,6 +191,7 @@ bool instance::dispatch(execution_unit* ctx, const strong_actor_ptr& sender,
write(ctx, callee_.get_buffer(path->hdl), hdr, &writer);
}
flush(*path);
//notify<hook::message_sent>(sender, path->next_hop, receiver, mid, msg);
return true;
}
......@@ -225,7 +225,7 @@ void instance::write_server_handshake(execution_unit* ctx, buffer_type& out_buf,
}
CAF_LOG_DEBUG_IF(!pa && port, "no actor published");
auto writer = make_callback([&](serializer& sink) -> error {
auto app_ids = get_or(config(), "middleman.app-identifiers",
auto app_ids = get_or(callee_.config(), "middleman.app-identifiers",
defaults::middleman::app_identifiers);
auto aid = invalid_actor_id;
auto iface = std::set<std::string>{};
......@@ -305,7 +305,7 @@ bool instance::handle(execution_unit* ctx, connection_handle hdl, header& hdr,
return false;
}
// Check the application ID.
auto whitelist = get_or(config(), "middleman.app-identifiers",
auto whitelist = get_or(callee_.config(), "middleman.app-identifiers",
defaults::middleman::app_identifiers);
auto i = std::find_first_of(app_ids.begin(), app_ids.end(),
whitelist.begin(), whitelist.end());
......@@ -365,6 +365,29 @@ bool instance::handle(execution_unit* ctx, connection_handle hdl, header& hdr,
callee_.learned_new_node_directly(source_node, was_indirect);
break;
}
case message_type::direct_message: {
// Deserialize payload.
binary_deserializer bd{ctx, *payload};
std::vector<strong_actor_ptr> forwarding_stack;
message msg;
if (auto err = bd(forwarding_stack, msg)) {
CAF_LOG_WARNING("unable to deserialize payload of direct message:"
<< ctx->system().render(err));
return false;
}
// Dispatch message to callee_.
auto source_node = tbl_.lookup_direct(hdl);
if (hdr.has(header::named_receiver_flag))
callee_.deliver(source_node, hdr.source_actor,
static_cast<atom_value>(hdr.dest_actor),
make_message_id(hdr.operation_data), forwarding_stack,
msg);
else
callee_.deliver(source_node, hdr.source_actor, hdr.dest_actor,
make_message_id(hdr.operation_data), forwarding_stack,
msg);
break;
}
case message_type::routed_message: {
// Deserialize payload.
binary_deserializer bd{ctx, *payload};
......@@ -380,48 +403,29 @@ bool instance::handle(execution_unit* ctx, connection_handle hdl, header& hdr,
forward(ctx, dest_node, hdr, *payload);
return true;
}
std::vector<strong_actor_ptr> forwarding_stack;
message msg;
if (auto err = bd(forwarding_stack, msg)) {
CAF_LOG_WARNING("unable to deserialize payload of routed message:"
<< ctx->system().render(err));
return false;
}
// in case the sender of this message was received via a third node,
// we assume that that node to offers a route to the original source
auto last_hop = tbl_.lookup_direct(hdl);
if (source_node != none && source_node != this_node_
&& last_hop != source_node
&& last_hop != source_node && !tbl_.lookup_direct(source_node)
&& tbl_.add_indirect(last_hop, source_node))
callee_.learned_new_node_indirectly(source_node);
}
// fall through
case message_type::direct_message: {
auto worker = hub_.pop();
auto last_hop = tbl_.lookup_direct(hdl);
if (worker != nullptr) {
CAF_LOG_DEBUG("launch BASP worker for deserializing a"
<< hdr.operation);
worker->launch(last_hop, hdr, *payload);
} else {
CAF_LOG_DEBUG("out of BASP workers, continue deserializing a"
<< hdr.operation);
// If no worker is available then we have no other choice than to take
// the performance hit and deserialize in this thread.
struct handler : remote_message_handler<handler> {
handler(message_queue* queue, proxy_registry* proxies,
actor_system* system, node_id last_hop, basp::header& hdr,
buffer_type& payload)
: queue_(queue),
proxies_(proxies),
system_(system),
last_hop_(std::move(last_hop)),
hdr_(hdr),
payload_(payload) {
msg_id_ = queue_->new_id();
}
message_queue* queue_;
proxy_registry* proxies_;
actor_system* system_;
node_id last_hop_;
basp::header& hdr_;
buffer_type& payload_;
uint64_t msg_id_;
};
handler f{&queue_, &proxies(), &system(), last_hop, hdr, *payload};
f.handle_remote_message(callee_.current_execution_unit());
}
if (hdr.has(header::named_receiver_flag))
callee_.deliver(source_node, hdr.source_actor,
static_cast<atom_value>(hdr.dest_actor),
make_message_id(hdr.operation_data), forwarding_stack,
msg);
else
callee_.deliver(source_node, hdr.source_actor, hdr.dest_actor,
make_message_id(hdr.operation_data), forwarding_stack,
msg);
break;
}
case message_type::monitor_message: {
......@@ -486,8 +490,10 @@ void instance::forward(execution_unit* ctx, const node_id& dest_node,
return;
}
flush(*path);
notify<hook::message_forwarded>(hdr, &payload);
} else {
CAF_LOG_WARNING("cannot forward message, no route to destination");
notify<hook::message_forwarding_failed>(hdr, &payload);
}
}
......
/******************************************************************************
* ____ _ _____ *
* / ___| / \ | ___| C++ *
* | | / _ \ | |_ Actor *
* | |___ / ___ \| _| Framework *
* \____/_/ \_|_| *
* *
* Copyright 2011-2019 Dominik Charousset *
* *
* Distributed under the terms and conditions of the BSD 3-Clause License or *
* (at your option) under the terms and conditions of the Boost Software *
* License 1.0. See accompanying files LICENSE and LICENSE_ALTERNATIVE. *
* *
* If you did not receive a copy of the license files, see *
* http://opensource.org/licenses/BSD-3-Clause and *
* http://www.boost.org/LICENSE_1_0.txt. *
******************************************************************************/
#include "caf/io/basp/message_queue.hpp"
#include <iterator>
namespace caf {
namespace io {
namespace basp {
message_queue::message_queue() : next_id(0), next_undelivered(0) {
// nop
}
void message_queue::push(execution_unit* ctx, uint64_t id,
strong_actor_ptr receiver,
mailbox_element_ptr content) {
std::unique_lock<std::mutex> guard{lock};
CAF_ASSERT(id >= next_undelivered);
CAF_ASSERT(id < next_id);
auto first = pending.begin();
auto last = pending.end();
if (id == next_undelivered) {
// Dispatch current head.
if (receiver != nullptr)
receiver->enqueue(std::move(content), ctx);
auto next = id + 1;
// Check whether we can deliver more.
if (first == last || first->id != next) {
next_undelivered = next;
CAF_ASSERT(next_undelivered <= next_id);
return;
}
// Deliver everything until reaching a non-consecutive ID or the end.
auto i = first;
for (; i != last && i->id == next; ++i, ++next)
if (i->receiver != nullptr)
i->receiver->enqueue(std::move(i->content), ctx);
next_undelivered = next;
pending.erase(first, i);
CAF_ASSERT(next_undelivered <= next_id);
return;
}
// Get the insertion point.
auto pred = [&](const actor_msg& x) {
return x.id >= id;
};
pending.emplace(std::find_if(first, last, pred),
actor_msg{id, std::move(receiver), std::move(content)});
}
void message_queue::drop(execution_unit* ctx, uint64_t id) {
push(ctx, id, nullptr, nullptr);
}
uint64_t message_queue::new_id() {
std::unique_lock<std::mutex> guard{lock};
return next_id++;
}
} // namespace basp
} // namespace io
} // namespace caf
......@@ -270,6 +270,9 @@ strong_actor_ptr middleman::remote_lookup(atom_value name, const node_id& nid) {
void middleman::start() {
CAF_LOG_TRACE("");
// Create hooks.
for (auto& f : system().config().hook_factories)
hooks_.emplace_back(f(system_));
// Launch backend.
if (!get_or(config(), "middleman.manual-multiplexing", false))
backend_supervisor_ = backend().make_supervisor();
......@@ -309,6 +312,7 @@ void middleman::stop() {
CAF_LOG_TRACE("");
backend().dispatch([=] {
CAF_LOG_TRACE("");
notify<hook::before_shutdown>();
// managers_ will be modified while we are stopping each manager,
// because each manager will call remove(...)
for (auto& kvp : named_brokers_) {
......@@ -329,18 +333,13 @@ void middleman::stop() {
while (backend().try_run_once())
; // nop
}
hooks_.clear();
named_brokers_.clear();
scoped_actor self{system(), true};
self->send_exit(manager_, exit_reason::kill);
if (!get_or(config(), "middleman.attach-utility-actors", false))
self->wait_for(manager_);
destroy(manager_);
// Note: we intentionally don't call `named_brokers_.clear()` here. The BASP
// broker must outlive the scheduler threads. The scheduler is stopped
// *after* the MM. However, the BASP workers still need to return to their
// hub safely, should some of them are still running at this point. By not
// clearing the container, we keep the BASP broker (and thus the BASP
// instance) alive until the MM module gets destroyed. At that point, all
// BASP workers are safe in their hub.
}
void middleman::init(actor_system_config& cfg) {
......
......@@ -34,23 +34,18 @@ routing_table::~routing_table() {
}
optional<routing_table::route> routing_table::lookup(const node_id& target) {
std::unique_lock<std::mutex> guard{mtx_};
// Check whether we have a direct path first.
{ // Lifetime scope of first iterator.
auto i = direct_by_nid_.find(target);
if (i != direct_by_nid_.end())
return route{target, i->second};
}
// Pick first available indirect route.
auto hdl = lookup_direct(target);
if (hdl)
return route{target, *hdl};
// pick first available indirect route
auto i = indirect_.find(target);
if (i != indirect_.end()) {
auto& hops = i->second;
while (!hops.empty()) {
auto& hop = *hops.begin();
auto j = direct_by_nid_.find(hop);
if (j != direct_by_nid_.end())
return route{hop, j->second};
// Erase hops that became invalid.
hdl = lookup_direct(hop);
if (hdl)
return route{hop, *hdl};
hops.erase(hops.begin());
}
}
......@@ -58,16 +53,11 @@ optional<routing_table::route> routing_table::lookup(const node_id& target) {
}
node_id routing_table::lookup_direct(const connection_handle& hdl) const {
std::unique_lock<std::mutex> guard{mtx_};
auto i = direct_by_hdl_.find(hdl);
if (i != direct_by_hdl_.end())
return i->second;
return none;
return get_opt(direct_by_hdl_, hdl, none);
}
optional<connection_handle>
routing_table::lookup_direct(const node_id& nid) const {
std::unique_lock<std::mutex> guard{mtx_};
auto i = direct_by_nid_.find(nid);
if (i != direct_by_nid_.end())
return i->second;
......@@ -75,58 +65,91 @@ routing_table::lookup_direct(const node_id& nid) const {
}
node_id routing_table::lookup_indirect(const node_id& nid) const {
std::unique_lock<std::mutex> guard{mtx_};
auto i = indirect_.find(nid);
if (i == indirect_.end())
return none;
if (!i->second.empty())
return *i->second.begin();
return none;
if (i->second.empty())
return none;
return *i->second.begin();
}
node_id routing_table::erase_direct(const connection_handle& hdl) {
std::unique_lock<std::mutex> guard{mtx_};
void routing_table::blacklist(const node_id& hop, const node_id& dest) {
blacklist_[dest].emplace(hop);
auto i = indirect_.find(dest);
if (i == indirect_.end())
return;
i->second.erase(hop);
if (i->second.empty())
indirect_.erase(i);
}
void routing_table::erase_direct(const connection_handle& hdl,
erase_callback& cb) {
auto i = direct_by_hdl_.find(hdl);
if (i == direct_by_hdl_.end())
return none;
return;
cb(i->second);
parent_->parent().notify<hook::connection_lost>(i->second);
direct_by_nid_.erase(i->second);
node_id result = std::move(i->second);
direct_by_hdl_.erase(i->first);
return result;
}
bool routing_table::erase_indirect(const node_id& dest) {
std::unique_lock<std::mutex> guard{mtx_};
auto i = indirect_.find(dest);
if (i == indirect_.end())
return false;
if (parent_->parent().has_hook())
for (auto& nid : i->second)
parent_->parent().notify<hook::route_lost>(nid, dest);
indirect_.erase(i);
return true;
}
void routing_table::add_direct(const connection_handle& hdl,
const node_id& nid) {
std::unique_lock<std::mutex> guard{mtx_};
auto hdl_added = direct_by_hdl_.emplace(hdl, nid).second;
auto nid_added = direct_by_nid_.emplace(nid, hdl).second;
CAF_ASSERT(hdl_added && nid_added);
CAF_IGNORE_UNUSED(hdl_added);
CAF_IGNORE_UNUSED(nid_added);
CAF_ASSERT(direct_by_hdl_.count(hdl) == 0);
CAF_ASSERT(direct_by_nid_.count(nid) == 0);
direct_by_hdl_.emplace(hdl, nid);
direct_by_nid_.emplace(nid, hdl);
parent_->parent().notify<hook::new_connection_established>(nid);
}
bool routing_table::add_indirect(const node_id& hop, const node_id& dest) {
std::unique_lock<std::mutex> guard{mtx_};
// Never add indirect entries if we already have direct connection.
if (direct_by_nid_.count(dest) != 0)
return false;
// Never add indirect entries if we don't have a connection to the hop.
if (direct_by_nid_.count(hop) == 0)
return false;
// Add entry to our node ID set.
auto& hops = indirect_[dest];
auto result = hops.empty();
hops.emplace(hop);
return result;
auto i = blacklist_.find(dest);
if (i == blacklist_.end() || i->second.count(hop) == 0) {
auto& hops = indirect_[dest];
auto added_first = hops.empty();
hops.emplace(hop);
parent_->parent().notify<hook::new_route_added>(hop, dest);
return added_first;
}
return false; // blacklisted
}
bool routing_table::reachable(const node_id& dest) {
return direct_by_nid_.count(dest) > 0 || indirect_.count(dest) > 0;
}
size_t routing_table::erase(const node_id& dest, erase_callback& cb) {
cb(dest);
size_t res = 0;
auto i = indirect_.find(dest);
if (i != indirect_.end()) {
res = i->second.size();
for (auto& nid : i->second) {
cb(nid);
parent_->parent().notify<hook::route_lost>(nid, dest);
}
indirect_.erase(i);
}
auto hdl = lookup_direct(dest);
if (hdl) {
direct_by_hdl_.erase(*hdl);
direct_by_nid_.erase(dest);
parent_->parent().notify<hook::connection_lost>(dest);
++res;
}
return res;
}
} // namespace basp
......
/******************************************************************************
* ____ _ _____ *
* / ___| / \ | ___| C++ *
* | | / _ \ | |_ Actor *
* | |___ / ___ \| _| Framework *
* \____/_/ \_|_| *
* *
* Copyright 2011-2019 Dominik Charousset *
* *
* Distributed under the terms and conditions of the BSD 3-Clause License or *
* (at your option) under the terms and conditions of the Boost Software *
* License 1.0. See accompanying files LICENSE and LICENSE_ALTERNATIVE. *
* *
* If you did not receive a copy of the license files, see *
* http://opensource.org/licenses/BSD-3-Clause and *
* http://www.boost.org/LICENSE_1_0.txt. *
******************************************************************************/
#include "caf/io/basp/worker.hpp"
#include "caf/actor_system.hpp"
#include "caf/io/basp/message_queue.hpp"
#include "caf/io/basp/worker_hub.hpp"
#include "caf/proxy_registry.hpp"
#include "caf/scheduler/abstract_coordinator.hpp"
namespace caf {
namespace io {
namespace basp {
// -- constructors, destructors, and assignment operators ----------------------
worker::worker(worker_hub& hub, message_queue& queue, proxy_registry& proxies)
: next_(nullptr),
hub_(&hub),
queue_(&queue),
proxies_(&proxies),
system_(&proxies.system()) {
CAF_IGNORE_UNUSED(pad_);
}
worker::~worker() {
// nop
}
// -- management ---------------------------------------------------------------
void worker::launch(const node_id& last_hop, const basp::header& hdr,
const buffer_type& payload) {
CAF_ASSERT(hdr.dest_actor != 0);
CAF_ASSERT(hdr.operation == basp::message_type::direct_message
|| hdr.operation == basp::message_type::routed_message);
msg_id_ = queue_->new_id();
last_hop_ = last_hop;
memcpy(&hdr_, &hdr, sizeof(basp::header));
payload_.assign(payload.begin(), payload.end());
system_->scheduler().enqueue(this);
}
// -- implementation of resumable ----------------------------------------------
resumable::subtype_t worker::subtype() const {
return resumable::function_object;
}
resumable::resume_result worker::resume(execution_unit* ctx, size_t) {
ctx->proxy_registry_ptr(proxies_);
handle_remote_message(ctx);
hub_->push(this);
return resumable::awaiting_message;
}
void worker::intrusive_ptr_add_ref_impl() {
// The basp::instance owns the hub (which owns this object) and must make
// sure to wait for pending workers at exit.
}
void worker::intrusive_ptr_release_impl() {
// nop
}
} // namespace basp
} // namespace io
} // namespace caf
......@@ -110,12 +110,11 @@ struct node {
class fixture {
public:
fixture(bool autoconn = false)
: sys(cfg.load<io::middleman, network::test_multiplexer>()
.set("middleman.enable-automatic-connections", autoconn)
.set("middleman.workers", size_t{0})
.set("scheduler.policy",
autoconn ? caf::atom("testing") : caf::atom("stealing"))
.set("middleman.attach-utility-actors", autoconn)) {
: sys(cfg.load<io::middleman, network::test_multiplexer>()
.set("middleman.enable-automatic-connections", autoconn)
.set("scheduler.policy", autoconn ? caf::atom("testing")
: caf::atom("stealing"))
.set("middleman.attach-utility-actors", autoconn)) {
auto& mm = sys.middleman();
mpx_ = dynamic_cast<network::test_multiplexer*>(&mm.backend());
CAF_REQUIRE(mpx_ != nullptr);
......@@ -200,17 +199,17 @@ public:
// implementation of the Binary Actor System Protocol
basp::instance& instance() {
return aut()->instance;
return aut()->state.instance;
}
// our routing table (filled by BASP)
basp::routing_table& tbl() {
return aut()->instance.tbl();
return aut()->state.instance.tbl();
}
// access to proxy instances
proxy_registry& proxies() {
return aut()->proxies();
return aut()->state.proxies();
}
// stores the singleton pointer for convenience
......@@ -313,7 +312,7 @@ public:
buffer buf;
std::tie(hdr, buf) = read_from_out_buf(hdl);
CAF_MESSAGE("dispatch output buffer for connection " << hdl.id());
CAF_REQUIRE_EQUAL(hdr.operation, basp::message_type::direct_message);
CAF_REQUIRE(hdr.operation == basp::message_type::direct_message);
binary_deserializer source{mpx_, buf};
std::vector<strong_actor_ptr> stages;
message msg;
......@@ -354,9 +353,8 @@ public:
buffer buf;
this_->to_payload(buf, xs...);
buffer& ob = this_->mpx()->output_buffer(hdl);
while (this_->mpx()->try_exec_runnable()) {
// repeat
}
while (ob.size() < basp::header_size)
this_->mpx()->exec_runnable();
CAF_MESSAGE("output buffer has " << ob.size() << " bytes");
basp::header hdr;
{ // lifetime scope of source
......@@ -377,6 +375,12 @@ public:
ob.erase(ob.begin(), ob.begin() + basp::header_size);
}
CAF_CHECK_EQUAL(operation, hdr.operation);
if (hdr.operation == basp::message_type::direct_message) {
binary_deserializer source{this_->mpx(), payload};
std::vector<strong_actor_ptr> fwd_stack;
message msg;
source(fwd_stack, msg);
}
CAF_CHECK_EQUAL(flags, static_cast<uint8_t>(hdr.flags));
CAF_CHECK_EQUAL(payload_len, hdr.payload_len);
CAF_CHECK_EQUAL(operation_data, hdr.operation_data);
......
/******************************************************************************
* ____ _ _____ *
* / ___| / \ | ___| C++ *
* | | / _ \ | |_ Actor *
* | |___ / ___ \| _| Framework *
* \____/_/ \_|_| *
* *
* Copyright 2011-2019 Dominik Charousset *
* *
* Distributed under the terms and conditions of the BSD 3-Clause License or *
* (at your option) under the terms and conditions of the Boost Software *
* License 1.0. See accompanying files LICENSE and LICENSE_ALTERNATIVE. *
* *
* If you did not receive a copy of the license files, see *
* http://opensource.org/licenses/BSD-3-Clause and *
* http://www.boost.org/LICENSE_1_0.txt. *
******************************************************************************/
#define CAF_SUITE io_basp_message_queue
#include "caf/io/basp/message_queue.hpp"
#include "caf/test/dsl.hpp"
#include "caf/actor_cast.hpp"
#include "caf/actor_system.hpp"
#include "caf/behavior.hpp"
using namespace caf;
namespace {
behavior testee_impl() {
return {
[](ok_atom, int) {
// nop
}
};
}
struct fixture : test_coordinator_fixture<> {
io::basp::message_queue queue;
strong_actor_ptr testee;
fixture() {
auto hdl = sys.spawn<lazy_init>(testee_impl);
testee = actor_cast<strong_actor_ptr>(hdl);
}
void acquire_ids(size_t num) {
for (size_t i = 0; i < num; ++i)
queue.new_id();
}
void push(int msg_id) {
queue.push(nullptr, static_cast<uint64_t>(msg_id), testee,
make_mailbox_element(self->ctrl(), make_message_id(), {},
ok_atom::value, msg_id));
}
};
} // namespace <anonymous>
CAF_TEST_FIXTURE_SCOPE(message_queue_tests, fixture)
CAF_TEST(default construction) {
CAF_CHECK_EQUAL(queue.next_id, 0u);
CAF_CHECK_EQUAL(queue.next_undelivered, 0u);
CAF_CHECK_EQUAL(queue.pending.size(), 0u);
}
CAF_TEST(ascending IDs) {
CAF_CHECK_EQUAL(queue.new_id(), 0u);
CAF_CHECK_EQUAL(queue.new_id(), 1u);
CAF_CHECK_EQUAL(queue.new_id(), 2u);
CAF_CHECK_EQUAL(queue.next_undelivered, 0u);
}
CAF_TEST(push order 0-1-2) {
acquire_ids(3);
push(0);
expect((ok_atom, int), from(self).to(testee).with(_, 0));
push(1);
expect((ok_atom, int), from(self).to(testee).with(_, 1));
push(2);
expect((ok_atom, int), from(self).to(testee).with(_, 2));
}
CAF_TEST(push order 0-2-1) {
acquire_ids(3);
push(0);
expect((ok_atom, int), from(self).to(testee).with(_, 0));
push(2);
disallow((ok_atom, int), from(self).to(testee));
push(1);
expect((ok_atom, int), from(self).to(testee).with(_, 1));
expect((ok_atom, int), from(self).to(testee).with(_, 2));
}
CAF_TEST(push order 1-0-2) {
acquire_ids(3);
push(1);
disallow((ok_atom, int), from(self).to(testee));
push(0);
expect((ok_atom, int), from(self).to(testee).with(_, 0));
expect((ok_atom, int), from(self).to(testee).with(_, 1));
push(2);
expect((ok_atom, int), from(self).to(testee).with(_, 2));
}
CAF_TEST(push order 1-2-0) {
acquire_ids(3);
push(1);
disallow((ok_atom, int), from(self).to(testee));
push(2);
disallow((ok_atom, int), from(self).to(testee));
push(0);
expect((ok_atom, int), from(self).to(testee).with(_, 0));
expect((ok_atom, int), from(self).to(testee).with(_, 1));
expect((ok_atom, int), from(self).to(testee).with(_, 2));
}
CAF_TEST(push order 2-0-1) {
acquire_ids(3);
push(2);
disallow((ok_atom, int), from(self).to(testee));
push(0);
expect((ok_atom, int), from(self).to(testee).with(_, 0));
push(1);
expect((ok_atom, int), from(self).to(testee).with(_, 1));
expect((ok_atom, int), from(self).to(testee).with(_, 2));
}
CAF_TEST(push order 2-1-0) {
acquire_ids(3);
push(2);
disallow((ok_atom, int), from(self).to(testee));
push(1);
disallow((ok_atom, int), from(self).to(testee));
push(0);
expect((ok_atom, int), from(self).to(testee).with(_, 0));
expect((ok_atom, int), from(self).to(testee).with(_, 1));
expect((ok_atom, int), from(self).to(testee).with(_, 2));
}
CAF_TEST(dropping) {
acquire_ids(3);
push(2);
disallow((ok_atom, int), from(self).to(testee));
queue.drop(nullptr, 1);
disallow((ok_atom, int), from(self).to(testee));
push(0);
expect((ok_atom, int), from(self).to(testee).with(_, 0));
expect((ok_atom, int), from(self).to(testee).with(_, 2));
}
CAF_TEST_FIXTURE_SCOPE_END()
/******************************************************************************
* ____ _ _____ *
* / ___| / \ | ___| C++ *
* | | / _ \ | |_ Actor *
* | |___ / ___ \| _| Framework *
* \____/_/ \_|_| *
* *
* Copyright 2011-2019 Dominik Charousset *
* *
* Distributed under the terms and conditions of the BSD 3-Clause License or *
* (at your option) under the terms and conditions of the Boost Software *
* License 1.0. See accompanying files LICENSE and LICENSE_ALTERNATIVE. *
* *
* If you did not receive a copy of the license files, see *
* http://opensource.org/licenses/BSD-3-Clause and *
* http://www.boost.org/LICENSE_1_0.txt. *
******************************************************************************/
#define CAF_SUITE io_basp_worker
#include "caf/io/basp/worker.hpp"
#include "caf/test/dsl.hpp"
#include "caf/actor_cast.hpp"
#include "caf/actor_control_block.hpp"
#include "caf/actor_system.hpp"
#include "caf/binary_serializer.hpp"
#include "caf/io/basp/message_queue.hpp"
#include "caf/io/basp/worker_hub.hpp"
#include "caf/make_actor.hpp"
#include "caf/proxy_registry.hpp"
using namespace caf;
namespace {
behavior testee_impl() {
return {
[](ok_atom) {
// nop
}
};
}
class mock_actor_proxy : public actor_proxy {
public:
explicit mock_actor_proxy(actor_config& cfg) : actor_proxy(cfg) {
// nop
}
void enqueue(mailbox_element_ptr, execution_unit*) override {
CAF_FAIL("mock_actor_proxy::enqueue called");
}
void kill_proxy(execution_unit*, error) override {
// nop
}
};
class mock_proxy_registry_backend : public proxy_registry::backend {
public:
mock_proxy_registry_backend(actor_system& sys) : sys_(sys) {
//nop
}
strong_actor_ptr make_proxy(node_id nid, actor_id aid) override {
actor_config cfg;
return make_actor<mock_actor_proxy, strong_actor_ptr>(aid, nid, &sys_, cfg);
}
void set_last_hop(node_id*) override {
// nop
}
private:
actor_system& sys_;
};
struct fixture : test_coordinator_fixture<> {
io::basp::worker_hub hub;
io::basp::message_queue queue;
mock_proxy_registry_backend proxies_backend;
proxy_registry proxies;
node_id last_hop;
actor testee;
fixture()
: proxies_backend(sys),
proxies(sys, proxies_backend),
last_hop(123, "0011223344556677889900112233445566778899") {
testee = sys.spawn<lazy_init>(testee_impl);
sys.registry().put(testee.id(), testee);
}
~fixture() {
sys.registry().erase(testee.id());
}
};
} // namespace <anonymous>
CAF_TEST_FIXTURE_SCOPE(worker_tests, fixture)
CAF_TEST(deliver serialized message) {
CAF_MESSAGE("create the BASP worker");
CAF_REQUIRE_EQUAL(hub.peek(), nullptr);
hub.push_new_worker(queue, proxies);
CAF_REQUIRE_NOT_EQUAL(hub.peek(), nullptr);
auto w = hub.pop();
CAF_MESSAGE("create a fake message + BASP header");
std::vector<char> payload;
std::vector<strong_actor_ptr> stages;
binary_serializer sink{sys, payload};
if (auto err = sink(stages, make_message(ok_atom::value)))
CAF_FAIL("unable to serialize message: " << sys.render(err));
io::basp::header hdr{io::basp::message_type::direct_message,
0,
static_cast<uint32_t>(payload.size()),
make_message_id().integer_value(),
42,
testee.id()};
CAF_MESSAGE("launch worker");
w->launch(last_hop, hdr, payload);
sched.run_once();
expect((ok_atom), from(_).to(testee));
}
CAF_TEST_FIXTURE_SCOPE_END()
......@@ -191,10 +191,6 @@ public:
return ptr_;
}
pointer operator->() const {
return get();
}
ptrdiff_t compare(const caf_handle& other) const {
return reinterpret_cast<ptrdiff_t>(ptr_)
- reinterpret_cast<ptrdiff_t>(other.ptr_);
......@@ -208,12 +204,32 @@ private:
caf::abstract_actor* ptr_;
};
// -- access to an actor's mailbox ---------------------------------------------
/// Returns a pointer to the next element in an actor's mailbox without taking
/// it out of the mailbox.
/// @pre `ptr` is alive and either a `scheduled_actor` or `blocking_actor`
inline caf::mailbox_element* next_mailbox_element(caf_handle x) {
CAF_ASSERT(x.get() != nullptr);
auto sptr = dynamic_cast<caf::scheduled_actor*>(x.get());
if (sptr != nullptr) {
return sptr->mailbox().closed() || sptr->mailbox().blocked()
? nullptr
: sptr->mailbox().peek();
}
auto bptr = dynamic_cast<caf::blocking_actor*>(x.get());
CAF_ASSERT(bptr != nullptr);
return bptr->mailbox().closed() || bptr->mailbox().blocked()
? nullptr
: bptr->mailbox().peek();
}
// -- introspection of the next mailbox element --------------------------------
/// @private
template <class... Ts>
caf::optional<std::tuple<Ts...>> default_extract(caf_handle x) {
auto ptr = x->peek_at_next_mailbox_element();
auto ptr = next_mailbox_element(x);
if (ptr == nullptr || !ptr->content().template match_elements<Ts...>())
return caf::none;
return ptr->content().template get_as_tuple<Ts...>();
......@@ -261,7 +277,7 @@ template <class T, class... Ts>
std::tuple<T, Ts...> extract(caf_handle x) {
auto result = try_extract<T, Ts...>(x);
if (result == caf::none) {
auto ptr = x->peek_at_next_mailbox_element();
auto ptr = next_mailbox_element(x);
if (ptr == nullptr)
CAF_FAIL("Mailbox is empty");
CAF_FAIL("Message does not match expected pattern: "
......@@ -310,8 +326,8 @@ public:
template <class Handle>
expect_clause& to(const Handle& whom) {
CAF_REQUIRE(sched_.prioritize(whom));
dest_ = &sched_.next_job<caf::abstract_actor>();
auto ptr = dest_->peek_at_next_mailbox_element();
dest_ = &sched_.next_job<caf::scheduled_actor>();
auto ptr = next_mailbox_element(dest_);
CAF_REQUIRE(ptr != nullptr);
if (src_)
CAF_REQUIRE_EQUAL(ptr->sender, src_);
......@@ -350,7 +366,7 @@ protected:
caf::scheduler::test_coordinator& sched_;
caf::strong_actor_ptr src_;
caf::abstract_actor* dest_;
caf::local_actor* dest_;
std::function<void ()> peek_;
};
......@@ -386,7 +402,7 @@ public:
template <class Handle>
allow_clause& to(const Handle& whom) {
if (sched_.prioritize(whom))
dest_ = &sched_.next_job<caf::abstract_actor>();
dest_ = &sched_.next_job<caf::scheduled_actor>();
return *this;
}
......@@ -432,7 +448,7 @@ protected:
caf::scheduler::test_coordinator& sched_;
caf::strong_actor_ptr src_;
caf::abstract_actor* dest_;
caf::local_actor* dest_;
std::function<bool ()> peek_;
};
......@@ -441,14 +457,14 @@ class disallow_clause {
public:
disallow_clause() {
check_ = [=] {
auto ptr = dest_->peek_at_next_mailbox_element();
auto ptr = next_mailbox_element(dest_);
if (ptr == nullptr)
return;
if (src_ != nullptr && ptr->sender != src_)
return;
auto res = try_extract<Ts...>(dest_);
if (res != caf::none)
CAF_FAIL("received disallowed message: " << caf::deep_to_string(*ptr));
CAF_FAIL("received disallowed message: " << CAF_ARG(*res));
};
}
......@@ -478,7 +494,7 @@ public:
// TODO: move tmp into lambda when switching to C++14
auto tmp = std::make_tuple(std::forward<Us>(xs)...);
check_ = [=] {
auto ptr = dest_->peek_at_next_mailbox_element();
auto ptr = next_mailbox_element(dest_);
if (ptr == nullptr)
return;
if (src_ != nullptr && ptr->sender != src_)
......@@ -547,7 +563,6 @@ public:
cfg.set("scheduler.policy", caf::atom("testing"));
cfg.set("logger.inline-output", true);
cfg.set("middleman.network-backend", caf::atom("testing"));
cfg.set("middleman.workers", size_t{0});
return cfg;
}
......
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