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

Add folly::coro::collectAllWindowed

Summary:
Adds some functions for concurrently awaiting a collection of tasks but with an upper bound on the number of tasks to process concurrently.

- `folly::coro::collectAllWindowed(range, maxConcurrency)`
- `folly::coro::collectAllTryWindowed(range, maxConcurrency)`

Reviewed By: andriigrynenko

Differential Revision: D15014011

fbshipit-source-id: 8f62dbae47e0548eacec8dc59239668040d3a771
parent 8869d108
......@@ -15,6 +15,7 @@
*/
#include <folly/ExceptionWrapper.h>
#include <folly/experimental/coro/Mutex.h>
#include <folly/experimental/coro/detail/Barrier.h>
#include <folly/experimental/coro/detail/BarrierTask.h>
......@@ -291,7 +292,7 @@ auto collectAllTryRange(InputRange awaitables)
{
std::size_t index = 0;
for (auto&& semiAwaitable : static_cast<InputRange&&>(awaitables)) {
for (auto&& semiAwaitable : awaitables) {
tasks.push_back(makeTask(
index++, static_cast<decltype(semiAwaitable)&&>(semiAwaitable)));
}
......@@ -314,5 +315,259 @@ auto collectAllTryRange(InputRange awaitables)
co_return std::move(results);
}
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::exception_ptr firstException;
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;
auto makeWorker = [&]() -> detail::BarrierTask {
auto lock =
co_await co_viaIfAsync(executor.copyDummy(), mutex.co_scoped_lock());
while (iter != iterEnd) {
awaitable_t awaitable = *iter;
try {
++iter;
} catch (...) {
if (!firstException) {
firstException = std::current_exception();
}
}
lock.unlock();
std::exception_ptr ex;
try {
co_await co_viaIfAsync(
executor.copyDummy(), static_cast<awaitable_t&&>(awaitable));
} catch (...) {
ex = std::current_exception();
}
lock =
co_await co_viaIfAsync(executor.copyDummy(), mutex.co_scoped_lock());
if (ex && !firstException) {
firstException = std::move(ex);
}
}
};
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 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);
lock = co_await mutex.co_scoped_lock();
}
} catch (...) {
workerCreationException = std::current_exception();
}
co_await detail::UnsafeResumeInlineSemiAwaitable{barrier.arriveAndWait()};
if (firstException) {
std::rethrow_exception(std::move(firstException));
} else if (workerTasks.empty() && workerCreationException) {
// Failed to create any workers to process the tasks.
std::rethrow_exception(std::move(workerCreationException));
}
}
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>>>> {
auto tryResults = co_await collectAllTryWindowed(
static_cast<InputRange&&>(awaitables), maxConcurrency);
std::vector<detail::collect_all_range_component_t<
detail::range_reference_t<InputRange>>>
results;
results.reserve(tryResults.size());
for (auto&& tryResult : tryResults) {
results.emplace_back(std::move(tryResult).value());
}
co_return std::move(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>>>> {
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.copyDummy(), 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.copyDummy(), static_cast<awaitable_t&&>(awaitable));
result.emplace();
} else {
result.emplace(co_await co_viaIfAsync(
executor.copyDummy(), 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.copyDummy(), 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 std::move(results);
}
} // namespace coro
} // namespace folly
......@@ -50,7 +50,7 @@ using collect_all_try_range_component_t =
collect_all_try_component_t<SemiAwaitable>;
template <typename Range>
using range_iterator_t = decltype(std::begin(std::declval<Range>()));
using range_iterator_t = decltype(std::begin(std::declval<Range&>()));
template <typename Iterator>
using iterator_reference_t = typename std::iterator_traits<Iterator>::reference;
......@@ -222,6 +222,88 @@ auto collectAllTryRange(std::vector<Task<T>> awaitables)
co_return co_await collectAllTryRange(awaitables | ranges::view::move);
}
///////////////////////////////////////////////////////////////////////////////
// collectAllWindowed(RangeOf<SemiAwaitable<T>>&&, size_t maxConcurrency)
// -> SemiAwaitable<std::vector<T>>
//
// collectAllWindowed(RangeOf<SemiAwaitable<void>>&&, size_t maxConcurrency)
// -> SemiAwaitable<void>
//
// Await each of the input awaitables in the range, allowing at most
// '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
// an exception then one of these exceptions 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 resulting std::vector will contain the results in the corresponding
// order of their respective awaitables in the input range.
template <
typename InputRange,
std::enable_if_t<
std::is_void_v<
semi_await_result_t<detail::range_reference_t<InputRange>>>,
int> = 0>
auto collectAllWindowed(InputRange awaitables, std::size_t maxConcurrency)
-> folly::coro::Task<void>;
template <
typename InputRange,
std::enable_if_t<
!std::is_void_v<
semi_await_result_t<detail::range_reference_t<InputRange>>>,
int> = 0>
auto collectAllWindowed(InputRange awaitables, std::size_t maxConcurrency)
-> folly::coro::Task<std::vector<detail::collect_all_range_component_t<
detail::range_reference_t<InputRange>>>>;
///////////////////////////////////////////////////////////////////////////////
// collectAllTryWindowed(RangeOf<SemiAwaitable<T>>&, size_t maxConcurrency)
// -> SemiAwaitable<std::vector<folly::Try<T>>>
//
// Concurrently awaits a collection of awaitable with bounded concurrency,
// producing a vector of Try values containing each of the results.
//
// The resulting std::vector will contain the results in the corresponding
// order of their respective awaitables in the input range.
//
// Note that the whole operation may still complete with an exception if
// iterating over the awaitables fails with an exception (eg. if you pass
// a Generator<Task<T>&&> and the generator throws an exception).
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>>>>;
// collectAllWindowed()/collectAllTryWindowed() overloads that simplify the
// use of these functions with std::vector<Task<T>>.
template <typename T>
auto collectAllWindowed(
std::vector<Task<T>> awaitables,
std::size_t maxConcurrency)
-> decltype(
collectAllWindowed(awaitables | ranges::view::move, maxConcurrency)) {
co_return co_await collectAllWindowed(
awaitables | ranges::view::move, maxConcurrency);
}
template <typename T>
auto collectAllTryWindowed(
std::vector<Task<T>> awaitables,
std::size_t maxConcurrency)
-> decltype(collectAllTryWindowed(
awaitables | ranges::view::move,
maxConcurrency)) {
co_return co_await collectAllTryWindowed(
awaitables | ranges::view::move, maxConcurrency);
}
} // namespace coro
} // namespace folly
......
......@@ -23,12 +23,15 @@
#include <folly/experimental/coro/Baton.h>
#include <folly/experimental/coro/BlockingWait.h>
#include <folly/experimental/coro/Collect.h>
#include <folly/experimental/coro/Generator.h>
#include <folly/experimental/coro/Mutex.h>
#include <folly/experimental/coro/Schedule.h>
#include <folly/experimental/coro/Task.h>
#include <folly/portability/GTest.h>
#include <numeric>
#include <string>
#include <vector>
////////////////////////////////////////////////////////
// folly::coro::collectAll() tests
......@@ -393,7 +396,7 @@ TEST(CollectAllRange, RangeOfNonVoid) {
using namespace std::literals::chrono_literals;
int x = count++;
if ((x % 20) == 0) {
co_await folly::futures::sleep(50ms);
co_await folly::coro::co_schedule;
}
co_return x;
};
......@@ -484,4 +487,250 @@ TEST(CollectAllTryRange, RangeOfValueSomeFailing) {
}());
}
////////////////////////////////////////////////////////////////////
// folly::coro::collectAllWindowed() tests
TEST(CollectAllWindowed, ConcurrentTasks) {
folly::CPUThreadPoolExecutor threadPool{
4, std::make_shared<folly::NamedThreadFactory>("TestThreadPool")};
using namespace folly::coro;
auto results = blockingWait(collectAllWindowed(
[&]() -> Generator<Task<std::string>&&> {
for (int i = 0; i < 10'000; ++i) {
co_yield[](int i)->Task<std::string> {
co_await folly::coro::co_schedule;
co_return folly::to<std::string>(i);
}
(i);
}
}(),
10));
CHECK_EQ(10'000, results.size());
for (int i = 0; i < 10'000; ++i) {
CHECK_EQ(folly::to<std::string>(i), results[i]);
}
}
TEST(CollectAllWindowed, WithGeneratorOfTaskOfValue) {
using namespace std::literals::chrono_literals;
const std::size_t maxConcurrency = 10;
std::atomic<int> activeCount{0};
std::atomic<int> completedCount{0};
auto makeTask = [&](int index) -> folly::coro::Task<int> {
auto count = ++activeCount;
CHECK_LE(count, maxConcurrency);
// Reschedule a variable number of times so that tasks may complete out of
// order.
for (int i = 0; i < index % 5; ++i) {
co_await folly::coro::co_schedule;
}
--activeCount;
++completedCount;
co_return index;
};
auto makeTaskGenerator = [&]()
-> folly::coro::Generator<folly::coro::Task<int>&&> {
for (int i = 0; i < 100; ++i) {
co_yield makeTask(i);
}
};
folly::coro::blockingWait([&]() -> folly::coro::Task<void> {
auto results = co_await folly::coro::collectAllWindowed(
makeTaskGenerator(), maxConcurrency);
CHECK_EQ(100, results.size());
for (int i = 0; i < 100; ++i) {
CHECK_EQ(i, results[i]);
}
}());
CHECK_EQ(0, activeCount.load());
CHECK_EQ(100, completedCount);
}
TEST(CollectAllWindowed, WithGeneratorOfTaskOfVoid) {
using namespace std::literals::chrono_literals;
const std::size_t maxConcurrency = 10;
std::atomic<int> activeCount{0};
std::atomic<int> completedCount{0};
auto makeTask = [&]() -> folly::coro::Task<void> {
auto count = ++activeCount;
CHECK_LE(count, maxConcurrency);
co_await folly::coro::co_schedule;
--activeCount;
++completedCount;
};
auto makeTaskGenerator = [&]()
-> folly::coro::Generator<folly::coro::Task<void>&&> {
for (int i = 0; i < 100; ++i) {
co_yield makeTask();
}
};
folly::coro::blockingWait([&]() -> folly::coro::Task<void> {
co_await folly::coro::collectAllWindowed(
makeTaskGenerator(), maxConcurrency);
}());
CHECK_EQ(0, activeCount.load());
CHECK_EQ(100, completedCount);
}
TEST(CollectAllWindowed, VectorOfVoidTask) {
using namespace std::literals::chrono_literals;
int count = 0;
auto makeTask = [&]() -> folly::coro::Task<void> {
co_await folly::coro::co_schedule;
++count;
};
std::vector<folly::coro::Task<void>> tasks;
for (int i = 0; i < 10; ++i) {
tasks.push_back(makeTask());
}
folly::coro::blockingWait(
folly::coro::collectAllWindowed(std::move(tasks), 5));
CHECK_EQ(10, count);
}
TEST(CollectAllWindowed, VectorOfValueTask) {
using namespace std::literals::chrono_literals;
int count = 0;
auto makeTask = [&](int i) -> folly::coro::Task<std::unique_ptr<int>> {
co_await folly::coro::co_schedule;
++count;
co_return std::make_unique<int>(i);
};
std::vector<folly::coro::Task<std::unique_ptr<int>>> tasks;
for (int i = 0; i < 10; ++i) {
tasks.push_back(makeTask(i));
}
auto results = folly::coro::blockingWait(
folly::coro::collectAllWindowed(std::move(tasks), 5));
CHECK_EQ(10, count);
CHECK_EQ(10, results.size());
for (int i = 0; i < 10; ++i) {
CHECK_EQ(i, *results[i]);
}
}
TEST(CollectAllWindowed, PartialFailure) {
try {
[[maybe_unused]] auto results =
folly::coro::blockingWait(folly::coro::collectAllWindowed(
[]() -> folly::coro::Generator<folly::coro::Task<int>&&> {
for (int i = 0; i < 10; ++i) {
co_yield[](int i)->folly::coro::Task<int> {
using namespace std::literals::chrono_literals;
if (i == 3) {
co_await folly::coro::co_schedule;
co_await folly::coro::co_schedule;
throw ErrorA{};
} else if (i == 7) {
co_await folly::coro::co_schedule;
throw ErrorB{};
}
co_return i;
}
(i);
}
}(),
5));
CHECK(false); // Should have thrown.
} catch (ErrorA) {
// Expected.
} catch (ErrorB) {
// Expected.
}
}
////////////////////////////////////////////////////////////////////
// folly::coro::collectAllTryWindowed() tests
TEST(CollectAllTryWindowed, PartialFailure) {
auto results = folly::coro::blockingWait(folly::coro::collectAllTryWindowed(
[]() -> folly::coro::Generator<folly::coro::Task<int>&&> {
for (int i = 0; i < 10; ++i) {
co_yield[](int i)->folly::coro::Task<int> {
using namespace std::literals::chrono_literals;
if (i == 3) {
co_await folly::coro::co_schedule;
co_await folly::coro::co_schedule;
throw ErrorA{};
} else if (i == 7) {
co_await folly::coro::co_schedule;
throw ErrorB{};
}
co_return i;
}
(i);
}
}(),
5));
CHECK_EQ(10, results.size());
for (int i = 0; i < 10; ++i) {
if (i == 3) {
CHECK(results[i].hasException());
CHECK(results[i].exception().is_compatible_with<ErrorA>());
} else if (i == 7) {
CHECK(results[i].hasException());
CHECK(results[i].exception().is_compatible_with<ErrorB>());
} else {
CHECK(results[i].hasValue());
CHECK_EQ(i, results[i].value());
}
}
}
TEST(CollectAllTryWindowed, GeneratorFailure) {
int active = 0;
int started = 0;
auto makeTask = [&](int i) -> folly::coro::Task<void> {
++active;
++started;
for (int j = 0; j < (i % 3); ++j) {
co_await folly::coro::co_schedule;
}
--active;
};
auto generateTasks = [&]()
-> folly::coro::Generator<folly::coro::Task<void>&&> {
for (int i = 0; i < 10; ++i) {
co_yield makeTask(i);
}
throw ErrorA{};
};
try {
[[maybe_unused]] auto results = folly::coro::blockingWait(
folly::coro::collectAllTryWindowed(generateTasks(), 5));
CHECK(false);
} catch (ErrorA) {
}
// Even if the generator throws an exception we should still have launched
// and waited for completion all of the prior tasks in the sequence.
CHECK_EQ(10, started);
CHECK_EQ(0, active);
}
#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