Commit 414ed837 authored by Dominik Charousset's avatar Dominik Charousset

Allow data in stream handshakes, remove filters

parent ab19c1c0
...@@ -37,9 +37,6 @@ class abstract_downstream { ...@@ -37,9 +37,6 @@ class abstract_downstream {
public: public:
using topics = std::vector<atom_value>; using topics = std::vector<atom_value>;
/// A set of downstream-defined topic subscriptions.
using topics_set = std::set<topics>;
using path = downstream_path; using path = downstream_path;
using path_cref = const path&; using path_cref = const path&;
...@@ -86,8 +83,7 @@ public: ...@@ -86,8 +83,7 @@ public:
/// Returns the size of the output buffer. /// Returns the size of the output buffer.
virtual size_t buf_size() const = 0; virtual size_t buf_size() const = 0;
bool add_path(strong_actor_ptr ptr, std::vector<atom_value> filter, bool add_path(strong_actor_ptr ptr, bool redeployable);
bool redeployable);
bool remove_path(strong_actor_ptr& ptr); bool remove_path(strong_actor_ptr& ptr);
...@@ -116,8 +112,6 @@ public: ...@@ -116,8 +112,6 @@ public:
} }
protected: protected:
void recalculate_active_filters();
void send_batch(downstream_path& dest, size_t chunk_size, message chunk); void send_batch(downstream_path& dest, size_t chunk_size, message chunk);
/// Sorts `paths_` in descending order by available credit. /// Sorts `paths_` in descending order by available credit.
...@@ -137,7 +131,6 @@ protected: ...@@ -137,7 +131,6 @@ protected:
stream_id sid_; stream_id sid_;
path_list paths_; path_list paths_;
std::unique_ptr<downstream_policy> policy_; std::unique_ptr<downstream_policy> policy_;
topics_set active_filters_;
}; };
} // namespace caf } // namespace caf
......
...@@ -32,17 +32,9 @@ namespace caf { ...@@ -32,17 +32,9 @@ namespace caf {
template <class T> template <class T>
class downstream final : public abstract_downstream { class downstream final : public abstract_downstream {
public: public:
/// Topic filters defined by a downstream actor.
using topics = abstract_downstream::topics;
using topics_set = abstract_downstream::topics_set;
/// A chunk of data for sending batches downstream. /// A chunk of data for sending batches downstream.
using chunk = std::vector<T>; using chunk = std::vector<T>;
/// Chunks split into multiple categories via filters.
using categorized_chunks = std::unordered_map<topics, chunk>;
downstream(local_actor* ptr, const stream_id& sid, downstream(local_actor* ptr, const stream_id& sid,
typename abstract_downstream::policy_ptr policy) typename abstract_downstream::policy_ptr policy)
: abstract_downstream(ptr, sid, std::move(policy)) { : abstract_downstream(ptr, sid, std::move(policy)) {
...@@ -139,101 +131,6 @@ private: ...@@ -139,101 +131,6 @@ private:
return xs; return xs;
} }
/// @pre `n <= buf_.size()`
chunk get_chunk(size_t n, const topics& filter) {
if (filter.empty())
return get_chunk(n);
chunk xs;
if (n > 0) {
xs.reserve(n);
auto in_filter = [&](atom_value x) {
auto e = filter.end();
return std::find(filter.begin(), e, x) != e;
};
// temporarily stores a T with the result of in_filter()
scratch_space ys;
move_from_buf_until_nth_match(ys, n, in_filter);
auto is_marked = [](const scratch_space_value& x) {
return std::get<1>(x);
};
// helper for iterating over ys as if using a move iterator
// over vector<T>
// parition range into result messages and message to move back into buf_
scratch_space_move_iter first{ys.begin()};
scratch_space_move_iter last{ys.end()};
scratch_space_move_iter pp{std::stable_partition(first.i, last.i,
is_marked)};
xs.insert(xs.end(), first, pp);
buf_.insert(buf_.begin(), pp, last);
}
return std::move(xs);
}
categorized_chunks get_chunks(size_t n, const topics_set& filters) {
categorized_chunks res;
if (filters.empty()) {
res.emplace(topics{}, get_chunk(n));
} else if (filters.size() == 1) {
res.emplace(*filters.begin(), get_chunk(n, *filters.begin()));
} else {
// reserve sufficient space for chunks
for (auto& filter : filters) {
chunk tmp;
tmp.reserve(n);
res.emplace(filter, std::move(tmp));
}
// get up to n elements from buffer
auto in_filter = [](const topics_set& filter, atom_value x) {
auto e = filter.end();
return filter.empty() || std::find(filter.begin(), e, x) != e;
};
auto in_any_filter = [&](atom_value x) {
for (auto& filter : filters)
if (in_filter(filter, x))
return true;
return false;
};
scratch_space ys;
move_from_buf_until_nth_match(ys, n, in_any_filter);
// parition range into result messages and message to move back into buf_
auto is_marked = [](const scratch_space_value& x) {
return std::get<1>(x);
};
auto first = ys.begin();
auto last = ys.end();
auto pp = std::stable_partition(first, last, is_marked);
// copy matched values into according chunks
for (auto& r : res)
for (auto& y : ys)
if (in_filter(r.first, std::get<2>(y)))
r.second.emplace_back(std::get<0>(y));
// move unused items back into place
buf_.insert(buf_.begin(), scratch_space_move_iter{pp},
scratch_space_move_iter{last});
}
return res;
}
// moves elements from `buf_` to `tmp` until n elements matching the
// predicate were moved or the buffer was fully consumed.
// @pre `n > 0`
template <class F>
void move_from_buf_until_nth_match(scratch_space& ys, size_t n, F pred) {
CAF_ASSERT(n > 0);
size_t included = 0; // nr. of included elements
for (auto i = buf_.begin(); i != buf_.end(); ++i) {
auto topic = this->policy().categorize(*i);
auto match = pred(topic);
ys.emplace_back(std::move(*i), match, topic);
if (match && ++included == n) {
buf_.erase(buf_.begin(), i + 1);
return;
}
}
// consumed whole buffer
buf_.clear();
}
std::deque<T> buf_; std::deque<T> buf_;
}; };
......
...@@ -34,8 +34,6 @@ namespace caf { ...@@ -34,8 +34,6 @@ namespace caf {
/// the stream ID registered with the hosting downstream object. /// the stream ID registered with the hosting downstream object.
class downstream_path { class downstream_path {
public: public:
using topics = std::vector<atom_value>;
/// Handle to the downstream actor. /// Handle to the downstream actor.
strong_actor_ptr ptr; strong_actor_ptr ptr;
...@@ -45,9 +43,6 @@ public: ...@@ -45,9 +43,6 @@ public:
/// Currently available credit for this path. /// Currently available credit for this path.
size_t open_credit; size_t open_credit;
/// Subscribed topics on this path (empty for all).
topics filter;
/// Stores whether the downstream actor is failsafe, i.e., allows the runtime /// Stores whether the downstream actor is failsafe, i.e., allows the runtime
/// to redeploy it on failure. If this field is set to `false` then /// to redeploy it on failure. If this field is set to `false` then
/// `unacknowledged_batches` is unused. /// `unacknowledged_batches` is unused.
...@@ -56,7 +51,7 @@ public: ...@@ -56,7 +51,7 @@ public:
/// Caches batches until receiving an ACK. /// Caches batches until receiving an ACK.
std::deque<std::pair<int64_t, stream_msg::batch>> unacknowledged_batches; std::deque<std::pair<int64_t, stream_msg::batch>> unacknowledged_batches;
downstream_path(strong_actor_ptr p, topics ts, bool redeploy); downstream_path(strong_actor_ptr p, bool redeploy);
}; };
} // namespace caf } // namespace caf
......
...@@ -112,6 +112,13 @@ struct message_factory { ...@@ -112,6 +112,13 @@ struct message_factory {
} }
}; };
/// Converts the tuple `xs` to a message.
template <class... Ts>
message make_message_from_tuple(std::tuple<Ts...> xs) {
message_factory f;
return detail::apply_moved_args(f, detail::get_indices(xs), xs);
}
} // namespace caf } // namespace caf
#endif // CAF_MAKE_MESSAGE_HPP #endif // CAF_MAKE_MESSAGE_HPP
...@@ -32,13 +32,11 @@ namespace mixin { ...@@ -32,13 +32,11 @@ namespace mixin {
template <class Base, class Subtype> template <class Base, class Subtype>
class has_downstreams : public Base { class has_downstreams : public Base {
public: public:
using topics = typename Base::topics; error add_downstream(strong_actor_ptr& ptr, size_t init,
bool is_redeployable) final {
error add_downstream(strong_actor_ptr& ptr, const topics& filter,
size_t init, bool is_redeployable) final {
if (!ptr) if (!ptr)
return sec::invalid_argument; return sec::invalid_argument;
if (out().add_path(ptr, filter, is_redeployable)) { if (out().add_path(ptr, is_redeployable)) {
dptr()->downstream_demand(ptr, init); dptr()->downstream_demand(ptr, init);
return none; return none;
} }
......
...@@ -299,11 +299,11 @@ public: ...@@ -299,11 +299,11 @@ public:
// -- stream management ------------------------------------------------------ // -- stream management ------------------------------------------------------
// Starts a new stream. // Starts a new stream.
template <class Handle, class Init, class Getter, class ClosedPredicate, template <class Handle, class... Ts, class Init, class Getter,
class ResHandler> class ClosedPredicate, class ResHandler>
stream<typename stream_source_trait_t<Getter>::output> stream<typename stream_source_trait_t<Getter>::output>
new_stream(const Handle& dest, Init init, Getter getter, ClosedPredicate pred, new_stream(const Handle& dest, std::tuple<Ts...> xs, Init init, Getter getter,
ResHandler res_handler) { ClosedPredicate pred, ResHandler res_handler) {
using type = typename stream_source_trait_t<Getter>::output; using type = typename stream_source_trait_t<Getter>::output;
using state_type = typename stream_source_trait_t<Getter>::state; using state_type = typename stream_source_trait_t<Getter>::state;
static_assert(std::is_same< static_assert(std::is_same<
...@@ -325,13 +325,14 @@ public: ...@@ -325,13 +325,14 @@ public:
stream_id sid{ctrl(), stream_id sid{ctrl(),
new_request_id(message_priority::normal).integer_value()}; new_request_id(message_priority::normal).integer_value()};
stream<type> token{sid}; stream<type> token{sid};
auto ys = std::tuple_cat(std::forward_as_tuple(token), std::move(xs));;
// generate new ID for the final response message and send handshake // generate new ID for the final response message and send handshake
auto res_id = new_request_id(message_priority::normal); auto res_id = new_request_id(message_priority::normal);
dest->enqueue(make_mailbox_element( dest->enqueue(
make_mailbox_element(
ctrl(), res_id, {}, ctrl(), res_id, {},
make<stream_msg::open>(sid, make_message(std::move(token)), make<stream_msg::open>(sid, make_message_from_tuple(std::move(ys)),
ctrl(), stream_priority::normal, ctrl(), stream_priority::normal, false)),
std::vector<atom_value>{}, false)),
context()); context());
// install response handler // install response handler
this->add_multiplexed_response_handler( this->add_multiplexed_response_handler(
...@@ -347,10 +348,22 @@ public: ...@@ -347,10 +348,22 @@ public:
return sid; return sid;
} }
// Starts a new stream.
template <class Handle, class Init, class Getter,
class ClosedPredicate, class ResHandler>
stream<typename stream_source_trait_t<Getter>::output>
new_stream(const Handle& dest, Init init, Getter getter,
ClosedPredicate pred, ResHandler res_handler) {
return new_stream(dest, std::make_tuple(), std::move(init),
std::move(getter), std::move(pred),
std::move(res_handler));
}
/// Adds a stream source to this actor. /// Adds a stream source to this actor.
template <class Init, class Getter, class ClosedPredicate> template <class Init, class... Ts, class Getter, class ClosedPredicate>
stream<typename stream_source_trait_t<Getter>::output> stream<typename stream_source_trait_t<Getter>::output>
add_source(Init init, Getter getter, ClosedPredicate pred) { add_source(std::tuple<Ts...> xs, Init init,
Getter getter, ClosedPredicate pred) {
CAF_ASSERT(current_mailbox_element() != nullptr); CAF_ASSERT(current_mailbox_element() != nullptr);
using type = typename stream_source_trait_t<Getter>::output; using type = typename stream_source_trait_t<Getter>::output;
using state_type = typename stream_source_trait_t<Getter>::state; using state_type = typename stream_source_trait_t<Getter>::state;
...@@ -373,7 +386,7 @@ public: ...@@ -373,7 +386,7 @@ public:
} }
stream_id sid{ctrl(), stream_id sid{ctrl(),
new_request_id(message_priority::normal).integer_value()}; new_request_id(message_priority::normal).integer_value()};
fwd_stream_handshake<type>(sid); fwd_stream_handshake<type>(sid, xs);
using impl = stream_source_impl<Getter, ClosedPredicate>; using impl = stream_source_impl<Getter, ClosedPredicate>;
std::unique_ptr<downstream_policy> p{new policy::anycast}; std::unique_ptr<downstream_policy> p{new policy::anycast};
auto ptr = make_counted<impl>(this, sid, std::move(p), std::move(getter), auto ptr = make_counted<impl>(this, sid, std::move(p), std::move(getter),
...@@ -383,10 +396,18 @@ public: ...@@ -383,10 +396,18 @@ public:
return sid; return sid;
} }
template <class Init, class Getter, class ClosedPredicate>
stream<typename stream_source_trait_t<Getter>::output>
add_source(Init init, Getter getter, ClosedPredicate pred) {
return add_source(std::make_tuple(), std::move(init),
std::move(getter), std::move(pred));
}
/// Adds a stream stage to this actor. /// Adds a stream stage to this actor.
template <class In, class Init, class Fun, class Cleanup> template <class In, class... Ts, class Init, class Fun, class Cleanup>
stream<typename stream_stage_trait_t<Fun>::output> stream<typename stream_stage_trait_t<Fun>::output>
add_stage(stream<In>& in, Init init, Fun fun, Cleanup cleanup) { add_stage(stream<In>& in, std::tuple<Ts...> xs,
Init init, Fun fun, Cleanup cleanup) {
CAF_ASSERT(current_mailbox_element() != nullptr); CAF_ASSERT(current_mailbox_element() != nullptr);
using output_type = typename stream_stage_trait_t<Fun>::output; using output_type = typename stream_stage_trait_t<Fun>::output;
using state_type = typename stream_stage_trait_t<Fun>::state; using state_type = typename stream_stage_trait_t<Fun>::state;
...@@ -400,7 +421,7 @@ public: ...@@ -400,7 +421,7 @@ public:
return stream_id{nullptr, 0}; return stream_id{nullptr, 0};
} }
auto sid = in.id(); auto sid = in.id();
fwd_stream_handshake<output_type>(in.id()); fwd_stream_handshake<output_type>(sid, xs);
using impl = stream_stage_impl<Fun, Cleanup>; using impl = stream_stage_impl<Fun, Cleanup>;
std::unique_ptr<downstream_policy> dptr{new policy::anycast}; std::unique_ptr<downstream_policy> dptr{new policy::anycast};
std::unique_ptr<upstream_policy> uptr{new policy::greedy}; std::unique_ptr<upstream_policy> uptr{new policy::greedy};
...@@ -411,6 +432,27 @@ public: ...@@ -411,6 +432,27 @@ public:
return std::move(sid); return std::move(sid);
} }
/// Adds a stream stage to this actor.
template <class In, class Init, class Fun, class Cleanup>
stream<typename stream_stage_trait_t<Fun>::output>
add_stage(stream<In>& in, Init init, Fun fun, Cleanup cleanup) {
return add_stage(in, std::make_tuple(), std::move(init),
std::move(fun), std::move(cleanup));
}
/// Adds `hdl` to an already existing stream as a new downstream path. Note
/// that the new downstream path will *not* receive previous (historic) data.
template <class T, class Handle>
expected<void> add_downstream_path(stream<T>& in, Handle hdl) {
auto i = streams_.find(in.id());
if (i != streams_.end()) {
auto ptr = actor_cast<strong_actor_ptr>(std::move(hdl));
i->second->add_downstream(ptr, 5, false);
return unit;
}
return sec::cannot_add_downstream;
}
/// Adds a stream sink to this actor. /// Adds a stream sink to this actor.
template <class In, class Init, class Fun, class Finalize> template <class In, class Init, class Fun, class Finalize>
result<typename stream_sink_trait_t<Fun, Finalize>::output> result<typename stream_sink_trait_t<Fun, Finalize>::output>
...@@ -562,8 +604,8 @@ protected: ...@@ -562,8 +604,8 @@ protected:
swap(g, f); swap(g, f);
} }
template <class T> template <class T, class... Ts>
void fwd_stream_handshake(const stream_id& sid) { void fwd_stream_handshake(const stream_id& sid, std::tuple<Ts...>& xs) {
auto mptr = current_mailbox_element(); auto mptr = current_mailbox_element();
auto& stages = mptr->stages; auto& stages = mptr->stages;
CAF_ASSERT(!stages.empty()); CAF_ASSERT(!stages.empty());
...@@ -571,11 +613,12 @@ protected: ...@@ -571,11 +613,12 @@ protected:
auto next = std::move(stages.back()); auto next = std::move(stages.back());
stages.pop_back(); stages.pop_back();
stream<T> token{sid}; stream<T> token{sid};
next->enqueue(make_mailbox_element( auto ys = std::tuple_cat(std::forward_as_tuple(token), std::move(xs));;
next->enqueue(
make_mailbox_element(
mptr->sender, mptr->mid, std::move(stages), mptr->sender, mptr->mid, std::move(stages),
make<stream_msg::open>(sid, make_message(token), ctrl(), make<stream_msg::open>(sid, make_message_from_tuple(std::move(ys)),
stream_priority::normal, ctrl(), stream_priority::normal, false)),
std::vector<atom_value>{}, false)),
context()); context());
mptr->mid.mark_as_answered(); mptr->mid.mark_as_answered();
} }
......
...@@ -34,21 +34,16 @@ class stream_handler : public ref_counted { ...@@ -34,21 +34,16 @@ class stream_handler : public ref_counted {
public: public:
~stream_handler() override; ~stream_handler() override;
// -- member types -----------------------------------------------------------
using topics = std::vector<atom_value>;
// -- handler for downstream events ------------------------------------------ // -- handler for downstream events ------------------------------------------
/// Add a new downstream actor to the stream. /// Add a new downstream actor to the stream.
/// @param hdl Handle to the new downstream actor. /// @param hdl Handle to the new downstream actor.
/// @param filter Subscribed topics (empty for all).
/// @param initial_demand Credit received with `ack_open`. /// @param initial_demand Credit received with `ack_open`.
/// @param redeployable Denotes whether the runtime can redeploy /// @param redeployable Denotes whether the runtime can redeploy
/// the downstream actor on failure. /// the downstream actor on failure.
/// @pre `hdl != nullptr` /// @pre `hdl != nullptr`
virtual error add_downstream(strong_actor_ptr& hdl, const topics& filter, virtual error add_downstream(strong_actor_ptr& hdl, size_t initial_demand,
size_t initial_demand, bool redeployable); bool redeployable);
/// Handles ACK message from a downstream actor. /// Handles ACK message from a downstream actor.
/// @pre `hdl != nullptr` /// @pre `hdl != nullptr`
......
...@@ -55,16 +55,13 @@ struct stream_msg : tag::boxing_type { ...@@ -55,16 +55,13 @@ struct stream_msg : tag::boxing_type {
static constexpr flow_label label = flows_downstream; static constexpr flow_label label = flows_downstream;
/// Allows the testing DSL to unbox this type automagically. /// Allows the testing DSL to unbox this type automagically.
using outer_type = stream_msg; using outer_type = stream_msg;
/// A type-erased stream<T> object for picking the correct message /// Contains a type-erased stream<T> object as first argument followed by
/// handler of the receiving actor. /// any number of user-defined additional handshake data.
message token; message msg;
/// A pointer to the previous stage in the pipeline. /// A pointer to the previous stage in the pipeline.
strong_actor_ptr prev_stage; strong_actor_ptr prev_stage;
/// Configures the priority for stream elements. /// Configures the priority for stream elements.
stream_priority priority; stream_priority priority;
/// Available topics for this stream. An empty vector indicates that the
/// upstream does provide only a single channel for this stream.
std::vector<atom_value> topics;
/// Tells the downstream whether rebindings can occur on this path. /// Tells the downstream whether rebindings can occur on this path.
bool redeployable; bool redeployable;
}; };
...@@ -78,9 +75,6 @@ struct stream_msg : tag::boxing_type { ...@@ -78,9 +75,6 @@ struct stream_msg : tag::boxing_type {
using outer_type = stream_msg; using outer_type = stream_msg;
/// Grants credit to the source. /// Grants credit to the source.
int32_t initial_demand; int32_t initial_demand;
/// Subscribes to a subset of the stream if non-empty. Otherwise, the
/// upstream sends all data of the stream.
std::vector<atom_value> filter;
/// Tells the upstream whether rebindings can occur on this path. /// Tells the upstream whether rebindings can occur on this path.
bool redeployable; bool redeployable;
}; };
...@@ -210,14 +204,13 @@ make(const stream_id& sid, Ts&&... xs) { ...@@ -210,14 +204,13 @@ make(const stream_id& sid, Ts&&... xs) {
template <class Inspector> template <class Inspector>
typename Inspector::result_type inspect(Inspector& f, stream_msg::open& x) { typename Inspector::result_type inspect(Inspector& f, stream_msg::open& x) {
return f(meta::type_name("open"), x.token, x.prev_stage, x.priority, return f(meta::type_name("open"), x.msg, x.prev_stage, x.priority,
x.topics, x.redeployable); x.redeployable);
} }
template <class Inspector> template <class Inspector>
typename Inspector::result_type inspect(Inspector& f, stream_msg::ack_open& x) { typename Inspector::result_type inspect(Inspector& f, stream_msg::ack_open& x) {
return f(meta::type_name("ack_open"), x.initial_demand, x.filter, return f(meta::type_name("ack_open"), x.initial_demand, x.redeployable);
x.redeployable);
} }
template <class Inspector> template <class Inspector>
......
...@@ -57,14 +57,10 @@ size_t abstract_downstream::min_credit() const { ...@@ -57,14 +57,10 @@ size_t abstract_downstream::min_credit() const {
return fold_paths(std::numeric_limits<size_t>::max(), f); return fold_paths(std::numeric_limits<size_t>::max(), f);
} }
bool abstract_downstream::add_path(strong_actor_ptr ptr, bool abstract_downstream::add_path(strong_actor_ptr ptr, bool redeployable) {
std::vector<atom_value> filter,
bool redeployable) {
auto predicate = [&](const path_uptr& x) { return x->ptr == ptr; }; auto predicate = [&](const path_uptr& x) { return x->ptr == ptr; };
if (std::none_of(paths_.begin(), paths_.end(), predicate)) { if (std::none_of(paths_.begin(), paths_.end(), predicate)) {
paths_.emplace_back( paths_.emplace_back(new path(std::move(ptr), redeployable));
new path(std::move(ptr), std::move(filter), redeployable));
recalculate_active_filters();
return true; return true;
} }
return false; return false;
...@@ -112,12 +108,6 @@ auto abstract_downstream::find(const strong_actor_ptr& ptr) const ...@@ -112,12 +108,6 @@ auto abstract_downstream::find(const strong_actor_ptr& ptr) const
return none; return none;
} }
void abstract_downstream::recalculate_active_filters() {
active_filters_.clear();
for (auto& x : paths_)
active_filters_.emplace(x->filter);
}
void abstract_downstream::send_batch(downstream_path& dest, size_t chunk_size, void abstract_downstream::send_batch(downstream_path& dest, size_t chunk_size,
message chunk) { message chunk) {
auto scs = static_cast<int32_t>(chunk_size); auto scs = static_cast<int32_t>(chunk_size);
......
...@@ -21,12 +21,10 @@ ...@@ -21,12 +21,10 @@
namespace caf { namespace caf {
downstream_path::downstream_path(strong_actor_ptr p, downstream_path::topics ts, downstream_path::downstream_path(strong_actor_ptr p, bool redeploy)
bool redeploy)
: ptr(std::move(p)), : ptr(std::move(p)),
next_batch_id(0), next_batch_id(0),
open_credit(0), open_credit(0),
filter(std::move(ts)),
redeployable(redeploy) { redeployable(redeploy) {
// nop // nop
} }
......
...@@ -66,9 +66,8 @@ void incoming_stream_multiplexer::operator()(stream_msg::open& x) { ...@@ -66,9 +66,8 @@ void incoming_stream_multiplexer::operator()(stream_msg::open& x) {
x.prev_stage = self_->ctrl(); x.prev_stage = self_->ctrl();
auto ptr = make_mailbox_element( auto ptr = make_mailbox_element(
cme->sender, cme->mid, std::move(cme->stages), cme->sender, cme->mid, std::move(cme->stages),
make<stream_msg::open>(current_stream_msg_->sid, std::move(x.token), make<stream_msg::open>(current_stream_msg_->sid, std::move(x.msg),
self_->ctrl(), x.priority, std::move(x.topics), self_->ctrl(), x.priority, x.redeployable));
x.redeployable));
successor->enqueue(std::move(ptr), self_->context()); successor->enqueue(std::move(ptr), self_->context());
// Send out demand upstream. // Send out demand upstream.
manage_credit(); manage_credit();
......
...@@ -78,9 +78,8 @@ void outgoing_stream_multiplexer::operator()(stream_msg::open& x) { ...@@ -78,9 +78,8 @@ void outgoing_stream_multiplexer::operator()(stream_msg::open& x) {
cme->sender, message_id::make(), {}, forward_atom::value, cme->sender, cme->sender, message_id::make(), {}, forward_atom::value, cme->sender,
std::move(cme->stages), path->hdl, cme->mid, std::move(cme->stages), path->hdl, cme->mid,
make_message(make<stream_msg::open>(current_stream_msg_->sid, make_message(make<stream_msg::open>(current_stream_msg_->sid,
std::move(x.token), self_->ctrl(), std::move(x.msg), self_->ctrl(),
x.priority, std::move(x.topics), x.priority, x.redeployable)));
x.redeployable)));
basp()->enqueue(std::move(ptr), self_->context()); basp()->enqueue(std::move(ptr), self_->context());
} }
......
...@@ -31,9 +31,7 @@ stream_handler::~stream_handler() { ...@@ -31,9 +31,7 @@ stream_handler::~stream_handler() {
// nop // nop
} }
error stream_handler::add_downstream(strong_actor_ptr&, error stream_handler::add_downstream(strong_actor_ptr&, size_t, bool) {
const stream_handler::topics&, size_t,
bool) {
CAF_LOG_ERROR("Cannot add downstream to a stream marked as no-downstreams"); CAF_LOG_ERROR("Cannot add downstream to a stream marked as no-downstreams");
return sec::cannot_add_downstream; return sec::cannot_add_downstream;
} }
......
...@@ -56,17 +56,16 @@ auto stream_msg_visitor::operator()(stream_msg::open& x) -> result_type { ...@@ -56,17 +56,16 @@ auto stream_msg_visitor::operator()(stream_msg::open& x) -> result_type {
return fail(sec::stream_init_failed); return fail(sec::stream_init_failed);
} }
auto bhvr = self_->bhvr_stack().back(); auto bhvr = self_->bhvr_stack().back();
auto res = bhvr(x.token); auto res = bhvr(x.msg);
if (!res) { if (!res) {
CAF_LOG_WARNING("stream handshake failed: actor did not respond to token:" CAF_LOG_WARNING("actor did not respond to handshake:" << CAF_ARG(x.msg));
<< CAF_ARG(x.token));
return fail(sec::stream_init_failed); return fail(sec::stream_init_failed);
} }
i_ = self_->streams().find(sid_); i_ = self_->streams().find(sid_);
if (i_ == e_) { if (i_ == e_) {
CAF_LOG_WARNING("stream handshake failed: actor did not provide a stream " CAF_LOG_WARNING("actor did not provide a stream "
"handler after receiving token:" "handler after receiving handshake:"
<< CAF_ARG(x.token)); << CAF_ARG(x.msg));
return fail(sec::stream_init_failed); return fail(sec::stream_init_failed);
} }
auto& handler = i_->second; auto& handler = i_->second;
...@@ -76,10 +75,9 @@ auto stream_msg_visitor::operator()(stream_msg::open& x) -> result_type { ...@@ -76,10 +75,9 @@ auto stream_msg_visitor::operator()(stream_msg::open& x) -> result_type {
// check whether we are a stage in a longer pipeline and send more // check whether we are a stage in a longer pipeline and send more
// stream_open messages if required // stream_open messages if required
auto ic = static_cast<int32_t>(*initial_credit); auto ic = static_cast<int32_t>(*initial_credit);
std::vector<atom_value> filter;
unsafe_send_as(self_, predecessor, unsafe_send_as(self_, predecessor,
make_message(make<stream_msg::ack_open>( make_message(make<stream_msg::ack_open>(std::move(sid_), ic,
std::move(sid_), ic, std::move(filter), false))); false)));
return {none, i_}; return {none, i_};
} }
self_->streams().erase(i_); self_->streams().erase(i_);
...@@ -107,7 +105,7 @@ auto stream_msg_visitor::operator()(stream_msg::abort& x) -> result_type { ...@@ -107,7 +105,7 @@ auto stream_msg_visitor::operator()(stream_msg::abort& x) -> result_type {
auto stream_msg_visitor::operator()(stream_msg::ack_open& x) -> result_type { auto stream_msg_visitor::operator()(stream_msg::ack_open& x) -> result_type {
CAF_LOG_TRACE(CAF_ARG(x)); CAF_LOG_TRACE(CAF_ARG(x));
if (i_ != e_) if (i_ != e_)
return {i_->second->add_downstream(self_->current_sender(), x.filter, return {i_->second->add_downstream(self_->current_sender(),
static_cast<size_t>(x.initial_demand), static_cast<size_t>(x.initial_demand),
false), false),
i_}; i_};
......
...@@ -41,8 +41,11 @@ namespace { ...@@ -41,8 +41,11 @@ namespace {
behavior file_reader(event_based_actor* self) { behavior file_reader(event_based_actor* self) {
using buf = std::deque<int>; using buf = std::deque<int>;
return { return {
[=](const std::string&) -> stream<int> { [=](std::string& fname) -> stream<int> {
CAF_CHECK_EQUAL(fname, "test.txt");
return self->add_source( return self->add_source(
// forward file name in handshake to next stage
std::forward_as_tuple(std::move(fname)),
// initialize state // initialize state
[&](buf& xs) { [&](buf& xs) {
xs = buf{1, 2, 3, 4, 5, 6, 7, 8, 9}; xs = buf{1, 2, 3, 4, 5, 6, 7, 8, 9};
...@@ -68,6 +71,8 @@ void streamer(event_based_actor* self, const actor& dest) { ...@@ -68,6 +71,8 @@ void streamer(event_based_actor* self, const actor& dest) {
self->new_stream( self->new_stream(
// destination of the stream // destination of the stream
dest, dest,
// "file name" as seen by the next stage
std::make_tuple("test.txt"),
// initialize state // initialize state
[&](buf& xs) { [&](buf& xs) {
xs = buf{1, 2, 3, 4, 5, 6, 7, 8, 9}; xs = buf{1, 2, 3, 4, 5, 6, 7, 8, 9};
...@@ -92,10 +97,13 @@ void streamer(event_based_actor* self, const actor& dest) { ...@@ -92,10 +97,13 @@ void streamer(event_based_actor* self, const actor& dest) {
behavior filter(event_based_actor* self) { behavior filter(event_based_actor* self) {
return { return {
[=](stream<int>& in) -> stream<int> { [=](stream<int>& in, std::string& fname) -> stream<int> {
CAF_CHECK_EQUAL(fname, "test.txt");
return self->add_stage( return self->add_stage(
// input stream // input stream
in, in,
// forward file name in handshake to next stage
std::forward_as_tuple(std::move(fname)),
// initialize state // initialize state
[=](unit_t&) { [=](unit_t&) {
// nop // nop
...@@ -116,7 +124,8 @@ behavior filter(event_based_actor* self) { ...@@ -116,7 +124,8 @@ behavior filter(event_based_actor* self) {
behavior broken_filter(event_based_actor*) { behavior broken_filter(event_based_actor*) {
return { return {
[=](stream<int>& x) -> stream<int> { [=](stream<int>& x, const std::string& fname) -> stream<int> {
CAF_CHECK_EQUAL(fname, "test.txt");
return x; return x;
} }
}; };
...@@ -124,7 +133,8 @@ behavior broken_filter(event_based_actor*) { ...@@ -124,7 +133,8 @@ behavior broken_filter(event_based_actor*) {
behavior sum_up(event_based_actor* self) { behavior sum_up(event_based_actor* self) {
return { return {
[=](stream<int>& in) { [=](stream<int>& in, std::string& fname) {
CAF_CHECK_EQUAL(fname, "test.txt");
return self->add_sink( return self->add_sink(
// input stream // input stream
in, in,
...@@ -147,7 +157,8 @@ behavior sum_up(event_based_actor* self) { ...@@ -147,7 +157,8 @@ behavior sum_up(event_based_actor* self) {
behavior drop_all(event_based_actor* self) { behavior drop_all(event_based_actor* self) {
return { return {
[=](stream<int>& in) { [=](stream<int>& in, std::string& fname) {
CAF_CHECK_EQUAL(fname, "test.txt");
return self->add_sink( return self->add_sink(
// input stream // input stream
in, in,
...@@ -174,6 +185,8 @@ void streamer_without_result(event_based_actor* self, const actor& dest) { ...@@ -174,6 +185,8 @@ void streamer_without_result(event_based_actor* self, const actor& dest) {
self->new_stream( self->new_stream(
// destination of the stream // destination of the stream
dest, dest,
// "file name" for the next stage
std::make_tuple("test.txt"),
// initialize state // initialize state
[&](buf& xs) { [&](buf& xs) {
xs = buf{1, 2, 3, 4, 5, 6, 7, 8, 9}; xs = buf{1, 2, 3, 4, 5, 6, 7, 8, 9};
...@@ -652,51 +665,4 @@ CAF_TEST(stream_crossing_the_wire) { ...@@ -652,51 +665,4 @@ CAF_TEST(stream_crossing_the_wire) {
mars.sched.run(); mars.sched.run();
anon_send_exit(source, exit_reason::user_shutdown); anon_send_exit(source, exit_reason::user_shutdown);
earth.sched.run(); earth.sched.run();
/*
// earth.stream_serv --('sys', 'ok', 5)--> earth.stream_serv
network_traffic();
expect_on(earth, (atom_value, atom_value, int32_t),
from(_).to(earth.stream_serv)
.with(sys_atom::value, ok_atom::value, 5));
network_traffic();
// mars.stream_serv --(stream_msg::ack_open)--> earth.stream_serv
expect_on(earth, (stream_msg::ack_open),
from(_).to(earth.stream_serv)
.with(5, _, false));
// earth.stream_serv --(stream_msg::ack_open)--> source
expect_on(earth, (stream_msg::ack_open),
from(earth.stream_serv).to(source)
.with(5, _, false));
// source --(stream_msg::ack_open)--> earth.stream_serv
expect_on_path(
(stream_msg::batch), with(5, std::vector<int>{1, 2, 3, 4, 5}, 0),
{earth, earth.stream_serv},
{mars, mars.stream_serv}, {mars, sink});
*/
/*
expect_on(earth, (stream_msg::batch),
from(source).to(earth.stream_serv)
.with(5, std::vector<int>{1, 2, 3, 4, 5}, 0));
*/
/*
// source <----(stream_msg::ack_open)------ sink
expect((stream_msg::ack_open), from(sink).to(source).with(5, _, false));
// source ----(stream_msg::batch)---> sink
expect((stream_msg::batch),
from(source).to(sink).with(5, std::vector<int>{1, 2, 3, 4, 5}, 0));
// source <--(stream_msg::ack_batch)---- sink
expect((stream_msg::ack_batch), from(sink).to(source).with(5, 0));
// source ----(stream_msg::batch)---> sink
expect((stream_msg::batch),
from(source).to(sink).with(4, std::vector<int>{6, 7, 8, 9}, 1));
// source <--(stream_msg::ack_batch)---- sink
expect((stream_msg::ack_batch), from(sink).to(source).with(4, 1));
// source ----(stream_msg::close)---> sink
expect((stream_msg::close), from(source).to(sink).with());
// sink ----(result: <empty>)---> source
expect((void), from(sink).to(source).with());
*/
} }
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