Commit 0d49d8d4 authored by Lewis Baker's avatar Lewis Baker Committed by Facebook Github Bot

Refactor folly::coro::Task to eliminate unnecessary synchronisation

Summary:
This ability to eagerly start a folly::coro::Task by calling .scheduleVia() introduced a potential race between the operation completing concurrently on another thread and the current thread then subsequently co_awaiting the returned folly::coro::Future.

Thus the implementation of coro::Task's promise_type required the use of an atomic variable to decide this race.

This diff eliminates the need for that synchronisation by separating the step of associating the task with an executor from the step of launching the task so that the Task is always lazily started.

The folly::coro::Task<T>::scheduleVia() method has now been replaced with a .scheduleOn() method that returns a folly::coro::TaskWithExecutor<T>. The TaskWithExecutor<T> type is awaitable and will schedule the Task's execution on the bound executor when it is awaited.

Also fixes a cyclic dependency issue between Promise.h and Task.h/Future.h by merging them all into Task.h.

folly/futures/Future.h now defines the `folly::coro::toSemiFuture()` function for wrapping an Awaitable up into a `folly::SemiFuture<T>`. The `folly::coro::Future<T>` type and folly/experimental/coro/Future.h header has been removed.

Reviewed By: andriigrynenko

Differential Revision: D9948564

