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).
`--foo=bar` syntax.
- The functions `make_message` and `make_error` now support `std::string_view`
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
......@@ -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
modules. Previously, it also included options from modules that were not
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
......
......@@ -41,7 +41,7 @@ void worker_impl(caf::event_based_actor* self,
// add this ID to the input to tag it.
using message_t = std::pair<caf::uuid, lp::frame>;
// 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.
auto messages = pub.as_observable().merge().share();
// Have one subscription for debug output. This also makes sure that the
......
......@@ -32,7 +32,7 @@ public:
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);
......
......@@ -234,6 +234,7 @@ caf_add_component(
async.consumer_adapter
async.producer_adapter
async.promise
async.publisher
async.spsc_buffer
behavior
binary_deserializer
......@@ -286,8 +287,8 @@ caf_add_component(
flow.flat_map
flow.for_each
flow.generation
flow.item_publisher
flow.mixed
flow.multicaster
flow.observe_on
flow.op.buffer
flow.op.cell
......
......@@ -18,10 +18,10 @@ class producer;
// -- template classes ---------------------------------------------------------
template <class T>
class spsc_buffer;
class consumer_resource;
template <class T>
class consumer_resource;
class future;
template <class T>
class producer_resource;
......@@ -30,7 +30,10 @@ template <class T>
class promise;
template <class T>
class future;
class publisher;
template <class T>
class spsc_buffer;
// -- 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:
/// disposes all elements individually.
static disposable make_composite(std::vector<disposable> entries);
/// Creates a disposable that simply represents a flag.
static disposable make_flag();
// -- mutators ---------------------------------------------------------------
/// Disposes the resource. Calling `dispose()` on a disposed resource is a
......
......@@ -44,6 +44,9 @@ class observable_def;
template <class Generator>
class generation_materializer;
template <class T>
class multicaster;
/// A blueprint for an @ref observer that generates items and applies any number
/// of processing steps immediately before emitting them.
template <class Generator, class... Steps>
......
......@@ -4,102 +4,11 @@
#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>
#include "caf/flow/multicaster.hpp"
namespace caf::flow {
template <class T>
class item_publisher {
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_;
};
using item_publisher [[deprecated("use multicaster instead")]] = multicaster<T>;
} // 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 @@
#include "caf/async/consumer.hpp"
#include "caf/async/producer.hpp"
#include "caf/async/publisher.hpp"
#include "caf/async/spsc_buffer.hpp"
#include "caf/cow_tuple.hpp"
#include "caf/cow_vector.hpp"
......@@ -377,6 +378,11 @@ public:
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
observable<output_type> observe_on(coordinator* other) && {
return materialize().observe_on(other);
......@@ -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)};
}
template <class T>
async::publisher<T> observable<T>::to_publisher() {
return async::publisher<T>::from(*this);
}
} // namespace caf::flow
......@@ -4,6 +4,7 @@
#pragma once
#include "caf/async/fwd.hpp"
#include "caf/cow_vector.hpp"
#include "caf/defaults.hpp"
#include "caf/detail/is_complete.hpp"
......@@ -250,6 +251,9 @@ public:
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 {
return *this;
}
......
......@@ -4,7 +4,7 @@
#pragma once
#include "caf/detail/plain_ref_counted.hpp"
#include "caf/detail/atomic_ref_counted.hpp"
#include "caf/flow/observer.hpp"
#include "caf/flow/op/base.hpp"
#include "caf/flow/subscription.hpp"
......@@ -13,7 +13,7 @@ namespace caf::flow::op {
/// Convenience base type for *cold* observable types.
template <class T>
class cold : public detail::plain_ref_counted, public base<T> {
class cold : public detail::atomic_ref_counted, public base<T> {
public:
// -- member types -----------------------------------------------------------
......
......@@ -4,7 +4,7 @@
#pragma once
#include "caf/detail/plain_ref_counted.hpp"
#include "caf/detail/atomic_ref_counted.hpp"
#include "caf/flow/observer.hpp"
#include "caf/flow/op/base.hpp"
#include "caf/flow/subscription.hpp"
......@@ -13,7 +13,7 @@ namespace caf::flow::op {
/// Convenience base type for *hot* observable types.
template <class T>
class hot : public detail::plain_ref_counted, public base<T> {
class hot : public detail::atomic_ref_counted, public base<T> {
public:
// -- member types -----------------------------------------------------------
......
......@@ -27,7 +27,8 @@
#include "caf/extend.hpp"
#include "caf/flow/coordinator.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/fwd.hpp"
#include "caf/intrusive/drr_cached_queue.hpp"
......@@ -177,7 +178,7 @@ public:
using exception_handler = std::function<error(pointer, std::exception_ptr&)>;
#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 {
public:
......
......@@ -71,6 +71,48 @@ disposable disposable::make_composite(std::vector<disposable> 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 ------------------------------------------------------------------
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 @@
// 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.item_publisher
#define CAF_SUITE flow.multicaster
#include "caf/flow/item_publisher.hpp"
#include "caf/flow/multicaster.hpp"
#include "core-test.hpp"
......@@ -29,11 +29,11 @@ struct fixture : test_coordinator_fixture<> {
BEGIN_FIXTURE_SCOPE(fixture)
SCENARIO("item publishers discard items that arrive before a subscriber") {
GIVEN("an item publisher") {
WHEN("publishing items") {
THEN("observers see only items that were published after subscribing") {
auto uut = flow::item_publisher<int>{ctx.get()};
SCENARIO("a multicaster discards items that arrive before a subscriber") {
GIVEN("an multicaster") {
WHEN("pushing items") {
THEN("observers see only items that were pushed after subscribing") {
auto uut = flow::multicaster<int>{ctx.get()};
uut.push({1, 2, 3});
auto snk = flow::make_auto_observer<int>();
uut.subscribe(snk->as_observer());
......
......@@ -9,7 +9,7 @@
#include "core-test.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_builder.hpp"
#include "caf/flow/observer.hpp"
......@@ -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>{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) {
pub.as_observable()
.observe_on(self) //
......
......@@ -8,7 +8,7 @@
#include "core-test.hpp"
#include "caf/flow/item_publisher.hpp"
#include "caf/flow/multicaster.hpp"
#include "caf/flow/observable_builder.hpp"
#include "caf/flow/scoped_coordinator.hpp"
......@@ -89,7 +89,7 @@ SCENARIO("mergers round-robin over their inputs") {
}
GIVEN("a merger with one input that completes") {
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(),
src.as_observable());
auto snk = flow::make_passive_observer<int>();
......@@ -122,7 +122,7 @@ SCENARIO("mergers round-robin over their inputs") {
}
}
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(),
src.as_observable());
ctx->run();
......@@ -158,7 +158,7 @@ SCENARIO("mergers round-robin over their inputs") {
}
GIVEN("a merger with one input that aborts after some items") {
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(),
src.as_observable());
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