Commit afa01fdb authored by Dominik Charousset's avatar Dominik Charousset

Add blocking producer and consumer interfaces

parent d7b8f7ee
......@@ -53,6 +53,8 @@ caf_add_component(
PRIVATE
CAF::internal
ENUM_TYPES
async.read_result
async.write_result
exit_reason
intrusive.inbox_result
intrusive.task_result
......
// This file is part of CAF, the C++ Actor Framework. See the file LICENSE in
// the main distribution directory for license terms and copyright or visit
// https://github.com/actor-framework/actor-framework/blob/master/LICENSE.
#pragma once
#include "caf/async/consumer.hpp"
#include "caf/async/read_result.hpp"
#include "caf/async/spsc_buffer.hpp"
#include "caf/intrusive_ptr.hpp"
#include "caf/make_counted.hpp"
#include "caf/ref_counted.hpp"
#include <chrono>
#include <condition_variable>
#include <type_traits>
namespace caf::async {
/// Blocking interface for emitting items to an asynchronous consumer.
template <class T>
class blocking_consumer {
public:
class impl : public ref_counted, public consumer {
public:
impl() = delete;
impl(const impl&) = delete;
impl& operator=(const impl&) = delete;
explicit impl(spsc_buffer_ptr<T> buf) : buf_(std::move(buf)) {
buf_->set_consumer(this);
}
template <class ErrorPolicy, class TimePoint>
read_result pull(ErrorPolicy policy, T& item, TimePoint timeout) {
val_ = &item;
std::unique_lock guard{buf_->mtx()};
if constexpr (std::is_same_v<TimePoint, none_t>) {
buf_->await_consumer_ready(guard, cv_);
} else {
if (!buf_->await_consumer_ready(guard, cv_, timeout))
return read_result::timeout;
}
auto [again, n] = buf_->pull_unsafe(guard, policy, 1u, *this);
CAF_IGNORE_UNUSED(again);
CAF_ASSERT(!again || n == 1);
if (n == 1) {
return read_result::ok;
} else if (buf_->abort_reason_unsafe()) {
return read_result::abort;
} else {
return read_result::stop;
}
}
template <class ErrorPolicy>
read_result pull(ErrorPolicy policy, T& item) {
return pull(policy, item, none);
}
void on_next(span<const T>items) {
CAF_ASSERT(items.size() == 1);
*val_ = items[0];
}
void on_complete() {
}
void on_error(const caf::error&) {
// nop
}
void cancel() {
if (buf_) {
buf_->cancel();
buf_ = nullptr;
}
}
void on_producer_ready() override {
// nop
}
void on_producer_wakeup() override {
// NOTE: buf_->mtx() is already locked at this point
cv_.notify_all();
}
void ref_consumer() const noexcept override {
ref();
}
void deref_consumer() const noexcept override {
deref();
}
error abort_reason() const {
buf_->abort_reason()();
}
CAF_INTRUSIVE_PTR_FRIENDS(impl)
private:
spsc_buffer_ptr<T> buf_;
std::condition_variable cv_;
T* val_ = nullptr;
};
using impl_ptr = intrusive_ptr<impl>;
blocking_consumer() = default;
blocking_consumer(const blocking_consumer&) = delete;
blocking_consumer& operator=(const blocking_consumer&) = delete;
blocking_consumer(blocking_consumer&&) = default;
blocking_consumer& operator=(blocking_consumer&&) = default;
explicit blocking_consumer(impl_ptr ptr) : impl_(std::move(ptr)) {
// nop
}
~blocking_consumer() {
if (impl_)
impl_->cancel();
}
/// Fetches the next item. If there is no item available, this functions
/// blocks unconditionally.
/// @param policy Either @ref instant_error, @ref delay_error or
/// @ref ignore_errors.
/// @param item Output parameter for storing the received item.
/// @returns the status of the read operation. The function writes to `item`
/// only when also returning `read_result::ok`.
template <class ErrorPolicy>
read_result pull(ErrorPolicy policy, T& item) {
return impl_->pull(policy, item);
}
/// Fetches the next item. If there is no item available, this functions
/// blocks until the absolute timeout was reached.
/// @param policy Either @ref instant_error, @ref delay_error or
/// @ref ignore_errors.
/// @param item Output parameter for storing the received item.
/// @param timeout Absolute timeout for the receive operation.
/// @returns the status of the read operation. The function writes to `item`
/// only when also returning `read_result::ok`.
template <class ErrorPolicy, class Clock,
class Duration = typename Clock::duration>
read_result pull(ErrorPolicy policy, T& item,
std::chrono::time_point<Clock, Duration> timeout) {
return impl_->pull(policy, item, timeout);
}
/// Fetches the next item. If there is no item available, this functions
/// blocks until the relative timeout was reached.
/// @param policy Either @ref instant_error, @ref delay_error or
/// @ref ignore_errors.
/// @param item Output parameter for storing the received item.
/// @param timeout Maximum duration before returning from the function.
/// @returns the status of the read operation. The function writes to `item`
/// only when also returning `read_result::ok`.
template <class ErrorPolicy, class Rep, class Period>
read_result pull(ErrorPolicy policy, T& item,
std::chrono::duration<Rep, Period> timeout) {
auto abs_timeout = std::chrono::system_clock::now() + timeout;
return impl_->pull(policy, item, abs_timeout);
}
error abort_reason() const {
impl_->abort_reason();
}
private:
intrusive_ptr<impl> impl_;
};
template <class T>
expected<blocking_consumer<T>>
make_blocking_consumer(consumer_resource<T> res) {
if (auto buf = res.try_open()) {
using impl_t = typename blocking_consumer<T>::impl;
return {blocking_consumer<T>{make_counted<impl_t>(std::move(buf))}};
} else {
return {make_error(sec::cannot_open_resource)};
}
}
} // namespace caf::async
// 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 <condition_variable>
#include <type_traits>
#include "caf/async/observer_buffer.hpp"
namespace caf::async {
/// Consumes all elements from a publisher and blocks the current thread until
/// completion.
template <class T>
class blocking_observer : public observer_buffer<T> {
public:
using super = observer_buffer<T>;
using super::super;
template <class OnNext, class OnError, class OnComplete>
auto run(OnNext fun, OnError err, OnComplete fin) {
static_assert(std::is_invocable_v<OnNext, T>,
"OnNext handlers must have signature 'void(T)' or 'bool(T)'");
static_assert(std::is_invocable_v<OnError, error>,
"OnError handlers must have signature 'void(const error&)'");
static_assert(std::is_invocable_v<OnComplete>,
"OnComplete handlers must have signature 'void()'");
auto wait_fn = [this](std::unique_lock<std::mutex>& guard) {
cv_.wait(guard);
};
for (;;) {
auto [val, done, what] = super::wait_with(wait_fn);
if (val) {
using fun_res = decltype(fun(*val));
if constexpr (std::is_same_v<bool, fun_res>) {
if (!fun(*val)) {
std::unique_lock guard{super::mtx_};
if (super::sub_)
super::sub_->cancel();
return;
}
} else {
static_assert(
std::is_same_v<void, fun_res>,
"OnNext handlers must have signature 'void(T)' or 'bool(T)'");
fun(*val);
}
} else if (done) {
if (!what)
fin();
else
err(*what);
return;
}
}
}
private:
std::condition_variable cv_;
void wakeup(std::unique_lock<std::mutex>&) override {
cv_.notify_all();
}
};
} // namespace caf::async
// 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 <condition_variable>
#include <type_traits>
#include "caf/async/producer.hpp"
#include "caf/async/spsc_buffer.hpp"
#include "caf/intrusive_ptr.hpp"
#include "caf/make_counted.hpp"
#include "caf/ref_counted.hpp"
namespace caf::async {
/// Blocking interface for emitting items to an asynchronous consumer.
template <class T>
class blocking_producer {
public:
class impl : public ref_counted, public producer {
public:
impl() = delete;
impl(const impl&) = delete;
impl& operator=(const impl&) = delete;
explicit impl(spsc_buffer_ptr<T> buf) : buf_(std::move(buf)) {
buf_->set_producer(this);
}
bool push(span<const T> items) {
std::unique_lock guard{mtx_};
while (items.size() > 0) {
while (demand_ == 0)
cv_.wait(guard);
if (demand_ < 0) {
return false;
} else {
auto n = std::min(static_cast<size_t>(demand_), items.size());
guard.unlock();
buf_->push(items.subspan(0, n));
guard.lock();
demand_ -= static_cast<ptrdiff_t>(n);
items = items.subspan(n);
}
}
return true;
}
bool push(const T& item) {
return push(make_span(&item, 1));
}
void close() {
if (buf_) {
buf_->close();
buf_ = nullptr;
}
}
void abort(error reason) {
if (buf_) {
buf_->abort(std::move(reason));
buf_ = nullptr;
}
}
bool cancelled() const {
std::unique_lock<std::mutex> guard{mtx_};
return demand_ == -1;
}
void on_consumer_ready() override {
// nop
}
void on_consumer_cancel() override {
std::unique_lock<std::mutex> guard{mtx_};
demand_ = -1;
cv_.notify_all();
}
void on_consumer_demand(size_t demand) override {
std::unique_lock<std::mutex> guard{mtx_};
if (demand_ == 0) {
demand_ += static_cast<ptrdiff_t>(demand);
cv_.notify_all();
} else if (demand_ > 0) {
demand_ += static_cast<ptrdiff_t>(demand);
}
}
void ref_producer() const noexcept override {
ref();
}
void deref_producer() const noexcept override {
deref();
}
CAF_INTRUSIVE_PTR_FRIENDS(impl)
private:
spsc_buffer_ptr<T> buf_;
mutable std::mutex mtx_;
std::condition_variable cv_;
ptrdiff_t demand_ = 0;
};
using impl_ptr = intrusive_ptr<impl>;
blocking_producer() = default;
blocking_producer(const blocking_producer&) = delete;
blocking_producer& operator=(const blocking_producer&) = delete;
blocking_producer(blocking_producer&&) = default;
blocking_producer& operator=(blocking_producer&&) = default;
explicit blocking_producer(impl_ptr ptr) : impl_(std::move(ptr)) {
// nop
}
~blocking_producer() {
if (impl_)
impl_->close();
}
/// Pushes an item to the consumer. If there is no demand by the consumer to
/// deliver the item, this functions blocks unconditionally.
/// @returns `true` if the item was delivered to the consumer or `false` if
/// the consumer no longer receives any additional item.
bool push(const T& item) {
return impl_->push(item);
}
/// Pushes multiple items to the consumer. If there is no demand by the
/// consumer to deliver all items, this functions blocks unconditionally until
/// all items have been delivered.
/// @returns `true` if all items were delivered to the consumer or `false` if
/// the consumer no longer receives any additional item.
bool push(span<const T> items) {
return impl_->push(items);
}
void close() {
if (impl_) {
impl_->close();
impl_ = nullptr;
}
}
void abort(error reason) {
if (impl_) {
impl_->abort(std::move(reason));
impl_ = nullptr;
}
}
/// Checks whether the consumer cancelled its subscription.
bool cancelled() const {
return impl_->cancelled();
}
private:
intrusive_ptr<impl> impl_;
};
template <class T>
expected<blocking_producer<T>>
make_blocking_producer(producer_resource<T> res) {
if (auto buf = res.try_open()) {
using impl_t = typename blocking_producer<T>::impl;
return {blocking_producer<T>{make_counted<impl_t>(std::move(buf))}};
} else {
return {make_error(sec::cannot_open_resource)};
}
}
} // namespace caf::async
// 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 <list>
#include <mutex>
#include <tuple>
#include "caf/async/batch.hpp"
#include "caf/defaults.hpp"
#include "caf/flow/observer.hpp"
#include "caf/flow/subscription.hpp"
namespace caf::async {
/// Enables buffered consumption of published items.
template <class T>
class observer_buffer : public flow::observer_impl<T> {
public:
observer_buffer() {
// nop
}
void on_complete() override {
std::unique_lock guard{mtx_};
if (!done_) {
sub_ = nullptr;
done_ = true;
deinit(guard);
}
}
void on_error(const error& what) override {
std::unique_lock guard{mtx_};
if (!done_) {
sub_ = nullptr;
done_ = true;
err_ = what;
deinit(guard);
}
}
void on_next(span<const T> items) override {
on_batch(make_batch(items));
}
void on_batch(const batch& buf) override {
std::list<batch> ls;
ls.emplace_back(buf);
std::unique_lock guard{mtx_};
batches_.splice(batches_.end(), std::move(ls));
if (batches_.size() == 1)
wakeup(guard);
}
void on_attach(flow::subscription sub) override {
CAF_ASSERT(sub.valid());
std::unique_lock guard{mtx_};
sub_ = std::move(sub);
init(guard);
}
bool has_data() {
if (local.pos_ != local.end_) {
return true;
} else {
std::unique_lock guard{mtx_};
return !batches_.empty();
}
}
/// Tries to fetch the next value. If no value exists, the first element in
/// the tuple is `nullptr`. The second value indicates whether the stream was
/// closed. If the stream was closed, the third value is `nullptr` if
/// `on_complete` was called and a pointer to the error if `on_error` was
/// called.
std::tuple<const T*, bool, const error*> poll() {
if (local.pos_ != local.end_) {
auto res = local.pos_;
if (++local.pos_ == local.end_) {
std::unique_lock guard{mtx_};
if (sub_)
sub_.request(local.cache_.size());
}
return {res, false, nullptr};
} else if (std::unique_lock guard{mtx_}; !batches_.empty()) {
batches_.front().swap(local.cache_);
batches_.pop_front();
guard.unlock();
auto items = local.cache_.template items<T>();
CAF_ASSERT(!items.empty());
local.pos_ = items.begin();
local.end_ = items.end();
return {local.pos_++, false, nullptr};
} else if (!err_) {
return {nullptr, done_, nullptr};
} else {
return {nullptr, true, &err_};
}
}
void dispose() override {
on_complete();
}
bool disposed() const noexcept override {
std::unique_lock guard{mtx_};
return done_;
}
protected:
template <class WaitFn>
std::tuple<const T*, bool, const error*> wait_with(WaitFn wait_fn) {
if (local.pos_ != local.end_) {
auto res = local.pos_;
if (++local.pos_ == local.end_) {
std::unique_lock guard{mtx_};
if (sub_)
sub_.request(local.cache_.size());
}
return {res, false, nullptr};
} else {
std::unique_lock guard{mtx_};
while (batches_.empty() && !done_)
wait_fn(guard);
if (!batches_.empty()) {
batches_.front().swap(local.cache_);
batches_.pop_front();
guard.unlock();
auto items = local.cache_.template items<T>();
local.pos_ = items.begin();
local.end_ = items.end();
return {local.pos_++, false, nullptr};
} else if (!err_) {
return {nullptr, done_, nullptr};
} else {
return {nullptr, true, &err_};
}
}
}
/// Wraps fields that we only access from the consumer's thread.
struct local_t {
const T* pos_ = nullptr;
const T* end_ = nullptr;
batch cache_;
} local;
static_assert(sizeof(local_t) < CAF_CACHE_LINE_SIZE);
/// Avoids false sharing.
char pad_[CAF_CACHE_LINE_SIZE - sizeof(local_t)];
// -- state for consumer and publisher ---------------------------------------
/// Protects fields that we access with both the consumer and the producer.
mutable std::mutex mtx_;
flow::subscription sub_;
bool done_ = false;
error err_;
std::list<batch> batches_;
private:
virtual void init(std::unique_lock<std::mutex>&) {
sub_.request(defaults::flow::buffer_size);
}
virtual void deinit(std::unique_lock<std::mutex>& guard) {
wakeup(guard);
}
virtual void wakeup(std::unique_lock<std::mutex>&) {
// Customization point.
}
};
} // namespace caf::async
// 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
namespace caf::async {
/// Policy type for having `consume` call `on_error` immediately after the
/// producer has aborted even if the buffer still contains items.
struct prioritize_errors_t {
static constexpr bool calls_on_error = true;
};
/// @relates prioritize_errors_t
constexpr auto prioritize_errors = prioritize_errors_t{};
/// Policy type for having `consume` call `on_error` only after processing all
/// items from the buffer.
struct delay_errors_t {
static constexpr bool calls_on_error = true;
};
/// @relates delay_errors_t
constexpr auto delay_errors = delay_errors_t{};
} // namespace caf::async
// 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/default_enum_inspect.hpp"
#include "caf/detail/core_export.hpp"
#include <type_traits>
namespace caf::async {
/// Encodes the result of an asynchronous read operation.
enum class read_result {
/// Signals that the read operation succeeded.
ok,
/// Signals that the reader reached the end of the input.
stop,
/// Signals that the source failed with an error.
abort,
/// Signals that the read operation timed out.
timeout,
};
/// @relates read_result
CAF_CORE_EXPORT std::string to_string(read_result);
/// @relates read_result
CAF_CORE_EXPORT bool from_string(string_view, read_result&);
/// @relates read_result
CAF_CORE_EXPORT bool from_integer(std::underlying_type_t<read_result>,
read_result&);
/// @relates read_result
template <class Inspector>
bool inspect(Inspector& f, read_result& x) {
return default_enum_inspect(f, x);
}
} // namespace caf::async
......@@ -4,10 +4,8 @@
#pragma once
#include <cstdlib>
#include <mutex>
#include "caf/async/consumer.hpp"
#include "caf/async/policy.hpp"
#include "caf/async/producer.hpp"
#include "caf/config.hpp"
#include "caf/defaults.hpp"
......@@ -20,25 +18,10 @@
#include "caf/span.hpp"
#include "caf/unit.hpp"
namespace caf::async {
/// Policy type for having `consume` call `on_error` immediately after the
/// producer has aborted even if the buffer still contains items.
struct prioritize_errors_t {
static constexpr bool calls_on_error = true;
};
/// @relates prioritize_errors_t
constexpr auto prioritize_errors = prioritize_errors_t{};
/// Policy type for having `consume` call `on_error` only after processing all
/// items from the buffer.
struct delay_errors_t {
static constexpr bool calls_on_error = true;
};
#include <cstdlib>
#include <mutex>
/// @relates delay_errors_t
constexpr auto delay_errors = delay_errors_t{};
namespace caf::async {
/// A Single Producer Single Consumer buffer. The buffer uses a "soft bound",
/// which means that the producer announces a desired maximum for in-flight
......@@ -55,6 +38,8 @@ class spsc_buffer : public ref_counted {
public:
using value_type = T;
using lock_type = std::unique_lock<std::mutex>;
spsc_buffer(uint32_t capacity, uint32_t min_pull_size)
: capacity_(capacity), min_pull_size_(min_pull_size) {
// Allocate some extra space in the buffer in case the producer goes beyond
......@@ -70,7 +55,7 @@ public:
/// the buffer becomes non-empty.
/// @returns the remaining capacity after inserting the items.
size_t push(span<const T> items) {
std::unique_lock guard{mtx_};
lock_type guard{mtx_};
CAF_ASSERT(producer_ != nullptr);
CAF_ASSERT(!closed_);
buf_.insert(buf_.end(), items.begin(), items.end());
......@@ -94,78 +79,41 @@ public:
/// first tuple element, the function has called `on_complete` or
/// `on_error` on the observer.
template <class Policy, class Observer>
std::pair<bool, size_t> pull(Policy, size_t demand, Observer& dst) {
std::unique_lock guard{mtx_};
CAF_ASSERT(consumer_ != nullptr);
CAF_ASSERT(consumer_buf_.empty());
if constexpr (std::is_same_v<Policy, prioritize_errors_t>) {
if (err_) {
consumer_ = nullptr;
dst.on_error(err_);
return {false, 0};
}
}
// We must not signal demand to the producer when reading excess elements
// from the buffer. Otherwise, we end up generating more demand than
// capacity_ allows us to.
auto overflow = buf_.size() <= capacity_ ? 0u : buf_.size() - capacity_;
auto next_n = [this, &demand] { return std::min(demand, buf_.size()); };
size_t consumed = 0;
for (auto n = next_n(); n > 0; n = next_n()) {
using std::make_move_iterator;
consumer_buf_.assign(make_move_iterator(buf_.begin()),
make_move_iterator(buf_.begin() + n));
buf_.erase(buf_.begin(), buf_.begin() + n);
if (overflow == 0) {
signal_demand(n);
} else if (n <= overflow) {
overflow -= n;
} else {
signal_demand(n - overflow);
overflow = 0;
}
guard.unlock();
dst.on_next(span<const T>{consumer_buf_.data(), n});
demand -= n;
consumed += n;
consumer_buf_.clear();
guard.lock();
}
if (!buf_.empty() || !closed_) {
return {true, consumed};
} else {
consumer_ = nullptr;
if (err_)
dst.on_error(err_);
else
dst.on_complete();
return {false, consumed};
}
std::pair<bool, size_t> pull(Policy policy, size_t demand, Observer& dst) {
lock_type guard{mtx_};
return pull_unsafe(guard, policy, demand, dst);
}
/// Checks whether there is any pending data in the buffer.
bool has_data() const noexcept {
std::unique_lock guard{mtx_};
lock_type guard{mtx_};
return !buf_.empty();
}
/// Checks whether the there is data available or whether the producer has
/// closed or aborted the flow.
bool has_consumer_event() const noexcept {
std::unique_lock guard{mtx_};
lock_type guard{mtx_};
return !buf_.empty() || closed_;
}
/// Returns how many items are currently available. This may be greater than
/// the `capacity`.
size_t available() const noexcept {
std::unique_lock guard{mtx_};
lock_type guard{mtx_};
return buf_.size();
}
/// Returns the error from the producer or a default-constructed error if
/// abort was not called yet.
error abort_reason() const {
lock_type guard{mtx_};
return err_;
}
/// Closes the buffer by request of the producer.
void close() {
std::unique_lock guard{mtx_};
lock_type guard{mtx_};
if (producer_) {
closed_ = true;
producer_ = nullptr;
......@@ -177,7 +125,7 @@ public:
/// Closes the buffer by request of the producer and signals an error to the
/// consumer.
void abort(error reason) {
std::unique_lock guard{mtx_};
lock_type guard{mtx_};
if (producer_) {
closed_ = true;
err_ = std::move(reason);
......@@ -189,7 +137,7 @@ public:
/// Closes the buffer by request of the consumer.
void cancel() {
std::unique_lock guard{mtx_};
lock_type guard{mtx_};
if (consumer_) {
consumer_ = nullptr;
if (producer_)
......@@ -200,7 +148,7 @@ public:
/// Consumer callback for the initial handshake between producer and consumer.
void set_consumer(consumer_ptr consumer) {
CAF_ASSERT(consumer != nullptr);
std::unique_lock guard{mtx_};
lock_type guard{mtx_};
if (consumer_)
CAF_RAISE_ERROR("SPSC buffer already has a consumer");
consumer_ = std::move(consumer);
......@@ -211,7 +159,7 @@ public:
/// Producer callback for the initial handshake between producer and consumer.
void set_producer(producer_ptr producer) {
CAF_ASSERT(producer != nullptr);
std::unique_lock guard{mtx_};
lock_type guard{mtx_};
if (producer_)
CAF_RAISE_ERROR("SPSC buffer already has a producer");
producer_ = std::move(producer);
......@@ -224,6 +172,8 @@ public:
return capacity_;
}
// -- unsafe interface for manual locking ------------------------------------
/// Returns the mutex for this object.
auto& mtx() const noexcept {
return mtx_;
......@@ -235,6 +185,82 @@ public:
return buf_.size();
}
/// Returns the error from the producer.
/// @pre 'mtx()' is locked.
const error& abort_reason_unsafe() const noexcept {
return err_;
}
/// Blocks until there is at least one item available or the producer stopped.
/// @pre the consumer calls `cv.notify_all()` in its `on_producer_wakeup`
void await_consumer_ready(lock_type& guard, std::condition_variable& cv) {
while (!closed_ && buf_.empty()) {
cv.wait(guard);
}
}
/// Blocks until there is at least one item available, the producer stopped,
/// or a timeout occurs.
/// @pre the consumer calls `cv.notify_all()` in its `on_producer_wakeup`
template <class TimePoint>
bool await_consumer_ready(lock_type& guard, std::condition_variable& cv,
TimePoint timeout) {
while (!closed_ && buf_.empty())
if (cv.wait_until(guard, timeout) == std::cv_status::timeout)
return false;
return true;
}
template <class Policy, class Observer>
std::pair<bool, size_t>
pull_unsafe(lock_type& guard, Policy, size_t demand, Observer& dst) {
CAF_ASSERT(consumer_ != nullptr);
CAF_ASSERT(consumer_buf_.empty());
if constexpr (std::is_same_v<Policy, prioritize_errors_t>) {
if (err_) {
consumer_ = nullptr;
dst.on_error(err_);
return {false, 0};
}
}
// We must not signal demand to the producer when reading excess elements
// from the buffer. Otherwise, we end up generating more demand than
// capacity_ allows us to.
auto overflow = buf_.size() <= capacity_ ? 0u : buf_.size() - capacity_;
auto next_n = [this, &demand] { return std::min(demand, buf_.size()); };
size_t consumed = 0;
for (auto n = next_n(); n > 0; n = next_n()) {
using std::make_move_iterator;
consumer_buf_.assign(make_move_iterator(buf_.begin()),
make_move_iterator(buf_.begin() + n));
buf_.erase(buf_.begin(), buf_.begin() + n);
if (overflow == 0) {
signal_demand(n);
} else if (n <= overflow) {
overflow -= n;
} else {
signal_demand(n - overflow);
overflow = 0;
}
guard.unlock();
dst.on_next(span<const T>{consumer_buf_.data(), n});
demand -= n;
consumed += n;
consumer_buf_.clear();
guard.lock();
}
if (!buf_.empty() || !closed_) {
return {true, consumed};
} else {
consumer_ = nullptr;
if (err_)
dst.on_error(err_);
else
dst.on_complete();
return {false, consumed};
}
}
private:
void ready() {
producer_->on_consumer_ready();
......@@ -359,6 +385,10 @@ public:
}
}
explicit operator bool() const noexcept {
return ctrl_ != nullptr;
}
private:
intrusive_ptr<resource_ctrl<T, false>> ctrl_;
};
......@@ -398,6 +428,10 @@ public:
}
}
explicit operator bool() const noexcept {
return ctrl_ != nullptr;
}
private:
intrusive_ptr<resource_ctrl<T, true>> ctrl_;
};
......
// 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/default_enum_inspect.hpp"
#include "caf/detail/core_export.hpp"
#include <type_traits>
namespace caf::async {
/// Encodes the result of an asynchronous write operation.
enum class write_result {
/// Signals that the write operation succeeded.
ok,
/// Signals that the item must be dropped because the write operation failed
/// with an unrecoverable error. Retries will fail with the same result. When
/// writing to a @ref producer_resource, this usually means the consumer
/// closed its end of the buffer.
drop,
/// Signals that the write operation timed out.
timeout,
};
/// @relates write_result
CAF_CORE_EXPORT std::string to_string(write_result);
/// @relates write_result
CAF_CORE_EXPORT bool from_string(string_view, write_result&);
/// @relates write_result
CAF_CORE_EXPORT bool from_integer(std::underlying_type_t<write_result>,
write_result&);
/// @relates write_result
template <class Inspector>
bool inspect(Inspector& f, write_result& x) {
return default_enum_inspect(f, x);
}
} // namespace caf::async
......@@ -11,6 +11,71 @@
namespace caf::flow {
// -- forward declarations -----------------------------------------------------
template <class T>
class repeater_source;
template <class Container>
class container_source;
template <class T>
class value_source;
template <class F>
class callable_source;
// -- builder interface --------------------------------------------------------
/// Factory for @ref observable objects.
class observable_builder {
public:
friend class coordinator;
observable_builder(const observable_builder&) noexcept = default;
observable_builder& operator=(const observable_builder&) noexcept = default;
/// Creates a @ref generation that emits `value` indefinitely.
template <class T>
[[nodiscard]] generation<repeater_source<T>> repeat(T value) const;
/// Creates a @ref generation that emits all values from `values`.
template <class Container>
[[nodiscard]] generation<container_source<Container>>
from_container(Container values) const;
/// Creates a @ref generation that emits `value` once.
template <class T>
[[nodiscard]] generation<value_source<T>> just(T value) const;
/// Creates a @ref generation that emits values by repeatedly calling `fn`.
template <class F>
[[nodiscard]] generation<callable_source<F>> from_callable(F fn) const;
/// Creates a @ref generation that emits values by repeatedly calling
/// `pullable.pull(...)`. For example implementations of the `Pullable`
/// concept, see @ref container_source, @ref repeater_source and
/// @ref callable_source.
template <class Pullable>
[[nodiscard]] generation<Pullable> lift(Pullable pullable) const;
/// Creates an @ref observable that reads and emits all values from `res`.
template <class T>
[[nodiscard]] observable<T>
from_resource(async::consumer_resource<T> res) const;
private:
explicit observable_builder(coordinator* ctx) : ctx_(ctx) {
// nop
}
coordinator* ctx_;
};
// -- generation ---------------------------------------------------------------
/// Implements the `Pullable` concept for emitting values from a container.
template <class Container>
class container_source {
public:
......@@ -43,6 +108,7 @@ private:
typename Container::const_iterator pos_;
};
/// Implements the `Pullable` concept for emitting the same value repeatedly.
template <class T>
class repeater_source {
public:
......@@ -68,6 +134,33 @@ private:
T value_;
};
/// Implements the `Pullable` concept for emitting the same value once.
template <class T>
class value_source {
public:
using output_type = T;
explicit value_source(T value) : value_(std::move(value)) {
// nop
}
value_source(value_source&&) = default;
value_source(const value_source&) = default;
value_source& operator=(value_source&&) = default;
value_source& operator=(const value_source&) = default;
template <class Step, class... Steps>
void pull(size_t, Step& step, Steps&... steps) {
if (step.on_next(value_, steps...))
step.on_complete(steps...);
}
private:
T value_;
};
/// Implements the `Pullable` concept for emitting values generated from a
/// function object.
template <class F>
class callable_source {
public:
......@@ -94,45 +187,8 @@ private:
F fn_;
};
class observable_builder {
public:
friend class coordinator;
observable_builder(const observable_builder&) noexcept = default;
observable_builder& operator=(const observable_builder&) noexcept = default;
template <class T>
[[nodiscard]] generation<repeater_source<T>> repeat(T value) const;
template <class Container>
[[nodiscard]] generation<container_source<Container>>
from_container(Container values) const;
template <class T>
[[nodiscard]] auto just(T value) const;
template <class F>
[[nodiscard]] generation<callable_source<F>> from_callable(F fn) const;
/// Opens an asynchronous, buffered resource and emits all inputs from the
/// buffer.
template <class T>
[[nodiscard]] observable<T>
from_resource(async::consumer_resource<T> hdl) const;
template <class Pullable>
[[nodiscard]] generation<Pullable> lift(Pullable pullable) const;
private:
explicit observable_builder(coordinator* ctx) : ctx_(ctx) {
// nop
}
coordinator* ctx_;
};
// -- generation ---------------------------------------------------------------
/// Helper class for combining multiple generation and transformation steps into
/// a single @ref observable object.
template <class Generator, class... Steps>
class generation final
: public observable_def<
......@@ -235,8 +291,8 @@ observable_builder::from_container(Container values) const {
// -- observable_builder::just -------------------------------------------------
template <class T>
auto observable_builder::just(T value) const {
return repeat(std::move(value)).take(1);
generation<value_source<T>> observable_builder::just(T value) const {
return {ctx_, value_source<T>{std::move(value)}};
}
// -- observable_builder::from_callable ----------------------------------------
......
......@@ -227,7 +227,7 @@ public:
}
}
// -- implementation of async::producer: these may get called concurrently ---
// -- implementation of async::producer: must be thread-safe -----------------
void on_consumer_ready() override {
// nop
......
......@@ -122,3 +122,27 @@ CAF_CORE_EXPORT config_value::dictionary& put_dictionary(settings& xs,
std::string name);
} // namespace caf
namespace caf::detail {
template <class T>
struct has_init {
private:
template <class U>
static auto sfinae(U* x, settings* y = nullptr)
-> decltype(x->init(*y), std::true_type());
template <class U>
static auto sfinae(...) -> std::false_type;
using sfinae_type
= decltype(sfinae<T>(nullptr, static_cast<settings*>(nullptr)));
public:
static constexpr bool value = sfinae_type::value;
};
template <class T>
constexpr bool has_init_v = has_init<T>::value;
} // namespace caf::detail
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