Commit 32321ab7 authored by Dominik Charousset's avatar Dominik Charousset

Eaglery create outbound paths and refine API

* create paths eagerly instead of delaying path creation until
  receiving ack_open
* create outbound paths in "pending mode", i.e., do not emit any batches
  on an outbound path with receiver slot 0
* remove unused state from `stream<T>`
* clean up implemenation for add_outbound_path and add_inbound_path
parent 994385fb
......@@ -41,6 +41,9 @@ public:
/// Type of `paths_`.
using typename super::map_type;
/// Unique pointer to an outbound path.
using typename super::unique_path_ptr;
/// Enables or disables output per path.
using filter_type = Filter;
......@@ -127,18 +130,24 @@ public:
// -- overridden functions ---------------------------------------------------
typename super::path_ptr add_path(stream_slots slots,
strong_actor_ptr target) override {
CAF_LOG_TRACE(CAF_ARG(slots) << CAF_ARG(target));
// Make sure we have state for the slot.
state_map_.emplace(slots.sender, path_state{});
// Append to `paths_`.
auto index = this->paths_.size();
auto result = super::add_path(slots, target);
if (result == nullptr)
return nullptr;
bool insert_path(unique_path_ptr ptr) override {
CAF_LOG_TRACE(CAF_ARG(ptr));
// Make sure state_map_ and paths_ are always equally sorted, otherwise
// we'll run into UB when calling `zip_foreach`.
CAF_ASSERT(state_map_.size() == this->paths_.size());
auto slot = ptr->slots.sender;
// Append to the regular path map.
if (!super::insert_path(std::move(ptr)))
return false;
// Append to the state map.
if (!state_map_.emplace(slot, path_state{}).second) {
super::remove_path(slot, none, true);
return false;
}
return true;
// Make sure state_map_ and paths_ are always equally sorted, otherwise
// we'll run into UB when calling `zip_foreach`.
/*
CAF_ASSERT(index == this->paths_.size() - 1);
CAF_ASSERT(result->slots == slots);
CAF_ASSERT(this->paths_.container().back().first == slots.sender);
......@@ -152,6 +161,7 @@ public:
swap(ys[index], *i);
}
return result;
*/
}
void emit_batches() override {
......
......@@ -88,10 +88,6 @@ public:
return buf_.size();
}
message make_handshake_token(stream_slot slot) const override {
return make_message(stream<T>{slot});
}
buffer_type& buf() {
return buf_;
}
......
......@@ -56,12 +56,6 @@ public:
delegate(x);
}
void operator()(stream_slot in, stream_slot out,
stream_manager_ptr& mgr) override {
// TODO: error handling
self_->build_pipeline(in, out, std::move(mgr));
}
private:
void deliver(response_promise& rp, error& x) {
CAF_LOG_DEBUG("report error back to requesting actor");
......
......@@ -61,11 +61,6 @@ public:
/// Called if the message handler returned any "ordinary" value.
virtual void operator()(message&) = 0;
/// Called if the message handler returned an `output_stream<...>` or a
/// `stream_result<...>`.
virtual void operator()(stream_slot in, stream_slot out,
stream_manager_ptr& mgr) = 0;
/// Called if the message handler returns "nothing", for example a
/// default-constructed `optional<T>`.
virtual void operator()(const none_t&) = 0;
......@@ -146,16 +141,14 @@ public:
/// Calls `(*this)(x.in(), x.out(), x.ptr())`.
template <class Out, class Tuple, class P>
void operator()(output_stream<Out, Tuple, P>& x) {
stream_manager_ptr ptr{std::move(x.ptr())};
(*this)(x.in(), x.out(), ptr);
void operator()(output_stream<Out, Tuple, P>&) {
(*this)();
}
/// Calls `(*this)(x.in(), 0, x.ptr())`.
template <class T, class P>
void operator()(stream_result<T, P>& x) {
stream_manager_ptr ptr{std::move(x.ptr())};
(*this)(x.in(), 0, ptr);
template <class T>
void operator()(stream_result<T>&) {
(*this)();
}
// -- visit API: return true if T was visited, false if T was skipped --------
......
......@@ -28,16 +28,16 @@ namespace caf {
namespace detail {
/// Identifies an unbound sequence of messages.
template <class Input, class Result, class Process, class Finalize>
class stream_sink_driver_impl final : public stream_sink_driver<Input, Result> {
template <class Input, class Process, class Finalize>
class stream_sink_driver_impl final : public stream_sink_driver<Input> {
public:
// -- member types -----------------------------------------------------------
using super = stream_sink_driver<Input, Result>;
using super = stream_sink_driver<Input>;
using typename super::input_type;
using trait = stream_sink_trait_t<Process, Finalize>;
using trait = stream_sink_trait_t<Process>;
using state_type = typename trait::state;
......@@ -48,12 +48,12 @@ public:
init(state_);
}
void process(std::vector<input_type>&& xs) override {
return trait::process::invoke(process_, state_, std::move(xs));
void process(std::vector<input_type>& xs) override {
return trait::process::invoke(process_, state_, xs);
}
message make_final_result() override {
return trait::finalize::invoke(finalize_, state_);
void finalize(const error& err) override {
finalize_(state_, err);
}
private:
......
......@@ -64,18 +64,13 @@ public:
CAF_LOG_TRACE(CAF_ARG(x));
using vec_type = std::vector<input_type>;
if (x.xs.match_elements<vec_type>()) {
auto& xs = x.xs.get_mutable_as<vec_type>(0);
driver_.process(std::move(xs));
driver_.process(x.xs.get_mutable_as<vec_type>(0));
return;
}
CAF_LOG_ERROR("received unexpected batch type (dropped)");
}
protected:
message make_final_result() override {
return driver_.make_final_result();
}
void finalize(const error& reason) override {
driver_.finalize(reason);
}
......
......@@ -27,48 +27,28 @@
namespace caf {
namespace detail {
template <class Scatterer, class Pull, class Done, class Finalize,
class HandshakeData>
class stream_source_driver_impl;
/// Identifies an unbound sequence of messages.
template <class Scatterer, class Pull, class Done, class Finalize, class... Ts>
class stream_source_driver_impl<Scatterer, Pull, Done, Finalize,
std::tuple<Ts...>>
final : public stream_source_driver<Scatterer, Ts...> {
template <class Scatterer, class Pull, class Done, class Finalize>
class stream_source_driver_impl final : public stream_source_driver<Scatterer> {
public:
// -- member types -----------------------------------------------------------
using super = stream_source_driver<Scatterer, Ts...>;
using super = stream_source_driver<Scatterer>;
using output_type = typename super::output_type;
using stream_type = stream<output_type>;
using output_stream_type = typename super::output_stream_type;
using tuple_type = std::tuple<Ts...>;
using handshake_tuple_type = typename super::handshake_tuple_type;
using trait = stream_source_trait_t<Pull>;
using state_type = typename trait::state;
template <class Init, class Tuple>
stream_source_driver_impl(Init init, Pull f, Done pred, Finalize fin,
Tuple&& hs)
template <class Init>
stream_source_driver_impl(Init init, Pull f, Done pred, Finalize fin)
: pull_(std::move(f)),
done_(std::move(pred)),
finalize_(std::move(fin)),
hs_(std::forward<Tuple>(hs)) {
finalize_(std::move(fin)) {
init(state_);
}
handshake_tuple_type make_handshake(stream_slot slot) const override {
return std::tuple_cat(std::make_tuple(stream_type{slot}), hs_);
}
void pull(downstream<output_type>& out, size_t num) override {
return pull_(state_, out, num);
}
......@@ -86,7 +66,6 @@ private:
Pull pull_;
Done done_;
Finalize finalize_;
tuple_type hs_;
};
} // namespace detail
......
......@@ -72,10 +72,6 @@ public:
return hint != this->out_.capacity();
}
message make_handshake(stream_slot slot) const override {
return make_message_from_tuple(driver_.make_handshake(slot));
}
protected:
void finalize(const error& reason) override {
driver_.finalize(reason);
......
......@@ -27,22 +27,16 @@
namespace caf {
namespace detail {
template <class Input, class Scatterer, class Process, class Finalize,
class HandshakeData>
class stream_stage_driver_impl;
/// Default implementation for a `stream_stage_driver` that hardwires `message`
/// as result type and implements `process` and `finalize` using user-provided
/// function objects (usually lambdas).
template <class Input, class Scatterer, class Process, class Finalize,
class... Ts>
class stream_stage_driver_impl<Input, Scatterer, Process, Finalize,
std::tuple<Ts...>>
final : public stream_stage_driver<Input, message, Scatterer, Ts...> {
template <class Input, class Scatterer, class Process, class Finalize>
class stream_stage_driver_impl final
: public stream_stage_driver<Input, Scatterer> {
public:
// -- member types -----------------------------------------------------------
using super = stream_stage_driver<Input, message, Scatterer, Ts...>;
using super = stream_stage_driver<Input, Scatterer>;
using typename super::input_type;
......@@ -50,47 +44,30 @@ public:
using typename super::stream_type;
using typename super::handshake_tuple_type;
using trait = stream_stage_trait_t<Process>;
using state_type = typename trait::state;
template <class Init, class Tuple>
stream_stage_driver_impl(Init init, Process f, Finalize fin, Tuple&& hs)
template <class Init>
stream_stage_driver_impl(Init init, Process f, Finalize fin)
: process_(std::move(f)),
fin_(std::move(fin)),
hs_(std::forward<Tuple>(hs)) {
fin_(std::move(fin)) {
init(state_);
}
handshake_tuple_type make_handshake(stream_slot slot) const override {
return std::tuple_cat(std::make_tuple(stream_type{slot}), hs_);
}
void process(std::vector<input_type>&& batch,
downstream<output_type>& out) override {
trait::process::invoke(process_, state_, std::move(batch), out);
}
void add_result(message& x) override {
// The default driver assumes to receive only a single result.
result_ = std::move(x);
}
message make_final_result() override {
return std::move(result_);
void process(downstream<output_type>& out,
std::vector<input_type>& batch) override {
trait::process::invoke(process_, state_, out, batch);
}
void finalize(const error&) override {
return fin_(state_);
void finalize(const error& err) override {
return fin_(state_, err);
}
private:
state_type state_;
Process process_;
Finalize fin_;
std::tuple<Ts...> hs_;
message result_;
};
......
......@@ -67,18 +67,13 @@ public:
CAF_LOG_TRACE(CAF_ARG(x));
using vec_type = std::vector<input_type>;
if (x.xs.match_elements<vec_type>()) {
auto& xs = x.xs.get_mutable_as<vec_type>(0);
downstream<output_type> ds{this->out_.buf()};
driver_.process(std::move(xs), ds);
driver_.process(ds, x.xs.get_mutable_as<vec_type>(0));
return;
}
CAF_LOG_ERROR("received unexpected batch type (dropped)");
}
message make_handshake(stream_slot slot) const override {
return make_message_from_tuple(driver_.make_handshake(slot));
}
bool congested() const noexcept override {
return this->out_.capacity() == 0;
}
......
......@@ -107,15 +107,25 @@ public:
/// A tuple holding all nested scatterers.
using nested_scatterers = std::tuple<T, Ts...>;
/// Pointer to an outbound path.
using typename super::path_ptr;
/// Unique pointer to an outbound path.
using typename super::unique_path_ptr;
/// State held for each slot.
struct non_owning_ptr {
outbound_path* ptr;
path_ptr ptr;
stream_scatterer* owner;
};
/// Maps slots to path and nested scatterer.
using map_type = detail::unordered_flat_map<stream_slot, non_owning_ptr>;
/// Maps slots to paths that haven't a scatterer assigned yet.
using unassigned_map_type = detail::unordered_flat_map<stream_slot,
unique_path_ptr>;
// -- constructors, destructors, and assignment operators --------------------
fused_scatterer(scheduled_actor* self)
......@@ -135,9 +145,34 @@ public:
// return std::get<U>(substreams_);
}
/// Requires a previous call to `add_path` for given slot.
template <class U>
void assign(stream_slot slot) {
paths_.emplace(slot, non_owning_ptr{nullptr, &get<U>()});
// Fetch pointer from the unassigned paths.
auto i = unassigned_paths_.find(slot);
if (i == unassigned_paths_.end()) {
CAF_LOG_ERROR("cannot assign nested scatterer to unknown slot");
return;
}
// Error or not, remove entry from unassigned_paths_ before leaving.
auto cleanup = detail::make_scope_guard([&] {
unassigned_paths_.erase(i);
});
// Transfer ownership to nested scatterer.
auto ptr = i->second.get();
CAF_ASSERT(ptr != nullptr);
auto owner = &get<U>();
if (!owner->insert_path(std::move(i->second))) {
CAF_LOG_ERROR("slot exists as unassigned and assigned");
return;
}
// Store owner and path in our map.
auto er = paths_.emplace(slot, non_owning_ptr{ptr, owner});
if (!er.second) {
CAF_LOG_ERROR("slot already mapped");
owner->remove_path(slot, sec::invalid_stream_state, false);
return;
}
}
// -- overridden functions ---------------------------------------------------
......@@ -146,32 +181,9 @@ public:
return paths_.size();
}
/// Requires a previous call to `assign<T>(slot.sender)`.
path_ptr add_path(stream_slots slots, strong_actor_ptr target) override {
CAF_LOG_TRACE(CAF_ARG(slots) << CAF_ARG(target));
auto e = paths_.end();
auto i = paths_.find(slots.sender);
if (i == e) {
CAF_LOG_ERROR("no scatterer assigned:" << CAF_ARG(slots.sender));
return nullptr;
}
if (i->second.ptr != nullptr) {
CAF_LOG_ERROR("multiple calls to add_path:" << CAF_ARG(slots.sender));
return nullptr;
}
auto result = i->second.owner->add_path(slots, target);
if (result == nullptr) {
CAF_LOG_ERROR("nested scatterer unable to add path:"
<< CAF_ARG(slots.sender));
paths_.erase(i);
return nullptr;
}
i->second.ptr = result;
return result;
}
bool remove_path(stream_slot slot, error reason,
bool silent) noexcept override {
CAF_LOG_TRACE(CAF_ARG(slot) << CAF_ARG(reason) << CAF_ARG(silent));
auto i = paths_.find(slot);
if (i == paths_.end())
return false;
......@@ -229,21 +241,21 @@ public:
return result;
}
message make_handshake_token(stream_slot slot) const override {
auto i = paths_.find(slot);
if (i != paths_.end())
return i->second.owner->make_handshake_token(slot);
CAF_LOG_ERROR("no scatterer available:" << CAF_ARG(slot));
return make_message(stream<message>{slot});
}
void clear_paths() override {
CAF_LOG_TRACE("");
for (auto ptr : ptrs_)
ptr->clear_paths();
paths_.clear();
}
protected:
bool insert_path(unique_path_ptr ptr) override {
CAF_LOG_TRACE(CAF_ARG(ptr));
CAF_ASSERT(ptr != nullptr);
auto slot = ptr->slots.sender;
return unassigned_paths_.emplace(slot, std::move(ptr)).second;
}
void for_each_path_impl(path_visitor& f) override {
for (auto& kvp : paths_)
f(*kvp.second.ptr);
......@@ -268,6 +280,7 @@ private:
nested_scatterers nested_;
stream_scatterer* ptrs_[sizeof...(Ts) + 1];
map_type paths_;
unassigned_map_type unassigned_paths_;
};
} // namespace caf
......
......@@ -34,6 +34,7 @@ template <class> class intrusive_ptr;
template <class> class optional;
template <class> class param;
template <class> class stream;
template <class> class stream_sink;
template <class> class trivial_match_case;
template <class> class weak_intrusive_ptr;
......@@ -41,7 +42,7 @@ template <class> struct timeout_definition;
// -- 2 param templates --------------------------------------------------------
template <class, class> class make_sink_result;
template <class, class> class stream_source;
// -- 3 param templates --------------------------------------------------------
......@@ -50,21 +51,19 @@ template <class, class, int> class actor_cast_access;
template <class, class, class> class broadcast_scatterer;
template <class, class, class> class broadcast_topic_scatterer;
template <class, class, class> class random_topic_scatterer;
template <class, class, class> class stream_stage;
// -- variadic templates -------------------------------------------------------
template <class...> class result;
template <class...> class delegated;
template <class...> class result;
template <class...> class typed_actor;
template <class...> class typed_actor_pointer;
template <class...> class typed_response_promise;
template <class...> class typed_event_based_actor;
template <class...> class typed_response_promise;
// -- variadic templates with fixed arguments ----------------------------------
template <class, class, class, class, class...> class stream_stage;
template <class, class, class...> class stream_source;
// -- classes ------------------------------------------------------------------
class actor;
......@@ -100,6 +99,7 @@ class proxy_registry;
class stream_manager;
class actor_companion;
class mailbox_element;
class message_builder;
class message_handler;
class scheduled_actor;
class stream_scatterer;
......@@ -231,7 +231,7 @@ using mailbox_element_ptr = std::unique_ptr<mailbox_element, detail::disposer>;
template <class, class = std::tuple<>, class = stream_manager_ptr>
class output_stream;
template <class, class = stream_manager_ptr>
template <class = stream_manager_ptr>
class stream_result;
} // namespace caf
......
......@@ -32,8 +32,6 @@ public:
size_t num_paths() const noexcept override;
path_ptr add_path(stream_slots slots, strong_actor_ptr target) override;
bool remove_path(stream_slot slots, error reason,
bool silent) noexcept override;
......@@ -47,9 +45,9 @@ public:
size_t buffered() const noexcept override;
message make_handshake_token(stream_slot slot) const override;
protected:
bool insert_path(unique_path_ptr) override;
void for_each_path_impl(path_visitor& f) override;
bool check_paths_impl(path_algorithm algo,
......
......@@ -276,6 +276,12 @@ public:
return current_element_;
}
/// Returns a pointer to the currently processed mailbox element.
/// @private
inline void current_mailbox_element(mailbox_element* ptr) {
current_element_ = ptr;
}
/// Adds a unidirectional `monitor` to `whom`.
/// @note Each call to `monitor` creates a new, independent monitor.
template <class Handle>
......
......@@ -23,9 +23,7 @@
#include "caf/stream_slot.hpp"
#include "caf/stream_source.hpp"
#include "caf/detail/type_traits.hpp"
#include "caf/meta/type_name.hpp"
#include "caf/detail/implicit_conversions.hpp"
namespace caf {
......@@ -37,7 +35,7 @@ struct make_source_result {
using value_type = typename Scatterer::value_type;
/// Fully typed stream manager as returned by `make_source`.
using source_type = stream_source<value_type, Scatterer, Ts...>;
using source_type = stream_source<value_type, Scatterer>;
/// Pointer to a fully typed stream manager.
using source_ptr_type = intrusive_ptr<source_type>;
......@@ -50,7 +48,8 @@ struct make_source_result {
/// the types of the handshake arguments.
template <class Scatterer, class... Ts>
using make_source_result_t =
typename make_source_result<Scatterer, detail::decay_t<Ts>...>::type;
typename make_source_result<Scatterer,
detail::strip_and_convert_t<Ts>...>::type;
} // namespace caf
......
......@@ -24,20 +24,20 @@
#include "caf/stream_slot.hpp"
#include "caf/stream_stage.hpp"
#include "caf/detail/type_traits.hpp"
#include "caf/detail/implicit_conversions.hpp"
namespace caf {
/// Helper trait for deducing an `output_stream` from the arguments to
/// `scheduled_actor::make_stage`.
template <class In, class Result, class Scatterer, class... Ts>
template <class In, class Scatterer, class... Ts>
class make_stage_result {
public:
/// Type of a single element.
using value_type = typename Scatterer::value_type;
/// Fully typed stream manager as returned by `make_stage`.
using stage_type = stream_stage<In, Result, value_type, Scatterer, Ts...>;
using stage_type = stream_stage<In, value_type, Scatterer>;
/// Pointer to a fully typed stream manager.
using stage_ptr_type = intrusive_ptr<stage_type>;
......@@ -50,8 +50,8 @@ public:
/// additional handshake types. Hardwires `message` as result type.
template <class In, class Scatterer, class... Ts>
using make_stage_result_t =
typename make_stage_result<In, message, Scatterer,
detail::decay_t<Ts>...>::type;
typename make_stage_result<In, Scatterer,
detail::strip_and_convert_t<Ts>...>::type;
} // namespace caf
......
......@@ -29,11 +29,12 @@
#include "caf/none.hpp"
#include "caf/intrusive_ptr.hpp"
#include "caf/message.hpp"
#include "caf/duration.hpp"
#include "caf/behavior.hpp"
#include "caf/ref_counted.hpp"
#include "caf/duration.hpp"
#include "caf/match_case.hpp"
#include "caf/may_have_timeout.hpp"
#include "caf/message.hpp"
#include "caf/ref_counted.hpp"
#include "caf/timeout_definition.hpp"
#include "caf/detail/behavior_impl.hpp"
......
......@@ -27,6 +27,7 @@
#include "caf/actor_control_block.hpp"
#include "caf/downstream_msg.hpp"
#include "caf/fwd.hpp"
#include "caf/logger.hpp"
#include "caf/stream_aborter.hpp"
#include "caf/stream_slot.hpp"
#include "caf/system_messages.hpp"
......@@ -64,8 +65,8 @@ public:
// -- constructors, destructors, and assignment operators --------------------
/// Constructs a path for given handle and stream ID.
outbound_path(stream_slots id, strong_actor_ptr ptr);
/// Constructs a pending path for given slot and handle.
outbound_path(stream_slot sender_slot, strong_actor_ptr receiver_hdl);
~outbound_path();
......@@ -83,10 +84,14 @@ public:
/// `xs_size` and increments `next_batch_id` by 1.
void emit_batch(local_actor* self, long xs_size, message xs);
/// Calls `emit_batch` for each
/// Calls `emit_batch` for each chunk in the cache, whereas each chunk is of
/// size `desired_batch_size`. Does nothing for pending paths.
template <class T>
void emit_batches(local_actor* self, std::vector<T>& cache,
bool force_underfull) {
CAF_LOG_TRACE(CAF_ARG(cache) << CAF_ARG(force_underfull));
if (pending())
return;
CAF_ASSERT(desired_batch_size > 0);
if (cache.size() == desired_batch_size) {
emit_batch(self, desired_batch_size, make_message(std::move(cache)));
......@@ -120,6 +125,14 @@ public:
const strong_actor_ptr& hdl,
error reason);
// -- properties -------------------------------------------------------------
/// Returns whether this path is pending, i.e., didn't receive an `ack_open`
/// yet.
inline bool pending() const noexcept {
return slots.receiver == invalid_stream_slot;
}
// -- member variables -------------------------------------------------------
/// Slot IDs for sender (self) and receiver (hdl).
......
......@@ -99,6 +99,15 @@ public:
return ptr_;
}
// -- conversions ------------------------------------------------------------
/// Rebinds the `ptr()` property.
template <class Manager>
output_stream<value_type, tuple_type, intrusive_ptr<Manager>>
rebind(Manager* ptr) const noexcept {
return {in_, out_, ptr};
}
private:
// -- member variables -------------------------------------------------------
......
This diff is collapsed.
......@@ -28,49 +28,9 @@
namespace caf {
/// Identifies an unbound sequence of elements.
/// Empty marker type for streaming handshakes.
template <class T>
class stream {
public:
// -- member types -----------------------------------------------------------
/// Type of a single element.
using value_type = T;
// -- constructors and destructors -------------------------------------------
stream(stream&&) = default;
stream(const stream&) = default;
stream& operator=(stream&&) = default;
stream& operator=(const stream&) = default;
explicit stream(stream_slot id = 0) : slot_(id) {
// nop
}
explicit stream(invalid_stream_t) : slot_(0) {
// nop
}
// -- properties -------------------------------------------------------------
/// Returns the actor-specific stream slot ID.
inline stream_slot slot() const {
return slot_;
}
// -- serialization support --------------------------------------------------
template <class Inspector>
friend typename Inspector::result_type inspect(Inspector& f, stream& x) {
return f(meta::type_name("stream"), x.slot_);
}
private:
// -- member variables -------------------------------------------------------
stream_slot slot_;
};
class stream {};
} // namespace caf
......
......@@ -23,10 +23,17 @@
#include <cstdint>
#include <cstddef>
#include "caf/actor.hpp"
#include "caf/actor_cast.hpp"
#include "caf/downstream_msg.hpp"
#include "caf/fwd.hpp"
#include "caf/mailbox_element.hpp"
#include "caf/make_message.hpp"
#include "caf/message_builder.hpp"
#include "caf/output_stream.hpp"
#include "caf/ref_counted.hpp"
#include "caf/stream_result.hpp"
#include "caf/stream_scatterer.hpp"
#include "caf/stream_slot.hpp"
#include "caf/upstream_msg.hpp"
......@@ -45,25 +52,6 @@ public:
~stream_manager() override;
/// Handles `stream_msg::open` messages by creating a new slot for incoming
/// traffic.
/// @param slot Slot ID used by the sender, i.e., the slot ID for upstream
/// messages back to the sender.
/// @param hdl Handle to the sender.
/// @param original_stage Handle to the initial receiver of the handshake.
/// @param priority Affects credit assignment and maximum bandwidth.
/// @param result_cb Callback for the listener of the final stream result.
/// Ignored when returning `nullptr`, because the previous
/// stage is responsible for it until this manager
/// acknowledges the handshake.
/// @returns An error if the stream manager rejects the handshake.
/// @pre `hdl != nullptr`
/*
virtual error open(stream_slot slot, strong_actor_ptr hdl,
strong_actor_ptr original_stage, stream_priority priority,
response_promise result_cb);
*/
virtual void handle(inbound_path* from, downstream_msg::batch& x);
virtual void handle(inbound_path* from, downstream_msg::close& x);
......@@ -98,14 +86,9 @@ public:
/// Sends a handshake to `dest`.
/// @pre `dest != nullptr`
virtual void send_handshake(strong_actor_ptr dest, stream_slot slot,
strong_actor_ptr stream_origin,
mailbox_element::forwarding_stack fwd_stack,
message_id handshake_mid);
/// Sends a handshake to `dest`.
/// @pre `dest != nullptr`
void send_handshake(strong_actor_ptr dest, stream_slot slot);
virtual void send_handshake(strong_actor_ptr next, stream_slot slot,
mailbox_element::forwarding_stack stages,
message handshake);
// -- implementation hooks for sources ---------------------------------------
......@@ -141,15 +124,6 @@ public:
/// Removes an input path
virtual void remove_input_path(stream_slot slot, error reason, bool silent);
// -- mutators ---------------------------------------------------------------
/// Adds a response promise to a sink for delivering the final result.
/// @pre `out().terminal() == true`
void add_promise(response_promise x);
/// Calls `x.deliver()`
void deliver_promises(message x);
// -- properties -------------------------------------------------------------
/// Returns whether this stream remains open even if no in- or outbound paths
......@@ -175,25 +149,82 @@ public:
return self_;
}
/// Creates an outbound path to the current sender without any type checking.
/// @pre `out().terminal() == false`
/// @private
template <class Out>
output_stream_t<Out> add_unsafe_outbound_path() {
auto handshake = make_message(stream<Out>{});
return {0, add_unsafe_outbound_path_impl(std::move(handshake)), this};
}
/// Creates an outbound path to the current sender without any type checking.
/// @pre `out().terminal() == false`
/// @private
template <class Out, class... Ts>
output_stream<Out, Ts...> add_unsafe_outbound_path();
output_stream_t<Out, detail::strip_and_convert_t<Ts>...>
add_unsafe_outbound_path(std::tuple<Ts...> xs) {
auto tk = std::make_tuple(stream<Out>{});
auto handshake = make_message_from_tuple(std::tuple_cat(tk, std::move(xs)));
return {0, add_unsafe_outbound_path_impl(std::move(handshake)), this};
}
/// Creates an outbound path to `next`, only checking whether the interface
/// of `next` allows handshakes of type `Out`.
/// @pre `next != nullptr`
/// @pre `self()->pending_stream_managers_[slot] == this`
/// @pre `out().terminal() == false`
/// @private
template <class Out, class Handle>
output_stream_t<Out> add_unsafe_outbound_path(Handle next) {
// TODO: type checking
auto handshake = make_message(stream<Out>{});
auto hdl = actor_cast<strong_actor_ptr>(std::move(next));
auto slot = add_unsafe_outbound_path_impl(std::move(hdl),
std::move(handshake));
return {0, slot, this};
}
/// Creates an outbound path to `next` without any type checking.
/// Creates an outbound path to `next`, only checking whether the interface
/// of `next` allows handshakes of type `Out` with arguments `Ts...`.
/// @pre `next != nullptr`
/// @pre `self()->pending_stream_managers_[slot] == this`
/// @pre `out().terminal() == false`
/// @private
void add_unsafe_outbound_path(strong_actor_ptr next, stream_slot slot,
strong_actor_ptr origin,
mailbox_element::forwarding_stack stages,
message_id mid);
template <class Out, class Handle, class... Ts>
output_stream_t<Out, detail::strip_and_convert_t<Ts>...>
add_unsafe_outbound_path(const Handle& next, std::tuple<Ts...> xs) {
// TODO: type checking
auto tk = std::make_tuple(stream<Out>{});
auto handshake = make_message_from_tuple(std::tuple_cat(tk, std::move(xs)));
auto hdl = actor_cast<strong_actor_ptr>(std::move(next));
auto slot = add_unsafe_outbound_path_impl(std::move(hdl),
std::move(handshake));
return {0, slot, this};
}
/// Creates an inbound path to the current sender without any type checking.
template <class Result, class In>
stream_result<Result> add_unsafe_inbound_path(const stream<In>& in);
/// @pre `current_sender() != nullptr`
/// @pre `out().terminal() == false`
/// @private
template <class In>
stream_result<> add_unsafe_inbound_path(const stream<In>&) {
return {add_unsafe_inbound_path_impl(), this};
}
/// Adds `next` as a new outbound path.
/// @private
stream_slot
add_unsafe_outbound_path_impl(strong_actor_ptr next, message handshake,
mailbox_element::forwarding_stack stages = {});
/// Adds `self_->current_sender()` as outbound path.
/// @private
stream_slot add_unsafe_outbound_path_impl(message handshake);
/// Adds
/// @pre Current message is an `open_stream_msg`.
stream_slot add_unsafe_inbound_path_impl();
protected:
// -- modifiers for self -----------------------------------------------------
......@@ -222,10 +253,6 @@ protected:
// -- implementation hooks for sources ---------------------------------------
/// Returns a type-erased `stream<T>` as handshake token for downstream
/// actors. Returns an empty message for sinks.
virtual message make_handshake(stream_slot slot) const;
/// Called whenever new credit becomes available. The default implementation
/// logs an error (sources are expected to override this hook).
virtual void downstream_demand(outbound_path* ptr, long demand);
......@@ -234,6 +261,8 @@ protected:
/// implementation does nothing.
virtual void output_closed(error reason);
// -- member variables -------------------------------------------------------
/// Points to the parent actor.
scheduled_actor* self_;
......@@ -246,13 +275,6 @@ protected:
/// Configures the importance of outgoing traffic.
stream_priority priority_;
/// Stores response promises for dellivering the final result.
std::vector<response_promise> promises_;
/// Stores promises while a handshake is active. The sink at the associated
/// key becomes responsible for the promise after receiving `ack_open`.
std::map<stream_slot, response_promise> in_flight_promises_;
/// Stores whether this stream shall remain open even if no in- or outbound
/// paths exist.
bool continuous_;
......@@ -264,23 +286,4 @@ using stream_manager_ptr = intrusive_ptr<stream_manager>;
} // namespace caf
#include "caf/output_stream.hpp"
#include "caf/stream_result.hpp"
namespace caf {
template <class Out, class... Ts>
output_stream<Out, Ts...>
stream_manager::add_unsafe_outbound_path() {
return {0, this->assign_next_pending_slot(), this};
}
template <class Result, class In>
stream_result<Result>
stream_manager::add_unsafe_inbound_path(const stream<In>&) {
return {this->assign_next_slot(), this};
}
} // namespace caf
#endif // CAF_STREAM_MANAGER_HPP
......@@ -21,15 +21,12 @@
#include "caf/fwd.hpp"
#include "caf/none.hpp"
#include "caf/stream_manager.hpp"
#include "caf/stream_slot.hpp"
#include "caf/meta/type_name.hpp"
namespace caf {
/// Terminates a stream by reducing it to a single value.
template <class T, class Pointer /* = stream_manager_ptr */>
template <class Pointer /* = stream_manager_ptr */>
class stream_result {
public:
// -- member types -----------------------------------------------------------
......@@ -54,14 +51,14 @@ public:
}
template <class CompatiblePointer>
stream_result(stream_result<T, CompatiblePointer> other)
stream_result(stream_result<CompatiblePointer> other)
: in_(other.in()),
ptr_(std::move(other.ptr())) {
// nop
}
template <class CompatiblePointer>
stream_result& operator=(stream_result<T, CompatiblePointer> other) {
stream_result& operator=(stream_result<CompatiblePointer> other) {
in_ = other.in();
ptr_ = std::move(other.ptr());
return *this;
......
......@@ -115,12 +115,12 @@ public:
/// Returns the current number of paths.
virtual size_t num_paths() const noexcept = 0;
/// Adds a path to `target` to the scatterer.
/// Adds a pending path to `target` to the scatterer.
/// @returns The added path on success, `nullptr` otherwise.
virtual path_ptr add_path(stream_slots slots, strong_actor_ptr target) = 0;
path_ptr add_path(stream_slot slot, strong_actor_ptr target);
/// Removes a path from the scatterer and returns it.
virtual bool remove_path(stream_slot slots, error reason,
virtual bool remove_path(stream_slot slot, error reason,
bool silent) noexcept = 0;
/// Returns the path associated to `slots` or `nullptr`.
......@@ -164,16 +164,15 @@ public:
/// Returns the size of the output buffer.
virtual size_t buffered() const noexcept = 0;
/// Returns `make_message(stream<T>{slot})`, where `T` is the value type of
/// this scatterer.
virtual message make_handshake_token(stream_slot slot) const = 0;
/// Silently removes all paths.
virtual void clear_paths() = 0;
protected:
// -- customization points ---------------------------------------------------
/// Inserts `ptr` to the implementation-specific container.
virtual bool insert_path(unique_path_ptr ptr) = 0;
/// Applies `f` to each path.
virtual void for_each_path_impl(path_visitor& f) = 0;
......
......@@ -59,8 +59,6 @@ public:
size_t num_paths() const noexcept override;
path_ptr add_path(stream_slots slots, strong_actor_ptr target) override;
bool remove_path(stream_slot slots, error reason,
bool silent) noexcept override;
......@@ -69,6 +67,8 @@ public:
void clear_paths() override;
protected:
bool insert_path(unique_path_ptr ptr) override;
void for_each_path_impl(path_visitor& f) override;
bool check_paths_impl(path_algorithm algo,
......
......@@ -29,15 +29,13 @@
namespace caf {
template <class In, class Result>
template <class In>
class stream_sink : public virtual stream_manager {
public:
// -- member types -----------------------------------------------------------
using input_type = In;
using result_type = Result;
// -- constructors, destructors, and assignment operators --------------------
stream_sink(scheduled_actor* self) : stream_manager(self) {
......@@ -47,14 +45,14 @@ public:
// -- properties -------------------------------------------------------------
/// Creates a new input path to the current sender.
stream_result<result_type, intrusive_ptr<stream_sink>>
stream_result<intrusive_ptr<stream_sink>>
add_inbound_path(const stream<input_type>&) {
return {this->assign_next_slot(), this};
return {add_unsafe_inbound_path_impl(), this};
}
};
template <class In, class Result>
using stream_sink_ptr = intrusive_ptr<stream_sink<In, Result>>;
template <class In>
using stream_sink_ptr = intrusive_ptr<stream_sink<In>>;
} // namespace caf
......
......@@ -31,17 +31,15 @@
namespace caf {
/// Identifies an unbound sequence of messages.
template <class Input, class Result>
template <class Input>
class stream_sink_driver {
public:
// -- member types -----------------------------------------------------------
using input_type = Input;
using result_type = Result;
/// Implemented `stream_sink` interface.
using sink_type = stream_sink<input_type, result_type>;
using sink_type = stream_sink<input_type>;
/// Smart pointer to the interface type.
using sink_ptr_type = intrusive_ptr<sink_type>;
......@@ -54,11 +52,7 @@ public:
// -- virtual functions ------------------------------------------------------
/// Produces a result message after closing the last inbound path.
virtual message make_final_result() {
return make_message();
}
/// Called after closing the last inbound path.
virtual void finalize(const error&) {
// nop
}
......@@ -66,7 +60,7 @@ public:
// -- pure virtual functions -------------------------------------------------
/// Processes a single batch.
virtual void process(std::vector<input_type>&& batch) = 0;
virtual void process(std::vector<input_type>& batch) = 0;
};
} // namespace caf
......
......@@ -33,7 +33,7 @@ namespace detail {
struct stream_sink_trait_invoke_one {
template <class F, class State, class In>
static void invoke(F& f, State& st, std::vector<In>&& xs) {
static void invoke(F& f, State& st, std::vector<In>& xs) {
for (auto& x : xs)
f(st, std::move(x));
}
......@@ -41,25 +41,8 @@ struct stream_sink_trait_invoke_one {
struct stream_sink_trait_invoke_all {
template <class F, class State, class In>
static void invoke(F& f, State& st, std::vector<In>&& xs) {
f(st, std::move(xs));
}
};
// -- result helper to support void and non-void functions ---------------------
struct stream_sink_trait_default_finalize {
template <class F, class State>
static message invoke(F& f, State& st) {
return make_message(f(st));
}
};
struct stream_sink_trait_void_finalize {
template <class F, class State>
static message invoke(F& f, State& st) {
f(st);
return make_message();
static void invoke(F& f, State& st, std::vector<In>& xs) {
f(st, xs);
}
};
......@@ -68,7 +51,7 @@ struct stream_sink_trait_void_finalize {
// -- trait implementation -----------------------------------------------------
/// Base type for all sink traits.
template <class State, class In, class Result>
template <class State, class In>
struct stream_sink_trait_base {
/// Defines the state element for the function objects.
using state = State;
......@@ -76,57 +59,26 @@ struct stream_sink_trait_base {
/// Defines the type of a single stream element.
using input = In;
/// Defines the result type of the sink.
using result = Result;
/// Defines a pointer to a sink.
using pointer = stream_sink_ptr<input, result>;
using pointer = stream_sink_ptr<input>;
};
/// Defines required type aliases for stream sinks.
template <class Fun, class Fin>
template <class Fun>
struct stream_sink_trait;
/// Specializes the trait for element-wise processing with result.
template <class State, class In, class Out>
struct stream_sink_trait<void(State&, In), Out(State&)>
: stream_sink_trait_base<State, In, Out> {
/// Defines a helper for dispatching to the finalizing function object.
using finalize = detail::stream_sink_trait_default_finalize;
/// Defines a helper for dispatching to the processing function object.
using process = detail::stream_sink_trait_invoke_one;
};
/// Specializes the trait for element-wise processing without result.
/// Specializes the trait for element-wise processing.
template <class State, class In>
struct stream_sink_trait<void(State&, In), void(State&)>
: stream_sink_trait_base<State, In, void> {
/// Defines a helper for dispatching to the finalizing function object.
using finalize = detail::stream_sink_trait_void_finalize;
struct stream_sink_trait<void(State&, In)>
: stream_sink_trait_base<State, In> {
/// Defines a helper for dispatching to the processing function object.
using process = detail::stream_sink_trait_invoke_one;
};
/// Specializes the trait for batch-wise processing with result.
template <class State, class In, class Out>
struct stream_sink_trait<void(State&, std::vector<In>&&), Out(State&)>
: stream_sink_trait_base<State, In, Out> {
/// Defines a helper for dispatching to the finalizing function object.
using finalize = detail::stream_sink_trait_default_finalize;
/// Defines a helper for dispatching to the processing function object.
using process = detail::stream_sink_trait_invoke_all;
};
/// Specializes the trait for batch-wise processing without result.
/// Specializes the trait for batch-wise processing.
template <class State, class In>
struct stream_sink_trait<void(State&, std::vector<In>&&), void(State&)>
: stream_sink_trait_base<State, In, void> {
/// Defines a helper for dispatching to the finalizing function object.
using finalize = detail::stream_sink_trait_void_finalize;
struct stream_sink_trait<void(State&, std::vector<In>&)>
: stream_sink_trait_base<State, In> {
/// Defines a helper for dispatching to the processing function object.
using process = detail::stream_sink_trait_invoke_all;
};
......@@ -134,10 +86,9 @@ struct stream_sink_trait<void(State&, std::vector<In>&&), void(State&)>
// -- convenience alias --------------------------------------------------------
/// Derives a sink trait from the signatures of Fun and Fin.
template <class Fun, class Fin>
template <class Fun>
using stream_sink_trait_t =
stream_sink_trait<typename detail::get_callable_trait<Fun>::fun_sig,
typename detail::get_callable_trait<Fin>::fun_sig>;
stream_sink_trait<typename detail::get_callable_trait<Fun>::fun_sig>;
} // namespace caf
......
......@@ -23,13 +23,14 @@
#include "caf/fwd.hpp"
#include "caf/intrusive_ptr.hpp"
#include "caf/logger.hpp"
#include "caf/stream_manager.hpp"
#include "caf/detail/type_traits.hpp"
namespace caf {
template <class Out, class Scatterer, class... Ts>
template <class Out, class Scatterer>
class stream_source : public virtual stream_manager {
public:
// -- member types -----------------------------------------------------------
......@@ -47,23 +48,49 @@ public:
}
/// Creates a new output path to the current sender.
output_stream<Out, std::tuple<Ts...>, intrusive_ptr<stream_source>>
output_stream<Out, std::tuple<>, intrusive_ptr<stream_source>>
add_outbound_path() {
return {0, this->assign_next_pending_slot(), this};
CAF_LOG_TRACE("");
return this->add_unsafe_outbound_path<Out>().rebind(this);
}
/// Creates a new output path to the current sender with custom handshake.
template <class... Ts>
output_stream<Out, std::tuple<detail::strip_and_convert_t<Ts>...>,
intrusive_ptr<stream_source>>
add_outbound_path(std::tuple<Ts...> xs) {
CAF_LOG_TRACE(CAF_ARG(xs));
return this->add_unsafe_outbound_path<Out>(std::move(xs)).rebind(this);
}
/// Creates a new output path to the current sender.
template <class Handle>
output_stream<Out, std::tuple<>, intrusive_ptr<stream_source>>
add_outbound_path(const Handle& next) {
CAF_LOG_TRACE(CAF_ARG(next));
return this->add_unsafe_outbound_path<Out>(next).rebind(this);
}
/// Creates a new output path to the current sender with custom handshake.
template <class Handle, class... Ts>
output_stream<Out, std::tuple<detail::strip_and_convert_t<Ts>...>,
intrusive_ptr<stream_source>>
add_outbound_path(const Handle& next, std::tuple<Ts...> xs) {
CAF_LOG_TRACE(CAF_ARG(next) << CAF_ARG(xs));
return this->add_unsafe_outbound_path<Out>(next, std::move(xs))
.rebind(this);
}
protected:
Scatterer out_;
};
template <class Out, class Scatterer, class... HandshakeData>
using stream_source_ptr =
intrusive_ptr<stream_source<Out, Scatterer, HandshakeData...>>;
template <class Out, class Scatterer>
using stream_source_ptr = intrusive_ptr<stream_source<Out, Scatterer>>;
template <class Scatterer, class... Ts>
template <class Scatterer>
using stream_source_ptr_t =
stream_source_ptr<typename Scatterer::value_type, Scatterer,
detail::decay_t<Ts>...>;
stream_source_ptr<typename Scatterer::value_type, Scatterer>;
} // namespace caf
......
......@@ -27,7 +27,7 @@
namespace caf {
/// Identifies an unbound sequence of messages.
template <class Scatterer, class... Ts>
template <class Scatterer>
class stream_source_driver {
public:
// -- member types -----------------------------------------------------------
......@@ -42,18 +42,14 @@ public:
using stream_type = stream<output_type>;
/// Tuple for creating the `open_stream_msg` handshake.
using handshake_tuple_type = std::tuple<stream_type, Ts...>;
using handshake_tuple_type = std::tuple<stream_type>;
/// Implemented `stream_source` interface.
using source_type = stream_source<output_type, scatterer_type, Ts...>;
using source_type = stream_source<output_type, scatterer_type>;
/// Smart pointer to the interface type.
using source_ptr_type = intrusive_ptr<source_type>;
/// Type of the output stream including handshake argument types.
using output_stream_type = output_stream<output_type, std::tuple<Ts...>,
source_ptr_type>;
// -- constructors, destructors, and assignment operators --------------------
virtual ~stream_source_driver() {
......@@ -69,9 +65,6 @@ public:
// -- pure virtual functions -------------------------------------------------
/// Generates handshake data for the next actor in the pipeline.
virtual handshake_tuple_type make_handshake(stream_slot slot) const = 0;
/// Generates more stream elements.
virtual void pull(downstream<output_type>& out, size_t num) = 0;
......
......@@ -28,16 +28,15 @@
namespace caf {
template <class In, class Result, class Out, class Scatterer,
class... HandshakeData>
class stream_stage : public stream_source<Out, Scatterer, HandshakeData...>,
public stream_sink<In, Result> {
template <class In, class Out, class Scatterer>
class stream_stage : public stream_source<Out, Scatterer>,
public stream_sink<In> {
public:
// -- member types -----------------------------------------------------------
using left_super = stream_source<Out, Scatterer, HandshakeData...>;
using left_super = stream_source<Out, Scatterer>;
using right_super = stream_sink<In, Result>;
using right_super = stream_sink<In>;
// -- constructors, destructors, and assignment operators --------------------
......@@ -49,10 +48,9 @@ public:
}
};
template <class In, class Result, class Out, class Scatterer,
class... HandshakeData>
template <class In, class Out, class Scatterer>
using stream_stage_ptr =
intrusive_ptr<stream_stage<In, Result, Out, Scatterer, HandshakeData...>>;
intrusive_ptr<stream_stage<In, Out, Scatterer>>;
} // namespace caf
......
......@@ -29,7 +29,7 @@
namespace caf {
/// Encapsulates user-provided functionality for generating a stream stage.
template <class Input, class Result, class Scatterer, class... Ts>
template <class Input, class Scatterer>
class stream_stage_driver {
public:
// -- member types -----------------------------------------------------------
......@@ -37,10 +37,6 @@ public:
/// Element type of the input stream.
using input_type = Input;
/// Result type shipped to the client after processing all elements of the
/// stream.
using result_type = Result;
/// Policy for distributing data to outbound paths.
using scatterer_type = Scatterer;
......@@ -50,21 +46,12 @@ public:
/// Type of the output stream.
using stream_type = stream<output_type>;
/// Tuple for creating the `open_stream_msg` handshake.
using handshake_tuple_type = std::tuple<stream_type, Ts...>;
/// Implemented `stream_stage` interface.
using stage_type = stream_stage<input_type, result_type,
output_type, Scatterer, Ts...>;
using stage_type = stream_stage<input_type, output_type, Scatterer>;
/// Smart pointer to the interface type.
using stage_ptr_type = intrusive_ptr<stage_type>;
/// Type of the output stream including handshake argument types.
using output_stream_type = output_stream<output_type, std::tuple<Ts...>,
stage_ptr_type>;
// -- constructors, destructors, and assignment operators --------------------
virtual ~stream_stage_driver() {
......@@ -73,22 +60,14 @@ public:
// -- virtual functions ------------------------------------------------------
/// Generates handshake data for the next actor in the pipeline.
virtual handshake_tuple_type make_handshake(stream_slot slot) const = 0;
/// Processes a single batch.
virtual void process(std::vector<input_type>&& batch,
downstream<output_type>& out) = 0;
/// Handles the result of an outbound path.
virtual void add_result(message&) = 0;
/// Produces a result message after receiving the result messages of all
/// outbound paths and closing all paths.
virtual message make_final_result() = 0;
virtual void process(downstream<output_type>& out,
std::vector<input_type>& batch) = 0;
/// Cleans up any state.
virtual void finalize(const error&) = 0;
virtual void finalize(const error&) {
// nop
}
};
} // namespace caf
......
......@@ -32,19 +32,19 @@ namespace detail {
// -- invoke helper to support element-wise and batch-wise processing ----------
struct stream_stage_trait_invoke_one {
template <class F, class State, class In, class Out>
static void invoke(F& f, State& st, std::vector<In>&& xs,
downstream<Out>& out) {
template <class F, class State, class Out, class In>
static void invoke(F& f, State& st, downstream<Out>& out,
std::vector<In>& xs) {
for (auto& x : xs)
f(st, out, std::move(x));
}
};
struct stream_stage_trait_invoke_all {
template <class F, class State, class In, class Out>
static void invoke(F& f, State& st, std::vector<In>&& xs,
downstream<Out>& out) {
f(st, out, std::move(xs));
template <class F, class State, class Out, class In>
static void invoke(F& f, State& st, downstream<Out>& out,
std::vector<In>& xs) {
f(st, out, xs);
}
};
......@@ -66,7 +66,7 @@ struct stream_stage_trait<void (State&, downstream<Out>&, In)> {
};
template <class State, class In, class Out>
struct stream_stage_trait<void (State&, downstream<Out>&, std::vector<In>&&)> {
struct stream_stage_trait<void (State&, downstream<Out>&, std::vector<In>&)> {
using state = State;
using input = In;
using output = Out;
......
......@@ -73,10 +73,6 @@ public:
void operator()(const none_t&) override {
(*this)();
}
void operator()(stream_slot, stream_slot, stream_manager_ptr&) override {
(*this)();
}
};
} // namespace <anonymous>
......
......@@ -19,8 +19,10 @@
#include "caf/invalid_stream_scatterer.hpp"
#include "caf/logger.hpp"
#include "caf/message_builder.hpp"
#include "caf/outbound_path.hpp"
#include "caf/stream.hpp"
#include "caf/stream_slot.hpp"
namespace caf {
......@@ -37,11 +39,6 @@ size_t invalid_stream_scatterer::num_paths() const noexcept {
return 0;
}
auto invalid_stream_scatterer::add_path(stream_slots, strong_actor_ptr)
-> path_ptr {
return nullptr;
}
bool invalid_stream_scatterer::remove_path(stream_slot, error, bool) noexcept {
return false;
}
......@@ -66,8 +63,8 @@ size_t invalid_stream_scatterer::buffered() const noexcept {
return 0u;
}
message invalid_stream_scatterer::make_handshake_token(stream_slot slot) const {
return make_message(stream<message>{slot});
bool invalid_stream_scatterer::insert_path(unique_path_ptr) {
return false;
}
void invalid_stream_scatterer::for_each_path_impl(path_visitor&) {
......
......@@ -25,9 +25,10 @@
namespace caf {
outbound_path::outbound_path(stream_slots id, strong_actor_ptr ptr)
: slots(id),
hdl(std::move(ptr)),
outbound_path::outbound_path(stream_slot sender_slot,
strong_actor_ptr receiver_hdl)
: slots(sender_slot, invalid_stream_slot),
hdl(std::move(receiver_hdl)),
next_batch_id(1),
open_credit(0),
desired_batch_size(0),
......
......@@ -400,44 +400,6 @@ void scheduled_actor::quit(error x) {
setf(is_terminated_flag);
}
// -- stream management --------------------------------------------------------
sec scheduled_actor::build_pipeline(stream_slot in, stream_slot out,
stream_manager_ptr mgr) {
CAF_LOG_TRACE(CAF_ARG(in) << CAF_ARG(out) << CAF_ARG(mgr));
auto fail = [&](sec code) {
auto rp = make_response_promise();
rp.deliver(code);
return code;
};
// Check arguments.
if (mgr == nullptr || (in | out) == 0) {
CAF_LOG_ERROR("build_pipeline called with invalid arguments");
return sec::invalid_stream_state;
}
// Get handle to the next stage in the pipeline.
auto next = take_current_next_stage();
if (in != 0) {
CAF_ASSERT(stream_managers_[in] == mgr);
// Sinks must always terminate the stream and store a response promise to
// ship the final result.
if (mgr->out().terminal()) {
CAF_ASSERT(out == 0);
if (next != nullptr)
return fail(sec::cannot_add_downstream);
mgr->add_promise(make_response_promise());
}
}
if (out != 0 && mgr->out().path(out) == nullptr) {
if (next == nullptr)
return fail(sec::no_downstream_stages_defined);
mgr->add_unsafe_outbound_path(std::move(next), out, current_sender(),
take_current_forwarding_stack(),
current_message_id());
}
return sec::none;
}
// -- timeout management -------------------------------------------------------
uint64_t scheduled_actor::set_receive_timeout(actor_clock::time_point x) {
......@@ -970,14 +932,30 @@ void scheduled_actor::erase_stream_manager(stream_slot id) {
stream_ticks_.stop();
}
void scheduled_actor::erase_pending_stream_manager(stream_slot id) {
CAF_LOG_TRACE(CAF_ARG(id));
pending_stream_managers_.erase(id);
}
void scheduled_actor::erase_stream_manager(const stream_manager_ptr& mgr) {
auto i = stream_managers_.begin();
auto e = stream_managers_.end();
while (i != e)
if (i->second == mgr)
i = stream_managers_.erase(i);
else
++i;
{ // Lifetime scope of first iterator pair.
auto i = stream_managers_.begin();
auto e = stream_managers_.end();
while (i != e)
if (i->second == mgr)
i = stream_managers_.erase(i);
else
++i;
}
{ // Lifetime scope of second iterator pair.
auto i = pending_stream_managers_.begin();
auto e = pending_stream_managers_.end();
while (i != e)
if (i->second == mgr)
i = pending_stream_managers_.erase(i);
else
++i;
}
if (stream_managers_.empty())
stream_ticks_.stop();
}
......@@ -987,9 +965,6 @@ scheduled_actor::handle_open_stream_msg(mailbox_element& x) {
CAF_LOG_TRACE(CAF_ARG(x));
// Fetches a stream manger from a behavior.
struct visitor : detail::invoke_result_visitor {
stream_manager_ptr ptr;
stream_slot in_slot;
stream_slot out_slot;
void operator()() override {
// nop
}
......@@ -1002,13 +977,6 @@ scheduled_actor::handle_open_stream_msg(mailbox_element& x) {
// nop
}
void operator()(stream_slot in, stream_slot out,
stream_manager_ptr& x) override {
ptr = std::move(x);
in_slot = in;
out_slot = out;
}
void operator()(const none_t&) override {
// nop
}
......@@ -1048,6 +1016,7 @@ scheduled_actor::handle_open_stream_msg(mailbox_element& x) {
CAF_LOG_DEBUG("no match in behavior, fall back to default handler");
return fallback();
case match_case::result::match: {
/*
if (f.ptr == nullptr) {
CAF_LOG_WARNING("actor did not return a stream manager after "
"handling open_stream_msg");
......@@ -1060,6 +1029,7 @@ scheduled_actor::handle_open_stream_msg(mailbox_element& x) {
path->emit_ack_open(this, actor_cast<actor_addr>(osm.original_stage));
// Propagate handshake down the pipeline.
build_pipeline(f.in_slot, f.out_slot, std::move(f.ptr));
*/
return im_success;
}
default:
......
......@@ -60,14 +60,25 @@ void stream_manager::handle(inbound_path*, downstream_msg::forced_close& x) {
bool stream_manager::handle(stream_slots slots, upstream_msg::ack_open& x) {
CAF_LOG_TRACE(CAF_ARG(slots) << CAF_ARG(x));
auto path = out().add_path(slots.invert(), x.rebind_to);
if (path == nullptr)
auto ptr = out().path(slots.receiver);
if (ptr == nullptr)
return false;
path->open_credit = x.initial_demand;
path->desired_batch_size = x.desired_batch_size;
if (!ptr->pending()) {
CAF_LOG_ERROR("received repeated ack_open");
return false;
}
if (ptr->hdl != x.rebind_from) {
CAF_LOG_ERROR("received ack_open with invalid rebind_from");
return false;
}
if (x.rebind_from != x.rebind_to) {
ptr->hdl = x.rebind_to;
}
ptr->slots.receiver = slots.sender;
ptr->open_credit = x.initial_demand;
ptr->desired_batch_size = x.desired_batch_size;
--pending_handshakes_;
push();
in_flight_promises_.erase(slots.sender);
return true;
}
......@@ -98,19 +109,10 @@ void stream_manager::stop() {
error tmp;
finalize(tmp);
self_->erase_inbound_paths_later(this);
if (!promises_.empty())
deliver_promises(make_final_result());
}
void stream_manager::abort(error reason) {
CAF_LOG_TRACE(CAF_ARG(reason));
if (!promises_.empty() || !in_flight_promises_.empty()) {
auto msg = make_message(reason);
deliver_promises(msg);
for (auto& kvp : in_flight_promises_)
kvp.second.deliver(msg);
in_flight_promises_.clear();
}
out().abort(reason);
finalize(reason);
self_->erase_inbound_paths_later(this, std::move(reason));
......@@ -127,27 +129,19 @@ bool stream_manager::congested() const noexcept {
return false;
}
void stream_manager::send_handshake(strong_actor_ptr dest, stream_slot slot,
strong_actor_ptr client,
mailbox_element::forwarding_stack fwd_stack,
message_id mid) {
CAF_ASSERT(dest != nullptr);
void stream_manager::send_handshake(strong_actor_ptr next, stream_slot slot,
mailbox_element::forwarding_stack stages,
message handshake) {
CAF_ASSERT(next != nullptr);
CAF_ASSERT(slot != invalid_stream_slot);
++pending_handshakes_;
in_flight_promises_.emplace(
slot, response_promise{self()->ctrl(), client, fwd_stack, mid});
dest->enqueue(
make_mailbox_element(std::move(client), mid, std::move(fwd_stack),
open_stream_msg{slot, make_handshake(slot),
self_->ctrl(), dest, priority_}),
next->enqueue(
make_mailbox_element(self_->ctrl(), make_message_id(), std::move(stages),
open_stream_msg{slot, std::move(handshake),
self_->ctrl(), next, priority_}),
self_->context());
}
void stream_manager::send_handshake(strong_actor_ptr dest, stream_slot slot) {
mailbox_element::forwarding_stack fwd_stack;
send_handshake(std::move(dest), slot, nullptr, std::move(fwd_stack),
make_message_id());
}
bool stream_manager::generate_messages() {
return false;
}
......@@ -184,28 +178,67 @@ void stream_manager::remove_input_path(stream_slot slot, error reason,
self_->erase_inbound_path_later(slot, std::move(reason));
}
void stream_manager::add_promise(response_promise x) {
CAF_LOG_TRACE(CAF_ARG(x));
CAF_ASSERT(out().terminal());
promises_.emplace_back(std::move(x));
stream_slot stream_manager::add_unsafe_outbound_path_impl(
strong_actor_ptr next, message handshake,
mailbox_element::forwarding_stack stages) {
CAF_LOG_TRACE(CAF_ARG(next) << CAF_ARG(handshake) << CAF_ARG(stages));
CAF_ASSERT(out().terminal() == false);
if (next == nullptr) {
CAF_LOG_WARNING("add_outbound_path called with next == nullptr");
auto rp = self_->make_response_promise();
rp.deliver(sec::no_downstream_stages_defined);
return invalid_stream_slot;
}
auto slot = self_->assign_next_pending_slot_to(this);
auto path = out().add_path(slot, next);
CAF_IGNORE_UNUSED(path);
CAF_ASSERT(path != nullptr);
// Build pipeline by forwarding handshake along the path.
send_handshake(std::move(next), slot, std::move(stages),
std::move(handshake));
generate_messages();
return slot;
}
void stream_manager::deliver_promises(message x) {
CAF_LOG_TRACE(CAF_ARG(x));
for (auto& p : promises_)
p.deliver(x);
promises_.clear();
stream_slot stream_manager::add_unsafe_outbound_path_impl(message handshake) {
CAF_LOG_TRACE(CAF_ARG(handshake));
// Call take_current_next_stage explicitly before calling
// take_current_forwarding_stack to avoid UB due to undefined evaluation
// order.
auto next = self_->take_current_next_stage();
auto stages = self_->take_current_forwarding_stack();
// Sources simply forward the original request.
return add_unsafe_outbound_path_impl(std::move(next), std::move(handshake),
std::move(stages));
}
void stream_manager::add_unsafe_outbound_path(
strong_actor_ptr next, stream_slot slot, strong_actor_ptr origin,
mailbox_element::forwarding_stack stages, message_id mid) {
CAF_ASSERT(next != nullptr);
CAF_ASSERT(out().terminal() == false);
// Build pipeline by forwarding handshake along the path.
send_handshake(std::move(next), slot, std::move(origin),
std::move(stages), mid);
generate_messages();
stream_slot stream_manager::add_unsafe_inbound_path_impl() {
CAF_LOG_TRACE("");
auto x = self_->current_mailbox_element();
if (x == nullptr || !x->content().match_elements<open_stream_msg>()) {
CAF_LOG_ERROR("add_unsafe_inbound_path called, but current message "
"is not an open_stream_msg");
return invalid_stream_slot;
}
auto& osm = x->content().get_mutable_as<open_stream_msg>(0);
auto fail = [&](sec code) {
stream_slots path_id{osm.slot, 0};
inbound_path::emit_irregular_shutdown(self_, path_id,
std::move(osm.prev_stage), code);
return invalid_stream_slot;
};
if (out().terminal() && self_->current_next_stage() != nullptr) {
// Sinks must always terminate the stream.
CAF_LOG_WARNING("add_unsafe_inbound_path called in a sink, but the "
"handshake has further stages");
return fail(sec::cannot_add_downstream);
}
auto slot = assign_next_slot();
stream_slots path_id{osm.slot, slot};
auto ptr = self_->make_inbound_path(this, path_id, std::move(osm.prev_stage));
CAF_ASSERT(ptr != nullptr);
ptr->emit_ack_open(self_, actor_cast<actor_addr>(osm.original_stage));
return slot;
}
stream_slot stream_manager::assign_next_slot() {
......@@ -233,11 +266,6 @@ void stream_manager::output_closed(error) {
// nop
}
message stream_manager::make_handshake(stream_slot) const {
CAF_LOG_ERROR("stream_manager::make_handshake called");
return none;
}
void stream_manager::downstream_demand(outbound_path*, long) {
CAF_LOG_ERROR("stream_manager::downstream_demand called");
}
......
......@@ -96,8 +96,17 @@ bool stream_scatterer::terminal() const noexcept {
return false;
}
stream_scatterer::path_ptr stream_scatterer::add_path(stream_slot slot,
strong_actor_ptr target) {
CAF_LOG_TRACE(CAF_ARG(slot) << CAF_ARG(target));
unique_path_ptr ptr{new outbound_path(slot, std::move(target))};
auto result = ptr.get();
return insert_path(std::move(ptr)) ? result : nullptr;
}
void stream_scatterer::about_to_erase(outbound_path* ptr, bool silent,
error* reason) {
CAF_LOG_TRACE(CAF_ARG(ptr) << CAF_ARG(silent) << CAF_ARG(reason));
if (!silent) {
if (reason == nullptr)
ptr->emit_regular_shutdown(self_);
......
......@@ -34,24 +34,13 @@ stream_scatterer_impl::~stream_scatterer_impl() {
// nop
}
outbound_path* stream_scatterer_impl::add_path(stream_slots slots,
strong_actor_ptr target) {
CAF_LOG_TRACE(CAF_ARG(slots) << CAF_ARG(target));
auto res = paths_.emplace(slots.sender, nullptr);
if (res.second) {
auto ptr = new outbound_path(slots, std::move(target));
res.first->second.reset(ptr);
return ptr;
}
return nullptr;
}
size_t stream_scatterer_impl::num_paths() const noexcept {
return paths_.size();
}
bool stream_scatterer_impl::remove_path(stream_slot slot, error reason,
bool silent) noexcept {
CAF_LOG_TRACE(CAF_ARG(slot) << CAF_ARG(reason) << CAF_ARG(silent));
auto i = paths_.find(slot);
if (i != paths_.end()) {
about_to_erase(i->second.get(), silent, &reason);
......@@ -70,6 +59,14 @@ void stream_scatterer_impl::clear_paths() {
paths_.clear();
}
bool stream_scatterer_impl::insert_path(unique_path_ptr ptr) {
CAF_LOG_TRACE(CAF_ARG(ptr));
CAF_ASSERT(ptr != nullptr);
auto slot = ptr->slots.sender;
CAF_ASSERT(slot != invalid_stream_slot);
return paths_.emplace(slot, std::move(ptr)).second;
}
void stream_scatterer_impl::for_each_path_impl(path_visitor& f) {
for (auto& kvp : paths_)
f(*kvp.second);
......
......@@ -86,9 +86,10 @@ public:
}
void add_path_to(entity& x, int desired_batch_size) {
auto ptr = bs.add_path({next_slot++, 0}, x.ctrl());
auto ptr = bs.add_path(next_slot++, x.ctrl());
CAF_REQUIRE(ptr != nullptr);
ptr->desired_batch_size = desired_batch_size;
ptr->slots.receiver = x.next_slot++;
paths.emplace_back(ptr);
}
......
......@@ -78,21 +78,21 @@ TESTEE(sum_up) {
return {
[=](stream<int>& in, const string& fname) {
CAF_CHECK_EQUAL(fname, "numbers.txt");
using int_ptr = int*;
return self->make_sink(
// input stream
in,
// initialize state
[](unit_t&) {
// nop
[=](int_ptr& x) {
x = &self->state.x;
},
// processing step
[=](unit_t&, int y) {
self->state.x += y;
[](int_ptr& x , int y) {
*x += y;
},
// cleanup and produce result message
[=](unit_t&) -> int {
// cleanup
[=](int_ptr&, const error&) {
CAF_MESSAGE(self->name() << " is done");
return self->state.x;
}
);
},
......@@ -104,13 +104,11 @@ TESTEE(sum_up) {
}
TESTEE_STATE(stream_multiplexer) {
stream_stage_ptr<int, message, int, broadcast_scatterer<int>, string> stage;
stream_stage_ptr<int, int, broadcast_scatterer<int>> stage;
};
TESTEE(stream_multiplexer) {
self->state.stage = self->make_continuous_stage(
// handshake data
std::make_tuple(std::string{"numbers.txt"}),
// initialize state
[](unit_t&) {
// nop
......@@ -120,14 +118,15 @@ TESTEE(stream_multiplexer) {
out.push(x);
},
// cleanup
[=](unit_t&) {
[=](unit_t&, const error&) {
CAF_MESSAGE(self->name() << " is done");
}
);
return {
[=](join_atom) {
CAF_MESSAGE("received 'join' request");
return self->state.stage->add_outbound_path();
return self->state.stage->add_outbound_path(
std::make_tuple("numbers.txt"));
},
[=](const stream<int>& in, std::string& fname) {
CAF_CHECK_EQUAL(fname, "numbers.txt");
......
......@@ -105,21 +105,22 @@ TESTEE_STATE(sum_up) {
};
TESTEE(sum_up) {
using intptr = int*;
return {
[=](stream<int>& in) {
return self->make_sink(
// input stream
in,
// initialize state
[](unit_t&) {
// nop
[=](intptr& x) {
x = &self->state.x;
},
// processing step
[=](unit_t&, int y) {
self->state.x += y;
[](intptr& x, int y) {
*x += y;
},
// cleanup and produce result message
[=](unit_t&) {
[=](intptr&, const error&) {
CAF_MESSAGE(self->name() << " is done");
}
);
......@@ -150,7 +151,7 @@ TESTEE(collect) {
self->state.strings.emplace_back(std::move(y));
},
// cleanup and produce result message
[=](unit_t&) {
[=](unit_t&, const error&) {
CAF_MESSAGE(self->name() << " is done");
}
);
......@@ -170,6 +171,8 @@ using scatterer = fused_scatterer<int_scatterer, string_scatterer>;
class fused_stage : public stream_manager {
public:
using super = stream_manager;
fused_stage(scheduled_actor* self) : stream_manager(self), out_(self) {
continuous(true);
}
......@@ -184,12 +187,14 @@ public:
using int_vec = std::vector<int>;
using string_vec = std::vector<string>;
if (batch.xs.match_elements<int_vec>()) {
CAF_MESSAGE("handle an integer batch");
auto& xs = batch.xs.get_mutable_as<int_vec>(0);
auto& buf = out_.get<int_scatterer>().buf();
buf.insert(buf.end(), xs.begin(), xs.end());
return;
}
if (batch.xs.match_elements<string_vec>()) {
CAF_MESSAGE("handle a string batch");
auto& xs = batch.xs.get_mutable_as<string_vec>(0);
auto& buf = out_.get<string_scatterer>().buf();
buf.insert(buf.end(), xs.begin(), xs.end());
......@@ -198,10 +203,6 @@ public:
CAF_LOG_ERROR("received unexpected batch type (dropped)");
}
message make_handshake(stream_slot slot) const override {
return out_.make_handshake_token(slot);
}
bool congested() const noexcept override {
return out_.capacity() == 0;
}
......@@ -212,7 +213,6 @@ public:
private:
scatterer out_;
std::map<stream_slot, stream_scatterer*> scatterers_;
};
TESTEE_STATE(stream_multiplexer) {
......@@ -231,18 +231,18 @@ TESTEE(stream_multiplexer) {
},
[=](join_atom, strings_atom) {
auto& stg = self->state.stage;
CAF_MESSAGE("received 'join' request for integers");
CAF_MESSAGE("received 'join' request for strings");
auto result = stg->add_unsafe_outbound_path<string>();
stg->out().assign<string_scatterer>(result.out());
return result;
},
[=](const stream<int>& in) {
CAF_MESSAGE("received handshake for integers");
return self->state.stage->add_unsafe_inbound_path<void>(in);
return self->state.stage->add_unsafe_inbound_path(in);
},
[=](const stream<string>& in) {
CAF_MESSAGE("received handshake for strings");
return self->state.stage->add_unsafe_inbound_path<void>(in);
return self->state.stage->add_unsafe_inbound_path(in);
}
};
}
......
This diff is collapsed.
......@@ -100,16 +100,20 @@ TESTEE(log_producer) {
}
return false;
},
unit,
policy::arg<scatterer>::value
);
res.ptr()->out().set_filter(res.out(), lvl);
auto& out = res.ptr()->out();
static_assert(std::is_same<decltype(out), scatterer&>::value,
"source has wrong scatterer type");
out.set_filter(res.out(), lvl);
return res;
}
};
}
TESTEE_STATE(log_dispatcher) {
stream_stage_ptr<value_type, message, value_type, scatterer> stage;
stream_stage_ptr<value_type, value_type, scatterer> stage;
};
TESTEE(log_dispatcher) {
......@@ -123,7 +127,7 @@ TESTEE(log_dispatcher) {
out.push(std::move(x));
},
// cleanup
[=](unit_t&) {
[=](unit_t&, const error&) {
CAF_MESSAGE(self->name() << " is done");
},
policy::arg<scatterer>::value
......@@ -161,7 +165,7 @@ TESTEE(log_consumer) {
self->state.log.emplace_back(std::move(x));
},
// cleanup and produce result message
[=](unit_t&) {
[=](unit_t&, const error&) {
CAF_MESSAGE(self->name() << " is done");
}
);
......
Markdown is supported
0%
or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment