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

Add CancellationToken support to collectAll()

Summary:
The `collectAll()`, `collectAllRange()` and `collectAllWindowed()` functions now cancel their subtasks when either the parent task is cancelled or when any of the subtasks fail with an exception.

The `collectAllTry()`, `collectAllTryRange()` and `collectAllTryWindowed()` functions now cancel their subtasks if the parent task is cancelled, but not if the child tasks fail with an exception.

The logic between the distinction here is that if you are calling one of the 'try' functions then you are intending to handle partial failure and so probably want each of the operations to run to completion independently of the others.

Reviewed By: kirkshoop

Differential Revision: D16820619

fbshipit-source-id: 5c203b41592a0145ff7be1c609dfccd39c41a3df
parent 412928c4
......@@ -38,19 +38,28 @@ template <typename SemiAwaitable, typename Result>
detail::BarrierTask makeCollectAllTask(
folly::Executor* executor,
SemiAwaitable&& awaitable,
const CancellationToken& cancelToken,
const CancellationSource& cancelSource,
Try<Result>& result) {
try {
if constexpr (std::is_void_v<Result>) {
co_await co_viaIfAsync(executor, static_cast<SemiAwaitable&&>(awaitable));
co_await co_viaIfAsync(
executor,
co_withCancellation(
cancelToken, static_cast<SemiAwaitable&&>(awaitable)));
result.emplace();
} else {
result.emplace(co_await co_viaIfAsync(
executor, static_cast<SemiAwaitable&&>(awaitable)));
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();
}
}
......@@ -67,11 +76,16 @@ auto collectAllTryImpl(
std::tuple<collect_all_try_component_t<SemiAwaitables>...> results;
Executor* executor = co_await co_current_executor;
const CancellationToken& cancelToken =
co_await co_current_cancellation_token;
const CancellationSource cancelSource = CancellationSource::invalid();
folly::coro::detail::BarrierTask tasks[sizeof...(SemiAwaitables)] = {
makeCollectAllTask(
executor,
static_cast<SemiAwaitables&&>(awaitables),
cancelToken,
cancelSource,
std::get<Indices>(results))...,
};
......@@ -105,10 +119,19 @@ auto collectAllImpl(
Executor* 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();
folly::coro::detail::BarrierTask tasks[sizeof...(SemiAwaitables)] = {
makeCollectAllTask(
executor,
static_cast<SemiAwaitables&&>(awaitables),
cancelToken,
cancelSource,
std::get<Indices>(results))...,
};
......@@ -130,6 +153,251 @@ auto collectAllImpl(
}
}
template <typename InputRange>
auto collectAllTryRangeImpl(
InputRange awaitables,
CancellationToken cancelToken,
CancellationSource cancelSource)
-> 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 =
folly::getKeepAliveToken(co_await co_current_executor);
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>>;
if constexpr (std::is_void_v<await_result>) {
co_await co_viaIfAsync(
executor.get_alias(),
co_withCancellation(cancelToken, std::move(semiAwaitable)));
result.emplace();
} else {
result.emplace(co_await co_viaIfAsync(
executor.get_alias(),
co_withCancellation(cancelToken, std::move(semiAwaitable))));
}
} catch (const std::exception& ex) {
result.emplaceException(std::current_exception(), ex);
cancelSource.requestCancellation();
} catch (...) {
result.emplaceException(std::current_exception());
cancelSource.requestCancellation();
}
};
// 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 index = 0;
for (auto&& semiAwaitable : awaitables) {
tasks.push_back(makeTask(
index++, static_cast<decltype(semiAwaitable)&&>(semiAwaitable)));
}
}
// Now that we know how many tasks there are, allocate that
// many Try objects to store the results before we start
// executing the tasks.
results.resize(tasks.size());
// Launch the tasks and wait for them all to finish.
{
detail::Barrier barrier{tasks.size() + 1};
for (auto&& task : tasks) {
task.start(&barrier);
}
co_await detail::UnsafeResumeInlineSemiAwaitable{barrier.arriveAndWait()};
}
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>>>> {
assert(maxConcurrency > 0);
std::vector<detail::collect_all_try_range_component_t<
detail::range_reference_t<InputRange>>>
results;
exception_wrapper iterationException;
folly::coro::Mutex mutex;
folly::Executor::KeepAlive<> executor =
folly::getKeepAliveToken(co_await co_current_executor);
using std::begin;
using std::end;
auto iter = begin(awaitables);
const auto iterEnd = end(awaitables);
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 (!iterationException && iter != iterEnd) {
try {
awaitable_t awaitable = *iter;
try {
++iter;
} catch (const std::exception& ex) {
iterationException = exception_wrapper{std::current_exception(), ex};
cancelSource.requestCancellation();
} catch (...) {
iterationException = exception_wrapper{std::current_exception()};
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;
}
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))));
}
} 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;
} catch (...) {
assert(lock.owns_lock());
if (!iterationException) {
iterationException = exception_wrapper{std::current_exception()};
cancelSource.requestCancellation();
}
co_return;
}
assert(lock.owns_lock());
}
};
std::vector<detail::BarrierTask> workerTasks;
detail::Barrier barrier{1};
exception_wrapper workerCreationException;
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
// it can consume as many results synchronously as it can before
// 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();
workerTasks.push_back(makeWorker());
barrier.add(1);
workerTasks.back().start(&barrier);
lock = co_await mutex.co_scoped_lock();
}
} catch (const std::exception& ex) {
workerCreationException = exception_wrapper{std::current_exception(), ex};
cancelSource.requestCancellation();
} catch (...) {
workerCreationException = exception_wrapper{std::current_exception()};
cancelSource.requestCancellation();
}
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_return results;
}
} // namespace detail
template <typename... SemiAwaitables>
......@@ -158,8 +426,23 @@ template <
auto collectAllRange(InputRange awaitables)
-> folly::coro::Task<std::vector<detail::collect_all_range_component_t<
detail::range_reference_t<InputRange>>>> {
auto results =
co_await folly::coro::collectAllTryRange(std::move(awaitables));
const CancellationSource cancelSource;
CancellationCallback cancelCallback(
co_await co_current_cancellation_token, [&]() noexcept {
cancelSource.requestCancellation();
});
auto resultsTry =
co_await folly::coro::co_awaitTry(detail::collectAllTryRangeImpl(
std::move(awaitables), cancelSource.getToken(), cancelSource));
if (resultsTry.hasException()) {
co_return Try<std::vector<detail::collect_all_range_component_t<
detail::range_reference_t<InputRange>>>>{
std::move(resultsTry.exception())};
}
auto& results = resultsTry.value();
// Collate the results into a single result vector.
std::vector<detail::collect_all_range_component_t<
......@@ -180,28 +463,34 @@ template <
semi_await_result_t<detail::range_reference_t<InputRange>>>,
int>>
auto collectAllRange(InputRange awaitables) -> folly::coro::Task<void> {
std::vector<detail::collect_all_try_range_component_t<
detail::range_reference_t<InputRange>>>
results;
folly::Executor::KeepAlive<> executor =
folly::getKeepAliveToken(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::atomic<bool> anyFailures{false};
exception_wrapper firstException;
using awaitable_type = remove_cvref_t<detail::range_reference_t<InputRange>>;
auto makeTask = [&](awaitable_type semiAwaitable) -> detail::BarrierTask {
try {
co_await coro::co_viaIfAsync(
executor.get_alias(), std::move(semiAwaitable));
co_await co_viaIfAsync(
executor.get_alias(),
co_withCancellation(cancelToken, std::move(semiAwaitable)));
} catch (const std::exception& ex) {
if (!anyFailures.exchange(true, std::memory_order_relaxed)) {
firstException = exception_wrapper{std::current_exception(), ex};
cancelSource.requestCancellation();
}
} catch (...) {
if (!anyFailures.exchange(true, std::memory_order_relaxed)) {
firstException = exception_wrapper{std::current_exception()};
cancelSource.requestCancellation();
}
}
};
......@@ -236,65 +525,10 @@ template <typename InputRange>
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 =
folly::getKeepAliveToken(co_await co_current_executor);
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>>;
if constexpr (std::is_void_v<await_result>) {
co_await coro::co_viaIfAsync(
executor.get_alias(), std::move(semiAwaitable));
result.emplace();
} else {
result.emplace(co_await coro::co_viaIfAsync(
executor.get_alias(), std::move(semiAwaitable)));
}
} catch (const std::exception& ex) {
result.emplaceException(std::current_exception(), ex);
} catch (...) {
result.emplaceException(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 index = 0;
for (auto&& semiAwaitable : awaitables) {
tasks.push_back(makeTask(
index++, static_cast<decltype(semiAwaitable)&&>(semiAwaitable)));
}
}
// Now that we know how many tasks there are, allocate that
// many Try objects to store the results before we start
// executing the tasks.
results.resize(tasks.size());
// Launch the tasks and wait for them all to finish.
{
detail::Barrier barrier{tasks.size() + 1};
for (auto&& task : tasks) {
task.start(&barrier);
}
co_await detail::UnsafeResumeInlineSemiAwaitable{barrier.arriveAndWait()};
}
co_return results;
co_return co_await detail::collectAllTryRangeImpl(
std::move(awaitables),
co_await co_current_cancellation_token,
CancellationSource::invalid());
}
template <
......@@ -307,13 +541,20 @@ auto collectAllWindowed(InputRange awaitables, std::size_t maxConcurrency)
-> folly::coro::Task<void> {
assert(maxConcurrency > 0);
std::exception_ptr firstException;
exception_wrapper firstException;
folly::coro::Mutex mutex;
folly::Executor::KeepAlive<> executor =
folly::getKeepAliveToken(co_await co_current_executor);
const folly::CancellationSource cancelSource;
folly::CancellationCallback cancelCallback(
co_await co_current_cancellation_token, [&]() noexcept {
cancelSource.requestCancellation();
});
const folly::CancellationToken cancelToken = cancelSource.getToken();
using std::begin;
using std::end;
auto iter = begin(awaitables);
......@@ -330,20 +571,30 @@ auto collectAllWindowed(InputRange awaitables, std::size_t maxConcurrency)
awaitable_t awaitable = *iter;
try {
++iter;
} catch (const std::exception& ex) {
if (!firstException) {
firstException = exception_wrapper{std::current_exception(), ex};
cancelSource.requestCancellation();
}
} catch (...) {
if (!firstException) {
firstException = std::current_exception();
firstException = exception_wrapper{std::current_exception()};
cancelSource.requestCancellation();
}
}
lock.unlock();
std::exception_ptr ex;
exception_wrapper ex;
try {
co_await co_viaIfAsync(
executor.get_alias(), static_cast<awaitable_t&&>(awaitable));
executor.get_alias(),
co_withCancellation(
cancelToken, static_cast<awaitable_t&&>(awaitable)));
} catch (const std::exception& e) {
ex = exception_wrapper{std::current_exception(), e};
} catch (...) {
ex = std::current_exception();
ex = exception_wrapper{std::current_exception()};
}
lock =
......@@ -351,6 +602,7 @@ auto collectAllWindowed(InputRange awaitables, std::size_t maxConcurrency)
if (ex && !firstException) {
firstException = std::move(ex);
cancelSource.requestCancellation();
}
}
};
......@@ -359,7 +611,7 @@ auto collectAllWindowed(InputRange awaitables, std::size_t maxConcurrency)
detail::Barrier barrier{1};
std::exception_ptr workerCreationException;
exception_wrapper workerCreationException;
try {
auto lock = co_await mutex.co_scoped_lock();
......@@ -378,17 +630,21 @@ 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();
} catch (...) {
workerCreationException = std::current_exception();
workerCreationException = exception_wrapper{std::current_exception()};
cancelSource.requestCancellation();
}
co_await detail::UnsafeResumeInlineSemiAwaitable{barrier.arriveAndWait()};
if (firstException) {
std::rethrow_exception(std::move(firstException));
firstException.throw_exception();
} else if (workerTasks.empty() && workerCreationException) {
// Failed to create any workers to process the tasks.
std::rethrow_exception(std::move(workerCreationException));
workerCreationException.throw_exception();
}
}
......@@ -401,8 +657,20 @@ template <
auto collectAllWindowed(InputRange awaitables, std::size_t maxConcurrency)
-> folly::coro::Task<std::vector<detail::collect_all_range_component_t<
detail::range_reference_t<InputRange>>>> {
auto tryResults = co_await collectAllTryWindowed(
static_cast<InputRange&&>(awaitables), maxConcurrency);
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>>>
......@@ -410,6 +678,11 @@ auto collectAllWindowed(InputRange awaitables, std::size_t maxConcurrency)
results.reserve(tryResults.size());
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());
}
......@@ -420,135 +693,11 @@ 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>>>> {
assert(maxConcurrency > 0);
std::vector<detail::collect_all_try_range_component_t<
detail::range_reference_t<InputRange>>>
results;
std::exception_ptr iterationException;
folly::coro::Mutex mutex;
folly::Executor::KeepAlive<> executor =
folly::getKeepAliveToken(co_await co_current_executor);
using std::begin;
using std::end;
auto iter = begin(awaitables);
const auto iterEnd = end(awaitables);
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 (!iterationException && iter != iterEnd) {
try {
awaitable_t awaitable = *iter;
try {
++iter;
} catch (...) {
iterationException = std::current_exception();
}
const auto thisIndex = results.size();
try {
results.emplace_back();
} catch (...) {
if (!iterationException) {
iterationException = std::current_exception();
}
// Failure to grow the results vector is fatal.
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(), static_cast<awaitable_t&&>(awaitable));
result.emplace();
} else {
result.emplace(co_await co_viaIfAsync(
executor.get_alias(), static_cast<awaitable_t&&>(awaitable)));
}
} catch (const std::exception& ex) {
result.emplaceException(std::current_exception(), ex);
} catch (...) {
result.emplaceException(std::current_exception());
}
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);
} catch (...) {
results[thisIndex].emplaceException(std::current_exception());
}
} catch (...) {
assert(lock.owns_lock());
if (!iterationException) {
iterationException = std::current_exception();
}
co_return;
}
assert(lock.owns_lock());
}
};
std::vector<detail::BarrierTask> workerTasks;
detail::Barrier barrier{1};
std::exception_ptr workerCreationException;
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
// it can consume as many results synchronously as it can before
// 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();
workerTasks.push_back(makeWorker());
barrier.add(1);
workerTasks.back().start(&barrier);
lock = co_await mutex.co_scoped_lock();
}
} catch (...) {
workerCreationException = std::current_exception();
}
co_await detail::UnsafeResumeInlineSemiAwaitable{barrier.arriveAndWait()};
if (iterationException) {
std::rethrow_exception(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.
std::rethrow_exception(std::move(workerCreationException));
}
co_return results;
co_return co_await detail::collectAllTryWindowedImpl(
std::move(awaitables),
maxConcurrency,
co_await co_current_cancellation_token,
CancellationSource::invalid());
}
} // namespace coro
......
......@@ -77,12 +77,13 @@ using range_reference_t = iterator_reference_t<range_iterator_t<Range>>;
// point. This means that awaiting multiple sub-tasks that all complete
// synchronously will still execute them sequentially on the current thread.
//
// If any of the input operations complete with an exception then the whole
// collectAll() operation will also 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.
// If any of the input operations complete with an exception then it will
// 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.
//
// 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
......@@ -116,6 +117,9 @@ auto collectAll(SemiAwaitables&&... awaitables) -> folly::coro::Task<std::tuple<
// operations and handle partial failures but has a less-convenient interface
// than collectAll().
//
// It also differs in that failure of one subtask does _not_ request
// cancellation of the other subtasks.
//
// Example: Handling partial failure with collectAllTry()
// folly::coro::Task<Foo> doSomething();
// folly::coro::Task<Bar> doSomethingElse();
......@@ -150,11 +154,11 @@ auto collectAllTry(SemiAwaitables&&... awaitables)
// of SemiAwaitable objects, returning a std::vector of the individual results
// once all operations have completed.
//
// If any of the operations fail with an exception the entire operation fails
// with an exception and any partial results are discarded. 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.
// 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.
//
// If you need to be able to distinguish which operation failed or handle
// partial failures then use collectAllTryRange() instead.
......@@ -233,8 +237,9 @@ auto collectAllTryRange(std::vector<Task<T>> awaitables)
// 'maxConcurrency' of these input awaitables to be concurrently awaited
// at any one point in time.
//
// If any of the input awaitables fail with an exception then the whole
// operation fails with an exception. If multiple input awaitables fail with
// 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.
//
......
......@@ -26,6 +26,7 @@
#include <folly/experimental/coro/CurrentExecutor.h>
#include <folly/experimental/coro/Generator.h>
#include <folly/experimental/coro/Mutex.h>
#include <folly/experimental/coro/Sleep.h>
#include <folly/experimental/coro/Task.h>
#include <folly/portability/GTest.h>
......@@ -196,6 +197,8 @@ TEST(CollectAll, SynchronousCompletionInLoopDoesntCauseStackOverflow) {
}());
}
struct OperationCancelled : std::exception {};
template <
typename Iter,
typename Sentinel,
......@@ -241,6 +244,69 @@ TEST(CollectAll, ParallelAccumulate) {
.scheduleOn(&threadPool));
}
TEST(CollectAll, CollectAllCancelsSubtasksWhenASubtaskFails) {
using namespace std::chrono_literals;
folly::coro::blockingWait([]() -> folly::coro::Task<void> {
auto start = std::chrono::steady_clock::now();
try {
auto [a, b, c] = co_await folly::coro::collectAll(
[]() -> folly::coro::Task<int> {
co_await folly::coro::sleep(10s);
co_return 42;
}(),
[]() -> folly::coro::Task<float> {
co_await folly::coro::sleep(5s);
co_return 3.14f;
}(),
[]() -> folly::coro::Task<void> {
co_await folly::coro::co_reschedule_on_current_executor;
throw ErrorA{};
}());
CHECK(false);
(void)a;
(void)b;
(void)c;
} catch (const ErrorA&) {
}
auto end = std::chrono::steady_clock::now();
CHECK((end - start) < 1s);
}());
}
TEST(CollectAll, CollectAllCancelsSubtasksWhenParentTaskCancelled) {
using namespace std::chrono_literals;
folly::coro::blockingWait([]() -> folly::coro::Task<void> {
auto start = std::chrono::steady_clock::now();
folly::CancellationSource cancelSource;
auto [a, b, c] = co_await folly::coro::co_withCancellation(
cancelSource.getToken(),
folly::coro::collectAll(
[&]() -> folly::coro::Task<int> {
co_await folly::coro::sleep(10s);
co_return 42;
}(),
[&]() -> folly::coro::Task<float> {
co_await folly::coro::sleep(5s);
co_return 3.14f;
}(),
[&]() -> folly::coro::Task<void> {
co_await folly::coro::co_reschedule_on_current_executor;
co_await folly::coro::co_reschedule_on_current_executor;
co_await folly::coro::co_reschedule_on_current_executor;
cancelSource.requestCancellation();
}()));
auto end = std::chrono::steady_clock::now();
CHECK((end - start) < 1s);
CHECK_EQ(42, a);
CHECK_EQ(3.14f, b);
(void)c;
}());
}
/////////////////////////////////////////////////////////
// folly::coro::collectAllTry() tests
......@@ -328,6 +394,76 @@ TEST(CollectAllTry, PartialFailure) {
}());
}
TEST(CollectAllTry, CollectAllTryDoesNotCancelSubtasksWhenASubtaskFails) {
using namespace std::chrono_literals;
folly::coro::blockingWait([]() -> folly::coro::Task<void> {
auto [a, b, c] = co_await folly::coro::collectAllTry(
[]() -> folly::coro::Task<int> {
co_await folly::coro::co_reschedule_on_current_executor;
co_await folly::coro::co_reschedule_on_current_executor;
co_await folly::coro::co_reschedule_on_current_executor;
co_await folly::coro::co_reschedule_on_current_executor;
co_await folly::coro::co_reschedule_on_current_executor;
CHECK(!(co_await folly::coro::co_current_cancellation_token)
.isCancellationRequested());
co_return 42;
}(),
[]() -> folly::coro::Task<float> {
co_await folly::coro::co_reschedule_on_current_executor;
co_await folly::coro::co_reschedule_on_current_executor;
co_await folly::coro::co_reschedule_on_current_executor;
co_await folly::coro::co_reschedule_on_current_executor;
co_await folly::coro::co_reschedule_on_current_executor;
CHECK(!(co_await folly::coro::co_current_cancellation_token)
.isCancellationRequested());
co_return 3.14f;
}(),
[]() -> folly::coro::Task<void> {
co_await folly::coro::co_reschedule_on_current_executor;
throw ErrorA{};
}());
CHECK(a.hasValue());
CHECK_EQ(42, a.value());
CHECK(b.hasValue());
CHECK_EQ(3.14f, b.value());
CHECK(c.hasException());
}());
}
TEST(CollectAllTry, CollectAllCancelsSubtasksWhenParentTaskCancelled) {
using namespace std::chrono_literals;
folly::coro::blockingWait([]() -> folly::coro::Task<void> {
auto start = std::chrono::steady_clock::now();
folly::CancellationSource cancelSource;
auto [a, b, c] = co_await folly::coro::co_withCancellation(
cancelSource.getToken(),
folly::coro::collectAllTry(
[&]() -> folly::coro::Task<int> {
co_await folly::coro::sleep(10s);
co_return 42;
}(),
[&]() -> folly::coro::Task<float> {
co_await folly::coro::sleep(5s);
co_return 3.14f;
}(),
[&]() -> folly::coro::Task<void> {
co_await folly::coro::co_reschedule_on_current_executor;
co_await folly::coro::co_reschedule_on_current_executor;
co_await folly::coro::co_reschedule_on_current_executor;
cancelSource.requestCancellation();
}()));
auto end = std::chrono::steady_clock::now();
CHECK((end - start) < 1s);
CHECK_EQ(42, a.value());
CHECK_EQ(3.14f, b.value());
CHECK(c.hasValue());
}());
}
/////////////////////////////////////////////////////////////
// collectAllRange() tests
......@@ -422,6 +558,81 @@ TEST(CollectAllRange, RangeOfNonVoid) {
}());
}
TEST(CollectAllRange, SubtasksCancelledWhenASubtaskFails) {
using namespace std::chrono_literals;
folly::coro::blockingWait([]() -> folly::coro::Task<void> {
bool consumedAllTasks = false;
auto generateTasks = [&]()
-> folly::coro::Generator<folly::coro::Task<void>&&> {
for (int i = 0; i < 10; ++i) {
co_yield folly::coro::sleep(10s);
}
co_yield[]()->folly::coro::Task<void> {
co_await folly::coro::co_reschedule_on_current_executor;
throw ErrorA{};
}
();
for (int i = 0; i < 10; ++i) {
co_yield folly::coro::sleep(10s);
}
consumedAllTasks = true;
};
auto start = std::chrono::steady_clock::now();
try {
co_await folly::coro::collectAllRange(generateTasks());
CHECK(false);
} catch (const ErrorA&) {
}
auto end = std::chrono::steady_clock::now();
CHECK((end - start) < 1s);
CHECK(consumedAllTasks);
}());
}
TEST(CollectAllRange, SubtasksCancelledWhenParentTaskCancelled) {
using namespace std::chrono_literals;
folly::coro::blockingWait([]() -> folly::coro::Task<void> {
folly::CancellationSource cancelSource;
bool consumedAllTasks = false;
auto generateTasks = [&]()
-> folly::coro::Generator<folly::coro::Task<void>&&> {
for (int i = 0; i < 10; ++i) {
co_yield folly::coro::sleep(10s);
}
co_yield[&]()->folly::coro::Task<void> {
co_await folly::coro::co_reschedule_on_current_executor;
co_await folly::coro::co_reschedule_on_current_executor;
co_await folly::coro::co_reschedule_on_current_executor;
co_await folly::coro::co_reschedule_on_current_executor;
cancelSource.requestCancellation();
auto token = co_await folly::coro::co_current_cancellation_token;
CHECK(token.isCancellationRequested());
}
();
consumedAllTasks = true;
};
auto start = std::chrono::steady_clock::now();
co_await folly::coro::co_withCancellation(
cancelSource.getToken(), folly::coro::collectAllRange(generateTasks()));
auto end = std::chrono::steady_clock::now();
CHECK((end - start) < 1s);
CHECK(consumedAllTasks);
}());
}
////////////////////////////////////////////////////////////////////
// folly::coro::collectAllTryRange() tests
......@@ -487,6 +698,84 @@ TEST(CollectAllTryRange, RangeOfValueSomeFailing) {
}());
}
TEST(CollectAllTryRange, NotCancelledWhenSubtaskFails) {
folly::coro::blockingWait([]() -> folly::coro::Task<void> {
auto generateTasks = [&]()
-> folly::coro::Generator<folly::coro::Task<void>&&> {
auto makeValidationTask = []() -> folly::coro::Task<void> {
co_await folly::coro::co_reschedule_on_current_executor;
co_await folly::coro::co_reschedule_on_current_executor;
co_await folly::coro::co_reschedule_on_current_executor;
auto token = co_await folly::coro::co_current_cancellation_token;
CHECK(!token.isCancellationRequested());
};
co_yield makeValidationTask();
co_yield makeValidationTask();
co_yield[]()->folly::coro::Task<void> {
co_await folly::coro::co_reschedule_on_current_executor;
throw ErrorA{};
}
();
co_yield makeValidationTask();
co_yield makeValidationTask();
};
auto results = co_await folly::coro::collectAllTryRange(generateTasks());
CHECK_EQ(5, results.size());
CHECK(results[0].hasValue());
CHECK(results[1].hasValue());
CHECK(results[2].hasException());
CHECK(results[3].hasValue());
CHECK(results[4].hasValue());
}());
}
TEST(CollectAllTryRange, SubtasksCancelledWhenParentTaskCancelled) {
using namespace std::chrono_literals;
folly::coro::blockingWait([]() -> folly::coro::Task<void> {
folly::CancellationSource cancelSource;
bool consumedAllTasks = false;
auto generateTasks = [&]()
-> folly::coro::Generator<folly::coro::Task<void>&&> {
for (int i = 0; i < 10; ++i) {
co_yield folly::coro::sleep(10s);
}
co_yield[&]()->folly::coro::Task<void> {
co_await folly::coro::co_reschedule_on_current_executor;
co_await folly::coro::co_reschedule_on_current_executor;
co_await folly::coro::co_reschedule_on_current_executor;
co_await folly::coro::co_reschedule_on_current_executor;
cancelSource.requestCancellation();
auto token = co_await folly::coro::co_current_cancellation_token;
CHECK(token.isCancellationRequested());
}
();
consumedAllTasks = true;
};
auto start = std::chrono::steady_clock::now();
auto results = co_await folly::coro::co_withCancellation(
cancelSource.getToken(),
folly::coro::collectAllTryRange(generateTasks()));
auto end = std::chrono::steady_clock::now();
CHECK_EQ(11, results.size());
for (auto& result : results) {
CHECK(result.hasValue());
}
CHECK((end - start) < 1s);
CHECK(consumedAllTasks);
}());
}
////////////////////////////////////////////////////////////////////
// folly::coro::collectAllWindowed() tests
......@@ -661,6 +950,80 @@ TEST(CollectAllWindowed, PartialFailure) {
}
}
TEST(CollectAllWindowed, SubtasksCancelledWhenASubtaskFails) {
using namespace std::chrono_literals;
folly::coro::blockingWait([]() -> folly::coro::Task<void> {
bool consumedAllTasks = false;
auto generateTasks = [&]()
-> folly::coro::Generator<folly::coro::Task<void>&&> {
co_yield[]()->folly::coro::Task<void> {
co_await folly::coro::co_reschedule_on_current_executor;
throw ErrorA{};
}
();
for (int i = 0; i < 10; ++i) {
co_yield folly::coro::sleep(10s);
}
consumedAllTasks = true;
};
auto start = std::chrono::steady_clock::now();
try {
co_await folly::coro::collectAllWindowed(generateTasks(), 2);
CHECK(false);
} catch (const ErrorA&) {
}
auto end = std::chrono::steady_clock::now();
CHECK((end - start) < 1s);
CHECK(consumedAllTasks);
}());
}
TEST(CollectAllWindowed, SubtasksCancelledWhenParentTaskCancelled) {
using namespace std::chrono_literals;
folly::coro::blockingWait([]() -> folly::coro::Task<void> {
folly::CancellationSource cancelSource;
bool consumedAllTasks = false;
auto generateTasks = [&]()
-> folly::coro::Generator<folly::coro::Task<void>&&> {
co_yield folly::coro::sleep(10s);
co_yield folly::coro::sleep(10s);
co_yield[&]()->folly::coro::Task<void> {
co_await folly::coro::co_reschedule_on_current_executor;
co_await folly::coro::co_reschedule_on_current_executor;
co_await folly::coro::co_reschedule_on_current_executor;
co_await folly::coro::co_reschedule_on_current_executor;
cancelSource.requestCancellation();
auto token = co_await folly::coro::co_current_cancellation_token;
CHECK(token.isCancellationRequested());
}
();
co_yield folly::coro::sleep(10s);
co_yield folly::coro::sleep(10s);
consumedAllTasks = true;
};
auto start = std::chrono::steady_clock::now();
co_await folly::coro::co_withCancellation(
cancelSource.getToken(),
folly::coro::collectAllWindowed(generateTasks(), 4));
auto end = std::chrono::steady_clock::now();
CHECK((end - start) < 1s);
CHECK(consumedAllTasks);
}());
}
////////////////////////////////////////////////////////////////////
// folly::coro::collectAllTryWindowed() tests
......@@ -733,4 +1096,80 @@ TEST(CollectAllTryWindowed, GeneratorFailure) {
CHECK_EQ(0, active);
}
TEST(CollectAllTryWindowed, NotCancelledWhenSubtaskFails) {
folly::coro::blockingWait([]() -> folly::coro::Task<void> {
auto generateTasks = [&]()
-> folly::coro::Generator<folly::coro::Task<void>&&> {
co_yield[]()->folly::coro::Task<void> {
co_await folly::coro::co_reschedule_on_current_executor;
throw ErrorA{};
}
();
auto makeValidationTask = []() -> folly::coro::Task<void> {
co_await folly::coro::co_reschedule_on_current_executor;
co_await folly::coro::co_reschedule_on_current_executor;
co_await folly::coro::co_reschedule_on_current_executor;
auto token = co_await folly::coro::co_current_cancellation_token;
CHECK(!token.isCancellationRequested());
};
co_yield makeValidationTask();
co_yield makeValidationTask();
};
auto results =
co_await folly::coro::collectAllTryWindowed(generateTasks(), 2);
CHECK_EQ(3, results.size());
CHECK(results[0].hasException());
CHECK(results[1].hasValue());
CHECK(results[2].hasValue());
}());
}
TEST(CollectAllTryWindowed, SubtasksCancelledWhenParentTaskCancelled) {
using namespace std::chrono_literals;
folly::coro::blockingWait([]() -> folly::coro::Task<void> {
folly::CancellationSource cancelSource;
bool consumedAllTasks = false;
auto generateTasks = [&]()
-> folly::coro::Generator<folly::coro::Task<void>&&> {
co_yield folly::coro::sleep(10s);
co_yield folly::coro::sleep(10s);
co_yield[&]()->folly::coro::Task<void> {
co_await folly::coro::co_reschedule_on_current_executor;
co_await folly::coro::co_reschedule_on_current_executor;
co_await folly::coro::co_reschedule_on_current_executor;
co_await folly::coro::co_reschedule_on_current_executor;
cancelSource.requestCancellation();
auto token = co_await folly::coro::co_current_cancellation_token;
CHECK(token.isCancellationRequested());
}
();
co_yield folly::coro::sleep(10s);
co_yield folly::coro::sleep(10s);
consumedAllTasks = true;
};
auto start = std::chrono::steady_clock::now();
auto results = co_await folly::coro::co_withCancellation(
cancelSource.getToken(),
folly::coro::collectAllTryWindowed(generateTasks(), 4));
auto end = std::chrono::steady_clock::now();
CHECK_EQ(5, results.size());
for (auto& result : results) {
CHECK(result.hasValue());
}
CHECK((end - start) < 1s);
CHECK(consumedAllTasks);
}());
}
#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