Commit 04e02412 authored by Dominik Charousset's avatar Dominik Charousset

Make most operators *cold* by default

parent a52cde49
......@@ -129,6 +129,7 @@ caf_add_component(
src/detail/print.cpp
src/detail/private_thread.cpp
src/detail/private_thread_pool.cpp
src/detail/ref_counted_base.cpp
src/detail/ripemd_160.cpp
src/detail/serialized_size.cpp
src/detail/set_thread_name.cpp
......@@ -147,8 +148,10 @@ caf_add_component(
src/error.cpp
src/event_based_actor.cpp
src/execution_unit.cpp
src/flow/coordinated.cpp
src/flow/coordinator.cpp
src/flow/observable_builder.cpp
src/flow/op/interval.cpp
src/flow/scoped_coordinator.cpp
src/flow/subscription.cpp
src/forwarding_actor_proxy.cpp
......@@ -285,17 +288,18 @@ caf_add_component(
dynamic_spawn
error
expected
flow.broadcaster
flow.concat
flow.concat_map
flow.defer
flow.empty
flow.error
flow.fail
flow.flat_map
flow.for_each
flow.generation
flow.interval
flow.item_publisher
flow.merge
flow.mixed
flow.never
flow.observe_on
flow.prefix_and_tail
......
// 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 <atomic>
#include <cstddef>
#include "caf/detail/core_export.hpp"
namespace caf::detail {
// Base class for reference counted objects with an atomic reference count.
class CAF_CORE_EXPORT ref_counted_base {
public:
virtual ~ref_counted_base();
ref_counted_base();
ref_counted_base(const ref_counted_base&);
ref_counted_base& operator=(const ref_counted_base&);
/// Increases reference count by one.
void ref() const noexcept {
rc_.fetch_add(1, std::memory_order_relaxed);
}
/// Decreases reference count by one and calls `request_deletion`
/// when it drops to zero.
void deref() const noexcept;
/// Queries whether there is exactly one reference.
bool unique() const noexcept {
return rc_ == 1;
}
size_t get_reference_count() const noexcept {
return rc_.load();
}
protected:
mutable std::atomic<size_t> rc_;
};
} // namespace caf::detail
......@@ -222,6 +222,9 @@ public:
template <class T>
constexpr bool is_iterable<T>::value;
template <class T>
constexpr bool is_iterable_v = is_iterable<T>::value;
/// Checks whether `T` is a non-const reference.
template <class T>
struct is_mutable_ref : std::false_type { };
......
......@@ -35,13 +35,13 @@ public:
virtual void deref_disposable() const noexcept = 0;
};
// -- constructors, destructors, and assignment operators --------------------
explicit disposable(intrusive_ptr<impl> pimpl) noexcept
: pimpl_(std::move(pimpl)) {
// nop
}
// -- constructors, destructors, and assignment operators --------------------
disposable() noexcept = default;
disposable(disposable&&) noexcept = default;
......@@ -57,6 +57,7 @@ public:
return *this;
}
// -- factories --------------------------------------------------------------
/// Combines multiple disposables into a single disposable. The new disposable
......
......@@ -7,21 +7,4 @@
#include "caf/flow/observable.hpp"
#include "caf/flow/observer.hpp"
namespace caf::flow {
/// Combines the items emitted from `pub` and `pubs...` to appear as a single
/// stream of items.
template <class Observable, class... Observables>
observable<typename Observable::output_type>
concat(Observable x, Observables... xs) {
using output_type = output_type_t<Observable>;
static_assert(
(std::is_same_v<output_type, output_type_t<Observables>> && ...));
auto hdl = std::move(x).as_observable();
auto ptr = make_counted<concat_impl<output_type>>(hdl.ptr()->ctx());
ptr->add(std::move(hdl));
(ptr->add(std::move(xs).as_observable()), ...);
return observable<output_type>{std::move(ptr)};
}
} // namespace caf::flow
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/detail/core_export.hpp"
namespace caf::flow {
/// An object that lives on a @ref coordinator.
class CAF_CORE_EXPORT coordinated {
public:
// -- constructors, destructors, and assignment operators --------------------
virtual ~coordinated();
// -- reference counting -----------------------------------------------------
/// Increases the reference count of the coordinated.
virtual void ref_coordinated() const noexcept = 0;
/// Decreases the reference count of the coordinated and destroys the object
/// if necessary.
virtual void deref_coordinated() const noexcept = 0;
friend void intrusive_ptr_add_ref(const coordinated* ptr) noexcept {
ptr->ref_coordinated();
}
friend void intrusive_ptr_release(const coordinated* ptr) noexcept {
ptr->deref_coordinated();
}
};
} // namespace caf::flow
......@@ -9,7 +9,6 @@
#include "caf/flow/fwd.hpp"
#include "caf/flow/subscription.hpp"
#include "caf/intrusive_ptr.hpp"
#include "caf/ref_counted.hpp"
#include "caf/timespan.hpp"
#include <chrono>
......@@ -22,10 +21,6 @@ namespace caf::flow {
/// objects since the coordinator guarantees synchronous execution.
class CAF_CORE_EXPORT coordinator {
public:
// -- friends ----------------------------------------------------------------
friend class subscription_impl;
// -- member types -----------------------------------------------------------
/// A time point of the monotonic clock.
......
......@@ -121,9 +121,6 @@ struct output_type_oracle {
template <class T>
using output_type_t = typename output_type_oracle<T>::type;
template <class T>
class merger_impl;
template <class>
struct has_impl_include {
static constexpr bool value = false;
......
// 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/coordinator.hpp"
#include "caf/flow/observable.hpp"
#include "caf/flow/op/mcast.hpp"
#include "caf/intrusive_ptr.hpp"
#include "caf/make_counted.hpp"
#include <cstdint>
namespace caf::flow {
template <class T>
class item_publisher {
public:
explicit item_publisher(coordinator* ctx) {
pimpl_ = make_counted<op::mcast<T>>(ctx);
}
item_publisher(item_publisher&&) = default;
item_publisher& operator=(item_publisher&&) = default;
~item_publisher() {
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:
intrusive_ptr<op::mcast<T>> pimpl_;
};
} // namespace caf::flow
......@@ -11,17 +11,5 @@ namespace caf::flow {
/// Combines the items emitted from `pub` and `pubs...` to appear as a single
/// stream of items.
template <class Observable, class... Observables>
observable<typename Observable::output_type>
merge(Observable x, Observables... xs) {
using output_type = output_type_t<Observable>;
static_assert(
(std::is_same_v<output_type, output_type_t<Observables>> && ...));
auto hdl = std::move(x).as_observable();
auto ptr = make_counted<merger_impl<output_type>>(hdl.ptr()->ctx());
ptr->add(std::move(hdl));
(ptr->add(std::move(xs).as_observable()), ...);
return observable<output_type>{std::move(ptr)};
}
} // namespace caf::flow
This diff is collapsed.
This diff is collapsed.
// 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/disposable.hpp"
#include "caf/flow/fwd.hpp"
#include "caf/flow/op/base.hpp"
#include "caf/flow/step.hpp"
#include "caf/fwd.hpp"
#include "caf/intrusive_ptr.hpp"
#include <cstddef>
#include <functional>
#include <utility>
#include <vector>
namespace caf::flow {
/// Represents a potentially unbound sequence of values.
template <class T>
class observable {
public:
/// The type of emitted items.
using output_type = T;
/// The pointer-to-implementation type.
using pimpl_type = intrusive_ptr<op::base<T>>;
explicit observable(pimpl_type pimpl) noexcept : pimpl_(std::move(pimpl)) {
// nop
}
observable& operator=(std::nullptr_t) noexcept {
pimpl_.reset();
return *this;
}
observable() noexcept = default;
observable(observable&&) noexcept = default;
observable(const observable&) noexcept = default;
observable& operator=(observable&&) noexcept = default;
observable& operator=(const observable&) noexcept = default;
/// @copydoc impl::subscribe
disposable subscribe(observer<T> what) {
if (pimpl_) {
return pimpl_->subscribe(std::move(what));
} else {
what.on_error(make_error(sec::invalid_observable));
return disposable{};
}
}
/// Creates a new observer that pushes all observed items to the resource.
disposable subscribe(async::producer_resource<T> resource);
/// Returns a transformation that applies a step function to each input.
template <class Step>
transformation<Step> transform(Step step);
/// Registers a callback for `on_next` events.
template <class F>
auto do_on_next(F f) {
return transform(do_on_next_step<T, F>{std::move(f)});
}
/// Registers a callback for `on_complete` events.
template <class F>
auto do_on_complete(F f) {
return transform(do_on_complete_step<T, F>{std::move(f)});
}
/// Registers a callback for `on_error` events.
template <class F>
auto do_on_error(F f) {
return transform(do_on_error_step<T, F>{std::move(f)});
}
/// Registers a callback that runs on `on_complete` or `on_error`.
template <class F>
auto do_finally(F f) {
return transform(do_finally_step<T, F>{std::move(f)});
}
/// Catches errors by converting them into errors instead.
auto on_error_complete() {
return transform(on_error_complete_step<T>{});
}
/// Returns a transformation that selects only the first `n` items.
transformation<limit_step<T>> take(size_t n);
/// Returns a transformation that selects only items that satisfy `predicate`.
template <class Predicate>
transformation<filter_step<Predicate>> filter(Predicate prediate);
/// Returns a transformation that selects all value until the `predicate`
/// returns false.
template <class Predicate>
transformation<take_while_step<Predicate>> take_while(Predicate prediate);
/// Reduces the entire sequence of items to a single value. Other names for
/// the algorithm are `accumulate` and `fold`.
template <class Reducer>
transformation<reduce_step<T, Reducer>> reduce(T init, Reducer reducer);
/// Accumulates all values and emits only the final result.
auto sum() {
return reduce(T{}, std::plus<T>{});
}
/// Makes all values unique by suppressing all items that have been emitted in
/// the past.
transformation<distinct_step<T>> distinct();
/// Returns a transformation that applies `f` to each input and emits the
/// result of the function application.
template <class F>
transformation<map_step<F>> map(F f);
/// Calls `on_next` for each item emitted by this observable.
template <class OnNext>
disposable for_each(OnNext on_next);
/// Calls `on_next` for each item and `on_error` for each error emitted by
/// this observable.
template <class OnNext, class OnError>
disposable for_each(OnNext on_next, OnError on_error);
/// Calls `on_next` for each item, `on_error` for each error and `on_complete`
/// for each completion signal emitted by this observable.
template <class OnNext, class OnError, class OnComplete>
disposable for_each(OnNext on_next, OnError on_error, OnComplete on_complete);
/// Combines the output of multiple @ref observable objects into one by
/// merging their outputs. May also be called without arguments if the `T` is
/// an @ref observable.
template <class... Inputs>
auto merge(Inputs&&... xs);
/// Combines the output of multiple @ref observable objects into one by
/// concatenating their outputs. May also be called without arguments if the
/// `T` is an @ref observable.
template <class... Inputs>
auto concat(Inputs&&...);
/// Returns a transformation that emits items by merging the outputs of all
/// observables returned by `f`.
template <class F>
auto flat_map(F f);
/// Returns a transformation that emits items from optional values returned by
/// `f`.
template <class F>
transformation<flat_map_optional_step<F>> flat_map_optional(F f);
/// Returns a transformation that emits items by concatenating the outputs of
/// all observables returned by `f`.
template <class F>
auto concat_map(F f);
/// Takes @p prefix_size elements from this observable and emits it in a tuple
/// containing an observable for the remaining elements as the second value.
/// The returned observable either emits a single element (the tuple) or none
/// if this observable never produces sufficient elements for the prefix.
/// @pre `prefix_size > 0`
observable<cow_tuple<std::vector<T>, observable<T>>>
prefix_and_tail(size_t prefix_size);
/// Similar to `prefix_and_tail(1)` but passes the single element directly in
/// the tuple instead of wrapping it in a list.
observable<cow_tuple<T, observable<T>>> head_and_tail();
/// Creates an asynchronous resource that makes emitted items available in a
/// spsc buffer.
async::consumer_resource<T> to_resource(size_t buffer_size,
size_t min_request_size);
async::consumer_resource<T> to_resource() {
return to_resource(defaults::flow::buffer_size, defaults::flow::min_demand);
}
observable observe_on(coordinator* other, size_t buffer_size,
size_t min_request_size);
observable observe_on(coordinator* other) {
return observe_on(other, defaults::flow::buffer_size,
defaults::flow::min_demand);
}
const observable& as_observable() const& noexcept {
return std::move(*this);
}
observable&& as_observable() && noexcept {
return std::move(*this);
}
const pimpl_type& pimpl() const noexcept {
return pimpl_;
}
bool valid() const noexcept {
return pimpl_ != nullptr;
}
explicit operator bool() const noexcept {
return valid();
}
bool operator!() const noexcept {
return !valid();
}
void swap(observable& other) {
pimpl_.swap(other.pimpl_);
}
/// @pre `valid()`
coordinator* ctx() const {
return pimpl_->ctx();
}
private:
pimpl_type pimpl_;
};
/// Convenience function for creating an @ref observable from a concrete
/// operator type.
/// @relates observable
template <class Operator, class... Ts>
observable<typename Operator::output_type>
make_observable(coordinator* ctx, Ts&&... xs) {
using out_t = typename Operator::output_type;
using ptr_t = intrusive_ptr<op::base<out_t>>;
ptr_t ptr{new Operator(ctx, std::forward<Ts>(xs)...), false};
return observable<out_t>{std::move(ptr)};
}
// Note: the definition of all member functions is in observable.hpp.
} // namespace caf::flow
This diff is collapsed.
The namespace `caf::flow::op` contains (private) implementation classes for
various operators.
// 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/detail/ref_counted_base.hpp"
#include "caf/flow/observer.hpp"
#include "caf/flow/subscription.hpp"
namespace caf::flow::op {
/// Abstract base type for all flow operators.
template <class T>
class base : public coordinated {
public:
/// The type of observed values.
using output_type = T;
/// Returns the @ref coordinator that executes this flow operator.
virtual coordinator* ctx() const noexcept = 0;
/// Subscribes a new observer to the operator.
virtual disposable subscribe(observer<T> what) = 0;
};
} // namespace caf::flow::op
// 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/observer.hpp"
#include "caf/flow/op/hot.hpp"
#include "caf/flow/subscription.hpp"
#include "caf/none.hpp"
#include <memory>
#include <utility>
#include <variant>
#include <vector>
namespace caf::flow::op {
/// State shared between one multicast operator and one subscribed observer.
template <class T>
struct cell_sub_state {
std::variant<none_t, unit_t, T, error> content;
std::vector<observer<T>> listeners;
void set_null() {
CAF_ASSERT(std::holds_alternative<none_t>(content));
content = unit;
std::vector<observer<T>> xs;
xs.swap(listeners);
for (auto& listener : xs) {
listener.on_complete();
}
}
void set_value(T item) {
CAF_ASSERT(std::holds_alternative<none_t>(content));
content = std::move(item);
auto& ref = std::get<T>(content);
std::vector<observer<T>> xs;
xs.swap(listeners);
for (auto& listener : xs) {
listener.on_next(ref);
listener.on_complete();
}
}
void set_error(error what) {
CAF_ASSERT(std::holds_alternative<none_t>(content));
content = std::move(what);
auto& ref = std::get<error>(content);
std::vector<observer<T>> xs;
xs.swap(listeners);
for (auto& listener : xs)
listener.on_error(ref);
}
void listen(observer<T> listener) {
switch (content.index()) {
case 1:
listener.on_complete();
break;
case 2:
listener.on_next(std::get<T>(content));
listener.on_complete();
break;
case 3:
listener.on_error(std::get<error>(content));
break;
default:
listeners.emplace_back(std::move(listener));
}
}
void drop(const observer<T>& listener) {
if (auto i = std::find(listeners.begin(), listeners.end(), listener);
i != listeners.end())
listeners.erase(i);
}
};
template <class T>
using cell_sub_state_ptr = std::shared_ptr<cell_sub_state<T>>;
template <class T>
class cell_sub : public subscription::impl_base {
public:
// -- constructors, destructors, and assignment operators --------------------
cell_sub(coordinator* ctx, cell_sub_state_ptr<T> state, observer<T> out)
: ctx_(ctx), state_(std::move(state)), out_(std::move(out)) {
// nop
}
// -- implementation of subscription -----------------------------------------
bool disposed() const noexcept override {
return !state_;
}
void cancel() override {
if (state_) {
state_->drop(out_);
state_ = nullptr;
out_ = nullptr;
}
}
void request(size_t) override {
if (!listening_) {
listening_ = true;
ctx_->delay_fn([state = state_, out = out_] { //
state->listen(std::move(out));
});
}
}
private:
coordinator* ctx_;
bool listening_ = false;
cell_sub_state_ptr<T> state_;
observer<T> out_;
};
// Base type for *hot* operators that multicast data to subscribed observers.
template <class T>
class cell : public hot<T> {
public:
// -- member types -----------------------------------------------------------
using super = hot<T>;
using state_type = cell_sub_state<T>;
using state_ptr_type = cell_sub_state_ptr<T>;
using observer_type = observer<T>;
// -- constructors, destructors, and assignment operators --------------------
explicit cell(coordinator* ctx)
: super(ctx), state_(std::make_shared<state_type>()) {
// nop
}
void set_null() {
state_->set_null();
}
void set_value(T item) {
state_->set_value(std::move(item));
}
void set_error(error what) {
state_->set_error(what);
}
disposable subscribe(observer<T> out) override {
auto ptr = make_counted<cell_sub<T>>(super::ctx_, state_, out);
out.on_subscribe(subscription{ptr});
return disposable{std::move(ptr)};
}
protected:
cell_sub_state_ptr<T> state_;
};
} // namespace caf::flow::op
// 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/detail/ref_counted_base.hpp"
#include "caf/flow/observer.hpp"
#include "caf/flow/op/base.hpp"
#include "caf/flow/subscription.hpp"
namespace caf::flow::op {
/// Convenience base type for *cold* observable types.
template <class T>
class cold : public detail::ref_counted_base, public base<T> {
public:
// -- member types -----------------------------------------------------------
using output_type = T;
// -- constructors, destructors, and assignment operators --------------------
explicit cold(coordinator* ctx) : ctx_(ctx) {
// nop
}
// -- implementation of disposable_impl --------------------------------------
void ref_coordinated() const noexcept final {
this->ref();
}
void deref_coordinated() const noexcept final {
this->deref();
}
// -- implementation of observable_impl<T> -----------------------------------
coordinator* ctx() const noexcept final {
return ctx_;
}
protected:
// -- member variables -------------------------------------------------------
coordinator* ctx_;
};
} // namespace caf::flow::op
// 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/detail/ref_counted_base.hpp"
#include "caf/flow/observer.hpp"
#include "caf/flow/op/cold.hpp"
#include "caf/flow/op/empty.hpp"
#include "caf/flow/subscription.hpp"
#include <deque>
#include <memory>
#include <numeric>
#include <utility>
#include <variant>
#include <vector>
namespace caf::flow::op {
/// Combines items from any number of observables.
template <class T>
class concat_sub : public subscription::impl_base {
public:
// -- member types -----------------------------------------------------------
using input_key = size_t;
using input_type = std::variant<observable<T>, observable<observable<T>>>;
// -- constructors, destructors, and assignment operators --------------------
concat_sub(coordinator* ctx, observer<T> out, std::vector<input_type> inputs)
: ctx_(ctx), out_(out), inputs_(std::move(inputs)) {
CAF_ASSERT(!inputs_.empty());
subscribe_next();
}
// -- input management -------------------------------------------------------
void subscribe_to(observable<T> what) {
CAF_ASSERT(!active_sub_);
active_key_ = next_key_++;
using fwd_t = forwarder<T, concat_sub, size_t>;
auto fwd = make_counted<fwd_t>(this, active_key_);
what.subscribe(fwd->as_observer());
}
void subscribe_to(observable<observable<T>> what) {
CAF_ASSERT(!active_sub_);
CAF_ASSERT(!factory_sub_);
factory_key_ = next_key_++;
using fwd_t = forwarder<observable<T>, concat_sub, size_t>;
auto fwd = make_counted<fwd_t>(this, factory_key_);
what.subscribe(fwd->as_observer());
}
void subscribe_next() {
if (factory_key_ != 0) {
// Ask the factory for the next observable.
CAF_ASSERT(!active_sub_);
factory_sub_.request(1);
} else if (!inputs_.empty()) {
// Subscribe to the next observable from the list.
std::visit([this](auto& x) { this->subscribe_to(x); }, inputs_.front());
inputs_.erase(inputs_.begin());
} else {
// Done!
fin();
}
}
// -- callbacks for the forwarders -------------------------------------------
void fwd_on_subscribe(input_key key, subscription sub) {
if (active_key_ == key && !active_sub_) {
active_sub_ = std::move(sub);
if (in_flight_ > 0)
active_sub_.request(in_flight_);
} else if (factory_key_ == key && !factory_sub_) {
CAF_ASSERT(!active_sub_);
factory_sub_ = std::move(sub);
factory_sub_.request(1);
} else {
sub.cancel();
}
}
void fwd_on_complete(input_key key) {
if (active_key_ == key && active_sub_) {
active_sub_ = nullptr;
subscribe_next();
} else if (factory_key_ == key && factory_sub_) {
factory_sub_ = nullptr;
factory_key_ = 0;
if (!active_sub_)
subscribe_next();
}
}
void fwd_on_error(input_key key, const error& what) {
if (key == active_key_ || key == factory_key_) {
CAF_ASSERT(out_);
if (delay_error_) {
if (!err_)
err_ = what;
subscribe_next();
} else {
err_ = what;
fin();
}
}
}
void fwd_on_next(input_key key, const T& item) {
if (key == active_key_) {
CAF_ASSERT(out_);
--in_flight_;
out_.on_next(item);
}
}
void fwd_on_next(input_key key, const observable<T>& item) {
if (key == factory_key_) {
CAF_ASSERT(!active_sub_);
subscribe_to(item);
}
}
// -- implementation of subscription -----------------------------------------
bool disposed() const noexcept override {
return !out_;
}
void cancel() override {
if (out_) {
ctx_->delay_fn([strong_this = intrusive_ptr<concat_sub>{this}] {
if (strong_this->out_) {
strong_this->err_.reset();
strong_this->fin();
}
});
}
}
void request(size_t n) override {
CAF_ASSERT(out_.valid());
if (active_sub_)
active_sub_.request(n);
in_flight_ += n;
}
private:
void fin() {
CAF_ASSERT(out_);
if (factory_sub_) {
factory_sub_.cancel();
factory_sub_ = nullptr;
}
if (active_sub_) {
active_sub_.cancel();
active_sub_ = nullptr;
}
factory_key_ = 0;
active_key_ = 0;
if (err_)
out_.on_error(err_);
else
out_.on_complete();
out_ = nullptr;
}
/// Stores the context (coordinator) that runs this flow.
coordinator* ctx_;
/// Stores a handle to the subscribed observer.
observer<T> out_;
/// Configures whether we carry on after an error.
bool delay_error_ = false;
/// Caches an on_error reason if delay_error_ is true.
error err_;
/// Stores our input sources. The first input is active (subscribed to) while
/// the others are pending (not subscribed to).
std::vector<input_type> inputs_;
/// If set, identifies the subscription to an observable factory.
subscription factory_sub_;
/// Our currently active subscription.
subscription active_sub_;
/// Identifies the active forwarder.
input_key factory_key_ = 0;
/// Identifies the active forwarder.
input_key active_key_ = 0;
/// Stores the next available key.
input_key next_key_ = 1;
/// Stores how much demand we have left. When switching to a new input, we
/// pass any demand unused by the previous input to the new one.
size_t in_flight_ = 0;
};
template <class T>
class concat : public cold<T> {
public:
// -- member types -----------------------------------------------------------
using super = cold<T>;
using input_type = std::variant<observable<T>, observable<observable<T>>>;
// -- constructors, destructors, and assignment operators --------------------
template <class... Ts, class... Inputs>
explicit concat(coordinator* ctx, Inputs&&... inputs) : super(ctx) {
(add(std::forward<Inputs>(inputs)), ...);
}
// -- properties -------------------------------------------------------------
size_t inputs() const noexcept {
return inputs_.size();
}
// -- implementation of observable<T> -----------------------------------
disposable subscribe(observer<T> out) override {
if (inputs() == 0) {
return make_counted<empty<T>>(super::ctx_)->subscribe(std::move(out));
} else {
auto ptr = make_counted<concat_sub<T>>(super::ctx_, out, inputs_);
out.on_subscribe(subscription{ptr});
return ptr->as_disposable();
}
}
private:
template <class Input>
void add(Input&& x) {
using input_t = std::decay_t<Input>;
if constexpr (detail::is_iterable_v<input_t>) {
for (auto& in : x)
add(in);
} else {
static_assert(is_observable_v<input_t>);
inputs_.emplace_back(std::move(x).as_observable());
}
}
std::vector<input_type> inputs_;
};
} // namespace caf::flow::op
// 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/observer.hpp"
#include "caf/flow/op/cold.hpp"
#include "caf/flow/subscription.hpp"
#include <utility>
namespace caf::flow::op {
template <class Fn>
struct defer_trait {
using fn_result_type = decltype(std::declval<Fn&>()());
static_assert(is_observable_v<fn_result_type>);
using output_type = typename fn_result_type::output_type;
};
/// Implementation of the `defer` operator.
template <class Factory>
class defer : public cold<typename defer_trait<Factory>::output_type> {
public:
// -- member types -----------------------------------------------------------
using output_type = typename defer_trait<Factory>::output_type;
using super = cold<output_type>;
// -- constructors, destructors, and assignment operators --------------------
defer(coordinator* ctx, Factory fn) : super(ctx), fn_(std::move(fn)) {
// nop
}
// -- implementation of observable<T>::impl ----------------------------------
disposable subscribe(observer<output_type> what) override {
return fn_().subscribe(what);
}
private:
Factory fn_;
};
} // namespace caf::flow::op
// 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/observable.hpp"
#include "caf/flow/observer.hpp"
#include "caf/flow/op/cold.hpp"
#include "caf/flow/subscription.hpp"
#include <cstdint>
namespace caf::flow::op {
template <class T>
class empty_sub : public subscription::impl_base {
public:
// -- constructors, destructors, and assignment operators --------------------
empty_sub(coordinator* ctx, observer<T> out)
: ctx_(ctx), out_(std::move(out)) {
// nop
}
// -- implementation of subscription -----------------------------------------
bool disposed() const noexcept override {
return !out_;
}
void cancel() override {
if (out_)
ctx_->delay_fn([out = std::move(out_)]() mutable { out.on_complete(); });
}
void request(size_t) override {
cancel();
}
private:
/// Stores the context (coordinator) that runs this flow.
coordinator* ctx_;
/// Stores a handle to the subscribed observer.
observer<T> out_;
};
/// An observable that represents an empty range. As soon as an observer
/// requests values from this observable, it calls `on_complete`.
template <class T>
class empty : public cold<T> {
public:
// -- member types -----------------------------------------------------------
using super = cold<T>;
using output_type = T;
// -- constructors, destructors, and assignment operators --------------------
explicit empty(coordinator* ctx) : super(ctx) {
// nop
}
// -- implementation of observable<T>::impl ----------------------------------
disposable subscribe(observer<output_type> out) override {
auto ptr = make_counted<empty_sub<T>>(super::ctx_, out);
out.on_subscribe(subscription{ptr});
return disposable{std::move(ptr)};
}
};
} // namespace caf::flow::op
// 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/error.hpp"
#include "caf/flow/observable.hpp"
#include "caf/flow/observer.hpp"
#include "caf/flow/op/cold.hpp"
#include "caf/flow/subscription.hpp"
namespace caf::flow::op {
template <class T>
class fail : public cold<T> {
public:
// -- member types -----------------------------------------------------------
using super = cold<T>;
// -- constructors, destructors, and assignment operators --------------------
fail(coordinator* ctx, error err) : super(ctx), err_(std::move(err)) {
// nop
}
// -- implementation of observable_impl<T> -----------------------------------
disposable subscribe(observer<T> out) override {
out.on_error(err_);
return {};
}
private:
error err_;
};
} // namespace caf::flow::op
// 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/detail/ref_counted_base.hpp"
#include "caf/detail/type_list.hpp"
#include "caf/flow/observer.hpp"
#include "caf/flow/op/hot.hpp"
#include "caf/flow/subscription.hpp"
#include "caf/sec.hpp"
#include <tuple>
#include <utility>
namespace caf::flow::op {
template <class Generator, class... Steps>
class from_generator_sub : public subscription::impl_base {
public:
// -- member types -----------------------------------------------------------
using output_type = steps_output_type_t<Generator, Steps...>;
// -- constructors, destructors, and assignment operators --------------------
from_generator_sub(coordinator* ctx, observer<output_type> out,
const Generator& gen, const std::tuple<Steps...>& steps)
: ctx_(ctx), out_(std::move(out)), gen_(gen), steps_(steps) {
// nop
}
// -- callbacks for the generator / steps ------------------------------------
bool on_next(const output_type& item) {
buf_.push_back(item);
return true;
}
void on_complete() {
completed_ = true;
}
void on_error(const error& what) {
completed_ = true;
err_ = what;
}
// -- implementation of subscription -----------------------------------------
bool disposed() const noexcept override {
return !out_;
}
void cancel() override {
if (out_) {
completed_ = true;
buf_.clear();
run_later();
}
}
void request(size_t n) override {
CAF_ASSERT(out_.valid());
demand_ += n;
run_later();
}
private:
void run_later() {
if (!running_) {
running_ = true;
ctx_->delay_fn([strong_this = intrusive_ptr<from_generator_sub>{this}] {
strong_this->do_run();
});
}
}
void do_run() {
while (out_ && demand_ > 0) {
while (buf_.empty() && !completed_)
pull(demand_);
if (!buf_.empty()) {
--demand_;
auto tmp = std::move(buf_.front());
buf_.pop_front();
out_.on_next(tmp);
} else if (completed_) {
fin();
running_ = false;
return;
}
}
if (out_ && buf_.empty() && completed_)
fin();
running_ = false;
}
void fin() {
if (!err_)
out_.on_complete();
else
out_.on_error(err_);
out_ = nullptr;
}
void pull(size_t n) {
auto fn = [this, n](auto&... steps) { gen_.pull(n, steps..., *this); };
std::apply(fn, steps_);
}
coordinator* ctx_;
bool running_ = false;
std::deque<output_type> buf_;
bool completed_ = false;
error err_;
size_t demand_ = 0;
observer<output_type> out_;
Generator gen_;
std::tuple<Steps...> steps_;
};
template <class Generator, class... Steps>
using from_generator_output_t = //
typename detail::tl_back_t< //
detail::type_list<Generator, Steps...> //
>::output_type;
template <class Generator, class... Steps>
class from_generator
: public op::cold<from_generator_output_t<Generator, Steps...>> {
public:
using output_type = from_generator_output_t<Generator, Steps...>;
using super = op::cold<output_type>;
from_generator(coordinator* ctx, Generator gen, std::tuple<Steps...> steps)
: super(ctx), gen_(std::move(gen)), steps_(std::move(steps)) {
// nop
}
// -- implementation of observable_impl<T> ---------------------------------
disposable subscribe(observer<output_type> out) override {
CAF_LOG_TRACE(CAF_ARG2("out", out.ptr()));
using impl_t = from_generator_sub<Generator, Steps...>;
auto ptr = make_counted<impl_t>(super::ctx_, out, gen_, steps_);
auto sub = subscription{std::move(ptr)};
out.on_subscribe(sub);
return std::move(sub).as_disposable();
}
private:
Generator gen_;
std::tuple<Steps...> steps_;
};
} // namespace caf::flow::op
This diff is collapsed.
This diff is collapsed.
// 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/detail/ref_counted_base.hpp"
#include "caf/flow/observer.hpp"
#include "caf/flow/op/base.hpp"
#include "caf/flow/subscription.hpp"
namespace caf::flow::op {
/// Convenience base type for *hot* observable types.
template <class T>
class hot : public detail::ref_counted_base, public base<T> {
public:
// -- member types -----------------------------------------------------------
using output_type = T;
// -- constructors, destructors, and assignment operators --------------------
explicit hot(coordinator* ctx) : ctx_(ctx) {
// nop
}
// -- implementation of disposable_impl --------------------------------------
void ref_coordinated() const noexcept final {
this->ref();
}
void deref_coordinated() const noexcept final {
this->deref();
}
// -- implementation of observable_impl<T> -----------------------------------
coordinator* ctx() const noexcept final {
return ctx_;
}
protected:
// -- member variables -------------------------------------------------------
coordinator* ctx_;
};
} // namespace caf::flow::op
// 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/detail/core_export.hpp"
#include "caf/flow/observer.hpp"
#include "caf/flow/op/cold.hpp"
#include "caf/timespan.hpp"
#include <cstdint>
namespace caf::flow::op {
class CAF_CORE_EXPORT interval : public cold<int64_t> {
public:
// -- member types -----------------------------------------------------------
using super = cold<int64_t>;
// -- constructors, destructors, and assignment operators --------------------
interval(coordinator* ctx, timespan initial_delay, timespan period);
interval(coordinator* ctx, timespan initial_delay, timespan period,
int64_t max_val);
// -- implementation of observable_impl<T> -----------------------------------
disposable subscribe(observer<int64_t> out) override;
private:
timespan initial_delay_;
timespan period_;
int64_t max_;
};
} // namespace caf::flow::op
This diff is collapsed.
This diff is collapsed.
// 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/observer.hpp"
#include "caf/flow/op/cold.hpp"
#include "caf/flow/subscription.hpp"
#include <utility>
namespace caf::flow::op {
template <class T>
class never_sub : public subscription::impl_base {
public:
// -- constructors, destructors, and assignment operators --------------------
never_sub(coordinator* ctx, observer<T> out)
: ctx_(ctx), out_(std::move(out)) {
// nop
}
// -- implementation of subscription -----------------------------------------
bool disposed() const noexcept override {
return !out_;
}
void cancel() override {
if (out_)
ctx_->delay_fn([out = std::move(out_)]() mutable { out.on_complete(); });
}
void request(size_t) override {
// nop
}
private:
/// Stores the context (coordinator) that runs this flow.
coordinator* ctx_;
/// Stores a handle to the subscribed observer.
observer<T> out_;
};
/// An observable that never calls any callbacks on its subscribers.
template <class T>
class never : public cold<T> {
public:
// -- member types -----------------------------------------------------------
using super = cold<T>;
using output_type = T;
// -- constructors, destructors, and assignment operators --------------------
explicit never(coordinator* ctx) : super(ctx) {
// nop
}
// -- implementation of observable_impl<T> -----------------------------------
disposable subscribe(observer<output_type> out) override {
auto ptr = make_counted<never_sub<T>>(super::ctx_, out);
out.ptr()->on_subscribe(subscription{ptr});
return ptr->as_disposable();
}
};
} // namespace caf::flow::op
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
......@@ -10,6 +10,7 @@
#include "caf/flow/observable.hpp"
#include "caf/flow/observable_builder.hpp"
#include "caf/flow/observer.hpp"
#include "caf/flow/op/cell.hpp"
#include "caf/flow/single.hpp"
#include "caf/scheduled_actor.hpp"
......@@ -26,20 +27,18 @@ namespace caf {
template <class T, class Policy>
flow::single<T> scheduled_actor::single_from_response_impl(Policy& policy) {
using output_type = T;
using impl_type = typename flow::single<output_type>::impl;
auto ptr = make_counted<impl_type>(this);
auto cell = make_counted<flow::op::cell<T>>(this);
policy.then(
this,
[this, ptr](T& val) {
ptr->set_value(std::move(val));
[this, cell](T& val) {
cell->set_value(std::move(val));
run_actions();
},
[this, ptr](error& err) {
ptr->set_error(std::move(err));
[this, cell](error& err) {
cell->set_error(std::move(err));
run_actions();
});
return flow::single<output_type>{std::move(ptr)};
return flow::single<T>{std::move(cell)};
}
} // namespace caf
This diff is collapsed.
// This file is part of CAF, the C++ Actor Framework. See the file LICENSE in
// the main distribution directory for license terms and copyright or visit
// https://github.com/actor-framework/actor-framework/blob/master/LICENSE.
#include "caf/flow/coordinated.hpp"
namespace caf::flow {
coordinated::~coordinated() {
// nop
}
} // namespace caf::flow
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
This diff is collapsed.
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