Commit e19b938b authored by Lewis Baker's avatar Lewis Baker Committed by Facebook GitHub Bot

Make collectAll, collectAllRange, collectAllWindowed consistently rethrow...

Make collectAll, collectAllRange, collectAllWindowed consistently rethrow exception from first failure

Summary:
The void-returning variants of collectAllRange() and collectAllWindowed() were
previously failing with the error from the first task to fail whereas the other
variants of collectAll() were failing with an arbitrary error (typically first
by argument order) in the case of multiple errors.

This was problematic because after the first task failed it requests cancellation
of the other tasks, which then often will complete with an 'OperationCancelled'
error. This mean that, unless the original error occurred on the first failed
task in argument order, the error that caused the cancellation would be discarded
and instead, the collectAll() operation would result in an 'OperationCancelled'
error.

This change updates those algorithms to now consistently fail with the error
from the first task to fail (in time) as this error will typically be the most
useful in determining the root cause of the failure.

Also it now discards any errors that occur in child tasks if they fail after
cancellation was requested on the parent task. In this case the collectAll()
will complete with an 'OperationCancelled' error.

Also made a few drive-by changes to avoid rethrowing exceptions where possible
by using `co_yield co_error(e)`.

Reviewed By: yfeldblum

Differential Revision: D21681812