fbshipit-source-id: 21c9ee1ec0e67d56321629b5901c3eb7cddcfe9b
parent 3020a18c
/*
* Copyright 2017-present Facebook, Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#pragma once
#include <glog/logging.h>
#include <folly/Executor.h>
#include <folly/experimental/coro/Promise.h>
#include <folly/experimental/coro/Task.h>
#include <folly/experimental/coro/Wait.h>
#include <folly/futures/Future.h>
namespace folly {
namespace coro {
/*
* Future object attached to a running coroutine. Implement await_* APIs.
*/
template <typename T>
class Future {
public:
Future(const Future&) = delete;
Future(Future&& other) : promise_(other.promise_) {
other.promise_ = nullptr;
}
class Awaiter {
public:
explicit Awaiter(Promise<T>* promise) noexcept : promise_(promise) {}
bool await_ready() noexcept {
return promise_->isReady();
}
bool await_suspend(std::experimental::coroutine_handle<> awaiter) noexcept {
auto state = promise_->state_.load(std::memory_order_acquire);
if (state == Promise<T>::State::HAS_RESULT) {
return false;
}
DCHECK(state == Promise<T>::State::EMPTY);
promise_->awaiter_ = std::move(awaiter);
if (promise_->state_.compare_exchange_strong(
state,
Promise<T>::State::HAS_AWAITER,
std::memory_order_release,
std::memory_order_acquire)) {
return true;
}
DCHECK(promise_->state_ == Promise<T>::State::HAS_RESULT);
return false;
}
T await_resume() {
DCHECK(promise_->state_ == Promise<T>::State::HAS_RESULT);
return std::move(promise_->result_).value();
}
private:
Promise<T>* promise_;
};
Awaiter operator co_await() && noexcept {
return Awaiter{promise_};
}
auto toFuture() &&;
bool isReady() const noexcept {
DCHECK(promise_);
return promise_->isReady();
}
~Future() {
if (!promise_) {
return;
}
auto state = promise_->state_.load(std::memory_order_acquire);
do {
DCHECK(state != Promise<T>::State::DETACHED);
DCHECK(state != Promise<T>::State::HAS_AWAITER);
if (state == Promise<T>::State::HAS_RESULT) {
auto ch = std::experimental::coroutine_handle<Promise<T>>::from_promise(
*promise_);
DCHECK(ch.done());
ch.destroy();
return;
}
DCHECK(state == Promise<T>::State::EMPTY);
} while (!promise_->state_.compare_exchange_weak(
state,
Promise<T>::State::DETACHED,
std::memory_order::memory_order_release,
std::memory_order::memory_order_acquire));
}
private:
friend class Task<T>;
template <typename U>
friend class Promise;
Future(Promise<T>& promise) : promise_(&promise) {}
Promise<T>* promise_;
};
namespace detail {
inline SemiFuture<Unit> toSemiFuture(Future<void> future) {
co_await std::move(future);
co_return folly::unit;
}
template <typename T>
SemiFuture<T> toSemiFuture(Future<T> future) {
return SemiFuture<T>::fromAwaitable(std::move(future));
}
} // namespace detail
template <typename T>
auto Future<T>::toFuture() && {
auto executor = promise_->executor_;
return detail::toSemiFuture(std::move(*this)).via(executor);
}
} // namespace coro
} // namespace folly
/*
* Copyright 2017-present Facebook, Inc.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
#pragma once
#include <type_traits>
#include <glog/logging.h>
#include <folly/ExceptionWrapper.h>
#include <folly/Try.h>
#include <folly/experimental/coro/Task.h>
#include <folly/experimental/coro/Utils.h>
#include <folly/experimental/coro/ViaIfAsync.h>
#include <folly/futures/Future.h>
namespace folly {
namespace coro {
enum class PromiseState {
// Coroutine hasn't started
EMPTY,
// Coroutine is running, but Future object managing this coroutine was
// destroyed
DETACHED,
// Some other coroutine is waiting on this coroutine to be complete
HAS_AWAITER,
// Coroutine is finished, result is stored inside Promise
HAS_RESULT
};
template <typename T>
class Future;
template <typename T>
class PromiseBase {
public:
template <typename U>
void return_value(U&& value) {
static_assert(
std::is_convertible<U&&, T>::value,
"Returned value is not convertible to task result type");
result_.emplace(static_cast<U&&>(value));
}
protected:
folly::Try<T> result_;
};
template <>
class PromiseBase<void> {
public:
void return_void() {}
protected:
folly::Try<void> result_;
};
template <typename T>
class Promise : public PromiseBase<T> {
public:
using State = PromiseState;
Promise() {}
~Promise() {}
Task<T> get_return_object() {
return {*this};
}
std::experimental::suspend_always initial_suspend() {
return {};
}
template <typename U>
auto await_transform(Task<U>&& task) {
return std::move(task).viaInline(executor_);
}
template <typename Awaitable>
decltype(auto) await_transform(Awaitable&& awaitable) {
using folly::coro::co_viaIfAsync;
return co_viaIfAsync(executor_, std::forward<Awaitable>(awaitable));
}
auto await_transform(getCurrentExecutor) {
return AwaitableReady<Executor*>(executor_);
}
class FinalSuspender;
FinalSuspender final_suspend() {
return {*this};
}
void unhandled_exception() {
this->result_ =
Try<T>(exception_wrapper::from_exception_ptr(std::current_exception()));
}
void start() {
std::experimental::coroutine_handle<Promise>::from_promise (*this)();
}
bool isReady() const noexcept {
return state_.load(std::memory_order_acquire) ==
Promise<T>::State::HAS_RESULT;
}
private:
friend class Future<T>;
friend class Task<T>;
template <typename U>
friend class Promise;
std::atomic<State> state_{State::EMPTY};
std::experimental::coroutine_handle<> awaiter_;
Executor* executor_{nullptr};
};
template <typename T>
class Promise<T>::FinalSuspender {
public:
bool await_ready() {
return promise_.state_.load(std::memory_order_acquire) == State::DETACHED;
}
bool await_suspend(std::experimental::coroutine_handle<>) {
auto state = promise_.state_.load(std::memory_order_acquire);
do {
if (state == State::DETACHED) {
return false;
}
DCHECK(state != State::HAS_RESULT);
} while (!promise_.state_.compare_exchange_weak(
state,
State::HAS_RESULT,
std::memory_order_release,
std::memory_order_acquire));
if (state == State::HAS_AWAITER) {
promise_.awaiter_.resume();
}
return true;
}
void await_resume() {}
private:
friend class Promise;
FinalSuspender(Promise& promise) : promise_(promise) {}
Promise& promise_;
};
} // namespace coro
} // namespace folly
...@@ -15,63 +15,351 @@ ...@@ -15,63 +15,351 @@
*/ */
#pragma once #pragma once
#include <experimental/coroutine>
#include <type_traits>
#include <glog/logging.h> #include <glog/logging.h>
#include <folly/Executor.h> #include <folly/Executor.h>
#include <folly/Portability.h>
#include <folly/ScopeGuard.h>
#include <folly/Try.h>
#include <folly/experimental/coro/Traits.h>
#include <folly/experimental/coro/Utils.h>
#include <folly/experimental/coro/ViaIfAsync.h>
#include <folly/futures/Future.h>
namespace folly { namespace folly {
namespace coro { namespace coro {
template <typename T> struct getCurrentExecutor {};
class Promise;
template <typename T = void>
class Task;
template <typename T = void>
class TaskWithExecutor;
namespace detail {
class TaskPromiseBase {
class FinalAwaiter {
public:
bool await_ready() noexcept {
return false;
}
template <typename Promise>
std::experimental::coroutine_handle<> await_suspend(
std::experimental::coroutine_handle<Promise> coro) noexcept {
TaskPromiseBase& promise = coro.promise();
return promise.continuation_;
}
void await_resume() noexcept {}
};
friend class FinalAwaiter;
protected:
TaskPromiseBase() noexcept : executor_(nullptr) {}
public:
std::experimental::suspend_always initial_suspend() noexcept {
return {};
}
FinalAwaiter final_suspend() noexcept {
return {};
}
template <typename U>
auto await_transform(Task<U>&& t) noexcept;
template <typename Awaitable>
auto await_transform(Awaitable&& awaitable) noexcept {
using folly::coro::co_viaIfAsync;
return co_viaIfAsync(executor_, static_cast<Awaitable&&>(awaitable));
}
auto await_transform(folly::coro::getCurrentExecutor) noexcept {
return AwaitableReady<folly::Executor*>{executor_};
}
private:
template <typename T>
friend class folly::coro::TaskWithExecutor;
template <typename T>
friend class folly::coro::Task;
std::experimental::coroutine_handle<> continuation_;
folly::Executor* executor_;
};
template <typename T> template <typename T>
class Future; class TaskPromise : public TaskPromiseBase {
public:
TaskPromise() noexcept = default;
/* Task<T> get_return_object() noexcept;
* Represents allocated, but not-started coroutine, which is not yet assigned to
* any executor. void unhandled_exception() noexcept {
*/ result_.emplaceException(
exception_wrapper::from_exception_ptr(std::current_exception()));
}
template <typename U>
void return_value(U&& value) {
static_assert(
std::is_convertible<U&&, T>::value,
"cannot convert return value to type T");
result_.emplace(static_cast<U&&>(value));
}
T getResult() {
return static_cast<T&&>(std::move(result_).value());
}
private:
using StorageType = std::conditional_t<
std::is_reference<T>::value,
std::reference_wrapper<std::remove_reference_t<T>>,
T>;
Try<StorageType> result_;
};
template <>
class TaskPromise<void> : public TaskPromiseBase {
public:
TaskPromise() noexcept = default;
Task<void> get_return_object() noexcept;
void unhandled_exception() noexcept {
result_.emplaceException(
exception_wrapper::from_exception_ptr(std::current_exception()));
}
void return_void() noexcept {}
void getResult() {
return std::move(result_).value();
}
private:
Try<void> result_;
};
} // namespace detail
/// Represents an allocated but not yet started coroutine that has already
/// been bound to an executor.
///
/// This task, when co_awaited, will launch the task on the bound executor
/// and will resume the awaiting coroutine on the bound executor when it
/// completes.
template <typename T> template <typename T>
class Task { class FOLLY_NODISCARD TaskWithExecutor {
using handle_t = std::experimental::coroutine_handle<detail::TaskPromise<T>>;
public: public:
using promise_type = Promise<T>; ~TaskWithExecutor() {
if (coro_) {
coro_.destroy();
}
}
TaskWithExecutor(TaskWithExecutor&& t) noexcept
: coro_(std::exchange(t.coro_, {})) {}
Task(const Task&) = delete; TaskWithExecutor& operator=(TaskWithExecutor t) noexcept {
Task(Task&& other) : promise_(other.promise_) { swap(t);
other.promise_ = nullptr; return *this;
} }
~Task() { folly::Executor* executor() const noexcept {
DCHECK(!promise_); return coro_.promise().executor_;
}
void swap(TaskWithExecutor& t) noexcept {
std::swap(coro_, t.coro_);
} }
Future<T> scheduleVia(folly::Executor* executor) && { // Start execution of this task eagerly and return a folly::SemiFuture<T>
promise_->executor_ = executor; // that will complete with the result.
promise_->executor_->add([promise = promise_] { promise->start(); }); auto start() && {
return {*std::exchange(promise_, nullptr)}; return folly::coro::toSemiFuture(std::move(*this));
}
class Awaiter {
public:
explicit Awaiter(handle_t coro) noexcept : coro_(coro) {}
~Awaiter() {
if (coro_) {
coro_.destroy();
}
}
bool await_ready() const {
return false;
}
void await_suspend(
std::experimental::coroutine_handle<> continuation) noexcept {
auto& promise = coro_.promise();
DCHECK(!promise.continuation_);
DCHECK(promise.executor_ != nullptr);
promise.continuation_ = continuation;
promise.executor_->add(coro_);
}
decltype(auto) await_resume() {
// Eagerly destroy the coroutine-frame once we have retrieved the result.
SCOPE_EXIT {
std::exchange(coro_, {}).destroy();
};
return coro_.promise().getResult();
}
private:
handle_t coro_;
};
Awaiter operator co_await() && noexcept {
return Awaiter{std::exchange(coro_, {})};
} }
private: private:
template <typename U> friend class Task<T>;
friend class Promise;
explicit TaskWithExecutor(handle_t coro) noexcept : coro_(coro) {}
handle_t coro_;
};
/// Represents an allocated, but not-started coroutine, which is not yet
/// been bound to an executor.
///
/// You can only co_await a Task from within another Task, in which case it
/// is implicitly bound to the same executor as the parent Task.
///
/// Alternatively, you can explicitly provide an executor by calling the
/// task.scheduleOn(executor) method, which will return a new not-yet-started
/// TaskWithExecutor that can be co_awaited anywhere and that will automatically
/// schedule the coroutine to start executing on the bound executor when it
/// is co_awaited.
///
/// Within the body of a Task's coroutine, it will ensure that it always
/// executes on the bound executor by implicitly transforming every
/// 'co_await expr' expression into
/// `co_await co_viaIfAsync(boundExecutor, expr)' to ensure that the coroutine
/// always resumes on the executor.
template <typename T>
class FOLLY_NODISCARD Task {
public:
using promise_type = detail::TaskPromise<T>;
private:
using handle_t = std::experimental::coroutine_handle<promise_type>;
public:
Task(const Task& t) = delete;
Task(Task&& t) noexcept : coro_(std::exchange(t.coro_, {})) {}
Future<T> viaInline(folly::Executor* executor) && { ~Task() {
promise_->executor_ = executor; if (coro_) {
promise_->start(); coro_.destroy();
return {*std::exchange(promise_, nullptr)}; }
} }
Task(promise_type& promise) : promise_(&promise) {} Task& operator=(Task t) noexcept {
swap(t);
return *this;
}
Promise<T>* promise_; void swap(Task& t) noexcept {
std::swap(coro_, t.coro_);
}
/// Specify the executor that this task should execute on.
///
/// Returns a new task that when co_awaited will launch execution of this
/// task on the specified executor.
FOLLY_NODISCARD
TaskWithExecutor<T> scheduleOn(Executor* executor) && noexcept {
coro_.promise().executor_ = executor;
return TaskWithExecutor<T>{std::exchange(coro_, {})};
}
private:
friend class detail::TaskPromiseBase;
friend class detail::TaskPromise<T>;
Task(handle_t coro) noexcept : coro_(coro) {}
handle_t coro_;
}; };
} // namespace coro template <typename T>
auto detail::TaskPromiseBase::await_transform(Task<T>&& t) noexcept {
class Awaiter {
using handle_t =
std::experimental::coroutine_handle<detail::TaskPromise<T>>;
public:
explicit Awaiter(handle_t coro) noexcept : coro_(coro) {}
Awaiter(Awaiter&& other) noexcept : coro_(std::exchange(other.coro_, {})) {}
Awaiter(const Awaiter&) = delete;
~Awaiter() {
if (coro_) {
coro_.destroy();
}
}
bool await_ready() noexcept {
return false;
}
handle_t await_suspend(
std::experimental::coroutine_handle<> continuation) noexcept {
coro_.promise().continuation_ = continuation;
return coro_;
}
decltype(auto) await_resume() {
SCOPE_EXIT {
std::exchange(coro_, {}).destroy();
};
return coro_.promise().getResult();
}
private:
handle_t coro_;
};
t.coro_.promise().executor_ = executor_;
return Awaiter{std::exchange(t.coro_, {})};
}
template <typename T> template <typename T>
coro::Future<T> via(folly::Executor* executor, coro::Task<T>&& task) { Task<T> detail::TaskPromise<T>::get_return_object() noexcept {
return std::move(task).scheduleVia(executor); return Task<T>{
std::experimental::coroutine_handle<detail::TaskPromise<T>>::from_promise(
*this)};
} }
inline Task<void> detail::TaskPromise<void>::get_return_object() noexcept {
return Task<void>{std::experimental::coroutine_handle<
detail::TaskPromise<void>>::from_promise(*this)};
}
} // namespace coro
} // namespace folly } // namespace folly
...@@ -59,20 +59,6 @@ class AwaitableReady<void> { ...@@ -59,20 +59,6 @@ class AwaitableReady<void> {
void await_resume() noexcept {} void await_resume() noexcept {}
}; };
struct getCurrentExecutor {};
struct yield {
bool await_ready() {
return false;
}
void await_suspend(std::experimental::coroutine_handle<> ch) {
ch();
}
void await_resume() {}
};
template <typename Awaitable> template <typename Awaitable>
class TimedWaitAwaitable { class TimedWaitAwaitable {
public: public:
......
...@@ -19,8 +19,6 @@ ...@@ -19,8 +19,6 @@
#include <folly/executors/InlineExecutor.h> #include <folly/executors/InlineExecutor.h>
#include <folly/experimental/coro/Baton.h> #include <folly/experimental/coro/Baton.h>
#include <folly/experimental/coro/Future.h>
#include <folly/experimental/coro/Promise.h>
#include <folly/experimental/coro/Task.h> #include <folly/experimental/coro/Task.h>
#include <folly/portability/GTest.h> #include <folly/portability/GTest.h>
...@@ -58,8 +56,7 @@ TEST(Baton, AwaitBaton) { ...@@ -58,8 +56,7 @@ TEST(Baton, AwaitBaton) {
CHECK(!reachedBeforeAwait); CHECK(!reachedBeforeAwait);
CHECK(!reachedAfterAwait); CHECK(!reachedAfterAwait);
auto& executor = InlineExecutor::instance(); auto f = std::move(t).scheduleOn(&InlineExecutor::instance()).start();
coro::Future<void> f = via(&executor, std::move(t));
CHECK(reachedBeforeAwait); CHECK(reachedBeforeAwait);
CHECK(!reachedAfterAwait); CHECK(!reachedAfterAwait);
......
...@@ -21,7 +21,8 @@ ...@@ -21,7 +21,8 @@
#include <folly/Chrono.h> #include <folly/Chrono.h>
#include <folly/executors/ManualExecutor.h> #include <folly/executors/ManualExecutor.h>
#include <folly/experimental/coro/BlockingWait.h> #include <folly/experimental/coro/BlockingWait.h>
#include <folly/experimental/coro/Future.h> #include <folly/experimental/coro/Task.h>
#include <folly/experimental/coro/Utils.h>
#include <folly/io/async/ScopedEventBaseThread.h> #include <folly/io/async/ScopedEventBaseThread.h>
#include <folly/portability/GTest.h> #include <folly/portability/GTest.h>
...@@ -33,24 +34,24 @@ coro::Task<int> task42() { ...@@ -33,24 +34,24 @@ coro::Task<int> task42() {
TEST(Coro, Basic) { TEST(Coro, Basic) {
ManualExecutor executor; ManualExecutor executor;
auto future = via(&executor, task42()); auto future = task42().scheduleOn(&executor).start();
EXPECT_FALSE(future.isReady()); EXPECT_FALSE(future.isReady());
executor.drive(); executor.drive();
EXPECT_TRUE(future.isReady()); EXPECT_TRUE(future.isReady());
EXPECT_EQ(42, folly::coro::blockingWait(std::move(future))); EXPECT_EQ(42, std::move(future).get());
} }
TEST(Coro, BasicFuture) { TEST(Coro, BasicFuture) {
ManualExecutor executor; ManualExecutor executor;
auto future = via(&executor, task42()).toFuture(); auto future = task42().scheduleOn(&executor).start();
EXPECT_FALSE(future.isReady()); EXPECT_FALSE(future.isReady());
EXPECT_EQ(42, future.getVia(&executor)); EXPECT_EQ(42, std::move(future).via(&executor).getVia(&executor));
} }
coro::Task<void> taskVoid() { coro::Task<void> taskVoid() {
...@@ -60,7 +61,7 @@ coro::Task<void> taskVoid() { ...@@ -60,7 +61,7 @@ coro::Task<void> taskVoid() {
TEST(Coro, Basic2) { TEST(Coro, Basic2) {
ManualExecutor executor; ManualExecutor executor;
auto future = via(&executor, taskVoid()); auto future = taskVoid().scheduleOn(&executor).start();
EXPECT_FALSE(future.isReady()); EXPECT_FALSE(future.isReady());
...@@ -69,30 +70,28 @@ TEST(Coro, Basic2) { ...@@ -69,30 +70,28 @@ TEST(Coro, Basic2) {
EXPECT_TRUE(future.isReady()); EXPECT_TRUE(future.isReady());
} }
coro::Task<void> taskSleep() {
(void)co_await futures::sleep(std::chrono::seconds{1});
co_return;
}
TEST(Coro, TaskOfMoveOnly) { TEST(Coro, TaskOfMoveOnly) {
auto f = []() -> coro::Task<std::unique_ptr<int>> { auto f = []() -> coro::Task<std::unique_ptr<int>> {
co_return std::make_unique<int>(123); co_return std::make_unique<int>(123);
}; };
auto p = coro::blockingWait(f().scheduleVia(&InlineExecutor::instance())); auto p = coro::blockingWait(f().scheduleOn(&InlineExecutor::instance()));
EXPECT_TRUE(p); EXPECT_TRUE(p);
EXPECT_EQ(123, *p); EXPECT_EQ(123, *p);
} }
coro::Task<void> taskSleep() {
(void)co_await futures::sleep(std::chrono::seconds{1});
co_return;
}
TEST(Coro, Sleep) { TEST(Coro, Sleep) {
ScopedEventBaseThread evbThread; ScopedEventBaseThread evbThread;
auto startTime = std::chrono::steady_clock::now(); auto startTime = std::chrono::steady_clock::now();
auto future = via(evbThread.getEventBase(), taskSleep()); auto task = taskSleep().scheduleOn(evbThread.getEventBase());
EXPECT_FALSE(future.isReady());
coro::blockingWait(std::move(future)); coro::blockingWait(std::move(task));
// The total time should be roughly 1 second. Some builds, especially // The total time should be roughly 1 second. Some builds, especially
// optimized ones, may result in slightly less than 1 second, so we perform // optimized ones, may result in slightly less than 1 second, so we perform
...@@ -109,19 +108,19 @@ coro::Task<int> taskException() { ...@@ -109,19 +108,19 @@ coro::Task<int> taskException() {
TEST(Coro, Throw) { TEST(Coro, Throw) {
ManualExecutor executor; ManualExecutor executor;
auto future = via(&executor, taskException()); auto future = taskException().scheduleOn(&executor).start();
EXPECT_FALSE(future.isReady()); EXPECT_FALSE(future.isReady());
executor.drive(); executor.drive();
EXPECT_TRUE(future.isReady()); EXPECT_TRUE(future.isReady());
EXPECT_THROW(coro::blockingWait(std::move(future)), std::runtime_error); EXPECT_THROW(std::move(future).get(), std::runtime_error);
} }
TEST(Coro, FutureThrow) { TEST(Coro, FutureThrow) {
ManualExecutor executor; ManualExecutor executor;
auto future = via(&executor, taskException()).toFuture(); auto future = taskException().scheduleOn(&executor).start();
EXPECT_FALSE(future.isReady()); EXPECT_FALSE(future.isReady());
...@@ -143,11 +142,17 @@ coro::Task<int> taskRecursion(int depth) { ...@@ -143,11 +142,17 @@ coro::Task<int> taskRecursion(int depth) {
TEST(Coro, LargeStack) { TEST(Coro, LargeStack) {
ScopedEventBaseThread evbThread; ScopedEventBaseThread evbThread;
auto future = via(evbThread.getEventBase(), taskRecursion(5000)); auto task = taskRecursion(5000).scheduleOn(evbThread.getEventBase());
EXPECT_EQ(5000, coro::blockingWait(std::move(future))); EXPECT_EQ(5000, coro::blockingWait(std::move(task)));
} }
#if defined(__clang__)
#define FOLLY_CORO_DONT_OPTIMISE_ON_CLANG __attribute__((optnone))
#else
#define FOLLY_CORO_DONT_OPTIMISE_ON_CLANG
#endif
coro::Task<void> taskThreadNested(std::thread::id threadId) { coro::Task<void> taskThreadNested(std::thread::id threadId) {
EXPECT_EQ(threadId, std::this_thread::get_id()); EXPECT_EQ(threadId, std::this_thread::get_id());
(void)co_await futures::sleep(std::chrono::seconds{1}); (void)co_await futures::sleep(std::chrono::seconds{1});
...@@ -155,12 +160,18 @@ coro::Task<void> taskThreadNested(std::thread::id threadId) { ...@@ -155,12 +160,18 @@ coro::Task<void> taskThreadNested(std::thread::id threadId) {
co_return; co_return;
} }
coro::Task<int> taskThread() { coro::Task<int> taskThread() FOLLY_CORO_DONT_OPTIMISE_ON_CLANG {
auto threadId = std::this_thread::get_id(); auto threadId = std::this_thread::get_id();
// BUG: Under @mode/clang-opt builds this object is placed on the coroutine
// frame and the code for the constructor assumes that it is allocated on
// a 16-byte boundary. However, when placed in the coroutine frame it can
// end up at a location that is not 16-byte aligned. This causes a SIGSEGV
// when performing a store to members that uses SSE instructions.
folly::ScopedEventBaseThread evbThread; folly::ScopedEventBaseThread evbThread;
co_await via(
evbThread.getEventBase(), taskThreadNested(evbThread.getThreadId())); co_await taskThreadNested(evbThread.getThreadId())
.scheduleOn(evbThread.getEventBase());
EXPECT_EQ(threadId, std::this_thread::get_id()); EXPECT_EQ(threadId, std::this_thread::get_id());
...@@ -169,30 +180,21 @@ coro::Task<int> taskThread() { ...@@ -169,30 +180,21 @@ coro::Task<int> taskThread() {
TEST(Coro, NestedThreads) { TEST(Coro, NestedThreads) {
ScopedEventBaseThread evbThread; ScopedEventBaseThread evbThread;
auto future = via(evbThread.getEventBase(), taskThread()); auto task = taskThread().scheduleOn(evbThread.getEventBase());
EXPECT_EQ(42, coro::blockingWait(std::move(task)));
EXPECT_EQ(42, coro::blockingWait(std::move(future)));
} }
coro::Task<int> taskYield(Executor* executor) { coro::Task<int> taskGetCurrentExecutor(Executor* executor) {
auto currentExecutor = co_await coro::getCurrentExecutor(); auto currentExecutor = co_await coro::getCurrentExecutor();
EXPECT_EQ(executor, currentExecutor); EXPECT_EQ(executor, currentExecutor);
co_return co_await task42().scheduleOn(currentExecutor);
auto future = via(currentExecutor, task42());
EXPECT_FALSE(future.isReady());
co_await coro::yield();
EXPECT_TRUE(future.isReady());
co_return co_await std::move(future);
} }
TEST(Coro, CurrentExecutor) { TEST(Coro, CurrentExecutor) {
ScopedEventBaseThread evbThread; ScopedEventBaseThread evbThread;
auto future = auto task = taskGetCurrentExecutor(evbThread.getEventBase())
via(evbThread.getEventBase(), taskYield(evbThread.getEventBase())); .scheduleOn(evbThread.getEventBase());
EXPECT_EQ(42, coro::blockingWait(std::move(task)));
EXPECT_EQ(42, coro::blockingWait(std::move(future)));
} }
coro::Task<void> taskTimedWait() { coro::Task<void> taskTimedWait() {
...@@ -231,7 +233,8 @@ coro::Task<void> taskTimedWait() { ...@@ -231,7 +233,8 @@ coro::Task<void> taskTimedWait() {
TEST(Coro, TimedWait) { TEST(Coro, TimedWait) {
ManualExecutor executor; ManualExecutor executor;
via(&executor, taskTimedWait()).toFuture().getVia(&executor); taskTimedWait().scheduleOn(&executor).start().via(&executor).getVia(
&executor);
} }
template <int value> template <int value>
...@@ -263,7 +266,11 @@ TEST(Coro, AwaitableWithOperator) { ...@@ -263,7 +266,11 @@ TEST(Coro, AwaitableWithOperator) {
ManualExecutor executor; ManualExecutor executor;
EXPECT_EQ( EXPECT_EQ(
42, 42,
via(&executor, taskAwaitableWithOperator()).toFuture().getVia(&executor)); taskAwaitableWithOperator()
.scheduleOn(&executor)
.start()
.via(&executor)
.getVia(&executor));
} }
struct AwaitableWithMemberOperator { struct AwaitableWithMemberOperator {
...@@ -284,8 +291,10 @@ TEST(Coro, AwaitableWithMemberOperator) { ...@@ -284,8 +291,10 @@ TEST(Coro, AwaitableWithMemberOperator) {
ManualExecutor executor; ManualExecutor executor;
EXPECT_EQ( EXPECT_EQ(
42, 42,
via(&executor, taskAwaitableWithMemberOperator()) taskAwaitableWithMemberOperator()
.toFuture() .scheduleOn(&executor)
.start()
.via(&executor)
.getVia(&executor)); .getVia(&executor));
} }
...@@ -297,7 +306,7 @@ coro::Task<int> taskBaton(fibers::Baton& baton) { ...@@ -297,7 +306,7 @@ coro::Task<int> taskBaton(fibers::Baton& baton) {
TEST(Coro, Baton) { TEST(Coro, Baton) {
ManualExecutor executor; ManualExecutor executor;
fibers::Baton baton; fibers::Baton baton;
auto future = via(&executor, taskBaton(baton)); auto future = taskBaton(baton).scheduleOn(&executor).start();
EXPECT_FALSE(future.isReady()); EXPECT_FALSE(future.isReady());
...@@ -309,7 +318,7 @@ TEST(Coro, Baton) { ...@@ -309,7 +318,7 @@ TEST(Coro, Baton) {
executor.run(); executor.run();
EXPECT_TRUE(future.isReady()); EXPECT_TRUE(future.isReady());
EXPECT_EQ(42, coro::blockingWait(std::move(future))); EXPECT_EQ(42, std::move(future).get());
} }
#endif #endif
...@@ -23,9 +23,7 @@ ...@@ -23,9 +23,7 @@
#include <folly/executors/ManualExecutor.h> #include <folly/executors/ManualExecutor.h>
#include <folly/experimental/coro/Baton.h> #include <folly/experimental/coro/Baton.h>
#include <folly/experimental/coro/BlockingWait.h> #include <folly/experimental/coro/BlockingWait.h>
#include <folly/experimental/coro/Future.h>
#include <folly/experimental/coro/Mutex.h> #include <folly/experimental/coro/Mutex.h>
#include <folly/experimental/coro/Promise.h>
#include <folly/experimental/coro/Task.h> #include <folly/experimental/coro/Task.h>
#include <folly/experimental/coro/detail/InlineTask.h> #include <folly/experimental/coro/detail/InlineTask.h>
#include <folly/portability/GTest.h> #include <folly/portability/GTest.h>
...@@ -75,11 +73,11 @@ TEST(Mutex, LockAsync) { ...@@ -75,11 +73,11 @@ TEST(Mutex, LockAsync) {
auto& inlineExecutor = InlineExecutor::instance(); auto& inlineExecutor = InlineExecutor::instance();
auto f1 = makeTask(b1).scheduleVia(&inlineExecutor); auto f1 = makeTask(b1).scheduleOn(&inlineExecutor).start();
CHECK_EQ(1, value); CHECK_EQ(1, value);
CHECK(!m.try_lock()); CHECK(!m.try_lock());
auto f2 = makeTask(b2).scheduleVia(&inlineExecutor); auto f2 = makeTask(b2).scheduleOn(&inlineExecutor).start();
CHECK_EQ(1, value); CHECK_EQ(1, value);
// This will resume f1 coroutine and let it release the // This will resume f1 coroutine and let it release the
...@@ -112,11 +110,11 @@ TEST(Mutex, ScopedLockAsync) { ...@@ -112,11 +110,11 @@ TEST(Mutex, ScopedLockAsync) {
auto& inlineExecutor = InlineExecutor::instance(); auto& inlineExecutor = InlineExecutor::instance();
auto f1 = makeTask(b1).scheduleVia(&inlineExecutor); auto f1 = makeTask(b1).scheduleOn(&inlineExecutor).start();
CHECK_EQ(1, value); CHECK_EQ(1, value);
CHECK(!m.try_lock()); CHECK(!m.try_lock());
auto f2 = makeTask(b2).scheduleVia(&inlineExecutor); auto f2 = makeTask(b2).scheduleOn(&inlineExecutor).start();
CHECK_EQ(1, value); CHECK_EQ(1, value);
// This will resume f1 coroutine and let it release the // This will resume f1 coroutine and let it release the
...@@ -147,13 +145,13 @@ TEST(Mutex, ThreadSafety) { ...@@ -147,13 +145,13 @@ TEST(Mutex, ThreadSafety) {
} }
}; };
auto f1 = makeTask().scheduleVia(&threadPool); auto f1 = makeTask().scheduleOn(&threadPool).start();
auto f2 = makeTask().scheduleVia(&threadPool); auto f2 = makeTask().scheduleOn(&threadPool).start();
auto f3 = makeTask().scheduleVia(&threadPool); auto f3 = makeTask().scheduleOn(&threadPool).start();
coro::blockingWait(std::move(f1)); std::move(f1).get();
coro::blockingWait(std::move(f2)); std::move(f2).get();
coro::blockingWait(std::move(f3)); std::move(f3).get();
CHECK_EQ(30'000, value); CHECK_EQ(30'000, value);
} }
......
...@@ -36,6 +36,7 @@ ...@@ -36,6 +36,7 @@
#include <folly/lang/Exception.h> #include <folly/lang/Exception.h>
#if FOLLY_HAS_COROUTINES #if FOLLY_HAS_COROUTINES
#include <folly/experimental/coro/Traits.h>
#include <experimental/coroutine> #include <experimental/coroutine>
#endif #endif
...@@ -864,13 +865,6 @@ class SemiFuture : private futures::detail::FutureBase<T> { ...@@ -864,13 +865,6 @@ class SemiFuture : private futures::detail::FutureBase<T> {
folly::Promise<T> promise_; folly::Promise<T> promise_;
}; };
template <typename Awaitable>
static SemiFuture fromAwaitable(Awaitable&& awaitable) {
return [](Awaitable awaitable) -> SemiFuture {
co_return co_await std::forward<Awaitable>(awaitable);
}(std::forward<Awaitable>(awaitable));
}
// Customise the co_viaIfAsync() operator so that SemiFuture<T> can be // Customise the co_viaIfAsync() operator so that SemiFuture<T> can be
// directly awaited within a folly::coro::Task coroutine. // directly awaited within a folly::coro::Task coroutine.
friend Future<T> co_viaIfAsync( friend Future<T> co_viaIfAsync(
...@@ -2083,6 +2077,31 @@ inline detail::FutureAwaitable<T> ...@@ -2083,6 +2077,31 @@ inline detail::FutureAwaitable<T>
/* implicit */ operator co_await(Future<T>&& future) noexcept { /* implicit */ operator co_await(Future<T>&& future) noexcept {
return detail::FutureAwaitable<T>(std::move(future)); return detail::FutureAwaitable<T>(std::move(future));
} }
namespace coro {
/// Convert an awaitable type into a SemiFuture that can then be consumed by
/// APIs that use folly::Future/SemiFuture.
///
/// This will eagerly start execution of 'co_await awaitable' and will make
/// the eventual result available via the returned SemiFuture.
template <typename Awaitable>
inline auto toSemiFuture(Awaitable awaitable) -> std::enable_if_t<
!std::is_void<folly::coro::await_result_t<Awaitable>>::value,
SemiFuture<await_result_t<Awaitable>>> {
co_return co_await static_cast<Awaitable&&>(awaitable);
}
template <typename Awaitable>
inline auto toSemiFuture(Awaitable awaitable) -> std::enable_if_t<
std::is_void<folly::coro::await_result_t<Awaitable>>::value,
SemiFuture<Unit>> {
co_await static_cast<Awaitable&&>(awaitable);
co_return Unit{};
}
} // namespace coro
} // namespace folly } // namespace folly
#endif #endif
......
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