Unverified Commit 12a5ee09 authored by Dominik Charousset's avatar Dominik Charousset Committed by GitHub

Merge pull request #1469

Add a new abstraction: publishers
parents 83d44fb3 3e5f62b3
...@@ -18,6 +18,13 @@ is based on [Keep a Changelog](https://keepachangelog.com). ...@@ -18,6 +18,13 @@ is based on [Keep a Changelog](https://keepachangelog.com).
`--foo=bar` syntax. `--foo=bar` syntax.
- The functions `make_message` and `make_error` now support `std::string_view` - The functions `make_message` and `make_error` now support `std::string_view`
as input and automatically convert it to `std::string`. as input and automatically convert it to `std::string`.
- To make it easier to set up asynchronous flows, CAF now provides a new class:
`caf::async::publisher`. Any observable can be transformed into a publisher by
calling `to_publisher`. The publisher can then be used to subscribe to the
observable from other actors or threads. The publisher has only a single
member function: `observe_on`. It converts the publisher back into an
observable. This new abstraction allows users to set up asynchronous flows
without having to manually deal with SPSC buffers.
### Changed ### Changed
...@@ -28,6 +35,9 @@ is based on [Keep a Changelog](https://keepachangelog.com). ...@@ -28,6 +35,9 @@ is based on [Keep a Changelog](https://keepachangelog.com).
- The output of `--dump-config` now only contains CAF options from loaded - The output of `--dump-config` now only contains CAF options from loaded
modules. Previously, it also included options from modules that were not modules. Previously, it also included options from modules that were not
loaded. loaded.
- We renamed `caf::flow::item_publisher` to `caf::flow::multicaster` to better
reflect its purpose and to avoid confusion with the new
`caf::async::publisher`.
### Fixed ### Fixed
......
...@@ -41,7 +41,7 @@ void worker_impl(caf::event_based_actor* self, ...@@ -41,7 +41,7 @@ void worker_impl(caf::event_based_actor* self,
// add this ID to the input to tag it. // add this ID to the input to tag it.
using message_t = std::pair<caf::uuid, lp::frame>; using message_t = std::pair<caf::uuid, lp::frame>;
// Allows us to push new flows into the central merge point. // Allows us to push new flows into the central merge point.
caf::flow::item_publisher<caf::flow::observable<message_t>> pub{self}; caf::flow::multicaster<caf::flow::observable<message_t>> pub{self};
// Our central merge point combines all inputs into a single, shared flow. // Our central merge point combines all inputs into a single, shared flow.
auto messages = pub.as_observable().merge().share(); auto messages = pub.as_observable().merge().share();
// Have one subscription for debug output. This also makes sure that the // Have one subscription for debug output. This also makes sure that the
......
...@@ -32,7 +32,7 @@ public: ...@@ -32,7 +32,7 @@ public:
using frame = caf::net::lp::frame; using frame = caf::net::lp::frame;
using publisher_type = caf::flow::item_publisher<QString>; using publisher_type = caf::flow::multicaster<QString>;
ChatWidget(QWidget* parent = nullptr); ChatWidget(QWidget* parent = nullptr);
......
...@@ -234,6 +234,7 @@ caf_add_component( ...@@ -234,6 +234,7 @@ caf_add_component(
async.consumer_adapter async.consumer_adapter
async.producer_adapter async.producer_adapter
async.promise async.promise
async.publisher
async.spsc_buffer async.spsc_buffer
behavior behavior
binary_deserializer binary_deserializer
...@@ -286,8 +287,8 @@ caf_add_component( ...@@ -286,8 +287,8 @@ caf_add_component(
flow.flat_map flow.flat_map
flow.for_each flow.for_each
flow.generation flow.generation
flow.item_publisher
flow.mixed flow.mixed
flow.multicaster
flow.observe_on flow.observe_on
flow.op.buffer flow.op.buffer
flow.op.cell flow.op.cell
......
...@@ -18,10 +18,10 @@ class producer; ...@@ -18,10 +18,10 @@ class producer;
// -- template classes --------------------------------------------------------- // -- template classes ---------------------------------------------------------
template <class T> template <class T>
class spsc_buffer; class consumer_resource;
template <class T> template <class T>
class consumer_resource; class future;
template <class T> template <class T>
class producer_resource; class producer_resource;
...@@ -30,7 +30,10 @@ template <class T> ...@@ -30,7 +30,10 @@ template <class T>
class promise; class promise;
template <class T> template <class T>
class future; class publisher;
template <class T>
class spsc_buffer;
// -- smart pointer aliases ---------------------------------------------------- // -- smart pointer aliases ----------------------------------------------------
......
// This file is part of CAF, the C++ Actor Framework. See the file LICENSE in
// the main distribution directory for license terms and copyright or visit
// https://github.com/actor-framework/actor-framework/blob/master/LICENSE.
#pragma once
#include "caf/async/spsc_buffer.hpp"
#include "caf/flow/coordinator.hpp"
#include "caf/flow/observable_decl.hpp"
#include "caf/flow/op/fail.hpp"
namespace caf::async {
/// Provides an interface for accessing an asynchronous data flow. Unlike a @ref
/// future, a publisher produces multiple values over time. Subscribers will
/// only receive items that are emitted after they have subscribed to the
/// publisher.
template <class T>
class publisher {
public:
publisher() noexcept = default;
publisher(publisher&&) noexcept = default;
publisher(const publisher&) noexcept = default;
publisher& operator=(publisher&&) noexcept = default;
publisher& operator=(const publisher&) noexcept = default;
/// Creates a @ref flow::observable that reads and emits all values from this
/// publisher.
flow::observable<T> observe_on(flow::coordinator* ctx, size_t buffer_size,
size_t min_request_size) {
if (impl_)
return impl_->observe_on(ctx, buffer_size, min_request_size);
auto err = make_error(sec::invalid_observable,
"cannot subscribe to default-constructed observable");
// Note: cannot use ctx->make_observable() here because it would create a
// circular dependency between observable_builder and publisher.
return flow::make_observable<flow::op::fail<T>>(ctx, std::move(err));
}
/// Creates a @ref flow::observable that reads and emits all values from this
/// publisher.
flow::observable<T> observe_on(flow::coordinator* ctx) {
return observe_on(ctx, defaults::flow::buffer_size,
defaults::flow::min_demand);
}
/// Creates a new asynchronous observable by decorating a regular observable.
static publisher from(flow::observable<T> decorated) {
if (!decorated)
return {};
auto* ctx = decorated.ctx();
auto flag = disposable::make_flag();
ctx->watch(flag);
auto pimpl = make_counted<impl>(ctx, std::move(decorated), std::move(flag));
return publisher{std::move(pimpl)};
}
private:
class impl : public ref_counted {
public:
impl(async::execution_context_ptr source, flow::observable<T> decorated,
disposable flag)
: source_(std::move(source)),
decorated_(std::move(decorated)),
flag_(std::move(flag)) {
// nop
}
flow::observable<T> observe_on(flow::coordinator* ctx, size_t buffer_size,
size_t min_request_size) {
// Short circuit if we are already on the target coordinator.
if (ctx == source_.get())
return decorated_;
// Otherwise, create a new SPSC buffer and connect it to the source.
auto [pull, push] = async::make_spsc_buffer_resource<T>(buffer_size,
min_request_size);
source_->schedule_fn(
[push = std::move(push), decorated = decorated_]() mutable {
decorated.subscribe(std::move(push));
});
return pull.observe_on(ctx);
}
~impl() {
source_->schedule_fn([flag = std::move(flag_)]() mutable {
// The source called `watch` on the flag to keep the event loop alive as
// long as there are still async references to this observable. We need
// to dispose the flag in the event loop of the source in order to make
// sure that the source cleans up properly.
flag.dispose();
});
}
private:
async::execution_context_ptr source_;
flow::observable<T> decorated_;
disposable flag_;
};
using impl_ptr = caf::intrusive_ptr<impl>;
explicit publisher(impl_ptr pimpl) : impl_(std::move(pimpl)) {
// nop
}
impl_ptr impl_;
};
} // namespace caf::async
...@@ -69,6 +69,9 @@ public: ...@@ -69,6 +69,9 @@ public:
/// disposes all elements individually. /// disposes all elements individually.
static disposable make_composite(std::vector<disposable> entries); static disposable make_composite(std::vector<disposable> entries);
/// Creates a disposable that simply represents a flag.
static disposable make_flag();
// -- mutators --------------------------------------------------------------- // -- mutators ---------------------------------------------------------------
/// Disposes the resource. Calling `dispose()` on a disposed resource is a /// Disposes the resource. Calling `dispose()` on a disposed resource is a
......
...@@ -44,6 +44,9 @@ class observable_def; ...@@ -44,6 +44,9 @@ class observable_def;
template <class Generator> template <class Generator>
class generation_materializer; class generation_materializer;
template <class T>
class multicaster;
/// A blueprint for an @ref observer that generates items and applies any number /// A blueprint for an @ref observer that generates items and applies any number
/// of processing steps immediately before emitting them. /// of processing steps immediately before emitting them.
template <class Generator, class... Steps> template <class Generator, class... Steps>
......
...@@ -4,102 +4,11 @@ ...@@ -4,102 +4,11 @@
#pragma once #pragma once
#include "caf/flow/fwd.hpp" #include "caf/flow/multicaster.hpp"
#include "caf/flow/observable_decl.hpp"
#include "caf/flow/op/mcast.hpp"
#include "caf/intrusive_ptr.hpp"
#include "caf/make_counted.hpp"
#include <cstdint>
namespace caf::flow { namespace caf::flow {
template <class T> template <class T>
class item_publisher { using item_publisher [[deprecated("use multicaster instead")]] = multicaster<T>;
public:
using impl_ptr = intrusive_ptr<op::mcast<T>>;
explicit item_publisher(coordinator* ctx) {
pimpl_ = make_counted<op::mcast<T>>(ctx);
}
explicit item_publisher(impl_ptr ptr) noexcept : pimpl_(std::move(ptr)) {
// nop
}
item_publisher(item_publisher&&) noexcept = default;
item_publisher& operator=(item_publisher&&) noexcept = default;
item_publisher(const item_publisher&) = delete;
item_publisher& operator=(const item_publisher&) = delete;
~item_publisher() {
if (pimpl_)
pimpl_->close();
}
/// Pushes an item to all subscribed observers. The publisher drops the item
/// if no subscriber exists.
void push(const T& item) {
pimpl_->push_all(item);
}
/// Pushes the items in range `[first, last)` to all subscribed observers. The
/// publisher drops the items if no subscriber exists.
template <class Iterator, class Sentinel>
void push(Iterator first, Sentinel last) {
while (first != last)
push(*first++);
}
/// Pushes the items from the initializer list to all subscribed observers.
/// The publisher drops the items if no subscriber exists.
void push(std::initializer_list<T> items) {
for (auto& item : items)
push(item);
}
/// Closes the publisher, eventually emitting on_complete on all observers.
void close() {
pimpl_->close();
}
/// Closes the publisher, eventually emitting on_error on all observers.
void abort(const error& reason) {
pimpl_->abort(reason);
}
/// Queries how many items the publisher may emit immediately to subscribed
/// observers.
size_t demand() const noexcept {
return pimpl_->min_demand();
}
/// Queries how many items are currently waiting in a buffer until the
/// observer requests additional items.
size_t buffered() const noexcept {
return pimpl_->max_buffered();
}
/// Queries whether there is at least one observer subscribed to the operator.
bool has_observers() const noexcept {
return pimpl_->has_observers();
}
/// Converts the publisher to an @ref observable.
observable<T> as_observable() const {
return observable<T>{pimpl_};
}
/// Subscribes a new @ref observer to the output of the publisher.
disposable subscribe(observer<T> out) {
return pimpl_->subscribe(out);
}
private:
impl_ptr pimpl_;
};
} // namespace caf::flow } // namespace caf::flow
// This file is part of CAF, the C++ Actor Framework. See the file LICENSE in
// the main distribution directory for license terms and copyright or visit
// https://github.com/actor-framework/actor-framework/blob/master/LICENSE.
#pragma once
#include "caf/flow/fwd.hpp"
#include "caf/flow/observable_decl.hpp"
#include "caf/flow/op/mcast.hpp"
#include "caf/intrusive_ptr.hpp"
#include "caf/make_counted.hpp"
#include <cstdint>
namespace caf::flow {
/// A multicaster pushes items to any number of subscribers.
template <class T>
class multicaster {
public:
using impl_ptr = intrusive_ptr<op::mcast<T>>;
explicit multicaster(coordinator* ctx) {
pimpl_ = make_counted<op::mcast<T>>(ctx);
}
explicit multicaster(impl_ptr ptr) noexcept : pimpl_(std::move(ptr)) {
// nop
}
multicaster(multicaster&&) noexcept = default;
multicaster& operator=(multicaster&&) noexcept = default;
multicaster(const multicaster&) = delete;
multicaster& operator=(const multicaster&) = delete;
~multicaster() {
if (pimpl_)
pimpl_->close();
}
/// Pushes an item to all subscribed observers. The publisher drops the item
/// if no subscriber exists.
void push(const T& item) {
pimpl_->push_all(item);
}
/// Pushes the items in range `[first, last)` to all subscribed observers. The
/// publisher drops the items if no subscriber exists.
template <class Iterator, class Sentinel>
void push(Iterator first, Sentinel last) {
while (first != last)
push(*first++);
}
/// Pushes the items from the initializer list to all subscribed observers.
/// The publisher drops the items if no subscriber exists.
void push(std::initializer_list<T> items) {
for (auto& item : items)
push(item);
}
/// Closes the publisher, eventually emitting on_complete on all observers.
void close() {
pimpl_->close();
}
/// Closes the publisher, eventually emitting on_error on all observers.
void abort(const error& reason) {
pimpl_->abort(reason);
}
/// Queries how many items the publisher may emit immediately to subscribed
/// observers.
size_t demand() const noexcept {
return pimpl_->min_demand();
}
/// Queries how many items are currently waiting in a buffer until the
/// observer requests additional items.
size_t buffered() const noexcept {
return pimpl_->max_buffered();
}
/// Queries whether there is at least one observer subscribed to the operator.
bool has_observers() const noexcept {
return pimpl_->has_observers();
}
/// Converts the publisher to an @ref observable.
observable<T> as_observable() const {
return observable<T>{pimpl_};
}
/// Subscribes a new @ref observer to the output of the publisher.
disposable subscribe(observer<T> out) {
return pimpl_->subscribe(out);
}
private:
impl_ptr pimpl_;
};
} // namespace caf::flow
...@@ -6,6 +6,7 @@ ...@@ -6,6 +6,7 @@
#include "caf/async/consumer.hpp" #include "caf/async/consumer.hpp"
#include "caf/async/producer.hpp" #include "caf/async/producer.hpp"
#include "caf/async/publisher.hpp"
#include "caf/async/spsc_buffer.hpp" #include "caf/async/spsc_buffer.hpp"
#include "caf/cow_tuple.hpp" #include "caf/cow_tuple.hpp"
#include "caf/cow_vector.hpp" #include "caf/cow_vector.hpp"
...@@ -377,6 +378,11 @@ public: ...@@ -377,6 +378,11 @@ public:
return materialize().to_resource(buffer_size, min_request_size); return materialize().to_resource(buffer_size, min_request_size);
} }
/// @copydoc observable::to_resource
async::publisher<output_type> to_publisher() && {
return materialize().to_publisher();
}
/// @copydoc observable::observe_on /// @copydoc observable::observe_on
observable<output_type> observe_on(coordinator* other) && { observable<output_type> observe_on(coordinator* other) && {
return materialize().observe_on(other); return materialize().observe_on(other);
...@@ -762,4 +768,9 @@ observable<T>::to_resource(size_t buffer_size, size_t min_request_size) { ...@@ -762,4 +768,9 @@ observable<T>::to_resource(size_t buffer_size, size_t min_request_size) {
return async::consumer_resource<T>{std::move(buf)}; return async::consumer_resource<T>{std::move(buf)};
} }
template <class T>
async::publisher<T> observable<T>::to_publisher() {
return async::publisher<T>::from(*this);
}
} // namespace caf::flow } // namespace caf::flow
...@@ -4,6 +4,7 @@ ...@@ -4,6 +4,7 @@
#pragma once #pragma once
#include "caf/async/fwd.hpp"
#include "caf/cow_vector.hpp" #include "caf/cow_vector.hpp"
#include "caf/defaults.hpp" #include "caf/defaults.hpp"
#include "caf/detail/is_complete.hpp" #include "caf/detail/is_complete.hpp"
...@@ -250,6 +251,9 @@ public: ...@@ -250,6 +251,9 @@ public:
return to_resource(defaults::flow::buffer_size, defaults::flow::min_demand); return to_resource(defaults::flow::buffer_size, defaults::flow::min_demand);
} }
/// Creates a publisher that makes emitted items available asynchronously.
async::publisher<T> to_publisher();
const observable& as_observable() const& noexcept { const observable& as_observable() const& noexcept {
return *this; return *this;
} }
......
...@@ -4,7 +4,7 @@ ...@@ -4,7 +4,7 @@
#pragma once #pragma once
#include "caf/detail/plain_ref_counted.hpp" #include "caf/detail/atomic_ref_counted.hpp"
#include "caf/flow/observer.hpp" #include "caf/flow/observer.hpp"
#include "caf/flow/op/base.hpp" #include "caf/flow/op/base.hpp"
#include "caf/flow/subscription.hpp" #include "caf/flow/subscription.hpp"
...@@ -13,7 +13,7 @@ namespace caf::flow::op { ...@@ -13,7 +13,7 @@ namespace caf::flow::op {
/// Convenience base type for *cold* observable types. /// Convenience base type for *cold* observable types.
template <class T> template <class T>
class cold : public detail::plain_ref_counted, public base<T> { class cold : public detail::atomic_ref_counted, public base<T> {
public: public:
// -- member types ----------------------------------------------------------- // -- member types -----------------------------------------------------------
......
...@@ -4,7 +4,7 @@ ...@@ -4,7 +4,7 @@
#pragma once #pragma once
#include "caf/detail/plain_ref_counted.hpp" #include "caf/detail/atomic_ref_counted.hpp"
#include "caf/flow/observer.hpp" #include "caf/flow/observer.hpp"
#include "caf/flow/op/base.hpp" #include "caf/flow/op/base.hpp"
#include "caf/flow/subscription.hpp" #include "caf/flow/subscription.hpp"
...@@ -13,7 +13,7 @@ namespace caf::flow::op { ...@@ -13,7 +13,7 @@ namespace caf::flow::op {
/// Convenience base type for *hot* observable types. /// Convenience base type for *hot* observable types.
template <class T> template <class T>
class hot : public detail::plain_ref_counted, public base<T> { class hot : public detail::atomic_ref_counted, public base<T> {
public: public:
// -- member types ----------------------------------------------------------- // -- member types -----------------------------------------------------------
......
...@@ -27,7 +27,8 @@ ...@@ -27,7 +27,8 @@
#include "caf/extend.hpp" #include "caf/extend.hpp"
#include "caf/flow/coordinator.hpp" #include "caf/flow/coordinator.hpp"
#include "caf/flow/fwd.hpp" #include "caf/flow/fwd.hpp"
#include "caf/flow/item_publisher.hpp" #include "caf/flow/item_publisher.hpp" // deprecated
#include "caf/flow/multicaster.hpp"
#include "caf/flow/observer.hpp" #include "caf/flow/observer.hpp"
#include "caf/fwd.hpp" #include "caf/fwd.hpp"
#include "caf/intrusive/drr_cached_queue.hpp" #include "caf/intrusive/drr_cached_queue.hpp"
...@@ -177,7 +178,7 @@ public: ...@@ -177,7 +178,7 @@ public:
using exception_handler = std::function<error(pointer, std::exception_ptr&)>; using exception_handler = std::function<error(pointer, std::exception_ptr&)>;
#endif // CAF_ENABLE_EXCEPTIONS #endif // CAF_ENABLE_EXCEPTIONS
using batch_publisher = flow::item_publisher<async::batch>; using batch_publisher = flow::multicaster<async::batch>;
class batch_forwarder : public ref_counted { class batch_forwarder : public ref_counted {
public: public:
......
...@@ -71,6 +71,48 @@ disposable disposable::make_composite(std::vector<disposable> entries) { ...@@ -71,6 +71,48 @@ disposable disposable::make_composite(std::vector<disposable> entries) {
return disposable{make_counted<composite_impl>(std::move(entries))}; return disposable{make_counted<composite_impl>(std::move(entries))};
} }
namespace {
class flag_impl : public ref_counted, public disposable::impl {
public:
flag_impl() : flag_(false) {
// nop
}
void dispose() {
flag_ = true;
}
bool disposed() const noexcept {
return flag_.load();
}
void ref_disposable() const noexcept {
ref();
}
void deref_disposable() const noexcept {
deref();
}
friend void intrusive_ptr_add_ref(const flag_impl* ptr) noexcept {
ptr->ref();
}
friend void intrusive_ptr_release(const flag_impl* ptr) noexcept {
ptr->deref();
}
private:
std::atomic<bool> flag_;
};
} // namespace
disposable disposable::make_flag() {
return disposable{make_counted<flag_impl>()};
}
// -- utility ------------------------------------------------------------------ // -- utility ------------------------------------------------------------------
size_t disposable::erase_disposed(std::vector<disposable>& xs) { size_t disposable::erase_disposed(std::vector<disposable>& xs) {
......
// 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 async.publisher
#include "caf/async/publisher.hpp"
#include "core-test.hpp"
#include "caf/flow/scoped_coordinator.hpp"
#include "caf/scheduled_actor/flow.hpp"
using namespace caf;
BEGIN_FIXTURE_SCOPE(test_coordinator_fixture<>)
SCENARIO("actors can subscribe to their own publishers") {
GIVEN("an observable") {
WHEN("converting it to a publisher") {
THEN("the owning actor can subscribe to it") {
auto values = std::make_shared<std::vector<int>>();
sys.spawn([values](caf::event_based_actor* self) {
self->make_observable()
.iota(1)
.take(7)
.to_publisher()
.observe_on(self)
.do_on_error([](const error& what) { FAIL("error: " << what); })
.for_each([values](int x) { values->push_back(x); });
});
run();
CHECK_EQ(*values, std::vector({1, 2, 3, 4, 5, 6, 7}));
}
}
}
}
SCENARIO("default-constructed publishers are invalid") {
GIVEN("a default-constructed publisher") {
WHEN("an actor subscribes to it") {
THEN("the actor observes an invalid_observable error") {
auto err = std::make_shared<error>();
sys.spawn([err](caf::event_based_actor* self) {
auto items = async::publisher<int>{};
items.observe_on(self)
.do_on_error([err](const error& what) { *err = what; })
.for_each([](int) { FAIL("unexpected value"); });
});
run();
CHECK_EQ(*err, sec::invalid_observable);
}
}
}
}
SCENARIO("publishers from default-constructed observables are invalid") {
GIVEN("publisher with a default-constructed observable") {
WHEN("an actor subscribes to it") {
THEN("the actor observes an invalid_observable error") {
auto err = std::make_shared<error>();
sys.spawn([err](caf::event_based_actor* self) {
auto items = async::publisher<int>::from({});
items.observe_on(self)
.do_on_error([err](const error& what) { *err = what; })
.for_each([](int) { FAIL("unexpected value"); });
});
run();
CHECK_EQ(*err, sec::invalid_observable);
}
}
}
}
SCENARIO("actors can subscribe to publishers from other actors") {
GIVEN("three actors") {
WHEN("creating a publisher on one and subscribing on the others") {
THEN("the other actors receive the values") {
auto v1 = std::make_shared<std::vector<int>>();
auto v2 = std::make_shared<std::vector<int>>();
auto items = std::make_shared<async::publisher<int>>();
sys.spawn([items](caf::event_based_actor* self) {
*items = self->make_observable().iota(1).take(7).to_publisher();
});
run();
auto consumer = [items](caf::event_based_actor* self,
std::shared_ptr<std::vector<int>> values) {
items->observe_on(self)
.do_on_error([](const error& what) { FAIL("error: " << what); })
.for_each([values](int x) { values->push_back(x); });
};
sys.spawn(consumer, v1);
sys.spawn(consumer, v2);
run();
CHECK_EQ(*v1, std::vector({1, 2, 3, 4, 5, 6, 7}));
CHECK_EQ(*v2, std::vector({1, 2, 3, 4, 5, 6, 7}));
}
}
}
}
SCENARIO("publishers from terminated actors produce errors") {
GIVEN("a publisher from a terminated actor") {
WHEN("another actors subscribes to it") {
THEN("the subscriber observe an error") {
auto items = std::make_shared<async::publisher<int>>();
sys.spawn([items](caf::event_based_actor* self) {
*items = self->make_observable().iota(1).take(7).to_publisher();
self->quit();
});
run();
auto err = std::make_shared<error>();
sys.spawn([items, err](caf::event_based_actor* self) {
items->observe_on(self)
.do_on_error([err](const error& what) { *err = what; })
.for_each([](int) { FAIL("unexpected value"); });
});
run();
CHECK_EQ(*err, sec::disposed);
}
}
}
}
END_FIXTURE_SCOPE()
...@@ -2,9 +2,9 @@ ...@@ -2,9 +2,9 @@
// the main distribution directory for license terms and copyright or visit // the main distribution directory for license terms and copyright or visit
// https://github.com/actor-framework/actor-framework/blob/master/LICENSE. // https://github.com/actor-framework/actor-framework/blob/master/LICENSE.
#define CAF_SUITE flow.item_publisher #define CAF_SUITE flow.multicaster
#include "caf/flow/item_publisher.hpp" #include "caf/flow/multicaster.hpp"
#include "core-test.hpp" #include "core-test.hpp"
...@@ -29,11 +29,11 @@ struct fixture : test_coordinator_fixture<> { ...@@ -29,11 +29,11 @@ struct fixture : test_coordinator_fixture<> {
BEGIN_FIXTURE_SCOPE(fixture) BEGIN_FIXTURE_SCOPE(fixture)
SCENARIO("item publishers discard items that arrive before a subscriber") { SCENARIO("a multicaster discards items that arrive before a subscriber") {
GIVEN("an item publisher") { GIVEN("an multicaster") {
WHEN("publishing items") { WHEN("pushing items") {
THEN("observers see only items that were published after subscribing") { THEN("observers see only items that were pushed after subscribing") {
auto uut = flow::item_publisher<int>{ctx.get()}; auto uut = flow::multicaster<int>{ctx.get()};
uut.push({1, 2, 3}); uut.push({1, 2, 3});
auto snk = flow::make_auto_observer<int>(); auto snk = flow::make_auto_observer<int>();
uut.subscribe(snk->as_observer()); uut.subscribe(snk->as_observer());
......
...@@ -9,7 +9,7 @@ ...@@ -9,7 +9,7 @@
#include "core-test.hpp" #include "core-test.hpp"
#include "caf/flow/coordinator.hpp" #include "caf/flow/coordinator.hpp"
#include "caf/flow/item_publisher.hpp" #include "caf/flow/multicaster.hpp"
#include "caf/flow/observable.hpp" #include "caf/flow/observable.hpp"
#include "caf/flow/observable_builder.hpp" #include "caf/flow/observable_builder.hpp"
#include "caf/flow/observer.hpp" #include "caf/flow/observer.hpp"
...@@ -118,7 +118,7 @@ SCENARIO("the buffer operator forces items at regular intervals") { ...@@ -118,7 +118,7 @@ SCENARIO("the buffer operator forces items at regular intervals") {
cow_vector<int>{}, cow_vector<int>{64}, cow_vector<int>{}, cow_vector<int>{64},
cow_vector<int>{}, cow_vector<int>{128, 256, 512}, cow_vector<int>{}, cow_vector<int>{128, 256, 512},
}; };
auto pub = flow::item_publisher<int>{ctx.get()}; auto pub = flow::multicaster<int>{ctx.get()};
sys.spawn([&pub, outputs](caf::event_based_actor* self) { sys.spawn([&pub, outputs](caf::event_based_actor* self) {
pub.as_observable() pub.as_observable()
.observe_on(self) // .observe_on(self) //
......
...@@ -8,7 +8,7 @@ ...@@ -8,7 +8,7 @@
#include "core-test.hpp" #include "core-test.hpp"
#include "caf/flow/item_publisher.hpp" #include "caf/flow/multicaster.hpp"
#include "caf/flow/observable_builder.hpp" #include "caf/flow/observable_builder.hpp"
#include "caf/flow/scoped_coordinator.hpp" #include "caf/flow/scoped_coordinator.hpp"
...@@ -89,7 +89,7 @@ SCENARIO("mergers round-robin over their inputs") { ...@@ -89,7 +89,7 @@ SCENARIO("mergers round-robin over their inputs") {
} }
GIVEN("a merger with one input that completes") { GIVEN("a merger with one input that completes") {
WHEN("subscribing to the merger and requesting before the first push") { WHEN("subscribing to the merger and requesting before the first push") {
auto src = flow::item_publisher<int>{ctx.get()}; auto src = flow::multicaster<int>{ctx.get()};
auto uut = make_counted<flow::op::merge<int>>(ctx.get(), auto uut = make_counted<flow::op::merge<int>>(ctx.get(),
src.as_observable()); src.as_observable());
auto snk = flow::make_passive_observer<int>(); auto snk = flow::make_passive_observer<int>();
...@@ -122,7 +122,7 @@ SCENARIO("mergers round-robin over their inputs") { ...@@ -122,7 +122,7 @@ SCENARIO("mergers round-robin over their inputs") {
} }
} }
WHEN("subscribing to the merger pushing before the first request") { WHEN("subscribing to the merger pushing before the first request") {
auto src = flow::item_publisher<int>{ctx.get()}; auto src = flow::multicaster<int>{ctx.get()};
auto uut = make_counted<flow::op::merge<int>>(ctx.get(), auto uut = make_counted<flow::op::merge<int>>(ctx.get(),
src.as_observable()); src.as_observable());
ctx->run(); ctx->run();
...@@ -158,7 +158,7 @@ SCENARIO("mergers round-robin over their inputs") { ...@@ -158,7 +158,7 @@ SCENARIO("mergers round-robin over their inputs") {
} }
GIVEN("a merger with one input that aborts after some items") { GIVEN("a merger with one input that aborts after some items") {
WHEN("subscribing to the merger") { WHEN("subscribing to the merger") {
auto src = flow::item_publisher<int>{ctx.get()}; auto src = flow::multicaster<int>{ctx.get()};
auto uut = make_counted<flow::op::merge<int>>(ctx.get(), auto uut = make_counted<flow::op::merge<int>>(ctx.get(),
src.as_observable()); src.as_observable());
auto snk = flow::make_passive_observer<int>(); auto snk = flow::make_passive_observer<int>();
......
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