fbshipit-source-id: dfe15d0672dcbdeb4e156962cdffe767ddf91a0b
parent 70f914ba
......@@ -26,40 +26,38 @@ namespace detail {
template <typename T>
T&& getValueOrUnit(Try<T>&& value) {
assert(value.hasValue());
return std::move(value).value();
}
inline Unit getValueOrUnit(Try<void>&& value) {
value.throwIfFailed();
assert(value.hasValue());
return Unit{};
}
template <typename SemiAwaitable, typename Result>
detail::BarrierTask makeCollectAllTask(
folly::Executor* executor,
SemiAwaitable&& awaitable,
BarrierTask makeCollectAllTryTask(
Executor::KeepAlive<> executor,
const CancellationToken& cancelToken,
const CancellationSource& cancelSource,
SemiAwaitable&& awaitable,
Try<Result>& result) {
try {
if constexpr (std::is_void_v<Result>) {
co_await co_viaIfAsync(
executor,
std::move(executor),
co_withCancellation(
cancelToken, static_cast<SemiAwaitable&&>(awaitable)));
result.emplace();
} else {
result.emplace(co_await co_viaIfAsync(
executor,
std::move(executor),
co_withCancellation(
cancelToken, static_cast<SemiAwaitable&&>(awaitable))));
}
} catch (const std::exception& ex) {
result.emplaceException(std::current_exception(), ex);
cancelSource.requestCancellation();
} catch (...) {
result.emplaceException(std::current_exception());
cancelSource.requestCancellation();
}
}
......@@ -73,19 +71,17 @@ auto collectAllTryImpl(
if constexpr (sizeof...(SemiAwaitables) == 0) {
co_return std::tuple<>{};
} else {
std::tuple<collect_all_try_component_t<SemiAwaitables>...> results;
Executor* executor = co_await co_current_executor;
const Executor::KeepAlive<> executor = co_await co_current_executor;
const CancellationToken& cancelToken =
co_await co_current_cancellation_token;
const CancellationSource cancelSource = CancellationSource::invalid();
std::tuple<collect_all_try_component_t<SemiAwaitables>...> results;
folly::coro::detail::BarrierTask tasks[sizeof...(SemiAwaitables)] = {
makeCollectAllTask(
executor,
static_cast<SemiAwaitables&&>(awaitables),
makeCollectAllTryTask(
executor.get_alias(),
cancelToken,
cancelSource,
static_cast<SemiAwaitables&&>(awaitables),
std::get<Indices>(results))...,
};
......@@ -108,7 +104,7 @@ auto collectAllTryImpl(
// Should be safe to avoid an executor transition here even if the
// operation completes asynchronously since all of the child tasks
// should already have transitioned to the correct executor due to
// the use of co_viaIfAsync() within makeBarrierTask().
// the use of co_viaIfAsync() within makeCollectAllTryTask().
co_await UnsafeResumeInlineSemiAwaitable{barrier.arriveAndWait()};
co_return results;
......@@ -124,23 +120,54 @@ auto collectAllImpl(
if constexpr (sizeof...(SemiAwaitables) == 0) {
co_return std::tuple<>{};
} else {
std::tuple<collect_all_try_component_t<SemiAwaitables>...> results;
Executor* executor = co_await co_current_executor;
const Executor::KeepAlive<> executor = co_await co_current_executor;
const CancellationToken& parentCancelToken =
co_await co_current_cancellation_token;
const CancellationSource cancelSource;
CancellationCallback cancelCallback(
co_await co_current_cancellation_token, [&]() noexcept {
CancellationCallback cancelCallback(parentCancelToken, [&]() noexcept {
cancelSource.requestCancellation();
});
const CancellationToken cancelToken = cancelSource.getToken();
exception_wrapper firstException;
std::atomic<bool> anyFailures{false};
auto makeTask = [&](auto&& awaitable, auto& result) -> BarrierTask {
using await_result = semi_await_result_t<decltype(awaitable)>;
try {
if constexpr (std::is_void_v<await_result>) {
co_await co_viaIfAsync(
executor.get_alias(),
co_withCancellation(
cancelToken, static_cast<decltype(awaitable)>(awaitable)));
result.emplace();
} else {
result.emplace(co_await co_viaIfAsync(
executor.get_alias(),
co_withCancellation(
cancelToken, static_cast<decltype(awaitable)>(awaitable))));
}
} catch (const std::exception& ex) {
anyFailures.store(true, std::memory_order_relaxed);
if (!cancelSource.requestCancellation()) {
// This was the first failure, remember it's error.
firstException = exception_wrapper{std::current_exception(), ex};
}
} catch (...) {
anyFailures.store(true, std::memory_order_relaxed);
if (!cancelSource.requestCancellation()) {
// This was the first failure, remember it's error.
firstException = exception_wrapper{std::current_exception()};
}
}
};
std::tuple<collect_all_try_component_t<SemiAwaitables>...> results;
folly::coro::detail::BarrierTask tasks[sizeof...(SemiAwaitables)] = {
makeCollectAllTask(
executor,
makeTask(
static_cast<SemiAwaitables&&>(awaitables),
cancelToken,
cancelSource,
std::get<Indices>(results))...,
};
......@@ -166,33 +193,233 @@ auto collectAllImpl(
// the use of co_viaIfAsync() within makeBarrierTask().
co_await UnsafeResumeInlineSemiAwaitable{barrier.arriveAndWait()};
if (anyFailures.load(std::memory_order_relaxed)) {
if (firstException) {
co_yield co_error(std::move(firstException));
}
// Parent task was cancelled before any child tasks failed.
// Complete with the OperationCancelled error instead of the
// child task's errors.
co_yield co_error(OperationCancelled{});
}
co_return std::tuple<collect_all_component_t<SemiAwaitables>...>{
getValueOrUnit(std::get<Indices>(std::move(results)))...};
}
}
} // namespace detail
template <typename... SemiAwaitables>
auto collectAll(SemiAwaitables&&... awaitables) -> folly::coro::Task<std::tuple<
detail::collect_all_component_t<remove_cvref_t<SemiAwaitables>>...>> {
return detail::collectAllImpl(
std::make_index_sequence<sizeof...(SemiAwaitables)>{},
static_cast<SemiAwaitables&&>(awaitables)...);
}
template <typename... SemiAwaitables>
auto collectAllTry(SemiAwaitables&&... awaitables)
-> folly::coro::Task<std::tuple<detail::collect_all_try_component_t<
remove_cvref_t<SemiAwaitables>>...>> {
return detail::collectAllTryImpl(
std::make_index_sequence<sizeof...(SemiAwaitables)>{},
static_cast<SemiAwaitables&&>(awaitables)...);
}
template <
typename InputRange,
std::enable_if_t<
!std::is_void_v<
semi_await_result_t<detail::range_reference_t<InputRange>>>,
int>>
auto collectAllRange(InputRange awaitables)
-> folly::coro::Task<std::vector<detail::collect_all_range_component_t<
detail::range_reference_t<InputRange>>>> {
const folly::Executor::KeepAlive<> executor = co_await co_current_executor;
const CancellationSource cancelSource;
CancellationCallback cancelCallback(
co_await co_current_cancellation_token, [&]() noexcept {
cancelSource.requestCancellation();
});
const CancellationToken cancelToken = cancelSource.getToken();
std::vector<detail::collect_all_try_range_component_t<
detail::range_reference_t<InputRange>>>
tryResults;
exception_wrapper firstException;
std::atomic<bool> anyFailures = false;
using awaitable_type = remove_cvref_t<detail::range_reference_t<InputRange>>;
auto makeTask = [&](awaitable_type semiAwaitable,
std::size_t index) -> detail::BarrierTask {
assert(index < tryResults.size());
try {
tryResults[index].emplace(co_await co_viaIfAsync(
executor.get_alias(),
co_withCancellation(cancelToken, std::move(semiAwaitable))));
} catch (const std::exception& ex) {
anyFailures.store(true, std::memory_order_relaxed);
if (!cancelSource.requestCancellation()) {
firstException = exception_wrapper{std::current_exception(), ex};
}
} catch (...) {
anyFailures.store(true, std::memory_order_relaxed);
if (!cancelSource.requestCancellation()) {
firstException = exception_wrapper{std::current_exception()};
}
}
};
// Create a task to await each input awaitable.
std::vector<detail::BarrierTask> tasks;
// TODO: Detect when the input range supports constant-time
// .size() and pre-reserve storage for that many elements in 'tasks'.
std::size_t taskCount = 0;
for (auto&& semiAwaitable : static_cast<InputRange&&>(awaitables)) {
tasks.push_back(makeTask(
static_cast<decltype(semiAwaitable)&&>(semiAwaitable), taskCount++));
}
tryResults.resize(taskCount);
// Save the initial context and restore it after starting each task
// as the task may have modified the context before suspending and we
// want to make sure the next task is started with the same initial
// context.
const auto context = RequestContext::saveContext();
// Launch the tasks and wait for them all to finish.
{
detail::Barrier barrier{tasks.size() + 1};
for (auto&& task : tasks) {
task.start(&barrier);
RequestContext::setContext(context);
}
co_await detail::UnsafeResumeInlineSemiAwaitable{barrier.arriveAndWait()};
}
// Check if there were any exceptions and rethrow the first one.
if (anyFailures.load(std::memory_order_relaxed)) {
if (firstException) {
co_yield co_error(std::move(firstException));
}
// Cancellation was requested of the parent Task before any of the
// child tasks failed.
co_yield co_error(OperationCancelled{});
}
std::vector<detail::collect_all_range_component_t<
detail::range_reference_t<InputRange>>>
results;
results.reserve(tryResults.size());
for (auto& result : tryResults) {
results.emplace_back(std::move(result).value());
}
co_return results;
}
template <
typename InputRange,
std::enable_if_t<
std::is_void_v<
semi_await_result_t<detail::range_reference_t<InputRange>>>,
int>>
auto collectAllRange(InputRange awaitables) -> folly::coro::Task<void> {
const folly::Executor::KeepAlive<> executor = co_await co_current_executor;
CancellationSource cancelSource;
CancellationCallback cancelCallback(
co_await co_current_cancellation_token, [&]() noexcept {
cancelSource.requestCancellation();
});
const CancellationToken cancelToken = cancelSource.getToken();
exception_wrapper firstException;
std::atomic<bool> anyFailures = false;
using awaitable_type = remove_cvref_t<detail::range_reference_t<InputRange>>;
auto makeTask = [&](awaitable_type semiAwaitable) -> detail::BarrierTask {
try {
co_await co_viaIfAsync(
executor.get_alias(),
co_withCancellation(cancelToken, std::move(semiAwaitable)));
} catch (const std::exception& ex) {
anyFailures.store(true, std::memory_order_relaxed);
if (!cancelSource.requestCancellation()) {
firstException = exception_wrapper{std::current_exception(), ex};
}
} catch (...) {
anyFailures.store(true, std::memory_order_relaxed);
if (!cancelSource.requestCancellation()) {
firstException = exception_wrapper{std::current_exception()};
}
}
};
// Create a task to await each input awaitable.
std::vector<detail::BarrierTask> tasks;
// TODO: Detect when the input range supports constant-time
// .size() and pre-reserve storage for that many elements in 'tasks'.
for (auto&& semiAwaitable : static_cast<InputRange&&>(awaitables)) {
tasks.push_back(
makeTask(static_cast<decltype(semiAwaitable)&&>(semiAwaitable)));
}
// Save the initial context and restore it after starting each task
// as the task may have modified the context before suspending and we
// want to make sure the next task is started with the same initial
// context.
const auto context = RequestContext::saveContext();
// Launch the tasks and wait for them all to finish.
{
detail::Barrier barrier{tasks.size() + 1};
for (auto&& task : tasks) {
task.start(&barrier);
RequestContext::setContext(context);
}
co_await detail::UnsafeResumeInlineSemiAwaitable{barrier.arriveAndWait()};
}
// Check if there were any exceptions and rethrow the first one.
if (anyFailures.load(std::memory_order_relaxed)) {
if (firstException) {
co_yield co_error(std::move(firstException));
}
}
}
template <typename InputRange>
auto collectAllTryRangeImpl(
InputRange awaitables,
CancellationToken cancelToken,
CancellationSource cancelSource)
auto collectAllTryRange(InputRange awaitables)
-> folly::coro::Task<std::vector<detail::collect_all_try_range_component_t<
detail::range_reference_t<InputRange>>>> {
std::vector<detail::collect_all_try_range_component_t<
detail::range_reference_t<InputRange>>>
results;
folly::Executor::KeepAlive<> executor =
const folly::Executor::KeepAlive<> executor =
folly::getKeepAliveToken(co_await co_current_executor);
const CancellationToken& cancelToken = co_await co_current_cancellation_token;
using awaitable_type = remove_cvref_t<detail::range_reference_t<InputRange>>;
auto makeTask = [&](std::size_t index,
awaitable_type semiAwaitable) -> detail::BarrierTask {
assert(index < results.size());
auto& result = results[index];
try {
using await_result =
semi_await_result_t<detail::range_reference_t<InputRange>>;
using await_result = semi_await_result_t<awaitable_type>;
if constexpr (std::is_void_v<await_result>) {
co_await co_viaIfAsync(
executor.get_alias(),
......@@ -205,10 +432,8 @@ auto collectAllTryRangeImpl(
}
} catch (const std::exception& ex) {
result.emplaceException(std::current_exception(), ex);
cancelSource.requestCancellation();
} catch (...) {
result.emplaceException(std::current_exception());
cancelSource.requestCancellation();
}
};
......@@ -250,26 +475,34 @@ auto collectAllTryRangeImpl(
co_return results;
}
template <typename InputRange>
auto collectAllTryWindowedImpl(
InputRange awaitables,
std::size_t maxConcurrency,
CancellationToken cancelToken,
CancellationSource cancelSource)
-> folly::coro::Task<std::vector<detail::collect_all_try_range_component_t<
detail::range_reference_t<InputRange>>>> {
template <
typename InputRange,
std::enable_if_t<
std::is_void_v<
semi_await_result_t<detail::range_reference_t<InputRange>>>,
int>>
auto collectAllWindowed(InputRange awaitables, std::size_t maxConcurrency)
-> folly::coro::Task<void> {
assert(maxConcurrency > 0);
std::vector<detail::collect_all_try_range_component_t<
detail::range_reference_t<InputRange>>>
results;
exception_wrapper iterationException;
const folly::Executor::KeepAlive<> executor = co_await co_current_executor;
const folly::CancellationSource cancelSource;
folly::CancellationCallback cancelCallback(
co_await folly::coro::co_current_cancellation_token, [&]() noexcept {
cancelSource.requestCancellation();
});
const folly::CancellationToken cancelToken = cancelSource.getToken();
folly::coro::Mutex mutex;
exception_wrapper firstException;
std::atomic<bool> anyFailures = false;
folly::Executor::KeepAlive<> executor =
folly::getKeepAliveToken(co_await co_current_executor);
const auto trySetFirstException = [&](exception_wrapper && e) noexcept {
anyFailures.store(true, std::memory_order_relaxed);
if (!cancelSource.requestCancellation()) {
// This is first entity to request cancellation.
firstException = std::move(e);
}
};
using std::begin;
using std::end;
......@@ -278,17 +511,19 @@ auto collectAllTryWindowedImpl(
using iterator_t = decltype(iter);
using awaitable_t = typename std::iterator_traits<iterator_t>::value_type;
using result_t = semi_await_result_t<awaitable_t>;
folly::coro::Mutex mutex;
exception_wrapper iterationException;
auto makeWorker = [&]() -> detail::BarrierTask {
auto lock =
co_await co_viaIfAsync(executor.get_alias(), mutex.co_scoped_lock());
while (!iterationException && iter != iterEnd) {
std::optional<awaitable_t> awaitable;
try {
awaitable_t awaitable = *iter;
try {
awaitable.emplace(*iter);
++iter;
} catch (const std::exception& ex) {
iterationException = exception_wrapper{std::current_exception(), ex};
......@@ -298,84 +533,24 @@ auto collectAllTryWindowedImpl(
cancelSource.requestCancellation();
}
const auto thisIndex = results.size();
try {
results.emplace_back();
} catch (const std::exception& ex) {
if (!iterationException) {
iterationException =
exception_wrapper{std::current_exception(), ex};
cancelSource.requestCancellation();
}
// Failure to grow the results vector is fatal.
co_return;
} catch (...) {
if (!iterationException) {
iterationException = exception_wrapper{std::current_exception()};
cancelSource.requestCancellation();
}
// Failure to grow the results vector is fatal.
co_return;
if (!awaitable) {
co_return;
}
lock.unlock();
detail::collect_all_try_range_component_t<
detail::range_reference_t<InputRange>>
result;
try {
if constexpr (std::is_void_v<result_t>) {
co_await co_viaIfAsync(
executor.get_alias(),
co_withCancellation(
cancelToken, static_cast<awaitable_t&&>(awaitable)));
result.emplace();
} else {
result.emplace(co_await co_viaIfAsync(
executor.get_alias(),
co_withCancellation(
cancelToken, static_cast<awaitable_t&&>(awaitable))));
}
co_withCancellation(cancelToken, std::move(*awaitable)));
} catch (const std::exception& ex) {
result.emplaceException(std::current_exception(), ex);
cancelSource.requestCancellation();
} catch (...) {
result.emplaceException(std::current_exception());
cancelSource.requestCancellation();
}
lock = co_await co_viaIfAsync(
executor.get_alias(), mutex.co_scoped_lock());
try {
results[thisIndex] = std::move(result);
} catch (const std::exception& ex) {
results[thisIndex].emplaceException(std::current_exception(), ex);
cancelSource.requestCancellation();
} catch (...) {
results[thisIndex].emplaceException(std::current_exception());
cancelSource.requestCancellation();
}
} catch (const std::exception& ex) {
assert(lock.owns_lock());
if (!iterationException) {
iterationException = exception_wrapper{std::current_exception(), ex};
cancelSource.requestCancellation();
}
co_return;
trySetFirstException(exception_wrapper{std::current_exception(), ex});
} catch (...) {
assert(lock.owns_lock());
if (!iterationException) {
iterationException = exception_wrapper{std::current_exception()};
cancelSource.requestCancellation();
}
co_return;
trySetFirstException(exception_wrapper{std::current_exception()});
}
assert(lock.owns_lock());
lock =
co_await co_viaIfAsync(executor.get_alias(), mutex.co_scoped_lock());
}
};
......@@ -383,8 +558,6 @@ auto collectAllTryWindowedImpl(
detail::Barrier barrier{1};
exception_wrapper workerCreationException;
// Save the initial context and restore it after starting each task
// as the task may have modified the context before suspending and we
// want to make sure the next task is started with the same initial
......@@ -393,10 +566,12 @@ auto collectAllTryWindowedImpl(
try {
auto lock = co_await mutex.co_scoped_lock();
while (iter != iterEnd && workerTasks.size() < maxConcurrency) {
// Unlock the mutex before starting the child operation so that
while (!iterationException && iter != iterEnd &&
workerTasks.size() < maxConcurrency) {
// Unlock the mutex before starting the worker so that
// it can consume as many results synchronously as it can before
// returning here and letting us potentially spawn another task.
// returning here and letting us spawn another task.
// This can avoid spawning more worker coroutines than is necessary
// to consume all of the awaitables.
lock.unlock();
......@@ -410,44 +585,27 @@ auto collectAllTryWindowedImpl(
lock = co_await mutex.co_scoped_lock();
}
} catch (const std::exception& ex) {
workerCreationException = exception_wrapper{std::current_exception(), ex};
cancelSource.requestCancellation();
// Only a fatal error if we failed to create any worker tasks.
if (workerTasks.empty()) {
iterationException = exception_wrapper{std::current_exception(), ex};
}
} catch (...) {
workerCreationException = exception_wrapper{std::current_exception()};
cancelSource.requestCancellation();
if (workerTasks.empty()) {
iterationException = exception_wrapper{std::current_exception()};
}
}
co_await detail::UnsafeResumeInlineSemiAwaitable{barrier.arriveAndWait()};
if (iterationException) {
co_return folly::Try<decltype(results)>{std::move(iterationException)};
} else if (workerTasks.empty() && workerCreationException) {
// Couldn't spawn any child workers to execute the tasks.
// TODO: We could update this code-path to try to execute them serially
// here but the system is probably in a bad state anyway.
co_return folly::Try<decltype(results)>{std::move(workerCreationException)};
co_yield co_error(std::move(iterationException));
} else if (anyFailures.load(std::memory_order_relaxed)) {
if (firstException) {
co_yield co_error(std::move(firstException));
}
co_return results;
}
} // namespace detail
template <typename... SemiAwaitables>
auto collectAll(SemiAwaitables&&... awaitables) -> folly::coro::Task<std::tuple<
detail::collect_all_component_t<remove_cvref_t<SemiAwaitables>>...>> {
return detail::collectAllImpl(
std::make_index_sequence<sizeof...(SemiAwaitables)>{},
static_cast<SemiAwaitables&&>(awaitables)...);
}
template <typename... SemiAwaitables>
auto collectAllTry(SemiAwaitables&&... awaitables)
-> folly::coro::Task<std::tuple<detail::collect_all_try_component_t<
remove_cvref_t<SemiAwaitables>>...>> {
return detail::collectAllTryImpl(
std::make_index_sequence<sizeof...(SemiAwaitables)>{},
static_cast<SemiAwaitables&&>(awaitables)...);
co_yield co_error(OperationCancelled{});
}
}
template <
......@@ -456,88 +614,105 @@ template <
!std::is_void_v<
semi_await_result_t<detail::range_reference_t<InputRange>>>,
int>>
auto collectAllRange(InputRange awaitables)
auto collectAllWindowed(InputRange awaitables, std::size_t maxConcurrency)
-> folly::coro::Task<std::vector<detail::collect_all_range_component_t<
detail::range_reference_t<InputRange>>>> {
const CancellationSource cancelSource;
CancellationCallback cancelCallback(
co_await co_current_cancellation_token, [&]() noexcept {
assert(maxConcurrency > 0);
const folly::Executor::KeepAlive<> executor = co_await co_current_executor;
const folly::CancellationSource cancelSource;
folly::CancellationCallback cancelCallback(
co_await folly::coro::co_current_cancellation_token, [&]() noexcept {
cancelSource.requestCancellation();
});
const folly::CancellationToken cancelToken = cancelSource.getToken();
auto resultsTry =
co_await folly::coro::co_awaitTry(detail::collectAllTryRangeImpl(
std::move(awaitables), cancelSource.getToken(), cancelSource));
exception_wrapper firstException;
std::atomic<bool> anyFailures = false;
if (resultsTry.hasException()) {
co_return Try<std::vector<detail::collect_all_range_component_t<
detail::range_reference_t<InputRange>>>>{
std::move(resultsTry.exception())};
auto trySetFirstException = [&](exception_wrapper && e) noexcept {
anyFailures.store(true, std::memory_order_relaxed);
if (!cancelSource.requestCancellation()) {
// This is first entity to request cancellation.
firstException = std::move(e);
}
};
using std::begin;
using std::end;
auto iter = begin(awaitables);
const auto iterEnd = end(awaitables);
auto& results = resultsTry.value();
using iterator_t = decltype(iter);
using awaitable_t = typename std::iterator_traits<iterator_t>::value_type;
// Collate the results into a single result vector.
std::vector<detail::collect_all_range_component_t<
folly::coro::Mutex mutex;
std::vector<detail::collect_all_try_range_component_t<
detail::range_reference_t<InputRange>>>
values;
values.reserve(results.size());
for (auto&& result : results) {
values.push_back(std::move(result).value());
}
tryResults;
co_return values;
}
exception_wrapper iterationException;
template <
typename InputRange,
std::enable_if_t<
std::is_void_v<
semi_await_result_t<detail::range_reference_t<InputRange>>>,
int>>
auto collectAllRange(InputRange awaitables) -> folly::coro::Task<void> {
folly::Executor::KeepAlive<> executor =
folly::getKeepAliveToken(co_await co_current_executor);
auto makeWorker = [&]() -> detail::BarrierTask {
auto lock =
co_await co_viaIfAsync(executor.get_alias(), mutex.co_scoped_lock());
const CancellationSource cancelSource;
CancellationCallback cancelCallback(
co_await co_current_cancellation_token, [&]() noexcept {
while (!iterationException && iter != iterEnd) {
const std::size_t thisIndex = tryResults.size();
std::optional<awaitable_t> awaitable;
try {
tryResults.emplace_back();
awaitable.emplace(*iter);
++iter;
} catch (const std::exception& ex) {
iterationException = exception_wrapper{std::current_exception(), ex};
cancelSource.requestCancellation();
});
const CancellationToken cancelToken = cancelSource.getToken();
} catch (...) {
iterationException = exception_wrapper{std::current_exception()};
cancelSource.requestCancellation();
}
std::atomic<bool> anyFailures{false};
exception_wrapper firstException;
if (!awaitable) {
co_return;
}
lock.unlock();
detail::collect_all_try_range_component_t<
detail::range_reference_t<InputRange>>
tryResult;
using awaitable_type = remove_cvref_t<detail::range_reference_t<InputRange>>;
auto makeTask = [&](awaitable_type semiAwaitable) -> detail::BarrierTask {
try {
co_await co_viaIfAsync(
tryResult.emplace(co_await co_viaIfAsync(
executor.get_alias(),
co_withCancellation(cancelToken, std::move(semiAwaitable)));
co_withCancellation(
cancelToken, static_cast<awaitable_t&&>(*awaitable))));
} catch (const std::exception& ex) {
if (!anyFailures.exchange(true, std::memory_order_relaxed)) {
firstException = exception_wrapper{std::current_exception(), ex};
cancelSource.requestCancellation();
trySetFirstException(exception_wrapper{std::current_exception(), ex});
} catch (...) {
trySetFirstException(exception_wrapper{std::current_exception()});
}
lock =
co_await co_viaIfAsync(executor.get_alias(), mutex.co_scoped_lock());
try {
tryResults[thisIndex] = std::move(tryResult);
} catch (const std::exception& ex) {
trySetFirstException(exception_wrapper{std::current_exception(), ex});
} catch (...) {
if (!anyFailures.exchange(true, std::memory_order_relaxed)) {
firstException = exception_wrapper{std::current_exception()};
cancelSource.requestCancellation();
trySetFirstException(exception_wrapper{std::current_exception()});
}
}
};
// Create a task to await each input awaitable.
std::vector<detail::BarrierTask> tasks;
std::vector<detail::BarrierTask> workerTasks;
// TODO: Detect when the input range supports constant-time
// .size() and pre-reserve storage for that many elements in 'tasks'.
detail::Barrier barrier{1};
for (auto&& semiAwaitable : static_cast<InputRange&&>(awaitables)) {
tasks.push_back(
makeTask(static_cast<decltype(semiAwaitable)&&>(semiAwaitable)));
}
exception_wrapper workerCreationException;
// Save the initial context and restore it after starting each task
// as the task may have modified the context before suspending and we
......@@ -545,55 +720,81 @@ auto collectAllRange(InputRange awaitables) -> folly::coro::Task<void> {
// context.
const auto context = RequestContext::saveContext();
// Launch the tasks and wait for them all to finish.
{
detail::Barrier barrier{tasks.size() + 1};
for (auto&& task : tasks) {
task.start(&barrier);
try {
auto lock = co_await mutex.co_scoped_lock();
while (!iterationException && iter != iterEnd &&
workerTasks.size() < maxConcurrency) {
// Unlock the mutex before starting the worker so that
// it can consume as many results synchronously as it can before
// returning here and letting us spawn another task.
// This can avoid spawning more worker coroutines than is necessary
// to consume all of the awaitables.
lock.unlock();
workerTasks.push_back(makeWorker());
barrier.add(1);
workerTasks.back().start(&barrier);
RequestContext::setContext(context);
lock = co_await mutex.co_scoped_lock();
}
} catch (const std::exception& ex) {
// Only a fatal error if we failed to create any worker tasks.
if (workerTasks.empty()) {
// No need to synchronise here. There are no concurrent tasks running.
iterationException = exception_wrapper{std::current_exception(), ex};
}
} catch (...) {
if (workerTasks.empty()) {
iterationException = exception_wrapper{std::current_exception()};
}
}
co_await detail::UnsafeResumeInlineSemiAwaitable{barrier.arriveAndWait()};
if (iterationException) {
co_yield co_error(std::move(iterationException));
} else if (anyFailures.load(std::memory_order_relaxed)) {
if (firstException) {
co_yield co_error(std::move(firstException));
}
// Check if there were any exceptions and rethrow the first one.
if (anyFailures.load(std::memory_order_relaxed)) {
firstException.throw_exception();
// Otherwise, cancellation was requested before any of the child tasks
// failed so complete with the OperationCancelled error.
co_yield co_error(OperationCancelled{});
}
std::vector<detail::collect_all_range_component_t<
detail::range_reference_t<InputRange>>>
results;
results.reserve(tryResults.size());
for (auto&& tryResult : tryResults) {
assert(tryResult.hasValue());
results.emplace_back(std::move(tryResult).value());
}
co_return results;
}
template <typename InputRange>
auto collectAllTryRange(InputRange awaitables)
auto collectAllTryWindowed(InputRange awaitables, std::size_t maxConcurrency)
-> folly::coro::Task<std::vector<detail::collect_all_try_range_component_t<
detail::range_reference_t<InputRange>>>> {
co_return co_await detail::collectAllTryRangeImpl(
std::move(awaitables),
co_await co_current_cancellation_token,
CancellationSource::invalid());
}
template <
typename InputRange,
std::enable_if_t<
std::is_void_v<
semi_await_result_t<detail::range_reference_t<InputRange>>>,
int>>
auto collectAllWindowed(InputRange awaitables, std::size_t maxConcurrency)
-> folly::coro::Task<void> {
assert(maxConcurrency > 0);
exception_wrapper firstException;
std::vector<detail::collect_all_try_range_component_t<
detail::range_reference_t<InputRange>>>
results;
folly::coro::Mutex mutex;
exception_wrapper iterationException;
folly::Executor::KeepAlive<> executor =
folly::getKeepAliveToken(co_await co_current_executor);
folly::coro::Mutex mutex;
const folly::CancellationSource cancelSource;
folly::CancellationCallback cancelCallback(
co_await co_current_cancellation_token, [&]() noexcept {
cancelSource.requestCancellation();
});
const folly::CancellationToken cancelToken = cancelSource.getToken();
const Executor::KeepAlive<> executor = co_await co_current_executor;
const CancellationToken& cancelToken = co_await co_current_cancellation_token;
using std::begin;
using std::end;
......@@ -602,47 +803,62 @@ auto collectAllWindowed(InputRange awaitables, std::size_t maxConcurrency)
using iterator_t = decltype(iter);
using awaitable_t = typename std::iterator_traits<iterator_t>::value_type;
using result_t = semi_await_result_t<awaitable_t>;
auto makeWorker = [&]() -> detail::BarrierTask {
auto lock =
co_await co_viaIfAsync(executor.get_alias(), mutex.co_scoped_lock());
while (iter != iterEnd) {
awaitable_t awaitable = *iter;
while (!iterationException && iter != iterEnd) {
const std::size_t thisIndex = results.size();
std::optional<awaitable_t> awaitable;
try {
results.emplace_back();
awaitable.emplace(*iter);
++iter;
} catch (const std::exception& ex) {
if (!firstException) {
firstException = exception_wrapper{std::current_exception(), ex};
cancelSource.requestCancellation();
}
iterationException = exception_wrapper{std::current_exception(), ex};
} catch (...) {
if (!firstException) {
firstException = exception_wrapper{std::current_exception()};
cancelSource.requestCancellation();
iterationException = exception_wrapper{std::current_exception()};
}
if (!awaitable) {
co_return;
}
lock.unlock();
exception_wrapper ex;
detail::collect_all_try_range_component_t<
detail::range_reference_t<InputRange>>
result;
try {
if constexpr (std::is_void_v<result_t>) {
co_await co_viaIfAsync(
executor.get_alias(),
co_withCancellation(
cancelToken, static_cast<awaitable_t&&>(awaitable)));
} catch (const std::exception& e) {
ex = exception_wrapper{std::current_exception(), e};
co_withCancellation(cancelToken, std::move(*awaitable)));
result.emplace();
} else {
result.emplace(co_await co_viaIfAsync(
executor.get_alias(),
co_withCancellation(cancelToken, std::move(*awaitable))));
}
} catch (const std::exception& ex) {
result.emplaceException(std::current_exception(), ex);
} catch (...) {
ex = exception_wrapper{std::current_exception()};
result.emplaceException(std::current_exception());
}
lock =
co_await co_viaIfAsync(executor.get_alias(), mutex.co_scoped_lock());
if (ex && !firstException) {
firstException = std::move(ex);
cancelSource.requestCancellation();
try {
results[thisIndex] = std::move(result);
} catch (const std::exception& ex) {
results[thisIndex].emplaceException(std::current_exception(), ex);
} catch (...) {
results[thisIndex].emplaceException(std::current_exception());
}
}
};
......@@ -651,8 +867,6 @@ auto collectAllWindowed(InputRange awaitables, std::size_t maxConcurrency)
detail::Barrier barrier{1};
exception_wrapper workerCreationException;
// Save the initial context and restore it after starting each task
// as the task may have modified the context before suspending and we
// want to make sure the next task is started with the same initial
......@@ -661,11 +875,11 @@ auto collectAllWindowed(InputRange awaitables, std::size_t maxConcurrency)
try {
auto lock = co_await mutex.co_scoped_lock();
while (iter != iterEnd && workerTasks.size() < maxConcurrency) {
// Unlock the mutex before starting the worker so that
while (!iterationException && iter != iterEnd &&
workerTasks.size() < maxConcurrency) {
// Unlock the mutex before starting the child operation so that
// it can consume as many results synchronously as it can before
// returning here and letting us spawn another task.
// returning here and letting us potentially spawn another task.
// This can avoid spawning more worker coroutines than is necessary
// to consume all of the awaitables.
lock.unlock();
......@@ -679,74 +893,26 @@ auto collectAllWindowed(InputRange awaitables, std::size_t maxConcurrency)
lock = co_await mutex.co_scoped_lock();
}
} catch (const std::exception& ex) {
workerCreationException = exception_wrapper{std::current_exception(), ex};
cancelSource.requestCancellation();
// Failure to create a worker is an error if we failed
// to create _any_ workers. As long as we created one then
// the algorithm should still be able to make forward progress.
if (workerTasks.empty()) {
iterationException = exception_wrapper{std::current_exception(), ex};
}
} catch (...) {
workerCreationException = exception_wrapper{std::current_exception()};
cancelSource.requestCancellation();
if (workerTasks.empty()) {
iterationException = exception_wrapper{std::current_exception()};
}
co_await detail::UnsafeResumeInlineSemiAwaitable{barrier.arriveAndWait()};
if (firstException) {
firstException.throw_exception();
} else if (workerTasks.empty() && workerCreationException) {
// Failed to create any workers to process the tasks.
workerCreationException.throw_exception();
}
}
template <
typename InputRange,
std::enable_if_t<
!std::is_void_v<
semi_await_result_t<detail::range_reference_t<InputRange>>>,
int>>
auto collectAllWindowed(InputRange awaitables, std::size_t maxConcurrency)
-> folly::coro::Task<std::vector<detail::collect_all_range_component_t<
detail::range_reference_t<InputRange>>>> {
assert(maxConcurrency > 0);
const folly::CancellationSource cancelSource;
folly::CancellationCallback cancelCallback(
co_await co_current_cancellation_token, [&]() noexcept {
cancelSource.requestCancellation();
});
const folly::CancellationToken cancelToken = cancelSource.getToken();
auto tryResults = co_await detail::collectAllTryWindowedImpl(
std::move(awaitables),
maxConcurrency,
cancelSource.getToken(),
cancelSource);
std::vector<detail::collect_all_range_component_t<
detail::range_reference_t<InputRange>>>
results;
results.reserve(tryResults.size());
co_await detail::UnsafeResumeInlineSemiAwaitable{barrier.arriveAndWait()};
for (auto&& tryResult : tryResults) {
if (tryResult.hasException()) {
co_return folly::Try<std::vector<detail::collect_all_range_component_t<
detail::range_reference_t<InputRange>>>>{
std::move(tryResult.exception())};
}
results.emplace_back(std::move(tryResult).value());
if (iterationException) {
co_yield co_error(std::move(iterationException));
}
co_return results;
}
template <typename InputRange>
auto collectAllTryWindowed(InputRange awaitables, std::size_t maxConcurrency)
-> folly::coro::Task<std::vector<detail::collect_all_try_range_component_t<
detail::range_reference_t<InputRange>>>> {
co_return co_await detail::collectAllTryWindowedImpl(
std::move(awaitables),
maxConcurrency,
co_await co_current_cancellation_token,
CancellationSource::invalid());
}
} // namespace coro
} // namespace folly
......@@ -82,9 +82,8 @@ using range_reference_t = iterator_reference_t<range_iterator_t<Range>>;
// request cancellation of any outstanding tasks and the whole collectAll()
// operation will complete with an exception once all of the operations
// have completed. Any partial results will be discarded. If multiple
// operations fail with an exception then one of the exceptions will be rethrown
// to the caller (which one is unspecified) and the other exceptions are
// discarded.
// operations fail with an exception then the exception from the first task
// to fail will be rethrown and subsequent errors are discarded.
//
// If you need to know which operation failed or you want to handle partial
// failures then you can use the folly::coro::collectAllTry() instead which
......@@ -148,7 +147,7 @@ auto collectAllTry(SemiAwaitables&&... awaitables)
remove_cvref_t<SemiAwaitables>>...>>;
////////////////////////////////////////////////////////////////////////
// rangeCollectAll(RangeOf<SemiAwaitable<T>>&&)
// collectAllRange(RangeOf<SemiAwaitable<T>>&&)
// -> SemiAwaitable<std::vector<T>>
//
// The collectAllRange() function can be used to concurrently await a collection
......@@ -158,8 +157,8 @@ auto collectAllTry(SemiAwaitables&&... awaitables)
// If any of the operations fail with an exception then requests cancellation of
// any outstanding operations and the entire operation fails with an exception,
// discarding any partial results. If more than one operation fails with an
// exception then the exception from the first failed operation in the input
// range is rethrown. Other results and exceptions are discarded.
// exception then the exception from task that failed first (in time) is
// rethrown. Other results and exceptions are discarded.
//
// If you need to be able to distinguish which operation failed or handle
// partial failures then use collectAllTryRange() instead.
......@@ -242,13 +241,13 @@ auto collectAllTryRange(std::vector<SemiAwaitable> awaitables)
// If any of the input awaitables fail with an exception then requests
// cancellation of any incomplete operations and fails the whole
// operation with an exception. If multiple input awaitables fail with
// an exception then one of these exceptions will be rethrown and the rest
// of the results will be discarded.
// an exception then the exeception from the first task to fail (in time)
// will be rethrown and the rest of the results will be discarded.
//
// If there is an exception thrown while iterating over the input-range then
// it will still guarantee that any prior awaitables in the input-range will
// run to completion before completing the collectAllWindowed() operation with
// an exception.
// the exception thrown during iteration.
//
// The resulting std::vector will contain the results in the corresponding
// order of their respective awaitables in the input range.
......
......@@ -144,13 +144,21 @@ struct ErrorA : std::exception {};
struct ErrorB : std::exception {};
struct ErrorC : std::exception {};
TEST_F(CollectAllTest, ThrowsOneOfMultipleErrors) {
TEST_F(CollectAllTest, ThrowsFirstError) {
bool caughtException = false;
folly::coro::blockingWait([&]() -> folly::coro::Task<void> {
try {
bool throwError = true;
// Child tasks are started in-order.
// The first task will reschedule itself onto the executor.
// The second task will fail immediately and will be the first
// task to fail.
// Then the third and first tasks will fail.
// As the second task failed first we should see its exception
// propagate out of collectAll().
auto [x, y, z] = co_await folly::coro::collectAll(
[&]() -> folly::coro::Task<int> {
co_await folly::coro::co_reschedule_on_current_executor;
if (throwError) {
throw ErrorA{};
}
......@@ -172,12 +180,8 @@ TEST_F(CollectAllTest, ThrowsOneOfMultipleErrors) {
(void)y;
(void)z;
CHECK(false);
} catch (const ErrorA&) {
caughtException = true;
} catch (const ErrorB&) {
caughtException = true;
} catch (const ErrorC&) {
caughtException = true;
}
}());
CHECK(caughtException);
......@@ -693,6 +697,32 @@ TEST_F(CollectAllRangeTest, SubtasksCancelledWhenASubtaskFails) {
}());
}
TEST_F(CollectAllRangeTest, FailsWithErrorOfFirstTaskToFailWhenMultipleErrors) {
using namespace std::chrono_literals;
folly::coro::blockingWait([]() -> folly::coro::Task<void> {
try {
co_await folly::coro::collectAllRange(
[]() -> folly::coro::Generator<folly::coro::Task<void>&&> {
co_yield folly::coro::sleep(1s);
co_yield[]()->folly::coro::Task<> {
co_await folly::coro::sleep(1s);
throw ErrorA{};
}
();
co_yield[]()->folly::coro::Task<> {
co_await folly::coro::co_reschedule_on_current_executor;
throw ErrorB{};
}
();
co_yield folly::coro::sleep(2s);
}());
CHECK(false);
} catch (const ErrorB&) {
}
}());
}
TEST_F(CollectAllRangeTest, SubtasksCancelledWhenParentTaskCancelled) {
using namespace std::chrono_literals;
......@@ -1119,7 +1149,7 @@ TEST_F(CollectAllWindowedTest, VectorOfValueTask) {
}
}
TEST_F(CollectAllWindowedTest, PartialFailure) {
TEST_F(CollectAllWindowedTest, MultipleFailuresPropagatesFirstError) {
try {
[[maybe_unused]] auto results =
folly::coro::blockingWait(folly::coro::collectAllWindowed(
......@@ -1142,8 +1172,6 @@ TEST_F(CollectAllWindowedTest, PartialFailure) {
}(),
5));
CHECK(false); // Should have thrown.
} catch (ErrorA) {
// Expected.
} catch (ErrorB) {
// Expected.
}
......
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