Commit 6d4d62e5 authored by Dominik Charousset's avatar Dominik Charousset Committed by Dominik Charousset

Remove unused redeployment state

parent 16351b4a
...@@ -146,12 +146,10 @@ public: ...@@ -146,12 +146,10 @@ public:
// -- overridden member functions of `stream_manager` ------------------------ // -- overridden member functions of `stream_manager` ------------------------
error ack_open(const stream_id& sid, const actor_addr& rebind_from, error ack_open(const stream_id& sid, const actor_addr& rebind_from,
strong_actor_ptr rebind_to, long initial_demand, strong_actor_ptr rebind_to, long initial_demand) override {
bool redeployable) override {
CAF_LOG_TRACE(CAF_ARG(sid) << CAF_ARG(rebind_from) << CAF_ARG(rebind_to) CAF_LOG_TRACE(CAF_ARG(sid) << CAF_ARG(rebind_from) << CAF_ARG(rebind_to)
<< CAF_ARG(initial_demand) << CAF_ARG(redeployable)); << CAF_ARG(initial_demand));
auto res = super::ack_open(sid, rebind_from, rebind_to, initial_demand, auto res = super::ack_open(sid, rebind_from, rebind_to, initial_demand);
redeployable);
if (res == none) if (res == none)
policy_.ack_open_success(sid, rebind_from, rebind_to); policy_.ack_open_success(sid, rebind_from, rebind_to);
else else
......
...@@ -61,7 +61,7 @@ public: ...@@ -61,7 +61,7 @@ public:
init(state_); init(state_);
} }
handshake_tuple_type make_handshake() override { handshake_tuple_type make_handshake() const override {
return std::tuple_cat(std::make_tuple(none), hs_); return std::tuple_cat(std::make_tuple(none), hs_);
} }
......
...@@ -63,7 +63,7 @@ public: ...@@ -63,7 +63,7 @@ public:
// -- implementation of virtual functions ------------------------------------ // -- implementation of virtual functions ------------------------------------
bool done() const override { bool done() const override {
return at_end_ && out_.clean(); return this->pending_handshakes_ == 0 && at_end_ && out_.clean();
} }
Scatterer& out() override { Scatterer& out() override {
......
...@@ -66,7 +66,8 @@ public: ...@@ -66,7 +66,8 @@ public:
} }
bool done() const override { bool done() const override {
return this->inbound_paths_.empty() && out_.clean(); return this->pending_handshakes_ == 0 && this->inbound_paths_.empty()
&& out_.clean();
} }
error handle(inbound_path*, downstream_msg::batch& x) override { error handle(inbound_path*, downstream_msg::batch& x) override {
...@@ -82,7 +83,7 @@ public: ...@@ -82,7 +83,7 @@ public:
return sec::unexpected_message; return sec::unexpected_message;
} }
message make_output_token(const stream_id&) const override { message make_handshake() const override {
return make_message_from_tuple(driver_.make_handshake()); return make_message_from_tuple(driver_.make_handshake());
} }
......
...@@ -67,10 +67,6 @@ public: ...@@ -67,10 +67,6 @@ public:
/// ID of the last received batch. /// ID of the last received batch.
int64_t last_batch_id; int64_t last_batch_id;
/// Stores whether the source actor is failsafe, i.e., allows the runtime to
/// redeploy it on failure.
bool redeployable;
/// Stores whether an error occurred during stream processing. Configures /// Stores whether an error occurred during stream processing. Configures
/// whether the destructor sends `close` or `forced_close` messages. /// whether the destructor sends `close` or `forced_close` messages.
error shutdown_reason; error shutdown_reason;
...@@ -122,8 +118,7 @@ public: ...@@ -122,8 +118,7 @@ public:
/// Emits a `stream_msg::ack_batch` on this path and sets `assigned_credit` /// Emits a `stream_msg::ack_batch` on this path and sets `assigned_credit`
/// to `initial_demand`. /// to `initial_demand`.
void emit_ack_open(local_actor* self, actor_addr rebind_from, void emit_ack_open(local_actor* self, actor_addr rebind_from);
bool is_redeployable);
/// Sends a `stream_msg::ack_batch` with credit for the next round. Credit is /// Sends a `stream_msg::ack_batch` with credit for the next round. Credit is
/// calculated as `max_queue_size - (assigned_credit - queued_items)`, whereas /// calculated as `max_queue_size - (assigned_credit - queued_items)`, whereas
......
...@@ -74,7 +74,7 @@ public: ...@@ -74,7 +74,7 @@ public:
/// Sends a stream handshake. /// Sends a stream handshake.
static void emit_open(local_actor* self, stream_slot slot, static void emit_open(local_actor* self, stream_slot slot,
strong_actor_ptr to, message handshake_data, strong_actor_ptr to, message handshake_data,
stream_priority prio, bool is_redeployable); stream_priority prio);
/// Sends a `stream_msg::batch` on this path, decrements `open_credit` by /// Sends a `stream_msg::batch` on this path, decrements `open_credit` by
/// Sets `open_credit` to `initial_credit` and clears `cached_handshake`. /// Sets `open_credit` to `initial_credit` and clears `cached_handshake`.
...@@ -111,18 +111,10 @@ public: ...@@ -111,18 +111,10 @@ public:
/// Batch size configured by the downstream actor. /// Batch size configured by the downstream actor.
uint64_t desired_batch_size; uint64_t desired_batch_size;
/// 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
/// `unacknowledged_batches` is unused.
bool redeployable;
/// Next expected batch ID to be acknowledged. Actors can receive a more /// Next expected batch ID to be acknowledged. Actors can receive a more
/// advanced batch ID in an ACK message, since CAF uses accumulative ACKs. /// advanced batch ID in an ACK message, since CAF uses accumulative ACKs.
int64_t next_ack_id; int64_t next_ack_id;
/// Caches batches until receiving an ACK.
cache_type unacknowledged_batches;
/// Caches the initiator of the stream (client) with the original request ID /// Caches the initiator of the stream (client) with the original request ID
/// until the stream handshake is either confirmed or aborted. Once /// until the stream handshake is either confirmed or aborted. Once
/// confirmed, the next stage takes responsibility for answering to the /// confirmed, the next stage takes responsibility for answering to the
...@@ -144,8 +136,8 @@ typename Inspector::result_type inspect(Inspector& f, ...@@ -144,8 +136,8 @@ typename Inspector::result_type inspect(Inspector& f,
template <class Inspector> template <class Inspector>
typename Inspector::result_type inspect(Inspector& f, outbound_path& x) { typename Inspector::result_type inspect(Inspector& f, outbound_path& x) {
return f(meta::type_name("outbound_path"), x.slots, x.hdl, x.next_batch_id, return f(meta::type_name("outbound_path"), x.slots, x.hdl, x.next_batch_id,
x.open_credit, x.desired_batch_size, x.redeployable, x.next_ack_id, x.open_credit, x.desired_batch_size, x.next_ack_id, x.cd,
x.unacknowledged_batches, x.cd, x.shutdown_reason); x.shutdown_reason);
} }
} // namespace caf } // namespace caf
......
...@@ -366,9 +366,23 @@ public: ...@@ -366,9 +366,23 @@ public:
return {slot, std::move(ptr)}; return {slot, std::move(ptr)};
} }
template <class... Ts, class Init, class Pull, class Done,
class Scatterer =
broadcast_scatterer<typename stream_source_trait_t<Pull>::output>,
class Trait = stream_source_trait_t<Pull>>
annotated_stream<typename Trait::output, detail::decay_t<Ts>...>
make_source(std::tuple<Ts...> xs, Init init, Pull pull, Done done,
policy::arg<Scatterer> = {}) {
using tuple_type = std::tuple<detail::decay_t<Ts>...>;
using driver = detail::stream_source_driver_impl<typename Trait::output,
Pull, Done, tuple_type>;
return make_source<driver, Scatterer>(std::move(init), std::move(pull),
std::move(done), std::move(xs));
}
template <class Driver, class Input, class... Ts> template <class Driver, class Input, class... Ts>
stream_result<typename Driver::output_type> stream_result<typename Driver::output_type> make_sink(const stream<Input>& in,
make_sink(const stream<Input>& in, Ts&&... xs) { Ts&&... xs) {
auto slot = next_slot(); auto slot = next_slot();
stream_slots id{in.slot(), slot}; stream_slots id{in.slot(), slot};
auto ptr = detail::make_stream_sink<Driver>(this, std::forward<Ts>(xs)...); auto ptr = detail::make_stream_sink<Driver>(this, std::forward<Ts>(xs)...);
...@@ -606,8 +620,7 @@ public: ...@@ -606,8 +620,7 @@ public:
auto ptr = make_counted<T>(this, std::forward<Ts>(xs)...); auto ptr = make_counted<T>(this, std::forward<Ts>(xs)...);
auto rp = make_response_promise(); auto rp = make_response_promise();
if (!add_source(ptr, sid, std::move(opn.prev_stage), if (!add_source(ptr, sid, std::move(opn.prev_stage),
std::move(opn.original_stage), opn.priority, std::move(opn.original_stage), opn.priority, rp)) {
opn.redeployable, rp)) {
CAF_LOG_ERROR("cannot create stream stage without source"); CAF_LOG_ERROR("cannot create stream stage without source");
rp.deliver(sec::cannot_add_upstream); rp.deliver(sec::cannot_add_upstream);
return none; return none;
...@@ -770,7 +783,6 @@ public: ...@@ -770,7 +783,6 @@ public:
/// @param sid The ID used for communicating to the sink. /// @param sid The ID used for communicating to the sink.
/// @param source_ptr Handle to the new source. /// @param source_ptr Handle to the new source.
/// @param prio Priority of the traffic from the source. /// @param prio Priority of the traffic from the source.
/// @param redeployable Configures whether source can re-appear after aborts.
/// @param result_cb Callback for the listener of the final stream result. /// @param result_cb Callback for the listener of the final stream result.
/// Ignored when returning `nullptr`, because the previous /// Ignored when returning `nullptr`, because the previous
/// stage is responsible for it until this manager /// stage is responsible for it until this manager
...@@ -779,8 +791,7 @@ public: ...@@ -779,8 +791,7 @@ public:
/// otherwise. /// otherwise.
bool add_source(const stream_manager_ptr& mgr, const stream_id& sid, bool add_source(const stream_manager_ptr& mgr, const stream_id& sid,
strong_actor_ptr source_ptr, strong_actor_ptr original_stage, strong_actor_ptr source_ptr, strong_actor_ptr original_stage,
stream_priority prio, bool redeployable, stream_priority prio, response_promise result_cb);
response_promise result_cb);
/// Adds a new source to the stream manager `mgr` if `current_message()` is a /// Adds a new source to the stream manager `mgr` if `current_message()` is a
/// `stream_msg::open` handshake. /// `stream_msg::open` handshake.
...@@ -829,7 +840,7 @@ public: ...@@ -829,7 +840,7 @@ public:
// from the next stage. // from the next stage.
if (!add_source(mgr, sid, std::move(opn.prev_stage), if (!add_source(mgr, sid, std::move(opn.prev_stage),
std::move(opn.original_stage), opn.priority, std::move(opn.original_stage), opn.priority,
opn.redeployable, none)) { none)) {
CAF_LOG_ERROR("cannot create stream stage without source"); CAF_LOG_ERROR("cannot create stream stage without source");
auto rp = make_response_promise(); auto rp = make_response_promise();
rp.deliver(sec::cannot_add_upstream); rp.deliver(sec::cannot_add_upstream);
......
...@@ -47,8 +47,6 @@ public: ...@@ -47,8 +47,6 @@ public:
/// @param hdl Handle to the sender. /// @param hdl Handle to the sender.
/// @param original_stage Handle to the initial receiver of the handshake. /// @param original_stage Handle to the initial receiver of the handshake.
/// @param priority Affects credit assignment and maximum bandwidth. /// @param priority Affects credit assignment and maximum bandwidth.
/// @param redeployable Configures whether `hdl` could get redeployed, i.e.,
/// can resume after an abort.
/// @param result_cb Callback for the listener of the final stream result. /// @param result_cb Callback for the listener of the final stream result.
/// Ignored when returning `nullptr`, because the previous /// Ignored when returning `nullptr`, because the previous
/// stage is responsible for it until this manager /// stage is responsible for it until this manager
...@@ -58,7 +56,7 @@ public: ...@@ -58,7 +56,7 @@ public:
/* /*
virtual error open(stream_slot slot, strong_actor_ptr hdl, virtual error open(stream_slot slot, strong_actor_ptr hdl,
strong_actor_ptr original_stage, stream_priority priority, strong_actor_ptr original_stage, stream_priority priority,
bool redeployable, response_promise result_cb); response_promise result_cb);
*/ */
virtual error handle(inbound_path* from, downstream_msg::batch& x); virtual error handle(inbound_path* from, downstream_msg::batch& x);
...@@ -93,6 +91,13 @@ public: ...@@ -93,6 +91,13 @@ public:
/// since it is unable to make progress sending on its own. /// since it is unable to make progress sending on its own.
virtual bool congested() const; virtual bool congested() const;
/// 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, stream_priority prio);
// -- implementation hooks for sources --------------------------------------- // -- implementation hooks for sources ---------------------------------------
/// Tries to generate new messages for the stream. This member function does /// Tries to generate new messages for the stream. This member function does
...@@ -102,7 +107,7 @@ public: ...@@ -102,7 +107,7 @@ public:
// -- pure virtual member functions ------------------------------------------ // -- pure virtual member functions ------------------------------------------
/// Returns the stream edge for outgoing data. /// Returns the scatterer for outgoing data.
virtual stream_scatterer& out() = 0; virtual stream_scatterer& out() = 0;
/// Returns whether the stream has reached the end and can be discarded /// Returns whether the stream has reached the end and can be discarded
...@@ -142,7 +147,7 @@ protected: ...@@ -142,7 +147,7 @@ protected:
/// Returns a type-erased `stream<T>` as handshake token for downstream /// Returns a type-erased `stream<T>` as handshake token for downstream
/// actors. Returns an empty message for sinks. /// actors. Returns an empty message for sinks.
virtual message make_output_token(const stream_id&) const; virtual message make_handshake() const;
/// Called whenever new credit becomes available. The default implementation /// Called whenever new credit becomes available. The default implementation
/// logs an error (sources are expected to override this hook). /// logs an error (sources are expected to override this hook).
...@@ -159,6 +164,7 @@ protected: ...@@ -159,6 +164,7 @@ protected:
std::vector<inbound_path*> inbound_paths_; std::vector<inbound_path*> inbound_paths_;
/// Keeps track of pending handshakes. /// Keeps track of pending handshakes.
long pending_handshakes_;
}; };
/// A reference counting pointer to a `stream_manager`. /// A reference counting pointer to a `stream_manager`.
......
...@@ -107,16 +107,13 @@ struct open_stream_msg { ...@@ -107,16 +107,13 @@ struct open_stream_msg {
/// Configures the priority for stream elements. /// Configures the priority for stream elements.
stream_priority priority; stream_priority priority;
/// Tells the downstream whether rebindings can occur on this path.
bool redeployable;
}; };
/// @relates stream_handshake_msg /// @relates stream_handshake_msg
template <class Inspector> template <class Inspector>
typename Inspector::result_type inspect(Inspector& f, open_stream_msg& x) { typename Inspector::result_type inspect(Inspector& f, open_stream_msg& x) {
return f(meta::type_name("stream_handshake_msg"), x.slot, x.msg, x.prev_stage, return f(meta::type_name("stream_handshake_msg"), x.slot, x.msg, x.prev_stage,
x.original_stage, x.priority, x.redeployable); x.original_stage, x.priority);
} }
} // namespace caf } // namespace caf
......
...@@ -61,9 +61,6 @@ struct upstream_msg : tag::boxing_type { ...@@ -61,9 +61,6 @@ struct upstream_msg : tag::boxing_type {
/// Desired size of individual batches. /// Desired size of individual batches.
int32_t desired_batch_size; int32_t desired_batch_size;
/// Tells the upstream whether rebindings can occur on this path.
bool redeployable;
}; };
/// Cumulatively acknowledges received batches and signalizes new demand from /// Cumulatively acknowledges received batches and signalizes new demand from
...@@ -163,7 +160,7 @@ template <class Inspector> ...@@ -163,7 +160,7 @@ template <class Inspector>
typename Inspector::result_type inspect(Inspector& f, typename Inspector::result_type inspect(Inspector& f,
upstream_msg::ack_open& x) { upstream_msg::ack_open& x) {
return f(meta::type_name("ack_open"), x.rebind_from, x.rebind_to, return f(meta::type_name("ack_open"), x.rebind_from, x.rebind_to,
x.initial_demand, x.desired_batch_size, x.redeployable); x.initial_demand, x.desired_batch_size);
} }
/// @relates upstream_msg::ack_batch /// @relates upstream_msg::ack_batch
......
...@@ -65,8 +65,7 @@ inbound_path::inbound_path(stream_manager_ptr mgr_ptr, stream_slots id, ...@@ -65,8 +65,7 @@ inbound_path::inbound_path(stream_manager_ptr mgr_ptr, stream_slots id,
assigned_credit(0), assigned_credit(0),
prio(stream_priority::normal), prio(stream_priority::normal),
last_acked_batch_id(0), last_acked_batch_id(0),
last_batch_id(0), last_batch_id(0) {
redeployable(false) {
mgr->register_input_path(this); mgr->register_input_path(this);
} }
...@@ -83,16 +82,14 @@ void inbound_path::handle(downstream_msg::batch& x) { ...@@ -83,16 +82,14 @@ void inbound_path::handle(downstream_msg::batch& x) {
mgr->push(); mgr->push();
} }
void inbound_path::emit_ack_open(local_actor* self, actor_addr rebind_from, void inbound_path::emit_ack_open(local_actor* self, actor_addr rebind_from) {
bool is_redeployable) {
assigned_credit = 50; // TODO: put constant in some header assigned_credit = 50; // TODO: put constant in some header
redeployable = is_redeployable;
int32_t desired_batch_size = 50; // TODO: put constant in some header int32_t desired_batch_size = 50; // TODO: put constant in some header
unsafe_send_as(self, hdl, unsafe_send_as(self, hdl,
make<upstream_msg::ack_open>( make<upstream_msg::ack_open>(
slots.invert(), self->address(), std::move(rebind_from), slots.invert(), self->address(), std::move(rebind_from),
self->ctrl(), static_cast<int32_t>(assigned_credit), self->ctrl(), static_cast<int32_t>(assigned_credit),
desired_batch_size, is_redeployable)); desired_batch_size));
} }
void inbound_path::emit_ack_batch(local_actor* self, long queued_items, void inbound_path::emit_ack_batch(local_actor* self, long queued_items,
......
...@@ -31,7 +31,6 @@ outbound_path::outbound_path(stream_slots id, strong_actor_ptr ptr) ...@@ -31,7 +31,6 @@ outbound_path::outbound_path(stream_slots id, strong_actor_ptr ptr)
next_batch_id(0), next_batch_id(0),
open_credit(0), open_credit(0),
desired_batch_size(0), desired_batch_size(0),
redeployable(false),
next_ack_id(0) { next_ack_id(0) {
// nop // nop
} }
...@@ -42,13 +41,13 @@ outbound_path::~outbound_path() { ...@@ -42,13 +41,13 @@ outbound_path::~outbound_path() {
void outbound_path::emit_open(local_actor* self, stream_slot slot, void outbound_path::emit_open(local_actor* self, stream_slot slot,
strong_actor_ptr to, message handshake_data, strong_actor_ptr to, message handshake_data,
stream_priority prio, bool is_redeployable) { stream_priority prio) {
CAF_ASSERT(self != nullptr); CAF_ASSERT(self != nullptr);
CAF_ASSERT(to != nullptr); CAF_ASSERT(to != nullptr);
// TODO: attach an aborter to `to` // TODO: attach an aborter to `to`
unsafe_send_as(self, to, unsafe_send_as(self, to,
open_stream_msg{slot, std::move(handshake_data), self->ctrl(), open_stream_msg{slot, std::move(handshake_data), self->ctrl(),
nullptr, prio, is_redeployable}); nullptr, prio});
} }
void outbound_path::emit_batch(local_actor* self, long xs_size, message xs) { void outbound_path::emit_batch(local_actor* self, long xs_size, message xs) {
...@@ -57,8 +56,6 @@ void outbound_path::emit_batch(local_actor* self, long xs_size, message xs) { ...@@ -57,8 +56,6 @@ void outbound_path::emit_batch(local_actor* self, long xs_size, message xs) {
auto bid = next_batch_id++; auto bid = next_batch_id++;
downstream_msg::batch batch{static_cast<int32_t>(xs_size), std::move(xs), downstream_msg::batch batch{static_cast<int32_t>(xs_size), std::move(xs),
bid}; bid};
if (redeployable)
unacknowledged_batches.emplace_back(bid, batch);
unsafe_send_as(self, hdl, unsafe_send_as(self, hdl,
downstream_msg{slots, self->address(), std::move(batch)}); downstream_msg{slots, self->address(), std::move(batch)});
} }
......
...@@ -704,11 +704,11 @@ bool scheduled_actor::add_source(const stream_manager_ptr& mgr, ...@@ -704,11 +704,11 @@ bool scheduled_actor::add_source(const stream_manager_ptr& mgr,
const stream_id& sid, const stream_id& sid,
strong_actor_ptr source_ptr, strong_actor_ptr source_ptr,
strong_actor_ptr original_stage, strong_actor_ptr original_stage,
stream_priority prio, bool redeployable, stream_priority prio,
response_promise result_cb) { response_promise result_cb) {
CAF_LOG_TRACE(CAF_ARG(mgr) << CAF_ARG(sid) << CAF_ARG(source_ptr) CAF_LOG_TRACE(CAF_ARG(mgr) << CAF_ARG(sid) << CAF_ARG(source_ptr)
<< CAF_ARG(original_stage) << CAF_ARG(prio) << CAF_ARG(original_stage) << CAF_ARG(prio)
<< CAF_ARG(redeployable) << CAF_ARG(result_cb)); << CAF_ARG(result_cb));
CAF_ASSERT(mgr != nullptr); CAF_ASSERT(mgr != nullptr);
if (!source_ptr) { if (!source_ptr) {
CAF_LOG_ERROR("cannot add invalid source"); CAF_LOG_ERROR("cannot add invalid source");
...@@ -719,7 +719,7 @@ bool scheduled_actor::add_source(const stream_manager_ptr& mgr, ...@@ -719,7 +719,7 @@ bool scheduled_actor::add_source(const stream_manager_ptr& mgr,
return false; return false;
} }
return mgr->add_source(sid, std::move(source_ptr), return mgr->add_source(sid, std::move(source_ptr),
std::move(original_stage), prio, redeployable, std::move(original_stage), prio,
std::move(result_cb)); std::move(result_cb));
} }
...@@ -744,7 +744,7 @@ bool scheduled_actor::add_source(const stream_manager_ptr& mgr, ...@@ -744,7 +744,7 @@ bool scheduled_actor::add_source(const stream_manager_ptr& mgr,
auto source_ptr = std::move(opn.prev_stage); auto source_ptr = std::move(opn.prev_stage);
return mgr->add_source(sid, std::move(source_ptr), return mgr->add_source(sid, std::move(source_ptr),
std::move(opn.original_stage), opn.priority, std::move(opn.original_stage), opn.priority,
opn.redeployable, std::move(result_cb)); std::move(result_cb));
} }
*/ */
......
...@@ -24,6 +24,7 @@ ...@@ -24,6 +24,7 @@
#include "caf/error.hpp" #include "caf/error.hpp"
#include "caf/expected.hpp" #include "caf/expected.hpp"
#include "caf/inbound_path.hpp" #include "caf/inbound_path.hpp"
#include "caf/local_actor.hpp"
#include "caf/logger.hpp" #include "caf/logger.hpp"
#include "caf/message.hpp" #include "caf/message.hpp"
#include "caf/outbound_path.hpp" #include "caf/outbound_path.hpp"
...@@ -33,7 +34,9 @@ ...@@ -33,7 +34,9 @@
namespace caf { namespace caf {
stream_manager::stream_manager(local_actor* selfptr) : self_(selfptr) { stream_manager::stream_manager(local_actor* selfptr)
: self_(selfptr),
pending_handshakes_(0) {
// nop // nop
} }
...@@ -41,34 +44,32 @@ stream_manager::~stream_manager() { ...@@ -41,34 +44,32 @@ stream_manager::~stream_manager() {
// nop // nop
} }
error stream_manager::handle(inbound_path* from, downstream_msg::batch& x) { error stream_manager::handle(inbound_path*, downstream_msg::batch&) {
return none; return none;
} }
error stream_manager::handle(inbound_path* from, downstream_msg::close& x) { error stream_manager::handle(inbound_path* from, downstream_msg::close&) {
out().take_path(from->slots); out().take_path(from->slots);
return none; return none;
} }
error stream_manager::handle(inbound_path* from, error stream_manager::handle(inbound_path*, downstream_msg::forced_close&) {
downstream_msg::forced_close& x) {
return none; return none;
} }
error stream_manager::handle(outbound_path* from, upstream_msg::ack_open& x) { error stream_manager::handle(outbound_path*, upstream_msg::ack_open&) {
return none; return none;
} }
error stream_manager::handle(outbound_path* from, upstream_msg::ack_batch& x) { error stream_manager::handle(outbound_path*, upstream_msg::ack_batch&) {
return none; return none;
} }
error stream_manager::handle(outbound_path* from, upstream_msg::drop& x) { error stream_manager::handle(outbound_path*, upstream_msg::drop&) {
return none; return none;
} }
error stream_manager::handle(outbound_path* from, error stream_manager::handle(outbound_path*, upstream_msg::forced_drop&) {
upstream_msg::forced_drop& x) {
return none; return none;
} }
...@@ -93,6 +94,19 @@ bool stream_manager::congested() const { ...@@ -93,6 +94,19 @@ bool stream_manager::congested() const {
return false; 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, stream_priority prio) {
CAF_ASSERT(dest != nullptr);
++pending_handshakes_;
dest->enqueue(
make_mailbox_element(
std::move(client), mid, std::move(fwd_stack),
open_stream_msg{slot, make_handshake(), self_->ctrl(), dest, prio}),
self_->context());
}
bool stream_manager::generate_messages() { bool stream_manager::generate_messages() {
return false; return false;
} }
...@@ -124,7 +138,7 @@ void stream_manager::output_closed(error) { ...@@ -124,7 +138,7 @@ void stream_manager::output_closed(error) {
// nop // nop
} }
message stream_manager::make_output_token(const stream_id&) const { message stream_manager::make_handshake() const {
CAF_LOG_ERROR("stream_manager::make_output_token called"); CAF_LOG_ERROR("stream_manager::make_output_token called");
return none; return none;
} }
......
...@@ -88,8 +88,7 @@ unique_pointer stream_scatterer::take_path_at(size_t index) noexcept { ...@@ -88,8 +88,7 @@ unique_pointer stream_scatterer::take_path_at(size_t index) noexcept {
bool stream_scatterer::clean() const noexcept { bool stream_scatterer::clean() const noexcept {
auto pred = [](const map_type::value_type& kvp) { auto pred = [](const map_type::value_type& kvp) {
auto& p = *kvp.second; auto& p = *kvp.second;
return p.next_batch_id == p.next_ack_id return p.next_batch_id == p.next_ack_id;
&& p.unacknowledged_batches.empty();
}; };
return buffered() == 0 && std::all_of(paths_.begin(), paths_.end(), pred); return buffered() == 0 && std::all_of(paths_.begin(), paths_.end(), pred);
} }
......
...@@ -318,7 +318,7 @@ public: ...@@ -318,7 +318,7 @@ public:
CAF_MESSAGE(name_ << " starts streaming to " << ref.name() CAF_MESSAGE(name_ << " starts streaming to " << ref.name()
<< " on slot " << slot); << " on slot " << slot);
outbound_path::emit_open(this, slot, ref.ctrl(), make_message(), outbound_path::emit_open(this, slot, ref.ctrl(), make_message(),
stream_priority::normal, false); stream_priority::normal);
struct driver final : public stream_source_driver<int> { struct driver final : public stream_source_driver<int> {
public: public:
driver(int sentinel) : x_(0), sentinel_(sentinel) { driver(int sentinel) : x_(0), sentinel_(sentinel) {
...@@ -353,7 +353,7 @@ public: ...@@ -353,7 +353,7 @@ public:
<< " on slot " << slot); << " on slot " << slot);
strong_actor_ptr to = ref.ctrl(); strong_actor_ptr to = ref.ctrl();
send(to, open_stream_msg{slot, make_message(), ctrl(), nullptr, send(to, open_stream_msg{slot, make_message(), ctrl(), nullptr,
stream_priority::normal, false}); stream_priority::normal});
struct driver final : public stream_stage_driver<int, int> { struct driver final : public stream_stage_driver<int, int> {
public: public:
driver(vector<int>* log) : log_(log) { driver(vector<int>* log) : log_(log) {
...@@ -405,7 +405,7 @@ public: ...@@ -405,7 +405,7 @@ public:
get<2>(mbox.queues()) get<2>(mbox.queues())
.queues() .queues()
.emplace(slot, std::unique_ptr<inbound_path>{ip}); .emplace(slot, std::unique_ptr<inbound_path>{ip});
ip->emit_ack_open(this, actor_cast<actor_addr>(hs.original_stage), false); ip->emit_ack_open(this, actor_cast<actor_addr>(hs.original_stage));
} }
void operator()(stream_slots slots, actor_addr& sender, void operator()(stream_slots slots, actor_addr& sender,
......
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