Commit 10304153 authored by Dominik Charousset's avatar Dominik Charousset

Merge branch 'topic/neverlord/future'

parents 042c79ed f30ebfdd
......@@ -86,6 +86,7 @@ caf_add_component(
src/actor_system_config.cpp
src/async/batch.cpp
src/async/consumer.cpp
src/async/execution_context.cpp
src/async/producer.cpp
src/attachable.cpp
src/behavior.cpp
......@@ -223,6 +224,7 @@ caf_add_component(
actor_system_config
actor_termination
aout
async.promise
async.spsc_buffer
behavior
binary_deserializer
......
// 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/action.hpp"
#include "caf/detail/core_export.hpp"
#include "caf/intrusive_ptr.hpp"
#include <type_traits>
namespace caf::async {
/// Represents a single execution context with an internal event-loop to
/// schedule @ref action objects.
class CAF_CORE_EXPORT execution_context {
public:
// -- constructors, destructors, and assignment operators --------------------
virtual ~execution_context();
// -- reference counting -----------------------------------------------------
/// Increases the reference count of the execution_context.
virtual void ref_execution_context() const noexcept = 0;
/// Decreases the reference count of the execution context and destroys the
/// object if necessary.
virtual void deref_execution_context() const noexcept = 0;
// -- scheduling of actions --------------------------------------------------
/// Schedules @p what to run on the event loop of the execution context. This
/// member function may get called from external sources or threads.
/// @thread-safe
virtual void schedule(action what) = 0;
///@copydoc schedule
template <class F>
void schedule_fn(F&& what) {
static_assert(std::is_invocable_v<F>);
return schedule(make_action(std::forward<F>(what)));
}
// -- lifetime management ----------------------------------------------------
/// Asks the coordinator to keep its event loop running until @p what becomes
/// disposed since it depends on external events or produces events that are
/// visible to outside observers. Must be called from within the event loop of
/// the execution context.
virtual void watch(disposable what) = 0;
};
/// @relates execution_context
inline void intrusive_ptr_add_ref(const execution_context* ptr) noexcept {
ptr->ref_execution_context();
}
/// @relates execution_context
inline void intrusive_ptr_release(const execution_context* ptr) noexcept {
ptr->deref_execution_context();
}
/// @relates execution_context
using execution_context_ptr = intrusive_ptr<execution_context>;
} // 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/async/execution_context.hpp"
#include "caf/async/fwd.hpp"
#include "caf/detail/async_cell.hpp"
#include "caf/disposable.hpp"
#include "caf/error.hpp"
#include "caf/sec.hpp"
namespace caf::async {
/// Provides an interface for accessing the result of an asynchronous
/// computation on an asynchronous @ref execution_context.
template <class T>
class bound_future {
public:
friend class future<T>;
bound_future() noexcept = default;
bound_future(bound_future&&) noexcept = default;
bound_future(const bound_future&) noexcept = default;
bound_future& operator=(bound_future&&) noexcept = default;
bound_future& operator=(const bound_future&) noexcept = default;
/// Retrieves the result at some point in the future and then calls either
/// @p on_success if the asynchronous operation generated a result or
/// @p on_error if the asynchronous operation resulted in an error.
template <class OnSuccess, class OnError>
disposable then(OnSuccess on_success, OnError on_error) {
static_assert(std::is_invocable_v<OnSuccess, const T&>);
static_assert(std::is_invocable_v<OnError, const error&>);
auto cb = [cp = cell_, f = std::move(on_success), g = std::move(on_error)] {
// Note: no need to lock the mutex. Once the cell has a value and actions
// are allowed to run, the value is immutable.
switch (cp->value.index()) {
default:
g(make_error(sec::broken_promise, "future found an invalid value"));
break;
case 1:
f(static_cast<const T&>(std::get<T>(cp->value)));
break;
case 2:
g(static_cast<const error&>(std::get<error>(cp->value)));
}
};
auto cb_action = make_action(std::move(cb));
auto event = typename cell_type::event{ctx_, cb_action};
bool fire_immediately = false;
{ // Critical section.
std::unique_lock guard{cell_->mtx};
if (std::holds_alternative<none_t>(cell_->value)) {
cell_->events.push_back(std::move(event));
} else {
fire_immediately = true;
}
}
if (fire_immediately)
event.first->schedule(std::move(event.second));
auto res = std::move(cb_action).as_disposable();
ctx_->watch(res);
return res;
}
private:
using cell_type = detail::async_cell<T>;
using cell_ptr = std::shared_ptr<cell_type>;
bound_future(execution_context* ctx, cell_ptr cell)
: ctx_(ctx), cell_(std::move(cell)) {
// nop
}
execution_context* ctx_;
cell_ptr cell_;
};
/// Represents the result of an asynchronous computation.
template <class T>
class future {
public:
friend class promise<T>;
future() noexcept = default;
future(future&&) noexcept = default;
future(const future&) noexcept = default;
future& operator=(future&&) noexcept = default;
future& operator=(const future&) noexcept = default;
bool valid() const noexcept {
return cell_ != nullptr;
}
explicit operator bool() const noexcept {
return valid();
}
bool operator!() const noexcept {
return !valid();
}
/// Binds this future to an @ref execution_context to run callbacks.
/// @pre `valid()`
bound_future<T> bind_to(execution_context* ctx) && {
return {ctx, std::move(cell_)};
}
/// Binds this future to an @ref execution_context to run callbacks.
/// @pre `valid()`
bound_future<T> bind_to(execution_context* ctx) const& {
return {ctx, cell_};
}
private:
using cell_ptr = std::shared_ptr<detail::async_cell<T>>;
explicit future(cell_ptr cell) : cell_(std::move(cell)) {
// nop
}
cell_ptr cell_;
};
} // namespace caf::async
......@@ -25,6 +25,12 @@ class consumer_resource;
template <class T>
class producer_resource;
template <class T>
class promise;
template <class T>
class future;
// -- free function templates --------------------------------------------------
template <class T>
......
// 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/execution_context.hpp"
#include "caf/async/future.hpp"
#include "caf/detail/async_cell.hpp"
#include "caf/disposable.hpp"
#include "caf/raise_error.hpp"
namespace caf::async {
/// Provides a facility to store a value or an error that is later acquired
/// asynchronously via a @ref future object. A promise may deliver only one
/// value.
template <class T>
class promise {
public:
promise(promise&&) noexcept = default;
promise(const promise&) noexcept = default;
promise& operator=(promise&&) noexcept = default;
promise& operator=(const promise&) noexcept = default;
promise() : cell_(std::make_shared<cell_type>()) {
// nop
}
~promise() {
if (cell_) {
auto& cnt = cell_->promises;
if (cnt == 1 || cnt.fetch_sub(1, std::memory_order_acq_rel) == 1) {
typename cell_type::event_list events;
{ // Critical section.
std::unique_lock guard{cell_->mtx};
if (std::holds_alternative<none_t>(cell_->value)) {
cell_->value = make_error(sec::broken_promise);
cell_->events.swap(events);
}
}
for (auto& [listener, callback] : events)
listener->schedule(std::move(callback));
}
}
}
bool valid() const noexcept {
return cell_ != nullptr;
}
explicit operator bool() const noexcept {
return valid();
}
bool operator!() const noexcept {
return !valid();
}
/// @pre `valid()`
void set_value(T value) {
if (cell_) {
do_set(value);
cell_ = nullptr;
}
}
/// @pre `valid()`
void set_error(error reason) {
if (cell_) {
do_set(reason);
cell_ = nullptr;
}
}
/// @pre `valid()`
future<T> get_future() const {
return future<T>{cell_};
}
private:
using cell_type = detail::async_cell<T>;
using cell_ptr = std::shared_ptr<cell_type>;
explicit promise(cell_type cell) : cell_(std::move(cell)) {
CAF_ASSERT(cell_ != nullptr);
cell_->promises.fetch_add(1, std::memory_order_relaxed);
}
template <class What>
void do_set(What& what) {
typename cell_type::event_list events;
{ // Critical section.
std::unique_lock guard{cell_->mtx};
if (std::holds_alternative<none_t>(cell_->value)) {
cell_->value = std::move(what);
cell_->events.swap(events);
} else {
CAF_RAISE_ERROR("promise already satisfied");
}
}
for (auto& [listener, callback] : events)
listener->schedule(std::move(callback));
}
cell_ptr cell_;
};
} // namespace caf::async
......@@ -398,7 +398,7 @@ public:
}
template <class Coordinator>
auto observe_on(Coordinator* ctx) {
auto observe_on(Coordinator* ctx) const {
return ctx->make_observable().from_resource(*this);
}
......
// 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/action.hpp"
#include "caf/async/execution_context.hpp"
#include "caf/config.hpp"
#include <atomic>
#include <memory>
#include <mutex>
#include <variant>
#include <vector>
namespace caf::detail {
/// Implementation detail for @ref async::future and @ref async::promise.
template <class T>
struct async_cell {
async_cell() : promises(1) {
// Make room for a couple of events to avoid frequent heap allocations in
// critical sections. We could also use a custom allocator to use
// small-buffer-optimization.
events.reserve(8);
}
async_cell(const async_cell&) = delete;
async_cell& operator=(const async_cell&) = delete;
using atomic_count = std::atomic<size_t>;
atomic_count promises;
std::byte padding[CAF_CACHE_LINE_SIZE - sizeof(atomic_count)];
using event = std::pair<async::execution_context_ptr, action>;
using event_list = std::vector<event>;
std::mutex mtx;
std::variant<none_t, T, error> value;
event_list events;
};
} // namespace caf::detail
......@@ -5,6 +5,7 @@
#pragma once
#include "caf/action.hpp"
#include "caf/async/execution_context.hpp"
#include "caf/detail/core_export.hpp"
#include "caf/flow/fwd.hpp"
#include "caf/flow/subscription.hpp"
......@@ -19,7 +20,7 @@ namespace caf::flow {
/// Coordinates any number of co-located observables and observers. The
/// co-located objects never need to synchronize calls to other co-located
/// objects since the coordinator guarantees synchronous execution.
class CAF_CORE_EXPORT coordinator {
class CAF_CORE_EXPORT coordinator : public async::execution_context {
public:
// -- member types -----------------------------------------------------------
......@@ -30,35 +31,11 @@ public:
virtual ~coordinator();
// -- reference counting -----------------------------------------------------
/// Increases the reference count of the coordinator.
virtual void ref_coordinator() const noexcept = 0;
/// Decreases the reference count of the coordinator and destroys the object
/// if necessary.
virtual void deref_coordinator() const noexcept = 0;
friend void intrusive_ptr_add_ref(const coordinator* ptr) noexcept {
ptr->ref_coordinator();
}
friend void intrusive_ptr_release(const coordinator* ptr) noexcept {
ptr->deref_coordinator();
}
// -- conversions ------------------------------------------------------------
/// Returns a factory object for new observable objects on this coordinator.
[[nodiscard]] observable_builder make_observable();
// -- lifetime management ----------------------------------------------------
/// Asks the coordinator to keep its event loop running until `obj` becomes
/// disposed since it depends on external events or produces events that are
/// visible to outside observers.
virtual void watch(disposable what) = 0;
// -- time -------------------------------------------------------------------
/// Returns the current time on the monotonic clock of this coordinator.
......@@ -66,17 +43,6 @@ public:
// -- scheduling of actions --------------------------------------------------
/// Schedules an action for execution on this coordinator. This member
/// function may get called from external sources or threads.
/// @thread-safe
virtual void schedule(action what) = 0;
///@copydoc schedule
template <class F>
void schedule_fn(F&& what) {
return schedule(make_action(std::forward<F>(what)));
}
/// Delays execution of an action until all pending actions were executed. May
/// call `schedule`.
/// @param what The action for delayed execution.
......
......@@ -31,13 +31,13 @@ public:
from_resource_sub(coordinator* ctx, buffer_ptr buf, observer<value_type> out)
: ctx_(ctx), buf_(buf), out_(std::move(out)) {
ctx_->ref_coordinator();
ctx_->ref_execution_context();
}
~from_resource_sub() {
if (buf_)
buf_->cancel();
ctx_->deref_coordinator();
ctx_->deref_execution_context();
}
// -- implementation of subscription_impl ------------------------------------
......@@ -157,9 +157,9 @@ private:
return {this};
}
/// Stores the context (coordinator) that runs this flow. Unlike other
/// observables, we need a strong reference to the context because otherwise
/// the buffer might call schedule_fn on a destroyed object.
/// Stores the @ref coordinator that runs this flow. Unlike other observables,
/// we need a strong reference to the coordinator because otherwise the buffer
/// might call `schedule_fn` on a destroyed object.
intrusive_ptr<coordinator> ctx_;
/// Stores a pointer to the asynchronous input buffer.
......
......@@ -27,9 +27,9 @@ public:
// -- reference counting -----------------------------------------------------
void ref_coordinator() const noexcept override;
void ref_execution_context() const noexcept override;
void deref_coordinator() const noexcept override;
void deref_execution_context() const noexcept override;
friend void intrusive_ptr_add_ref(const scoped_coordinator* ptr) {
ptr->ref();
......
......@@ -427,9 +427,9 @@ public:
steady_time_point steady_time() override;
void ref_coordinator() const noexcept override;
void ref_execution_context() const noexcept override;
void deref_coordinator() const noexcept override;
void deref_execution_context() const noexcept override;
void schedule(action what) override;
......
// 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/async/execution_context.hpp"
namespace caf::async {
execution_context::~execution_context() {
// nop
}
} // namespace caf::async
......@@ -28,11 +28,11 @@ void scoped_coordinator::run() {
// -- reference counting -------------------------------------------------------
void scoped_coordinator::ref_coordinator() const noexcept {
void scoped_coordinator::ref_execution_context() const noexcept {
ref();
}
void scoped_coordinator::deref_coordinator() const noexcept {
void scoped_coordinator::deref_execution_context() const noexcept {
deref();
}
......
......@@ -385,11 +385,11 @@ flow::coordinator::steady_time_point scheduled_actor::steady_time() {
return clock().now();
}
void scheduled_actor::ref_coordinator() const noexcept {
void scheduled_actor::ref_execution_context() const noexcept {
intrusive_ptr_add_ref(ctrl());
}
void scheduled_actor::deref_coordinator() const noexcept {
void scheduled_actor::deref_execution_context() const noexcept {
intrusive_ptr_release(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.
#define CAF_SUITE async.promise
#include "caf/async/promise.hpp"
#include "core-test.hpp"
#include "caf/scheduled_actor/flow.hpp"
using namespace caf;
using namespace std::literals;
BEGIN_FIXTURE_SCOPE(test_coordinator_fixture<>)
SCENARIO("actors can observe futures") {
GIVEN("a promise and future pair") {
WHEN("passing a non-ready future to an actor") {
THEN("the actor can observe the value via .then() later") {
auto val = std::make_shared<std::variant<none_t, std::string, error>>();
auto uut = async::promise<std::string>{};
auto fut = uut.get_future();
auto testee = sys.spawn([val, fut](event_based_actor* self) {
fut.bind_to(self).then([val](const std::string& str) { *val = str; },
[val](const error& err) { *val = err; });
});
run();
CHECK(std::holds_alternative<none_t>(*val));
uut.set_value("hello world"s);
expect((action), to(testee));
if (CHECK(std::holds_alternative<std::string>(*val)))
CHECK_EQ(std::get<std::string>(*val), "hello world");
}
}
WHEN("passing a ready future to an actor") {
THEN("the actor can observe the value via .then() immediately") {
auto val = std::make_shared<std::variant<none_t, std::string, error>>();
auto uut = async::promise<std::string>{};
auto fut = uut.get_future();
uut.set_value("hello world"s);
auto testee = sys.spawn([val, fut](event_based_actor* self) {
fut.bind_to(self).then([val](const std::string& str) { *val = str; },
[val](const error& err) { *val = err; });
});
run();
if (CHECK(std::holds_alternative<std::string>(*val)))
CHECK_EQ(std::get<std::string>(*val), "hello world");
}
}
WHEN("passing a non-ready future to an actor and disposing the action") {
THEN("the actor never observes the value") {
auto val = std::make_shared<std::variant<none_t, std::string, error>>();
auto uut = async::promise<std::string>{};
auto fut = uut.get_future();
auto hdl = disposable{};
auto testee = sys.spawn([val, fut, &hdl](event_based_actor* self) {
hdl = fut.bind_to(self).then(
[val](const std::string& str) { *val = str; },
[val](const error& err) { *val = err; });
});
run();
CHECK(std::holds_alternative<none_t>(*val));
hdl.dispose();
uut.set_value("hello world"s);
run();
CHECK(std::holds_alternative<none_t>(*val));
}
}
}
}
END_FIXTURE_SCOPE()
Markdown is supported
0%
or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment