Commit 4c16673b authored by Dominik Charousset's avatar Dominik Charousset

Merge 'topic/streaming'

parents 227921f6 8b47ed51
......@@ -64,7 +64,6 @@ add_library(libcaf_core_obj OBJECT ${CAF_CORE_HEADERS}
src/binary_deserializer.cpp
src/binary_serializer.cpp
src/blocking_actor.cpp
src/complexity_based_credit_controller.cpp
src/config_option.cpp
src/config_option_adder.cpp
src/config_option_set.cpp
......@@ -97,11 +96,14 @@ add_library(libcaf_core_obj OBJECT ${CAF_CORE_HEADERS}
src/detail/set_thread_name.cpp
src/detail/shared_spinlock.cpp
src/detail/simple_actor_clock.cpp
src/detail/size_based_credit_controller.cpp
src/detail/stringification_inspector.cpp
src/detail/sync_request_bouncer.cpp
src/detail/test_actor_clock.cpp
src/detail/thread_safe_actor_clock.cpp
src/detail/tick_emitter.cpp
src/detail/token_based_credit_controller.cpp
src/detail/token_based_credit_controller.cpp
src/detail/type_id_list_builder.cpp
src/detail/uri_impl.cpp
src/downstream_manager.cpp
......@@ -156,7 +158,6 @@ add_library(libcaf_core_obj OBJECT ${CAF_CORE_HEADERS}
src/sec_strings.cpp
src/serializer.cpp
src/settings.cpp
src/size_based_credit_controller.cpp
src/skip.cpp
src/stream_aborter.cpp
src/stream_manager.cpp
......@@ -170,7 +171,6 @@ add_library(libcaf_core_obj OBJECT ${CAF_CORE_HEADERS}
src/telemetry/metric_family.cpp
src/telemetry/metric_registry.cpp
src/term.cpp
src/test_credit_controller.cpp
src/thread_hook.cpp
src/timestamp.cpp
src/tracing_data.cpp
......
......@@ -204,18 +204,12 @@ public:
// -- stream parameters ------------------------------------------------------
/// @private
timespan stream_desired_batch_complexity;
/// @private
timespan stream_max_batch_delay;
/// @private
timespan stream_credit_round_interval;
/// @private
timespan stream_tick_duration() const noexcept;
// -- OpenSSL parameters -----------------------------------------------------
std::string openssl_certificate;
......
......@@ -94,18 +94,6 @@ public:
return i != state_map_.end() ? i->second.buf.size() : 0u;
}
int32_t max_capacity() const noexcept override {
// The maximum capacity is limited by the slowest downstream path.
auto result = std::numeric_limits<int32_t>::max();
for (auto& kvp : this->paths_) {
auto mc = kvp.second->max_capacity;
// max_capacity is 0 if and only if we didn't receive an ack_batch yet.
if (mc > 0)
result = std::min(result, mc);
}
return result;
}
/// Sets the filter for `slot` to `filter`. Inserts a new element if `slot`
/// is a new path.
void set_filter(stream_slot slot, filter_type new_filter) {
......@@ -307,7 +295,10 @@ private:
}
auto new_size = buffered();
CAF_ASSERT(old_size >= new_size);
this->shipped_messages(old_size - new_size);
auto shipped = old_size - new_size;
this->shipped_messages(shipped);
if (shipped > 0)
this->last_send_ = this->self()->now();
}
state_map_type state_map_;
......
......@@ -100,7 +100,7 @@ public:
desired);
desired *= 2;
auto stored = buffered();
return stored < desired ? desired - stored : 0u;
return desired > stored ? desired - stored : 0u;
}
size_t buffered() const noexcept override {
......
......@@ -32,67 +32,34 @@ public:
// -- member types -----------------------------------------------------------
/// Wraps an assignment of the controller to its source.
struct CAF_CORE_EXPORT assignment {
/// Stores how much credit we assign to the source.
int32_t credit;
struct calibration {
/// Stores how much credit the path may emit at most.
int32_t max_credit;
/// Stores how many elements we demand per batch.
int32_t batch_size;
/// Stores how many batches the caller should wait before calling
/// `calibrate` again.
int32_t next_calibration;
};
// -- constructors, destructors, and assignment operators --------------------
explicit credit_controller(scheduled_actor* self);
virtual ~credit_controller();
// -- properties -------------------------------------------------------------
scheduled_actor* self() noexcept {
return self_;
}
// -- pure virtual functions -------------------------------------------------
/// Called before processing the batch `x` in order to allow the controller
/// to keep statistics on incoming batches.
virtual void before_processing(downstream_msg::batch& x) = 0;
/// Called after processing the batch `x` in order to allow the controller to
/// keep statistics on incoming batches.
/// @note The consumer may alter the batch while processing it, for example
/// by moving each element of the batch downstream.
virtual void after_processing(downstream_msg::batch& x) = 0;
virtual void before_processing(downstream_msg::batch& batch) = 0;
/// Assigns initial credit during the stream handshake.
/// @returns The initial credit for the new sources.
virtual assignment compute_initial() = 0;
/// Assigs new credit to the source after a cycle ends.
/// @param cycle Duration of a cycle.
/// @param max_downstream_credit Maximum downstream capacity as reported by
/// the downstream manager. Controllers may use
/// this capacity as an upper bound.
virtual assignment compute(timespan cycle, int32_t max_downstream_credit) = 0;
// -- virtual functions ------------------------------------------------------
/// Returns an initial calibration for the path.
virtual calibration init() = 0;
/// Computes a credit assignment to the source after crossing the
/// low-threshold. May assign zero credit.
virtual assignment compute_bridge();
/// Returns the threshold for when we may give extra credit to a source
/// during a cycle.
/// @returns Zero or a negative value if the controller never grants bridge
/// credit, otherwise the threshold for calling `compute_bridge` to
/// generate additional credit.
virtual int32_t threshold() const noexcept;
private:
// -- member variables -------------------------------------------------------
/// Points to the parent system.
scheduled_actor* self_;
virtual calibration calibrate() = 0;
};
} // namespace caf
......@@ -33,20 +33,60 @@
namespace caf::defaults::stream {
constexpr auto desired_batch_complexity = timespan{50'000};
constexpr auto max_batch_delay = timespan{5'000'000};
constexpr auto credit_round_interval = timespan{10'000'000};
constexpr auto credit_policy = string_view{"complexity"};
constexpr auto max_batch_delay = timespan{1'000'000};
/// Configures an algorithm for assigning credit and adjusting batch sizes.
///
/// The `size-based` controller (default) samples how many Bytes stream elements
/// occupy when serialized to CAF's binary wire format.
///
/// The `token-based` controller associates each stream element with one token.
/// Input buffer and batch sizes are then statically defined in terms of tokens.
/// This strategy makes no dynamic adjustment or sampling.
constexpr auto credit_policy = string_view{"size-based"};
[[deprecated("this parameter no longer has any effect")]] //
constexpr auto credit_round_interval
= max_batch_delay;
} // namespace caf::defaults::stream
namespace caf::defaults::stream::size_policy {
constexpr auto bytes_per_batch = int32_t{02 * 1024}; // 2 KB
/// Desired size of a single batch in Bytes, when serialized into CAF's binary
/// wire format.
constexpr auto bytes_per_batch = int32_t{2 * 1024}; // 2 KB
/// Number of Bytes (over all received elements) an inbound path may buffer.
/// Actors use heuristics for calculating the estimated memory use, so actors
/// may still allocate more memory in practice.
constexpr auto buffer_capacity = int32_t{64 * 1024}; // 64 KB
/// Frequency of computing the serialized size of incoming batches. Smaller
/// values may increase accuracy, but also add computational overhead.
constexpr auto sampling_rate = int32_t{100};
/// Frequency of re-calibrating batch sizes. For example, a calibration interval
/// of 10 and a sampling rate of 20 causes the actor to re-calibrate every 200
/// batches.
constexpr auto calibration_interval = int32_t{20};
/// Value between 0 and 1 representing the degree of weighting decrease for
/// adjusting batch sizes. A higher factor discounts older observations faster.
constexpr auto smoothing_factor = float{0.6};
} // namespace caf::defaults::stream::size_policy
namespace caf::defaults::stream::token_policy {
/// Number of elements in a single batch.
constexpr auto batch_size = int32_t{256}; // 2 KB for elements of size 8.
/// Maximum number of elements in the input buffer.
constexpr auto buffer_size = int32_t{4096}; // // 32 KB for elements of size 8.
} // namespace caf::defaults::stream::token_policy
namespace caf::defaults::scheduler {
constexpr auto policy = string_view{"stealing"};
......
......@@ -30,9 +30,7 @@ public:
using super::super;
size_t result() const noexcept {
return result_;
}
size_t result = 0;
result_type begin_object(type_id_t) override;
......@@ -75,9 +73,6 @@ public:
result_type apply(span<const byte> x) override;
result_type apply(const std::vector<bool>& xs) override;
private:
size_t result_ = 0;
};
template <class T>
......@@ -85,7 +80,7 @@ size_t serialized_size(actor_system& sys, const T& x) {
serialized_size_inspector f{sys};
auto err = f(x);
static_cast<void>(err);
return f.result();
return f.result;
}
template <class T>
......@@ -93,7 +88,7 @@ size_t serialized_size(const T& x) {
serialized_size_inspector f{nullptr};
auto err = f(x);
static_cast<void>(err);
return f.result();
return f.result;
}
} // namespace caf::detail
......@@ -19,24 +19,21 @@
#pragma once
#include "caf/credit_controller.hpp"
#include "caf/detail/core_export.hpp"
#include "caf/detail/serialized_size.hpp"
#include "caf/downstream_msg.hpp"
#include "caf/stream.hpp"
namespace caf::detail {
/// A credit controller that estimates the bytes required to store incoming
/// batches and constrains credit based on upper bounds for memory usage.
class size_based_credit_controller : public credit_controller {
class CAF_CORE_EXPORT size_based_credit_controller : public credit_controller {
public:
// -- member types -----------------------------------------------------------
using super = credit_controller;
// -- constants --------------------------------------------------------------
/// Configures at what buffer level we grant bridge credit (0 to 1).
static constexpr float buffer_threshold = 0.75f;
/// Configures how many samples we require for recalculating buffer sizes.
static constexpr int32_t min_samples = 10;
static constexpr int32_t min_samples = 50;
/// Stores how many elements we buffer at most after the handshake.
int32_t initial_buffer_size = 10;
......@@ -46,54 +43,75 @@ public:
// -- constructors, destructors, and assignment operators --------------------
explicit size_based_credit_controller(scheduled_actor* self);
explicit size_based_credit_controller(local_actor* self);
~size_based_credit_controller() override;
// -- overrides --------------------------------------------------------------
void before_processing(downstream_msg::batch& x) override;
void after_processing(downstream_msg::batch& x) override;
assignment compute_initial() override;
// -- interface functions ----------------------------------------------------
calibration init() override;
calibration calibrate() override;
// -- factory functions ------------------------------------------------------
template <class T>
static auto make(local_actor* self, stream<T>) {
class impl : public size_based_credit_controller {
using size_based_credit_controller::size_based_credit_controller;
void before_processing(downstream_msg::batch& x) override {
if (++this->sample_counter_ == this->sampling_rate_) {
this->sample_counter_ = 0;
this->inspector_.result = 0;
this->sampled_elements_ += x.xs_size;
for (auto& element : x.xs.get_as<std::vector<T>>(0)) {
auto res = this->inspector_(element);
static_cast<void>(res); // This inspector never produces an error.
}
this->sampled_total_size_
+= static_cast<int64_t>(this->inspector_.result);
}
}
};
return std::make_unique<impl>(self);
}
protected:
// -- member variables -------------------------------------------------------
assignment compute(timespan cycle, int32_t) override;
local_actor* self_;
assignment compute_bridge() override;
/// Keeps track of when to sample a batch.
int32_t sample_counter_ = 0;
int32_t threshold() const noexcept override;
/// Stores the last computed (moving) average for the serialized size per
/// element in the stream.
int32_t bytes_per_element_ = 0;
private:
// -- member variables -------------------------------------------------------
/// Stores how many elements were sampled since last calling `calibrate`.
int32_t sampled_elements_ = 0;
/// Total number of elements in all processed batches in the current cycle.
int64_t num_batches_ = 0;
/// Stores how many bytes the sampled batches required when serialized.
int64_t sampled_total_size_ = 0;
/// Stores how many elements the buffer should hold at most.
int32_t buffer_size_ = initial_buffer_size;
/// Computes how many bytes elements require on the wire.
serialized_size_inspector inspector_;
/// Stores how many elements each batch should contain.
int32_t batch_size_ = initial_batch_size;
/// Stores whether this is the first run.
bool initializing_ = true;
/// Configures how many bytes we store in total.
int32_t buffer_capacity_;
// -- see caf::defaults::stream::size_policy --------------------------------
/// Configures how many bytes we transfer per batch.
int32_t bytes_per_batch_;
/// Stores how many elements we have sampled during the current cycle.
int32_t sampled_elements_ = 0;
int32_t buffer_capacity_;
/// Stores approximately how many bytes the sampled elements require when
/// serialized.
int32_t sampled_total_size_ = 0;
int32_t sampling_rate_ = 1;
/// Counter for keeping track of when to sample a batch.
int32_t sample_counter_ = 0;
int32_t calibration_interval_;
/// Configured how many batches we skip for the size sampling.
int32_t sample_rate_ = 1;
float smoothing_factor_;
};
} // namespace caf::detail
......@@ -61,10 +61,6 @@ public:
return driver_.acquire_credit(path, desired);
}
bool congested() const noexcept override {
return driver_.congested();
}
protected:
void finalize(const error& reason) override {
driver_.finalize(reason);
......
......@@ -73,10 +73,6 @@ public:
CAF_LOG_ERROR("received unexpected batch type (dropped)");
}
bool congested() const noexcept override {
return driver_.congested();
}
int32_t acquire_credit(inbound_path* path, int32_t desired) override {
return driver_.acquire_credit(path, desired);
}
......
......@@ -19,58 +19,52 @@
#pragma once
#include "caf/credit_controller.hpp"
#include "caf/detail/core_export.hpp"
namespace caf {
namespace detail {
namespace caf::detail {
/// Computes credit for an attached source based on measuring the complexity of
/// incoming batches.
class complexity_based_credit_controller : public credit_controller {
/// A credit controller that estimates the bytes required to store incoming
/// batches and constrains credit based on upper bounds for memory usage.
class CAF_CORE_EXPORT token_based_credit_controller : public credit_controller {
public:
// -- member types -----------------------------------------------------------
using super = credit_controller;
// -- constants --------------------------------------------------------------
/// Configures how many samples we require for recalculating buffer sizes.
static constexpr int32_t min_samples = 50;
/// Stores how many elements we buffer at most after the handshake.
int32_t initial_buffer_size = 50;
int32_t initial_buffer_size = 10;
/// Stores how many elements we allow per batch after the handshake.
int32_t initial_batch_size = 10;
int32_t initial_batch_size = 2;
// -- constructors, destructors, and assignment operators --------------------
explicit complexity_based_credit_controller(scheduled_actor* self);
explicit token_based_credit_controller(local_actor* self);
~complexity_based_credit_controller() override;
~token_based_credit_controller() override;
// -- overrides --------------------------------------------------------------
// -- interface functions ----------------------------------------------------
void before_processing(downstream_msg::batch& x) override;
void before_processing(downstream_msg::batch& batch) override;
void after_processing(downstream_msg::batch& x) override;
calibration init() override;
assignment compute_initial() override;
calibration calibrate() override;
assignment compute(timespan cycle, int32_t) override;
// -- factory functions ------------------------------------------------------
private:
// -- member variables -------------------------------------------------------
/// Total number of elements in all processed batches in the current cycle.
int64_t num_elements_ = 0;
template <class T>
static auto make(local_actor* self, stream<T>) {
return std::make_unique<token_based_credit_controller>(self);
}
/// Elapsed time for processing all elements of all batches in the current
/// cycle.
timespan processing_time_;
private:
// -- see caf::defaults::stream::token_policy -------------------------------
/// Timestamp of the last call to `before_processing`.
timestamp processing_begin_;
int32_t batch_size_;
/// Stores the desired per-batch complexity.
timespan complexity_;
int32_t buffer_size_;
};
} // namespace detail
} // namespace caf
} // namespace caf::detail
......@@ -21,9 +21,11 @@
#include <memory>
#include <vector>
#include "caf/actor_clock.hpp"
#include "caf/detail/core_export.hpp"
#include "caf/fwd.hpp"
#include "caf/stream_slot.hpp"
#include "caf/timespan.hpp"
namespace caf {
......@@ -48,6 +50,9 @@ public:
/// Unique pointer to an outbound path.
using unique_path_ptr = std::unique_ptr<path_type>;
/// Discrete point in time, as reported by the actor clock.
using time_point = typename actor_clock::time_point;
/// Function object for iterating over all paths.
struct CAF_CORE_EXPORT path_visitor {
virtual ~path_visitor();
......@@ -83,6 +88,11 @@ public:
/// stream and never has outbound paths.
virtual bool terminal() const noexcept;
// -- time management --------------------------------------------------------
/// Forces underful batches after reaching the maximum delay.
void tick(time_point now, timespan max_batch_delay);
// -- path management --------------------------------------------------------
/// Applies `f` to each path.
......@@ -193,9 +203,6 @@ public:
/// Queries an estimate of the size of the output buffer for `slot`.
virtual size_t buffered(stream_slot slot) const noexcept;
/// Computes the maximum available downstream capacity.
virtual int32_t max_capacity() const noexcept;
/// Queries whether the manager cannot make any progress, because its buffer
/// is full and no more credit is available.
bool stalled() const noexcept;
......@@ -244,6 +251,9 @@ protected:
// -- member variables -------------------------------------------------------
stream_manager* parent_;
/// Stores the time stamp of our last batch.
time_point last_send_;
};
} // namespace caf
......@@ -25,10 +25,13 @@
#include "caf/actor_control_block.hpp"
#include "caf/credit_controller.hpp"
#include "caf/detail/core_export.hpp"
#include "caf/detail/size_based_credit_controller.hpp"
#include "caf/detail/token_based_credit_controller.hpp"
#include "caf/downstream_msg.hpp"
#include "caf/logger.hpp"
#include "caf/meta/type_name.hpp"
#include "caf/settings.hpp"
#include "caf/stream_aborter.hpp"
#include "caf/stream_manager.hpp"
#include "caf/stream_priority.hpp"
#include "caf/stream_slot.hpp"
#include "caf/telemetry/counter.hpp"
......@@ -41,14 +44,56 @@ namespace caf {
/// State for a path to an upstream actor (source).
class CAF_CORE_EXPORT inbound_path {
public:
// -- member types -----------------------------------------------------------
/// Message type for propagating graceful shutdowns.
using regular_shutdown = upstream_msg::drop;
/// Message type for propagating errors.
using irregular_shutdown = upstream_msg::forced_drop;
/// Wraps optional actor metrics collected by this path.
struct metrics_t {
telemetry::int_counter* processed_elements;
telemetry::int_gauge* input_buffer_size;
};
/// Discrete point in time, as reported by the actor clock.
using time_point = typename actor_clock::time_point;
/// Time interval, as reported by the actor clock.
using duration_type = typename actor_clock::duration_type;
// -- constructors, destructors, and assignment operators --------------------
template <class T>
inbound_path(stream_manager* mgr, stream<T> in)
: inbound_path(mgr, type_id_v<T>) {
auto& cfg = config();
auto set_default = [this, in] {
controller_ = detail::size_based_credit_controller::make(self(), in);
};
if (auto str = get_if<std::string>(&cfg, "caf.stream.credit-policy")) {
if (*str == "token-based")
controller_ = detail::token_based_credit_controller::make(self(), in);
else if (*str == "size-based")
set_default();
else
CAF_LOG_WARNING("unrecognized credit policy:"
<< *str << "(falling back to 'size-based')");
} else {
set_default();
}
}
void init(strong_actor_ptr source_hdl, stream_slots id);
~inbound_path();
// -- member variables -------------------------------------------------------
/// Points to the manager responsible for incoming traffic.
stream_manager_ptr mgr;
stream_manager* mgr;
/// Handle to the source.
strong_actor_ptr hdl;
......@@ -56,18 +101,24 @@ public:
/// Stores slot IDs for sender (hdl) and receiver (self).
stream_slots slots;
/// Optionally stores pointers to telemetry objects.
struct metrics_t {
telemetry::int_counter* processed_elements;
telemetry::int_gauge* input_buffer_size;
} metrics;
/// Stores pointers to optional telemetry objects.
metrics_t metrics;
/// Stores the last computed desired batch size.
/// Stores the last computed desired batch size. Adjusted at run-time by the
/// controller.
int32_t desired_batch_size = 0;
/// Amount of credit we have signaled upstream.
int32_t assigned_credit = 0;
/// Maximum amount of credit that the path may signal upstream. Adjusted at
/// run-time by the controller.
int32_t max_credit = 0;
/// Decremented whenever receiving a batch. Triggers a re-calibration by the
/// controller when reaching zero.
int32_t calibration_countdown = 10;
/// Priority of incoming batches from this source.
stream_priority prio = stream_priority::normal;
......@@ -80,46 +131,49 @@ public:
/// Controller for assigning credit to the source.
std::unique_ptr<credit_controller> controller_;
/// Stores the time point of the last credit decision for this source.
actor_clock::time_point last_credit_decision;
/// Stores when the last ACK was emitted.
time_point last_ack_time;
/// Stores the time point of the last credit decision for this source.
actor_clock::time_point next_credit_decision;
// -- properties -------------------------------------------------------------
/// Constructs a path for given handle and stream ID.
inbound_path(stream_manager_ptr mgr_ptr, stream_slots id,
strong_actor_ptr ptr, type_id_t input_type);
/// Returns whether the path received no input since last emitting
/// `ack_batch`, i.e., `last_acked_batch_id == last_batch_id`.
bool up_to_date() const noexcept;
~inbound_path();
/// Returns a pointer to the parent actor.
scheduled_actor* self() const noexcept;
/// Returns currently unassigned credit that we could assign to the source.
int32_t available_credit() const noexcept;
/// Returns the system-wide configuration.
const settings& config() const noexcept;
// -- callbacks --------------------------------------------------------------
/// Updates `last_batch_id` and `assigned_credit` before dispatching to the
/// manager.
void handle(downstream_msg::batch& x);
/// Dispatches any `downstream_msg` other than `batch` directly to the
/// manager.
template <class T>
void handle(T& x) {
mgr->handle(this, x);
}
/// Forward the `close` message to the manager.
void handle(downstream_msg::close& x);
/// Forward the `forced_close` message to the manager.
void handle(downstream_msg::forced_close& x);
/// Forces an ACK message after receiving no input for a considerable amount
/// of time.
void tick(time_point now, duration_type max_batch_delay);
// -- messaging --------------------------------------------------------------
/// Emits an `upstream_msg::ack_batch`.
void emit_ack_open(local_actor* self, actor_addr rebind_from);
/// Sends an `upstream_msg::ack_batch` for granting new credit. Credit is
/// calculated from sampled batch durations, the cycle duration and the
/// desired batch complexity.
/// Sends an `upstream_msg::ack_batch` for granting new credit.
/// @param self Points to the parent actor, i.e., sender of the message.
/// @param queued_items Accumulated size of all batches that are currently
/// waiting in the mailbox.
/// @param now Current timestamp.
/// @param cycle Time between credit rounds.
void emit_ack_batch(local_actor* self, int32_t queued_items,
actor_clock::time_point now, timespan cycle);
/// Returns whether the path received no input since last emitting
/// `ack_batch`, i.e., `last_acked_batch_id == last_batch_id`.
bool up_to_date();
/// @param new_credit Amount of new credit to assign to the source.
void emit_ack_batch(local_actor* self, int32_t new_credit);
/// Sends an `upstream_msg::drop` on this path.
void emit_regular_shutdown(local_actor* self);
......@@ -132,8 +186,8 @@ public:
emit_irregular_shutdown(local_actor* self, stream_slots slots,
const strong_actor_ptr& hdl, error reason);
/// Returns a pointer to the parent actor.
scheduled_actor* self();
private:
inbound_path(stream_manager* mgr_ptr, type_id_t input_type);
};
/// @relates inbound_path
......
......@@ -165,9 +165,6 @@ public:
/// ACKs, i.e., receiving an ACK with a higher ID is not an error.
int64_t next_ack_id;
/// Stores the maximum capacity of the downstream actor.
int32_t max_capacity;
/// Stores whether an outbound path is marked for removal. The
/// `downstream_manger` no longer sends new batches to a closing path, but
/// buffered batches are still shipped. The owning `stream_manager` removes
......
......@@ -67,6 +67,14 @@ public:
return x + x;
}
template <template <class> class Queue>
static deficit_type
quantum(const Queue<upstream_messages>& q, deficit_type) noexcept {
// Allow actors to consume *all* upstream messages. They are lightweight by
// design and require little processing.
return q.total_task_size();
}
template <class Queue>
static deficit_type quantum(const Queue&, deficit_type x) noexcept {
return x;
......
......@@ -32,7 +32,6 @@
#include "caf/actor_traits.hpp"
#include "caf/detail/behavior_stack.hpp"
#include "caf/detail/core_export.hpp"
#include "caf/detail/tick_emitter.hpp"
#include "caf/detail/unordered_flat_map.hpp"
#include "caf/error.hpp"
#include "caf/extend.hpp"
......@@ -201,52 +200,6 @@ public:
using exception_handler = std::function<error(pointer, std::exception_ptr&)>;
#endif // CAF_ENABLE_EXCEPTIONS
/// Consumes messages from the mailbox.
struct mailbox_visitor {
scheduled_actor* self;
size_t& handled_msgs;
size_t max_throughput;
bool collect_metrics;
/// Consumes upstream messages.
intrusive::task_result operator()(size_t, upstream_queue&,
mailbox_element&);
/// Consumes downstream messages.
intrusive::task_result
operator()(size_t, downstream_queue&, stream_slot slot,
policy::downstream_messages::nested_queue_type&,
mailbox_element&);
// Dispatches asynchronous messages with high and normal priority to the
// same handler.
template <class Queue>
intrusive::task_result operator()(size_t, Queue&, mailbox_element& x) {
return (*this)(x);
}
// Consumes asynchronous messages.
intrusive::task_result operator()(mailbox_element& x);
template <class F>
intrusive::task_result run(mailbox_element& x, F body) {
if (collect_metrics) {
auto t0 = std::chrono::steady_clock::now();
auto mbox_time = x.seconds_until(t0);
auto res = body();
if (res != intrusive::task_result::skip) {
auto& builtins = self->builtin_metrics();
telemetry::timer::observe(builtins.processing_time, t0);
builtins.mailbox_time->observe(mbox_time);
builtins.mailbox_size->dec();
}
return res;
} else {
return body();
}
}
};
// -- static helper functions ------------------------------------------------
static void default_error_handler(pointer ptr, error& x);
......@@ -518,6 +471,9 @@ public:
/// Pushes `ptr` to the cache of the default queue.
void push_to_cache(mailbox_element_ptr ptr);
/// Returns the queue of the mailbox that stores high priority messages.
urgent_queue& get_urgent_queue();
/// Returns the default queue of the mailbox that stores ordinary messages.
normal_queue& get_normal_queue();
......@@ -527,15 +483,11 @@ public:
/// Returns the queue of the mailbox that stores `downstream_msg` messages.
downstream_queue& get_downstream_queue();
/// Returns the queue of the mailbox that stores high priority messages.
urgent_queue& get_urgent_queue();
// -- inbound_path management ------------------------------------------------
/// Creates a new path for incoming stream traffic from `sender`.
virtual inbound_path*
make_inbound_path(stream_manager_ptr mgr, stream_slots slots,
strong_actor_ptr sender, type_id_t rtti);
virtual bool add_inbound_path(type_id_t input_type,
std::unique_ptr<inbound_path> path);
/// Silently closes incoming stream traffic on `slot`.
virtual void erase_inbound_path_later(stream_slot slot);
......@@ -571,8 +523,7 @@ public:
return;
}
CAF_LOG_INFO("no manager found:" << CAF_ARG(slots));
// TODO: replace with `if constexpr` when switching to C++17
if (std::is_same<T, upstream_msg::ack_batch>::value) {
if constexpr (std::is_same<T, upstream_msg::ack_batch>::value) {
// Make sure the other actor does not falsely believe us a source.
inbound_path::emit_irregular_shutdown(this, slots, current_sender(),
sec::invalid_upstream);
......@@ -659,6 +610,12 @@ public:
/// Removes the stream manager mapped to `id` in `O(log n)`.
void erase_pending_stream_manager(stream_slot id);
/// Moves a pending stream manager to the list of active stream managers.
/// @returns `true` and a pointer to the moved stream manager on success,
/// `false` and `nullptr` otherwise.
[[nodiscard]] std::pair<bool, stream_manager*>
ack_pending_stream_manager(stream_slot id);
/// Removes all entries for `mgr` in `O(n)`.
void erase_stream_manager(const stream_manager_ptr& mgr);
......@@ -681,6 +638,14 @@ public:
|| !pending_stream_managers_.empty();
}
auto max_batch_delay() const noexcept {
return max_batch_delay_;
}
void active_stream_managers(std::vector<stream_manager*>& result);
std::vector<stream_manager*> active_stream_managers();
/// @endcond
protected:
......@@ -723,15 +688,13 @@ protected:
/// yet received an ACK.
stream_manager_map pending_stream_managers_;
/// Controls batch and credit timeouts.
detail::tick_emitter stream_ticks_;
/// Stores how long the actor should try to accumulate more items in order to
/// send a full stream batch.
timespan max_batch_delay_;
/// Number of ticks per batch delay.
size_t max_batch_delay_ticks_;
/// Number of ticks of each credit round.
size_t credit_round_ticks_;
/// Pointer to a private thread object associated with a detached actor.
detail::private_thread* private_thread_;
......@@ -746,7 +709,23 @@ protected:
exception_handler exception_handler_;
#endif // CAF_ENABLE_EXCEPTIONS
/// @endcond
private:
template <class F>
intrusive::task_result run_with_metrics(mailbox_element& x, F body) {
if (metrics_.mailbox_time) {
auto t0 = std::chrono::steady_clock::now();
auto mbox_time = x.seconds_until(t0);
auto res = body();
if (res != intrusive::task_result::skip) {
telemetry::timer::observe(metrics_.processing_time, t0);
metrics_.mailbox_time->observe(mbox_time);
metrics_.mailbox_size->dec();
}
return res;
} else {
return body();
}
}
};
} // namespace caf
......@@ -19,10 +19,7 @@
#pragma once
#include "caf/fwd.hpp"
#include "caf/invalid_stream.hpp"
#include "caf/meta/type_name.hpp"
#include "caf/stream_manager.hpp"
#include "caf/stream_slot.hpp"
#include "caf/type_id.hpp"
namespace caf {
......
......@@ -25,12 +25,15 @@
#include "caf/actor.hpp"
#include "caf/actor_cast.hpp"
#include "caf/actor_clock.hpp"
#include "caf/detail/core_export.hpp"
#include "caf/downstream_msg.hpp"
#include "caf/fwd.hpp"
#include "caf/inbound_path.hpp"
#include "caf/ref_counted.hpp"
#include "caf/stream.hpp"
#include "caf/stream_slot.hpp"
#include "caf/timespan.hpp"
#include "caf/upstream_msg.hpp"
namespace caf {
......@@ -54,8 +57,15 @@ public:
// -- member types -----------------------------------------------------------
using inbound_path_ptr = std::unique_ptr<inbound_path>;
using inbound_paths_list = std::vector<inbound_path*>;
/// Discrete point in time.
using time_point = typename actor_clock::time_point;
// -- constructors, destructors, and assignment operators --------------------
stream_manager(scheduled_actor* selfptr,
stream_priority prio = stream_priority::normal);
......@@ -83,10 +93,6 @@ public:
/// buffers of in- and outbound paths.
virtual void shutdown();
/// Tries to advance the stream by generating more credit or by sending
/// batches.
void advance();
/// Pushes new data to downstream actors by sending batches. The amount of
/// pushed data is limited by the available credit.
virtual void push();
......@@ -107,7 +113,7 @@ public:
/// messages.
virtual bool generate_messages();
// -- pure virtual member functions ------------------------------------------
// -- interface functions ----------------------------------------------------
/// Returns the manager for downstream communication.
virtual downstream_manager& out() = 0;
......@@ -127,6 +133,11 @@ public:
/// Advances time.
virtual void cycle_timeout(size_t cycle_nr);
/// Acquires credit on an inbound path. The calculated credit to fill our
/// queue for two cycles is `desired`, but the manager is allowed to return
/// any non-negative value.
virtual int32_t acquire_credit(inbound_path* path, int32_t desired);
// -- input path management --------------------------------------------------
/// Informs the manager that a new input path opens.
......@@ -139,6 +150,16 @@ public:
/// This function is called from the destructor of `inbound_path`.
virtual void deregister_input_path(inbound_path* x) noexcept;
/// Creates an inbound path to the current sender without any type checking.
/// @pre `current_sender() != nullptr`
/// @pre `out().terminal() == false`
/// @private
template <class In>
stream_slot add_unchecked_inbound_path(stream<In> in) {
auto path = std::make_unique<inbound_path>(this, in);
return add_unchecked_inbound_path_impl(type_id_v<In>, std::move(path));
}
/// Removes an input path
virtual void remove_input_path(stream_slot slot, error reason, bool silent);
......@@ -180,14 +201,11 @@ public:
bool inbound_paths_idle() const noexcept;
/// Returns the parent actor.
scheduled_actor* self() {
scheduled_actor* self() noexcept {
return self_;
}
/// Acquires credit on an inbound path. The calculated credit to fill our
/// queue for two cycles is `desired`, but the manager is allowed to return
/// any non-negative value.
virtual int32_t acquire_credit(inbound_path* path, int32_t desired);
// -- output path management -------------------------------------------------
/// Creates an outbound path to the current sender without any type checking.
/// @pre `out().terminal() == false`
......@@ -241,15 +259,6 @@ public:
std::move(handshake));
}
/// Creates an inbound path to the current sender without any type checking.
/// @pre `current_sender() != nullptr`
/// @pre `out().terminal() == false`
/// @private
template <class In>
stream_slot add_unchecked_inbound_path(stream<In>) {
return add_unchecked_inbound_path_impl(type_id_v<In>);
}
/// Adds a new outbound path to `rp.next()`.
/// @private
stream_slot
......@@ -264,9 +273,9 @@ public:
/// @private
stream_slot add_unchecked_outbound_path_impl(message handshake);
/// Adds the current sender as an inbound path.
/// @pre Current message is an `open_stream_msg`.
stream_slot add_unchecked_inbound_path_impl(type_id_t rtti);
// -- time management --------------------------------------------------------
void tick(time_point now);
protected:
// -- modifiers for self -----------------------------------------------------
......@@ -295,6 +304,11 @@ protected:
/// implementation does nothing.
virtual void output_closed(error reason);
/// Adds the current sender as an inbound path.
/// @pre Current message is an `open_stream_msg`.
stream_slot add_unchecked_inbound_path_impl(type_id_t input_type,
inbound_path_ptr path);
// -- member variables -------------------------------------------------------
/// Points to the parent actor.
......@@ -312,6 +326,10 @@ protected:
/// Stores individual flags, for continuous streaming or when shutting down.
int flags_;
/// Stores the maximum amount of time outbound paths should buffer elements
/// before sending underful batches.
timespan max_batch_delay_;
private:
void setf(int flag) noexcept {
auto x = flags_;
......
......@@ -62,8 +62,8 @@ public:
// -- properties -------------------------------------------------------------
/// Creates a new input path to the current sender.
inbound_stream_slot<input_type> add_inbound_path(const stream<input_type>&) {
return {this->add_unchecked_inbound_path_impl(type_id_v<input_type>)};
inbound_stream_slot<input_type> add_inbound_path(stream<input_type> in) {
return {this->add_unchecked_inbound_path(in)};
}
private:
......
......@@ -60,12 +60,6 @@ public:
/// Processes a single batch.
virtual void process(std::vector<input_type>& batch) = 0;
/// Can mark the sink as congested, e.g., when writing into a buffer that
/// fills up faster than it is drained.
virtual bool congested() const noexcept {
return false;
}
/// Acquires credit on an inbound path. The calculated credit to fill our
/// queue for two cycles is `desired`, but the driver is allowed to return
/// any non-negative value.
......
......@@ -59,7 +59,7 @@ public:
}
DownstreamManager& out() override {
return left_super::out();
return this->out_;
}
};
......
......@@ -73,12 +73,6 @@ public:
// nop
}
/// Can mark the stage as congested. The default implementation signals a
/// congestion if the downstream manager has no capacity left in its buffer.
virtual bool congested() const noexcept {
return out_.capacity() == 0;
}
/// Acquires credit on an inbound path. The calculated credit to fill our
/// queue for two cycles is `desired`, but the driver is allowed to return
/// any non-negative value.
......
......@@ -75,10 +75,6 @@ struct CAF_CORE_EXPORT upstream_msg : tag::boxing_type {
/// Cumulative ack ID.
int64_t acknowledged_id;
/// Maximum capacity on this path. Stages can consider this metric for
/// downstream actors when calculating their own maximum capactiy.
int32_t max_capacity;
};
/// Asks the source to discard any remaining credit and close this path
......@@ -169,7 +165,7 @@ template <class Inspector>
typename Inspector::result_type
inspect(Inspector& f, upstream_msg::ack_batch& x) {
return f(meta::type_name("ack_batch"), x.new_capacity, x.desired_batch_size,
x.acknowledged_id, x.max_capacity);
x.acknowledged_id);
}
/// @relates upstream_msg::drop
......
......@@ -57,9 +57,8 @@ actor_system_config::actor_system_config()
config_file_path(default_config_file),
slave_mode_fun(nullptr) {
// (1) hard-coded defaults
stream_desired_batch_complexity = defaults::stream::desired_batch_complexity;
stream_max_batch_delay = defaults::stream::max_batch_delay;
stream_credit_round_interval = defaults::stream::credit_round_interval;
stream_credit_round_interval = 2 * stream_max_batch_delay;
// fill our options vector for creating config file and CLI parsers
using std::string;
using string_list = std::vector<string>;
......@@ -70,14 +69,19 @@ actor_system_config::actor_system_config()
.add<string>(config_file_path, "config-file",
"set config file (default: caf-application.conf)");
opt_group{custom_options_, "caf.stream"}
.add<timespan>(stream_desired_batch_complexity, "desired-batch-complexity",
"processing time per batch")
.add<timespan>(stream_max_batch_delay, "max-batch-delay",
"maximum delay for partial batches")
.add<timespan>(stream_credit_round_interval, "credit-round-interval",
"time between emitting credit")
.add<string>("credit-policy",
"selects an algorithm for credit computation");
"selects an implementation for credit computation");
opt_group{custom_options_, "caf.stream.size-based-policy"}
.add<int32_t>("bytes-per-batch", "desired batch size in bytes")
.add<int32_t>("buffer-capacity", "maximum input buffer size in bytes")
.add<int32_t>("sampling-rate", "frequency of collecting batch sizes")
.add<int32_t>("calibration-interval", "frequency of re-calibrations")
.add<float>("smoothing-factor", "factor for discounting older samples");
opt_group{custom_options_, "caf.stream.token-based-policy"}
.add<int32_t>("batch-size", "number of elements per batch")
.add<int32_t>("buffer-size", "max. number of elements in the input buffer");
opt_group{custom_options_, "caf.scheduler"}
.add<string>("policy", "'stealing' (default) or 'sharing'")
.add<size_t>("max-threads", "maximum number of worker threads")
......@@ -117,12 +121,8 @@ settings actor_system_config::dump_content() const {
auto& caf_group = result["caf"].as_dictionary();
// -- streaming parameters
auto& stream_group = caf_group["stream"].as_dictionary();
put_missing(stream_group, "desired-batch-complexity",
defaults::stream::desired_batch_complexity);
put_missing(stream_group, "max-batch-delay",
defaults::stream::max_batch_delay);
put_missing(stream_group, "credit-round-interval",
defaults::stream::credit_round_interval);
put_missing(stream_group, "credit-policy", defaults::stream::credit_policy);
put_missing(stream_group, "size-policy.buffer-capacity",
defaults::stream::size_policy::buffer_capacity);
......@@ -354,12 +354,6 @@ actor_system_config& actor_system_config::set_impl(string_view name,
return *this;
}
timespan actor_system_config::stream_tick_duration() const noexcept {
auto ns_count = caf::detail::gcd(stream_credit_round_interval.count(),
stream_max_batch_delay.count());
return timespan{ns_count};
}
std::string actor_system_config::render(const error& x) {
return to_string(x);
}
......
......@@ -20,20 +20,8 @@
namespace caf {
credit_controller::credit_controller(scheduled_actor* self) : self_(self) {
// nop
}
credit_controller::~credit_controller() {
// nop
}
credit_controller::assignment credit_controller::compute_bridge() {
return {0, 0};
}
int32_t credit_controller::threshold() const noexcept {
return -1;
}
} // namespace caf
......@@ -46,7 +46,7 @@ error serialized_size_inspector::begin_sequence(size_t list_size) {
x >>= 7;
}
*i++ = static_cast<uint8_t>(x) & 0x7f;
result_ += static_cast<size_t>(i - buf);
result += static_cast<size_t>(i - buf);
return none;
}
......@@ -55,57 +55,57 @@ error serialized_size_inspector::end_sequence() {
}
error serialized_size_inspector::apply(bool) {
result_ += sizeof(uint8_t);
result += sizeof(uint8_t);
return none;
}
error serialized_size_inspector::apply(int8_t x) {
result_ += sizeof(x);
result += sizeof(x);
return none;
}
error serialized_size_inspector::apply(uint8_t x) {
result_ += sizeof(x);
result += sizeof(x);
return none;
}
error serialized_size_inspector::apply(int16_t x) {
result_ += sizeof(x);
result += sizeof(x);
return none;
}
error serialized_size_inspector::apply(uint16_t x) {
result_ += sizeof(x);
result += sizeof(x);
return none;
}
error serialized_size_inspector::apply(int32_t x) {
result_ += sizeof(x);
result += sizeof(x);
return none;
}
error serialized_size_inspector::apply(uint32_t x) {
result_ += sizeof(x);
result += sizeof(x);
return none;
}
error serialized_size_inspector::apply(int64_t x) {
result_ += sizeof(x);
result += sizeof(x);
return none;
}
error serialized_size_inspector::apply(uint64_t x) {
result_ += sizeof(x);
result += sizeof(x);
return none;
}
error serialized_size_inspector::apply(float x) {
result_ += sizeof(x);
result += sizeof(x);
return none;
}
error serialized_size_inspector::apply(double x) {
result_ += sizeof(x);
result += sizeof(x);
return none;
}
......@@ -120,30 +120,30 @@ error serialized_size_inspector::apply(long double x) {
error serialized_size_inspector::apply(string_view x) {
CAF_IGNORE_UNUSED(begin_sequence(x.size()));
result_ += x.size();
result += x.size();
return end_sequence();
}
error serialized_size_inspector::apply(const std::u16string& x) {
CAF_IGNORE_UNUSED(begin_sequence(x.size()));
result_ += x.size() * sizeof(uint16_t);
result += x.size() * sizeof(uint16_t);
return end_sequence();
}
error serialized_size_inspector::apply(const std::u32string& x) {
CAF_IGNORE_UNUSED(begin_sequence(x.size()));
result_ += x.size() * sizeof(uint32_t);
result += x.size() * sizeof(uint32_t);
return end_sequence();
}
error serialized_size_inspector::apply(span<const byte> x) {
result_ += x.size();
result += x.size();
return none;
}
error serialized_size_inspector::apply(const std::vector<bool>& xs) {
CAF_IGNORE_UNUSED(begin_sequence(xs.size()));
result_ += (xs.size() + static_cast<size_t>(xs.size() % 8 != 0)) / 8;
result += (xs.size() + static_cast<size_t>(xs.size() % 8 != 0)) / 8;
return end_sequence();
}
......
......@@ -16,51 +16,60 @@
* http://www.boost.org/LICENSE_1_0.txt. *
******************************************************************************/
#include "caf/detail/complexity_based_credit_controller.hpp"
#include "caf/detail/size_based_credit_controller.hpp"
#include "caf/actor_system.hpp"
#include "caf/actor_system_config.hpp"
#include "caf/defaults.hpp"
#include "caf/detail/serialized_size.hpp"
#include "caf/scheduled_actor.hpp"
// Safe us some typing and very ugly formatting.
#define impl complexity_based_credit_controller
namespace {
namespace caf::detail {
// Sample the first 10 batches when starting up.
constexpr int32_t initial_sample_size = 10;
impl::impl(scheduled_actor* self) : super(self) {
auto& cfg = self->system().config();
complexity_ = cfg.stream_desired_batch_complexity;
}
} // namespace
impl::~impl() {
// nop
}
namespace caf::detail {
void impl::before_processing(downstream_msg::batch& x) {
num_elements_ += x.xs_size;
processing_begin_ = make_timestamp();
size_based_credit_controller::size_based_credit_controller(local_actor* ptr)
: self_(ptr), inspector_(ptr->system()) {
namespace fallback = defaults::stream::size_policy;
// Initialize from the config parameters.
auto& cfg = ptr->system().config();
if (auto section = get_if<settings>(&cfg, "caf.stream.size-based-policy")) {
bytes_per_batch_ = get_or(*section, "bytes-per-batch",
fallback::bytes_per_batch);
buffer_capacity_ = get_or(*section, "buffer-capacity",
fallback::buffer_capacity);
calibration_interval_ = get_or(*section, "calibration-interval",
fallback::calibration_interval);
smoothing_factor_ = get_or(*section, "smoothing-factor",
fallback::smoothing_factor);
} else {
buffer_capacity_ = fallback::buffer_capacity;
bytes_per_batch_ = fallback::bytes_per_batch;
calibration_interval_ = fallback::calibration_interval;
smoothing_factor_ = fallback::smoothing_factor;
}
}
void impl::after_processing(downstream_msg::batch&) {
processing_time_ += make_timestamp() - processing_begin_;
size_based_credit_controller::~size_based_credit_controller() {
// nop
}
credit_controller::assignment impl::compute_initial() {
return {initial_buffer_size, initial_batch_size};
credit_controller::calibration size_based_credit_controller::init() {
// Initially, we simply assume that the size of one element equals
// bytes-per-batch.
return {buffer_capacity_ / bytes_per_batch_, 1, initial_sample_size};
}
credit_controller::assignment
impl::compute(timespan cycle, int32_t downstream_capacity) {
// Max throughput = C * (N / t), where C = cycle length, N = measured items,
// and t = measured time. Desired batch size is the same formula with D
// (desired complexity) instead of C. We compute our values in 64-bit for
// more precision before truncating to a 32-bit integer type at the end.
int64_t total_ns = processing_time_.count();
if (total_ns == 0)
return {1, 1};
credit_controller::calibration size_based_credit_controller::calibrate() {
CAF_ASSERT(sample_counter_ == 0);
// Helper for truncating a 64-bit integer to a 32-bit integer with a minimum
// value of 1.
auto clamp = [](int64_t x) -> int32_t {
auto clamp_i32 = [](int64_t x) -> int32_t {
static constexpr auto upper_bound = std::numeric_limits<int32_t>::max();
if (x > upper_bound)
return upper_bound;
......@@ -68,17 +77,27 @@ impl::compute(timespan cycle, int32_t downstream_capacity) {
return 1;
return static_cast<int32_t>(x);
};
// Instead of C * (N / t) we calculate (C * N) / t to avoid double conversion
// and rounding errors.
assignment result;
// Give enough credit to last 2 cycles, but don't exceed downstream capacity.
result.credit = 2 * clamp((cycle.count() * num_elements_) / total_ns);
result.credit = std::min(result.credit, downstream_capacity);
result.batch_size = clamp((complexity_.count() * num_elements_) / total_ns);
// Reset state and return.
num_elements_ = 0;
processing_time_ = timespan{0};
return result;
if (!initializing_) {
auto bpe = clamp_i32(sampled_total_size_ / sampled_elements_);
bytes_per_element_ = static_cast<int32_t>(
smoothing_factor_ * bpe // weighted current measurement
+ (1.0 - smoothing_factor_) * bytes_per_element_ // past values
);
} else {
// After our first run, we continue with the actual sampling rate.
initializing_ = false;
sampling_rate_ = get_or(self_->config(),
"caf.stream.size-based-policy.sampling-rate",
defaults::stream::size_policy::sampling_rate);
bytes_per_element_ = clamp_i32(sampled_total_size_ / sampled_elements_);
}
sampled_elements_ = 0;
sampled_total_size_ = 0;
return {
clamp_i32(buffer_capacity_ / bytes_per_element_),
clamp_i32(bytes_per_batch_ / bytes_per_element_),
sampling_rate_ * calibration_interval_,
};
}
} // namespace caf::detail
......@@ -16,40 +16,45 @@
* http://www.boost.org/LICENSE_1_0.txt. *
******************************************************************************/
#pragma once
#include "caf/credit_controller.hpp"
namespace caf {
namespace detail {
/// Computes predictable credit in unit tests.
class test_credit_controller : public credit_controller {
public:
// -- member types -----------------------------------------------------------
using super = credit_controller;
// -- constructors, destructors, and assignment operators --------------------
using super::super;
~test_credit_controller() override;
// -- overrides --------------------------------------------------------------
void before_processing(downstream_msg::batch& x) override;
void after_processing(downstream_msg::batch& x) override;
assignment compute_initial() override;
assignment compute(timespan cycle, int32_t) override;
private:
/// Total number of elements in all processed batches in the current cycle.
int64_t num_elements_ = 0;
};
} // namespace detail
} // namespace caf
#include "caf/detail/token_based_credit_controller.hpp"
#include "caf/actor_system.hpp"
#include "caf/actor_system_config.hpp"
#include "caf/config_value.hpp"
#include "caf/defaults.hpp"
#include "caf/detail/serialized_size.hpp"
#include "caf/local_actor.hpp"
#include "caf/settings.hpp"
namespace caf::detail {
token_based_credit_controller::token_based_credit_controller(local_actor* ptr) {
namespace fallback = defaults::stream::token_policy;
// Initialize from the config parameters.
auto& cfg = ptr->system().config();
if (auto section = get_if<settings>(&cfg, "caf.stream.token-based-policy")) {
batch_size_ = get_or(*section, "batch-size", fallback::batch_size);
buffer_size_ = get_or(*section, "buffer-size", fallback::buffer_size);
} else {
batch_size_ = fallback::batch_size;
buffer_size_ = fallback::buffer_size;
}
}
token_based_credit_controller::~token_based_credit_controller() {
// nop
}
void token_based_credit_controller::before_processing(downstream_msg::batch&) {
// nop
}
credit_controller::calibration token_based_credit_controller::init() {
return calibrate();
}
credit_controller::calibration token_based_credit_controller::calibrate() {
return {buffer_size_, batch_size_, std::numeric_limits<int32_t>::max()};
}
} // namespace caf::detail
......@@ -41,7 +41,7 @@ downstream_manager::downstream_manager::path_predicate::~path_predicate() {
downstream_manager::downstream_manager(stream_manager* parent)
: parent_(parent) {
// nop
last_send_ = parent->self()->now();
}
downstream_manager::~downstream_manager() {
......@@ -62,6 +62,13 @@ bool downstream_manager::terminal() const noexcept {
return true;
}
// -- time management ----------------------------------------------------------
void downstream_manager::tick(time_point now, timespan max_batch_delay) {
if (now >= last_send_ + max_batch_delay && buffered() > 0)
force_emit_batches();
}
// -- path management ----------------------------------------------------------
std::vector<stream_slot> downstream_manager::path_slots() {
......@@ -199,10 +206,6 @@ size_t downstream_manager::buffered(stream_slot) const noexcept {
return 0;
}
int32_t downstream_manager::max_capacity() const noexcept {
return std::numeric_limits<int32_t>::max();
}
bool downstream_manager::stalled() const noexcept {
auto no_credit = [](const outbound_path& x) {
return x.open_credit == 0;
......
......@@ -20,58 +20,33 @@
#include "caf/actor_system_config.hpp"
#include "caf/defaults.hpp"
#include "caf/detail/complexity_based_credit_controller.hpp"
#include "caf/detail/meta_object.hpp"
#include "caf/detail/size_based_credit_controller.hpp"
#include "caf/detail/test_credit_controller.hpp"
#include "caf/logger.hpp"
#include "caf/no_stages.hpp"
#include "caf/scheduled_actor.hpp"
#include "caf/send.hpp"
#include "caf/settings.hpp"
namespace caf {
namespace {
constexpr bool force_ack = true;
void emit_ack_batch(inbound_path& path, credit_controller::assignment x,
bool force_ack_msg = false) {
CAF_ASSERT(x.batch_size > 0);
auto& out = path.mgr->out();
path.desired_batch_size = x.batch_size;
int32_t new_credit = 0;
auto used = static_cast<int32_t>(out.buffered()) + path.assigned_credit;
auto guard = detail::make_scope_guard([&] {
if (!force_ack_msg || path.up_to_date())
return;
unsafe_send_as(path.self(), path.hdl,
make<upstream_msg::ack_batch>(
path.slots.invert(), path.self()->address(), 0,
x.batch_size, path.last_batch_id, x.credit));
path.last_acked_batch_id = path.last_batch_id;
});
if (x.credit <= used)
return;
new_credit = path.mgr->acquire_credit(&path, x.credit - used);
if (new_credit < 1)
return;
guard.disable();
unsafe_send_as(path.self(), path.hdl,
make<upstream_msg::ack_batch>(
path.slots.invert(), path.self()->address(), new_credit,
x.batch_size, path.last_batch_id, x.credit));
path.last_acked_batch_id = path.last_batch_id;
path.assigned_credit += new_credit;
template <class T>
void set_controller(std::unique_ptr<caf::credit_controller>& ptr,
caf::local_actor* self) {
ptr = std::make_unique<T>(self);
}
} // namespace
inbound_path::inbound_path(stream_manager_ptr mgr_ptr, stream_slots id,
strong_actor_ptr ptr,
[[maybe_unused]] type_id_t in_type)
: mgr(std::move(mgr_ptr)), hdl(std::move(ptr)), slots(id) {
namespace caf {
// -- constructors, destructors, and assignment operators ----------------------
inbound_path::inbound_path(stream_manager* ptr, type_id_t in_type) : mgr(ptr) {
// Note: we can't include stream_manager.hpp in the header of inbound_path,
// because that would cause a circular reference. Hence, we also can't use an
// intrusive_ptr as member and instead call `ref/deref` manually.
CAF_ASSERT(mgr != nullptr);
mgr->ref();
auto self = mgr->self();
auto [processed_elements, input_buffer_size]
= self->inbound_stream_metrics(in_type);
......@@ -79,59 +54,96 @@ inbound_path::inbound_path(stream_manager_ptr mgr_ptr, stream_slots id,
mgr->register_input_path(this);
CAF_STREAM_LOG_DEBUG(self->name()
<< "opens input stream with element type"
<< detail::global_meta_object(in_type)->type_name
<< "at slot" << id.receiver << "from" << hdl);
if (auto str = get_if<std::string>(&self->system().config(),
"caf.stream.credit-policy")) {
if (*str == "testing")
controller_.reset(new detail::test_credit_controller(self));
else if (*str == "size")
controller_.reset(new detail::size_based_credit_controller(self));
else
controller_.reset(new detail::complexity_based_credit_controller(self));
} else {
controller_.reset(new detail::complexity_based_credit_controller(self));
}
<< detail::global_meta_object(in_type)->type_name);
last_ack_time = self->now();
}
inbound_path::~inbound_path() {
mgr->deregister_input_path(this);
mgr->deref();
}
void inbound_path::init(strong_actor_ptr source_hdl, stream_slots id) {
hdl = std::move(source_hdl);
slots = id;
}
// -- properties ---------------------------------------------------------------
bool inbound_path::up_to_date() const noexcept {
return last_acked_batch_id == last_batch_id;
}
scheduled_actor* inbound_path::self() const noexcept {
return mgr->self();
}
int32_t inbound_path::available_credit() const noexcept {
// The max_credit may decrease as a result of re-calibration, in which case
// the source can have more than the maximum amount for a brief period.
return std::max(max_credit - assigned_credit, int32_t{0});
}
const settings& inbound_path::config() const noexcept {
return content(mgr->self()->config());
}
// -- callbacks ----------------------------------------------------------------
void inbound_path::handle(downstream_msg::batch& batch) {
CAF_LOG_TRACE(CAF_ARG(slots) << CAF_ARG(batch));
// Handle batch.
auto batch_size = batch.xs_size;
last_batch_id = batch.id;
CAF_STREAM_LOG_DEBUG(self()->name() << "handles batch of size" << batch_size
<< "on slot" << slots.receiver << "with"
<< assigned_credit << "assigned credit");
assigned_credit -= batch_size;
CAF_ASSERT(assigned_credit >= 0);
controller_->before_processing(batch);
mgr->handle(this, batch);
// Update settings as necessary.
if (--calibration_countdown == 0) {
auto [cmax, bsize, countdown] = controller_->calibrate();
max_credit = cmax;
desired_batch_size = bsize;
calibration_countdown = countdown;
}
// Send ACK whenever we can assign credit for another batch to the source.
if (auto available = available_credit(); available >= desired_batch_size)
if (auto acquired = mgr->acquire_credit(this, available); acquired > 0)
emit_ack_batch(self(), acquired);
}
void inbound_path::handle(downstream_msg::batch& x) {
CAF_LOG_TRACE(CAF_ARG(slots) << CAF_ARG(x));
auto batch_size = x.xs_size;
last_batch_id = x.id;
CAF_STREAM_LOG_DEBUG(mgr->self()->name() << "handles batch of size"
<< batch_size << "on slot" << slots.receiver << "with"
<< assigned_credit << "assigned credit");
if (assigned_credit <= batch_size) {
assigned_credit = 0;
// Do not log a message when "running out of credit" for the first batch
// that can easily consume the initial credit in one shot.
CAF_STREAM_LOG_DEBUG_IF(next_credit_decision.time_since_epoch().count() > 0,
mgr->self()->name()
<< "ran out of credit at slot" << slots.receiver);
} else {
assigned_credit -= batch_size;
CAF_ASSERT(assigned_credit >= 0);
void inbound_path::tick(time_point now, duration_type max_batch_delay) {
if (now >= last_ack_time + (2 * max_batch_delay)) {
int32_t new_credit = 0;
if (auto available = available_credit(); available > 0)
new_credit = mgr->acquire_credit(this, available);
emit_ack_batch(self(), new_credit);
}
auto threshold = controller_->threshold();
if (threshold >= 0 && assigned_credit <= threshold)
caf::emit_ack_batch(*this, controller_->compute_bridge());
controller_->before_processing(x);
}
void inbound_path::handle(downstream_msg::close& x) {
mgr->handle(this, x);
}
void inbound_path::handle(downstream_msg::forced_close& x) {
mgr->handle(this, x);
controller_->after_processing(x);
mgr->push();
}
// -- messaging ----------------------------------------------------------------
void inbound_path::emit_ack_open(local_actor* self, actor_addr rebind_from) {
CAF_LOG_TRACE(CAF_ARG(slots) << CAF_ARG(rebind_from));
// Update state.
auto initial = controller_->compute_initial();
assigned_credit = mgr->acquire_credit(this, initial.credit);
CAF_ASSERT(assigned_credit >= 0);
desired_batch_size = std::min(initial.batch_size, assigned_credit);
auto [cmax, bsize, countdown] = controller_->init();
max_credit = cmax;
desired_batch_size = bsize;
calibration_countdown = countdown;
if (auto available = available_credit(); available > 0)
if (auto acquired = mgr->acquire_credit(this, available); acquired > 0)
assigned_credit += acquired;
// Make sure we receive errors from this point on.
stream_aborter::add(hdl, self->address(), slots.receiver,
stream_aborter::source_aborter);
......@@ -140,21 +152,21 @@ void inbound_path::emit_ack_open(local_actor* self, actor_addr rebind_from) {
make<upstream_msg::ack_open>(
slots.invert(), self->address(), std::move(rebind_from),
self->ctrl(), assigned_credit, desired_batch_size));
last_credit_decision = self->clock().now();
last_ack_time = self->now();
}
void inbound_path::emit_ack_batch(local_actor*, int32_t,
actor_clock::time_point now, timespan cycle) {
CAF_LOG_TRACE(CAF_ARG(slots) << CAF_ARG(cycle));
last_credit_decision = now;
next_credit_decision = now + cycle;
auto max_capacity = static_cast<int32_t>(mgr->out().max_capacity());
caf::emit_ack_batch(*this, controller_->compute(cycle, max_capacity),
force_ack);
}
bool inbound_path::up_to_date() {
return last_acked_batch_id == last_batch_id;
void inbound_path::emit_ack_batch(local_actor* self, int32_t new_credit) {
CAF_LOG_TRACE(CAF_ARG(new_credit));
CAF_ASSERT(desired_batch_size > 0);
if (last_acked_batch_id == last_batch_id && new_credit == 0)
return;
unsafe_send_as(self, hdl,
make<upstream_msg::ack_batch>(slots.invert(), self->address(),
new_credit, desired_batch_size,
last_batch_id));
last_acked_batch_id = last_batch_id;
assigned_credit += new_credit;
last_ack_time = self->now();
}
void inbound_path::emit_regular_shutdown(local_actor* self) {
......@@ -185,8 +197,4 @@ void inbound_path::emit_irregular_shutdown(local_actor* self,
std::move(reason)));
}
scheduled_actor* inbound_path::self() {
return mgr->self();
}
} // namespace caf
......@@ -94,6 +94,10 @@ void local_actor::setup_metrics() {
metrics_ = make_instance_metrics(this);
}
auto local_actor::now() const noexcept -> clock_type::time_point {
return clock().now();
}
void local_actor::request_response_timeout(timespan timeout, message_id mid) {
CAF_LOG_TRACE(CAF_ARG(timeout) << CAF_ARG(mid));
if (timeout == infinite)
......
......@@ -34,14 +34,13 @@ constexpr int32_t max_batch_size = 128 * 1024;
outbound_path::outbound_path(stream_slot sender_slot,
strong_actor_ptr receiver_hdl)
: slots(sender_slot, invalid_stream_slot),
hdl(std::move(receiver_hdl)),
next_batch_id(1),
open_credit(0),
desired_batch_size(50),
next_ack_id(1),
max_capacity(0),
closing(false) {
: slots(sender_slot, invalid_stream_slot),
hdl(std::move(receiver_hdl)),
next_batch_id(1),
open_credit(0),
desired_batch_size(50),
next_ack_id(1),
closing(false) {
// nop
}
......
......@@ -22,6 +22,7 @@
#include "caf/downstream_msg.hpp"
#include "caf/inbound_path.hpp"
#include "caf/logger.hpp"
#include "caf/stream_manager.hpp"
namespace caf::policy {
......
......@@ -21,6 +21,7 @@
#include "caf/actor_ostream.hpp"
#include "caf/actor_system_config.hpp"
#include "caf/config.hpp"
#include "caf/defaults.hpp"
#include "caf/detail/default_invoke_result_visitor.hpp"
#include "caf/detail/meta_object.hpp"
#include "caf/detail/private_thread.hpp"
......@@ -136,19 +137,8 @@ scheduled_actor::scheduled_actor(actor_config& cfg)
#endif // CAF_ENABLE_EXCEPTIONS
{
auto& sys_cfg = home_system().config();
auto interval = sys_cfg.stream_tick_duration();
CAF_ASSERT(interval.count() > 0);
stream_ticks_.interval(interval);
CAF_ASSERT(sys_cfg.stream_max_batch_delay.count() > 0);
auto div = [](timespan x, timespan y) {
return static_cast<size_t>(x.count() / y.count());
};
max_batch_delay_ticks_ = div(sys_cfg.stream_max_batch_delay, interval);
CAF_ASSERT(max_batch_delay_ticks_ > 0);
credit_round_ticks_ = div(sys_cfg.stream_credit_round_interval, interval);
CAF_ASSERT(credit_round_ticks_ > 0);
CAF_LOG_DEBUG(CAF_ARG(interval) << CAF_ARG(max_batch_delay_ticks_)
<< CAF_ARG(credit_round_ticks_));
max_batch_delay_ = get_or(sys_cfg, "caf.stream.max_batch_delay",
defaults::stream::max_batch_delay);
}
scheduled_actor::~scheduled_actor() {
......@@ -288,127 +278,17 @@ void scheduled_actor::intrusive_ptr_release_impl() {
intrusive_ptr_release(ctrl());
}
namespace {
// TODO: replace with generic lambda when switching to C++14
struct upstream_msg_visitor {
scheduled_actor* selfptr;
upstream_msg& um;
template <class T>
void operator()(T& x) {
selfptr->handle_upstream_msg(um.slots, um.sender, x);
}
};
} // namespace
intrusive::task_result
scheduled_actor::mailbox_visitor::operator()(size_t, upstream_queue&,
mailbox_element& x) {
CAF_ASSERT(x.content().match_elements<upstream_msg>());
return run(x, [&] {
self->current_mailbox_element(&x);
CAF_LOG_RECEIVE_EVENT((&x));
CAF_BEFORE_PROCESSING(self, x);
auto& um = x.content().get_mutable_as<upstream_msg>(0);
upstream_msg_visitor f{self, um};
visit(f, um.content);
CAF_AFTER_PROCESSING(self, invoke_message_result::consumed);
return ++handled_msgs < max_throughput ? intrusive::task_result::resume
: intrusive::task_result::stop_all;
});
}
intrusive::task_result scheduled_actor::mailbox_visitor::operator()(
size_t, downstream_queue& qs, stream_slot,
policy::downstream_messages::nested_queue_type& q, mailbox_element& x) {
CAF_LOG_TRACE(CAF_ARG(x) << CAF_ARG(handled_msgs));
return run(x, [&, this] {
self->current_mailbox_element(&x);
CAF_LOG_RECEIVE_EVENT((&x));
CAF_BEFORE_PROCESSING(self, x);
CAF_ASSERT(x.content().match_elements<downstream_msg>());
auto& dm = x.content().get_mutable_as<downstream_msg>(0);
auto f = [&, this](auto& content) {
using content_type = std::decay_t<decltype(content)>;
auto& inptr = q.policy().handler;
if (inptr == nullptr)
return intrusive::task_result::stop;
if (auto processed_elements = inptr->metrics.processed_elements) {
auto num_elements = q.policy().task_size(content);
auto input_buffer_size = inptr->metrics.input_buffer_size;
CAF_ASSERT(input_buffer_size != nullptr);
processed_elements->inc(num_elements);
input_buffer_size->dec(num_elements);
}
// Do *not* store a reference here since we potentially reset `inptr`.
auto mgr = inptr->mgr;
inptr->handle(content);
// The sender slot can be 0. This is the case for forced_close or
// forced_drop messages from stream aborters.
CAF_ASSERT(inptr->slots == dm.slots
|| (dm.slots.sender == 0
&& dm.slots.receiver == inptr->slots.receiver));
if constexpr (std::is_same<content_type, downstream_msg::close>::value
|| std::is_same<content_type,
downstream_msg::forced_close>::value) {
if (auto input_buffer_size = inptr->metrics.input_buffer_size)
input_buffer_size->dec(q.total_task_size());
inptr.reset();
qs.erase_later(dm.slots.receiver);
self->erase_stream_manager(dm.slots.receiver);
if (mgr->done()) {
CAF_LOG_DEBUG("path is done receiving and closes its manager");
self->erase_stream_manager(mgr);
mgr->stop();
}
return intrusive::task_result::stop;
} else if (mgr->done()) {
CAF_LOG_DEBUG("path is done receiving and closes its manager");
self->erase_stream_manager(mgr);
mgr->stop();
return intrusive::task_result::stop;
}
return intrusive::task_result::resume;
};
auto res = visit(f, dm.content);
CAF_AFTER_PROCESSING(self, invoke_message_result::consumed);
return ++handled_msgs < max_throughput ? res
: intrusive::task_result::stop_all;
});
}
intrusive::task_result
scheduled_actor::mailbox_visitor::operator()(mailbox_element& x) {
CAF_LOG_TRACE(CAF_ARG(x) << CAF_ARG(handled_msgs));
return run(x, [&, this] {
switch (self->reactivate(x)) {
case activation_result::terminated:
return intrusive::task_result::stop;
case activation_result::success:
return ++handled_msgs < max_throughput
? intrusive::task_result::resume
: intrusive::task_result::stop_all;
case activation_result::skipped:
return intrusive::task_result::skip;
default:
return intrusive::task_result::resume;
}
});
}
resumable::resume_result scheduled_actor::resume(execution_unit* ctx,
size_t max_throughput) {
CAF_PUSH_AID(id());
CAF_LOG_TRACE(CAF_ARG(max_throughput));
if (!activate(ctx))
return resumable::done;
size_t handled_msgs = 0;
size_t consumed = 0;
actor_clock::time_point tout{actor_clock::duration_type{0}};
auto reset_timeouts_if_needed = [&] {
// Set a new receive timeout if we called our behavior at least once.
if (handled_msgs > 0)
if (consumed > 0)
set_receive_timeout();
// Set a new stream timeout.
if (!stream_managers_.empty()) {
......@@ -418,30 +298,153 @@ resumable::resume_result scheduled_actor::resume(execution_unit* ctx,
set_stream_timeout(tout);
}
};
mailbox_visitor f{this, handled_msgs, max_throughput,
getf(abstract_actor::collects_metrics_flag)};
// Callback for handling urgent and normal messages.
auto handle_async = [this, max_throughput, &consumed](mailbox_element& x) {
return run_with_metrics(x, [this, max_throughput, &consumed, &x] {
switch (reactivate(x)) {
case activation_result::terminated:
return intrusive::task_result::stop;
case activation_result::success:
return ++consumed < max_throughput ? intrusive::task_result::resume
: intrusive::task_result::stop_all;
case activation_result::skipped:
return intrusive::task_result::skip;
default:
return intrusive::task_result::resume;
}
});
};
// Callback for handling upstream messages (e.g., ACKs).
auto handle_umsg = [this, max_throughput, &consumed](mailbox_element& x) {
return run_with_metrics(x, [this, max_throughput, &consumed, &x] {
current_mailbox_element(&x);
CAF_LOG_RECEIVE_EVENT((&x));
CAF_BEFORE_PROCESSING(this, x);
CAF_ASSERT(x.content().match_elements<upstream_msg>());
auto& um = x.content().get_mutable_as<upstream_msg>(0);
auto f = [&](auto& content) {
handle_upstream_msg(um.slots, um.sender, content);
};
visit(f, um.content);
CAF_AFTER_PROCESSING(this, invoke_message_result::consumed);
return ++consumed < max_throughput ? intrusive::task_result::resume
: intrusive::task_result::stop_all;
});
};
// Callback for handling downstream messages (e.g., batches).
auto handle_dmsg = [this, &consumed, max_throughput](stream_slot, auto& q,
mailbox_element& x) {
return run_with_metrics(x, [this, max_throughput, &consumed, &q, &x] {
current_mailbox_element(&x);
CAF_LOG_RECEIVE_EVENT((&x));
CAF_BEFORE_PROCESSING(this, x);
CAF_ASSERT(x.content().match_elements<downstream_msg>());
auto& dm = x.content().get_mutable_as<downstream_msg>(0);
auto f = [&, this](auto& content) {
using content_type = std::decay_t<decltype(content)>;
auto& inptr = q.policy().handler;
if (inptr == nullptr)
return intrusive::task_result::stop;
if (auto processed_elements = inptr->metrics.processed_elements) {
auto num_elements = q.policy().task_size(content);
auto input_buffer_size = inptr->metrics.input_buffer_size;
CAF_ASSERT(input_buffer_size != nullptr);
processed_elements->inc(num_elements);
input_buffer_size->dec(num_elements);
}
// Hold onto a strong reference since we might reset `inptr`.
auto mgr = stream_manager_ptr{inptr->mgr};
inptr->handle(content);
// The sender slot can be 0. This is the case for forced_close or
// forced_drop messages from stream aborters.
CAF_ASSERT(inptr->slots == dm.slots
|| (dm.slots.sender == 0
&& dm.slots.receiver == inptr->slots.receiver));
if constexpr (std::is_same<content_type, downstream_msg::close>::value
|| std::is_same<content_type,
downstream_msg::forced_close>::value) {
if (auto input_buffer_size = inptr->metrics.input_buffer_size)
input_buffer_size->dec(q.total_task_size());
inptr.reset();
get_downstream_queue().erase_later(dm.slots.receiver);
erase_stream_manager(dm.slots.receiver);
if (mgr->done()) {
CAF_LOG_DEBUG("path is done receiving and closes its manager");
erase_stream_manager(mgr);
mgr->stop();
}
return intrusive::task_result::stop;
} else if (mgr->done()) {
CAF_LOG_DEBUG("path is done receiving and closes its manager");
erase_stream_manager(mgr);
mgr->stop();
return intrusive::task_result::stop;
}
return intrusive::task_result::resume;
};
auto res = visit(f, dm.content);
CAF_AFTER_PROCESSING(this, invoke_message_result::consumed);
return ++consumed < max_throughput ? res
: intrusive::task_result::stop_all;
});
};
std::vector<stream_manager*> managers;
mailbox_element_ptr ptr;
// Timeout for calling `advance_streams`.
while (handled_msgs < max_throughput) {
while (consumed < max_throughput) {
CAF_LOG_DEBUG("start new DRR round");
mailbox_.fetch_more();
auto prev = consumed; // Caches the value before processing more.
// TODO: maybe replace '3' with configurable / adaptive value?
// Dispatch on the different message categories in our mailbox.
auto consumed = mailbox_.new_round(3, f).consumed_items;
if (consumed == 0) {
static constexpr size_t quantum = 3;
// Dispatch urgent and normal (asynchronous) messages.
get_urgent_queue().new_round(quantum * 3, handle_async);
get_normal_queue().new_round(quantum, handle_async);
// Consume all upstream messages. They are lightweight by design and ACKs
// come with new credit, allowing us to advance stream traffic.
if (auto tts = get_upstream_queue().total_task_size(); tts > 0) {
get_upstream_queue().new_round(tts, handle_umsg);
// After processing ACKs, we may have new credit that enables us to ship
// some batches from our output buffers. This step also may re-enable
// inbound paths by draining output buffers here.
active_stream_managers(managers);
for (auto mgr : managers)
mgr->push();
}
// Note: a quantum of 1 means "1 batch" for this queue.
if (get_downstream_queue().new_round(quantum, handle_dmsg).consumed_items
> 0) {
do {
// Processing batches, enables stages to push more data downstream. This
// in turn may allow the stage to process more batches again. Hence the
// loop. By not giving additional quanta, we simply allow the stage to
// consume what it was allowed to in the first place.
active_stream_managers(managers);
for (auto mgr : managers)
mgr->push();
} while (
consumed < max_throughput
&& get_downstream_queue().new_round(0, handle_dmsg).consumed_items > 0);
}
// Update metrics or try returning if the actor consumed nothing.
auto delta = consumed - prev;
CAF_LOG_DEBUG("consumed" << delta << "messages this round");
if (delta > 0) {
auto signed_val = static_cast<int64_t>(delta);
home_system().base_metrics().processed_messages->inc(signed_val);
} else {
reset_timeouts_if_needed();
if (mailbox().try_block())
return resumable::awaiting_message;
} else {
auto signed_val = static_cast<int64_t>(consumed);
home_system().base_metrics().processed_messages->inc(signed_val);
CAF_LOG_DEBUG("mailbox().try_block() returned false");
}
// Check whether the visitor left the actor without behavior.
if (finalize()) {
CAF_LOG_DEBUG("allow stream managers to send batches");
active_stream_managers(managers);
for (auto mgr : managers)
mgr->push();
CAF_LOG_DEBUG("check for shutdown or advance streams");
if (finalize())
return resumable::done;
}
// Advance streams, i.e., try to generating credit or to emit batches.
auto now = clock().now();
if (now >= tout)
if (auto now = clock().now(); now >= tout)
tout = advance_streams(now);
}
CAF_LOG_DEBUG("max throughput reached");
......@@ -911,8 +914,6 @@ bool scheduled_actor::finalize() {
i = stream_managers_.erase(i);
else
++i;
if (stream_managers_.empty())
stream_ticks_.stop();
}
}
// An actor is considered alive as long as it has a behavior and didn't set
......@@ -944,6 +945,10 @@ void scheduled_actor::push_to_cache(mailbox_element_ptr ptr) {
}
}
scheduled_actor::urgent_queue& scheduled_actor::get_urgent_queue() {
return get<urgent_queue_index>(mailbox_.queue().queues());
}
scheduled_actor::normal_queue& scheduled_actor::get_normal_queue() {
return get<normal_queue_index>(mailbox_.queue().queues());
}
......@@ -956,24 +961,16 @@ scheduled_actor::downstream_queue& scheduled_actor::get_downstream_queue() {
return get<downstream_queue_index>(mailbox_.queue().queues());
}
scheduled_actor::urgent_queue& scheduled_actor::get_urgent_queue() {
return get<urgent_queue_index>(mailbox_.queue().queues());
}
inbound_path* scheduled_actor::make_inbound_path(stream_manager_ptr mgr,
stream_slots slots,
strong_actor_ptr sender,
type_id_t input_type) {
bool scheduled_actor::add_inbound_path(type_id_t,
std::unique_ptr<inbound_path> path) {
static constexpr size_t queue_index = downstream_queue_index;
using policy_type = policy::downstream_messages::nested;
auto& qs = get<queue_index>(mailbox_.queue().queues()).queues();
auto res = qs.emplace(slots.receiver, policy_type{nullptr});
auto res = qs.emplace(path->slots.receiver, policy_type{nullptr});
if (!res.second)
return nullptr;
auto path = new inbound_path(std::move(mgr), slots, std::move(sender),
input_type);
res.first->second.policy().handler.reset(path);
return path;
return false;
res.first->second.policy().handler = std::move(path);
return true;
}
void scheduled_actor::erase_inbound_path_later(stream_slot slot) {
......@@ -1033,20 +1030,12 @@ void scheduled_actor::handle_upstream_msg(stream_slots slots,
CAF_IGNORE_UNUSED(sender);
CAF_LOG_TRACE(CAF_ARG(slots) << CAF_ARG(sender) << CAF_ARG(x));
CAF_ASSERT(sender == x.rebind_to);
// Get the manager for that stream, move it from `pending_managers_` to
// `managers_`, and handle `x`.
auto i = pending_stream_managers_.find(slots.receiver);
if (i == pending_stream_managers_.end()) {
if (auto [moved, ptr] = ack_pending_stream_manager(slots.receiver); moved) {
CAF_ASSERT(ptr != nullptr);
ptr->handle(slots, x);
} else {
CAF_LOG_WARNING("found no corresponding manager for received ack_open");
return;
}
auto ptr = std::move(i->second);
pending_stream_managers_.erase(i);
if (!add_stream_manager(slots.receiver, ptr)) {
CAF_LOG_WARNING("unable to add stream manager after receiving ack_open");
return;
}
ptr->handle(slots, x);
}
uint64_t scheduled_actor::set_timeout(std::string type,
......@@ -1082,8 +1071,6 @@ stream_slot scheduled_actor::next_slot() {
void scheduled_actor::assign_slot(stream_slot x, stream_manager_ptr mgr) {
CAF_LOG_TRACE(CAF_ARG(x));
if (stream_managers_.empty())
stream_ticks_.start(clock().now());
CAF_ASSERT(stream_managers_.count(x) == 0);
stream_managers_.emplace(x, std::move(mgr));
}
......@@ -1113,15 +1100,12 @@ scheduled_actor::assign_next_pending_slot_to(stream_manager_ptr mgr) {
bool scheduled_actor::add_stream_manager(stream_slot id,
stream_manager_ptr mgr) {
CAF_LOG_TRACE(CAF_ARG(id));
if (stream_managers_.empty())
stream_ticks_.start(clock().now());
return stream_managers_.emplace(id, std::move(mgr)).second;
}
void scheduled_actor::erase_stream_manager(stream_slot id) {
CAF_LOG_TRACE(CAF_ARG(id));
if (stream_managers_.erase(id) != 0 && stream_managers_.empty())
stream_ticks_.stop();
stream_managers_.erase(id);
CAF_LOG_DEBUG(CAF_ARG2("stream_managers_.size", stream_managers_.size()));
}
......@@ -1130,6 +1114,19 @@ void scheduled_actor::erase_pending_stream_manager(stream_slot id) {
pending_stream_managers_.erase(id);
}
std::pair<bool, stream_manager*>
scheduled_actor::ack_pending_stream_manager(stream_slot id) {
CAF_LOG_TRACE(CAF_ARG(id));
if (auto i = pending_stream_managers_.find(id);
i != pending_stream_managers_.end()) {
auto ptr = std::move(i->second);
auto raw_ptr = ptr.get();
pending_stream_managers_.erase(i);
return {add_stream_manager(id, std::move(ptr)), raw_ptr};
}
return {false, nullptr};
}
void scheduled_actor::erase_stream_manager(const stream_manager_ptr& mgr) {
CAF_LOG_TRACE("");
if (!stream_managers_.empty()) {
......@@ -1140,8 +1137,6 @@ void scheduled_actor::erase_stream_manager(const stream_manager_ptr& mgr) {
i = stream_managers_.erase(i);
else
++i;
if (stream_managers_.empty())
stream_ticks_.stop();
}
{ // Lifetime scope of second iterator pair.
auto i = pending_stream_managers_.begin();
......@@ -1200,41 +1195,34 @@ scheduled_actor::handle_open_stream_msg(mailbox_element& x) {
actor_clock::time_point
scheduled_actor::advance_streams(actor_clock::time_point now) {
CAF_LOG_TRACE("");
if (!stream_ticks_.started()) {
CAF_LOG_DEBUG("tick emitter not started yet");
CAF_LOG_TRACE(CAF_ARG(now));
if (stream_managers_.empty())
return actor_clock::time_point::max();
}
/// Advance time for driving forced batches and credit.
auto bitmask = stream_ticks_.timeouts(now, {max_batch_delay_ticks_,
credit_round_ticks_});
// Force batches on all output paths.
if ((bitmask & 0x01) != 0 && !stream_managers_.empty()) {
std::vector<stream_manager*> managers;
managers.reserve(stream_managers_.size());
for (auto& kvp : stream_managers_)
managers.emplace_back(kvp.second.get());
std::sort(managers.begin(), managers.end());
auto e = std::unique(managers.begin(), managers.end());
for (auto i = managers.begin(); i != e; ++i)
(*i)->out().force_emit_batches();
}
// Fill up credit on each input path.
if ((bitmask & 0x02) != 0) {
CAF_LOG_DEBUG("new credit round");
auto cycle = stream_ticks_.interval();
cycle *= static_cast<decltype(cycle)::rep>(credit_round_ticks_);
auto& qs = get_downstream_queue().queues();
for (auto& kvp : qs) {
auto inptr = kvp.second.policy().handler.get();
if (inptr != nullptr) {
auto tts = static_cast<int32_t>(kvp.second.total_task_size());
inptr->emit_ack_batch(this, tts, now, cycle);
}
}
}
return stream_ticks_.next_timeout(now, {max_batch_delay_ticks_,
credit_round_ticks_});
auto managers = active_stream_managers();
for (auto ptr : managers)
ptr->tick(now);
auto idle = [](const stream_manager* mgr) { return mgr->idle(); };
if (std::all_of(managers.begin(), managers.end(), idle))
return actor_clock::time_point::max();
return now + max_batch_delay_;
}
void scheduled_actor::active_stream_managers(std::vector<stream_manager*>& xs) {
xs.clear();
if (stream_managers_.empty())
return;
xs.reserve(stream_managers_.size());
for (auto& kvp : stream_managers_)
xs.emplace_back(kvp.second.get());
std::sort(xs.begin(), xs.end());
auto e = std::unique(xs.begin(), xs.end());
xs.erase(e, xs.end());
}
std::vector<stream_manager*> scheduled_actor::active_stream_managers() {
std::vector<stream_manager*> result;
active_stream_managers(result);
return result;
}
} // namespace caf
/******************************************************************************
* ____ _ _____ *
* / ___| / \ | ___| C++ *
* | | / _ \ | |_ Actor *
* | |___ / ___ \| _| Framework *
* \____/_/ \_|_| *
* *
* Copyright 2011-2019 Dominik Charousset *
* *
* Distributed under the terms and conditions of the BSD 3-Clause License or *
* (at your option) under the terms and conditions of the Boost Software *
* License 1.0. See accompanying files LICENSE and LICENSE_ALTERNATIVE. *
* *
* If you did not receive a copy of the license files, see *
* http://opensource.org/licenses/BSD-3-Clause and *
* http://www.boost.org/LICENSE_1_0.txt. *
******************************************************************************/
#include "caf/detail/size_based_credit_controller.hpp"
#include "caf/actor_system.hpp"
#include "caf/actor_system_config.hpp"
#include "caf/defaults.hpp"
#include "caf/detail/serialized_size.hpp"
#include "caf/scheduled_actor.hpp"
// Safe us some typing and very ugly formatting.
#define impl size_based_credit_controller
namespace caf::detail {
impl::impl(scheduled_actor* self) : super(self) {
auto& cfg = self->system().config();
buffer_capacity_ = get_or(cfg, "caf.stream.size-policy.buffer-capacity",
defaults::stream::size_policy::buffer_capacity);
bytes_per_batch_ = get_or(cfg, "caf.stream.size-policy.bytes-per-batch",
defaults::stream::size_policy::bytes_per_batch);
}
impl::~impl() {
// nop
}
void impl::before_processing(downstream_msg::batch& x) {
if (++sample_counter_ == sample_rate_) {
sampled_elements_ += x.xs_size;
sampled_total_size_ += serialized_size(self()->system(), x.xs);
sample_counter_ = 0;
}
++num_batches_;
}
void impl::after_processing(downstream_msg::batch&) {
// nop
}
credit_controller::assignment impl::compute_initial() {
return {buffer_size_, batch_size_};
}
credit_controller::assignment impl::compute(timespan, int32_t) {
if (sampled_elements_ >= min_samples) {
// Helper for truncating a 64-bit integer to a 32-bit integer with a
// minimum value of 1.
auto clamp_i32 = [](int64_t x) -> int32_t {
static constexpr auto upper_bound = std::numeric_limits<int32_t>::max();
if (x > upper_bound)
return upper_bound;
if (x <= 0)
return 1;
return static_cast<int32_t>(x);
};
// Calculate ideal batch size by size.
auto bytes_per_element = clamp_i32(sampled_total_size_ / sampled_elements_);
batch_size_ = clamp_i32(bytes_per_batch_ / bytes_per_element);
buffer_size_ = clamp_i32(buffer_capacity_ / bytes_per_element);
// Reset bookkeeping state.
sampled_elements_ = 0;
sampled_total_size_ = 0;
// Adjust the sample rate to reach min_samples in the next cycle.
sample_rate_ = clamp_i32(num_batches_ / min_samples);
if (sample_counter_ >= sample_rate_)
sample_counter_ = 0;
num_batches_ = 0;
}
return {buffer_size_, batch_size_};
}
credit_controller::assignment impl::compute_bridge() {
CAF_ASSERT(batch_size_ > 0);
CAF_ASSERT(buffer_size_ > batch_size_);
return {buffer_size_, batch_size_};
}
int32_t impl::threshold() const noexcept {
return static_cast<int32_t>(buffer_size_ * buffer_threshold);
}
} // namespace caf::detail
......@@ -23,6 +23,7 @@
#include "caf/actor_control_block.hpp"
#include "caf/actor_system.hpp"
#include "caf/actor_system_config.hpp"
#include "caf/defaults.hpp"
#include "caf/error.hpp"
#include "caf/expected.hpp"
#include "caf/inbound_path.hpp"
......@@ -37,7 +38,9 @@ namespace caf {
stream_manager::stream_manager(scheduled_actor* selfptr, stream_priority prio)
: self_(selfptr), pending_handshakes_(0), priority_(prio), flags_(0) {
// nop
auto& cfg = selfptr->config();
max_batch_delay_ = get_or(cfg, "caf.stream.max-batch-delay",
defaults::stream::max_batch_delay);
}
stream_manager::~stream_manager() {
......@@ -89,25 +92,20 @@ bool stream_manager::handle(stream_slots slots, upstream_msg::ack_open& x) {
CAF_ASSERT(ptr->open_credit >= 0);
ptr->set_desired_batch_size(x.desired_batch_size);
--pending_handshakes_;
push();
return true;
}
void stream_manager::handle(stream_slots slots, upstream_msg::ack_batch& x) {
CAF_LOG_TRACE(CAF_ARG(slots) << CAF_ARG(x));
CAF_ASSERT(x.desired_batch_size > 0);
auto path = out().path(slots.receiver);
if (path != nullptr) {
if (auto path = out().path(slots.receiver); path != nullptr) {
path->open_credit += x.new_capacity;
path->max_capacity = x.max_capacity;
CAF_ASSERT(path->open_credit >= 0);
CAF_ASSERT(path->max_capacity >= 0);
path->set_desired_batch_size(x.desired_batch_size);
path->next_ack_id = x.acknowledged_id + 1;
// Gravefully remove path after receiving its final ACK.
if (path->closing && out().clean(slots.receiver))
out().remove_path(slots.receiver, none, false);
push();
}
}
......@@ -149,28 +147,6 @@ void stream_manager::shutdown() {
ipath->emit_regular_shutdown(self_);
}
void stream_manager::advance() {
CAF_LOG_TRACE("");
// Try to emit more credit.
if (!inbound_paths_.empty()) {
auto now = self_->clock().now();
auto& cfg = self_->system().config();
auto interval = cfg.stream_credit_round_interval;
auto& qs = self_->get_downstream_queue().queues();
// Iterate all queues for inbound traffic.
for (auto& kvp : qs) {
auto inptr = kvp.second.policy().handler.get();
// Ignore inbound paths of other managers.
if (inptr->mgr.get() == this) {
auto tts = static_cast<int32_t>(kvp.second.total_task_size());
inptr->emit_ack_batch(self_, tts, now, interval);
}
}
}
// Try to generate more batches.
push();
}
void stream_manager::push() {
CAF_LOG_TRACE("");
do {
......@@ -179,7 +155,7 @@ void stream_manager::push() {
}
bool stream_manager::congested() const noexcept {
return false;
return out().capacity() == 0;
}
void stream_manager::deliver_handshake(response_promise& rp, stream_slot slot,
......@@ -288,7 +264,8 @@ stream_manager::add_unchecked_outbound_path_impl(message handshake) {
}
stream_slot
stream_manager::add_unchecked_inbound_path_impl(type_id_t input_type) {
stream_manager::add_unchecked_inbound_path_impl(type_id_t input_type,
inbound_path_ptr ptr) {
CAF_LOG_TRACE("");
auto x = self_->current_mailbox_element();
if (x == nullptr || !x->content().match_elements<open_stream_msg>()) {
......@@ -311,13 +288,22 @@ stream_manager::add_unchecked_inbound_path_impl(type_id_t input_type) {
}
auto slot = assign_next_slot();
stream_slots path_id{osm.slot, slot};
auto ptr = self_->make_inbound_path(this, path_id, std::move(osm.prev_stage),
input_type);
CAF_ASSERT(ptr != nullptr);
ptr->emit_ack_open(self_, actor_cast<actor_addr>(osm.original_stage));
auto raw_ptr = ptr.get();
ptr->init(std::move(osm.prev_stage), path_id);
if (!self_->add_inbound_path(input_type, std::move(ptr)))
return invalid_stream_slot;
raw_ptr->emit_ack_open(self_, actor_cast<actor_addr>(osm.original_stage));
return slot;
}
void stream_manager::tick(time_point now) {
do {
out().tick(now, max_batch_delay_);
for (auto path : inbound_paths_)
path->tick(now, max_batch_delay_);
} while (generate_messages());
}
stream_slot stream_manager::assign_next_slot() {
return self_->assign_next_slot_to(this);
}
......
/******************************************************************************
* ____ _ _____ *
* / ___| / \ | ___| C++ *
* | | / _ \ | |_ Actor *
* | |___ / ___ \| _| Framework *
* \____/_/ \_|_| *
* *
* Copyright 2011-2019 Dominik Charousset *
* *
* Distributed under the terms and conditions of the BSD 3-Clause License or *
* (at your option) under the terms and conditions of the Boost Software *
* License 1.0. See accompanying files LICENSE and LICENSE_ALTERNATIVE. *
* *
* If you did not receive a copy of the license files, see *
* http://opensource.org/licenses/BSD-3-Clause and *
* http://www.boost.org/LICENSE_1_0.txt. *
******************************************************************************/
#include "caf/detail/test_credit_controller.hpp"
#include "caf/actor_system.hpp"
#include "caf/actor_system_config.hpp"
#include "caf/scheduled_actor.hpp"
namespace caf {
namespace detail {
test_credit_controller::~test_credit_controller() {
// nop
}
void test_credit_controller::before_processing(downstream_msg::batch& x) {
num_elements_ += x.xs_size;
}
void test_credit_controller::after_processing(downstream_msg::batch&) {
// nop
}
credit_controller::assignment test_credit_controller::compute_initial() {
return {50, 50};
}
credit_controller::assignment
test_credit_controller::compute(timespan cycle, int32_t) {
auto& cfg = self()->system().config();
auto complexity = cfg.stream_desired_batch_complexity;
// Max throughput = C * (N / t), where C = cycle length, N = measured items,
// and t = measured time. Desired batch size is the same formula with D
// (desired complexity) instead of C. We compute our values in 64-bit for
// more precision before truncating to a 32-bit integer type at the end.
int64_t total_ns = num_elements_ * 1000; // calculate with 1us per element
if (total_ns == 0)
return {1, 1};
// Helper for truncating a 64-bit integer to a 32-bit integer with a minimum
// value of 1.
auto clamp = [](int64_t x) -> int32_t {
static constexpr auto upper_bound = std::numeric_limits<int32_t>::max();
if (x > upper_bound)
return upper_bound;
if (x <= 0)
return 1;
return static_cast<int32_t>(x);
};
// Instead of C * (N / t) we calculate (C * N) / t to avoid double conversion
// and rounding errors.
assignment result;
result.credit = clamp((cycle.count() * num_elements_) / total_ns);
result.batch_size = clamp((complexity.count() * num_elements_) / total_ns);
// Reset state and return.
num_elements_ = 0;
return result;
}
} // namespace detail
} // namespace caf
......@@ -49,37 +49,39 @@ TESTEE_STATE(file_reader) {
};
VARARGS_TESTEE(file_reader, size_t buf_size) {
return {[=](string& fname) -> result<stream<int32_t>, string> {
CAF_CHECK_EQUAL(fname, "numbers.txt");
CAF_CHECK_EQUAL(self->mailbox().empty(), true);
return attach_stream_source(
self,
// forward file name in handshake to next stage
std::forward_as_tuple(std::move(fname)),
// initialize state
[=](unit_t&) {
auto& xs = self->state.buf;
xs.resize(buf_size);
std::iota(xs.begin(), xs.end(), 1);
},
// get next element
[=](unit_t&, downstream<int32_t>& out, size_t num) {
auto& xs = self->state.buf;
CAF_MESSAGE("push " << num << " messages downstream");
auto n = std::min(num, xs.size());
for (size_t i = 0; i < n; ++i)
out.push(xs[i]);
xs.erase(xs.begin(), xs.begin() + static_cast<ptrdiff_t>(n));
},
// check whether we reached the end
[=](const unit_t&) {
if (self->state.buf.empty()) {
CAF_MESSAGE(self->name() << " is done");
return true;
}
return false;
});
}};
return {
[=](string& fname) -> result<stream<int32_t>, string> {
CAF_CHECK_EQUAL(fname, "numbers.txt");
CAF_CHECK_EQUAL(self->mailbox().empty(), true);
return attach_stream_source(
self,
// forward file name in handshake to next stage
std::forward_as_tuple(std::move(fname)),
// initialize state
[=](unit_t&) {
auto& xs = self->state.buf;
xs.resize(buf_size);
std::iota(xs.begin(), xs.end(), 1);
},
// get next element
[=](unit_t&, downstream<int32_t>& out, size_t num) {
auto& xs = self->state.buf;
CAF_MESSAGE("push " << num << " messages downstream");
auto n = std::min(num, xs.size());
for (size_t i = 0; i < n; ++i)
out.push(xs[i]);
xs.erase(xs.begin(), xs.begin() + static_cast<ptrdiff_t>(n));
},
// check whether we reached the end
[=](const unit_t&) {
if (self->state.buf.empty()) {
CAF_MESSAGE(self->name() << " is done");
return true;
}
return false;
});
},
};
}
TESTEE_STATE(sum_up) {
......@@ -87,26 +89,28 @@ TESTEE_STATE(sum_up) {
};
TESTEE(sum_up) {
return {[=](stream<int32_t>& in, const string& fname) {
CAF_CHECK_EQUAL(fname, "numbers.txt");
using int_ptr = int32_t*;
return attach_stream_sink(
self,
// input stream
in,
// initialize state
[=](int_ptr& x) { x = &self->state.x; },
// processing step
[](int_ptr& x, int32_t y) { *x += y; },
// cleanup
[=](int_ptr&, const error&) {
CAF_MESSAGE(self->name() << " is done");
});
},
[=](join_atom atm, actor src) {
CAF_MESSAGE(self->name() << " joins a stream");
self->send(self * src, atm);
}};
return {
[=](stream<int32_t>& in, const string& fname) {
CAF_CHECK_EQUAL(fname, "numbers.txt");
using int_ptr = int32_t*;
return attach_stream_sink(
self,
// input stream
in,
// initialize state
[=](int_ptr& x) { x = &self->state.x; },
// processing step
[](int_ptr& x, int32_t y) { *x += y; },
// cleanup
[=](int_ptr&, const error&) {
CAF_MESSAGE(self->name() << " is done");
});
},
[=](join_atom atm, actor src) {
CAF_MESSAGE(self->name() << " joins a stream");
self->send(self * src, atm);
},
};
}
TESTEE_STATE(stream_multiplexer) {
......@@ -150,7 +154,7 @@ using fixture = test_coordinator_fixture<>;
CAF_TEST_FIXTURE_SCOPE(local_streaming_tests, fixture)
CAF_TEST(depth_3_pipeline_with_fork) {
auto src = sys.spawn(file_reader, 50u);
auto src = sys.spawn(file_reader, 60u);
auto stg = sys.spawn(stream_multiplexer);
auto snk1 = sys.spawn(sum_up);
auto snk2 = sys.spawn(sum_up);
......@@ -168,14 +172,14 @@ CAF_TEST(depth_3_pipeline_with_fork) {
run();
CAF_CHECK_EQUAL(st.stage->out().num_paths(), 2u);
CAF_CHECK_EQUAL(st.stage->inbound_paths().size(), 0u);
CAF_CHECK_EQUAL(deref<sum_up_actor>(snk1).state.x, 1275);
CAF_CHECK_EQUAL(deref<sum_up_actor>(snk2).state.x, 1275);
CAF_CHECK_EQUAL(deref<sum_up_actor>(snk1).state.x, sum(60));
CAF_CHECK_EQUAL(deref<sum_up_actor>(snk2).state.x, sum(60));
self->send_exit(stg, exit_reason::kill);
}
CAF_TEST(depth_3_pipeline_with_join) {
auto src1 = sys.spawn(file_reader, 50u);
auto src2 = sys.spawn(file_reader, 50u);
auto src1 = sys.spawn(file_reader, 60u);
auto src2 = sys.spawn(file_reader, 60u);
auto stg = sys.spawn(stream_multiplexer);
auto snk = sys.spawn(sum_up);
auto& st = deref<stream_multiplexer_actor>(stg).state;
......@@ -192,46 +196,7 @@ CAF_TEST(depth_3_pipeline_with_join) {
run();
CAF_CHECK_EQUAL(st.stage->out().num_paths(), 1u);
CAF_CHECK_EQUAL(st.stage->inbound_paths().size(), 0u);
CAF_CHECK_EQUAL(deref<sum_up_actor>(snk).state.x, 2550);
self->send_exit(stg, exit_reason::kill);
}
CAF_TEST(closing_downstreams_before_end_of_stream) {
auto src = sys.spawn(file_reader, 10000u);
auto stg = sys.spawn(stream_multiplexer);
auto snk1 = sys.spawn(sum_up);
auto snk2 = sys.spawn(sum_up);
auto& st = deref<stream_multiplexer_actor>(stg).state;
CAF_MESSAGE("connect sinks to the stage (fork)");
self->send(snk1, join_atom_v, stg);
self->send(snk2, join_atom_v, stg);
consume_messages();
CAF_CHECK_EQUAL(st.stage->out().num_paths(), 2u);
CAF_MESSAGE("connect source to the stage (fork)");
self->send(stg * src, "numbers.txt");
consume_messages();
CAF_CHECK_EQUAL(st.stage->out().num_paths(), 2u);
CAF_CHECK_EQUAL(st.stage->inbound_paths().size(), 1u);
CAF_MESSAGE("do a single round of credit");
trigger_timeouts();
consume_messages();
CAF_MESSAGE("make sure the stream isn't done yet");
CAF_REQUIRE(!deref<file_reader_actor>(src).state.buf.empty());
CAF_CHECK_EQUAL(st.stage->out().num_paths(), 2u);
CAF_CHECK_EQUAL(st.stage->inbound_paths().size(), 1u);
CAF_MESSAGE("get the next not-yet-buffered integer");
auto next_pending = deref<file_reader_actor>(src).state.buf.front();
CAF_REQUIRE_GREATER(next_pending, 0);
auto sink1_result = sum(next_pending - 1);
CAF_MESSAGE("gracefully close sink 1, next pending: " << next_pending);
self->send(stg, close_atom_v, 0);
expect((close_atom, int32_t), from(self).to(stg));
CAF_MESSAGE("ship remaining elements");
run();
CAF_CHECK_EQUAL(st.stage->out().num_paths(), 1u);
CAF_CHECK_EQUAL(st.stage->inbound_paths().size(), 0u);
CAF_CHECK_LESS(deref<sum_up_actor>(snk1).state.x, sink1_result);
CAF_CHECK_EQUAL(deref<sum_up_actor>(snk2).state.x, sum(10000));
CAF_CHECK_EQUAL(deref<sum_up_actor>(snk).state.x, sum(60) * 2);
self->send_exit(stg, exit_reason::kill);
}
......
......@@ -200,10 +200,6 @@ public:
CAF_LOG_ERROR("received unexpected batch type (dropped)");
}
bool congested() const noexcept override {
return out_.capacity() == 0;
}
fused_manager& out() noexcept override {
return out_;
}
......
......@@ -45,7 +45,6 @@
#include "caf/detail/stream_sink_impl.hpp"
#include "caf/detail/stream_source_impl.hpp"
#include "caf/detail/stream_stage_impl.hpp"
#include "caf/detail/tick_emitter.hpp"
#include "caf/downstream_manager.hpp"
#include "caf/downstream_msg.hpp"
#include "caf/inbound_path.hpp"
......@@ -149,9 +148,6 @@ public:
/// Defines the container for storing message handlers.
using behavior_type = behavior;
/// The type of a single tick.
using clock_type = detail::tick_emitter::clock_type;
/// The type of a single tick.
using time_point = clock_type::time_point;
......@@ -163,21 +159,12 @@ public:
// -- constructors, destructors, and assignment operators --------------------
entity(actor_config& cfg, const char* cstr_name, time_point* global_time,
duration_type credit_interval, duration_type force_batches_interval)
: super(cfg),
mbox(unit, unit, unit, unit, unit),
name_(cstr_name),
global_time_(global_time),
tick_emitter_(global_time == nullptr ? clock_type::now()
: *global_time) {
auto cycle = detail::gcd(credit_interval.count(),
force_batches_interval.count());
ticks_per_force_batches_interval =
static_cast<size_t>(force_batches_interval.count() / cycle);
ticks_per_credit_interval =
static_cast<size_t>(credit_interval.count() / cycle);
tick_emitter_.interval(duration_type{cycle});
entity(actor_config& cfg, const char* cstr_name, time_point* global_time)
: super(cfg),
mbox(unit, unit, unit, unit, unit),
name_(cstr_name),
global_time_(global_time) {
CAF_ASSERT(global_time_ != nullptr);
}
void enqueue(mailbox_element_ptr what, execution_unit*) override {
......@@ -256,13 +243,13 @@ public:
public:
using super = stream_stage_driver<int32_t, downstream_manager>;
driver(downstream_manager& out, vector<int32_t>* log)
: super(out),
log_(log) {
driver(downstream_manager& out, vector<int32_t>* log, const char* name)
: super(out), log_(log), name(name) {
// nop
}
void process(downstream<int>& out, vector<int>& batch) override {
CAF_MESSAGE(name << " forwards " << batch.size() << " elements");
log_->insert(log_->end(), batch.begin(), batch.end());
out.append(batch.begin(), batch.end());
}
......@@ -273,8 +260,9 @@ public:
private:
vector<int>* log_;
const char* name;
};
forwarder = detail::make_stream_stage<driver>(this, &data);
forwarder = detail::make_stream_stage<driver>(this, &data, name_);
auto res = forwarder->add_outbound_path(ref.ctrl());
CAF_MESSAGE(name_ << " starts forwarding to " << ref.name()
<< " on slot " << res.value());
......@@ -321,39 +309,21 @@ public:
scheduled_actor::handle_upstream_msg(slots, sender, x);
}
void advance_time() {
auto cycle = std::chrono::milliseconds(100);
auto f = [&](tick_type x) {
if (x % ticks_per_force_batches_interval == 0) {
// Force batches on all output paths.
for (auto& kvp : stream_managers())
kvp.second->out().force_emit_batches();
}
if (x % ticks_per_credit_interval == 0) {
// Fill credit on each input path up to 30.
auto& qs = get<dmsg_id::value>(mbox.queues()).queues();
for (auto& kvp : qs) {
auto inptr = kvp.second.policy().handler.get();
auto tts = static_cast<int32_t>(kvp.second.total_task_size());
inptr->emit_ack_batch(this, tts, now(), cycle);
}
}
};
tick_emitter_.update(now(), f);
void tick() {
for (auto& kvp : stream_managers())
kvp.second->tick(now());
}
inbound_path* make_inbound_path(stream_manager_ptr mgr, stream_slots slots,
strong_actor_ptr sender,
type_id_t input_type) override {
virtual bool add_inbound_path(type_id_t,
std::unique_ptr<inbound_path> path) override {
using policy_type = policy::downstream_messages::nested;
auto res = get<dmsg_id::value>(mbox.queues())
.queues().emplace(slots.receiver, policy_type{nullptr});
.queues()
.emplace(path->slots.receiver, policy_type{nullptr});
if (!res.second)
return nullptr;
auto path = new inbound_path(std::move(mgr), slots, std::move(sender),
input_type);
res.first->second.policy().handler.reset(path);
return path;
return false;
res.first->second.policy().handler = std::move(path);
return true;
}
void erase_inbound_path_later(stream_slot slot) override {
......@@ -375,7 +345,14 @@ public:
}
time_point now() {
return global_time_ == nullptr ? clock_type::now() : *global_time_;
return *global_time_;
}
void push() {
if (forwarder)
forwarder->push();
for (auto mgr : active_stream_managers())
mgr->push();
}
// -- member variables -------------------------------------------------------
......@@ -388,7 +365,6 @@ public:
tick_type ticks_per_force_batches_interval;
tick_type ticks_per_credit_interval;
time_point* global_time_;
detail::tick_emitter tick_emitter_;
};
struct msg_visitor {
......@@ -433,6 +409,7 @@ struct msg_visitor {
);
visit(f, um.content);
self->current_mailbox_element(nullptr);
self->push();
return intrusive::task_result::resume;
}
......@@ -447,6 +424,7 @@ struct msg_visitor {
auto& dm = x.content().get_mutable_as<downstream_msg>(0);
auto f = detail::make_overload(
[&](downstream_msg::batch& y) {
TRACE(self->name(), batch, CAF_ARG(dm.slots), CAF_ARG(y.xs_size));
inptr->handle(y);
if (inptr->mgr->done()) {
CAF_MESSAGE(self->name()
......@@ -492,15 +470,7 @@ struct msg_visitor {
struct fixture {
using scheduler_type = scheduler::test_coordinator;
struct timing_config {
timespan credit_interval = std::chrono::milliseconds(100);
timespan force_batches_interval = std::chrono::milliseconds(50);
timespan step = force_batches_interval;
};
timing_config tc;
timespan max_batch_delay = defaults::stream::max_batch_delay;
actor_system_config cfg;
actor_system sys;
......@@ -513,13 +483,11 @@ struct fixture {
entity& bob;
entity& carl;
static actor spawn(actor_system& sys, actor_id id, const char* name,
timing_config& tc) {
static actor spawn(actor_system& sys, actor_id id, const char* name) {
actor_config conf;
auto& clock = dynamic_cast<scheduler_type&>(sys.scheduler()).clock();
auto global_time = &clock.current_time;
return make_actor<entity>(id, node_id{}, &sys, conf, name, global_time,
tc.credit_interval, tc.force_batches_interval);
return make_actor<entity>(id, node_id{}, &sys, conf, name, global_time);
}
static entity& fetch(const actor& hdl) {
......@@ -531,18 +499,21 @@ struct fixture {
caf::test::engine::argv()))
CAF_FAIL("parsing the config failed: " << to_string(err));
cfg.set("caf.scheduler.policy", "testing");
cfg.set("caf.stream.credit-policy", "token-based");
cfg.set("caf.stream.token-based-policy.batch-size", 50);
cfg.set("caf.stream.token-based-policy.buffer-size", 200);
return cfg;
}
fixture()
: sys(init_config(cfg)),
sched(dynamic_cast<scheduler_type&>(sys.scheduler())),
alice_hdl(spawn(sys, 0, "alice", tc)),
bob_hdl(spawn(sys, 1, "bob", tc)),
carl_hdl(spawn(sys, 2, "carl", tc)),
alice(fetch(alice_hdl)),
bob(fetch(bob_hdl)),
carl(fetch(carl_hdl)) {
: sys(init_config(cfg)),
sched(dynamic_cast<scheduler_type&>(sys.scheduler())),
alice_hdl(spawn(sys, 0, "alice")),
bob_hdl(spawn(sys, 1, "bob")),
carl_hdl(spawn(sys, 2, "carl")),
alice(fetch(alice_hdl)),
bob(fetch(bob_hdl)),
carl(fetch(carl_hdl)) {
// nop
}
......@@ -568,10 +539,10 @@ struct fixture {
template <class... Ts>
void next_cycle(Ts&... xs) {
entity* es[] = {&xs...};
CAF_MESSAGE("advance clock by " << tc.credit_interval.count() << "ns");
sched.clock().current_time += tc.credit_interval;
CAF_MESSAGE("advance clock by " << max_batch_delay);
sched.clock().current_time += max_batch_delay;
for (auto e : es)
e->advance_time();
e->tick();
}
template <class F, class... Ts>
......@@ -583,10 +554,10 @@ struct fixture {
while (!std::all_of(std::begin(fs), std::end(fs), mailbox_empty))
for (auto& f : fs)
f.self->mbox.new_round(1, f);
CAF_MESSAGE("advance clock by " << tc.step.count() << "ns");
sched.clock().current_time += tc.step;
CAF_MESSAGE("advance clock by " << max_batch_delay);
sched.clock().current_time += max_batch_delay;
for (auto e : es)
e->advance_time();
e->tick();
}
while (!pred());
}
......@@ -614,16 +585,12 @@ CAF_TEST_FIXTURE_SCOPE(native_streaming_classes_tests, fixture)
CAF_TEST(depth_2_pipeline_30_items) {
alice.start_streaming(bob, 30);
loop(alice, bob);
next_cycle(alice, bob); // emit first ack_batch
loop(alice, bob);
next_cycle(alice, bob); // to emit final ack_batch
loop(alice, bob);
loop_until([&] { return done_streaming(); }, alice, bob);
CAF_CHECK_EQUAL(bob.data, make_iota(0, 30));
}
CAF_TEST(depth_2_pipeline_2000_items) {
constexpr size_t num_messages = 2000;
CAF_TEST(depth_2_pipeline_500_items) {
constexpr size_t num_messages = 500;
alice.start_streaming(bob, num_messages);
loop_until([&] { return done_streaming(); }, alice, bob);
CAF_CHECK_EQUAL(bob.data, make_iota(0, num_messages));
......@@ -632,19 +599,13 @@ CAF_TEST(depth_2_pipeline_2000_items) {
CAF_TEST(depth_3_pipeline_30_items) {
bob.forward_to(carl);
alice.start_streaming(bob, 30);
loop(alice, bob, carl);
next_cycle(alice, bob, carl); // emit first ack_batch
loop(alice, bob, carl);
next_cycle(alice, bob, carl);
loop(alice, bob, carl);
next_cycle(alice, bob, carl); // emit final ack_batch
loop(alice, bob, carl);
loop_until([&] { return done_streaming(); }, alice, bob, carl);
CAF_CHECK_EQUAL(bob.data, make_iota(0, 30));
CAF_CHECK_EQUAL(carl.data, make_iota(0, 30));
}
CAF_TEST(depth_3_pipeline_2000_items) {
constexpr size_t num_messages = 2000;
CAF_TEST(depth_3_pipeline_500_items) {
constexpr size_t num_messages = 500;
bob.forward_to(carl);
alice.start_streaming(bob, num_messages);
CAF_MESSAGE("loop over alice and bob until bob is congested");
......
......@@ -40,33 +40,8 @@ TESTEE_SETUP();
using buf = std::deque<int>;
std::function<void(buf&)> init(size_t buf_size) {
return [=](buf& xs) {
xs.resize(buf_size);
std::iota(xs.begin(), xs.end(), 1);
};
}
void push_from_buf(buf& xs, downstream<int>& out, size_t num) {
CAF_MESSAGE("push " << num << " messages downstream");
auto n = std::min(num, xs.size());
for (size_t i = 0; i < n; ++i)
out.push(xs[i]);
xs.erase(xs.begin(), xs.begin() + static_cast<ptrdiff_t>(n));
}
std::function<bool(const buf&)> is_done(scheduled_actor* self) {
return [=](const buf& xs) {
if (xs.empty()) {
CAF_MESSAGE(self->name() << " exhausted its buffer");
return true;
}
return false;
};
}
template <class T, class Self>
std::function<void(T&, const error&)> fin(Self* self) {
auto fin(Self* self) {
return [=](T&, const error& err) {
self->state.fin_called += 1;
if (err == none) {
......@@ -102,18 +77,38 @@ TESTEE_STATE(file_reader) {
};
VARARGS_TESTEE(file_reader, size_t buf_size) {
auto init = [](size_t buf_size) {
return [=](buf& xs) {
xs.resize(buf_size);
std::iota(xs.begin(), xs.end(), 1);
};
};
auto push_from_buf = [](buf& xs, downstream<int>& out, size_t num) {
CAF_MESSAGE("push " << num << " messages downstream");
auto n = std::min(num, xs.size());
for (size_t i = 0; i < n; ++i)
out.push(xs[i]);
xs.erase(xs.begin(), xs.begin() + static_cast<ptrdiff_t>(n));
};
auto is_done = [self](const buf& xs) {
if (xs.empty()) {
CAF_MESSAGE(self->name() << " exhausted its buffer");
return true;
}
return false;
};
return {
[=](string& fname) -> result<stream<int>> {
CAF_CHECK_EQUAL(fname, "numbers.txt");
CAF_CHECK_EQUAL(self->mailbox().empty(), true);
return attach_stream_source(self, init(buf_size), push_from_buf,
is_done(self), fin<buf>(self));
return attach_stream_source(self, init(buf_size), push_from_buf, is_done,
fin<buf>(self));
},
[=](string& fname, actor next) {
CAF_CHECK_EQUAL(fname, "numbers.txt");
CAF_CHECK_EQUAL(self->mailbox().empty(), true);
attach_stream_source(self, next, init(buf_size), push_from_buf,
is_done(self), fin<buf>(self));
attach_stream_source(self, next, init(buf_size), push_from_buf, is_done,
fin<buf>(self));
},
};
}
......@@ -259,11 +254,7 @@ CAF_TEST(depth_2_pipeline_50_items) {
expect((upstream_msg::ack_open), from(snk).to(src));
CAF_MESSAGE("start data transmission (a single batch)");
expect((downstream_msg::batch), from(src).to(snk));
tick();
expect((timeout_msg), from(snk).to(snk));
expect((timeout_msg), from(src).to(src));
expect((upstream_msg::ack_batch), from(snk).to(src));
CAF_MESSAGE("expect close message from src and then result from snk");
expect((downstream_msg::close), from(src).to(snk));
CAF_CHECK_EQUAL(deref<sum_up_actor>(snk).state.x, 1275);
CAF_MESSAGE("verify that each actor called its finalizer once");
......@@ -278,15 +269,11 @@ CAF_TEST(depth_2_pipeline_setup2_50_items) {
CAF_MESSAGE("initiate stream handshake");
self->send(src, "numbers.txt", snk);
expect((string, actor), from(self).to(src).with("numbers.txt", snk));
expect((open_stream_msg), from(strong_actor_ptr{nullptr}).to(snk));
expect((open_stream_msg), to(snk));
expect((upstream_msg::ack_open), from(snk).to(src));
CAF_MESSAGE("start data transmission (a single batch)");
expect((downstream_msg::batch), from(src).to(snk));
tick();
expect((timeout_msg), from(snk).to(snk));
expect((timeout_msg), from(src).to(src));
expect((upstream_msg::ack_batch), from(snk).to(src));
CAF_MESSAGE("expect close message from src and then result from snk");
expect((downstream_msg::close), from(src).to(snk));
CAF_CHECK_EQUAL(deref<sum_up_actor>(snk).state.x, 1275);
CAF_MESSAGE("verify that each actor called its finalizer once");
......@@ -311,9 +298,6 @@ CAF_TEST(delayed_depth_2_pipeline_50_items) {
expect((upstream_msg::ack_open), from(snk).to(src));
CAF_MESSAGE("start data transmission (a single batch)");
expect((downstream_msg::batch), from(src).to(snk));
tick();
expect((timeout_msg), from(snk).to(snk));
expect((timeout_msg), from(src).to(src));
expect((upstream_msg::ack_batch), from(snk).to(src));
CAF_MESSAGE("expect close message from src and then result from snk");
expect((downstream_msg::close), from(src).to(snk));
......@@ -381,6 +365,9 @@ CAF_TEST(depth_2_pipeline_error_at_source) {
CAF_MESSAGE("start data transmission (and abort source)");
hard_kill(src);
expect((downstream_msg::batch), from(src).to(snk));
expect((downstream_msg::batch), from(src).to(snk));
expect((downstream_msg::batch), from(src).to(snk));
expect((downstream_msg::batch), from(src).to(snk));
expect((downstream_msg::forced_close), from(_).to(snk));
CAF_MESSAGE("verify that the sink called its finalizer once");
CAF_CHECK_EQUAL(deref<sum_up_actor>(snk).state.fin_called, 1);
......@@ -407,12 +394,6 @@ CAF_TEST(depth_3_pipeline_50_items) {
auto src = sys.spawn(file_reader, 50u);
auto stg = sys.spawn(filter);
auto snk = sys.spawn(sum_up);
auto next_cycle = [&] {
tick();
allow((timeout_msg), from(snk).to(snk));
allow((timeout_msg), from(stg).to(stg));
allow((timeout_msg), from(src).to(src));
};
CAF_MESSAGE(CAF_ARG(self) << CAF_ARG(src) << CAF_ARG(stg) << CAF_ARG(snk));
CAF_MESSAGE("initiate stream handshake");
self->send(snk * stg * src, "numbers.txt");
......@@ -425,17 +406,9 @@ CAF_TEST(depth_3_pipeline_50_items) {
expect((downstream_msg::batch), from(src).to(stg));
CAF_MESSAGE("the stage should delay its first batch since its underfull");
disallow((downstream_msg::batch), from(stg).to(snk));
next_cycle();
CAF_MESSAGE("the source shuts down and the stage sends the final batch");
expect((upstream_msg::ack_batch), from(stg).to(src));
expect((downstream_msg::close), from(src).to(stg));
expect((downstream_msg::batch), from(stg).to(snk));
next_cycle();
CAF_MESSAGE("the stage shuts down and the sink produces its final result");
expect((upstream_msg::ack_batch), from(snk).to(stg));
expect((downstream_msg::close), from(stg).to(snk));
CAF_MESSAGE("after running the pipeline the sink received all batches");
run();
CAF_CHECK_EQUAL(deref<sum_up_actor>(snk).state.x, 625);
CAF_MESSAGE("verify that each actor called its finalizer once");
CAF_CHECK_EQUAL(deref<file_reader_actor>(src).state.fin_called, 1);
CAF_CHECK_EQUAL(deref<filter_actor>(stg).state.fin_called, 1);
CAF_CHECK_EQUAL(deref<sum_up_actor>(snk).state.fin_called, 1);
......
......@@ -684,7 +684,9 @@ public:
cfg.set("caf.middleman.manual-multiplexing", true);
cfg.set("caf.middleman.workers", size_t{0});
}
cfg.set("caf.stream.credit-policy", "testing");
cfg.set("caf.stream.credit-policy", "token-based");
cfg.set("caf.stream.token-based-policy.batch-size", 50);
cfg.set("caf.stream.token-based-policy.buffer-size", 200);
return cfg;
}
......
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