Commit da65c083 authored by Dominik Charousset's avatar Dominik Charousset Committed by Dominik Charousset

Improve performance of the mcast operator

(cherry picked from commit 4a4c0276)
parent 4f580d89
......@@ -16,6 +16,7 @@
#include <algorithm>
#include <deque>
#include <memory>
#include <numeric>
namespace caf::flow::op {
......@@ -39,12 +40,13 @@ public:
// -- implementation of subscription -----------------------------------------
bool disposed() const noexcept override {
return !state_;
return !state_ || state_->disposed;
}
void dispose() override {
if (state_) {
ctx_->delay_fn([state = std::move(state_)]() { state->do_dispose(); });
auto state = std::move(state_);
state->dispose();
}
}
......@@ -66,7 +68,7 @@ private:
// Base type for *hot* operators that multicast data to subscribed observers.
template <class T>
class mcast : public hot<T> {
class mcast : public hot<T>, public ucast_sub_state_listener<T> {
public:
// -- member types -----------------------------------------------------------
......@@ -84,18 +86,30 @@ public:
// nop
}
~mcast() override {
close();
}
// -- broadcasting -----------------------------------------------------------
/// Pushes @p item to all subscribers.
void push_all(const T& item) {
for (auto& state : states_)
state->push(item);
/// @returns `true` if all observers consumed the item immediately without
/// buffering it, `false` otherwise.
bool push_all(const T& item) {
return std::accumulate(states_.begin(), states_.end(), true,
[&item](bool res, const state_ptr_type& ptr) {
return res & ptr->push(item);
});
}
/// Closes the operator, eventually emitting on_complete on all observers.
void close() {
if (!closed_) {
closed_ = true;
for (auto& state : states_)
for (auto& state : states_) {
state->listener = nullptr;
state->close();
}
states_.clear();
}
}
......@@ -104,58 +118,59 @@ public:
void abort(const error& reason) {
if (!closed_) {
closed_ = true;
for (auto& state : states_)
for (auto& state : states_) {
state->listener = nullptr;
state->abort(reason);
}
states_.clear();
err_ = reason;
}
}
// -- properties -------------------------------------------------------------
size_t max_demand() const noexcept {
if (states_.empty()) {
return 0;
} else {
}
auto pred = [](const state_ptr_type& x, const state_ptr_type& y) {
return x->demand < y->demand;
};
auto& ptr = *std::max_element(states_.begin(), states_.end(), pred);
return ptr->demand;
}
}
size_t min_demand() const noexcept {
if (states_.empty()) {
return 0;
} else {
}
auto pred = [](const state_ptr_type& x, const state_ptr_type& y) {
return x->demand < y->demand;
};
auto& ptr = *std::min_element(states_.begin(), states_.end(), pred);
ptr->demand;
}
return ptr->demand;
}
size_t max_buffered() const noexcept {
if (states_.empty()) {
return 0;
} else {
}
auto pred = [](const state_ptr_type& x, const state_ptr_type& y) {
return x->buf.size() < y->buf.size();
};
auto& ptr = *std::max_element(states_.begin(), states_.end(), pred);
return ptr->buf.size();
}
}
size_t min_buffered() const noexcept {
if (states_.empty()) {
return 0;
} else {
}
auto pred = [](const state_ptr_type& x, const state_ptr_type& y) {
return x->buf.size() < y->buf.size();
};
auto& ptr = *std::min_element(states_.begin(), states_.end(), pred);
ptr->buf.size();
}
return ptr->buf.size();
}
/// Queries whether there is at least one observer subscribed to the operator.
......@@ -168,20 +183,20 @@ public:
return states_.size();
}
// -- state management -------------------------------------------------------
/// Adds state for a new observer to the operator.
state_ptr_type add_state(observer_type out) {
auto state = make_counted<state_type>(super::ctx_, std::move(out));
auto mc = strong_this();
state->when_disposed = make_action([mc, state]() mutable { //
mc->do_dispose(state);
});
state->when_consumed_some = make_action([mc, state]() mutable { //
mc->on_consumed_some(*state);
});
state->listener = this;
states_.push_back(state);
return state;
}
disposable subscribe(observer<T> out) override {
// -- implementation of observable -------------------------------------------
/// Adds a new observer to the operator.
disposable subscribe(observer_type out) override {
if (!closed_) {
auto ptr = make_counted<mcast_sub<T>>(super::ctx_, add_state(out));
out.on_subscribe(subscription{ptr});
......@@ -194,6 +209,23 @@ public:
}
}
// -- implementation of ucast_sub_state_listener -----------------------------
void on_disposed(state_type* ptr) final {
super::ctx_->delay_fn([mc = strong_this(), sptr = state_ptr_type{ptr}] {
if (auto i = std::find(mc->states_.begin(), mc->states_.end(), sptr);
i != mc->states_.end()) {
// We don't care about preserving the order of elements in the vector.
// Hence, we can swap the element to the back and then pop it.
auto last = mc->states_.end() - 1;
if (i != last)
std::swap(*i, *last);
mc->states_.pop_back();
mc->do_dispose(sptr);
}
});
}
protected:
bool closed_ = false;
error err_;
......@@ -204,19 +236,8 @@ private:
return {this};
}
void do_dispose(state_ptr_type& state) {
auto e = states_.end();
if (auto i = std::find(states_.begin(), e, state); i != e) {
states_.erase(i);
on_dispose(*state);
}
}
virtual void on_dispose(state_type&) {
// nop
}
virtual void on_consumed_some(state_type&) {
/// Called whenever a state is disposed.
virtual void do_dispose(const state_ptr_type&) {
// nop
}
};
......
......@@ -23,16 +23,32 @@ namespace caf::flow::op {
template <class T>
class prefix_and_tail_sub : public detail::plain_ref_counted,
public observer_impl<T>,
public subscription_impl {
public subscription_impl,
public ucast_sub_state_listener<T> {
public:
// -- member types -----------------------------------------------------------
using tuple_t = cow_tuple<cow_vector<T>, observable<T>>;
using state_type = ucast_sub_state<T>;
// -- constructors, destructors, and assignment operators --------------------
prefix_and_tail_sub(coordinator* ctx, observer<tuple_t> out,
size_t prefix_size)
: ctx_(ctx), out_(std::move(out)), prefix_size_(prefix_size) {
prefix_buf_.reserve(prefix_size);
}
~prefix_and_tail_sub() {
if (sink_) {
sink_->state().listener = nullptr;
sink_->close();
}
}
// -- implementation of observer ---------------------------------------------
void ref_coordinated() const noexcept override {
ref();
}
......@@ -53,7 +69,7 @@ public:
if (prefix_buf_.size() == prefix_size_) {
// Create the sink to deliver to tail lazily and deliver the prefix.
sink_ = make_counted<ucast<T>>(ctx_);
set_callbacks();
sink_->state().listener = this;
// Force member to be null before calling on_next / on_complete.
auto out = std::move(out_);
auto tup = make_cow_tuple(cow_vector<T>{std::move(prefix_buf_)},
......@@ -66,7 +82,7 @@ public:
void on_error(const error& reason) override {
if (sink_) {
sink_->state().when_demand_changed = nullptr;
sink_->state().listener= nullptr;
sink_->abort(reason);
sub_ = nullptr;
} else if (out_) {
......@@ -77,7 +93,7 @@ public:
void on_complete() override {
if (sink_) {
sink_->state().when_demand_changed = nullptr;
sink_->state().listener = nullptr;
sink_->close();
sub_ = nullptr;
} else if (out_) {
......@@ -86,6 +102,8 @@ public:
}
}
// -- implementation of observable -------------------------------------------
void on_subscribe(flow::subscription sub) override {
if (!sub_ && out_) {
sub_ = std::move(sub);
......@@ -98,6 +116,8 @@ public:
}
}
// -- implementation of disposable -------------------------------------------
void dispose() override {
if (out_) {
out_ = nullptr;
......@@ -129,20 +149,13 @@ public:
}
}
private:
intrusive_ptr<prefix_and_tail_sub> strong_this() {
return {this};
}
// -- implementation of ucast_sub_state_listener -----------------------------
void set_callbacks() {
auto sptr = strong_this();
auto demand_cb = [sptr] { sptr->on_sink_demand_change(); };
sink_->state().when_demand_changed = make_action(std::move(demand_cb));
auto disposed_cb = [sptr] { sptr->on_sink_dispose(); };
sink_->state().when_disposed = make_action(std::move(disposed_cb));
void on_disposed(state_type*) override {
ctx_->delay_fn([sptr = strong_this()] { sptr->do_dispose(); });
}
void on_sink_demand_change() {
void on_demand_changed(state_type*) override {
if (sink_ && sub_) {
auto& st = sink_->state();
auto pending = in_flight_ + st.buf.size();
......@@ -154,14 +167,20 @@ private:
}
}
void on_sink_dispose() {
private:
intrusive_ptr<prefix_and_tail_sub> strong_this() {
return {this};
}
void do_dispose() {
sink_ = nullptr;
if (sub_) {
sub_.dispose();
sub_ = nullptr;
if (out_) {
auto tmp = std::move(out_);
tmp.on_complete();
}
}
/// Our scheduling context.
coordinator* ctx_;
......
......@@ -21,12 +21,20 @@ public:
using state_type = typename super::state_type;
using state_ptr_type = mcast_sub_state_ptr<T>;
using src_ptr = intrusive_ptr<base<T>>;
// -- constructors, destructors, and assignment operators --------------------
publish(coordinator* ctx, src_ptr src) : super(ctx), source_(std::move(src)) {
// nop
publish(coordinator* ctx, src_ptr src,
size_t max_buf_size = defaults::flow::buffer_size)
: super(ctx), max_buf_size_(max_buf_size), source_(std::move(src)) {
try_request_more_ = make_action([this] { this->try_request_more(); });
}
~publish() override {
try_request_more_.dispose();
}
// -- ref counting (and disambiguation due to multiple base types) -----------
......@@ -85,7 +93,16 @@ public:
void on_next(const T& item) override {
--in_flight_;
this->push_all(item);
if (this->push_all(item)) {
if (in_ && this->has_observers()) {
// If push_all returns `true`, it means that all observers have consumed
// the item without buffering it. Hence, we know that
// this->max_buffered() is 0 and we can request more items from the
// source right away.
++in_flight_;
in_.request(1);
}
}
}
void on_complete() override {
......@@ -106,8 +123,18 @@ public:
}
}
// -- implementation of ucast_sub_state_listener -----------------------------
void on_consumed_some(state_type*, size_t, size_t) override {
if (!try_request_more_pending_) {
try_request_more_pending_ = true;
super::ctx_->delay(try_request_more_);
}
}
protected:
void try_request_more() {
try_request_more_pending_ = false;
if (in_ && this->has_observers()) {
if (auto buf_size = this->max_buffered() + in_flight_;
max_buf_size_ > buf_size) {
......@@ -119,7 +146,7 @@ protected:
}
private:
void on_dispose(state_type&) override {
void do_dispose(const state_ptr_type&) override {
try_request_more();
if (auto_disconnect_ && connected_ && super::observer_count() == 0) {
in_.dispose();
......@@ -128,17 +155,36 @@ private:
}
}
void on_consumed_some(state_type&) override {
try_request_more();
}
/// Keeps track of the number of items that have been requested but that have
/// not yet been delivered.
size_t in_flight_ = 0;
size_t max_buf_size_ = defaults::flow::buffer_size;
/// Maximum number of items to buffer.
size_t max_buf_size_;
/// Our subscription for fetching items.
subscription in_;
/// The source operator we subscribe to lazily.
src_ptr source_;
/// Keeps track of whether we are connected to the source operator.
bool connected_ = false;
/// The number of observers that need to connect before we connect to the
/// source operator.
size_t auto_connect_threshold_ = std::numeric_limits<size_t>::max();
/// Whether to disconnect from the source operator when the last observer
/// unsubscribes.
bool auto_disconnect_ = false;
/// Scheduled when on_consumed_some() is called. Having this as a member
/// variable avoids allocating a new action object for each call.
action try_request_more_;
/// Guards against scheduling `try_request_more_` while it is already pending.
bool try_request_more_pending_ = false;
};
} // namespace caf::flow::op
......@@ -17,10 +17,13 @@
namespace caf::flow::op {
/// State shared between one multicast operator and one subscribed observer.
/// Shared state between an operator that emits values and the subscribed
/// observer.
template <class T>
class ucast_sub_state : public detail::plain_ref_counted {
public:
// -- friends ----------------------------------------------------------------
friend void intrusive_ptr_add_ref(const ucast_sub_state* ptr) noexcept {
ptr->ref();
}
......@@ -29,6 +32,39 @@ public:
ptr->deref();
}
// -- member types -----------------------------------------------------------
/// Interface for listeners that want to be notified when a `ucast_sub_state`
/// is disposed, has consumed some items, or when its demand hast changed.
class abstract_listener {
public:
virtual ~abstract_listener() {
// nop
}
/// Called when the `ucast_sub_state` is disposed.
virtual void on_disposed(ucast_sub_state*) = 0;
/// Called when the `ucast_sub_state` receives new demand.
virtual void on_demand_changed(ucast_sub_state*) {
// nop
}
/// Called when the `ucast_sub_state` has consumed some items.
/// @param state The `ucast_sub_state` that consumed items.
/// @param old_buffer_size The number of items in the buffer before
/// consuming items.
/// @param new_buffer_size The number of items in the buffer after
/// consuming items.
virtual void on_consumed_some([[maybe_unused]] ucast_sub_state* state,
[[maybe_unused]] size_t old_buffer_size,
[[maybe_unused]] size_t new_buffer_size) {
// nop
}
};
// -- constructors, destructors, and assignment operators --------------------
explicit ucast_sub_state(coordinator* ptr) : ctx(ptr) {
// nop
}
......@@ -38,31 +74,49 @@ public:
// nop
}
/// The coordinator for scheduling delayed function calls.
coordinator* ctx;
/// The buffer for storing items until the observer requests them.
std::deque<T> buf;
/// The number items that the observer has requested but not yet received.
size_t demand = 0;
/// The observer to send items to.
observer<T> out;
/// Keeps track of whether this object has been disposed.
bool disposed = false;
/// Keeps track of whether this object has been closed.
bool closed = false;
/// Keeps track of whether `do_run` is currently running.
bool running = false;
/// The error to pass to the observer after the last `on_next` call. If this
/// error is default-constructed, then the observer receives `on_complete`.
/// Otherwise, the observer receives `on_error`.
error err;
action when_disposed;
action when_consumed_some;
action when_demand_changed;
/// The listener for state changes. We hold a non-owning pointer to the
/// listener, because the listener owns the state.
abstract_listener* listener = nullptr;
void push(const T& item) {
/// Returns `true` if `item` was consumed, `false` when it was buffered.
[[nodiscard]] bool push(const T& item) {
if (disposed) {
// nop
return true;
} else if (demand > 0 && !running) {
CAF_ASSERT(out);
CAF_ASSERT(buf.empty());
--demand;
out.on_next(item);
if (when_consumed_some)
ctx->delay(when_consumed_some);
return true;
} else {
buf.push_back(item);
return false;
}
}
......@@ -71,13 +125,11 @@ public:
closed = true;
if (!running && buf.empty()) {
disposed = true;
listener = nullptr;
if (out) {
out.on_complete();
out = nullptr;
auto tmp = std::move(out);
tmp.on_complete();
}
when_disposed = nullptr;
when_consumed_some = nullptr;
when_demand_changed = nullptr;
}
}
}
......@@ -88,58 +140,60 @@ public:
err = reason;
if (!running && buf.empty()) {
disposed = true;
listener = nullptr;
if (out) {
out.on_error(reason);
out = nullptr;
auto tmp = std::move(out);
tmp.on_error(reason);
}
when_disposed = nullptr;
when_consumed_some = nullptr;
when_demand_changed = nullptr;
}
}
}
void do_dispose() {
if (out) {
out.on_complete();
out = nullptr;
}
if (when_disposed) {
ctx->delay(std::move(when_disposed));
}
if (when_consumed_some) {
when_consumed_some.dispose();
when_consumed_some = nullptr;
}
when_demand_changed = nullptr;
void dispose() {
buf.clear();
demand = 0;
disposed = true;
if (listener) {
auto* lptr = listener;
listener = nullptr;
lptr->on_disposed(this);
}
if (out) {
auto tmp = std::move(out);
tmp.on_complete();
}
}
void do_run() {
auto guard = detail::make_scope_guard([this] { running = false; });
if (!disposed) {
auto old_buf_size = buf.size();
auto got_some = demand > 0 && !buf.empty();
for (bool run = got_some; run; run = demand > 0 && !buf.empty()) {
out.on_next(buf.front());
// Note: on_next may call dispose().
if (disposed)
return;
buf.pop_front();
--demand;
}
if (buf.empty() && closed) {
auto tmp = std::move(out);
if (err)
out.on_error(err);
tmp.on_error(err);
else
out.on_complete();
out = nullptr;
do_dispose();
} else if (got_some && when_consumed_some) {
ctx->delay(when_consumed_some);
tmp.on_complete();
dispose();
} else if (got_some && listener) {
listener->on_consumed_some(this, old_buf_size, buf.size());
}
}
}
};
template <class T>
using ucast_sub_state_listener = typename ucast_sub_state<T>::abstract_listener;
template <class T>
using ucast_sub_state_ptr = intrusive_ptr<ucast_sub_state<T>>;
......@@ -156,20 +210,23 @@ public:
// -- implementation of subscription -----------------------------------------
bool disposed() const noexcept override {
return !state_;
return !state_ || state_->disposed;
}
void dispose() override {
if (state_) {
ctx_->delay_fn([state = std::move(state_)]() { state->do_dispose(); });
auto state = std::move(state_);
state->dispose();
}
}
void request(size_t n) override {
if (!state_)
return;
state_->demand += n;
if (state_->when_demand_changed)
state_->when_demand_changed.run();
if (!state_->running) {
if (state_->listener)
state_->listener->on_demand_changed(state_.get());
if (!state_->running && !state_->buf.empty()) {
state_->running = true;
ctx_->delay_fn([state = state_] { state->do_run(); });
}
......@@ -205,7 +262,7 @@ public:
/// Pushes @p item to the subscriber or buffers them until subscribed.
void push(const T& item) {
state_->push(item);
std::ignore = state_->push(item);
}
/// Closes the operator, eventually emitting on_complete on all observers.
......
......@@ -16,27 +16,23 @@ intrusive_ptr<scoped_coordinator> scoped_coordinator::make() {
void scoped_coordinator::run() {
for (;;) {
auto f = next(!watched_disposables_.empty());
if (f.ptr() != nullptr) {
f.run();
drop_disposed_flows();
} else {
auto f = next(!watched_disposables_.empty());
if (!f)
return;
}
f.run();
}
}
size_t scoped_coordinator::run_some() {
size_t result = 0;
for (;;) {
drop_disposed_flows();
auto f = next(false);
if (f.ptr() != nullptr) {
if (!f)
return result;
++result;
f.run();
drop_disposed_flows();
} else {
return result;
}
}
}
......
// This file is part of CAF, the C++ Actor Framework. See the file LICENSE in
// the main distribution directory for license terms and copyright or visit
// https://github.com/actor-framework/actor-framework/blob/master/LICENSE.
#define CAF_SUITE flow.op.buffer
#include "caf/flow/op/buffer.hpp"
#include "core-test.hpp"
#include "caf/flow/coordinator.hpp"
#include "caf/flow/item_publisher.hpp"
#include "caf/flow/observable.hpp"
#include "caf/flow/observable_builder.hpp"
#include "caf/flow/observer.hpp"
#include "caf/flow/scoped_coordinator.hpp"
using namespace caf;
using namespace std::literals;
namespace {
constexpr auto fwd_data = flow::op::buffer_input_t{};
constexpr auto fwd_ctrl = flow::op::buffer_emit_t{};
struct skip_trait {
static constexpr bool skip_empty = true;
using input_type = int;
using output_type = cow_vector<int>;
using select_token_type = int64_t;
output_type operator()(const std::vector<input_type>& xs) {
return output_type{xs};
}
};
struct noskip_trait {
static constexpr bool skip_empty = false;
using input_type = int;
using output_type = cow_vector<int>;
using select_token_type = int64_t;
output_type operator()(const std::vector<input_type>& xs) {
return output_type{xs};
}
};
struct fixture : test_coordinator_fixture<> {
flow::scoped_coordinator_ptr ctx = flow::make_scoped_coordinator();
~fixture() {
ctx->run();
}
// Similar to buffer::subscribe, but returns a buffer_sub pointer instead of
// type-erasing it into a disposable.
template <class Trait = noskip_trait>
auto raw_sub(size_t max_items, flow::observable<int> in,
flow::observable<int64_t> select,
flow::observer<cow_vector<int>> out) {
using sub_t = flow::op::buffer_sub<Trait>;
auto ptr = make_counted<sub_t>(ctx.get(), max_items, out);
ptr->init(in, select);
out.on_subscribe(flow::subscription{ptr});
return ptr;
}
template <class Impl>
void add_subs(intrusive_ptr<Impl> uut) {
auto data_sub = make_counted<flow::passive_subscription_impl>();
uut->fwd_on_subscribe(fwd_data, flow::subscription{std::move(data_sub)});
auto ctrl_sub = make_counted<flow::passive_subscription_impl>();
uut->fwd_on_subscribe(fwd_ctrl, flow::subscription{std::move(ctrl_sub)});
}
template <class T>
auto trivial_obs() {
return flow::make_trivial_observable<T>(ctx.get());
}
};
} // namespace
BEGIN_FIXTURE_SCOPE(fixture)
SCENARIO("the buffer operator groups items together") {
GIVEN("an observable") {
WHEN("calling .buffer(3)") {
THEN("the observer receives values in groups of three") {
auto inputs = std::vector<int>{1, 2, 4, 8, 16, 32, 64, 128};
auto outputs = std::vector<cow_vector<int>>{};
auto expected = std::vector<cow_vector<int>>{
cow_vector<int>{1, 2, 4},
cow_vector<int>{8, 16, 32},
cow_vector<int>{64, 128},
};
ctx->make_observable()
.from_container(inputs) //
.buffer(3)
.for_each([&outputs](const cow_vector<int>& xs) {
outputs.emplace_back(xs);
});
ctx->run();
CHECK_EQ(outputs, expected);
}
}
}
}
SCENARIO("the buffer operator forces items at regular intervals") {
GIVEN("an observable") {
WHEN("calling .buffer(3, 1s)") {
THEN("the observer receives values in groups of three or after 1s") {
auto outputs = std::make_shared<std::vector<cow_vector<int>>>();
auto expected = std::vector<cow_vector<int>>{
cow_vector<int>{1, 2, 4}, cow_vector<int>{8, 16, 32},
cow_vector<int>{}, cow_vector<int>{64},
cow_vector<int>{}, cow_vector<int>{128, 256, 512},
};
auto closed = std::make_shared<bool>(false);
auto pub = flow::item_publisher<int>{ctx.get()};
sys.spawn([&pub, outputs, closed](caf::event_based_actor* self) {
pub.as_observable()
.observe_on(self) //
.buffer(3, 1s)
.do_on_complete([closed] { *closed = true; })
.for_each([outputs](const cow_vector<int>& xs) {
outputs->emplace_back(xs);
});
});
sched.run();
MESSAGE("emit the first six items");
pub.push({1, 2, 4, 8, 16, 32});
ctx->run_some();
sched.run();
MESSAGE("force an empty buffer");
advance_time(1s);
sched.run();
MESSAGE("force a buffer with a single element");
pub.push(64);
ctx->run_some();
sched.run();
advance_time(1s);
sched.run();
MESSAGE("force an empty buffer");
advance_time(1s);
sched.run();
MESSAGE("emit the last items and close the source");
pub.push({128, 256, 512});
pub.close();
ctx->run_some();
sched.run();
advance_time(1s);
sched.run();
CHECK_EQ(*outputs, expected);
CHECK(*closed);
}
}
}
}
SCENARIO("the buffer operator forwards errors") {
GIVEN("an observable that produces some values followed by an error") {
WHEN("calling .buffer() on it") {
THEN("the observer receives the values and then the error") {
auto outputs = std::make_shared<std::vector<cow_vector<int>>>();
auto err = std::make_shared<error>();
sys.spawn([outputs, err](caf::event_based_actor* self) {
auto obs = self->make_observable();
obs.iota(1)
.take(17)
.concat(obs.fail<int>(make_error(caf::sec::runtime_error)))
.buffer(7, 1s)
.do_on_error([err](const error& what) { *err = what; })
.for_each([outputs](const cow_vector<int>& xs) {
outputs->emplace_back(xs);
});
});
sched.run();
auto expected = std::vector<cow_vector<int>>{
cow_vector<int>{1, 2, 3, 4, 5, 6, 7},
cow_vector<int>{8, 9, 10, 11, 12, 13, 14},
cow_vector<int>{15, 16, 17},
};
CHECK_EQ(*outputs, expected);
CHECK_EQ(*err, caf::sec::runtime_error);
}
}
}
GIVEN("an observable that produces only an error") {
WHEN("calling .buffer() on it") {
THEN("the observer receives the error") {
auto outputs = std::make_shared<std::vector<cow_vector<int>>>();
auto err = std::make_shared<error>();
sys.spawn([outputs, err](caf::event_based_actor* self) {
self->make_observable()
.fail<int>(make_error(caf::sec::runtime_error))
.buffer(3, 1s)
.do_on_error([err](const error& what) { *err = what; })
.for_each([outputs](const cow_vector<int>& xs) {
outputs->emplace_back(xs);
});
});
sched.run();
CHECK(outputs->empty());
CHECK_EQ(*err, caf::sec::runtime_error);
}
}
}
}
SCENARIO("buffers start to emit items once subscribed") {
GIVEN("a buffer operator") {
WHEN("the selector never calls on_subscribe") {
THEN("the buffer still emits batches") {
auto snk = flow::make_passive_observer<cow_vector<int>>();
auto grd = make_unsubscribe_guard(snk);
auto uut = raw_sub(3, flow::make_nil_observable<int>(ctx.get()),
flow::make_nil_observable<int64_t>(ctx.get()),
snk->as_observer());
auto data_sub = make_counted<flow::passive_subscription_impl>();
uut->fwd_on_subscribe(fwd_data, flow::subscription{data_sub});
ctx->run();
REQUIRE_GE(data_sub->demand, 3u);
for (int i = 0; i < 3; ++i)
uut->fwd_on_next(fwd_data, i);
ctx->run();
CHECK_EQ(snk->buf.size(), 0u);
snk->request(17);
ctx->run();
if (CHECK_EQ(snk->buf.size(), 1u))
CHECK_EQ(snk->buf[0], cow_vector<int>({0, 1, 2}));
}
}
}
}
SCENARIO("buffers never subscribe to their control observable on error") {
GIVEN("a buffer operator") {
WHEN("the data observable calls on_error on subscribing it") {
THEN("the buffer never tries to subscribe to their control observable") {
auto snk = flow::make_passive_observer<cow_vector<int>>();
auto cnt = std::make_shared<size_t>(0);
auto uut = raw_sub(3,
ctx->make_observable().fail<int>(sec::runtime_error),
flow::make_nil_observable<int64_t>(ctx.get(), cnt),
snk->as_observer());
ctx->run();
CHECK(snk->aborted());
CHECK_EQ(*cnt, 0u);
}
}
}
}
SCENARIO("buffers dispose unexpected subscriptions") {
GIVEN("an initialized buffer operator") {
WHEN("calling on_subscribe with unexpected subscriptions") {
THEN("the buffer disposes them immediately") {
auto snk = flow::make_passive_observer<cow_vector<int>>();
auto grd = make_unsubscribe_guard(snk);
auto uut = raw_sub(3, flow::make_nil_observable<int>(ctx.get()),
flow::make_nil_observable<int64_t>(ctx.get()),
snk->as_observer());
auto data_sub = make_counted<flow::passive_subscription_impl>();
auto ctrl_sub = make_counted<flow::passive_subscription_impl>();
uut->fwd_on_subscribe(fwd_data, flow::subscription{data_sub});
uut->fwd_on_subscribe(fwd_ctrl, flow::subscription{ctrl_sub});
ctx->run();
auto data_sub_2 = make_counted<flow::passive_subscription_impl>();
auto ctrl_sub_2 = make_counted<flow::passive_subscription_impl>();
uut->fwd_on_subscribe(fwd_data, flow::subscription{data_sub_2});
uut->fwd_on_subscribe(fwd_ctrl, flow::subscription{ctrl_sub_2});
ctx->run();
CHECK(!uut->disposed());
CHECK(!data_sub->disposed());
CHECK(!ctrl_sub->disposed());
CHECK(data_sub_2->disposed());
CHECK(ctrl_sub_2->disposed());
}
}
}
}
SCENARIO("buffers emit final items after an on_error event") {
GIVEN("an initialized buffer operator") {
WHEN("calling on_error(data) on a buffer without pending data") {
THEN("the buffer forward on_error immediately") {
auto snk = flow::make_passive_observer<cow_vector<int>>();
auto uut = raw_sub(3, trivial_obs<int>(), trivial_obs<int64_t>(),
snk->as_observer());
snk->request(42);
ctx->run();
uut->fwd_on_next(fwd_data, 1);
uut->fwd_on_next(fwd_data, 2);
uut->fwd_on_next(fwd_data, 3);
CHECK_EQ(uut->pending(), 0u);
uut->fwd_on_error(fwd_data, sec::runtime_error);
CHECK_EQ(snk->buf, std::vector{cow_vector<int>({1, 2, 3})});
CHECK(snk->aborted());
}
}
WHEN("calling on_error(data) on a buffer with pending data") {
THEN("the buffer still emits pending data before closing") {
auto snk = flow::make_passive_observer<cow_vector<int>>();
auto uut = raw_sub(3, trivial_obs<int>(), trivial_obs<int64_t>(),
snk->as_observer());
ctx->run();
uut->fwd_on_next(fwd_data, 1);
uut->fwd_on_next(fwd_data, 2);
CHECK_EQ(uut->pending(), 2u);
uut->fwd_on_error(fwd_data, sec::runtime_error);
CHECK(snk->buf.empty());
CHECK(!snk->aborted());
snk->request(42);
ctx->run();
CHECK_EQ(snk->buf, std::vector{cow_vector<int>({1, 2})});
CHECK(snk->aborted());
}
}
WHEN("calling on_error(control) on a buffer without pending data") {
THEN("the buffer forward on_error immediately") {
auto snk = flow::make_passive_observer<cow_vector<int>>();
auto uut = raw_sub(3, trivial_obs<int>(), trivial_obs<int64_t>(),
snk->as_observer());
snk->request(42);
ctx->run();
uut->fwd_on_next(fwd_data, 1);
uut->fwd_on_next(fwd_data, 2);
uut->fwd_on_next(fwd_data, 3);
CHECK_EQ(uut->pending(), 0u);
uut->fwd_on_error(fwd_ctrl, sec::runtime_error);
CHECK_EQ(snk->buf, std::vector{cow_vector<int>({1, 2, 3})});
CHECK(snk->aborted());
}
}
WHEN("calling on_error(control) on a buffer with pending data") {
THEN("the buffer still emits pending data before closing") {
auto snk = flow::make_passive_observer<cow_vector<int>>();
auto uut = raw_sub(3, trivial_obs<int>(), trivial_obs<int64_t>(),
snk->as_observer());
ctx->run();
uut->fwd_on_next(fwd_data, 1);
uut->fwd_on_next(fwd_data, 2);
CHECK_EQ(uut->pending(), 2u);
uut->fwd_on_error(fwd_ctrl, sec::runtime_error);
CHECK(snk->buf.empty());
CHECK(!snk->aborted());
snk->request(42);
ctx->run();
CHECK_EQ(snk->buf, std::vector{cow_vector<int>({1, 2})});
CHECK(snk->aborted());
}
}
}
}
SCENARIO("buffers emit final items after an on_complete event") {
GIVEN("an initialized buffer operator") {
WHEN("calling on_complete(data) on a buffer without pending data") {
THEN("the buffer forward on_complete immediately") {
auto snk = flow::make_passive_observer<cow_vector<int>>();
auto uut = raw_sub(3, trivial_obs<int>(), trivial_obs<int64_t>(),
snk->as_observer());
snk->request(42);
ctx->run();
uut->fwd_on_next(fwd_data, 1);
uut->fwd_on_next(fwd_data, 2);
uut->fwd_on_next(fwd_data, 3);
CHECK_EQ(uut->pending(), 0u);
uut->fwd_on_complete(fwd_data);
CHECK_EQ(snk->buf, std::vector{cow_vector<int>({1, 2, 3})});
CHECK(snk->completed());
}
}
WHEN("calling on_complete(data) on a buffer with pending data") {
THEN("the buffer still emits pending data before closing") {
auto snk = flow::make_passive_observer<cow_vector<int>>();
auto uut = raw_sub(3, trivial_obs<int>(), trivial_obs<int64_t>(),
snk->as_observer());
ctx->run();
uut->fwd_on_next(fwd_data, 1);
uut->fwd_on_next(fwd_data, 2);
CHECK_EQ(uut->pending(), 2u);
uut->fwd_on_complete(fwd_data);
CHECK(snk->buf.empty());
CHECK(!snk->completed());
snk->request(42);
ctx->run();
CHECK_EQ(snk->buf, std::vector{cow_vector<int>({1, 2})});
CHECK(snk->completed());
}
}
WHEN("calling on_complete(control) on a buffer without pending data") {
THEN("the buffer raises an error immediately") {
auto snk = flow::make_passive_observer<cow_vector<int>>();
auto uut = raw_sub(3, trivial_obs<int>(), trivial_obs<int64_t>(),
snk->as_observer());
snk->request(42);
ctx->run();
uut->fwd_on_next(fwd_data, 1);
uut->fwd_on_next(fwd_data, 2);
uut->fwd_on_next(fwd_data, 3);
CHECK_EQ(uut->pending(), 0u);
uut->fwd_on_complete(fwd_ctrl);
CHECK_EQ(snk->buf, std::vector{cow_vector<int>({1, 2, 3})});
CHECK(snk->aborted());
}
}
WHEN("calling on_complete(control) on a buffer with pending data") {
THEN("the buffer raises an error after shipping pending items") {
auto snk = flow::make_passive_observer<cow_vector<int>>();
auto uut = raw_sub(3, trivial_obs<int>(), trivial_obs<int64_t>(),
snk->as_observer());
ctx->run();
uut->fwd_on_next(fwd_data, 1);
uut->fwd_on_next(fwd_data, 2);
CHECK_EQ(uut->pending(), 2u);
uut->fwd_on_complete(fwd_ctrl);
CHECK(snk->buf.empty());
CHECK(!snk->completed());
snk->request(42);
ctx->run();
CHECK_EQ(snk->buf, std::vector{cow_vector<int>({1, 2})});
CHECK(snk->aborted());
}
}
}
}
SCENARIO("skip policies suppress empty batches") {
GIVEN("a buffer operator") {
WHEN("the control observable fires with no pending data") {
THEN("the operator omits the batch") {
auto snk = flow::make_passive_observer<cow_vector<int>>();
auto grd = make_unsubscribe_guard(snk);
auto uut = raw_sub<skip_trait>(3, trivial_obs<int>(),
trivial_obs<int64_t>(),
snk->as_observer());
add_subs(uut);
snk->request(42);
ctx->run();
uut->fwd_on_next(fwd_ctrl, 1);
ctx->run();
CHECK(snk->buf.empty());
}
}
WHEN("the control observable fires with pending data") {
THEN("the operator emits a partial batch") {
auto snk = flow::make_passive_observer<cow_vector<int>>();
auto grd = make_unsubscribe_guard(snk);
auto uut = raw_sub<skip_trait>(3, trivial_obs<int>(),
trivial_obs<int64_t>(),
snk->as_observer());
add_subs(uut);
snk->request(42);
ctx->run();
uut->fwd_on_next(fwd_data, 17);
uut->fwd_on_next(fwd_ctrl, 1);
ctx->run();
CHECK_EQ(snk->buf, std::vector{cow_vector<int>{17}});
}
}
}
}
SCENARIO("no-skip policies emit empty batches") {
GIVEN("a buffer operator") {
WHEN("the control observable fires with no pending data") {
THEN("the operator emits an empty batch") {
auto snk = flow::make_passive_observer<cow_vector<int>>();
auto grd = make_unsubscribe_guard(snk);
auto uut = raw_sub<noskip_trait>(3, trivial_obs<int>(),
trivial_obs<int64_t>(),
snk->as_observer());
add_subs(uut);
snk->request(42);
ctx->run();
uut->fwd_on_next(fwd_ctrl, 1);
ctx->run();
CHECK_EQ(snk->buf, std::vector{cow_vector<int>()});
}
}
WHEN("the control observable fires with pending data") {
THEN("the operator emits a partial batch") {
auto snk = flow::make_passive_observer<cow_vector<int>>();
auto grd = make_unsubscribe_guard(snk);
auto uut = raw_sub<noskip_trait>(3, trivial_obs<int>(),
trivial_obs<int64_t>(),
snk->as_observer());
add_subs(uut);
snk->request(42);
ctx->run();
uut->fwd_on_next(fwd_data, 17);
uut->fwd_on_next(fwd_ctrl, 1);
ctx->run();
CHECK_EQ(snk->buf, std::vector{cow_vector<int>{17}});
}
}
}
}
SCENARIO("disposing a buffer operator completes the flow") {
GIVEN("a buffer operator") {
WHEN("disposing the subscription operator of the operator") {
THEN("the observer receives an on_complete event") {
auto snk = flow::make_passive_observer<cow_vector<int>>();
auto uut = raw_sub<skip_trait>(3, trivial_obs<int>(),
trivial_obs<int64_t>(),
snk->as_observer());
add_subs(uut);
snk->request(42);
ctx->run();
uut->dispose();
ctx->run();
CHECK(snk->completed());
}
}
}
}
SCENARIO("on_request actions can turn into no-ops") {
GIVEN("a buffer operator") {
WHEN("the sink requests more data right before a timeout triggers") {
THEN("the batch gets shipped and the on_request action does nothing") {
auto snk = flow::make_passive_observer<cow_vector<int>>();
auto grd = make_unsubscribe_guard(snk);
auto uut = raw_sub<skip_trait>(3, trivial_obs<int>(),
trivial_obs<int64_t>(),
snk->as_observer());
add_subs(uut);
ctx->run();
// Add three items that we can't push yet because no downstream demand.
for (int i = 0; i < 3; ++i)
uut->fwd_on_next(fwd_data, i);
CHECK(uut->can_emit());
CHECK_EQ(uut->pending(), 3u);
// Add demand, which triggers an action - but don't run it yet.
snk->request(42);
CHECK_EQ(uut->pending(), 3u);
// Fire on_next on the control channel to force the batch out.
uut->fwd_on_next(fwd_ctrl, 1);
CHECK_EQ(uut->pending(), 0u);
// Run the scheduled action: turns into a no-op now.
ctx->run();
CHECK_EQ(snk->buf, std::vector{cow_vector<int>({0, 1, 2})});
}
}
}
}
END_FIXTURE_SCOPE()
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