Commit 8e94420e authored by Dominik Charousset's avatar Dominik Charousset Committed by Dominik Charousset

Remove broken UDP option for BASP

parent b173485d
......@@ -18,13 +18,14 @@
#pragma once
#include "caf/io/basp/buffer_type.hpp"
#include "caf/io/basp/connection_state.hpp"
#include "caf/io/basp/endpoint_context.hpp"
#include "caf/io/basp/header.hpp"
#include "caf/io/basp/version.hpp"
#include "caf/io/basp/instance.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/connection_state.hpp"
#include "caf/io/basp/version.hpp"
/// @defgroup BASP Binary Actor Sytem Protocol
///
......
......@@ -35,14 +35,12 @@ namespace basp {
// stores meta information for active endpoints
struct endpoint_context {
using pending_map = std::map<sequence_type, std::pair<basp::header,
std::vector<char>>>;
// denotes what message we expect from the remote node next
basp::connection_state cstate;
// our currently processed BASP header
basp::header hdr;
// the handle for I/O operations
variant<connection_handle, datagram_handle> hdl;
connection_handle hdl;
// network-agnostic node identifier
node_id id;
// ports
......@@ -50,15 +48,6 @@ struct endpoint_context {
uint16_t local_port;
// pending operations to be performed after handshake completed
optional<response_promise> callback;
// protocols that do not implement ordering are ordered by CAF
bool requires_ordering;
// sequence numbers and a buffer to establish order
sequence_type seq_incoming;
sequence_type seq_outgoing;
// pending messages due to ordering
pending_map pending;
// track if a timeout to deliver pending messages is set
bool did_set_timeout;
};
} // namespace basp
......
......@@ -35,9 +35,6 @@ namespace basp {
/// @addtogroup BASP
/// Sequence number type for BASP headers.
using sequence_type = uint16_t;
/// The header of a Binary Actor System Protocol (BASP) message.
/// A BASP header consists of a routing part, i.e., source and
/// destination, as well as an operation and operation data. Several
......@@ -53,11 +50,9 @@ struct header {
node_id dest_node;
actor_id source_actor;
actor_id dest_actor;
sequence_type sequence_number;
inline header(message_type m_operation, uint8_t m_flags,
uint32_t m_payload_len, uint64_t m_operation_data,
node_id m_source_node, node_id m_dest_node,
header(message_type m_operation, uint8_t m_flags, uint32_t m_payload_len,
uint64_t m_operation_data, node_id m_source_node, node_id m_dest_node,
actor_id m_source_actor, actor_id m_dest_actor)
: operation(m_operation),
flags(m_flags),
......@@ -66,25 +61,7 @@ struct header {
source_node(std::move(m_source_node)),
dest_node(std::move(m_dest_node)),
source_actor(m_source_actor),
dest_actor(m_dest_actor),
sequence_number(0) {
// nop
}
inline header(message_type m_operation, uint8_t m_flags,
uint32_t m_payload_len, uint64_t m_operation_data,
node_id m_source_node, node_id m_dest_node,
actor_id m_source_actor, actor_id m_dest_actor,
sequence_type m_sequence_number)
: operation(m_operation),
flags(m_flags),
payload_len(m_payload_len),
operation_data(m_operation_data),
source_node(std::move(m_source_node)),
dest_node(std::move(m_dest_node)),
source_actor(m_source_actor),
dest_actor(m_dest_actor),
sequence_number(m_sequence_number) {
dest_actor(m_dest_actor) {
// nop
}
......@@ -94,7 +71,7 @@ struct header {
static const uint8_t named_receiver_flag = 0x01;
/// Queries whether this header has the given flag.
inline bool has(uint8_t flag) const {
bool has(uint8_t flag) const {
return (flags & flag) != 0;
}
};
......@@ -109,8 +86,7 @@ typename Inspector::result_type inspect(Inspector& f, header& hdr) {
meta::omittable(), pad,
hdr.flags, hdr.payload_len, hdr.operation_data,
hdr.source_node, hdr.dest_node,
hdr.source_actor, hdr.dest_actor,
hdr.sequence_number);
hdr.source_actor, hdr.dest_actor);
}
/// @relates header
......@@ -140,8 +116,7 @@ bool valid(const header& hdr);
constexpr size_t header_size = node_id::serialized_size * 2
+ sizeof(actor_id) * 2
+ sizeof(uint32_t) * 2
+ sizeof(uint64_t)
+ sizeof(sequence_type);
+ sizeof(uint64_t);
/// @}
......
......@@ -33,7 +33,6 @@
#include "caf/io/basp/message_type.hpp"
#include "caf/io/basp/routing_table.hpp"
#include "caf/io/basp/connection_state.hpp"
#include "caf/io/basp/endpoint_context.hpp"
namespace caf {
namespace io {
......@@ -48,7 +47,6 @@ public:
class callee {
protected:
using buffer_type = std::vector<char>;
using endpoint_handle = variant<connection_handle, datagram_handle>;
public:
explicit callee(actor_system& sys, proxy_registry::backend& backend);
......@@ -95,65 +93,23 @@ public:
virtual void handle_heartbeat(const node_id& nid) = 0;
/// Returns the actor namespace associated to this BASP protocol instance.
inline proxy_registry& proxies() {
proxy_registry& proxies() {
return namespace_;
}
/// Returns the hosting actor system.
inline actor_system& system() {
actor_system& system() {
return namespace_.system();
}
/// Returns the system-wide configuration.
inline const actor_system_config& config() const {
const actor_system_config& config() const {
return namespace_.system().config();
}
/// Returns the next outgoing sequence number for a connection.
virtual sequence_type next_sequence_number(connection_handle hdl) = 0;
/// Returns the next outgoing sequence number for an endpoint.
virtual sequence_type next_sequence_number(datagram_handle hdl) = 0;
/// Adds a message with a future sequence number to the pending messages
/// of a given endpoint context.
virtual void add_pending(execution_unit* ctx, endpoint_context& ep,
sequence_type seq, header hdr,
std::vector<char> payload) = 0;
/// Delivers a pending incoming messages for an endpoint `ep` with
/// application layer ordering. Delivery of the next available packet can
/// be forced to simply skip undeliverd messages via the `force` flag.
virtual bool deliver_pending(execution_unit* ctx, endpoint_context& ep,
bool force) = 0;
/// Drop pending messages with sequence number `seq`.
virtual void drop_pending(endpoint_context& ep, sequence_type seq) = 0;
/// Returns a reference to the current sent buffer, dispatching the call
/// based on the type contained in `hdl`.
virtual buffer_type& get_buffer(endpoint_handle hdl) = 0;
/// Returns a reference to the current sent buffer. The callee may cache
/// buffers to reuse them for multiple datagrams. Subsequent calls will
/// return the same buffer until `pop_datagram_buffer` is called.
virtual buffer_type& get_buffer(datagram_handle hdl) = 0;
/// Returns a reference to the sent buffer.
virtual buffer_type& get_buffer(connection_handle hdl) = 0;
/// Returns the buffer accessed through a call to `get_buffer` when
/// passing a datagram handle and removes it from the callee.
virtual buffer_type pop_datagram_buffer(datagram_handle hdl) = 0;
/// Flushes the underlying write buffer of `hdl`, dispatches the call based
/// on the type contained in `hdl`.
virtual void flush(endpoint_handle hdl) = 0;
/// Flushes the underlying write buffer of `hdl`. Implicitly pops the
/// current buffer and enqueues it for sending.
virtual void flush(datagram_handle hdl) = 0;
/// Flushes the underlying write buffer of `hdl`.
virtual void flush(connection_handle hdl) = 0;
......@@ -175,9 +131,6 @@ public:
connection_state handle(execution_unit* ctx,
new_data_msg& dm, header& hdr, bool is_payload);
/// Handles a received datagram.
bool handle(execution_unit* ctx, new_datagram_msg& dm, endpoint_context& ep);
/// Sends heartbeat messages to all valid nodes those are directly connected.
void handle_heartbeat(execution_unit* ctx);
......@@ -206,12 +159,6 @@ public:
size_t remove_published_actor(const actor_addr& whom, uint16_t port,
removed_published_actor* cb = nullptr);
/// Compare two sequence numbers
static bool is_greater(sequence_type lhs, sequence_type rhs,
sequence_type max_distance
= std::numeric_limits<sequence_type>::max() / 2);
/// Returns `true` if a path to destination existed, `false` otherwise.
bool dispatch(execution_unit* ctx, const strong_actor_ptr& sender,
const std::vector<strong_actor_ptr>& forwarding_stack,
......@@ -237,7 +184,7 @@ public:
/// Returns the current mapping of ports to addresses
/// and interfaces of published actors.
inline const published_actor_map& published_actors() const {
const published_actor_map& published_actors() const {
return published_actors_;
}
......@@ -250,38 +197,33 @@ public:
/// if no actor is published at this port then a standard handshake is
/// written (e.g. used when establishing direct connections on-the-fly).
void write_server_handshake(execution_unit* ctx,
buffer_type& out_buf, optional<uint16_t> port,
uint16_t sequence_number = 0);
buffer_type& out_buf, optional<uint16_t> port);
/// Writes the client handshake to `buf`.
static void write_client_handshake(execution_unit* ctx,
buffer_type& buf,
const node_id& remote_side,
const node_id& this_node,
const std::string& app_identifier,
uint16_t sequence_number = 0);
const std::string& app_identifier);
/// Writes the client handshake to `buf`.
void write_client_handshake(execution_unit* ctx,
buffer_type& buf, const node_id& remote_side,
uint16_t sequence_number = 0);
buffer_type& buf, const node_id& remote_side);
/// Writes an `announce_proxy` to `buf`.
void write_announce_proxy(execution_unit* ctx, buffer_type& buf,
const node_id& dest_node, actor_id aid,
uint16_t sequence_number = 0);
const node_id& dest_node, actor_id aid);
/// Writes a `kill_proxy` to `buf`.
void write_kill_proxy(execution_unit* ctx, buffer_type& buf,
const node_id& dest_node, actor_id aid,
const error& rsn, uint16_t sequence_number = 0);
const error& rsn);
/// Writes a `heartbeat` to `buf`.
void write_heartbeat(execution_unit* ctx,
buffer_type& buf, const node_id& remote_side,
uint16_t sequence_number = 0);
void write_heartbeat(execution_unit* ctx, buffer_type& buf,
const node_id& remote_side);
inline const node_id& this_node() const {
const node_id& this_node() const {
return this_node_;
}
......@@ -291,186 +233,12 @@ public:
system().middleman().template notify<Event>(std::forward<Ts>(xs)...);
}
inline actor_system& system() {
actor_system& system() {
return callee_.system();
}
template <class Handle>
bool handle(execution_unit* ctx, const Handle& hdl, header& hdr,
std::vector<char>* payload, bool tcp_based,
optional<endpoint_context&> ep, optional<uint16_t> port) {
// function object for checking payload validity
auto payload_valid = [&]() -> bool {
return payload != nullptr && payload->size() == hdr.payload_len;
};
// handle message to ourselves
switch (hdr.operation) {
case message_type::server_handshake: {
actor_id aid = invalid_actor_id;
std::set<std::string> sigs;
if (!payload_valid()) {
CAF_LOG_ERROR("fail to receive the app identifier");
return false;
} else {
binary_deserializer bd{ctx, *payload};
std::string remote_appid;
auto e = bd(remote_appid);
if (e)
return false;
auto appid = get_if<std::string>(&callee_.config(),
"middleman.app-identifier");
if ((appid && *appid != remote_appid)
|| (!appid && !remote_appid.empty())) {
CAF_LOG_ERROR("app identifier mismatch");
return false;
}
e = bd(aid, sigs);
if (e)
return false;
}
// close self connection after handshake is done
if (hdr.source_node == this_node_) {
CAF_LOG_DEBUG("close connection to self immediately");
callee_.finalize_handshake(hdr.source_node, aid, sigs);
return false;
}
// close this connection if we already have a direct connection
if (tbl_.lookup_direct(hdr.source_node)) {
CAF_LOG_DEBUG("close connection since we already have a "
"direct connection: " << CAF_ARG(hdr.source_node));
callee_.finalize_handshake(hdr.source_node, aid, sigs);
return false;
}
// add direct route to this node and remove any indirect entry
CAF_LOG_DEBUG("new direct connection:" << CAF_ARG(hdr.source_node));
tbl_.add_direct(hdl, hdr.source_node);
auto was_indirect = tbl_.erase_indirect(hdr.source_node);
// write handshake as client in response
auto path = tbl_.lookup(hdr.source_node);
if (!path) {
CAF_LOG_ERROR("no route to host after server handshake");
return false;
}
if (tcp_based) {
auto ch = get<connection_handle>(path->hdl);
write_client_handshake(ctx, callee_.get_buffer(ch),
hdr.source_node);
}
callee_.learned_new_node_directly(hdr.source_node, was_indirect);
callee_.finalize_handshake(hdr.source_node, aid, sigs);
flush(*path);
break;
}
case message_type::client_handshake: {
if (!payload_valid()) {
CAF_LOG_ERROR("fail to receive the app identifier");
return false;
} else {
binary_deserializer bd{ctx, *payload};
std::string remote_appid;
auto e = bd(remote_appid);
if (e)
return false;
auto appid = get_if<std::string>(&callee_.config(),
"middleman.app-identifier");
if ((appid && *appid != remote_appid)
|| (!appid && !remote_appid.empty())) {
CAF_LOG_ERROR("app identifier mismatch");
return false;
}
}
if (tcp_based) {
if (tbl_.lookup_direct(hdr.source_node)) {
CAF_LOG_DEBUG("received second client handshake:"
<< CAF_ARG(hdr.source_node));
break;
}
// add direct route to this node and remove any indirect entry
CAF_LOG_DEBUG("new direct connection:" << CAF_ARG(hdr.source_node));
tbl_.add_direct(hdl, hdr.source_node);
auto was_indirect = tbl_.erase_indirect(hdr.source_node);
callee_.learned_new_node_directly(hdr.source_node, was_indirect);
} else {
auto new_node = (this_node() != hdr.source_node
&& !tbl_.lookup_direct(hdr.source_node));
if (new_node) {
// add direct route to this node and remove any indirect entry
CAF_LOG_DEBUG("new direct connection:" << CAF_ARG(hdr.source_node));
tbl_.add_direct(hdl, hdr.source_node);
}
uint16_t seq = (ep && ep->requires_ordering) ? ep->seq_outgoing++ : 0;
write_server_handshake(ctx,
callee_.get_buffer(hdl),
port, seq);
callee_.flush(hdl);
if (new_node) {
auto was_indirect = tbl_.erase_indirect(hdr.source_node);
callee_.learned_new_node_directly(hdr.source_node, was_indirect);
}
}
break;
}
case message_type::dispatch_message: {
if (!payload_valid())
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 (hdr.source_node != none
&& hdr.source_node != this_node_
&& last_hop != hdr.source_node
&& !tbl_.lookup_direct(hdr.source_node)
&& tbl_.add_indirect(last_hop, hdr.source_node))
callee_.learned_new_node_indirectly(hdr.source_node);
binary_deserializer bd{ctx, *payload};
auto receiver_name = static_cast<atom_value>(0);
std::vector<strong_actor_ptr> forwarding_stack;
message msg;
if (hdr.has(header::named_receiver_flag)) {
auto e = bd(receiver_name);
if (e)
return false;
}
auto e = bd(forwarding_stack, msg);
if (e)
return false;
CAF_LOG_DEBUG(CAF_ARG(forwarding_stack) << CAF_ARG(msg));
if (hdr.has(header::named_receiver_flag))
callee_.deliver(hdr.source_node, hdr.source_actor, receiver_name,
make_message_id(hdr.operation_data),
forwarding_stack, msg);
else
callee_.deliver(hdr.source_node, hdr.source_actor, hdr.dest_actor,
make_message_id(hdr.operation_data),
forwarding_stack, msg);
break;
}
case message_type::announce_proxy:
callee_.proxy_announced(hdr.source_node, hdr.dest_actor);
break;
case message_type::kill_proxy: {
if (!payload_valid())
return false;
binary_deserializer bd{ctx, *payload};
error fail_state;
auto e = bd(fail_state);
if (e)
return false;
callee_.proxies().erase(hdr.source_node, hdr.source_actor,
std::move(fail_state));
break;
}
case message_type::heartbeat: {
CAF_LOG_TRACE("received heartbeat: " << CAF_ARG(hdr.source_node));
callee_.handle_heartbeat(hdr.source_node);
break;
}
default:
CAF_LOG_ERROR("invalid operation");
return false;
}
return true;
}
bool handle(execution_unit* ctx, connection_handle hdl, header& hdr,
std::vector<char>* payload);
private:
routing_table tbl_;
......
......@@ -21,25 +21,10 @@
#include <unordered_map>
#include <unordered_set>
#include "caf/node_id.hpp"
#include "caf/callback.hpp"
#include "caf/io/visitors.hpp"
#include "caf/io/abstract_broker.hpp"
#include "caf/io/basp/buffer_type.hpp"
namespace std {
template<>
struct hash<caf::variant<caf::io::connection_handle,caf::io::datagram_handle>> {
size_t operator()(const caf::variant<caf::io::connection_handle,
caf::io::datagram_handle>& hdl) const {
return caf::visit(caf::io::hash_visitor{}, hdl);
}
};
} // namespace std
#include "caf/node_id.hpp"
namespace caf {
namespace io {
......@@ -51,7 +36,6 @@ namespace basp {
/// BASP peer and provides both direct and indirect paths.
class routing_table {
public:
using endpoint_handle = variant<connection_handle, datagram_handle>;
explicit routing_table(abstract_broker* parent);
......@@ -60,7 +44,7 @@ public:
/// Describes a routing path to a node.
struct route {
const node_id& next_hop;
endpoint_handle hdl;
connection_handle hdl;
};
/// Describes a function object for erase operations that
......@@ -72,11 +56,11 @@ public:
/// Returns the ID of the peer connected via `hdl` or
/// `none` if `hdl` is unknown.
node_id lookup_direct(const endpoint_handle& hdl) const;
node_id lookup_direct(const connection_handle& hdl) const;
/// Returns the handle offering a direct connection to `nid` or
/// `invalid_connection_handle` if no direct connection to `nid` exists.
optional<endpoint_handle> lookup_direct(const node_id& nid) const;
optional<connection_handle> lookup_direct(const node_id& nid) const;
/// Returns the next hop that would be chosen for `nid`
/// or `none` if there's no indirect route to `nid`.
......@@ -84,7 +68,7 @@ public:
/// Adds a new direct route to the table.
/// @pre `hdl != invalid_connection_handle && nid != none`
void add_direct(const endpoint_handle& hdl, const node_id& nid);
void add_direct(const connection_handle& hdl, const node_id& nid);
/// Adds a new indirect route to the table.
bool add_indirect(const node_id& hop, const node_id& dest);
......@@ -95,7 +79,7 @@ public:
/// 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 endpoint_handle& hdl, erase_callback& cb);
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`.
......@@ -131,8 +115,8 @@ public:
node_id_set>; // hop
abstract_broker* parent_;
std::unordered_map<endpoint_handle, node_id> direct_by_hdl_;
std::unordered_map<node_id, endpoint_handle> direct_by_nid_;
std::unordered_map<connection_handle, node_id> direct_by_hdl_;
std::unordered_map<node_id, connection_handle> direct_by_nid_;
indirect_entries indirect_;
indirect_entries blacklist_;
};
......
......@@ -35,11 +35,8 @@
#include "caf/io/basp/all.hpp"
#include "caf/io/broker.hpp"
#include "caf/io/visitors.hpp"
#include "caf/io/typed_broker.hpp"
#include "caf/io/basp/endpoint_context.hpp"
namespace caf {
namespace io {
......@@ -89,42 +86,9 @@ struct basp_broker_state : proxy_registry::backend, basp::instance::callee {
// inherited from basp::instance::callee
void learned_new_node_indirectly(const node_id& nid) override;
// inherited from basp::instance::callee
uint16_t next_sequence_number(connection_handle hdl) override;
// inherited from basp::instance::callee
uint16_t next_sequence_number(datagram_handle hdl) override;
// inherited from basp::instance::callee
void add_pending(execution_unit* ctx, basp::endpoint_context& ep,
uint16_t seq, basp::header hdr,
std::vector<char> payload) override;
// inherited from basp::instance::callee
bool deliver_pending(execution_unit* ctx, basp::endpoint_context& ep,
bool force) override;
// inherited from basp::instance::callee
void drop_pending(basp::endpoint_context& ep, uint16_t seq) override;
// inherited from basp::instance::callee
buffer_type& get_buffer(endpoint_handle hdl) override;
// inherited from basp::instance::callee
buffer_type& get_buffer(datagram_handle hdl) override;
// inherited from basp::instance::callee
buffer_type& get_buffer(connection_handle hdl) override;
// inherited from basp::instance::callee
buffer_type pop_datagram_buffer(datagram_handle hdl) override;
// inherited from basp::instance::callee
void flush(endpoint_handle hdl) override;
// inherited from basp::instance::callee
void flush(datagram_handle hdl) override;
// inherited from basp::instance::callee
void flush(connection_handle hdl) override;
......@@ -134,11 +98,9 @@ struct basp_broker_state : proxy_registry::backend, basp::instance::callee {
/// Sets `this_context` by either creating or accessing state for `hdl`.
void set_context(connection_handle hdl);
void set_context(datagram_handle hdl);
/// Cleans up any state for `hdl`.
void cleanup(connection_handle hdl);
void cleanup(datagram_handle hdl);
// pointer to ourselves
broker* self;
......@@ -146,14 +108,10 @@ struct basp_broker_state : proxy_registry::backend, basp::instance::callee {
// protocol instance of BASP
basp::instance instance;
using ctx_tcp_map = std::unordered_map<connection_handle,
basp::endpoint_context>;
using ctx_udp_map = std::unordered_map<datagram_handle,
basp::endpoint_context>;
using ctx_map = std::unordered_map<connection_handle, basp::endpoint_context>;
// keeps context information for all open connections
ctx_tcp_map ctx_tcp;
ctx_udp_map ctx_udp;
ctx_map ctx;
// points to the current context for callbacks such as `make_proxy`
basp::endpoint_context* this_context = nullptr;
......@@ -167,19 +125,6 @@ struct basp_broker_state : proxy_registry::backend, basp::instance::callee {
/// routing paths by forming a mesh between all nodes.
bool automatic_connections = false;
/// Configures whether BASP allows TCP connections.
bool allow_tcp = true;
/// Configures whether BASP allows UDP connections.
bool allow_udp = false;
// reusable send buffers for UDP communication
const size_t max_buffers;
std::stack<buffer_type> cached_buffers;
// maximum queue size for pending messages of endpoints with ordering
const size_t max_pending_messages;
// timeout for delivery of pending messages of endpoints with ordering
const std::chrono::milliseconds pending_to = std::chrono::milliseconds(100);
......
......@@ -76,22 +76,6 @@ public:
system().message_types(tk), port, in, reuse);
}
/// Tries to publish `whom` at `port` and returns either an
/// `error` or the bound port.
/// @param whom Actor that should be published at `port`.
/// @param port Unused UDP port.
/// @param in The IP address to listen to or `INADDR_ANY` if `in == nullptr`.
/// @param reuse Create socket using `SO_REUSEADDR`.
/// @returns The actual port the OS uses after `bind()`. If `port == 0`
/// the OS chooses a random high-level port.
template <class Handle>
expected<uint16_t> publish_udp(Handle&& whom, uint16_t port,
const char* in = nullptr, bool reuse = false) {
detail::type_list<typename std::decay<Handle>::type> tk;
return publish_udp(actor_cast<strong_actor_ptr>(std::forward<Handle>(whom)),
system().message_types(tk), port, in, reuse);
}
/// Makes *all* local groups accessible via network
/// on address `addr` and `port`.
/// @returns The actual port the OS uses after `bind()`. If `port == 0`
......@@ -108,14 +92,6 @@ public:
return unpublish(whom.address(), port);
}
/// Unpublishes `whom` by closing `port` or all assigned ports if `port == 0`.
/// @param whom Actor that should be unpublished at `port`.
/// @param port UDP port.
template <class Handle>
expected<void> unpublish_udp(const Handle& whom, uint16_t port = 0) {
return unpublish_udp(whom.address(), port);
}
/// Establish a new connection to the actor at `host` on given `port`.
/// @param host Valid hostname or IP address.
/// @param port TCP port.
......@@ -131,21 +107,6 @@ public:
return actor_cast<ActorHandle>(std::move(*x));
}
/// Contacts the actor at `host` on given `port`.
/// @param host Valid hostname or IP address.
/// @param port UDP port.
/// @returns An `actor` to the proxy instance representing
/// a remote actor or an `error`.
template <class ActorHandle = actor>
expected<ActorHandle> remote_actor_udp(std::string host, uint16_t port) {
detail::type_list<ActorHandle> tk;
auto x = remote_actor_udp(system().message_types(tk), std::move(host), port);
if (!x)
return x.error();
CAF_ASSERT(x && *x);
return actor_cast<ActorHandle>(std::move(*x));
}
/// <group-name>@<host>:<port>
expected<group> remote_group(const std::string& group_uri);
......@@ -372,20 +333,11 @@ private:
std::set<std::string> sigs,
uint16_t port, const char* cstr, bool ru);
expected<uint16_t> publish_udp(const strong_actor_ptr& whom,
std::set<std::string> sigs,
uint16_t port, const char* cstr, bool ru);
expected<void> unpublish(const actor_addr& whom, uint16_t port);
expected<void> unpublish_udp(const actor_addr& whom, uint16_t port);
expected<strong_actor_ptr> remote_actor(std::set<std::string> ifs,
std::string host, uint16_t port);
expected<strong_actor_ptr> remote_actor_udp(std::set<std::string> ifs,
std::string host, uint16_t port);
static int exec_slave_mode(actor_system&, const actor_system_config&);
// environment
......
......@@ -68,12 +68,6 @@ namespace io {
/// (unpublish_atom, strong_actor_ptr whom, uint16_t port)
/// -> void
///
/// // Closes `port` if it is mapped to `whom`.
/// // whom: A published actor.
/// // port: Used UDP port.
/// (unpublish_udp_atom, strong_actor_ptr whom, uint16_t port)
/// -> void
///
/// // Unconditionally closes `port`, removing any actor
/// // published at this port.
/// // port: Used TCP port.
......@@ -98,23 +92,14 @@ using middleman_actor =
::with<uint16_t>,
replies_to<publish_udp_atom, uint16_t, strong_actor_ptr,
std::set<std::string>, std::string, bool>
::with<uint16_t>,
replies_to<open_atom, uint16_t, std::string, bool>
::with<uint16_t>,
replies_to<connect_atom, std::string, uint16_t>
::with<node_id, strong_actor_ptr, std::set<std::string>>,
replies_to<contact_atom, std::string, uint16_t>
::with<node_id, strong_actor_ptr, std::set<std::string>>,
reacts_to<unpublish_atom, actor_addr, uint16_t>,
reacts_to<unpublish_udp_atom, actor_addr, uint16_t>,
reacts_to<close_atom, uint16_t>,
replies_to<spawn_atom, node_id, std::string, message, std::set<std::string>>
......
/******************************************************************************
* ____ _ _____ *
* / ___| / \ | ___| C++ *
* | | / _ \ | |_ Actor *
* | |___ / ___ \| _| Framework *
* \____/_/ \_|_| *
* *
* Copyright (C) 2011 - 2016 *
* *
* 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 "caf/io/abstract_broker.hpp"
namespace caf {
namespace io {
struct addr_visitor {
using result_type = std::string;
addr_visitor(abstract_broker* ptr) : ptr_(ptr) { }
template <class Handle>
result_type operator()(const Handle& hdl) { return ptr_->remote_addr(hdl); }
abstract_broker* ptr_;
};
struct port_visitor {
using result_type = uint16_t;
port_visitor(abstract_broker* ptr) : ptr_(ptr) { }
template <class Handle>
result_type operator()(const Handle& hdl) { return ptr_->remote_port(hdl); }
abstract_broker* ptr_;
};
struct id_visitor {
using result_type = int64_t;
template <class Handle>
result_type operator()(const Handle& hdl) { return hdl.id(); }
};
struct hash_visitor {
using result_type = size_t;
template <class Handle>
result_type operator()(const Handle& hdl) const {
std::hash<Handle> f;
return f(hdl);
}
};
} // namespace io
} // namespace caf
......@@ -88,9 +88,11 @@ void abstract_broker::ack_writes(connection_handle hdl, bool enable) {
}
std::vector<char>& abstract_broker::wr_buf(connection_handle hdl) {
CAF_ASSERT(hdl != invalid_connection_handle);
auto x = by_id(hdl);
if (!x) {
CAF_LOG_ERROR("tried to access wr_buf() of an unknown connection_handle");
CAF_LOG_ERROR("tried to access wr_buf() of an unknown connection_handle:"
<< CAF_ARG(hdl));
return dummy_wr_buf_;
}
return x->wr_buf();
......
......@@ -39,21 +39,6 @@
namespace caf {
namespace io {
namespace {
// visitors to access handle variant of the context
struct seq_num_visitor {
using result_type = basp::sequence_type;
seq_num_visitor(basp_broker_state* ptr) : state(ptr) { }
template <class T>
result_type operator()(const T& hdl) {
return state->next_sequence_number(hdl);
}
basp_broker_state* state;
};
} // namespace anonymous
const char* basp_broker_state::name = "basp_broker";
/******************************************************************************
......@@ -64,11 +49,7 @@ basp_broker_state::basp_broker_state(broker* selfptr)
: basp::instance::callee(selfptr->system(),
static_cast<proxy_registry::backend&>(*this)),
self(selfptr),
instance(selfptr, *this),
max_buffers(get_or(self->config(), "middleman.cached-udp-buffers",
defaults::middleman::cached_udp_buffers)),
max_pending_messages(get_or(self->config(), "middleman.max-pending-msgs",
defaults::middleman::max_pending_msgs)) {
instance(selfptr, *this) {
CAF_ASSERT(this_node() != none);
}
......@@ -119,13 +100,11 @@ strong_actor_ptr basp_broker_state::make_proxy(node_id nid, actor_id aid) {
});
CAF_LOG_DEBUG("successfully created proxy instance, "
"write announce_proxy_instance:"
<< CAF_ARG(nid) << CAF_ARG(aid));
auto& ctx = *this_context;
<< CAF_ARG(nid) << CAF_ARG(aid)
<< CAF_ARG2("hdl", this_context->hdl));
// tell remote side we are monitoring this actor now
instance.write_announce_proxy(self->context(),
get_buffer(this_context->hdl),
nid, aid,
ctx.requires_ordering ? ctx.seq_outgoing++ : 0);
instance.write_announce_proxy(self->context(), get_buffer(this_context->hdl),
nid, aid);
instance.flush(*path);
mm->notify<hook::new_remote_actor>(res);
return res;
......@@ -177,10 +156,8 @@ void basp_broker_state::send_kill_proxy_instance(const node_id& nid,
<< CAF_ARG(nid));
return;
}
instance.write_kill_proxy(self->context(),
get_buffer(path->hdl),
nid, aid, rsn,
visit(seq_num_visitor{this}, path->hdl));
instance.write_kill_proxy(self->context(), get_buffer(path->hdl), nid, aid,
rsn);
instance.flush(*path);
}
......@@ -363,8 +340,7 @@ void basp_broker_state::learned_new_node(const node_id& nid) {
basp::header hdr{basp::message_type::dispatch_message,
basp::header::named_receiver_flag,
0, make_message_id().integer_value(), this_node(), nid,
tmp.id(), invalid_actor_id,
visit(seq_num_visitor{this}, path->hdl)};
tmp.id(), invalid_actor_id};
// writing std::numeric_limits<actor_id>::max() is a hack to get
// this send-to-named-actor feature working with older CAF releases
instance.write(self->context(), get_buffer(path->hdl),
......@@ -414,54 +390,25 @@ void basp_broker_state::learned_new_node_indirectly(const node_id& nid) {
basp::header hdr{basp::message_type::dispatch_message,
basp::header::named_receiver_flag,
0, make_message_id().integer_value(), this_node(), nid,
tmp.id(), invalid_actor_id,
visit(seq_num_visitor{this}, path->hdl)};
tmp.id(), invalid_actor_id};
instance.write(self->context(), get_buffer(path->hdl),
hdr, &writer);
instance.flush(*path);
};
if (allow_tcp)
try_connect("basp.default-connectivity-tcp");
if (allow_udp)
try_connect("basp.default-connectivity-udp");
}
void basp_broker_state::set_context(connection_handle hdl) {
CAF_LOG_TRACE(CAF_ARG(hdl));
auto i = ctx_tcp.find(hdl);
if (i == ctx_tcp.end()) {
CAF_LOG_DEBUG("create new BASP context:" << CAF_ARG(hdl));
i = ctx_tcp.emplace(
hdl,
basp::endpoint_context{
basp::await_header,
basp::header{basp::message_type::server_handshake, 0,
0, 0, none, none,
invalid_actor_id, invalid_actor_id},
hdl, none, 0, 0, none, false, 0, 0,
basp::endpoint_context::pending_map(), false
}
).first;
}
this_context = &i->second;
}
void basp_broker_state::set_context(datagram_handle hdl) {
CAF_LOG_TRACE(CAF_ARG(hdl));
auto i = ctx_udp.find(hdl);
if (i == ctx_udp.end()) {
auto i = ctx.find(hdl);
if (i == ctx.end()) {
CAF_LOG_DEBUG("create new BASP context:" << CAF_ARG(hdl));
i = ctx_udp.emplace(
hdl,
basp::endpoint_context{
basp::await_header,
basp::header{basp::message_type::server_handshake,
0, 0, 0, none, none,
invalid_actor_id, invalid_actor_id},
hdl, none, 0, 0, none, true, 0, 0,
basp::endpoint_context::pending_map(), false
}
).first;
basp::header hdr{basp::message_type::server_handshake, 0, 0, 0, none, none,
invalid_actor_id, invalid_actor_id};
i = ctx
.emplace(hdl, basp::endpoint_context{basp::await_header, hdr, hdl,
none, 0, 0, none})
.first;
}
this_context = &i->second;
}
......@@ -477,142 +424,23 @@ void basp_broker_state::cleanup(connection_handle hdl) {
instance.tbl().erase_direct(hdl, cb);
// Remove the context for `hdl`, making sure clients receive an error in case
// this connection was closed during handshake.
auto i = ctx_tcp.find(hdl);
if (i != ctx_tcp.end()) {
auto& ref = i->second;
CAF_ASSERT(i->first == get<connection_handle>(ref.hdl));
if (ref.callback) {
CAF_LOG_DEBUG("connection closed during handshake");
ref.callback->deliver(sec::disconnect_during_handshake);
}
ctx_tcp.erase(i);
}
}
void basp_broker_state::cleanup(datagram_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.
auto cb = make_callback([&](const node_id& nid) -> error {
purge_state(nid);
return none;
});
instance.tbl().erase_direct(hdl, cb);
// Remove the context for `hdl`, making sure clients receive an error in case
// this connection was closed during handshake.
auto i = ctx_udp.find(hdl);
if (i != ctx_udp.end()) {
auto i = ctx.find(hdl);
if (i != ctx.end()) {
auto& ref = i->second;
CAF_ASSERT(i->first == get<datagram_handle>(ref.hdl));
CAF_ASSERT(i->first == ref.hdl);
if (ref.callback) {
CAF_LOG_DEBUG("connection closed during handshake");
ref.callback->deliver(sec::disconnect_during_handshake);
}
ctx_udp.erase(i);
ctx.erase(i);
}
}
basp::sequence_type basp_broker_state::next_sequence_number(connection_handle) {
return 0;
}
basp::sequence_type
basp_broker_state::next_sequence_number(datagram_handle hdl) {
auto i = ctx_udp.find(hdl);
if (i != ctx_udp.end() && i->second.requires_ordering)
return i->second.seq_outgoing++;
return 0;
}
void basp_broker_state::add_pending(execution_unit* ctx,
basp::endpoint_context& ep,
basp::sequence_type seq,
basp::header hdr,
std::vector<char> payload) {
if (!ep.requires_ordering)
return;
ep.pending.emplace(seq, std::make_pair(std::move(hdr), std::move(payload)));
if (ep.pending.size() >= max_pending_messages)
deliver_pending(ctx, ep, true);
else if (!ep.did_set_timeout)
self->delayed_send(self, pending_to, pending_atom::value,
get<datagram_handle>(ep.hdl));
}
bool basp_broker_state::deliver_pending(execution_unit* ctx,
basp::endpoint_context& ep,
bool force) {
if (!ep.requires_ordering || ep.pending.empty())
return true;
std::vector<char>* payload = nullptr;
auto i = ep.pending.begin();
// Force delivery of at least the first messages, if desired.
if (force)
ep.seq_incoming = i->first;
while (i != ep.pending.end() && i->first == ep.seq_incoming) {
ep.hdr = std::move(i->second.first);
payload = &i->second.second;
if (!instance.handle(ctx, get<datagram_handle>(ep.hdl),
ep.hdr, payload, false, ep, none))
return false;
i = ep.pending.erase(i);
ep.seq_incoming += 1;
}
// Set a timeout if there are still pending messages.
if (!ep.pending.empty() && !ep.did_set_timeout)
self->delayed_send(self, pending_to, pending_atom::value,
get<datagram_handle>(ep.hdl));
return true;
}
void basp_broker_state::drop_pending(basp::endpoint_context& ep,
basp::sequence_type seq) {
if (!ep.requires_ordering)
return;
ep.pending.erase(seq);
}
basp_broker_state::buffer_type&
basp_broker_state::get_buffer(endpoint_handle hdl) {
if (hdl.is<connection_handle>())
return get_buffer(get<connection_handle>(hdl));
else
return get_buffer(get<datagram_handle>(hdl));
}
basp_broker_state::buffer_type&
basp_broker_state::get_buffer(datagram_handle) {
if (cached_buffers.empty())
cached_buffers.emplace();
return cached_buffers.top();
}
basp_broker_state::buffer_type&
basp_broker_state::get_buffer(connection_handle hdl) {
return self->wr_buf(hdl);
}
basp_broker_state::buffer_type
basp_broker_state::pop_datagram_buffer(datagram_handle) {
std::vector<char> res;
std::swap(res, cached_buffers.top());
cached_buffers.pop();
return res;
}
void basp_broker_state::flush(endpoint_handle hdl) {
if (hdl.is<connection_handle>())
flush(get<connection_handle>(hdl));
else
flush(get<datagram_handle>(hdl));
}
void basp_broker_state::flush(datagram_handle hdl) {
if (!cached_buffers.empty() && !cached_buffers.top().empty())
self->enqueue_datagram(hdl, pop_datagram_buffer(hdl));
self->flush(hdl);
}
void basp_broker_state::flush(connection_handle hdl) {
self->flush(hdl);
}
......@@ -630,13 +458,10 @@ basp_broker::basp_broker(actor_config& cfg)
behavior basp_broker::make_behavior() {
CAF_LOG_TRACE(CAF_ARG(system().node()));
state.allow_tcp = !get_or(config(), "middleman.disable-tcp", false);
state.allow_udp = get_or(config(), "middleman.enable-udp", false);
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
if (state.allow_tcp) {
auto res = add_tcp_doorman(uint16_t{0});
if (res) {
auto port = res->second;
......@@ -646,18 +471,6 @@ behavior basp_broker::make_behavior() {
"basp.default-connectivity-tcp",
make_message(port, std::move(addrs)));
}
}
if (state.allow_udp) {
auto res = add_udp_datagram_servant(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-udp",
make_message(port, std::move(addrs)));
}
}
state.automatic_connections = true;
}
auto heartbeat_interval = get_or(config(), "middleman.heartbeat-interval",
......@@ -687,43 +500,6 @@ behavior basp_broker::make_behavior() {
ctx.cstate = next;
}
},
// received from auto connect broker for UDP communication
[=](new_datagram_msg& msg, datagram_servant_ptr ptr, uint16_t port) {
CAF_LOG_TRACE(CAF_ARG(msg.handle));
auto hdl = ptr->hdl();
move_datagram_servant(ptr);
auto& ctx = state.ctx_udp[hdl];
ctx.hdl = hdl;
ctx.remote_port = port;
ctx.local_port = local_port(hdl);
ctx.requires_ordering = true;
ctx.seq_incoming = 0;
ctx.seq_outgoing = 1; // already sent the client handshake
// Let's not implement this twice
send(this, std::move(msg));
},
// received from underlying broker implementation
[=](new_datagram_msg& msg) {
CAF_LOG_TRACE(CAF_ARG(msg.handle));
state.set_context(msg.handle);
auto& ctx = *state.this_context;
if (ctx.local_port == 0)
ctx.local_port = local_port(msg.handle);
if (!state.instance.handle(context(), msg, ctx)) {
if (ctx.callback) {
CAF_LOG_WARNING("failed to handshake with remote node"
<< CAF_ARG(msg.handle));
ctx.callback->deliver(make_error(sec::disconnect_during_handshake));
}
state.cleanup(msg.handle);
close(msg.handle);
}
},
// received from the underlying broker implementation
[=](datagram_sent_msg& msg) {
if (state.cached_buffers.size() < state.max_buffers)
state.cached_buffers.emplace(std::move(msg.buf));
},
// received from proxy instances
[=](forward_atom, strong_actor_ptr& src,
const std::vector<strong_actor_ptr>& fwd_stack,
......@@ -770,8 +546,7 @@ behavior basp_broker::make_behavior() {
basp::header hdr{basp::message_type::dispatch_message,
basp::header::named_receiver_flag,
0, cme->mid.integer_value(), state.this_node(),
dest_node, src->id(), invalid_actor_id,
visit(seq_num_visitor{&state}, path->hdl)};
dest_node, src->id(), invalid_actor_id};
state.instance.write(context(), state.get_buffer(path->hdl),
hdr, &writer);
state.instance.flush(*path);
......@@ -815,53 +590,14 @@ behavior basp_broker::make_behavior() {
auto rp = make_response_promise();
auto hdl = ptr->hdl();
add_scribe(std::move(ptr));
auto& ctx = state.ctx_tcp[hdl];
auto& ctx = state.ctx[hdl];
ctx.hdl = hdl;
ctx.remote_port = port;
ctx.cstate = basp::await_header;
ctx.callback = rp;
ctx.requires_ordering = false;
// await server handshake
configure_read(hdl, receive_policy::exactly(basp::header_size));
},
[=](publish_udp_atom, datagram_servant_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_datagram_servant(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)
[=](contact_atom, datagram_servant_ptr& ptr, uint16_t port) {
CAF_LOG_TRACE(CAF_ARG(ptr) << CAF_ARG(port));
auto rp = make_response_promise();
auto hdl = ptr->hdl();
add_datagram_servant(std::move(ptr));
auto& ctx = state.ctx_udp[hdl];
ctx.hdl = hdl;
ctx.remote_port = port;
ctx.local_port = local_port(hdl);
ctx.callback = rp;
ctx.requires_ordering = true;
ctx.seq_incoming = 0;
ctx.seq_outgoing = 0;
auto& bi = state.instance;
bi.write_client_handshake(context(), state.get_buffer(hdl),
none, ctx.seq_outgoing++);
state.flush(hdl);
},
// received from underlying broker implementation
[=](const datagram_servant_closed_msg& msg) {
CAF_LOG_TRACE("");
// since all handles share a port, we can take any of them to query for
// port information
CAF_ASSERT(msg.handles.size() > 0);
auto port = local_port(msg.handles.front());
state.instance.remove_published_actor(port);
},
[=](delete_atom, const node_id& nid, actor_id aid) {
CAF_LOG_TRACE(CAF_ARG(nid) << ", " << CAF_ARG(aid));
state.proxies().erase(nid, aid);
......@@ -878,18 +614,6 @@ behavior basp_broker::make_behavior() {
return sec::no_actor_published_at_port;
return unit;
},
[=](unpublish_udp_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(datagram_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;
......@@ -907,8 +631,8 @@ behavior basp_broker::make_behavior() {
uint16_t port = 0;
auto hdl = state.instance.tbl().lookup_direct(x);
if (hdl) {
addr = visit(addr_visitor{this}, *hdl);
port = visit(port_visitor{this}, *hdl);
addr = remote_addr(*hdl);
port = remote_port(*hdl);
}
return std::make_tuple(x, std::move(addr), port);
},
......@@ -916,21 +640,6 @@ behavior basp_broker::make_behavior() {
state.instance.handle_heartbeat(context());
delayed_send(this, std::chrono::milliseconds{interval},
tick_atom::value, interval);
},
[=](pending_atom, datagram_handle hdl) {
auto& ep = state.ctx_udp[hdl];
ep.did_set_timeout = false;
if (ep.pending.empty())
return;
auto i = ep.pending.begin();
auto seq = i->first;
if (seq == ep.seq_incoming ||
basp::instance::is_greater(seq, ep.seq_incoming)) {
// Skip missing messages and force delivery.
state.deliver_pending(context(), ep, true);
} else {
state.drop_pending(ep, seq);
}
}
};
}
......
......@@ -43,8 +43,7 @@ std::string to_string(const header &hdr) {
<< to_string(hdr.source_node) << ", "
<< to_string(hdr.dest_node) << ", "
<< hdr.source_actor << ", "
<< hdr.dest_actor << ", "
<< hdr.sequence_number
<< hdr.dest_actor
<< "}";
return oss.str();
}
......@@ -57,8 +56,7 @@ bool operator==(const header& lhs, const header& rhs) {
&& lhs.source_node == rhs.source_node
&& lhs.dest_node == rhs.dest_node
&& lhs.source_actor == rhs.source_actor
&& lhs.dest_actor == rhs.dest_actor
&& lhs.sequence_number == rhs.sequence_number;
&& lhs.dest_actor == rhs.dest_actor;
}
namespace {
......
......@@ -29,20 +29,6 @@ namespace caf {
namespace io {
namespace basp {
namespace {
struct seq_num_visitor {
using result_type = uint16_t;
seq_num_visitor(instance::callee& c) : cal(c) { }
template <class T>
result_type operator()(const T& hdl) {
return cal.next_sequence_number(hdl);
}
instance::callee& cal;
};
} // namespace <anonymous>
instance::callee::callee(actor_system& sys, proxy_registry::backend& backend)
: namespace_(sys, backend) {
// nop
......@@ -123,103 +109,16 @@ connection_state instance::handle(execution_unit* ctx,
}
return await_header;
}
if (!handle(ctx, dm.handle, hdr, payload, true, none, none))
if (!handle(ctx, dm.handle, hdr, payload))
return err();
return await_header;
}
bool instance::handle(execution_unit* ctx, new_datagram_msg& dm,
endpoint_context& ep) {
using itr_t = network::receive_buffer::iterator;
// function object providing cleanup code on errors
auto err = [&]() -> bool {
auto cb = make_callback([&](const node_id& nid) -> error {
callee_.purge_state(nid);
return none;
});
tbl_.erase_direct(dm.handle, cb);
return false;
};
// extract payload
std::vector<char> pl_buf{std::move_iterator<itr_t>(std::begin(dm.buf) +
basp::header_size),
std::move_iterator<itr_t>(std::end(dm.buf))};
// resize header
dm.buf.resize(basp::header_size);
// extract header
binary_deserializer bd{ctx, dm.buf};
auto e = bd(ep.hdr);
if (e || !valid(ep.hdr)) {
CAF_LOG_WARNING("received invalid header:" << CAF_ARG(ep.hdr));
return err();
}
CAF_LOG_DEBUG(CAF_ARG(ep.hdr));
std::vector<char>* payload = nullptr;
if (ep.hdr.payload_len > 0) {
payload = &pl_buf;
if (payload->size() != ep.hdr.payload_len) {
CAF_LOG_WARNING("received invalid payload");
return err();
}
}
// Handle ordering of datagrams.
if (is_greater(ep.hdr.sequence_number, ep.seq_incoming)) {
// Add early messages to the pending message buffer.
auto s = ep.hdr.sequence_number;
callee_.add_pending(ctx, ep, s, std::move(ep.hdr), std::move(pl_buf));
return true;
} else if (ep.hdr.sequence_number != ep.seq_incoming) {
// Drop messages that arrive late.
CAF_LOG_DEBUG("dropping message " << CAF_ARG(dm));
return true;
}
// This is the expected message.
ep.seq_incoming += 1;
// TODO: add optional reliability here
if (!is_handshake(ep.hdr) && !is_heartbeat(ep.hdr)
&& ep.hdr.dest_node != this_node_) {
CAF_LOG_DEBUG("forward message");
auto path = lookup(ep.hdr.dest_node);
if (path) {
binary_serializer bs{ctx, callee_.get_buffer(path->hdl)};
auto ex = bs(ep.hdr);
if (ex)
return err();
if (payload != nullptr)
bs.apply_raw(payload->size(), payload->data());
flush(*path);
notify<hook::message_forwarded>(ep.hdr, payload);
} else {
CAF_LOG_INFO("cannot forward message, no route to destination");
if (ep.hdr.source_node != this_node_) {
// TODO: signalize error back to sending node
auto reverse_path = lookup(ep.hdr.source_node);
if (!reverse_path) {
CAF_LOG_WARNING("cannot send error message: no route to source");
} else {
CAF_LOG_WARNING("not implemented yet: signalize forward failure");
}
} else {
CAF_LOG_WARNING("lost packet with probably spoofed source");
}
notify<hook::message_forwarding_failed>(ep.hdr, payload);
}
return true;
}
if (!handle(ctx, dm.handle, ep.hdr, payload, false, ep, ep.local_port))
return err();
// See if the next message was delivered early and is already bufferd.
if (!callee_.deliver_pending(ctx, ep, false))
return err();
return true;
}
void instance::handle_heartbeat(execution_unit* ctx) {
CAF_LOG_TRACE("");
for (auto& kvp: tbl_.direct_by_hdl_) {
CAF_LOG_TRACE(CAF_ARG(kvp.first) << CAF_ARG(kvp.second));
write_heartbeat(ctx, callee_.get_buffer(kvp.first),
kvp.second, visit(seq_num_visitor{callee_}, kvp.first));
write_heartbeat(ctx, callee_.get_buffer(kvp.first), kvp.second);
callee_.flush(kvp.first);
}
}
......@@ -293,13 +192,6 @@ size_t instance::remove_published_actor(const actor_addr& whom,
return result;
}
bool instance::is_greater(sequence_type lhs, sequence_type rhs,
sequence_type max_distance) {
// distance between lhs and rhs is smaller than max_distance.
return ((lhs > rhs) && (lhs - rhs <= max_distance)) ||
((lhs < rhs) && (rhs - lhs > max_distance));
}
bool instance::dispatch(execution_unit* ctx, const strong_actor_ptr& sender,
const std::vector<strong_actor_ptr>& forwarding_stack,
const strong_actor_ptr& receiver, message_id mid,
......@@ -318,8 +210,7 @@ bool instance::dispatch(execution_unit* ctx, const strong_actor_ptr& sender,
});
header hdr{message_type::dispatch_message, 0, 0, mid.integer_value(),
sender ? sender->node() : this_node(), receiver->node(),
sender ? sender->id() : invalid_actor_id, receiver->id(),
visit(seq_num_visitor{callee_}, path->hdl)};
sender ? sender->id() : invalid_actor_id, receiver->id()};
write(ctx, callee_.get_buffer(path->hdl), hdr, &writer);
flush(*path);
notify<hook::message_sent>(sender, path->next_hop, receiver, mid, msg);
......@@ -350,10 +241,8 @@ void instance::write(execution_unit* ctx, buffer_type& buf,
CAF_LOG_ERROR(CAF_ARG(err));
}
void instance::write_server_handshake(execution_unit* ctx,
buffer_type& out_buf,
optional<uint16_t> port,
uint16_t sequence_number) {
void instance::write_server_handshake(execution_unit* ctx, buffer_type& out_buf,
optional<uint16_t> port) {
CAF_LOG_TRACE(CAF_ARG(port));
using namespace detail;
published_actor* pa = nullptr;
......@@ -380,7 +269,7 @@ void instance::write_server_handshake(execution_unit* ctx,
header hdr{message_type::server_handshake, 0, 0, version,
this_node_, none,
(pa != nullptr) && pa->first ? pa->first->id() : invalid_actor_id,
invalid_actor_id, sequence_number};
invalid_actor_id};
write(ctx, out_buf, hdr, &writer);
}
......@@ -388,62 +277,206 @@ void instance::write_client_handshake(execution_unit* ctx,
buffer_type& buf,
const node_id& remote_side,
const node_id& this_node,
const std::string& app_identifier,
uint16_t sequence_number) {
const std::string& app_identifier) {
CAF_LOG_TRACE(CAF_ARG(remote_side));
auto writer = make_callback([&](serializer& sink) -> error {
return sink(const_cast<std::string&>(app_identifier));
});
header hdr{message_type::client_handshake, 0, 0, 0,
this_node, remote_side, invalid_actor_id, invalid_actor_id,
sequence_number};
this_node, remote_side, invalid_actor_id, invalid_actor_id};
write(ctx, buf, hdr, &writer);
}
void instance::write_client_handshake(execution_unit* ctx,
buffer_type& buf,
const node_id& remote_side,
uint16_t sequence_number) {
const node_id& remote_side) {
write_client_handshake(ctx, buf, remote_side, this_node_,
get_or(callee_.config(), "middleman.app-identifier",
defaults::middleman::app_identifier),
sequence_number);
defaults::middleman::app_identifier));
}
void instance::write_announce_proxy(execution_unit* ctx, buffer_type& buf,
const node_id& dest_node, actor_id aid,
uint16_t sequence_number) {
const node_id& dest_node, actor_id aid) {
CAF_LOG_TRACE(CAF_ARG(dest_node) << CAF_ARG(aid));
header hdr{message_type::announce_proxy, 0, 0, 0,
this_node_, dest_node, invalid_actor_id, aid,
sequence_number};
this_node_, dest_node, invalid_actor_id, aid};
write(ctx, buf, hdr);
}
void instance::write_kill_proxy(execution_unit* ctx, buffer_type& buf,
const node_id& dest_node, actor_id aid,
const error& rsn, uint16_t sequence_number) {
const error& rsn) {
CAF_LOG_TRACE(CAF_ARG(dest_node) << CAF_ARG(aid) << CAF_ARG(rsn));
auto writer = make_callback([&](serializer& sink) -> error {
return sink(const_cast<error&>(rsn));
});
header hdr{message_type::kill_proxy, 0, 0, 0,
this_node_, dest_node, aid, invalid_actor_id,
sequence_number};
this_node_, dest_node, aid, invalid_actor_id};
write(ctx, buf, hdr, &writer);
}
void instance::write_heartbeat(execution_unit* ctx,
buffer_type& buf,
const node_id& remote_side,
uint16_t sequence_number) {
const node_id& remote_side) {
CAF_LOG_TRACE(CAF_ARG(remote_side));
header hdr{message_type::heartbeat, 0, 0, 0,
this_node_, remote_side, invalid_actor_id, invalid_actor_id,
sequence_number};
this_node_, remote_side, invalid_actor_id, invalid_actor_id};
write(ctx, buf, hdr);
}
bool instance::handle(execution_unit* ctx, connection_handle hdl, header& hdr,
std::vector<char>* payload) {
// function object for checking payload validity
auto payload_valid = [&]() -> bool {
return payload != nullptr && payload->size() == hdr.payload_len;
};
// handle message to ourselves
switch (hdr.operation) {
case message_type::server_handshake: {
actor_id aid = invalid_actor_id;
std::set<std::string> sigs;
if (!payload_valid()) {
CAF_LOG_ERROR("fail to receive the app identifier");
return false;
} else {
binary_deserializer bd{ctx, *payload};
std::string remote_appid;
auto e = bd(remote_appid);
if (e)
return false;
auto appid = get_if<std::string>(&callee_.config(),
"middleman.app-identifier");
if ((appid && *appid != remote_appid)
|| (!appid && !remote_appid.empty())) {
CAF_LOG_ERROR("app identifier mismatch");
return false;
}
e = bd(aid, sigs);
if (e)
return false;
}
// close self connection after handshake is done
if (hdr.source_node == this_node_) {
CAF_LOG_DEBUG("close connection to self immediately");
callee_.finalize_handshake(hdr.source_node, aid, sigs);
return false;
}
// close this connection if we already have a direct connection
if (tbl_.lookup_direct(hdr.source_node)) {
CAF_LOG_DEBUG("close connection since we already have a "
"direct connection: " << CAF_ARG(hdr.source_node));
callee_.finalize_handshake(hdr.source_node, aid, sigs);
return false;
}
// add direct route to this node and remove any indirect entry
CAF_LOG_DEBUG("new direct connection:" << CAF_ARG(hdr.source_node));
tbl_.add_direct(hdl, hdr.source_node);
auto was_indirect = tbl_.erase_indirect(hdr.source_node);
// write handshake as client in response
auto path = tbl_.lookup(hdr.source_node);
if (!path) {
CAF_LOG_ERROR("no route to host after server handshake");
return false;
}
write_client_handshake(ctx, callee_.get_buffer(path->hdl),
hdr.source_node);
callee_.learned_new_node_directly(hdr.source_node, was_indirect);
callee_.finalize_handshake(hdr.source_node, aid, sigs);
flush(*path);
break;
}
case message_type::client_handshake: {
if (!payload_valid()) {
CAF_LOG_ERROR("fail to receive the app identifier");
return false;
} else {
binary_deserializer bd{ctx, *payload};
std::string remote_appid;
auto e = bd(remote_appid);
if (e)
return false;
auto appid = get_if<std::string>(&callee_.config(),
"middleman.app-identifier");
if ((appid && *appid != remote_appid)
|| (!appid && !remote_appid.empty())) {
CAF_LOG_ERROR("app identifier mismatch");
return false;
}
}
if (tbl_.lookup_direct(hdr.source_node)) {
CAF_LOG_DEBUG("received second client handshake:"
<< CAF_ARG(hdr.source_node));
break;
}
// add direct route to this node and remove any indirect entry
CAF_LOG_DEBUG("new direct connection:" << CAF_ARG(hdr.source_node));
tbl_.add_direct(hdl, hdr.source_node);
auto was_indirect = tbl_.erase_indirect(hdr.source_node);
callee_.learned_new_node_directly(hdr.source_node, was_indirect);
break;
}
case message_type::dispatch_message: {
if (!payload_valid())
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 (hdr.source_node != none
&& hdr.source_node != this_node_
&& last_hop != hdr.source_node
&& !tbl_.lookup_direct(hdr.source_node)
&& tbl_.add_indirect(last_hop, hdr.source_node))
callee_.learned_new_node_indirectly(hdr.source_node);
binary_deserializer bd{ctx, *payload};
auto receiver_name = static_cast<atom_value>(0);
std::vector<strong_actor_ptr> forwarding_stack;
message msg;
if (hdr.has(header::named_receiver_flag)) {
auto e = bd(receiver_name);
if (e)
return false;
}
auto e = bd(forwarding_stack, msg);
if (e)
return false;
CAF_LOG_DEBUG(CAF_ARG(forwarding_stack) << CAF_ARG(msg));
if (hdr.has(header::named_receiver_flag))
callee_.deliver(hdr.source_node, hdr.source_actor, receiver_name,
make_message_id(hdr.operation_data),
forwarding_stack, msg);
else
callee_.deliver(hdr.source_node, hdr.source_actor, hdr.dest_actor,
make_message_id(hdr.operation_data),
forwarding_stack, msg);
break;
}
case message_type::announce_proxy:
callee_.proxy_announced(hdr.source_node, hdr.dest_actor);
break;
case message_type::kill_proxy: {
if (!payload_valid())
return false;
binary_deserializer bd{ctx, *payload};
error fail_state;
auto e = bd(fail_state);
if (e)
return false;
callee_.proxies().erase(hdr.source_node, hdr.source_actor,
std::move(fail_state));
break;
}
case message_type::heartbeat: {
CAF_LOG_TRACE("received heartbeat: " << CAF_ARG(hdr.source_node));
callee_.handle_heartbeat(hdr.source_node);
break;
}
default:
CAF_LOG_ERROR("invalid operation");
return false;
}
return true;
}
} // namespace basp
} // namespace io
} // namespace caf
......@@ -144,21 +144,6 @@ expected<uint16_t> middleman::publish(const strong_actor_ptr& whom,
return f(publish_atom::value, port, std::move(whom), std::move(sigs), in, ru);
}
expected<uint16_t> middleman::publish_udp(const strong_actor_ptr& whom,
std::set<std::string> sigs,
uint16_t port, const char* cstr,
bool ru) {
CAF_LOG_TRACE(CAF_ARG(whom) << CAF_ARG(sigs) << CAF_ARG(port));
if (!whom)
return sec::cannot_publish_invalid_actor;
std::string in;
if (cstr != nullptr)
in = cstr;
auto f = make_function_view(actor_handle());
return f(publish_udp_atom::value, port, std::move(whom),
std::move(sigs), in, ru);
}
expected<uint16_t> middleman::publish_local_groups(uint16_t port,
const char* in, bool reuse) {
CAF_LOG_TRACE(CAF_ARG(port) << CAF_ARG(in));
......@@ -185,12 +170,6 @@ expected<void> middleman::unpublish(const actor_addr& whom, uint16_t port) {
return f(unpublish_atom::value, whom, port);
}
expected<void> middleman::unpublish_udp(const actor_addr& whom, uint16_t port) {
CAF_LOG_TRACE(CAF_ARG(whom) << CAF_ARG(port));
auto f = make_function_view(actor_handle());
return f(unpublish_udp_atom::value, whom, port);
}
expected<strong_actor_ptr> middleman::remote_actor(std::set<std::string> ifs,
std::string host,
uint16_t port) {
......@@ -208,23 +187,6 @@ expected<strong_actor_ptr> middleman::remote_actor(std::set<std::string> ifs,
return ptr;
}
expected<strong_actor_ptr>
middleman::remote_actor_udp(std::set<std::string> ifs, std::string host,
uint16_t port) {
CAF_LOG_TRACE(CAF_ARG(ifs) << CAF_ARG(host) << CAF_ARG(port));
auto f = make_function_view(actor_handle());
auto res = f(contact_atom::value, std::move(host), port);
if (!res)
return std::move(res.error());
strong_actor_ptr ptr = std::move(std::get<1>(*res));
if (!ptr)
return make_error(sec::no_actor_published_at_port, port);
if (!system().assignable(std::get<2>(*res), ifs))
return make_error(sec::unexpected_actor_messaging_interface, std::move(ifs),
std::move(std::get<2>(*res)));
return ptr;
}
expected<group> middleman::remote_group(const std::string& group_uri) {
CAF_LOG_TRACE(CAF_ARG(group_uri));
// format of group_identifier is group@host:port
......
......@@ -53,14 +53,6 @@ middleman_actor_impl::middleman_actor_impl(actor_config& cfg,
else
++i;
}
i = cached_udp_.begin();
e = cached_udp_.end();
while (i != e) {
if (get<1>(i->second) == dm.source)
i = cached_udp_.erase(i);
else
++i;
}
});
set_exit_handler([=](exit_msg&) {
// ignored, the MM links group nameservers
......@@ -72,7 +64,6 @@ void middleman_actor_impl::on_exit() {
CAF_LOG_TRACE("");
broker_ = nullptr;
cached_tcp_.clear();
cached_udp_.clear();
for (auto& kvp : pending_)
for (auto& promise : kvp.second)
promise.deliver(make_error(sec::cannot_connect_to_node));
......@@ -85,32 +76,20 @@ const char* middleman_actor_impl::name() const {
auto middleman_actor_impl::make_behavior() -> behavior_type {
CAF_LOG_TRACE("");
auto tcp_disabled = [=] {
return get_or(config(), "middleman.disable-tcp", false);
};
auto udp_disabled = [=] {
return !get_or(config(), "middleman.enable-udp", false);
};
return {
[=](publish_atom, uint16_t port, strong_actor_ptr& whom, mpi_set& sigs,
std::string& addr, bool reuse) -> put_res {
CAF_LOG_TRACE("");
if (tcp_disabled())
return make_error(sec::feature_disabled);
return put(port, whom, sigs, addr.c_str(), reuse);
},
[=](open_atom, uint16_t port, std::string& addr, bool reuse) -> put_res {
CAF_LOG_TRACE("");
if (tcp_disabled())
return make_error(sec::feature_disabled);
strong_actor_ptr whom;
mpi_set sigs;
return put(port, whom, sigs, addr.c_str(), reuse);
},
[=](connect_atom, std::string& hostname, uint16_t port) -> get_res {
CAF_LOG_TRACE(CAF_ARG(hostname) << CAF_ARG(port));
if (tcp_disabled())
return make_error(sec::feature_disabled);
auto rp = make_response_promise();
endpoint key{std::move(hostname), port};
// respond immediately if endpoint is cached
......@@ -162,78 +141,11 @@ auto middleman_actor_impl::make_behavior() -> behavior_type {
});
return get_delegated{};
},
[=](publish_udp_atom, uint16_t port, strong_actor_ptr& whom,
mpi_set& sigs, std::string& addr, bool reuse) -> put_res {
CAF_LOG_TRACE("");
if (udp_disabled())
return make_error(sec::feature_disabled);
return put_udp(port, whom, sigs, addr.c_str(), reuse);
},
[=](contact_atom, std::string& hostname, uint16_t port) -> get_res {
CAF_LOG_TRACE(CAF_ARG(hostname) << CAF_ARG(port));
if (udp_disabled())
return make_error(sec::feature_disabled);
auto rp = make_response_promise();
endpoint key{std::move(hostname), port};
// respond immediately if endpoint is cached
auto x = cached_udp(key);
if (x) {
CAF_LOG_DEBUG("found cached entry" << CAF_ARG(*x));
rp.deliver(get<0>(*x), get<1>(*x), get<2>(*x));
return get_delegated{};
}
// attach this promise to a pending request if possible
auto rps = pending(key);
if (rps) {
CAF_LOG_DEBUG("attach to pending request");
rps->emplace_back(std::move(rp));
return get_delegated{};
}
// connect to endpoint and initiate handshake etc.
auto r = contact(key.first, port);
if (!r) {
rp.deliver(std::move(r.error()));
return get_delegated{};
}
auto& ptr = *r;
std::vector<response_promise> tmp{std::move(rp)};
pending_.emplace(key, std::move(tmp));
request(broker_, infinite, contact_atom::value, std::move(ptr), port)
.then(
[=](node_id& nid, strong_actor_ptr& addr, mpi_set& sigs) {
auto i = pending_.find(key);
if (i == pending_.end())
return;
if (nid && addr) {
monitor(addr);
cached_udp_.emplace(key, std::make_tuple(nid, addr, sigs));
}
auto res = make_message(std::move(nid), std::move(addr),
std::move(sigs));
for (auto& promise : i->second)
promise.deliver(res);
pending_.erase(i);
},
[=](error& err) {
auto i = pending_.find(key);
if (i == pending_.end())
return;
for (auto& promise : i->second)
promise.deliver(err);
pending_.erase(i);
});
return get_delegated{};
},
[=](unpublish_atom atm, actor_addr addr, uint16_t p) -> del_res {
CAF_LOG_TRACE("");
delegate(broker_, atm, std::move(addr), p);
return {};
},
[=](unpublish_udp_atom atm, actor_addr addr, uint16_t p) -> del_res {
CAF_LOG_TRACE("");
delegate(broker_, atm, std::move(addr), p);
return {};
},
[=](close_atom atm, uint16_t p) -> del_res {
CAF_LOG_TRACE("");
delegate(broker_, atm, p);
......
......@@ -52,11 +52,11 @@ optional<routing_table::route> routing_table::lookup(const node_id& target) {
return none;
}
node_id routing_table::lookup_direct(const endpoint_handle& hdl) const {
node_id routing_table::lookup_direct(const connection_handle& hdl) const {
return get_opt(direct_by_hdl_, hdl, none);
}
optional<routing_table::endpoint_handle>
optional<connection_handle>
routing_table::lookup_direct(const node_id& nid) const {
auto i = direct_by_nid_.find(nid);
if (i != direct_by_nid_.end())
......@@ -83,7 +83,7 @@ void routing_table::blacklist(const node_id& hop, const node_id& dest) {
indirect_.erase(i);
}
void routing_table::erase_direct(const endpoint_handle& hdl,
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())
......@@ -105,7 +105,7 @@ bool routing_table::erase_indirect(const node_id& dest) {
return true;
}
void routing_table::add_direct(const endpoint_handle& hdl,
void routing_table::add_direct(const connection_handle& hdl,
const node_id& nid) {
CAF_ASSERT(direct_by_hdl_.count(hdl) == 0);
CAF_ASSERT(direct_by_nid_.count(nid) == 0);
......
......@@ -18,7 +18,7 @@
#include "caf/config.hpp"
#define CAF_SUITE io_basp_tcp
#define CAF_SUITE io_basp
#include "caf/test/dsl.hpp"
#include <array>
......@@ -771,10 +771,8 @@ CAF_TEST(automatic_connection) {
// (this node receives a message from jupiter via mars and responds via mars,
// but then also establishes a connection to jupiter directly)
auto check_node_in_tbl = [&](node& n) {
io::id_visitor id_vis;
auto hdl = tbl().lookup_direct(n.id);
CAF_REQUIRE(hdl);
CAF_CHECK_EQUAL(visit(id_vis, *hdl), n.connection.id());
};
mpx()->provide_scribe("jupiter", 8080, jupiter().connection);
CAF_CHECK(mpx()->has_pending_scribe("jupiter", 8080));
......
/******************************************************************************
* ____ _ _____ *
* / ___| / \ | ___| C++ *
* | | / _ \ | |_ Actor *
* | |___ / ___ \| _| Framework *
* \____/_/ \_|_| *
* *
* Copyright 2011-2018 Dominik Charousset *
* *
* Distributed under the terms and conditions of the BSD 3-Clause License or *
* (at your option) under the terms and conditions of the Boost Software *
* License 1.0. See accompanying files LICENSE and LICENSE_ALTERNATIVE. *
* *
* If you did not receive a copy of the license files, see *
* http://opensource.org/licenses/BSD-3-Clause and *
* http://www.boost.org/LICENSE_1_0.txt. *
******************************************************************************/
#include "caf/config.hpp"
#define CAF_SUITE io_basp_udp
#include "caf/test/dsl.hpp"
#include <array>
#include <mutex>
#include <memory>
#include <limits>
#include <vector>
#include <iostream>
#include <condition_variable>
#include "caf/all.hpp"
#include "caf/io/all.hpp"
#include "caf/io/datagram_handle.hpp"
#include "caf/io/datagram_servant.hpp"
#include "caf/deep_to_string.hpp"
#include "caf/io/network/interfaces.hpp"
#include "caf/io/network/ip_endpoint.hpp"
#include "caf/io/network/test_multiplexer.hpp"
namespace {
struct anything { };
anything any_vals;
template <class T>
struct maybe {
maybe(T x) : val(std::move(x)) {
// nop
}
maybe(anything) {
// nop
}
caf::optional<T> val;
};
template <class T>
std::string to_string(const maybe<T>& x) {
return to_string(x.val);
}
template <class T>
bool operator==(const maybe<T>& x, const T& y) {
return x.val ? x.val == y : true;
}
constexpr uint8_t no_flags = 0;
constexpr uint32_t no_payload = 0;
constexpr uint64_t no_operation_data = 0;
constexpr auto basp_atom = caf::atom("BASP");
constexpr auto spawn_serv_atom = caf::atom("SpawnServ");
constexpr auto config_serv_atom = caf::atom("ConfigServ");
} // namespace <anonymous>
using namespace std;
using namespace caf;
using namespace caf::io;
namespace {
constexpr uint32_t num_remote_nodes = 2;
using buffer = std::vector<char>;
std::string hexstr(const buffer& buf) {
return deep_to_string(meta::hex_formatted(), buf);
}
struct node {
std::string name;
node_id id;
datagram_handle endpoint;
union { scoped_actor dummy_actor; };
node() {
// nop
}
~node() {
// nop
}
};
class fixture {
public:
fixture(bool autoconn = false, bool use_test_coordinator = false)
: sys(cfg.load<io::middleman, network::test_multiplexer>()
.set("middleman.enable-automatic-connections", autoconn)
.set("middleman.enable-udp", true)
.set("middleman.disable-tcp", true)
.set("scheduler.policy", autoconn || use_test_coordinator
? caf::atom("testing")
: caf::atom("stealing"))
.set("middleman.attach-utility-actors",
autoconn || use_test_coordinator)) {
auto& mm = sys.middleman();
mpx_ = dynamic_cast<network::test_multiplexer*>(&mm.backend());
CAF_REQUIRE(mpx_ != nullptr);
CAF_REQUIRE(&sys == &mpx_->system());
auto hdl = mm.named_broker<basp_broker>(basp_atom);
aut_ = static_cast<basp_broker*>(actor_cast<abstract_actor*>(hdl));
this_node_ = sys.node();
self_.reset(new scoped_actor{sys});
dhdl_ = datagram_handle::from_int(1);
aut_->add_datagram_servant(mpx_->new_datagram_servant(dhdl_, 1u));
registry_ = &sys.registry();
registry_->put((*self_)->id(), actor_cast<strong_actor_ptr>(*self_));
// first remote node is everything of this_node + 1, then +2, etc.
for (uint32_t i = 0; i < num_remote_nodes; ++i) {
auto& n = nodes_[i];
node_id::host_id_type tmp = this_node_.host_id();
for (auto& c : tmp)
c = static_cast<uint8_t>(c + i + 1);
n.id = node_id{this_node_.process_id() + i + 1, tmp};
n.endpoint = datagram_handle::from_int(i + 2);
new (&n.dummy_actor) scoped_actor(sys);
// register all pseudo remote actors in the registry
registry_->put(n.dummy_actor->id(),
actor_cast<strong_actor_ptr>(n.dummy_actor));
}
// make sure all init messages are handled properly
mpx_->flush_runnables();
nodes_[0].name = "Jupiter";
nodes_[1].name = "Mars";
CAF_REQUIRE_NOT_EQUAL(jupiter().endpoint, mars().endpoint);
CAF_MESSAGE("Earth: " << to_string(this_node_) << ", ID = "
<< endpoint_handle().id());
CAF_MESSAGE("Jupiter: " << to_string(jupiter().id) << ", ID = "
<< jupiter().endpoint.id());
CAF_MESSAGE("Mars: " << to_string(mars().id) << ", ID = "
<< mars().endpoint.id());
CAF_REQUIRE_NOT_EQUAL(this_node_, jupiter().id);
CAF_REQUIRE_NOT_EQUAL(jupiter().id, mars().id);
}
~fixture() {
this_node_ = none;
self_.reset();
for (auto& n : nodes_) {
n.id = none;
n.dummy_actor.~scoped_actor();
}
}
uint32_t serialized_size(const message& msg) {
buffer buf;
binary_serializer bs{mpx_, buf};
auto e = bs(const_cast<message&>(msg));
CAF_REQUIRE(!e);
return static_cast<uint32_t>(buf.size());
}
node& jupiter() {
return nodes_[0];
}
node& mars() {
return nodes_[1];
}
// our "virtual communication backend"
network::test_multiplexer* mpx() {
return mpx_;
}
// actor-under-test
basp_broker* aut() {
return aut_;
}
// our node ID
node_id& this_node() {
return this_node_;
}
// an actor reference representing a local actor
scoped_actor& self() {
return *self_;
}
datagram_handle endpoint_handle() {
return dhdl_;
}
// implementation of the Binary Actor System Protocol
basp::instance& instance() {
return aut()->state.instance;
}
// our routing table (filled by BASP)
basp::routing_table& tbl() {
return aut()->state.instance.tbl();
}
// access to proxy instances
proxy_registry& proxies() {
return aut()->state.proxies();
}
// stores the singleton pointer for convenience
actor_registry* registry() {
return registry_;
}
using payload_writer = basp::instance::payload_writer;
template <class... Ts>
void to_payload(binary_serializer& bs, const Ts&... xs) {
bs(const_cast<Ts&>(xs)...);
}
template <class... Ts>
void to_payload(buffer& buf, const Ts&... xs) {
binary_serializer bs{mpx_, buf};
to_payload(bs, xs...);
}
void to_buf(buffer& buf, basp::header& hdr, payload_writer* writer) {
instance().write(mpx_, buf, hdr, writer);
}
template <class T, class... Ts>
void to_buf(buffer& buf, basp::header& hdr, payload_writer* writer,
const T& x, const Ts&... xs) {
auto pw = make_callback([&](serializer& sink) -> error {
if (writer)
return error::eval([&] { return (*writer)(sink); },
[&] { return sink(const_cast<T&>(x)); });
return sink(const_cast<T&>(x));
});
to_buf(buf, hdr, &pw, xs...);
}
std::pair<basp::header, buffer> from_buf(const buffer& buf) {
basp::header hdr;
binary_deserializer bd{mpx_, buf};
auto e = bd(hdr);
CAF_REQUIRE(!e);
buffer payload;
if (hdr.payload_len > 0) {
std::copy(buf.begin() + basp::header_size, buf.end(),
std::back_inserter(payload));
}
return {hdr, std::move(payload)};
}
void establish_communication(node& n,
optional<datagram_handle> dx = none,
actor_id published_actor_id = invalid_actor_id,
const set<string>& published_actor_ifs
= std::set<std::string>{}) {
auto src = dx ? *dx : dhdl_;
CAF_MESSAGE("establish communication on node " << n.name
<< ", delegated servant ID = " << n.endpoint.id()
<< ", initial servant ID = " << src.id());
// send the client handshake and receive the server handshake
// and a dispatch_message as answers
auto hdl = n.endpoint;
mpx_->add_pending_endpoint(src, hdl);
CAF_MESSAGE("Send client handshake");
mock(src, hdl,
{basp::message_type::client_handshake, 0, 0, 0,
n.id, this_node(),
invalid_actor_id, invalid_actor_id}, std::string{})
// upon receiving the client handshake, the server should answer
// with the server handshake and send the dispatch_message blow
.receive(hdl,
basp::message_type::server_handshake, no_flags,
any_vals, basp::version, this_node(), node_id{none},
published_actor_id, invalid_actor_id, std::string{},
published_actor_id, published_actor_ifs)
// upon receiving our client handshake, BASP will check
// whether there is a SpawnServ actor on this node
.receive(hdl,
basp::message_type::dispatch_message,
basp::header::named_receiver_flag, any_vals,
no_operation_data,
this_node(), n.id,
any_vals, invalid_actor_id,
spawn_serv_atom,
std::vector<actor_addr>{},
make_message(sys_atom::value, get_atom::value, "info"));
// test whether basp instance correctly updates the
// routing table upon receiving client handshakes
auto path = tbl().lookup(n.id);
CAF_REQUIRE(path);
CAF_CHECK_EQUAL(path->hdl, n.endpoint);
CAF_CHECK_EQUAL(path->next_hop, n.id);
}
std::pair<basp::header, buffer> read_from_out_buf(datagram_handle hdl) {
CAF_MESSAGE("read from output buffer for endpoint " << hdl.id());
auto& que = mpx_->output_queue(hdl);
while (que.empty())
mpx()->exec_runnable();
auto result = from_buf(que.front().second);
que.pop_front();
return result;
}
void dispatch_out_buf(datagram_handle hdl) {
basp::header hdr;
buffer buf;
std::tie(hdr, buf) = read_from_out_buf(hdl);
CAF_MESSAGE("dispatch output buffer for endpoint " << hdl.id());
CAF_REQUIRE(hdr.operation == basp::message_type::dispatch_message);
binary_deserializer source{mpx_, buf};
std::vector<strong_actor_ptr> stages;
message msg;
auto e = source(stages, msg);
CAF_REQUIRE(!e);
auto src = actor_cast<strong_actor_ptr>(registry_->get(hdr.source_actor));
auto dest = registry_->get(hdr.dest_actor);
CAF_REQUIRE(dest);
dest->enqueue(make_mailbox_element(src, make_message_id(),
std::move(stages), std::move(msg)),
nullptr);
}
class mock_t {
public:
mock_t(fixture* thisptr) : this_(thisptr) {
// nop
}
mock_t(mock_t&&) = default;
~mock_t() {
if (num > 1)
CAF_MESSAGE("implementation under test responded with "
<< (num - 1) << " BASP message" << (num > 2 ? "s" : ""));
}
template <class... Ts>
mock_t& receive(datagram_handle hdl,
maybe<basp::message_type> operation,
maybe<uint8_t> flags,
maybe<uint32_t> payload_len,
maybe<uint64_t> operation_data,
maybe<node_id> source_node,
maybe<node_id> dest_node,
maybe<actor_id> source_actor,
maybe<actor_id> dest_actor,
const Ts&... xs) {
CAF_MESSAGE("expect #" << num << " on endpoint ID = " << hdl.id());
buffer buf;
this_->to_payload(buf, xs...);
auto& oq = this_->mpx()->output_queue(hdl);
while (oq.empty())
this_->mpx()->exec_runnable();
CAF_MESSAGE("output queue has " << oq.size() << " messages");
auto dh = oq.front().first;
buffer& ob = oq.front().second;
CAF_MESSAGE("next datagram has " << ob.size()
<< " bytes, servant ID = " << dh.id());
CAF_CHECK_EQUAL(dh.id(), hdl.id());
basp::header hdr;
{ // lifetime scope of source
binary_deserializer source{this_->mpx(), ob};
auto e = source(hdr);
CAF_REQUIRE_EQUAL(e, none);
}
buffer payload;
if (hdr.payload_len > 0) {
CAF_REQUIRE(ob.size() >= (basp::header_size + hdr.payload_len));
auto first = ob.begin() + basp::header_size;
auto end = first + hdr.payload_len;
payload.assign(first, end);
}
CAF_MESSAGE("erase message from output queue");
oq.pop_front();
CAF_CHECK_EQUAL(operation, hdr.operation);
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);
CAF_CHECK_EQUAL(source_node, hdr.source_node);
CAF_CHECK_EQUAL(dest_node, hdr.dest_node);
CAF_CHECK_EQUAL(source_actor, hdr.source_actor);
CAF_CHECK_EQUAL(dest_actor, hdr.dest_actor);
CAF_REQUIRE_EQUAL(buf.size(), payload.size());
CAF_REQUIRE_EQUAL(hexstr(buf), hexstr(payload));
++num;
return *this;
}
template <class... Ts>
mock_t& enqueue_back(datagram_handle hdl, datagram_handle ep,
basp::header hdr, const Ts&... xs) {
buffer buf;
this_->to_buf(buf, hdr, nullptr, xs...);
CAF_MESSAGE("adding msg " << to_string(hdr.operation)
<< " with " << (buf.size() - basp::header_size)
<< " bytes payload to back of queue");
this_->mpx()->virtual_network_buffer(hdl).emplace_back(ep, buf);
return *this;
}
template <class... Ts>
mock_t& enqueue_back(datagram_handle hdl, basp::header hdr,
Ts&&... xs) {
return enqueue_back(hdl, hdl, hdr, std::forward<Ts>(xs)...);
}
template <class... Ts>
mock_t& enqueue_front(datagram_handle hdl, datagram_handle ep,
basp::header hdr, const Ts&... xs) {
buffer buf;
this_->to_buf(buf, hdr, nullptr, xs...);
CAF_MESSAGE("adding msg " << to_string(hdr.operation)
<< " with " << (buf.size() - basp::header_size)
<< " bytes payload to front of queue");
this_->mpx()->virtual_network_buffer(hdl).emplace_front(ep, buf);
return *this;
}
template <class... Ts>
mock_t& enqueue_front(datagram_handle hdl, basp::header hdr,
Ts&&... xs) {
return enqueue_front(hdl, hdl, hdr, std::forward<Ts>(xs)...);
}
mock_t& deliver(datagram_handle hdl, size_t num_messages = 1) {
for (size_t i = 0; i < num_messages; ++i)
this_->mpx()->read_data(hdl);
return *this;
}
private:
fixture* this_;
size_t num = 1;
};
template <class... Ts>
mock_t mock(datagram_handle hdl, basp::header hdr,
const Ts&... xs) {
buffer buf;
to_buf(buf, hdr, nullptr, xs...);
CAF_MESSAGE("virtually send " << to_string(hdr.operation)
<< " with " << (buf.size() - basp::header_size)
<< " bytes payload");
mpx()->virtual_send(hdl, hdl, buf);
return {this};
}
template <class... Ts>
mock_t mock(datagram_handle hdl, datagram_handle ep, basp::header hdr,
const Ts&... xs) {
buffer buf;
to_buf(buf, hdr, nullptr, xs...);
CAF_MESSAGE("virtually send " << to_string(hdr.operation)
<< " with " << (buf.size() - basp::header_size)
<< " bytes payload");
mpx()->virtual_send(hdl, ep, buf);
return {this};
}
mock_t mock() {
return {this};
}
actor_system_config cfg;
actor_system sys;
private:
basp_broker* aut_;
datagram_handle dhdl_;
network::test_multiplexer* mpx_;
node_id this_node_;
unique_ptr<scoped_actor> self_;
array<node, num_remote_nodes> nodes_;
/*
array<node_id, num_remote_nodes> remote_node_;
array<connection_handle, num_remote_nodes> remote_hdl_;
array<unique_ptr<scoped_actor>, num_remote_nodes> pseudo_remote_;
*/
actor_registry* registry_;
};
class manual_timer_fixture : public fixture {
public:
using scheduler_type = caf::scheduler::test_coordinator;
scheduler_type& sched;
manual_timer_fixture()
: fixture(false, true),
sched(dynamic_cast<scheduler_type&>(sys.scheduler())) {
// nop
}
};
class autoconn_enabled_fixture : public fixture {
public:
using scheduler_type = caf::scheduler::test_coordinator;
scheduler_type& sched;
middleman_actor mma;
autoconn_enabled_fixture()
: fixture(true),
sched(dynamic_cast<scheduler_type&>(sys.scheduler())),
mma(sys.middleman().actor_handle()) {
// nop
}
void publish(const actor& whom, uint16_t port, bool is_udp = false) {
using sig_t = std::set<std::string>;
scoped_actor tmp{sys};
sig_t sigs;
CAF_MESSAGE("publish whom on port " << port);
if (is_udp)
tmp->send(mma, publish_udp_atom::value, port,
actor_cast<strong_actor_ptr>(whom), std::move(sigs), "", false);
else
tmp->send(mma, publish_atom::value, port,
actor_cast<strong_actor_ptr>(whom), std::move(sigs), "", false);
CAF_MESSAGE("publish from tmp to mma with port _");
expect((atom_value, uint16_t, strong_actor_ptr, sig_t, std::string, bool),
from(tmp).to(mma));
CAF_MESSAGE("publish: from mma to tmp with port " << port);
expect((uint16_t), from(mma).to(tmp).with(port));
}
};
} // namespace <anonymous>
CAF_TEST_FIXTURE_SCOPE(basp_udp_tests, fixture)
CAF_TEST_DISABLED(empty_server_handshake_udp) {
// test whether basp instance correctly sends a
// client handshake when there's no actor published
buffer buf;
instance().write_server_handshake(mpx(), buf, none);
basp::header hdr;
buffer payload;
std::tie(hdr, payload) = from_buf(buf);
basp::header expected{basp::message_type::server_handshake, 0,
static_cast<uint32_t>(payload.size()),
basp::version,
this_node(), none,
invalid_actor_id, invalid_actor_id,
0};
CAF_CHECK(basp::valid(hdr));
CAF_CHECK(basp::is_handshake(hdr));
CAF_CHECK_EQUAL(to_string(hdr), to_string(expected));
}
CAF_TEST_DISABLED(empty_client_handshake_udp) {
// test whether basp instance correctly sends a
// client handshake when there's no actor published
buffer buf;
instance().write_client_handshake(mpx(), buf, none);
basp::header hdr;
buffer payload;
std::tie(hdr, payload) = from_buf(buf);
basp::header expected{basp::message_type::client_handshake, 0,
static_cast<uint32_t>(payload.size()), 0,
this_node(), none,
invalid_actor_id, invalid_actor_id,
0};
CAF_CHECK(basp::valid(hdr));
CAF_CHECK(basp::is_handshake(hdr));
CAF_MESSAGE("got : " << to_string(hdr));
CAF_MESSAGE("expecting: " << to_string(expected));
CAF_CHECK_EQUAL(to_string(hdr), to_string(expected));
}
CAF_TEST_DISABLED(non_empty_server_handshake_udp) {
// test whether basp instance correctly sends a
// server handshake with published actors
buffer buf;
instance().add_published_actor(4242, actor_cast<strong_actor_ptr>(self()),
{"caf::replies_to<@u16>::with<@u16>"});
instance().write_server_handshake(mpx(), buf, uint16_t{4242});
buffer expected_buf;
basp::header expected{basp::message_type::server_handshake, 0, 0,
basp::version, this_node(), none,
self()->id(), invalid_actor_id, 0};
to_buf(expected_buf, expected, nullptr, std::string{},
self()->id(), set<string>{"caf::replies_to<@u16>::with<@u16>"});
CAF_CHECK_EQUAL(hexstr(buf), hexstr(expected_buf));
}
CAF_TEST_DISABLED(remote_address_and_port_udp) {
CAF_MESSAGE("connect to Mars");
establish_communication(mars());
auto mm = sys.middleman().actor_handle();
CAF_MESSAGE("ask MM about node ID of Mars");
self()->send(mm, get_atom::value, mars().id);
do {
mpx()->exec_runnable();
} while (self()->mailbox().empty());
CAF_MESSAGE("receive result of MM");
self()->receive(
[&](const node_id& nid, const std::string& addr, uint16_t port) {
CAF_CHECK_EQUAL(nid, mars().id);
// all test nodes have address "test" and connection handle ID as port
CAF_CHECK_EQUAL(addr, "test");
CAF_CHECK_EQUAL(port, mars().endpoint.id());
}
);
}
CAF_TEST_DISABLED(client_handshake_and_dispatch_udp) {
CAF_MESSAGE("establish communication with Jupiter");
establish_communication(jupiter());
CAF_MESSAGE("send dispatch message");
// send a message via `dispatch` from node 0 on endpoint 1
mock(endpoint_handle(), jupiter().endpoint,
{basp::message_type::dispatch_message, 0, 0, 0,
jupiter().id, this_node(), jupiter().dummy_actor->id(), self()->id(),
1}, // increment sequence number
std::vector<actor_addr>{},
make_message(1, 2, 3))
.receive(jupiter().endpoint,
basp::message_type::announce_proxy, no_flags, no_payload,
no_operation_data, this_node(), jupiter().id,
invalid_actor_id, jupiter().dummy_actor->id());
// must've created a proxy for our remote actor
CAF_REQUIRE(proxies().count_proxies(jupiter().id) == 1);
// must've send remote node a message that this proxy is monitored now
// receive the message
self()->receive(
[](int a, int b, int c) {
CAF_CHECK_EQUAL(a, 1);
CAF_CHECK_EQUAL(b, 2);
CAF_CHECK_EQUAL(c, 3);
return a + b + c;
}
);
CAF_MESSAGE("exec message of forwarding proxy");
mpx()->exec_runnable();
// deserialize and send message from out buf
dispatch_out_buf(jupiter().endpoint);
jupiter().dummy_actor->receive(
[](int i) {
CAF_CHECK_EQUAL(i, 6);
}
);
}
CAF_TEST_DISABLED(message_forwarding_udp) {
// connect two remote nodes
CAF_MESSAGE("establish communication with Jupiter");
establish_communication(jupiter());
CAF_MESSAGE("establish communication with Mars");
establish_communication(mars());
CAF_MESSAGE("send dispatch message to ... ");
auto msg = make_message(1, 2, 3);
// send a message from node 0 to node 1, forwarded by this node
mock(endpoint_handle(), jupiter().endpoint,
{basp::message_type::dispatch_message, 0, 0, 0,
jupiter().id, mars().id,
invalid_actor_id, mars().dummy_actor->id(),
1}, // increment sequence number
msg)
.receive(mars().endpoint,
basp::message_type::dispatch_message, no_flags, any_vals,
no_operation_data, jupiter().id, mars().id,
invalid_actor_id, mars().dummy_actor->id(),
msg);
}
CAF_TEST_DISABLED(publish_and_connect_udp) {
auto dx = datagram_handle::from_int(4242);
mpx()->provide_datagram_servant(4242, dx);
auto res = sys.middleman().publish_udp(self(), 4242);
CAF_REQUIRE(res == 4242);
mpx()->flush_runnables(); // process publish message in basp_broker
establish_communication(jupiter(), dx, self()->id());
}
CAF_TEST_DISABLED(remote_actor_and_send_udp) {
constexpr const char* lo = "localhost";
CAF_MESSAGE("self: " << to_string(self()->address()));
mpx()->provide_datagram_servant(lo, 4242, jupiter().endpoint);
CAF_REQUIRE(mpx()->has_pending_remote_endpoint(lo, 4242));
auto mm1 = sys.middleman().actor_handle();
actor result;
auto f = self()->request(mm1, infinite, contact_atom::value,
lo, uint16_t{4242});
// wait until BASP broker has received and processed the connect message
while (!aut()->valid(jupiter().endpoint))
mpx()->exec_runnable();
CAF_REQUIRE(!mpx()->has_pending_scribe(lo, 4242));
// build a fake server handshake containing the id of our first pseudo actor
CAF_MESSAGE("client handshake => server handshake => proxy announcement");
auto na = registry()->named_actors();
mock()
.receive(jupiter().endpoint,
basp::message_type::client_handshake, no_flags, 1u,
no_operation_data, this_node(), node_id(),
invalid_actor_id, invalid_actor_id, std::string{});
mock(jupiter().endpoint,
{basp::message_type::server_handshake, 0, 0, basp::version,
jupiter().id, none,
jupiter().dummy_actor->id(), invalid_actor_id,
0}, // sequence number, first message
std::string{},
jupiter().dummy_actor->id(),
uint32_t{0})
.receive(jupiter().endpoint,
basp::message_type::dispatch_message,
basp::header::named_receiver_flag, any_vals,
no_operation_data, this_node(), jupiter().id,
any_vals, invalid_actor_id,
spawn_serv_atom,
std::vector<actor_id>{},
make_message(sys_atom::value, get_atom::value, "info"))
.receive(jupiter().endpoint,
basp::message_type::announce_proxy, no_flags, no_payload,
no_operation_data, this_node(), jupiter().id,
invalid_actor_id, jupiter().dummy_actor->id());
CAF_MESSAGE("BASP broker should've send the proxy");
f.receive(
[&](node_id nid, strong_actor_ptr res, std::set<std::string> ifs) {
CAF_REQUIRE(res);
auto aptr = actor_cast<abstract_actor*>(res);
CAF_REQUIRE(dynamic_cast<forwarding_actor_proxy*>(aptr) != nullptr);
CAF_CHECK_EQUAL(proxies().count_proxies(jupiter().id), 1u);
CAF_CHECK_EQUAL(nid, jupiter().id);
CAF_CHECK_EQUAL(res->node(), jupiter().id);
CAF_CHECK_EQUAL(res->id(), jupiter().dummy_actor->id());
CAF_CHECK(ifs.empty());
auto proxy = proxies().get(jupiter().id, jupiter().dummy_actor->id());
CAF_REQUIRE(proxy != nullptr);
CAF_REQUIRE(proxy == res);
result = actor_cast<actor>(res);
},
[&](error& err) {
CAF_FAIL("error: " << sys.render(err));
}
);
CAF_MESSAGE("send message to proxy");
anon_send(actor_cast<actor>(result), 42);
mpx()->flush_runnables();
mock()
.receive(jupiter().endpoint,
basp::message_type::dispatch_message, no_flags, any_vals,
no_operation_data, this_node(), jupiter().id,
invalid_actor_id, jupiter().dummy_actor->id(),
std::vector<actor_id>{},
make_message(42));
auto msg = make_message("hi there!");
CAF_MESSAGE("send message via BASP (from proxy)");
mock(jupiter().endpoint,
{basp::message_type::dispatch_message, 0, 0, 0,
jupiter().id, this_node(),
jupiter().dummy_actor->id(), self()->id(),
1}, // sequence number, second message
std::vector<actor_id>{},
make_message("hi there!"));
self()->receive(
[&](const string& str) {
CAF_CHECK_EQUAL(to_string(self()->current_sender()), to_string(result));
CAF_CHECK_EQUAL(self()->current_sender(), result.address());
CAF_CHECK_EQUAL(str, "hi there!");
}
);
}
CAF_TEST_DISABLED(actor_serialize_and_deserialize_udp) {
auto testee_impl = [](event_based_actor* testee_self) -> behavior {
testee_self->set_default_handler(reflect_and_quit);
return {
[] {
// nop
}
};
};
establish_communication(jupiter());
auto prx = proxies().get_or_put(jupiter().id, jupiter().dummy_actor->id());
mock()
.receive(jupiter().endpoint,
basp::message_type::announce_proxy, no_flags, no_payload,
no_operation_data, this_node(), prx->node(),
invalid_actor_id, prx->id());
CAF_CHECK_EQUAL(prx->node(), jupiter().id);
CAF_CHECK_EQUAL(prx->id(), jupiter().dummy_actor->id());
auto testee = sys.spawn(testee_impl);
registry()->put(testee->id(), actor_cast<strong_actor_ptr>(testee));
CAF_MESSAGE("send message via BASP (from proxy)");
auto msg = make_message(actor_cast<actor_addr>(prx));
mock(endpoint_handle(), jupiter().endpoint,
{basp::message_type::dispatch_message, 0, 0, 0,
prx->node(), this_node(),
prx->id(), testee->id(),
1}, // sequence number, first message after handshake
std::vector<actor_id>{},
msg);
// testee must've responded (process forwarded message in BASP broker)
CAF_MESSAGE("wait until BASP broker writes to its output buffer");
while (mpx()->output_queue(jupiter().endpoint).empty())
mpx()->exec_runnable(); // process forwarded message in basp_broker
// output buffer must contain the reflected message
mock()
.receive(jupiter().endpoint,
basp::message_type::dispatch_message, no_flags, any_vals,
no_operation_data, this_node(), prx->node(), testee->id(), prx->id(),
std::vector<actor_id>{}, msg);
}
CAF_TEST_DISABLED(indirect_connections_udp) {
// this node receives a message from jupiter via mars and responds via mars
// and any ad-hoc automatic connection requests are ignored
CAF_MESSAGE("self: " << to_string(self()->address()));
auto dx = datagram_handle::from_int(4242);
mpx()->provide_datagram_servant(4242, dx);
sys.middleman().publish_udp(self(), 4242);
mpx()->flush_runnables(); // process publish message in basp_broker
CAF_MESSAGE("connect to Mars");
establish_communication(mars(), dx, self()->id());
CAF_MESSAGE("actor from Jupiter sends a message to us via Mars");
auto mx = mock(dx, mars().endpoint,
{basp::message_type::dispatch_message, 0, 0, 0,
jupiter().id, this_node(),
jupiter().dummy_actor->id(), self()->id(),
1}, // sequence number
std::vector<actor_id>{},
make_message("hello from jupiter!"));
CAF_MESSAGE("expect ('sys', 'get', \"info\") from Earth to Jupiter at Mars");
// this asks Jupiter if it has a 'SpawnServ'
mx.receive(mars().endpoint,
basp::message_type::dispatch_message,
basp::header::named_receiver_flag, any_vals,
no_operation_data, this_node(), jupiter().id,
any_vals, invalid_actor_id,
spawn_serv_atom,
std::vector<actor_id>{},
make_message(sys_atom::value, get_atom::value, "info"));
CAF_MESSAGE("expect announce_proxy message at Mars from Earth to Jupiter");
mx.receive(mars().endpoint,
basp::message_type::announce_proxy, no_flags, no_payload,
no_operation_data, this_node(), jupiter().id,
invalid_actor_id, jupiter().dummy_actor->id());
CAF_MESSAGE("receive message from jupiter");
self()->receive(
[](const std::string& str) -> std::string {
CAF_CHECK_EQUAL(str, "hello from jupiter!");
return "hello from earth!";
}
);
mpx()->exec_runnable(); // process forwarded message in basp_broker
mock()
.receive(mars().endpoint,
basp::message_type::dispatch_message, no_flags, any_vals,
no_operation_data, this_node(), jupiter().id,
self()->id(), jupiter().dummy_actor->id(),
std::vector<actor_id>{},
make_message("hello from earth!"));
}
CAF_TEST_FIXTURE_SCOPE_END()
CAF_TEST_FIXTURE_SCOPE(basp_udp_tests_with_manual_timer, manual_timer_fixture)
CAF_TEST_DISABLED(out_of_order_delivery_udp) {
// This test uses the test_coordinator to get control over the
// timeouts that deliver pending message.
constexpr const char* lo = "localhost";
CAF_MESSAGE("self: " << to_string(self()->address()));
mpx()->provide_datagram_servant(lo, 4242, jupiter().endpoint);
CAF_REQUIRE(mpx()->has_pending_remote_endpoint(lo, 4242));
auto mm1 = sys.middleman().actor_handle();
actor result;
auto f = self()->request(mm1, infinite, contact_atom::value,
lo, uint16_t{4242});
// wait until BASP broker has received and processed the connect message
while (!aut()->valid(jupiter().endpoint)) {
sched.run();
mpx()->exec_runnable();
}
CAF_REQUIRE(!mpx()->has_pending_scribe(lo, 4242));
// build a fake server handshake containing the id of our first pseudo actor
CAF_MESSAGE("client handshake => server handshake => proxy announcement");
auto na = registry()->named_actors();
mock()
.receive(jupiter().endpoint,
basp::message_type::client_handshake, no_flags, 1u,
no_operation_data, this_node(), node_id(),
invalid_actor_id, invalid_actor_id, std::string{});
mock(jupiter().endpoint, jupiter().endpoint,
{basp::message_type::server_handshake, 0, 0, basp::version,
jupiter().id, none,
jupiter().dummy_actor->id(), invalid_actor_id,
0}, // sequence number, first message
std::string{},
jupiter().dummy_actor->id(),
uint32_t{0})
.receive(jupiter().endpoint,
basp::message_type::dispatch_message,
basp::header::named_receiver_flag, any_vals,
no_operation_data, this_node(), jupiter().id,
any_vals, invalid_actor_id,
spawn_serv_atom,
std::vector<actor_id>{},
make_message(sys_atom::value, get_atom::value, "info"))
.receive(jupiter().endpoint,
basp::message_type::announce_proxy, no_flags, no_payload,
no_operation_data, this_node(), jupiter().id,
invalid_actor_id, jupiter().dummy_actor->id());
sched.run();
CAF_MESSAGE("BASP broker should've send the proxy");
f.receive(
[&](node_id nid, strong_actor_ptr res, std::set<std::string> ifs) {
CAF_REQUIRE(res);
auto aptr = actor_cast<abstract_actor*>(res);
CAF_REQUIRE(dynamic_cast<forwarding_actor_proxy*>(aptr) != nullptr);
CAF_CHECK_EQUAL(proxies().count_proxies(jupiter().id), 1u);
CAF_CHECK_EQUAL(nid, jupiter().id);
CAF_CHECK_EQUAL(res->node(), jupiter().id);
CAF_CHECK_EQUAL(res->id(), jupiter().dummy_actor->id());
CAF_CHECK(ifs.empty());
auto proxy = proxies().get(jupiter().id, jupiter().dummy_actor->id());
CAF_REQUIRE(proxy != nullptr);
CAF_REQUIRE(proxy == res);
result = actor_cast<actor>(res);
},
[&](error& err) {
CAF_FAIL("error: " << sys.render(err));
}
);
CAF_MESSAGE("send message to proxy");
anon_send(actor_cast<actor>(result), 42);
mpx()->flush_runnables();
mock()
.receive(jupiter().endpoint,
basp::message_type::dispatch_message, no_flags, any_vals,
no_operation_data, this_node(), jupiter().id,
invalid_actor_id, jupiter().dummy_actor->id(),
std::vector<actor_id>{},
make_message(42));
auto header_with_seq = [&](uint16_t seq) -> basp::header {
return {basp::message_type::dispatch_message, 0, 0, 0,
jupiter().id, this_node(),
jupiter().dummy_actor->id(), self()->id(),
seq};
};
CAF_MESSAGE("send 10 messages out of order");
mock()
.enqueue_back(jupiter().endpoint, header_with_seq(1),
std::vector<actor_id>{}, make_message(0))
.enqueue_back(jupiter().endpoint, header_with_seq(2),
std::vector<actor_id>{}, make_message(1))
.enqueue_front(jupiter().endpoint, header_with_seq(3),
std::vector<actor_id>{}, make_message(2))
.enqueue_back(jupiter().endpoint, header_with_seq(4),
std::vector<actor_id>{}, make_message(3))
.enqueue_back(jupiter().endpoint, header_with_seq(5),
std::vector<actor_id>{}, make_message(4))
.enqueue_back(jupiter().endpoint, header_with_seq(6),
std::vector<actor_id>{}, make_message(5))
.enqueue_front(jupiter().endpoint, header_with_seq(7),
std::vector<actor_id>{}, make_message(6))
.enqueue_back(jupiter().endpoint, header_with_seq(8),
std::vector<actor_id>{}, make_message(7))
.enqueue_back(jupiter().endpoint, header_with_seq(9),
std::vector<actor_id>{}, make_message(8))
.enqueue_front(jupiter().endpoint, header_with_seq(10),
std::vector<actor_id>{}, make_message(9))
.deliver(jupiter().endpoint, 10);
int expected_next = 0;
self()->receive_while([&] { return expected_next < 10; }) (
[&](int val) {
CAF_CHECK_EQUAL(to_string(self()->current_sender()), to_string(result));
CAF_CHECK_EQUAL(self()->current_sender(), result.address());
CAF_CHECK_EQUAL(expected_next, val);
++expected_next;
}
);
sched.trigger_timeouts();
mpx()->flush_runnables();
CAF_MESSAGE("force delivery via timeout that skips messages");
const basp::sequence_type seq_and_payload = 23;
mock()
.enqueue_back(jupiter().endpoint, header_with_seq(seq_and_payload),
std::vector<actor_id>{}, make_message(seq_and_payload))
.deliver(jupiter().endpoint, 1);
sched.trigger_timeouts();
mpx()->exec_runnable();
self()->receive(
[&](basp::sequence_type val) {
CAF_CHECK_EQUAL(to_string(self()->current_sender()), to_string(result));
CAF_CHECK_EQUAL(self()->current_sender(), result.address());
CAF_CHECK_EQUAL(seq_and_payload, val);
}
);
}
CAF_TEST_FIXTURE_SCOPE_END()
CAF_TEST_FIXTURE_SCOPE(basp_udp_tests_with_autoconn, autoconn_enabled_fixture)
CAF_TEST_DISABLED(automatic_connection_udp) {
// this tells our BASP broker to enable the automatic connection feature
//anon_send(aut(), ok_atom::value,
// "middleman.enable-automatic-connections", make_message(true));
//mpx()->exec_runnable(); // process publish message in basp_broker
// jupiter [remote hdl 0] -> mars [remote hdl 1] -> earth [this_node]
// (this node receives a message from jupiter via mars and responds via mars,
// but then also establishes a connection to jupiter directly)
auto check_node_in_tbl = [&](node& n) {
io::id_visitor id_vis;
auto hdl = tbl().lookup_direct(n.id);
CAF_REQUIRE(hdl);
CAF_CHECK_EQUAL(visit(id_vis, *hdl), n.endpoint.id());
};
mpx()->provide_datagram_servant("jupiter", 8080, jupiter().endpoint);
CAF_CHECK(mpx()->has_pending_remote_endpoint("jupiter", 8080));
CAF_MESSAGE("self: " << to_string(self()->address()));
auto dx = datagram_handle::from_int(4242);
mpx()->provide_datagram_servant(4242, dx);
publish(self(), 4242, true);
mpx()->flush_runnables(); // process publish message in basp_broker
CAF_MESSAGE("connect to mars");
establish_communication(mars(), dx, self()->id());
//CAF_CHECK_EQUAL(tbl().lookup_direct(mars().id).id(), mars().connection.id());
check_node_in_tbl(mars());
CAF_MESSAGE("simulate that an actor from jupiter "
"sends a message to us via mars");
mock(dx, mars().endpoint,
{basp::message_type::dispatch_message, 0, 0, 0,
jupiter().id, this_node(),
jupiter().dummy_actor->id(), self()->id(),
1}, // sequence number
std::vector<actor_id>{},
make_message("hello from jupiter!"))
.receive(mars().endpoint,
basp::message_type::dispatch_message,
basp::header::named_receiver_flag, any_vals, no_operation_data,
this_node(), jupiter().id, any_vals, invalid_actor_id,
spawn_serv_atom,
std::vector<actor_id>{},
make_message(sys_atom::value, get_atom::value, "info"))
// if we do not disable TCP, we would receive a second message here asking
// for the tcp related addresses
.receive(mars().endpoint,
basp::message_type::dispatch_message,
basp::header::named_receiver_flag, any_vals,
no_operation_data, this_node(), jupiter().id,
any_vals, // actor ID of an actor spawned by the BASP broker
invalid_actor_id,
config_serv_atom,
std::vector<actor_id>{},
make_message(get_atom::value, "basp.default-connectivity-udp"))
.receive(mars().endpoint,
basp::message_type::announce_proxy, no_flags, no_payload,
no_operation_data, this_node(), jupiter().id,
invalid_actor_id, jupiter().dummy_actor->id());
CAF_CHECK_EQUAL(mpx()->output_queue(mars().endpoint).size(), 0u);
CAF_CHECK_EQUAL(tbl().lookup_indirect(jupiter().id), mars().id);
CAF_CHECK_EQUAL(tbl().lookup_indirect(mars().id), none);
auto connection_helper_actor = sys.latest_actor_id();
CAF_CHECK_EQUAL(mpx()->output_queue(mars().endpoint).size(), 0u);
// create a dummy config server and respond to the name lookup
CAF_MESSAGE("receive ConfigServ of jupiter");
network::address_listing res;
res[network::protocol::ipv4].emplace_back("jupiter");
mock(dx, mars().endpoint,
{basp::message_type::dispatch_message, 0, 0, 0,
this_node(), this_node(),
invalid_actor_id, connection_helper_actor,
2}, // sequence number
std::vector<actor_id>{},
make_message("basp.default-connectivity-udp",
make_message(uint16_t{8080}, std::move(res))));
// our connection helper should now connect to jupiter and
// send the scribe handle over to the BASP broker
while (mpx()->has_pending_remote_endpoint("jupiter", 8080)) {
sched.run();
mpx()->flush_runnables();
}
CAF_REQUIRE(mpx()->output_queue(mars().endpoint).empty());
CAF_MESSAGE("Let's do the handshake.");
mock()
.receive(jupiter().endpoint,
basp::message_type::client_handshake, no_flags, 1u,
no_operation_data, this_node(), node_id(),
invalid_actor_id, invalid_actor_id, std::string{});
CAF_MESSAGE("Received client handshake.");
// send handshake from jupiter
mock(jupiter().endpoint, jupiter().endpoint,
{basp::message_type::server_handshake, 0, 0, basp::version,
jupiter().id, none,
jupiter().dummy_actor->id(), invalid_actor_id},
std::string{},
jupiter().dummy_actor->id(),
uint32_t{0});
mpx()->exec_runnable(); // Receive message from connection helper
mpx()->exec_runnable(); // Process BASP server handshake
CAF_CHECK_EQUAL(tbl().lookup_indirect(jupiter().id), none);
CAF_CHECK_EQUAL(tbl().lookup_indirect(mars().id), none);
check_node_in_tbl(jupiter());
check_node_in_tbl(mars());
CAF_MESSAGE("receive message from jupiter");
self()->receive(
[](const std::string& str) -> std::string {
CAF_CHECK_EQUAL(str, "hello from jupiter!");
return "hello from earth!";
}
);
mpx()->exec_runnable(); // process forwarded message in basp_broker
CAF_MESSAGE("response message must take direct route now");
mock()
.receive(jupiter().endpoint,
basp::message_type::dispatch_message, no_flags, any_vals,
no_operation_data, this_node(), jupiter().id,
self()->id(), jupiter().dummy_actor->id(),
std::vector<actor_id>{},
make_message("hello from earth!"));
CAF_CHECK_EQUAL(mpx()->output_queue(mars().endpoint).size(), 0u);
}
CAF_TEST_FIXTURE_SCOPE_END()
/******************************************************************************
* ____ _ _____ *
* / ___| / \ | ___| C++ *
* | | / _ \ | |_ Actor *
* | |___ / ___ \| _| Framework *
* \____/_/ \_|_| *
* *
* Copyright 2011-2018 Dominik Charousset *
* *
* Distributed under the terms and conditions of the BSD 3-Clause License or *
* (at your option) under the terms and conditions of the Boost Software *
* License 1.0. See accompanying files LICENSE and LICENSE_ALTERNATIVE. *
* *
* If you did not receive a copy of the license files, see *
* http://opensource.org/licenses/BSD-3-Clause and *
* http://www.boost.org/LICENSE_1_0.txt. *
******************************************************************************/
#include "caf/config.hpp"
#define CAF_SUITE io_dynamic_remote_actor_udp
#include "caf/test/dsl.hpp"
#include <vector>
#include <sstream>
#include <utility>
#include <algorithm>
#include "caf/all.hpp"
#include "caf/io/all.hpp"
using namespace caf;
namespace {
constexpr char local_host[] = "127.0.0.1";
class config : public actor_system_config {
public:
config() {
load<io::middleman>();
set("middleman.enable-udp", true);
add_message_type<std::vector<int>>("std::vector<int>");
if (auto err = parse(test::engine::argc(), test::engine::argv()))
CAF_FAIL("failed to parse config: " << to_string(err));
}
};
struct fixture {
// State for the server.
config server_side_config;
actor_system server_side;
io::middleman& server_side_mm;
// State for the client.
config client_side_config;
actor_system client_side;
io::middleman& client_side_mm;
fixture()
: server_side(server_side_config),
server_side_mm(server_side.middleman()),
client_side(client_side_config),
client_side_mm(client_side.middleman()) {
// nop
}
};
behavior make_pong_behavior() {
return {
[](int val) -> int {
++val;
CAF_MESSAGE("pong with " << val);
return val;
}
};
}
behavior make_ping_behavior(event_based_actor* self, const actor& pong) {
CAF_MESSAGE("ping with " << 0);
self->send(pong, 0);
return {
[=](int val) -> int {
if (val == 3) {
CAF_MESSAGE("ping with exit");
self->send_exit(self->current_sender(),
exit_reason::user_shutdown);
CAF_MESSAGE("ping quits");
self->quit();
}
CAF_MESSAGE("ping with " << val);
return val;
}
};
}
behavior make_sort_behavior() {
return {
[](std::vector<int>& vec) -> std::vector<int> {
CAF_MESSAGE("sorter received: " << deep_to_string(vec));
std::sort(vec.begin(), vec.end());
CAF_MESSAGE("sorter sent: " << deep_to_string(vec));
return std::move(vec);
}
};
}
behavior make_sort_requester_behavior(event_based_actor* self,
const actor& sorter) {
self->send(sorter, std::vector<int>{5, 4, 3, 2, 1});
return {
[=](const std::vector<int>& vec) {
CAF_MESSAGE("sort requester received: " << deep_to_string(vec));
std::vector<int> expected_vec{1, 2, 3, 4, 5};
CAF_CHECK_EQUAL(vec, expected_vec);
self->send_exit(sorter, exit_reason::user_shutdown);
self->quit();
}
};
}
behavior fragile_mirror(event_based_actor* self) {
return {
[=](int i) {
self->quit(exit_reason::user_shutdown);
return i;
}
};
}
behavior linking_actor(event_based_actor* self, const actor& buddy) {
CAF_MESSAGE("link to mirror and send dummy message");
self->link_to(buddy);
self->send(buddy, 42);
return {
[](int i) {
CAF_CHECK_EQUAL(i, 42);
}
};
}
} // namespace <anonymous>
CAF_TEST_FIXTURE_SCOPE(dynamic_remote_actor_tests_udp, fixture)
CAF_TEST(identity_semantics_udp) {
// server side
auto server = server_side.spawn(make_pong_behavior);
auto port1 = unbox(server_side_mm.publish_udp(server, 0, local_host));
auto port2 = unbox(server_side_mm.publish_udp(server, 0, local_host));
CAF_REQUIRE_NOT_EQUAL(port1, port2);
auto same_server = unbox(server_side_mm.remote_actor_udp(local_host, port2));
CAF_REQUIRE_EQUAL(same_server, server);
CAF_CHECK_EQUAL(same_server->node(), server_side.node());
auto server1 = unbox(client_side_mm.remote_actor_udp(local_host, port1));
auto server2 = unbox(client_side_mm.remote_actor_udp(local_host, port2));
CAF_CHECK_EQUAL(server1, client_side_mm.remote_actor_udp(local_host, port1));
CAF_CHECK_EQUAL(server2, client_side_mm.remote_actor_udp(local_host, port2));
anon_send_exit(server, exit_reason::user_shutdown);
}
CAF_TEST(ping_pong_udp) {
// server side
auto port = unbox(server_side_mm.publish_udp(
server_side.spawn(make_pong_behavior), 0, local_host));
// client side
auto pong = unbox(client_side_mm.remote_actor_udp(local_host, port));
client_side.spawn(make_ping_behavior, pong);
}
CAF_TEST(custom_message_type_udp) {
// server side
auto port = unbox(server_side_mm.publish_udp(
server_side.spawn(make_sort_behavior), 0, local_host));
// client side
auto sorter = unbox(client_side_mm.remote_actor_udp(local_host, port));
client_side.spawn(make_sort_requester_behavior, sorter);
}
CAF_TEST(remote_link_udp) {
// server side
auto port = unbox(server_side_mm.publish_udp(
server_side.spawn(fragile_mirror), 0, local_host));
// client side
auto mirror = unbox(client_side_mm.remote_actor_udp(local_host, port));
auto linker = client_side.spawn(linking_actor, mirror);
scoped_actor self{client_side};
self->wait_for(linker);
CAF_MESSAGE("linker exited");
self->wait_for(mirror);
CAF_MESSAGE("mirror exited");
}
CAF_TEST_DISABLED(multiple_endpoints_udp) {
config cfg;
// Setup server.
CAF_MESSAGE("creating server");
actor_system server_sys{cfg};
auto mirror = server_sys.spawn([]() -> behavior {
return {
[] (std::string str) {
std::reverse(begin(str), end(str));
return str;
}
};
});
auto port = server_sys.middleman().publish_udp(mirror, 0);
CAF_REQUIRE(port);
CAF_MESSAGE("server running on port " << port);
auto client_fun = [](event_based_actor* self) -> behavior {
return {
[=](actor s) {
self->send(s, "hellow, world");
},
[=](const std::string& str) {
CAF_CHECK_EQUAL(str, "dlrow ,wolleh");
self->quit();
CAF_MESSAGE("done");
}
};
};
// Setup a client.
CAF_MESSAGE("creating first client");
config client_cfg;
actor_system client_sys{client_cfg};
auto client = client_sys.spawn(client_fun);
// Acquire remote actor from the server.
auto client_srv = client_sys.middleman().remote_actor_udp("localhost", *port);
CAF_REQUIRE(client_srv);
// Setup other clients.
for (int i = 0; i < 5; ++i) {
config other_cfg;
actor_system other_sys{other_cfg};
CAF_MESSAGE("creating new client");
auto other = other_sys.spawn(client_fun);
// Acquire remote actor from the new server.
auto other_srv = other_sys.middleman().remote_actor_udp("localhost", *port);
CAF_REQUIRE(other_srv);
// Establish communication and exit.
CAF_MESSAGE("client contacts server and exits");
anon_send(other, *other_srv);
other_sys.await_all_actors_done();
}
// Start communicate from the first actor.
CAF_MESSAGE("first client contacts server and exits");
anon_send(client, *client_srv);
client_sys.await_all_actors_done();
anon_send_exit(mirror, exit_reason::user_shutdown);
}
CAF_TEST_FIXTURE_SCOPE_END()
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