Commit 6cdf7973 authored by Dominik Charousset's avatar Dominik Charousset

Integrate streams into scheduled actors

parent f094d85b
...@@ -132,6 +132,7 @@ caf_add_component( ...@@ -132,6 +132,7 @@ caf_add_component(
src/detail/private_thread_pool.cpp src/detail/private_thread_pool.cpp
src/detail/ripemd_160.cpp src/detail/ripemd_160.cpp
src/detail/set_thread_name.cpp src/detail/set_thread_name.cpp
src/detail/stream_bridge.cpp
src/detail/stringification_inspector.cpp src/detail/stringification_inspector.cpp
src/detail/sync_request_bouncer.cpp src/detail/sync_request_bouncer.cpp
src/detail/test_actor_clock.cpp src/detail/test_actor_clock.cpp
......
...@@ -8,7 +8,6 @@ ...@@ -8,7 +8,6 @@
#include "caf/config.hpp" #include "caf/config.hpp"
#include "caf/detail/core_export.hpp" #include "caf/detail/core_export.hpp"
#include "caf/error.hpp"
#include "caf/fwd.hpp" #include "caf/fwd.hpp"
#include "caf/intrusive_ptr.hpp" #include "caf/intrusive_ptr.hpp"
#include "caf/node_id.hpp" #include "caf/node_id.hpp"
......
...@@ -12,17 +12,18 @@ ...@@ -12,17 +12,18 @@
#include "caf/error.hpp" #include "caf/error.hpp"
#include "caf/execution_unit.hpp" #include "caf/execution_unit.hpp"
#include "caf/exit_reason.hpp" #include "caf/exit_reason.hpp"
#include "caf/fwd.hpp"
#include "caf/message_priority.hpp"
namespace caf { namespace caf {
class abstract_actor; /// @relates attachable
using attachable_ptr = std::unique_ptr<attachable>;
/// Callback utility class. /// Callback utility class.
class CAF_CORE_EXPORT attachable { class CAF_CORE_EXPORT attachable {
public: public:
attachable() = default; // -- member types -----------------------------------------------------------
attachable(const attachable&) = delete;
attachable& operator=(const attachable&) = delete;
/// Represents a pointer to a value with its subtype as type ID number. /// Represents a pointer to a value with its subtype as type ID number.
struct token { struct token {
...@@ -52,8 +53,16 @@ public: ...@@ -52,8 +53,16 @@ public:
token(size_t typenr, const void* vptr); token(size_t typenr, const void* vptr);
}; };
// -- constructors and destructors -------------------------------------------
attachable() = default;
attachable(const attachable&) = delete;
attachable& operator=(const attachable&) = delete;
virtual ~attachable(); virtual ~attachable();
// -- interface for the actor ------------------------------------------------
/// Executed if the actor finished execution with given `reason`. /// Executed if the actor finished execution with given `reason`.
/// The default implementation does nothing. /// The default implementation does nothing.
/// @warning `host` can be `nullptr` /// @warning `host` can be `nullptr`
...@@ -68,10 +77,21 @@ public: ...@@ -68,10 +77,21 @@ public:
return matches(token{T::token_type, &what}); return matches(token{T::token_type, &what});
} }
std::unique_ptr<attachable> next; // -- factory functions ------------------------------------------------------
};
/// @relates attachable static attachable_ptr
using attachable_ptr = std::unique_ptr<attachable>; make_monitor(actor_addr observed, actor_addr observer,
message_priority prio = message_priority::normal);
static attachable_ptr make_link(actor_addr observed, actor_addr observer);
static attachable_ptr make_stream_aborter(actor_addr observed,
actor_addr observer,
uint64_t sink_flow_id);
// -- member variables -------------------------------------------------------
attachable_ptr next;
};
} // namespace caf } // namespace caf
// This file is part of CAF, the C++ Actor Framework. See the file LICENSE in
// the main distribution directory for license terms and copyright or visit
// https://github.com/actor-framework/actor-framework/blob/master/LICENSE.
#pragma once
#include "caf/actor_control_block.hpp"
#include "caf/flow/observer.hpp"
#include "caf/flow/op/hot.hpp"
#include "caf/flow/subscription.hpp"
#include <cstddef>
#include <cstdint>
#include <deque>
namespace caf::detail {
class stream_bridge_sub : public flow::subscription::impl_base {
public:
stream_bridge_sub(scheduled_actor* self, strong_actor_ptr src,
flow::observer<async::batch> out, uint64_t snk_flow_id,
size_t max_in_flight, size_t request_threshold)
: self_(self),
src_(std::move(src)),
out_(std::move(out)),
snk_flow_id_(snk_flow_id),
max_in_flight_(max_in_flight),
request_threshold_(request_threshold) {
// nop
}
// -- callbacks for the actor ------------------------------------------------
void ack(uint64_t src_flow_id, uint32_t max_items_per_batch);
void drop();
void drop(const error& reason);
void push(const async::batch& input);
void push();
// -- implementation of subscription -----------------------------------------
bool disposed() const noexcept override;
void dispose() override;
void request(size_t n) override;
private:
bool initialized() const noexcept {
return src_flow_id_ != 0;
}
void do_abort(const error& reason);
void do_check_credit();
scheduled_actor* self_;
strong_actor_ptr src_;
flow::observer<async::batch> out_;
uint64_t src_flow_id_ = 0;
uint64_t snk_flow_id_;
size_t max_in_flight_batches_ = 0;
size_t in_flight_batches_ = 0;
size_t low_batches_threshold_ = 0;
size_t demand_ = 0;
std::deque<async::batch> buf_;
size_t max_in_flight_;
size_t request_threshold_;
};
using stream_bridge_sub_ptr = intrusive_ptr<stream_bridge_sub>;
class stream_bridge : public flow::op::hot<async::batch> {
public:
using super = flow::op::hot<async::batch>;
explicit stream_bridge(scheduled_actor* self, strong_actor_ptr src,
uint64_t stream_id, size_t buf_capacity,
size_t request_threshold);
disposable subscribe(flow::observer<async::batch> out) override;
private:
scheduled_actor* self_ptr();
strong_actor_ptr src_;
uint64_t stream_id_;
size_t buf_capacity_;
size_t request_threshold_;
};
} // namespace caf::detail
...@@ -125,8 +125,8 @@ struct assert_scheduled_actor_hdr { ...@@ -125,8 +125,8 @@ struct assert_scheduled_actor_hdr {
"include 'caf/scheduled_actor/flow.hpp' for this method"); "include 'caf/scheduled_actor/flow.hpp' for this method");
}; };
template <class T> template <class T, class V = T>
using assert_scheduled_actor_hdr_t using assert_scheduled_actor_hdr_t
= std::enable_if_t<assert_scheduled_actor_hdr<T>::value, T>; = std::enable_if_t<assert_scheduled_actor_hdr<T>::value, V>;
} // namespace caf::flow } // namespace caf::flow
...@@ -4,8 +4,8 @@ ...@@ -4,8 +4,8 @@
#pragma once #pragma once
#include "caf/flow/coordinator.hpp" #include "caf/flow/fwd.hpp"
#include "caf/flow/observable.hpp" #include "caf/flow/observable_decl.hpp"
#include "caf/flow/op/mcast.hpp" #include "caf/flow/op/mcast.hpp"
#include "caf/intrusive_ptr.hpp" #include "caf/intrusive_ptr.hpp"
#include "caf/make_counted.hpp" #include "caf/make_counted.hpp"
...@@ -17,14 +17,26 @@ namespace caf::flow { ...@@ -17,14 +17,26 @@ namespace caf::flow {
template <class T> template <class T>
class item_publisher { class item_publisher {
public: public:
using impl_ptr = intrusive_ptr<op::mcast<T>>;
explicit item_publisher(coordinator* ctx) { explicit item_publisher(coordinator* ctx) {
pimpl_ = make_counted<op::mcast<T>>(ctx); pimpl_ = make_counted<op::mcast<T>>(ctx);
} }
item_publisher(item_publisher&&) = default; explicit item_publisher(impl_ptr ptr) noexcept : pimpl_(std::move(ptr)) {
item_publisher& operator=(item_publisher&&) = default; // nop
}
item_publisher(item_publisher&&) noexcept = default;
item_publisher& operator=(item_publisher&&) noexcept = default;
item_publisher(const item_publisher&) = delete;
item_publisher& operator=(const item_publisher&) = delete;
~item_publisher() { ~item_publisher() {
if (pimpl_)
pimpl_->close(); pimpl_->close();
} }
...@@ -87,7 +99,7 @@ public: ...@@ -87,7 +99,7 @@ public:
} }
private: private:
intrusive_ptr<op::mcast<T>> pimpl_; impl_ptr pimpl_;
}; };
} // namespace caf::flow } // namespace caf::flow
...@@ -208,6 +208,12 @@ public: ...@@ -208,6 +208,12 @@ public:
return fn(std::move(*this)); return fn(std::move(*this));
} }
// -- batching ---------------------------------------------------------------
/// Like @c buffer, but wraps the collected items into type-erased batches.
observable<async::batch> collect_batches(timespan max_delay,
size_t max_items);
// -- observing -------------------------------------------------------------- // -- observing --------------------------------------------------------------
/// Observes items from this observable on another @ref coordinator. /// Observes items from this observable on another @ref coordinator.
...@@ -284,9 +290,9 @@ private: ...@@ -284,9 +290,9 @@ private:
/// Convenience function for creating an @ref observable from a concrete /// Convenience function for creating an @ref observable from a concrete
/// operator type. /// operator type.
/// @relates observable /// @relates observable
template <class Operator, class... Ts> template <class Operator, class CoordinatorType, class... Ts>
observable<typename Operator::output_type> observable<typename Operator::output_type>
make_observable(coordinator* ctx, Ts&&... xs) { make_observable(CoordinatorType* ctx, Ts&&... xs) {
using out_t = typename Operator::output_type; using out_t = typename Operator::output_type;
using ptr_t = intrusive_ptr<op::base<out_t>>; using ptr_t = intrusive_ptr<op::base<out_t>>;
ptr_t ptr{new Operator(ctx, std::forward<Ts>(xs)...), false}; ptr_t ptr{new Operator(ctx, std::forward<Ts>(xs)...), false};
......
...@@ -84,9 +84,9 @@ public: ...@@ -84,9 +84,9 @@ public:
void fwd_on_subscribe(buffer_input_t, subscription sub) { void fwd_on_subscribe(buffer_input_t, subscription sub) {
if (!value_sub_ && out_) { if (!value_sub_ && out_) {
value_sub_ = std::move(sub); value_sub_ = std::move(sub);
if (pending_demand_ > 0) { if (demand_ > 0) {
value_sub_.request(pending_demand_); in_flight_ += max_buf_size_;
pending_demand_ = 0; value_sub_.request(max_buf_size_);
} }
} else { } else {
sub.dispose(); sub.dispose();
...@@ -94,7 +94,8 @@ public: ...@@ -94,7 +94,8 @@ public:
} }
void fwd_on_complete(buffer_input_t) { void fwd_on_complete(buffer_input_t) {
CAF_ASSERT(value_sub_.valid()); if (!value_sub_.valid())
return;
CAF_ASSERT(out_.valid()); CAF_ASSERT(out_.valid());
value_sub_ = nullptr; value_sub_ = nullptr;
if (!buf_.empty()) if (!buf_.empty())
...@@ -113,6 +114,8 @@ public: ...@@ -113,6 +114,8 @@ public:
} }
void fwd_on_next(buffer_input_t, const input_type& item) { void fwd_on_next(buffer_input_t, const input_type& item) {
CAF_ASSERT(in_flight_ > 0);
--in_flight_;
buf_.push_back(item); buf_.push_back(item);
if (buf_.size() == max_buf_size_) if (buf_.size() == max_buf_size_)
do_emit(); do_emit();
...@@ -163,17 +166,27 @@ public: ...@@ -163,17 +166,27 @@ public:
void request(size_t n) override { void request(size_t n) override {
CAF_ASSERT(out_.valid()); CAF_ASSERT(out_.valid());
if (value_sub_) demand_ += n;
value_sub_.request(n); if (value_sub_ && pending() == 0) {
else in_flight_ = max_buf_size_;
pending_demand_ += n; value_sub_.request(max_buf_size_);
}
} }
private: private:
size_t pending() const noexcept {
return buf_.size() + in_flight_;
}
void do_emit() { void do_emit() {
Trait f; Trait f;
out_.on_next(f(buf_)); out_.on_next(f(buf_));
auto buffered = buf_.size();
buf_.clear(); buf_.clear();
if (value_sub_ && buffered > 0) {
in_flight_ += buffered;
value_sub_.request(buffered);
}
} }
void do_dispose() { void do_dispose() {
...@@ -210,6 +223,9 @@ private: ...@@ -210,6 +223,9 @@ private:
/// Stores the maximum buffer size before forcing a batch. /// Stores the maximum buffer size before forcing a batch.
size_t max_buf_size_; size_t max_buf_size_;
/// Keeps track of how many items we have already requested.
size_t in_flight_ = 0;
/// Stores the elements until we can emit them. /// Stores the elements until we can emit them.
std::vector<input_type> buf_; std::vector<input_type> buf_;
...@@ -222,8 +238,8 @@ private: ...@@ -222,8 +238,8 @@ private:
/// Our subscription for the control tokens. /// Our subscription for the control tokens.
subscription control_sub_; subscription control_sub_;
/// /// Demand signaled by the observer.
size_t pending_demand_ = 0; size_t demand_ = 0;
}; };
template <class Trait> template <class Trait>
......
...@@ -86,6 +86,7 @@ class actor_proxy; ...@@ -86,6 +86,7 @@ class actor_proxy;
class actor_registry; class actor_registry;
class actor_system; class actor_system;
class actor_system_config; class actor_system_config;
class attachable;
class behavior; class behavior;
class binary_deserializer; class binary_deserializer;
class binary_serializer; class binary_serializer;
...@@ -125,6 +126,7 @@ class scoped_actor; ...@@ -125,6 +126,7 @@ class scoped_actor;
class serializer; class serializer;
class skip_t; class skip_t;
class skippable_result; class skippable_result;
class stream;
class tracing_data; class tracing_data;
class tracing_data_factory; class tracing_data_factory;
class type_id_list; class type_id_list;
...@@ -148,6 +150,13 @@ struct invalid_actor_t; ...@@ -148,6 +150,13 @@ struct invalid_actor_t;
struct node_down_msg; struct node_down_msg;
struct none_t; struct none_t;
struct prohibit_top_level_spawn_marker; struct prohibit_top_level_spawn_marker;
struct stream_abort_msg;
struct stream_ack_msg;
struct stream_batch_msg;
struct stream_cancel_msg;
struct stream_close_msg;
struct stream_demand_msg;
struct stream_open_msg;
struct unit_t; struct unit_t;
// -- free template functions -------------------------------------------------- // -- free template functions --------------------------------------------------
...@@ -334,6 +343,8 @@ class dynamic_message_data; ...@@ -334,6 +343,8 @@ class dynamic_message_data;
class group_manager; class group_manager;
class message_data; class message_data;
class private_thread; class private_thread;
class stream_bridge;
class stream_bridge_sub;
struct meta_object; struct meta_object;
......
...@@ -419,7 +419,10 @@ public: ...@@ -419,7 +419,10 @@ public:
bool cleanup(error&& fail_state, execution_unit* host) override; bool cleanup(error&& fail_state, execution_unit* host) override;
message_id new_request_id(message_priority mp); message_id new_request_id(message_priority mp) noexcept;
/// Returns a 64-bit ID that is unique on this actor.
uint64_t new_u64_id() noexcept;
template <class T> template <class T>
void respond(T& x) { void respond(T& x) {
......
...@@ -17,13 +17,18 @@ ...@@ -17,13 +17,18 @@
#include "caf/action.hpp" #include "caf/action.hpp"
#include "caf/actor_traits.hpp" #include "caf/actor_traits.hpp"
#include "caf/async/fwd.hpp"
#include "caf/cow_string.hpp"
#include "caf/detail/behavior_stack.hpp" #include "caf/detail/behavior_stack.hpp"
#include "caf/detail/core_export.hpp" #include "caf/detail/core_export.hpp"
#include "caf/detail/stream_bridge.hpp"
#include "caf/disposable.hpp" #include "caf/disposable.hpp"
#include "caf/error.hpp" #include "caf/error.hpp"
#include "caf/extend.hpp" #include "caf/extend.hpp"
#include "caf/flow/coordinator.hpp" #include "caf/flow/coordinator.hpp"
#include "caf/flow/fwd.hpp" #include "caf/flow/fwd.hpp"
#include "caf/flow/item_publisher.hpp"
#include "caf/flow/observer.hpp"
#include "caf/fwd.hpp" #include "caf/fwd.hpp"
#include "caf/intrusive/drr_cached_queue.hpp" #include "caf/intrusive/drr_cached_queue.hpp"
#include "caf/intrusive/drr_queue.hpp" #include "caf/intrusive/drr_queue.hpp"
...@@ -76,6 +81,10 @@ class CAF_CORE_EXPORT scheduled_actor : public local_actor, ...@@ -76,6 +81,10 @@ class CAF_CORE_EXPORT scheduled_actor : public local_actor,
public: public:
// -- friends ---------------------------------------------------------------- // -- friends ----------------------------------------------------------------
friend class detail::stream_bridge;
friend class detail::stream_bridge_sub;
template <class, class> template <class, class>
friend class response_handle; friend class response_handle;
...@@ -128,6 +137,13 @@ public: ...@@ -128,6 +137,13 @@ public:
urgent_queue, normal_queue>; urgent_queue, normal_queue>;
}; };
using batch_op_ptr = intrusive_ptr<flow::op::base<async::batch>>;
struct stream_source_state {
batch_op_ptr obs;
size_t max_items_per_batch;
};
static constexpr size_t urgent_queue_index = 0; static constexpr size_t urgent_queue_index = 0;
static constexpr size_t normal_queue_index = 1; static constexpr size_t normal_queue_index = 1;
...@@ -161,6 +177,19 @@ public: ...@@ -161,6 +177,19 @@ public:
using exception_handler = std::function<error(pointer, std::exception_ptr&)>; using exception_handler = std::function<error(pointer, std::exception_ptr&)>;
#endif // CAF_ENABLE_EXCEPTIONS #endif // CAF_ENABLE_EXCEPTIONS
using batch_publisher = flow::item_publisher<async::batch>;
class batch_forwarder : public ref_counted {
public:
~batch_forwarder() override;
virtual void cancel() = 0;
virtual void request(size_t num_items) = 0;
};
using batch_forwarder_ptr = intrusive_ptr<batch_forwarder>;
// -- static helper functions ------------------------------------------------ // -- static helper functions ------------------------------------------------
static void default_error_handler(pointer ptr, error& x); static void default_error_handler(pointer ptr, error& x);
...@@ -439,6 +468,56 @@ public: ...@@ -439,6 +468,56 @@ public:
void watch(disposable what) override; void watch(disposable what) override;
template <class Observable>
flow::assert_scheduled_actor_hdr_t<Observable, stream>
to_stream(std::string name, timespan max_delay, size_t max_items_per_batch,
Observable&&);
template <class Observable>
flow::assert_scheduled_actor_hdr_t<Observable, stream>
to_stream(cow_string name, timespan max_delay, size_t max_items_per_batch,
Observable&&);
struct to_stream_t {
scheduled_actor* self;
cow_string name;
timespan max_delay;
size_t max_items_per_batch;
template <class Observable>
auto operator()(Observable&& what) {
return self->to_stream(name, max_delay, max_items_per_batch,
std::forward<Observable>(what));
}
};
/// Returns a function object for passing it to @c compose.
to_stream_t to_stream(std::string name, timespan max_delay,
size_t max_items_per_batch) {
return {this, cow_string{std::move(name)}, max_delay, max_items_per_batch};
}
to_stream_t to_stream(cow_string name, timespan max_delay,
size_t max_items_per_batch) {
return {this, std::move(name), max_delay, max_items_per_batch};
}
/// Lifts a stream into an @ref observable.
/// @param what The input stream.
/// @param buf_capacity Upper bound for caching inputs from the stream.
/// @param demand_threshold Minimal free buffer capacity before signaling
/// demand upstream.
/// @note Both @p buf_capacity and @p demand_threshold are considered hints.
/// The actor may increase (or decrease) the effective settings
/// depending on the amount of messages per batch or other factors.
template <class T>
flow::assert_scheduled_actor_hdr_t<flow::observable<T>>
observe_as(stream what, size_t buf_capacity, size_t demand_threshold);
/// Deregisters a local stream. After calling this function, other actors can
/// no longer access the flow that has been attached to the stream. Current
/// flows remain unaffected.
void deregister_stream(uint64_t stream_id);
/// @cond PRIVATE /// @cond PRIVATE
// -- utility functions for invoking default handler ------------------------- // -- utility functions for invoking default handler -------------------------
...@@ -518,7 +597,8 @@ public: ...@@ -518,7 +597,8 @@ public:
/// @private /// @private
bool alive() const noexcept { bool alive() const noexcept {
return !bhvr_stack_.empty() || !awaited_responses_.empty() return !bhvr_stack_.empty() || !awaited_responses_.empty()
|| !multiplexed_responses_.empty() || !watched_disposables_.empty(); || !multiplexed_responses_.empty() || !watched_disposables_.empty()
|| !stream_sources_.empty();
} }
/// Runs all pending actions. /// Runs all pending actions.
...@@ -610,12 +690,46 @@ private: ...@@ -610,12 +690,46 @@ private:
/// Removes any watched object that became disposed since the last update. /// Removes any watched object that became disposed since the last update.
void update_watched_disposables(); void update_watched_disposables();
/// Implementation detail for observe_as.
flow::observable<async::batch> do_observe(stream what, size_t buf_capacity,
size_t request_threshold);
/// Implementation detail for to_stream.
stream to_stream_impl(cow_string name, batch_op_ptr batch_op,
type_id_t item_type, size_t max_items_per_batch);
/// Registers a stream bridge at the actor (callback for
/// detail::stream_bridge).
void register_flow_state(uint64_t local_id,
detail::stream_bridge_sub_ptr sub);
/// Drops the state for a stream bridge (callback for
/// detail::stream_bridge_sub).
void drop_flow_state(uint64_t local_id);
/// Tries to emit more items on a stream bridge (callback for
/// detail::stream_bridge_sub).
void try_push_stream(uint64_t local_id);
/// Stores actions that the actor executes after processing the current /// Stores actions that the actor executes after processing the current
/// message. /// message.
std::vector<action> actions_; std::vector<action> actions_;
/// Stores resources that block the actor from terminating. /// Stores ongoing activities such as flows that block the actor from
/// terminating.
std::vector<disposable> watched_disposables_; std::vector<disposable> watched_disposables_;
/// Stores open streams that other actors may access. An actor is considered
/// alive as long as it has open streams.
std::unordered_map<uint64_t, stream_source_state> stream_sources_;
/// Maps the ID of outgoing streams to local forwarder objects to allow the
/// actor to signal demand from the receiver to the flow.
std::unordered_map<uint64_t, batch_forwarder_ptr> stream_subs_;
/// Maps the ID of incoming stream batches to local state that allows the
/// actor to push received batches into the local flow.
std::unordered_map<uint64_t, detail::stream_bridge_sub_ptr> stream_bridges_;
}; };
} // namespace caf } // namespace caf
...@@ -10,9 +10,11 @@ ...@@ -10,9 +10,11 @@
#include "caf/flow/observable.hpp" #include "caf/flow/observable.hpp"
#include "caf/flow/observable_builder.hpp" #include "caf/flow/observable_builder.hpp"
#include "caf/flow/observer.hpp" #include "caf/flow/observer.hpp"
#include "caf/flow/op/buffer.hpp"
#include "caf/flow/op/cell.hpp" #include "caf/flow/op/cell.hpp"
#include "caf/flow/single.hpp" #include "caf/flow/single.hpp"
#include "caf/scheduled_actor.hpp" #include "caf/scheduled_actor.hpp"
#include "caf/stream.hpp"
namespace caf::flow { namespace caf::flow {
...@@ -23,6 +25,48 @@ struct has_impl_include<scheduled_actor> { ...@@ -23,6 +25,48 @@ struct has_impl_include<scheduled_actor> {
} // namespace caf::flow } // namespace caf::flow
namespace caf::detail {
template <class T>
class unbatch {
public:
using input_type = async::batch;
using output_type = T;
template <class Next, class... Steps>
bool on_next(const async::batch& xs, Next& next, Steps&... steps) {
for (const auto& item : xs.template items<T>())
if (!next.on_next(item, steps...))
return false;
return true;
}
template <class Next, class... Steps>
void on_complete(Next& next, Steps&... steps) {
next.on_complete(steps...);
}
template <class Next, class... Steps>
void on_error(const error& what, Next& next, Steps&... steps) {
next.on_error(what, steps...);
}
};
template <class T>
struct batching_trait {
static constexpr bool skip_empty = true;
using input_type = T;
using output_type = async::batch;
using select_token_type = int64_t;
output_type operator()(const std::vector<input_type>& xs) {
return async::make_batch(make_span(xs));
}
};
} // namespace caf::detail
namespace caf { namespace caf {
template <class T, class Policy> template <class T, class Policy>
...@@ -41,4 +85,38 @@ flow::single<T> scheduled_actor::single_from_response_impl(Policy& policy) { ...@@ -41,4 +85,38 @@ flow::single<T> scheduled_actor::single_from_response_impl(Policy& policy) {
return flow::single<T>{std::move(cell)}; return flow::single<T>{std::move(cell)};
} }
template <class Observable>
flow::assert_scheduled_actor_hdr_t<Observable, stream>
scheduled_actor::to_stream(std::string name, timespan max_delay,
size_t max_items_per_batch, Observable&& obs) {
return to_stream(cow_string{std::move(name)}, max_delay, max_items_per_batch,
std::forward<Observable>(obs));
}
template <class Observable>
flow::assert_scheduled_actor_hdr_t<Observable, stream>
scheduled_actor::to_stream(cow_string name, timespan max_delay,
size_t max_items_per_batch, Observable&& obs) {
using obs_t = std::decay_t<Observable>;
using val_t = typename obs_t::output_type;
using trait_t = detail::batching_trait<val_t>;
using impl_t = flow::op::buffer<trait_t>;
auto batch_op = make_counted<impl_t>(
this, max_items_per_batch, std::forward<Observable>(obs).as_observable(),
flow::make_observable<flow::op::interval>(this, max_delay, max_delay));
return to_stream_impl(std::move(name), std::move(batch_op), type_id_v<val_t>,
max_items_per_batch);
}
template <class T>
flow::assert_scheduled_actor_hdr_t<flow::observable<T>>
scheduled_actor::observe_as(stream what, size_t buf_capacity,
size_t demand_threshold) {
if (what.template has_element_type<T>())
return do_observe(what, buf_capacity, demand_threshold)
.transform(detail::unbatch<T>{})
.as_observable();
return make_observable().fail<T>(make_error(sec::type_clash));
}
} // namespace caf } // namespace caf
...@@ -83,26 +83,10 @@ enum class sec : uint8_t { ...@@ -83,26 +83,10 @@ enum class sec : uint8_t {
runtime_error, runtime_error,
/// Linking to a remote actor failed because actor no longer exists. /// Linking to a remote actor failed because actor no longer exists.
remote_linking_failed, remote_linking_failed,
/// Adding an upstream to a stream failed. /// Subscribing to a stream failed because it was invalid.
cannot_add_upstream = 30, invalid_stream = 30,
/// Adding an upstream to a stream failed because it already exists. /// Subscribing to a stream failed because it can only be subscribed to once.
upstream_already_exists, cannot_resubscribe_stream,
/// Unable to process upstream messages because upstream is invalid.
invalid_upstream,
/// Adding a downstream to a stream failed.
cannot_add_downstream,
/// Adding a downstream to a stream failed because it already exists.
downstream_already_exists,
/// Unable to process downstream messages because downstream is invalid.
invalid_downstream = 35,
/// Cannot start streaming without next stage.
no_downstream_stages_defined,
/// Actor failed to initialize state after receiving a stream handshake.
stream_init_failed,
/// Unable to process a stream since due to missing state.
invalid_stream_state,
/// Stream aborted due to unexpected error.
unhandled_stream_error,
/// A function view was called without assigning an actor first. /// A function view was called without assigning an actor first.
bad_function_call = 40, bad_function_call = 40,
/// Feature is disabled in the actor system config. /// Feature is disabled in the actor system config.
...@@ -175,7 +159,7 @@ enum class sec : uint8_t { ...@@ -175,7 +159,7 @@ enum class sec : uint8_t {
disposed, disposed,
/// Failed to open a resource. /// Failed to open a resource.
cannot_open_resource, cannot_open_resource,
/// Received malformed data on a network socket. /// Received malformed data.
protocol_error, protocol_error,
/// Encountered faulty logic in the program. /// Encountered faulty logic in the program.
logic_error, logic_error,
......
...@@ -5,10 +5,11 @@ ...@@ -5,10 +5,11 @@
#pragma once #pragma once
#include "caf/actor_control_block.hpp" #include "caf/actor_control_block.hpp"
#include "caf/async/batch.hpp"
#include "caf/cow_string.hpp" #include "caf/cow_string.hpp"
#include "caf/detail/comparable.hpp" #include "caf/detail/comparable.hpp"
#include "caf/detail/core_export.hpp" #include "caf/detail/core_export.hpp"
#include "caf/type_id.hpp" #include "caf/fwd.hpp"
#include <cstddef> #include <cstddef>
#include <cstdint> #include <cstdint>
...@@ -16,6 +17,10 @@ ...@@ -16,6 +17,10 @@
namespace caf { namespace caf {
/// Provides access to a potentially unbound sequence of items emitted by an
/// actor. Each stream is uniquely identified by the address of the hosting
/// actor plus an integer value. Further, streams have human-readable names
/// attached to them in order to make help with observability and logging.
class CAF_CORE_EXPORT stream : private detail::comparable<stream> { class CAF_CORE_EXPORT stream : private detail::comparable<stream> {
public: public:
// -- constructors, destructors, and assignment operators -------------------- // -- constructors, destructors, and assignment operators --------------------
...@@ -30,31 +35,42 @@ public: ...@@ -30,31 +35,42 @@ public:
stream& operator=(const stream&) noexcept = default; stream& operator=(const stream&) noexcept = default;
stream(caf::strong_actor_ptr source, type_id_t type, std::string name, stream(strong_actor_ptr source, type_id_t type, std::string name, uint64_t id)
uint64_t id) : source_(std::move(source)), type_(type), name_(std::move(name)), id_(id) {
// nop
}
stream(strong_actor_ptr source, type_id_t type, cow_string name, uint64_t id)
: source_(std::move(source)), type_(type), name_(std::move(name)), id_(id) { : source_(std::move(source)), type_(type), name_(std::move(name)), id_(id) {
// nop // nop
} }
// -- properties ------------------------------------------------------------- // -- properties -------------------------------------------------------------
/// Checks whether this stream emits elements of type @c T.
template <class T> template <class T>
bool has_element_type() const noexcept { bool has_element_type() const noexcept {
return type_id_v<T> == type_; return type_id_v<T> == type_;
} }
const caf::strong_actor_ptr& source() { /// Queries the source of this stream. Default-constructed streams return a
/// @c null pointer.
const strong_actor_ptr& source() {
return source_; return source_;
} }
/// Returns the type ID of the items emitted by the source.
type_id_t type() const noexcept { type_id_t type() const noexcept {
return type_; return type_;
} }
/// Returns the human-readable name for this stream, as announced by the
/// source.
const std::string& name() const noexcept { const std::string& name() const noexcept {
return name_.str(); return name_.str();
} }
/// Returns the source-specific identifier for this stream.
uint64_t id() const noexcept { uint64_t id() const noexcept {
return id_; return id_;
} }
...@@ -74,7 +90,7 @@ public: ...@@ -74,7 +90,7 @@ public:
} }
private: private:
caf::strong_actor_ptr source_; strong_actor_ptr source_;
type_id_t type_ = invalid_type_id; type_id_t type_ = invalid_type_id;
cow_string name_; cow_string name_;
uint64_t id_ = 0; uint64_t id_ = 0;
......
...@@ -9,9 +9,11 @@ ...@@ -9,9 +9,11 @@
#include <vector> #include <vector>
#include "caf/actor_addr.hpp" #include "caf/actor_addr.hpp"
#include "caf/async/batch.hpp"
#include "caf/deep_to_string.hpp" #include "caf/deep_to_string.hpp"
#include "caf/fwd.hpp" #include "caf/fwd.hpp"
#include "caf/group.hpp" #include "caf/group.hpp"
#include "caf/type_id.hpp"
namespace caf { namespace caf {
...@@ -105,4 +107,128 @@ bool inspect(Inspector& f, node_down_msg& x) { ...@@ -105,4 +107,128 @@ bool inspect(Inspector& f, node_down_msg& x) {
f.field("reason", x.reason)); f.field("reason", x.reason));
} }
/// Asks a source to add another sink.
/// @note The sender is always the sink.
struct stream_open_msg {
/// The ID of the requested stream.
uint64_t id;
/// A handle to the new sink.
strong_actor_ptr sink;
/// The ID of the flow at the sink.
uint64_t sink_flow_id;
};
/// @relates stream_open_msg
template <class Inspector>
bool inspect(Inspector& f, stream_open_msg& msg) {
return f.object(msg).fields(f.field("id", msg.id), f.field("sink", msg.sink),
f.field("sink-flow-id", msg.sink_flow_id));
}
/// Asks the source for more data.
/// @note The sender is always the sink.
struct stream_demand_msg {
/// The ID of the flow at the source.
uint64_t source_flow_id;
/// Additional demand from the sink.
uint32_t demand;
};
/// @relates stream_demand_msg
template <class Inspector>
bool inspect(Inspector& f, stream_demand_msg& msg) {
return f.object(msg).fields(f.field("source-flow-id", msg.source_flow_id),
f.field("demand", msg.demand));
}
/// Informs the source that the sender is no longer interest in receiving
/// items from this stream.
/// @note The sender is always the sink.
struct stream_cancel_msg {
/// The ID of the flow at the source.
uint64_t source_flow_id;
};
/// @relates stream_cancel_msg
template <class Inspector>
bool inspect(Inspector& f, stream_cancel_msg& msg) {
return f.object(msg).fields(f.field("source-flow-id", msg.source_flow_id));
}
/// Informs the sink that the source has added it to the flow.
/// @note The sender is always the source.
struct stream_ack_msg {
/// Pointer to the source actor.
strong_actor_ptr source;
/// The ID of the flow at the sink.
uint64_t sink_flow_id;
/// The ID of the flow at the source.
uint64_t source_flow_id;
/// Maximum amounts of items per batch.
uint32_t max_items_per_batch;
};
/// @relates stream_ack_msg
template <class Inspector>
bool inspect(Inspector& f, stream_ack_msg& msg) {
return f.object(msg).fields(
f.field("source", msg.source), f.field("sink-flow-id", msg.sink_flow_id),
f.field("source-flow-id", msg.source_flow_id),
f.field("max-items-per-batch", msg.max_items_per_batch));
}
/// Transfers items from a source to a sink.
/// @note The sender is always the source.
struct stream_batch_msg {
/// The ID of the flow at the sink.
uint64_t sink_flow_id;
/// Contains the new items from the source.
async::batch content;
};
/// @relates stream_batch_msg
template <class Inspector>
bool inspect(Inspector& f, stream_batch_msg& msg) {
return f.object(msg).fields(f.field("sink-flow-id", msg.sink_flow_id),
f.field("content", msg.content));
}
/// Informs the sink that a stream has reached the end.
/// @note The sender is always the source.
struct stream_close_msg {
/// The ID of the flow at the sink.
uint64_t sink_flow_id;
};
/// @relates stream_close_msg
template <class Inspector>
bool inspect(Inspector& f, stream_close_msg& msg) {
return f.object(msg).fields(f.field("sink-flow-id", msg.sink_flow_id));
}
/// Informs the sink that a stream has been aborted due to an unrecoverable
/// error.
/// @note The sender is always the source.
struct stream_abort_msg {
/// The ID of the flow at the sink.
uint64_t sink_flow_id;
/// Contains details about the abort reason.
error reason;
};
/// @relates stream_abort_msg
template <class Inspector>
bool inspect(Inspector& f, stream_abort_msg& msg) {
return f.object(msg).fields(f.field("sink-flow-id", msg.sink_flow_id),
f.field("reason", msg.reason));
}
} // namespace caf } // namespace caf
...@@ -373,7 +373,7 @@ public: ...@@ -373,7 +373,7 @@ public:
CAF_BEGIN_TYPE_ID_BLOCK(core_module, 0) CAF_BEGIN_TYPE_ID_BLOCK(core_module, 0)
// -- C types // -- C types ----------------------------------------------------------------
CAF_ADD_TYPE_ID_FROM_EXPR(core_module, (bool) ) CAF_ADD_TYPE_ID_FROM_EXPR(core_module, (bool) )
CAF_ADD_TYPE_ID_FROM_EXPR(core_module, (double) ) CAF_ADD_TYPE_ID_FROM_EXPR(core_module, (double) )
...@@ -388,14 +388,15 @@ CAF_BEGIN_TYPE_ID_BLOCK(core_module, 0) ...@@ -388,14 +388,15 @@ CAF_BEGIN_TYPE_ID_BLOCK(core_module, 0)
CAF_ADD_TYPE_ID_FROM_EXPR(core_module, (uint64_t)) CAF_ADD_TYPE_ID_FROM_EXPR(core_module, (uint64_t))
CAF_ADD_TYPE_ID_FROM_EXPR(core_module, (uint8_t)) CAF_ADD_TYPE_ID_FROM_EXPR(core_module, (uint8_t))
// -- STL types // -- STL types --------------------------------------------------------------
CAF_ADD_TYPE_ID(core_module, (std::string)) CAF_ADD_TYPE_ID(core_module, (std::string))
CAF_ADD_TYPE_ID(core_module, (std::u16string)) CAF_ADD_TYPE_ID(core_module, (std::u16string))
CAF_ADD_TYPE_ID(core_module, (std::u32string)) CAF_ADD_TYPE_ID(core_module, (std::u32string))
CAF_ADD_TYPE_ID(core_module, (std::set<std::string>) ) CAF_ADD_TYPE_ID(core_module, (std::set<std::string>) )
// -- CAF types // -- CAF types --------------------------------------------------------------
CAF_ADD_TYPE_ID(core_module, (caf::action)) CAF_ADD_TYPE_ID(core_module, (caf::action))
CAF_ADD_TYPE_ID(core_module, (caf::actor)) CAF_ADD_TYPE_ID(core_module, (caf::actor))
CAF_ADD_TYPE_ID(core_module, (caf::actor_addr)) CAF_ADD_TYPE_ID(core_module, (caf::actor_addr))
...@@ -426,6 +427,14 @@ CAF_BEGIN_TYPE_ID_BLOCK(core_module, 0) ...@@ -426,6 +427,14 @@ CAF_BEGIN_TYPE_ID_BLOCK(core_module, 0)
CAF_ADD_TYPE_ID(core_module, (caf::pec)) CAF_ADD_TYPE_ID(core_module, (caf::pec))
CAF_ADD_TYPE_ID(core_module, (caf::sec)) CAF_ADD_TYPE_ID(core_module, (caf::sec))
CAF_ADD_TYPE_ID(core_module, (caf::shared_action_ptr)) CAF_ADD_TYPE_ID(core_module, (caf::shared_action_ptr))
CAF_ADD_TYPE_ID(core_module, (caf::stream))
CAF_ADD_TYPE_ID(core_module, (caf::stream_abort_msg))
CAF_ADD_TYPE_ID(core_module, (caf::stream_ack_msg))
CAF_ADD_TYPE_ID(core_module, (caf::stream_batch_msg))
CAF_ADD_TYPE_ID(core_module, (caf::stream_cancel_msg))
CAF_ADD_TYPE_ID(core_module, (caf::stream_close_msg))
CAF_ADD_TYPE_ID(core_module, (caf::stream_demand_msg))
CAF_ADD_TYPE_ID(core_module, (caf::stream_open_msg))
CAF_ADD_TYPE_ID(core_module, (caf::strong_actor_ptr)) CAF_ADD_TYPE_ID(core_module, (caf::strong_actor_ptr))
CAF_ADD_TYPE_ID(core_module, (caf::timespan)) CAF_ADD_TYPE_ID(core_module, (caf::timespan))
CAF_ADD_TYPE_ID(core_module, (caf::timestamp)) CAF_ADD_TYPE_ID(core_module, (caf::timestamp))
......
...@@ -4,6 +4,10 @@ ...@@ -4,6 +4,10 @@
#include "caf/attachable.hpp" #include "caf/attachable.hpp"
#include "caf/actor_cast.hpp"
#include "caf/default_attachable.hpp"
#include "caf/system_messages.hpp"
namespace caf { namespace caf {
attachable::~attachable() { attachable::~attachable() {
...@@ -30,4 +34,57 @@ bool attachable::matches(const token&) { ...@@ -30,4 +34,57 @@ bool attachable::matches(const token&) {
return false; return false;
} }
attachable_ptr attachable::make_monitor(actor_addr observed,
actor_addr observer,
message_priority prio) {
return default_attachable::make_monitor(std::move(observed),
std::move(observer), prio);
}
attachable_ptr attachable::make_link(actor_addr observed, actor_addr observer) {
return default_attachable::make_link(std::move(observed),
std::move(observer));
}
namespace {
class stream_aborter : public attachable {
public:
stream_aborter(actor_addr observed, actor_addr observer,
uint64_t sink_flow_id)
: observed_(std::move(observed)),
observer_(std::move(observer)),
sink_flow_id_(sink_flow_id) {
// nop
}
void actor_exited(const error& rsn, execution_unit* host) override {
if (auto observer = actor_cast<strong_actor_ptr>(observer_)) {
auto observed = actor_cast<strong_actor_ptr>(observed_);
observer->enqueue(std::move(observed), make_message_id(),
make_message(stream_abort_msg{sink_flow_id_, rsn}),
host);
}
}
private:
/// Holds a weak reference to the observed actor.
actor_addr observed_;
/// Holds a weak reference to the observing actor.
actor_addr observer_;
/// Identifies the aborted flow at the observer.
uint64_t sink_flow_id_;
};
} // namespace
attachable_ptr attachable::make_stream_aborter(actor_addr observed,
actor_addr observer,
uint64_t sink_flow_id) {
return std::make_unique<stream_aborter>(std::move(observed),
std::move(observer), sink_flow_id);
}
} // namespace caf } // namespace caf
...@@ -23,12 +23,13 @@ message make(abstract_actor* self, const error& reason) { ...@@ -23,12 +23,13 @@ message make(abstract_actor* self, const error& reason) {
void default_attachable::actor_exited(const error& rsn, execution_unit* host) { void default_attachable::actor_exited(const error& rsn, execution_unit* host) {
CAF_ASSERT(observed_ != observer_); CAF_ASSERT(observed_ != observer_);
auto factory = type_ == monitor ? &make<down_msg> : &make<exit_msg>; auto factory = type_ == monitor ? &make<down_msg> : &make<exit_msg>;
auto observer = actor_cast<strong_actor_ptr>(observer_);
if (auto observer = actor_cast<strong_actor_ptr>(observer_)) {
auto observed = actor_cast<strong_actor_ptr>(observed_); auto observed = actor_cast<strong_actor_ptr>(observed_);
if (observer)
observer->enqueue(std::move(observed), make_message_id(priority_), observer->enqueue(std::move(observed), make_message_id(priority_),
factory(actor_cast<abstract_actor*>(observed_), rsn), factory(actor_cast<abstract_actor*>(observed_), rsn),
host); host);
}
} }
bool default_attachable::matches(const token& what) { bool default_attachable::matches(const token& what) {
......
// This file is part of CAF, the C++ Actor Framework. See the file LICENSE in
// the main distribution directory for license terms and copyright or visit
// https://github.com/actor-framework/actor-framework/blob/master/LICENSE.
#include "caf/detail/stream_bridge.hpp"
#include "caf/scheduled_actor.hpp"
namespace caf::detail {
namespace {
/// Configures how many (full) batches the bridge must be able to cache at the
/// very least.
constexpr size_t min_batch_buffering = 5;
/// Configures how many batches we request in one go. This is to avoid sending
/// one demand message for each batch we receive.
constexpr size_t min_batch_request_threshold = 3;
} // namespace
void stream_bridge_sub::ack(uint64_t src_flow_id,
uint32_t max_items_per_batch) {
CAF_LOG_TRACE(CAF_ARG(src_flow_id) << CAF_ARG(max_items_per_batch));
// Sanity checking.
if (max_items_per_batch == 0) {
CAF_LOG_ERROR("stream ACK announced a batch size of 0");
do_abort(make_error(sec::protocol_error));
return;
}
// Update our state. Streams operate on batches, so we translate the
// user-defined bounds on per-item level to a rough equivalent on batches.
// Batches may be "under-full", so this isn't perfect in practice.
src_flow_id_ = src_flow_id;
max_in_flight_batches_ = std::max(min_batch_buffering,
max_in_flight_ / max_items_per_batch);
low_batches_threshold_ = std::max(min_batch_request_threshold,
request_threshold_ / max_items_per_batch);
// Go get some data.
in_flight_batches_ = max_in_flight_batches_;
unsafe_send_as(self_, src_,
stream_demand_msg{src_flow_id_,
static_cast<uint32_t>(in_flight_batches_)});
}
void stream_bridge_sub::drop() {
CAF_LOG_TRACE("");
src_ = nullptr;
out_.on_complete();
out_ = nullptr;
}
void stream_bridge_sub::drop(const error& reason) {
CAF_LOG_TRACE(CAF_ARG(reason));
src_ = nullptr;
out_.on_error(reason);
out_ = nullptr;
}
void stream_bridge_sub::push(const async::batch& input) {
CAF_LOG_TRACE(CAF_ARG2("input.size", input.size()));
// Sanity checking.
if (in_flight_batches_ == 0) {
CAF_LOG_ERROR("source exceeded its allowed credit!");
do_abort(make_error(sec::protocol_error));
return;
}
// Push batch downstream or buffer it.
--in_flight_batches_;
if (demand_ > 0) {
CAF_ASSERT(buf_.empty());
--demand_;
out_.on_next(input);
do_check_credit();
} else {
buf_.push_back(input);
}
}
void stream_bridge_sub::push() {
CAF_LOG_TRACE("");
while (!buf_.empty() && demand_ > 0) {
--demand_;
out_.on_next(buf_.front());
buf_.pop_front();
}
do_check_credit();
}
bool stream_bridge_sub::disposed() const noexcept {
return src_ != nullptr;
}
void stream_bridge_sub::dispose() {
if (!src_)
return;
unsafe_send_as(self_, src_, stream_cancel_msg{src_flow_id_});
auto fn = make_action([self = self_, snk_flow_id = snk_flow_id_] {
self->drop_flow_state(snk_flow_id);
});
self_->delay(std::move(fn));
src_ = nullptr;
}
void stream_bridge_sub::request(size_t n) {
demand_ += n;
if (!buf_.empty()) {
auto fn = make_action([self = self_, snk_flow_id = snk_flow_id_] {
self->try_push_stream(snk_flow_id);
});
self_->delay(std::move(fn));
}
}
void stream_bridge_sub::do_abort(const error& reason) {
auto fn = make_action([self = self_, snk_flow_id = snk_flow_id_] {
self->drop_flow_state(snk_flow_id);
});
self_->delay(std::move(fn));
out_.on_error(reason);
out_ = nullptr;
unsafe_send_as(self_, src_, stream_cancel_msg{src_flow_id_});
src_ = nullptr;
}
void stream_bridge_sub::do_check_credit() {
auto capacity = max_in_flight_batches_ - in_flight_batches_ - buf_.size();
if (capacity >= low_batches_threshold_) {
in_flight_batches_ += capacity;
unsafe_send_as(self_, src_,
stream_demand_msg{src_flow_id_,
static_cast<uint32_t>(capacity)});
}
}
stream_bridge::stream_bridge(scheduled_actor* self, strong_actor_ptr src,
uint64_t stream_id, size_t buf_capacity,
size_t request_threshold)
: super(self),
src_(std::move(src)),
stream_id_(stream_id),
buf_capacity_(buf_capacity),
request_threshold_(request_threshold) {
// nop
}
disposable stream_bridge::subscribe(flow::observer<async::batch> out) {
if (!src_) {
out.on_error(make_error(sec::cannot_resubscribe_stream));
return {};
}
auto self = self_ptr();
auto local_id = self->new_u64_id();
unsafe_send_as(self, src_,
stream_open_msg{stream_id_, self->ctrl(), local_id});
auto sub = make_counted<stream_bridge_sub>(self, std::move(src_), out,
local_id, buf_capacity_,
request_threshold_);
self->register_flow_state(local_id, sub);
out.on_subscribe(flow::subscription{sub});
return sub->as_disposable();
}
scheduled_actor* stream_bridge::self_ptr() {
// This cast is safe, because the stream_bridge may only be constructed with
// a scheduled actor pointer.
return static_cast<scheduled_actor*>(super::ctx());
}
} // namespace caf::detail
...@@ -24,6 +24,7 @@ ...@@ -24,6 +24,7 @@
#include "caf/message.hpp" #include "caf/message.hpp"
#include "caf/message_id.hpp" #include "caf/message_id.hpp"
#include "caf/node_id.hpp" #include "caf/node_id.hpp"
#include "caf/stream.hpp"
#include "caf/system_messages.hpp" #include "caf/system_messages.hpp"
#include "caf/timespan.hpp" #include "caf/timespan.hpp"
#include "caf/timestamp.hpp" #include "caf/timestamp.hpp"
......
...@@ -129,11 +129,16 @@ void local_actor::on_exit() { ...@@ -129,11 +129,16 @@ void local_actor::on_exit() {
// nop // nop
} }
message_id local_actor::new_request_id(message_priority mp) { message_id local_actor::new_request_id(message_priority mp) noexcept {
auto result = ++last_request_id_; auto result = ++last_request_id_;
return mp == message_priority::normal ? result : result.with_high_priority(); return mp == message_priority::normal ? result : result.with_high_priority();
} }
uint64_t local_actor::new_u64_id() noexcept {
auto result = ++last_request_id_;
return result.integer_value();
}
void local_actor::send_exit(const actor_addr& whom, error reason) { void local_actor::send_exit(const actor_addr& whom, error reason) {
send_exit(actor_cast<strong_actor_ptr>(whom), std::move(reason)); send_exit(actor_cast<strong_actor_ptr>(whom), std::move(reason));
} }
......
...@@ -13,7 +13,11 @@ ...@@ -13,7 +13,11 @@
#include "caf/detail/meta_object.hpp" #include "caf/detail/meta_object.hpp"
#include "caf/detail/private_thread.hpp" #include "caf/detail/private_thread.hpp"
#include "caf/detail/sync_request_bouncer.hpp" #include "caf/detail/sync_request_bouncer.hpp"
#include "caf/flow/observable.hpp"
#include "caf/flow/observable_builder.hpp"
#include "caf/flow/op/mcast.hpp"
#include "caf/scheduler/abstract_coordinator.hpp" #include "caf/scheduler/abstract_coordinator.hpp"
#include "caf/stream.hpp"
using namespace std::string_literals; using namespace std::string_literals;
...@@ -108,6 +112,10 @@ error scheduled_actor::default_exception_handler(local_actor* ptr, ...@@ -108,6 +112,10 @@ error scheduled_actor::default_exception_handler(local_actor* ptr,
// -- constructors and destructors --------------------------------------------- // -- constructors and destructors ---------------------------------------------
scheduled_actor::batch_forwarder::~batch_forwarder() {
// nop
}
scheduled_actor::scheduled_actor(actor_config& cfg) scheduled_actor::scheduled_actor(actor_config& cfg)
: super(cfg), : super(cfg),
mailbox_(unit, unit, unit), mailbox_(unit, unit, unit),
...@@ -381,6 +389,83 @@ void scheduled_actor::set_receive_timeout() { ...@@ -381,6 +389,83 @@ void scheduled_actor::set_receive_timeout() {
// -- caf::flow API ------------------------------------------------------------ // -- caf::flow API ------------------------------------------------------------
namespace {
// Forwards batches from a local flow to another actor.
class batch_forwarder_impl : public scheduled_actor::batch_forwarder,
public flow::observer_impl<async::batch> {
public:
batch_forwarder_impl(scheduled_actor* self, actor sink_hdl,
uint64_t sink_flow_id)
: self_(self), sink_hdl_(sink_hdl), sink_flow_id_(sink_flow_id) {
// nop
}
void cancel() override {
if (sub_) {
sub_.dispose();
sink_hdl_ = nullptr;
sub_ = nullptr;
}
}
void request(size_t num_items) override {
if (sub_)
sub_.request(num_items);
}
void ref_coordinated() const noexcept final {
ref();
}
void deref_coordinated() const noexcept final {
deref();
}
bool subscribed() const noexcept {
return sub_.valid();
}
void on_next(const async::batch& content) override {
unsafe_send_as(self_, sink_hdl_, stream_batch_msg{sink_flow_id_, content});
}
void on_error(const error& err) override {
unsafe_send_as(self_, sink_hdl_, stream_abort_msg{sink_flow_id_, err});
sink_hdl_ = nullptr;
sub_ = nullptr;
}
void on_complete() override {
unsafe_send_as(self_, sink_hdl_, stream_close_msg{sink_flow_id_});
sink_hdl_ = nullptr;
sub_ = nullptr;
}
void on_subscribe(flow::subscription sub) override {
if (!sub_ && sink_hdl_)
sub_ = sub;
else
sub.dispose();
}
friend void intrusive_ptr_add_ref(const batch_forwarder_impl* ptr) noexcept {
ptr->ref();
}
friend void intrusive_ptr_release(const batch_forwarder_impl* ptr) noexcept {
ptr->deref();
}
private:
scheduled_actor* self_;
actor sink_hdl_;
uint64_t sink_flow_id_;
flow::subscription sub_;
};
} // namespace
flow::coordinator::steady_time_point scheduled_actor::steady_time() { flow::coordinator::steady_time_point scheduled_actor::steady_time() {
return clock().now(); return clock().now();
} }
...@@ -441,8 +526,11 @@ scheduled_actor::categorize(mailbox_element& x) { ...@@ -441,8 +526,11 @@ scheduled_actor::categorize(mailbox_element& x) {
} }
return message_category::internal; return message_category::internal;
} }
if (auto view = make_typed_message_view<exit_msg>(content)) { if (content.size() != 1)
auto& em = get<0>(view); return message_category::ordinary;
switch (content.type_at(0)) {
case type_id_v<exit_msg>: {
auto& em = content.get_mutable_as<exit_msg>(0);
// make sure to get rid of attachables if they're no longer needed // make sure to get rid of attachables if they're no longer needed
unlink_from(em.source); unlink_from(em.source);
// exit_reason::kill is always fatal // exit_reason::kill is always fatal
...@@ -453,29 +541,95 @@ scheduled_actor::categorize(mailbox_element& x) { ...@@ -453,29 +541,95 @@ scheduled_actor::categorize(mailbox_element& x) {
} }
return message_category::internal; return message_category::internal;
} }
if (auto view = make_typed_message_view<down_msg>(content)) { case type_id_v<down_msg>: {
auto& dm = get<0>(view); auto& dm = content.get_mutable_as<down_msg>(0);
call_handler(down_handler_, this, dm); call_handler(down_handler_, this, dm);
return message_category::internal; return message_category::internal;
} }
if (auto view = make_typed_message_view<action>(content)) { case type_id_v<action>: {
auto ptr = get<0>(view).ptr(); auto ptr = content.get_as<action>(0).ptr();
CAF_ASSERT(ptr != nullptr); CAF_ASSERT(ptr != nullptr);
CAF_LOG_DEBUG("run action"); CAF_LOG_DEBUG("run action");
ptr->run(); ptr->run();
return message_category::internal; return message_category::internal;
} }
if (auto view = make_typed_message_view<node_down_msg>(content)) { case type_id_v<node_down_msg>: {
auto& dm = get<0>(view); auto& dm = content.get_mutable_as<node_down_msg>(0);
call_handler(node_down_handler_, this, dm); call_handler(node_down_handler_, this, dm);
return message_category::internal; return message_category::internal;
} }
if (auto view = make_typed_message_view<error>(content)) { case type_id_v<error>: {
auto& err = get<0>(view); auto& err = content.get_mutable_as<error>(0);
call_handler(error_handler_, this, err); call_handler(error_handler_, this, err);
return message_category::internal; return message_category::internal;
} }
case type_id_v<stream_open_msg>: {
auto& [str_id, ptr, sink_id] = content.get_as<stream_open_msg>(0);
auto sink_hdl = actor_cast<actor>(ptr);
if (auto i = stream_sources_.find(str_id); i != stream_sources_.end()) {
auto fwd = make_counted<batch_forwarder_impl>(this, sink_hdl, sink_id);
auto sub = i->second.obs->subscribe(flow::observer<async::batch>{fwd});
if (fwd->subscribed()) {
auto flow_id = new_u64_id();
stream_subs_.emplace(flow_id, std::move(fwd));
auto mipb = static_cast<uint32_t>(i->second.max_items_per_batch);
unsafe_send_as(this, sink_hdl,
stream_ack_msg{ctrl(), sink_id, flow_id, mipb});
} else {
CAF_LOG_ERROR("failed to subscribe a batch forwarder");
sub.dispose();
}
}
return message_category::internal;
}
case type_id_v<stream_demand_msg>: {
auto [sub_id, new_demand] = content.get_as<stream_demand_msg>(0);
if (auto i = stream_subs_.find(sub_id); i != stream_subs_.end()) {
i->second->request(new_demand);
}
return message_category::internal;
}
case type_id_v<stream_cancel_msg>: {
auto [sub_id] = content.get_as<stream_cancel_msg>(0);
if (auto i = stream_subs_.find(sub_id); i != stream_subs_.end()) {
i->second->cancel();
stream_subs_.erase(i);
}
return message_category::internal;
}
case type_id_v<stream_ack_msg>: {
auto [ptr, sink_id, src_id, mipb] = content.get_as<stream_ack_msg>(0);
if (auto i = stream_bridges_.find(sink_id); i != stream_bridges_.end()) {
i->second->ack(src_id, mipb);
}
return message_category::internal;
}
case type_id_v<stream_batch_msg>: {
const auto& [sink_id, xs] = content.get_as<stream_batch_msg>(0);
if (auto i = stream_bridges_.find(sink_id); i != stream_bridges_.end()) {
i->second->push(xs);
}
return message_category::internal;
}
case type_id_v<stream_close_msg>: {
auto [sink_id] = content.get_as<stream_close_msg>(0);
if (auto i = stream_bridges_.find(sink_id); i != stream_bridges_.end()) {
i->second->drop();
stream_bridges_.erase(i);
}
return message_category::internal;
}
case type_id_v<stream_abort_msg>: {
const auto& [sink_id, reason] = content.get_as<stream_abort_msg>(0);
if (auto i = stream_bridges_.find(sink_id); i != stream_bridges_.end()) {
i->second->drop(reason);
stream_bridges_.erase(i);
}
return message_category::internal;
}
default:
return message_category::ordinary; return message_category::ordinary;
}
} }
invoke_message_result scheduled_actor::consume(mailbox_element& x) { invoke_message_result scheduled_actor::consume(mailbox_element& x) {
...@@ -730,7 +884,31 @@ disposable scheduled_actor::run_delayed(timespan delay, action what) { ...@@ -730,7 +884,31 @@ disposable scheduled_actor::run_delayed(timespan delay, action what) {
strong_actor_ptr{ctrl()}); strong_actor_ptr{ctrl()});
} }
// -- scheduling of caf::flow events ------------------------------------------- // -- caf::flow bindings -------------------------------------------------------
stream scheduled_actor::to_stream_impl(cow_string name, batch_op_ptr batch_op,
type_id_t item_type,
size_t max_items_per_batch) {
CAF_LOG_TRACE(CAF_ARG(name)
<< CAF_ARG2("item_type", query_type_name(item_type)));
auto local_id = new_u64_id();
stream_sources_.emplace(local_id, stream_source_state{std::move(batch_op),
max_items_per_batch});
return {ctrl(), item_type, std::move(name), local_id};
}
flow::observable<async::batch>
scheduled_actor::do_observe(stream what, size_t buf_capacity,
size_t request_threshold) {
CAF_LOG_TRACE(CAF_ARG(what)
<< CAF_ARG(buf_capacity) << CAF_ARG(request_threshold));
if (const auto& src = what.source()) {
using impl_t = detail::stream_bridge;
return flow::make_observable<impl_t>(this, src, what.id(), buf_capacity,
request_threshold);
}
return make_observable().fail<async::batch>(make_error(sec::invalid_stream));
}
void scheduled_actor::watch(disposable obj) { void scheduled_actor::watch(disposable obj) {
CAF_ASSERT(obj.valid()); CAF_ASSERT(obj.valid());
...@@ -738,6 +916,10 @@ void scheduled_actor::watch(disposable obj) { ...@@ -738,6 +916,10 @@ void scheduled_actor::watch(disposable obj) {
CAF_LOG_DEBUG("now watching" << watched_disposables_.size() << "disposables"); CAF_LOG_DEBUG("now watching" << watched_disposables_.size() << "disposables");
} }
void scheduled_actor::deregister_stream(uint64_t stream_id) {
stream_sources_.erase(stream_id);
}
void scheduled_actor::run_actions() { void scheduled_actor::run_actions() {
if (!actions_.empty()) { if (!actions_.empty()) {
// Note: can't use iterators here since actions may add to the vector. // Note: can't use iterators here since actions may add to the vector.
...@@ -757,4 +939,19 @@ void scheduled_actor::update_watched_disposables() { ...@@ -757,4 +939,19 @@ void scheduled_actor::update_watched_disposables() {
<< "disposables"); << "disposables");
} }
void scheduled_actor::register_flow_state(uint64_t local_id,
detail::stream_bridge_sub_ptr sub) {
stream_bridges_.emplace(local_id, std::move(sub));
}
void scheduled_actor::drop_flow_state(uint64_t local_id) {
stream_bridges_.erase(local_id);
}
void scheduled_actor::try_push_stream(uint64_t local_id) {
CAF_LOG_TRACE(CAF_ARG(local_id));
if (auto i = stream_bridges_.find(local_id); i != stream_bridges_.end())
i->second->push();
}
} // namespace caf } // namespace caf
...@@ -8,7 +8,10 @@ ...@@ -8,7 +8,10 @@
#include "core-test.hpp" #include "core-test.hpp"
#include "caf/scheduled_actor/flow.hpp"
using namespace caf; using namespace caf;
using namespace std::literals;
namespace { namespace {
...@@ -31,6 +34,18 @@ struct fixture : test_coordinator_fixture<> { ...@@ -31,6 +34,18 @@ struct fixture : test_coordinator_fixture<> {
} }
}; };
using ivec = std::vector<int>;
behavior int_sink(event_based_actor* self, std::shared_ptr<ivec> results) {
return {
[self, results](stream input) {
self //
->observe_as<int>(input, 30, 10)
.for_each([results](int x) { results->push_back(x); });
},
};
}
} // namespace } // namespace
BEGIN_FIXTURE_SCOPE(fixture) BEGIN_FIXTURE_SCOPE(fixture)
...@@ -57,4 +72,34 @@ TEST_CASE("value-constructed") { ...@@ -57,4 +72,34 @@ TEST_CASE("value-constructed") {
CHECK_EQ(uut, deep_copy(uut)); CHECK_EQ(uut, deep_copy(uut));
} }
TEST_CASE("streams allow actors to transmit flow items to others") {
auto res = ivec{};
res.resize(256);
std::iota(res.begin(), res.end(), 1);
auto r1 = std::make_shared<ivec>();
auto s1 = sys.spawn(int_sink, r1);
auto r2 = std::make_shared<ivec>();
auto s2 = sys.spawn(int_sink, r2);
run();
auto src = sys.spawn([s1, s2](event_based_actor* self) {
auto vals = self //
->make_observable()
.iota(1)
.take(256)
.compose(self->to_stream("foo", 10ms, 10));
self->send(s1, vals);
self->send(s2, vals);
});
run_once();
expect((stream), from(src).to(s1));
expect((stream), from(src).to(s2));
expect((stream_open_msg), from(s1).to(src));
expect((stream_open_msg), from(s2).to(src));
expect((stream_ack_msg), from(src).to(s1));
expect((stream_ack_msg), from(src).to(s2));
run();
CHECK_EQ(*r1, res);
CHECK_EQ(*r2, res);
}
END_FIXTURE_SCOPE() END_FIXTURE_SCOPE()
...@@ -12,7 +12,7 @@ namespace caf::io::basp { ...@@ -12,7 +12,7 @@ namespace caf::io::basp {
/// @{ /// @{
/// The current BASP version. Note: BASP is not backwards compatible. /// The current BASP version. Note: BASP is not backwards compatible.
constexpr uint64_t version = 5; constexpr uint64_t version = 6;
/// @} /// @}
......
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