Commit fc608d33 authored by Dominik Charousset's avatar Dominik Charousset Committed by Marian Triebe

Enable streams to cross the wire, relates #531

parent 5de3e082
......@@ -14,32 +14,38 @@ set (LIBCAF_CORE_SRCS
src/abstract_actor.cpp
src/abstract_channel.cpp
src/abstract_composable_behavior.cpp
src/abstract_group.cpp
src/abstract_coordinator.cpp
src/abstract_downstream.cpp
src/abstract_group.cpp
src/abstract_upstream.cpp
src/actor.cpp
src/actor_addr.cpp
src/actor_companion.cpp
src/actor_config.cpp
src/actor_control_block.cpp
src/actor_companion.cpp
src/actor_ostream.cpp
src/actor_pool.cpp
src/actor_proxy.cpp
src/actor_registry.cpp
src/actor_system.cpp
src/actor_system_config.cpp
src/anycast.cpp
src/atom.cpp
src/attachable.cpp
src/behavior.cpp
src/behavior_stack.cpp
src/behavior_impl.cpp
src/behavior_stack.cpp
src/blocking_actor.cpp
src/blocking_behavior.cpp
src/broadcast.cpp
src/concatenated_tuple.cpp
src/config_option.cpp
src/continue_helper.cpp
src/decorated_tuple.cpp
src/default_attachable.cpp
src/deserializer.cpp
src/downstream_path.cpp
src/downstream_policy.cpp
src/duration.cpp
src/dynamic_message_data.cpp
src/error.cpp
......@@ -50,69 +56,67 @@ set (LIBCAF_CORE_SRCS
src/get_mac_addresses.cpp
src/get_process_id.cpp
src/get_root_uuid.cpp
src/greedy.cpp
src/group.cpp
src/group_manager.cpp
src/group_module.cpp
src/incoming_stream_multiplexer.cpp
src/invoke_result_visitor.cpp
src/match_case.cpp
src/merged_tuple.cpp
src/monitorable_actor.cpp
src/local_actor.cpp
src/logger.cpp
src/mailbox_element.cpp
src/match_case.cpp
src/memory.cpp
src/memory_managed.cpp
src/merged_tuple.cpp
src/message.cpp
src/message_builder.cpp
src/message_data.cpp
src/message_handler.cpp
src/message_view.cpp
src/monitorable_actor.cpp
src/node_id.cpp
src/outgoing_stream_multiplexer.cpp
src/parse_ini.cpp
src/private_thread.cpp
src/ref_counted.cpp
src/proxy_registry.cpp
src/response_promise.cpp
src/raw_event_based_actor.cpp
src/ref_counted.cpp
src/replies_to.cpp
src/response_promise.cpp
src/resumable.cpp
src/ripemd_160.cpp
src/scheduled_actor.cpp
src/scoped_actor.cpp
src/scoped_execution_unit.cpp
src/sec.cpp
src/serializer.cpp
src/sequencer.cpp
src/serializer.cpp
src/shared_spinlock.cpp
src/skip.cpp
src/splitter.cpp
src/sync_request_bouncer.cpp
src/stringification_inspector.cpp
src/test_coordinator.cpp
src/term.cpp
src/timestamp.cpp
src/try_match.cpp
src/type_erased_value.cpp
src/type_erased_tuple.cpp
src/uniform_type_info_map.cpp
src/unprofiled.cpp
src/greedy.cpp
src/broadcast.cpp
src/anycast.cpp
src/work_stealing.cpp
src/work_sharing.cpp
src/abstract_downstream.cpp
src/abstract_upstream.cpp
src/downstream_path.cpp
src/downstream_policy.cpp
src/stream.cpp
src/stream_handler.cpp
src/stream_msg_visitor.cpp
src/stream_multiplexer.cpp
src/stream_priority.cpp
src/stream_sink.cpp
src/stream_source.cpp
src/stream_stage.cpp
src/stringification_inspector.cpp
src/sync_request_bouncer.cpp
src/term.cpp
src/test_coordinator.cpp
src/timestamp.cpp
src/try_match.cpp
src/type_erased_tuple.cpp
src/type_erased_value.cpp
src/uniform_type_info_map.cpp
src/unprofiled.cpp
src/upstream_path.cpp
src/upstream_policy.cpp)
src/upstream_policy.cpp
src/work_sharing.cpp
src/work_stealing.cpp)
add_custom_target(libcaf_core)
......
......@@ -20,6 +20,7 @@
#ifndef CAF_ACTOR_SYSTEM_HPP
#define CAF_ACTOR_SYSTEM_HPP
#include <array>
#include <mutex>
#include <atomic>
#include <string>
......@@ -127,6 +128,32 @@ public:
friend class io::middleman;
friend class abstract_actor;
/// The number of actors implictly spawned by the actor system on startup.
static constexpr size_t num_internal_actors = 3;
/// Returns the ID of an internal actor by its name.
/// @pre x in {'SpawnServ', 'ConfigServ', 'StreamServ'}
static constexpr size_t internal_actor_id(atom_value x) {
return x == atom("SpawnServ") ? 0 : (x == atom("ConfigServ") ? 1 : 2);
}
/// Returns the internal actor for dynamic spawn operations.
inline const strong_actor_ptr& spawn_serv() const {
return internal_actors_[internal_actor_id(atom("SpawnServ"))];
}
/// Returns the internal actor for storing the runtime configuration
/// for this actor system.
inline const strong_actor_ptr& config_serv() const {
return internal_actors_[internal_actor_id(atom("ConfigServ"))];
}
/// Returns the internal actor for managing streams that
/// cross network boundaries.
inline const strong_actor_ptr& stream_serv() const {
return internal_actors_[internal_actor_id(atom("StreamServ"))];
}
actor_system() = delete;
actor_system(const actor_system&) = delete;
actor_system& operator=(const actor_system&) = delete;
......@@ -518,6 +545,20 @@ private:
return res;
}
/// Sets the internal actor for dynamic spawn operations.
inline void spawn_serv(strong_actor_ptr x) {
internal_actors_[internal_actor_id(atom("SpawnServ"))] = std::move(x);
}
/// Sets the internal actor for storing the runtime configuration.
inline void config_serv(strong_actor_ptr x) {
internal_actors_[internal_actor_id(atom("ConfigServ"))] = std::move(x);
}
/// Sets the internal actor for managing streams that
/// cross network boundaries. Called in middleman::start.
void stream_serv(strong_actor_ptr x);
std::atomic<size_t> ids_;
uniform_type_info_map types_;
node_id node_;
......@@ -530,8 +571,8 @@ private:
opencl::manager* opencl_manager_;
crdt::replicator* replicator_;
bool await_actors_before_shutdown_;
strong_actor_ptr config_serv_;
strong_actor_ptr spawn_serv_;
// Stores SpawnServ, ConfigServ, and StreamServ
std::array<strong_actor_ptr, num_internal_actors> internal_actors_;
std::atomic<size_t> detached;
mutable std::mutex detached_mtx;
mutable std::condition_variable detached_cv;
......
/******************************************************************************
* ____ _ _____ *
* / ___| / \ | ___| C++ *
* | | / _ \ | |_ Actor *
* | |___ / ___ \| _| Framework *
* \____/_/ \_|_| *
* *
* Copyright (C) 2011 - 2016 *
* Dominik Charousset <dominik.charousset (at) haw-hamburg.de> *
* *
* Distributed under the terms and conditions of the BSD 3-Clause License or *
* (at your option) under the terms and conditions of the Boost Software *
* License 1.0. See accompanying files LICENSE and LICENSE_ALTERNATIVE. *
* *
* If you did not receive a copy of the license files, see *
* http://opensource.org/licenses/BSD-3-Clause and *
* http://www.boost.org/LICENSE_1_0.txt. *
******************************************************************************/
#ifndef CAF_DETAIL_INCOMING_STREAM_MULTIPLEXER_HPP
#define CAF_DETAIL_INCOMING_STREAM_MULTIPLEXER_HPP
#include <deque>
#include <cstdint>
#include <algorithm>
#include <unordered_map>
#include "caf/actor.hpp"
#include "caf/error.hpp"
#include "caf/local_actor.hpp"
#include "caf/stream_msg.hpp"
#include "caf/detail/stream_multiplexer.hpp"
namespace caf {
namespace detail {
// Forwards messages from local actors to a remote stream_serv.
class incoming_stream_multiplexer : public stream_multiplexer {
public:
/// Allow `variant` to recognize this type as a visitor.
using result_type = void;
incoming_stream_multiplexer(local_actor* self, backend& service);
void operator()(stream_msg& x);
/// @pre `self_->current_sender() != nullptr`
/// @pre `current_stream_state_ == streams_.end()`
void operator()(stream_msg::open&);
/// @pre `self_->current_sender() != nullptr`
/// @pre `current_stream_state_ != streams_.end()`
void operator()(stream_msg::ack_open&);
/// @pre `self_->current_sender() != nullptr`
/// @pre `current_stream_state_ != streams_.end()`
void operator()(stream_msg::batch&);
/// @pre `self_->current_sender() != nullptr`
/// @pre `current_stream_state_ != streams_.end()`
void operator()(stream_msg::ack_batch&);
/// @pre `self_->current_sender() != nullptr`
/// @pre `current_stream_state_ != streams_.end()`
void operator()(stream_msg::close&);
/// @pre `self_->current_sender() != nullptr`
/// @pre `current_stream_state_ != streams_.end()`
void operator()(stream_msg::abort&);
/// @pre `self_->current_sender() != nullptr`
/// @pre `current_stream_state_ != streams_.end()`
void operator()(stream_msg::downstream_failed&);
/// @pre `self_->current_sender() != nullptr`
/// @pre `current_stream_state_ != streams_.end()`
void operator()(stream_msg::upstream_failed&);
private:
// Forwards the current stream_msg upstream.
// @pre `current_stream_msg != nullptr`
void forward_to_upstream();
// Forwards the current stream_msg downstream.
// @pre `current_stream_msg != nullptr`
void forward_to_downstream();
};
} // namespace detail
} // namespace caf
#endif // CAF_DETAIL_INCOMING_STREAM_MULTIPLEXER_HPP
/******************************************************************************
* ____ _ _____ *
* / ___| / \ | ___| C++ *
* | | / _ \ | |_ Actor *
* | |___ / ___ \| _| Framework *
* \____/_/ \_|_| *
* *
* Copyright (C) 2011 - 2016 *
* Dominik Charousset <dominik.charousset (at) haw-hamburg.de> *
* *
* Distributed under the terms and conditions of the BSD 3-Clause License or *
* (at your option) under the terms and conditions of the Boost Software *
* License 1.0. See accompanying files LICENSE and LICENSE_ALTERNATIVE. *
* *
* If you did not receive a copy of the license files, see *
* http://opensource.org/licenses/BSD-3-Clause and *
* http://www.boost.org/LICENSE_1_0.txt. *
******************************************************************************/
#ifndef CAF_DETAIL_STREAM_SERV_DOWNSTREAM_HPP
#define CAF_DETAIL_STREAM_SERV_DOWNSTREAM_HPP
#include <deque>
#include <cstdint>
#include <algorithm>
#include <unordered_map>
#include "caf/actor.hpp"
#include "caf/error.hpp"
#include "caf/stream_msg.hpp"
#include "caf/local_actor.hpp"
#include "caf/detail/stream_multiplexer.hpp"
namespace caf {
namespace detail {
// Forwards messages from local actors to a remote stream_serv.
class outgoing_stream_multiplexer : public stream_multiplexer {
public:
/// Allow `variant` to recognize this type as a visitor.
using result_type = void;
outgoing_stream_multiplexer(local_actor* self, backend& service);
void operator()(stream_msg& x);
void operator()(stream_msg::open&);
void operator()(stream_msg::ack_open&);
void operator()(stream_msg::batch&);
void operator()(stream_msg::ack_batch&);
void operator()(stream_msg::close&);
void operator()(stream_msg::abort&);
void operator()(stream_msg::downstream_failed&);
void operator()(stream_msg::upstream_failed&);
private:
// Forwards the current stream_msg upstream.
// @pre `current_stream_msg != nullptr`
void forward_to_upstream();
// Forwards the current stream_msg downstream.
// @pre `current_stream_msg != nullptr`
void forward_to_downstream();
};
} // namespace detail
} // namespace caf
#endif // CAF_DETAIL_STREAM_SERV_DOWNSTREAM_HPP
/******************************************************************************
* ____ _ _____ *
* / ___| / \ | ___| C++ *
* | | / _ \ | |_ Actor *
* | |___ / ___ \| _| Framework *
* \____/_/ \_|_| *
* *
* Copyright (C) 2011 - 2016 *
* Dominik Charousset <dominik.charousset (at) haw-hamburg.de> *
* *
* Distributed under the terms and conditions of the BSD 3-Clause License or *
* (at your option) under the terms and conditions of the Boost Software *
* License 1.0. See accompanying files LICENSE and LICENSE_ALTERNATIVE. *
* *
* If you did not receive a copy of the license files, see *
* http://opensource.org/licenses/BSD-3-Clause and *
* http://www.boost.org/LICENSE_1_0.txt. *
******************************************************************************/
#ifndef CAF_DETAIL_STREAM_MULTIPLEXER_HPP
#define CAF_DETAIL_STREAM_MULTIPLEXER_HPP
#include <deque>
#include <vector>
#include <cstdint>
#include <algorithm>
#include <unordered_map>
#include "caf/node_id.hpp"
#include "caf/optional.hpp"
#include "caf/stream_msg.hpp"
#include "caf/message_id.hpp"
#include "caf/local_actor.hpp"
#include "caf/mailbox_element.hpp"
#include "caf/actor_control_block.hpp"
namespace caf {
namespace detail {
// Forwards messages from local actors to a remote stream_serv.
class stream_multiplexer {
public:
/// Represents a path to a remote spawn server and stores required meta data.
struct remote_path {
remote_path(remote_path&&) = default;
remote_path& operator=(remote_path&&) = default;
inline remote_path(strong_actor_ptr ptr)
: hdl(ptr),
credit(1),
in_flight(1) {
// We start at credit 1 and in_flight 1. This means sending the first
// message to a remote spawn server does not require previous handshaking
// (other than establishing a connection).
}
/// Handle to a remote stream server.
strong_actor_ptr hdl;
/// Buffer for outgoing messages (sent to the BASP broker).
std::deque<mailbox_element_ptr> buf;
/// Available credit for sending messages.
int32_t credit;
/// Capacity that we have granted the remote stream server.
int32_t in_flight;
};
/// Maps node IDs to remote paths.
using remote_paths = std::unordered_map<node_id, remote_path>;
/// The backend of a stream server downstream establishes connections to
/// remote stream servers via node ID.
class backend {
public:
backend(actor basp_ref);
virtual ~backend();
/// Returns a remote actor representing the stream serv of node `nid`.
/// Returns an invalid handle if a) `nid` is invalid or identitifies this
/// node, or b) the backend could not establish a connection.
virtual strong_actor_ptr remote_stream_serv(const node_id& nid) = 0;
/// Returns a reference to the basp_ broker.
inline actor& basp() {
return basp_;
}
/// Returns all known remote stream servers and available credit.
inline remote_paths& remotes() {
return remotes_;
}
inline const remote_paths& remotes() const {
return remotes_;
}
/// Queries whether `x` is a known remote node.
bool has_remote_path(const node_id& x) const {
return remotes().count(x) > 0;
}
/// Adds `ptr` as remote stream serv on `x`. This is a no-op if `x` already
/// has a known path.
void add_remote_path(const node_id& x, strong_actor_ptr ptr) {
remotes().emplace(x, std::move(ptr));
}
/// Called whenever `nid` grants us `x` more credit.
// @pre `current_stream_state_ != streams_.end()`
void add_credit(const node_id& nid, int32_t x);
// Drains as much from the buffer by sending messages to the remote
// spawn_serv as possible, i.e., as many messages as credit is available.
void drain_buf(remote_path& path);
protected:
/// A reference to the basp_ broker.
actor basp_;
/// Known remote stream servers and available credit.
remote_paths remotes_;
};
/// Stores previous and next stage for a stream as well as the corresponding
/// remote path.
struct stream_state {
strong_actor_ptr prev_stage;
strong_actor_ptr next_stage;
remote_path* rpath;
};
// Maps stream ID to stream states.
using stream_states = std::unordered_map<stream_id, stream_state>;
/// Creates a new stream multiplexer for `self`, using `service` to connect to
/// remote spawn servers, and `basp` to send messages to remotes.
/// @pre `self != nullptr && basp != nullptr`
stream_multiplexer(local_actor* self, backend& service);
/// Queries whether stream `x` is managed by this multiplexer.
bool has_stream(const stream_id& x) const {
return streams_.count(x) > 0;
}
protected:
// Dispatches `x` on the subtype `T`.
template <class T>
static void dispatch(T& derived, stream_msg& x) {
// Reject anonymous messages.
auto prev = derived.self_->current_sender();
if (prev != nullptr) {
// Set state for the message handlers.
derived.current_stream_msg_ = &x;
derived.current_stream_state_ = derived.streams_.find(x.sid);
// Make sure that handshakes are not received twice and drop
// non-handshake messages if no state for the stream is found.
if (holds_alternative<stream_msg::open>(x.content)) {
if (derived.current_stream_state_ == derived.streams_.end()) {
derived(get<stream_msg::open>(x.content));
} else {
CAF_LOG_ERROR("Received multiple handshakes for stream.");
derived.fail(sec::upstream_already_exists);
}
} else {
if (derived.current_stream_state_ != derived.streams_.end()) {
apply_visitor(derived, x.content);
} else {
CAF_LOG_ERROR("Unable to access required stream and/or path state.");
derived.fail(sec::invalid_stream_state);
}
}
}
}
// Returns a reference to the remote stream server instance for `nid`
// if a remote stream_serv is known or connecting is successful.
optional<remote_path&> get_remote_or_try_connect(const node_id& nid);
// Returns a reference to the stream state for `sid`.
optional<stream_state&> state_for(const stream_id& sid);
/// Assings new capacity (credit) to remote stream servers.
/// @pre `current_remote_path_ != remotes().end()`
void manage_credit();
// Aborts the current stream with error `reason`.
// @pre `current_stream_msg != nullptr`
void fail(error reason, strong_actor_ptr predecessor,
strong_actor_ptr successor = nullptr);
// Aborts the current stream with error `reason`, assuming `state_for` returns
// valid predecessor and successor.
// @pre `current_stream_msg != nullptr`
void fail(error reason);
// Sends message `x` to the local actor `dest`.
void send_local(strong_actor_ptr& dest, stream_msg&& x,
std::vector<strong_actor_ptr> stages = {},
message_id mid = message_id::make());
// Creates a new message for the BASP broker.
inline mailbox_element_ptr
make_basp_message(remote_path& path, message&& x,
std::vector<strong_actor_ptr> stages = {},
message_id mid = message_id::make()) {
return make_mailbox_element(self_->ctrl(), message_id::make(), {},
forward_atom::value,
strong_actor_ptr{self_->ctrl()},
std::move(stages), path.hdl, mid, std::move(x));
}
// Sends message `x` to the remote stream server `dest`.
inline void send_remote(remote_path& path, stream_msg&& x,
std::vector<strong_actor_ptr> stages = {},
message_id mid = message_id::make()) {
path.buf.emplace_back(make_basp_message(path, make_message(std::move(x)),
std::move(stages), mid));
service_.drain_buf(path);
}
// Sends the control message `x` to the remote stream server `dest`. A control
// message signals capaticity and therefore does not use credit on its own and
// is sent immediately.
inline void send_remote_ctrl(remote_path& path, message&& x) {
basp()->enqueue(make_basp_message(path, std::move(x)), self_->context());
}
/// Returns a reference to the basp_ broker.
inline actor& basp() const {
return service_.basp();
}
/// Returns all known remote stream servers and available credit.
inline remote_paths& remotes() {
return service_.remotes();
}
/// Returns all known remote stream servers and available credit.
inline const remote_paths& remotes() const {
return service_.remotes();
}
// Stores which stream is currently processed.
stream_msg* current_stream_msg_ = nullptr;
/// Stores which stream state belongs to `current_stream_msg_`.
stream_states::iterator current_stream_state_;
// The parent actor.
local_actor* self_;
// The remoting backend.
backend& service_;
// Open streams.
stream_states streams_;
};
} // namespace detail
} // namespace caf
#endif // CAF_DETAIL_STREAM_MULTIPLEXER_HPP
......@@ -32,11 +32,11 @@ class forwarding_actor_proxy : public actor_proxy {
public:
using forwarding_stack = std::vector<strong_actor_ptr>;
forwarding_actor_proxy(actor_config& cfg, actor mgr);
forwarding_actor_proxy(actor_config& cfg, actor dest, actor stream_serv1);
~forwarding_actor_proxy() override;
void enqueue(mailbox_element_ptr what, execution_unit* host) override;
void enqueue(mailbox_element_ptr what, execution_unit* context) override;
bool link_impl(linking_operation op, abstract_actor* other) override;
......@@ -46,16 +46,13 @@ public:
void kill_proxy(execution_unit* ctx, error rsn) override;
actor manager() const;
void manager(actor new_manager);
private:
void forward_msg(strong_actor_ptr sender, message_id mid, message msg,
const forwarding_stack* fwd = nullptr);
mutable detail::shared_spinlock manager_mtx_;
actor manager_;
mutable detail::shared_spinlock mtx_;
actor broker_;
actor stream_serv_;
};
} // namespace caf
......
/******************************************************************************
* ____ _ _____ *
* / ___| / \ | ___| C++ *
* | | / _ \ | |_ Actor *
* | |___ / ___ \| _| Framework *
* \____/_/ \_|_| *
* *
* Copyright (C) 2011 - 2016 *
* Dominik Charousset <dominik.charousset (at) haw-hamburg.de> *
* *
* Distributed under the terms and conditions of the BSD 3-Clause License or *
* (at your option) under the terms and conditions of the Boost Software *
* License 1.0. See accompanying files LICENSE and LICENSE_ALTERNATIVE. *
* *
* If you did not receive a copy of the license files, see *
* http://opensource.org/licenses/BSD-3-Clause and *
* http://www.boost.org/LICENSE_1_0.txt. *
******************************************************************************/
#ifndef CAF_RAW_EVENT_BASED_ACTOR_HPP
#define CAF_RAW_EVENT_BASED_ACTOR_HPP
#include "caf/event_based_actor.hpp"
namespace caf {
/// A cooperatively raw scheduled actor is a dynamically typed actor that does
/// not handle any system messages. All handler for system messages as well as
/// the default handler are ignored. This actor type is for testing and
/// system-level actors.
/// @extends event_based_actor
class raw_event_based_actor : public event_based_actor {
public:
// -- member types -----------------------------------------------------------
/// Required by `spawn` for type deduction.
using signatures = none_t;
/// Required by `spawn` for type deduction.
using behavior_type = behavior;
// -- constructors and destructors -------------------------------------------
explicit raw_event_based_actor(actor_config& cfg);
invoke_message_result consume(mailbox_element& x) override;
};
} // namespace caf
#endif // CAF_RAW_EVENT_BASED_ACTOR_HPP
......@@ -29,6 +29,7 @@
#include <type_traits>
#include "caf/fwd.hpp"
#include "caf/sec.hpp"
#include "caf/error.hpp"
#include "caf/extend.hpp"
#include "caf/local_actor.hpp"
......@@ -40,6 +41,8 @@
#include "caf/stream_source_impl.hpp"
#include "caf/stream_result_trait.hpp"
#include "caf/to_string.hpp"
#include "caf/policy/greedy.hpp"
#include "caf/policy/anycast.hpp"
#include "caf/policy/broadcast.hpp"
......@@ -476,7 +479,7 @@ public:
message_category categorize(mailbox_element& x);
/// Tries to consume `x`.
invoke_message_result consume(mailbox_element& x);
virtual invoke_message_result consume(mailbox_element& x);
/// Tries to consume `x`.
void consume(mailbox_element_ptr x);
......@@ -503,7 +506,8 @@ public:
inline bool has_behavior() const {
return !bhvr_stack_.empty()
|| !awaited_responses_.empty()
|| !multiplexed_responses_.empty();
|| !multiplexed_responses_.empty()
|| !streams_.empty();
}
inline behavior& current_behavior() {
......
......@@ -106,6 +106,8 @@ enum class sec : uint8_t {
no_downstream_stages_defined,
/// Actor failed to initialize state after receiving a stream handshake.
stream_init_failed,
/// Unable to process a stream since due to missing state.
invalid_stream_state,
/// A function view was called without assigning an actor first.
bad_function_call
};
......
......@@ -40,8 +40,19 @@ namespace caf {
/// Stream communication messages for handshaking, ACKing, data transmission,
/// etc.
struct stream_msg : tag::boxing_type {
/// A flow label characterizes nested types.
enum flow_label {
/// Identifies content types that only flow downstream.
flows_downstream,
/// Identifies content types that only flow upstream.
flows_upstream,
/// Identifies content types that propagate errors in both directions.
flows_both_ways
};
/// Initiates a stream handshake.
struct open {
/// Allows visitors to dispatch on this tag.
static constexpr flow_label label = flows_downstream;
/// Allows the testing DSL to unbox this type automagically.
using outer_type = stream_msg;
/// A type-erased stream<T> object for picking the correct message
......@@ -61,6 +72,8 @@ struct stream_msg : tag::boxing_type {
/// Acknowledges a previous `open` message and finalizes a stream handshake.
/// Also signalizes initial demand.
struct ack_open {
/// Allows visitors to dispatch on this tag.
static constexpr flow_label label = flows_upstream;
/// Allows the testing DSL to unbox this type automagically.
using outer_type = stream_msg;
/// Grants credit to the source.
......@@ -74,6 +87,8 @@ struct stream_msg : tag::boxing_type {
/// Transmits stream data.
struct batch {
/// Allows visitors to dispatch on this tag.
static constexpr flow_label label = flows_downstream;
/// Allows the testing DSL to unbox this type automagically.
using outer_type = stream_msg;
/// Size of the type-erased vector<T> (used credit).
......@@ -87,6 +102,8 @@ struct stream_msg : tag::boxing_type {
/// Cumulatively acknowledges received batches and signalizes new demand from
/// a sink to its source.
struct ack_batch {
/// Allows visitors to dispatch on this tag.
static constexpr flow_label label = flows_upstream;
/// Allows the testing DSL to unbox this type automagically.
using outer_type = stream_msg;
/// Newly available credit.
......@@ -97,12 +114,16 @@ struct stream_msg : tag::boxing_type {
/// Closes a stream after receiving an ACK for the last batch.
struct close {
/// Allows visitors to dispatch on this tag.
static constexpr flow_label label = flows_downstream;
/// Allows the testing DSL to unbox this type automagically.
using outer_type = stream_msg;
};
/// Propagates fatal errors.
struct abort {
/// Allows visitors to dispatch on this tag.
static constexpr flow_label label = flows_both_ways;
/// Allows the testing DSL to unbox this type automagically.
using outer_type = stream_msg;
/// Reason for shutting down the stream.
......@@ -113,6 +134,8 @@ struct stream_msg : tag::boxing_type {
/// awaits a `resume` message afterwards if the downstream path was
/// redeployable. Otherwise, this results in a fatal error.
struct downstream_failed {
/// Allows visitors to dispatch on this tag.
static constexpr flow_label label = flows_upstream;
/// Allows the testing DSL to unbox this type automagically.
using outer_type = stream_msg;
/// Exit reason of the failing downstream path.
......@@ -123,6 +146,8 @@ struct stream_msg : tag::boxing_type {
/// awaits a `resume` message afterwards if the upstream path was
/// redeployable. Otherwise, this results in a fatal error.
struct upstream_failed {
/// Allows visitors to dispatch on this tag.
static constexpr flow_label label = flows_downstream;
/// Allows the testing DSL to unbox this type automagically.
using outer_type = stream_msg;
/// Exit reason of the failing upstream path.
......@@ -165,6 +190,12 @@ const T& get(const stream_msg& x) {
return get<T>(x.content);
}
/// Allows the testing DSL to check whether `stream_msg` holds a `T`.
template <class T>
bool is(const stream_msg& x) {
return holds_alternative<T>(x.content);
}
template <class T, class... Ts>
typename std::enable_if<
detail::tl_contains<
......@@ -185,7 +216,8 @@ typename Inspector::result_type inspect(Inspector& f, stream_msg::open& x) {
template <class Inspector>
typename Inspector::result_type inspect(Inspector& f, stream_msg::ack_open& x) {
return f(meta::type_name("ok"), x.initial_demand, x.filter, x.redeployable);
return f(meta::type_name("ack_open"), x.initial_demand, x.filter,
x.redeployable);
}
template <class Inspector>
......@@ -196,7 +228,7 @@ typename Inspector::result_type inspect(Inspector& f, stream_msg::batch& x) {
template <class Inspector>
typename Inspector::result_type inspect(Inspector& f,
stream_msg::ack_batch& x) {
return f(meta::type_name("demand"), x.new_capacity, x.acknowledged_id);
return f(meta::type_name("ack_batch"), x.new_capacity, x.acknowledged_id);
}
template <class Inspector>
......
......@@ -26,14 +26,22 @@ namespace caf {
/// Categorizes individual streams.
enum class stream_priority {
/// Denotes best-effort traffic.
low,
/// Denotes soft-realtime traffic.
very_high,
/// Denotes time-sensitive traffic.
high,
/// Denotes traffic with moderate timing requirements.
normal,
/// Denotes soft-realtime traffic.
high
/// Denotes uncritical traffic without timing requirements.
low,
/// Denotes best-effort traffic.
very_low
};
/// Stores the number of `stream_priority` classes.
static constexpr size_t stream_priorities = 5;
/// @relates stream_priority
std::string to_string(stream_priority x);
} // namespace caf
......
......@@ -42,7 +42,6 @@ abstract_downstream::~abstract_downstream() {
size_t abstract_downstream::total_credit() const {
auto f = [](size_t x, path_cref y) {
// printf("f(%d, %d)\n", (int) x, (int) y.open_credit);
return x + y.open_credit;
};
return fold_paths(0, f);
......
......@@ -25,6 +25,7 @@
#include "caf/to_string.hpp"
#include "caf/event_based_actor.hpp"
#include "caf/actor_system_config.hpp"
#include "caf/raw_event_based_actor.hpp"
#include "caf/policy/work_sharing.hpp"
#include "caf/policy/work_stealing.hpp"
......@@ -145,6 +146,12 @@ behavior config_serv_impl(stateful_actor<kvstate>* self) {
};
}
// -- spawn server -------------------------------------------------------------
// A spawn server allows users to spawn actors dynamically with a name and a
// message containing the data for initialization. By accessing the spawn server
// on another node, users can spwan actors remotely.
struct spawn_serv_state {
static const char* name;
};
......@@ -164,6 +171,20 @@ behavior spawn_serv_impl(stateful_actor<spawn_serv_state>* self) {
};
}
// -- stream server ------------------------------------------------------------
// The stream server acts as a man-in-the-middle for all streams that cross the
// network. It manages any number of unrelated streams by placing itself and the
// stream server on the next remote node into the pipeline.
// Outgoing messages are buffered in FIFO order to ensure fairness. However, the
// stream server uses five different FIFO queues: on for each priority level.
// A high priority grants more network bandwidth.
// Note that stream servers do not actively take part in the streams they
// process. Batch messages and ACKs are treated equally. Open, close, and error
// messages are evaluated to add and remove state as needed.
class dropping_execution_unit : public execution_unit {
public:
dropping_execution_unit(actor_system* sys) : execution_unit(sys) {
......@@ -262,12 +283,12 @@ actor_system::actor_system(actor_system_config& cfg)
groups_.init(cfg);
// spawn config and spawn servers (lazily to not access the scheduler yet)
static constexpr auto Flags = hidden + lazy_init;
spawn_serv_ = actor_cast<strong_actor_ptr>(spawn<Flags>(spawn_serv_impl));
config_serv_ = actor_cast<strong_actor_ptr>(spawn<Flags>(config_serv_impl));
spawn_serv(actor_cast<strong_actor_ptr>(spawn<Flags>(spawn_serv_impl)));
config_serv(actor_cast<strong_actor_ptr>(spawn<Flags>(config_serv_impl)));
// fire up remaining modules
registry_.start();
registry_.put(atom("SpawnServ"), spawn_serv_);
registry_.put(atom("ConfigServ"), config_serv_);
registry_.put(atom("SpawnServ"), spawn_serv());
registry_.put(atom("ConfigServ"), config_serv());
for (auto& mod : modules_)
if (mod)
mod->start();
......@@ -279,14 +300,14 @@ actor_system::~actor_system() {
CAF_LOG_DEBUG("shutdown actor system");
if (await_actors_before_shutdown_)
await_all_actors_done();
// shutdown system-level servers
anon_send_exit(spawn_serv_, exit_reason::user_shutdown);
anon_send_exit(config_serv_, exit_reason::user_shutdown);
// release memory as soon as possible
spawn_serv_ = nullptr;
config_serv_ = nullptr;
// shutdown internal actors
for (auto& x : internal_actors_) {
anon_send_exit(x, exit_reason::user_shutdown);
x = nullptr;
}
registry_.erase(atom("SpawnServ"));
registry_.erase(atom("ConfigServ"));
registry_.erase(atom("StreamServ"));
// group module is the first one, relies on MM
groups_.stop();
// stop modules in reverse order
......@@ -423,4 +444,9 @@ actor_system::dyn_spawn_impl(const std::string& name, message& args,
return std::move(res.first);
}
void actor_system::stream_serv(strong_actor_ptr x) {
internal_actors_[internal_actor_id(atom("StreamServ"))] = std::move(x);
registry_.put(atom("StreamServ"), stream_serv());
}
} // namespace caf
......@@ -24,29 +24,21 @@
#include "caf/send.hpp"
#include "caf/locks.hpp"
#include "caf/logger.hpp"
#include "caf/stream_msg.hpp"
#include "caf/mailbox_element.hpp"
namespace caf {
forwarding_actor_proxy::forwarding_actor_proxy(actor_config& cfg, actor mgr)
forwarding_actor_proxy::forwarding_actor_proxy(actor_config& cfg, actor dest,
actor stream_serv)
: actor_proxy(cfg),
manager_(std::move(mgr)) {
broker_(std::move(dest)),
stream_serv_(std::move(stream_serv)) {
// nop
}
forwarding_actor_proxy::~forwarding_actor_proxy() {
if (manager_)
anon_send(manager_, make_message(delete_atom::value, node(), id()));
}
actor forwarding_actor_proxy::manager() const {
shared_lock<detail::shared_spinlock> guard_(manager_mtx_);
return manager_;
}
void forwarding_actor_proxy::manager(actor new_manager) {
std::unique_lock<detail::shared_spinlock> guard_(manager_mtx_);
manager_.swap(new_manager);
anon_send(broker_, make_message(delete_atom::value, node(), id()));
}
void forwarding_actor_proxy::forward_msg(strong_actor_ptr sender,
......@@ -55,25 +47,44 @@ void forwarding_actor_proxy::forward_msg(strong_actor_ptr sender,
CAF_LOG_TRACE(CAF_ARG(id()) << CAF_ARG(sender)
<< CAF_ARG(mid) << CAF_ARG(msg));
forwarding_stack tmp;
shared_lock<detail::shared_spinlock> guard_(manager_mtx_);
if (manager_)
manager_->enqueue(nullptr, invalid_message_id,
make_message(forward_atom::value, std::move(sender),
fwd != nullptr ? *fwd : tmp,
strong_actor_ptr{ctrl()},
mid, std::move(msg)),
nullptr);
shared_lock<detail::shared_spinlock> guard(mtx_);
if (broker_)
broker_->enqueue(nullptr, invalid_message_id,
make_message(forward_atom::value, std::move(sender),
fwd != nullptr ? *fwd : tmp,
strong_actor_ptr{ctrl()}, mid,
std::move(msg)),
nullptr);
}
void forwarding_actor_proxy::enqueue(mailbox_element_ptr what,
execution_unit*) {
execution_unit* context) {
CAF_PUSH_AID(0);
CAF_ASSERT(what);
forward_msg(std::move(what->sender), what->mid,
what->move_content_to_message(), &what->stages);
if (what->content().type_token() != make_type_token<stream_msg>()) {
forward_msg(std::move(what->sender), what->mid,
what->move_content_to_message(), &what->stages);
} else {
shared_lock<detail::shared_spinlock> guard(mtx_);
if (stream_serv_) {
// Push this actor the the forwarding stack and move the message
// to the stream_serv, which will intercept stream handshakes.
// Since the stream_serv becomes part of the pipeline, the proxy
// will never receive a stream_msg unless it is the initial handshake.
what->stages.emplace_back(ctrl());
auto msg = what->move_content_to_message();
auto prefix = make_message(sys_atom::value);
stream_serv_->enqueue(make_mailbox_element(std::move(what->sender),
what->mid,
std::move(what->stages),
prefix + msg),
context);
//what->stages.emplace_back(ctrl());
//stream_serv_->enqueue(std::move(what), context);
}
}
}
bool forwarding_actor_proxy::link_impl(linking_operation op,
abstract_actor* other) {
switch (op) {
......@@ -124,7 +135,12 @@ void forwarding_actor_proxy::local_unlink_from(abstract_actor* other) {
void forwarding_actor_proxy::kill_proxy(execution_unit* ctx, error rsn) {
CAF_ASSERT(ctx != nullptr);
actor tmp{std::move(manager_)}; // manually break cycle
actor tmp[2];
{ // lifetime scope of guard
std::unique_lock<detail::shared_spinlock> guard(mtx_);
broker_.swap(tmp[0]); // manually break cycle
stream_serv_.swap(tmp[1]); // manually break cycle
}
cleanup(std::move(rsn), ctx);
}
......
/******************************************************************************
* ____ _ _____ *
* / ___| / \ | ___| C++ *
* | | / _ \ | |_ Actor *
* | |___ / ___ \| _| Framework *
* \____/_/ \_|_| *
* *
* Copyright (C) 2011 - 2016 *
* Dominik Charousset <dominik.charousset (at) haw-hamburg.de> *
* *
* Distributed under the terms and conditions of the BSD 3-Clause License or *
* (at your option) under the terms and conditions of the Boost Software *
* License 1.0. See accompanying files LICENSE and LICENSE_ALTERNATIVE. *
* *
* If you did not receive a copy of the license files, see *
* http://opensource.org/licenses/BSD-3-Clause and *
* http://www.boost.org/LICENSE_1_0.txt. *
******************************************************************************/
#include "caf/detail/incoming_stream_multiplexer.hpp"
#include "caf/send.hpp"
#include "caf/variant.hpp"
#include "caf/to_string.hpp"
#include "caf/local_actor.hpp"
namespace caf {
namespace detail {
incoming_stream_multiplexer::incoming_stream_multiplexer(local_actor* self,
backend& service)
: stream_multiplexer(self, service) {
// nop
}
void incoming_stream_multiplexer::operator()(stream_msg& x) {
CAF_LOG_TRACE(CAF_ARG(x));
CAF_ASSERT(self_->current_mailbox_element() != nullptr);
dispatch(*this, x);
}
void incoming_stream_multiplexer::operator()(stream_msg::open& x) {
CAF_LOG_TRACE(CAF_ARG(x));
CAF_ASSERT(current_stream_msg_ != nullptr);
auto prev = std::move(x.prev_stage);
// Make sure we have a previous stage.
if (!prev) {
CAF_LOG_WARNING("received stream_msg::open without previous stage");
return fail(sec::invalid_upstream, nullptr);
}
// Make sure we have a next stage.
auto cme = self_->current_mailbox_element();
if (!cme || cme->stages.empty()) {
CAF_LOG_WARNING("received stream_msg::open without next stage");
return fail(sec::invalid_downstream, std::move(prev));
}
auto successor = cme->stages.back();
cme->stages.pop_back();
// Our prev always is the remote stream server proxy.
auto current_remote_path = remotes().emplace(prev->node(), prev).first;
current_stream_state_ = streams_.emplace(current_stream_msg_->sid,
stream_state{std::move(prev),
successor,
&current_remote_path->second}).first;
// Rewrite handshake and forward it to the next stage.
x.prev_stage = self_->ctrl();
auto ptr = make_mailbox_element(
cme->sender, cme->mid, std::move(cme->stages),
make<stream_msg::open>(current_stream_msg_->sid, std::move(x.token),
self_->ctrl(), x.priority, std::move(x.topics),
x.redeployable));
successor->enqueue(std::move(ptr), self_->context());
// Send out demand upstream.
manage_credit();
}
void incoming_stream_multiplexer::operator()(stream_msg::ack_open&) {
CAF_ASSERT(current_stream_msg_ != nullptr);
CAF_ASSERT(current_stream_state_ != streams_.end());
forward_to_upstream();
}
void incoming_stream_multiplexer::operator()(stream_msg::batch&) {
CAF_ASSERT(current_stream_msg_ != nullptr);
CAF_ASSERT(current_stream_state_ != streams_.end());
forward_to_downstream();
}
void incoming_stream_multiplexer::operator()(stream_msg::ack_batch&) {
CAF_ASSERT(current_stream_msg_ != nullptr);
CAF_ASSERT(current_stream_state_ != streams_.end());
forward_to_upstream();
}
void incoming_stream_multiplexer::operator()(stream_msg::close&) {
CAF_ASSERT(current_stream_msg_ != nullptr);
CAF_ASSERT(current_stream_state_ != streams_.end());
auto i = streams_.find(current_stream_msg_->sid);
if (i != streams_.end()) {
forward_to_downstream();
streams_.erase(i);
}
}
void incoming_stream_multiplexer::operator()(stream_msg::abort& x) {
CAF_ASSERT(current_stream_msg_ != nullptr);
CAF_ASSERT(current_stream_state_ != streams_.end());
auto i = streams_.find(current_stream_msg_->sid);
if (i != streams_.end()) {
if (i->second.prev_stage == self_->current_sender())
fail(x.reason, nullptr, i->second.next_stage);
else
fail(x.reason, i->second.prev_stage);
streams_.erase(i);
}
}
void incoming_stream_multiplexer::operator()(stream_msg::downstream_failed&) {
CAF_ASSERT(current_stream_msg_ != nullptr);
CAF_ASSERT(current_stream_state_ != streams_.end());
}
void incoming_stream_multiplexer::operator()(stream_msg::upstream_failed&) {
CAF_ASSERT(current_stream_msg_ != nullptr);
CAF_ASSERT(current_stream_state_ != streams_.end());
}
void incoming_stream_multiplexer::forward_to_upstream() {
CAF_ASSERT(current_stream_msg_ != nullptr);
CAF_ASSERT(current_stream_state_ != streams_.end());
send_remote(*current_stream_state_->second.rpath,
std::move(*current_stream_msg_));
}
void incoming_stream_multiplexer::forward_to_downstream() {
CAF_ASSERT(current_stream_msg_ != nullptr);
// When forwarding downstream, we also have to manage upstream credit.
manage_credit();
send_local(current_stream_state_->second.next_stage,
std::move(*current_stream_msg_));
}
} // namespace detail
} // namespace caf
/******************************************************************************
* ____ _ _____ *
* / ___| / \ | ___| C++ *
* | | / _ \ | |_ Actor *
* | |___ / ___ \| _| Framework *
* \____/_/ \_|_| *
* *
* Copyright (C) 2011 - 2016 *
* Dominik Charousset <dominik.charousset (at) haw-hamburg.de> *
* *
* Distributed under the terms and conditions of the BSD 3-Clause License or *
* (at your option) under the terms and conditions of the Boost Software *
* License 1.0. See accompanying files LICENSE and LICENSE_ALTERNATIVE. *
* *
* If you did not receive a copy of the license files, see *
* http://opensource.org/licenses/BSD-3-Clause and *
* http://www.boost.org/LICENSE_1_0.txt. *
******************************************************************************/
#include "caf/detail/outgoing_stream_multiplexer.hpp"
#include "caf/send.hpp"
#include "caf/variant.hpp"
#include "caf/to_string.hpp"
#include "caf/local_actor.hpp"
namespace caf {
namespace detail {
outgoing_stream_multiplexer::outgoing_stream_multiplexer(local_actor* self,
backend& service)
: stream_multiplexer(self, service) {
// nop
}
void outgoing_stream_multiplexer::operator()(stream_msg& x) {
CAF_LOG_TRACE(CAF_ARG(x));
CAF_ASSERT(self_->current_mailbox_element() != nullptr);
dispatch(*this, x);
}
void outgoing_stream_multiplexer::operator()(stream_msg::open& x) {
CAF_LOG_TRACE(CAF_ARG(x));
CAF_ASSERT(current_stream_msg_ != nullptr);
auto predecessor = std::move(x.prev_stage);
// Make sure we a previous stage.
if (!predecessor) {
CAF_LOG_WARNING("received stream_msg::open without previous stage");
return fail(sec::invalid_upstream, nullptr);
}
// Make sure we don't receive a handshake for an already open stream.
if (streams_.count(current_stream_msg_->sid) > 0) {
CAF_LOG_WARNING("received stream_msg::open twice");
return fail(sec::upstream_already_exists, std::move(predecessor));
}
// Make sure we have a next stage.
auto cme = self_->current_mailbox_element();
if (cme->stages.empty()) {
CAF_LOG_WARNING("received stream_msg::open without next stage");
return fail(sec::invalid_downstream, std::move(predecessor));
}
auto successor = cme->stages.back();
// Get a connection to the responsible stream server.
auto path = get_remote_or_try_connect(successor->node());
if (!path) {
CAF_LOG_WARNING("cannot connect to remote stream server");
return fail(sec::cannot_connect_to_node, std::move(predecessor));
}
// Update state and send handshake to remote stream_serv (via
// middleman/basp_broker).
streams_.emplace(current_stream_msg_->sid,
stream_state{std::move(predecessor), path->hdl,
&(*path)});
// Send handshake to remote stream_serv (via middleman/basp_broker). We need
// to send this message as `current_sender`. We have do bypass the queue
// since `send_remote` always sends the message from `self_`.
auto ptr = make_mailbox_element(
cme->sender, message_id::make(), {}, forward_atom::value, cme->sender,
std::move(cme->stages), path->hdl, cme->mid,
make_message(make<stream_msg::open>(current_stream_msg_->sid,
std::move(x.token), self_->ctrl(),
x.priority, std::move(x.topics),
x.redeployable)));
basp()->enqueue(std::move(ptr), self_->context());
}
void outgoing_stream_multiplexer::operator()(stream_msg::ack_open&) {
forward_to_upstream();
}
void outgoing_stream_multiplexer::operator()(stream_msg::batch&) {
forward_to_downstream();
}
void outgoing_stream_multiplexer::operator()(stream_msg::ack_batch&) {
forward_to_upstream();
}
void outgoing_stream_multiplexer::operator()(stream_msg::close&) {
CAF_ASSERT(current_stream_msg_ != nullptr);
auto i = streams_.find(current_stream_msg_->sid);
if (i != streams_.end()) {
forward_to_downstream();
streams_.erase(i);
}
}
void outgoing_stream_multiplexer::operator()(stream_msg::abort& x) {
CAF_ASSERT(current_stream_msg_ != nullptr);
auto i = streams_.find(current_stream_msg_->sid);
if (i != streams_.end()) {
if (i->second.prev_stage == self_->current_sender())
fail(x.reason, nullptr, i->second.next_stage);
else
fail(x.reason, i->second.prev_stage);
streams_.erase(i);
}
}
void outgoing_stream_multiplexer::operator()(stream_msg::downstream_failed&) {
CAF_ASSERT(current_stream_msg_ != nullptr);
}
void outgoing_stream_multiplexer::operator()(stream_msg::upstream_failed&) {
CAF_ASSERT(current_stream_msg_ != nullptr);
}
void outgoing_stream_multiplexer::forward_to_upstream() {
CAF_ASSERT(current_stream_msg_ != nullptr);
CAF_ASSERT(current_stream_state_ != streams_.end());
manage_credit();
send_local(current_stream_state_->second.prev_stage,
std::move(*current_stream_msg_));
}
void outgoing_stream_multiplexer::forward_to_downstream() {
CAF_ASSERT(current_stream_msg_ != nullptr);
CAF_ASSERT(current_stream_state_ != streams_.end());
send_remote(*current_stream_state_->second.rpath,
std::move(*current_stream_msg_));
}
} // namespace detail
} // namespace caf
/******************************************************************************
* ____ _ _____ *
* / ___| / \ | ___| C++ *
* | | / _ \ | |_ Actor *
* | |___ / ___ \| _| Framework *
* \____/_/ \_|_| *
* *
* Copyright (C) 2011 - 2016 *
* Dominik Charousset <dominik.charousset (at) haw-hamburg.de> *
* *
* Distributed under the terms and conditions of the BSD 3-Clause License or *
* (at your option) under the terms and conditions of the Boost Software *
* License 1.0. See accompanying files LICENSE and LICENSE_ALTERNATIVE. *
* *
* If you did not receive a copy of the license files, see *
* http://opensource.org/licenses/BSD-3-Clause and *
* http://www.boost.org/LICENSE_1_0.txt. *
******************************************************************************/
#include "caf/raw_event_based_actor.hpp"
#include "caf/detail/default_invoke_result_visitor.hpp"
namespace caf {
raw_event_based_actor::raw_event_based_actor(actor_config& cfg)
: event_based_actor(cfg) {
// nop
}
invoke_message_result raw_event_based_actor::consume(mailbox_element& x) {
CAF_LOG_TRACE(CAF_ARG(x));
current_element_ = &x;
CAF_LOG_RECEIVE_EVENT(current_element_);
// short-circuit awaited responses
if (!awaited_responses_.empty()) {
auto& pr = awaited_responses_.front();
// skip all messages until we receive the currently awaited response
if (x.mid != pr.first)
return im_skipped;
if (!pr.second(x.content())) {
// try again with error if first attempt failed
auto msg = make_message(make_error(sec::unexpected_response,
x.move_content_to_message()));
pr.second(msg);
}
awaited_responses_.pop_front();
return im_success;
}
// handle multiplexed responses
if (x.mid.is_response()) {
auto mrh = multiplexed_responses_.find(x.mid);
// neither awaited nor multiplexed, probably an expired timeout
if (mrh == multiplexed_responses_.end())
return im_dropped;
if (!mrh->second(x.content())) {
// try again with error if first attempt failed
auto msg = make_message(make_error(sec::unexpected_response,
x.move_content_to_message()));
mrh->second(msg);
}
multiplexed_responses_.erase(mrh);
return im_success;
}
auto& content = x.content();
// handle timeout messages
if (x.content().type_token() == make_type_token<timeout_msg>()) {
auto& tm = content.get_as<timeout_msg>(0);
auto tid = tm.timeout_id;
CAF_ASSERT(!x.mid.valid());
if (is_active_timeout(tid)) {
CAF_LOG_DEBUG("handle timeout message");
if (bhvr_stack_.empty())
return im_dropped;
bhvr_stack_.back().handle_timeout();
return im_success;
}
CAF_LOG_DEBUG("dropped expired timeout message");
return im_dropped;
}
// handle everything else as ordinary message
detail::default_invoke_result_visitor<event_based_actor> visitor{this};
bool skipped = false;
auto had_timeout = getf(has_timeout_flag);
if (had_timeout)
unsetf(has_timeout_flag);
// restore timeout at scope exit if message was skipped
auto timeout_guard = detail::make_scope_guard([&] {
if (skipped && had_timeout)
setf(has_timeout_flag);
});
auto call_default_handler = [&] {
auto sres = call_handler(default_handler_, this, x);
switch (sres.flag) {
default:
break;
case rt_error:
case rt_value:
visitor.visit(sres);
break;
case rt_skip:
skipped = true;
}
};
if (bhvr_stack_.empty()) {
call_default_handler();
return !skipped ? im_success : im_skipped;
}
auto& bhvr = bhvr_stack_.back();
switch (bhvr(visitor, x.content())) {
default:
break;
case match_case::skip:
skipped = true;
break;
case match_case::no_match:
call_default_handler();
}
return !skipped ? im_success : im_skipped;
// should be unreachable
CAF_CRITICAL("invalid message type");
}
} // namespace caf
......@@ -392,6 +392,8 @@ scheduled_actor::categorize(mailbox_element& x) {
} else if (i != e) {
if (i->second->done()) {
streams_.erase(i);
if (streams_.empty() && !has_behavior())
quit(exit_reason::normal);
}
}
return message_category::internal;
......
......@@ -64,6 +64,7 @@ const char* sec_strings[] = {
"invalid_downstream",
"no_downstream_stages_defined",
"stream_init_failed",
"invalid_stream_state",
"bad_function_call"
};
......
......@@ -82,7 +82,6 @@ auto stream_msg_visitor::operator()(stream_msg::open& x) -> result_type {
std::move(sid_), ic, std::move(filter), false)));
return {none, i_};
}
printf("error: %s\n", self_->system().render(initial_credit.error()).c_str());
self_->streams().erase(i_);
return fail(std::move(initial_credit.error()));
}
......
/******************************************************************************
* ____ _ _____ *
* / ___| / \ | ___| C++ *
* | | / _ \ | |_ Actor *
* | |___ / ___ \| _| Framework *
* \____/_/ \_|_| *
* *
* Copyright (C) 2011 - 2016 *
* Dominik Charousset <dominik.charousset (at) haw-hamburg.de> *
* *
* Distributed under the terms and conditions of the BSD 3-Clause License or *
* (at your option) under the terms and conditions of the Boost Software *
* License 1.0. See accompanying files LICENSE and LICENSE_ALTERNATIVE. *
* *
* If you did not receive a copy of the license files, see *
* http://opensource.org/licenses/BSD-3-Clause and *
* http://www.boost.org/LICENSE_1_0.txt. *
******************************************************************************/
#include "caf/detail/stream_multiplexer.hpp"
#include "caf/send.hpp"
#include "caf/variant.hpp"
#include "caf/to_string.hpp"
#include "caf/local_actor.hpp"
namespace caf {
namespace detail {
stream_multiplexer::backend::backend(actor basp_ref) : basp_(basp_ref) {
// nop
}
stream_multiplexer::backend::~backend() {
// nop
}
void stream_multiplexer::backend::add_credit(const node_id& nid, int32_t x) {
auto i = remotes().find(nid);
if (i != remotes().end()) {
auto& path = i->second;
path.credit += x;
drain_buf(path);
}
}
void stream_multiplexer::backend::drain_buf(remote_path& path) {
CAF_LOG_TRACE(CAF_ARG(path));
auto n = std::min(path.credit, static_cast<int32_t>(path.buf.size()));
if (n > 0) {
auto b = path.buf.begin();
auto e = b + n;
for (auto i = b; i != e; ++i)
basp()->enqueue(std::move(*i), nullptr);
path.buf.erase(b, e);
path.credit -= static_cast<int32_t>(n);
}
}
stream_multiplexer::stream_multiplexer(local_actor* self, backend& service)
: self_(self),
service_(service) {
CAF_ASSERT(self_ != nullptr);
}
optional<stream_multiplexer::remote_path&>
stream_multiplexer::get_remote_or_try_connect(const node_id& nid) {
auto i = remotes().find(nid);
if (i != remotes().end())
return i->second;
auto res = service_.remote_stream_serv(nid);
if (res)
return remotes().emplace(nid, std::move(res)).first->second;
return none;
}
optional<stream_multiplexer::stream_state&>
stream_multiplexer::state_for(const stream_id& sid) {
auto i = streams_.find(sid);
if (i != streams_.end())
return i->second;
return none;
}
void stream_multiplexer::manage_credit() {
auto& path = *current_stream_state_->second.rpath;
// todo: actual, adaptive credit management
if (--path.in_flight == 0) {
int32_t new_remote_credit = 5;
path.in_flight += new_remote_credit;
send_remote_ctrl(
path, make_message(sys_atom::value, ok_atom::value, new_remote_credit));
}
}
void stream_multiplexer::fail(error reason, strong_actor_ptr predecessor,
strong_actor_ptr successor) {
CAF_ASSERT(current_stream_msg_ != nullptr);
if (predecessor)
unsafe_send_as(self_, predecessor,
make<stream_msg::abort>(current_stream_msg_->sid, reason));
if (successor)
unsafe_send_as(self_, successor,
make<stream_msg::abort>(current_stream_msg_->sid, reason));
auto rp = self_->make_response_promise();
rp.deliver(std::move(reason));
}
void stream_multiplexer::fail(error reason) {
CAF_ASSERT(current_stream_msg_ != nullptr);
auto i = streams_.find(current_stream_msg_->sid);
if (i != streams_.end()) {
fail(std::move(reason), std::move(i->second.prev_stage),
std::move(i->second.next_stage));
streams_.erase(i);
} else {
fail(std::move(reason), nullptr, nullptr);
}
}
void stream_multiplexer::send_local(strong_actor_ptr& dest, stream_msg&& x,
std::vector<strong_actor_ptr> stages,
message_id mid) {
CAF_ASSERT(dest != nullptr);
dest->enqueue(make_mailbox_element(self_->ctrl(), mid, std::move(stages),
std::move(x)),
self_->context());
}
} // namespace detail
} // namespace caf
......@@ -23,10 +23,18 @@ namespace caf {
std::string to_string(stream_priority x) {
switch (x) {
default: return "invalid";
case stream_priority::low: return "low";
case stream_priority::normal: return "normal";
case stream_priority::high: return "high";
default:
return "invalid";
case stream_priority::very_high:
return "very_high";
case stream_priority::high:
return "high";
case stream_priority::normal:
return "normal";
case stream_priority::low:
return "low";
case stream_priority::very_low:
return "very_low";
}
}
......
This diff is collapsed.
This diff is collapsed.
......@@ -36,7 +36,7 @@ public:
~test_multiplexer() override;
expected<connection_handle> new_tcp_scribe(const std::string& host,
uint16_t port_hint) override;
uint16_t port_hint) override;
expected<void> assign_tcp_scribe(abstract_broker* ptr,
connection_handle hdl) override;
......@@ -70,6 +70,8 @@ public:
/// A buffer storing bytes.
using buffer_type = std::vector<char>;
using shared_buffer_type = std::shared_ptr<buffer_type>;
/// Models pending data on the network, i.e., the network
/// input buffer usually managed by the operating system.
buffer_type& virtual_network_buffer(connection_handle hdl);
......@@ -109,6 +111,13 @@ public:
/// Stores `hdl` as a pending connection for `src`.
void add_pending_connect(accept_handle src, connection_handle hdl);
/// Add `hdl` as a pending connect to `src` and provide a scribe on `peer`
/// that connects the buffers of `hdl` and `peer_hdl`. Calls
/// `add_pending_connect(...)` and `peer.provide_scribe(...)`.
void prepare_connection(accept_handle src, connection_handle hdl,
test_multiplexer& peer, std::string host,
uint16_t port, connection_handle peer_hdl);
using pending_connects_map = std::unordered_multimap<accept_handle,
connection_handle>;
......@@ -122,9 +131,12 @@ public:
/// Accepts a pending connect on `hdl`.
bool accept_connection(accept_handle hdl);
/// Poll data on all scribes.
bool read_data();
/// Reads data from the external input buffer until
/// the configured read policy no longer allows receiving.
void read_data(connection_handle hdl);
bool read_data(connection_handle hdl);
/// Appends `buf` to the virtual network buffer of `hdl`
/// and calls `read_data(hdl)` afterwards.
......@@ -150,14 +162,25 @@ private:
using guard_type = std::unique_lock<std::mutex>;
struct scribe_data {
buffer_type xbuf;
shared_buffer_type vn_buf_ptr;
shared_buffer_type wr_buf_ptr;
buffer_type& vn_buf;
buffer_type rd_buf;
buffer_type wr_buf;
buffer_type& wr_buf;
receive_policy::config recv_conf;
bool stopped_reading = false;
bool passive_mode = false;
bool stopped_reading;
bool passive_mode;
intrusive_ptr<scribe> ptr;
bool ack_writes = false;
bool ack_writes;
// Creates a mock-only scribe.
scribe_data();
// Creates an entangled scribe where the input of this scribe is
// output of another scribe and vice versa.
scribe_data(shared_buffer_type input, shared_buffer_type output);
scribe_data(const scribe_data&);
};
struct doorman_data {
......
......@@ -88,7 +88,8 @@ strong_actor_ptr basp_broker_state::make_proxy(node_id nid, actor_id aid) {
auto mm = &system().middleman();
actor_config cfg;
auto res = make_actor<forwarding_actor_proxy, strong_actor_ptr>(
aid, nid, &(self->home_system()), cfg, self);
aid, nid, &(self->home_system()), cfg, self,
actor_cast<actor>(self->home_system().stream_serv()));
strong_actor_ptr selfptr{self->ctrl()};
res->get()->attach_functor([=](const error& rsn) {
mm->backend().post([=] {
......
......@@ -17,6 +17,8 @@
* http://www.boost.org/LICENSE_1_0.txt. *
******************************************************************************/
#include "caf/io/middleman.hpp"
#include <tuple>
#include <cerrno>
#include <memory>
......@@ -35,11 +37,12 @@
#include "caf/make_counted.hpp"
#include "caf/scoped_actor.hpp"
#include "caf/function_view.hpp"
#include "caf/actor_registry.hpp"
#include "caf/event_based_actor.hpp"
#include "caf/actor_system_config.hpp"
#include "caf/raw_event_based_actor.hpp"
#include "caf/typed_event_based_actor.hpp"
#include "caf/io/middleman.hpp"
#include "caf/io/basp_broker.hpp"
#include "caf/io/system_messages.hpp"
......@@ -51,8 +54,9 @@
#include "caf/detail/ripemd_160.hpp"
#include "caf/detail/safe_equal.hpp"
#include "caf/detail/get_root_uuid.hpp"
#include "caf/actor_registry.hpp"
#include "caf/detail/get_mac_addresses.hpp"
#include "caf/detail/incoming_stream_multiplexer.hpp"
#include "caf/detail/outgoing_stream_multiplexer.hpp"
#ifdef CAF_USE_ASIO
#include "caf/io/network/asio_multiplexer.hpp"
......@@ -270,6 +274,92 @@ void middleman::start() {
}
auto basp = named_broker<basp_broker>(atom("BASP"));
manager_ = make_middleman_actor(system(), basp);
// Install stream serv into the actor system.
class stream_serv : public raw_event_based_actor,
public detail::stream_multiplexer::backend {
public:
stream_serv(actor_config& cfg, actor basp)
: raw_event_based_actor(cfg),
detail::stream_multiplexer::backend(std::move(basp)),
incoming_(this, *this),
outgoing_(this, *this) {
// nop
}
const char* name() const override {
return "stream_serv";
}
behavior make_behavior() override {
return {
[=](stream_msg& x) -> delegated<message> {
// Dispatching depends on the direction of the message.
if (outgoing_.has_stream(x.sid)) {
outgoing_(x);
} else {
incoming_(x);
}
return {};
},
[=](sys_atom, stream_msg& x) -> delegated<message> {
// Stream message received from a proxy, always results in a new
// stream from a local actor to a remote node.
CAF_ASSERT(holds_alternative<stream_msg::open>(x.content));
outgoing_(x);
return {};
},
[=](sys_atom, ok_atom, int32_t credit) {
CAF_ASSERT(current_mailbox_element() != nullptr);
auto cme = current_mailbox_element();
if (cme->sender != nullptr) {
auto& nid = cme->sender->node();
add_credit(nid, credit);
} else {
CAF_LOG_ERROR("Received credit from an anonmyous stream server.");
}
},
[=](exit_msg& x) {
if (x.reason)
quit(x.reason);
},
// Connects both incoming_ and outgoing_ to nid.
[=](connect_atom, const node_id& nid) {
send(basp_, forward_atom::value, nid, atom("ConfigServ"),
make_message(get_atom::value, atom("StreamServ")));
},
// Assumes `ptr` is a remote spawn server.
[=](strong_actor_ptr& ptr) {
if (ptr) {
add_remote_path(ptr->node(), ptr);
}
}
};
}
strong_actor_ptr remote_stream_serv(const node_id& nid) override {
strong_actor_ptr result;
// Ask remote config server for a handle to the remote spawn server.
scoped_actor self{system()};
self->send(basp_, forward_atom::value, nid, atom("ConfigServ"),
make_message(get_atom::value, atom("StreamServ")));
// Time out after 5 minutes.
self->receive(
[&](strong_actor_ptr& addr) {
result = std::move(addr);
},
after(std::chrono::minutes(5)) >> [] {
CAF_LOG_INFO("Accessing a remote spawn server timed out.");
}
);
return result;
}
private:
detail::incoming_stream_multiplexer incoming_;
detail::outgoing_stream_multiplexer outgoing_;
};
auto ssi = system().spawn<stream_serv, lazy_init + hidden>(actor_cast<actor>(basp));
system().stream_serv(actor_cast<strong_actor_ptr>(std::move(ssi)));
}
void middleman::stop() {
......
......@@ -28,6 +28,29 @@ namespace caf {
namespace io {
namespace network {
test_multiplexer::scribe_data::scribe_data()
: scribe_data(std::make_shared<buffer_type>(),
std::make_shared<buffer_type>()) {
// nop
}
test_multiplexer::scribe_data::scribe_data(shared_buffer_type input,
shared_buffer_type output)
: vn_buf_ptr(std::move(input)),
wr_buf_ptr(std::move(output)),
vn_buf(*vn_buf_ptr),
wr_buf(*wr_buf_ptr),
stopped_reading(false),
passive_mode(false),
ack_writes(false) {
// nop
}
test_multiplexer::scribe_data::scribe_data(const scribe_data& other)
: scribe_data(other.vn_buf_ptr, other.wr_buf_ptr) {
// nop
}
test_multiplexer::test_multiplexer(actor_system* sys) : multiplexer(sys) {
CAF_ASSERT(sys != nullptr);
}
......@@ -40,6 +63,7 @@ test_multiplexer::~test_multiplexer() {
expected<connection_handle>
test_multiplexer::new_tcp_scribe(const std::string& host, uint16_t port_hint) {
CAF_LOG_DEBUG("new tcp scribe " << CAF_ARG(host) << CAF_ARG(port_hint)); // DELME
guard_type guard{mx_};
connection_handle result;
auto i = scribes_.find(std::make_pair(host, port_hint));
......@@ -52,6 +76,8 @@ test_multiplexer::new_tcp_scribe(const std::string& host, uint16_t port_hint) {
expected<void> test_multiplexer::assign_tcp_scribe(abstract_broker* ptr,
connection_handle hdl) {
CAF_LOG_TRACE(CAF_ARG(hdl));
CAF_LOG_DEBUG("assign tcp scribe " << CAF_ARG(hdl)); // DELME
class impl : public scribe {
public:
impl(abstract_broker* self, connection_handle ch, test_multiplexer* mpx)
......@@ -121,6 +147,7 @@ connection_handle test_multiplexer::add_tcp_scribe(abstract_broker*,
expected<connection_handle>
test_multiplexer::add_tcp_scribe(abstract_broker* ptr, const std::string& host,
uint16_t desired_port) {
CAF_LOG_DEBUG("add tcp scribe " << CAF_ARG(host) << CAF_ARG(desired_port)); // DELME
auto hdl = new_tcp_scribe(host, desired_port);
if (!hdl)
return std::move(hdl.error());
......@@ -223,6 +250,8 @@ void test_multiplexer::run() {
void test_multiplexer::provide_scribe(std::string host, uint16_t desired_port,
connection_handle hdl) {
CAF_LOG_TRACE(CAF_ARG(host) << CAF_ARG(desired_port) << CAF_ARG(hdl));
CAF_LOG_DEBUG(CAF_ARG(host) << CAF_ARG(desired_port) << CAF_ARG(hdl)); // DELME
guard_type guard{mx_};
scribes_.emplace(std::make_pair(std::move(host), desired_port), hdl);
}
......@@ -237,7 +266,7 @@ void test_multiplexer::provide_acceptor(uint16_t desired_port,
/// the test program.
test_multiplexer::buffer_type&
test_multiplexer::virtual_network_buffer(connection_handle hdl) {
return scribe_data_[hdl].xbuf;
return scribe_data_[hdl].vn_buf;
}
test_multiplexer::buffer_type&
......@@ -291,6 +320,29 @@ void test_multiplexer::add_pending_connect(accept_handle src,
pending_connects_.emplace(src, hdl);
}
void test_multiplexer::prepare_connection(accept_handle src,
connection_handle hdl,
test_multiplexer& peer,
std::string host, uint16_t port,
connection_handle peer_hdl) {
CAF_ASSERT(this != &peer);
CAF_LOG_TRACE(CAF_ARG(src) << CAF_ARG(hdl) << CAF_ARG(host) << CAF_ARG(port)
<< CAF_ARG(peer_hdl));
auto input = std::make_shared<buffer_type>();
auto output = std::make_shared<buffer_type>();
CAF_LOG_DEBUG("insert scribe data for" << CAF_ARG(hdl));
auto res1 = scribe_data_.emplace(hdl, scribe_data{input, output});
if (!res1.second)
throw std::runtime_error("prepare_connection: handle already in use");
CAF_LOG_DEBUG("insert scribe data on peer for" << CAF_ARG(peer_hdl));
auto res2 = peer.scribe_data_.emplace(peer_hdl, scribe_data{output, input});
if (!res2.second)
throw std::runtime_error("prepare_connection: peer handle already in use");
provide_acceptor(port, src);
add_pending_connect(src, hdl);
peer.provide_scribe(std::move(host), port, peer_hdl);
}
test_multiplexer::pending_connects_map& test_multiplexer::pending_connects() {
return pending_connects_;
}
......@@ -301,55 +353,92 @@ bool test_multiplexer::has_pending_scribe(std::string x, uint16_t y) {
}
bool test_multiplexer::accept_connection(accept_handle hdl) {
CAF_LOG_TRACE(CAF_ARG(hdl));
if (passive_mode(hdl))
return false;
auto& dd = doorman_data_[hdl];
if (!dd.ptr)
return false;
// assign scribes to all pending connects if needed
auto rng = pending_connects_.equal_range(hdl);
for (auto i = rng.first; i != rng.second; ++i)
if (impl_ptr(i->second) == nullptr)
assign_tcp_scribe(dd.ptr->parent(), i->second);
if (!dd.ptr->new_connection())
passive_mode(hdl) = true;
return true;
}
void test_multiplexer::read_data(connection_handle hdl) {
bool test_multiplexer::read_data() {
CAF_LOG_TRACE("");
// scribe_data might change while we traverse it
std::vector<connection_handle> xs;
xs.reserve(scribe_data_.size());
for (auto& kvp : scribe_data_)
xs.emplace_back(kvp.first);
long hits = 0;
for (auto x : xs)
if (scribe_data_.count(x) > 0)
if (read_data(x))
++hits;
return hits > 0;
}
bool test_multiplexer::read_data(connection_handle hdl) {
CAF_LOG_TRACE(CAF_ARG(hdl));
if (passive_mode(hdl))
return;
return false;
flush_runnables();
scribe_data& sd = scribe_data_[hdl];
while (!sd.ptr)
exec_runnable();
switch (sd.recv_conf.first) {
case receive_policy_flag::exactly:
while (sd.xbuf.size() >= sd.recv_conf.second) {
sd.rd_buf.clear();
auto first = sd.xbuf.begin();
auto last = first + static_cast<ptrdiff_t>(sd.recv_conf.second);
sd.rd_buf.insert(sd.rd_buf.end(), first, last);
sd.xbuf.erase(first, last);
if (!sd.ptr->consume(this, sd.rd_buf.data(), sd.rd_buf.size()))
passive_mode(hdl) = true;
}
break;
case receive_policy_flag::at_least:
if (sd.xbuf.size() >= sd.recv_conf.second) {
sd.rd_buf.clear();
sd.rd_buf.swap(sd.xbuf);
if (!sd.ptr->consume(this, sd.rd_buf.data(), sd.rd_buf.size()))
passive_mode(hdl) = true;
}
break;
case receive_policy_flag::at_most:
auto max_bytes = static_cast<ptrdiff_t>(sd.recv_conf.second);
while (!sd.xbuf.empty()) {
sd.rd_buf.clear();
auto xbuf_size = static_cast<ptrdiff_t>(sd.xbuf.size());
auto first = sd.xbuf.begin();
auto last = (max_bytes < xbuf_size) ? first + max_bytes : sd.xbuf.end();
sd.rd_buf.insert(sd.rd_buf.end(), first, last);
sd.xbuf.erase(first, last);
if (!sd.ptr->consume(this, sd.rd_buf.data(), sd.rd_buf.size()))
passive_mode(hdl) = true;
}
if (!sd.ptr) {
CAF_LOG_DEBUG("No scribe available yet on" << CAF_ARG(hdl));
return false;
}
// count how many data packets we could dispatch
long hits = 0;
for (;;) {
switch (sd.recv_conf.first) {
case receive_policy_flag::exactly:
if (sd.vn_buf.size() >= sd.recv_conf.second) {
++hits;
sd.rd_buf.clear();
auto first = sd.vn_buf.begin();
auto last = first + static_cast<ptrdiff_t>(sd.recv_conf.second);
sd.rd_buf.insert(sd.rd_buf.end(), first, last);
sd.vn_buf.erase(first, last);
if (!sd.ptr->consume(this, sd.rd_buf.data(), sd.rd_buf.size()))
passive_mode(hdl) = true;
} else {
return hits > 0;
}
break;
case receive_policy_flag::at_least:
if (sd.vn_buf.size() >= sd.recv_conf.second) {
++hits;
sd.rd_buf.clear();
sd.rd_buf.swap(sd.vn_buf);
if (!sd.ptr->consume(this, sd.rd_buf.data(), sd.rd_buf.size()))
passive_mode(hdl) = true;
} else {
return hits > 0;
}
break;
case receive_policy_flag::at_most:
auto max_bytes = static_cast<ptrdiff_t>(sd.recv_conf.second);
if (!sd.vn_buf.empty()) {
++hits;
sd.rd_buf.clear();
auto xbuf_size = static_cast<ptrdiff_t>(sd.vn_buf.size());
auto first = sd.vn_buf.begin();
auto last = (max_bytes < xbuf_size) ? first + max_bytes : sd.vn_buf.end();
sd.rd_buf.insert(sd.rd_buf.end(), first, last);
sd.vn_buf.erase(first, last);
if (!sd.ptr->consume(this, sd.rd_buf.data(), sd.rd_buf.size()))
passive_mode(hdl) = true;
} else {
return hits > 0;
}
}
}
}
......
......@@ -20,7 +20,7 @@
#include "caf/config.hpp"
#define CAF_SUITE io_typed_remote_actor
#include "caf/test/unit_test.hpp"
#include "caf/test/dsl.hpp"
#include <thread>
#include <string>
......@@ -116,3 +116,4 @@ CAF_TEST(test_typed_remote_actor) {
auto argv = test::engine::argv();
run_server(argc, argv);
}
......@@ -50,6 +50,10 @@ struct has_outer_type {
template <class T, class U>
T get(const U&);
// enables ADL in `with_content`
template <class T, class U>
bool is(const U&);
struct wildcard { };
constexpr wildcard _ = wildcard{};
......@@ -223,7 +227,7 @@ public:
template <class... Us>
void with(Us&&... xs) {
auto tmp = std::make_tuple(std::forward<Ts>(xs)...);
auto tmp = std::make_tuple(std::forward<Us>(xs)...);
elementwise_compare_inspector<decltype(tmp)> inspector{tmp};
auto ys = this->template peek<Ts...>();
CAF_CHECK(inspector(get<0>(ys)));
......@@ -262,7 +266,9 @@ private:
void with_content(std::integral_constant<bool, true>, const U& x) {
elementwise_compare_inspector<U> inspector{x};
auto xs = this->template peek<typename T::outer_type>();
CAF_CHECK(inspect(inspector, const_cast<T&>(get<T>(get<0>(xs)))));
auto& x0 = get<0>(xs);
CAF_REQUIRE(is<T>(x0));
CAF_CHECK(inspect(inspector, const_cast<T&>(get<T>(x0))));
}
};
......@@ -344,3 +350,7 @@ struct test_coordinator_fixture {
CAF_MESSAGE("expect" << #types << "." << #fields); \
expect< CAF_EXPAND(CAF_DSL_LIST types) >().fields
#define expect_on(where, types, fields) \
CAF_MESSAGE(#where << ": expect" << #types << "." << #fields); \
where . expect< CAF_EXPAND(CAF_DSL_LIST types) >().fields
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