From e256f40156811169e2725a969212d1677f16ffff Mon Sep 17 00:00:00 2001 From: Theodore Tsirpanis Date: Thu, 13 Jun 2024 15:05:29 +0300 Subject: [PATCH 01/11] Remove `Status` from the thread pool. --- .../thread_pool/test/unit_thread_pool.cc | 160 +++--------------- tiledb/common/thread_pool/thread_pool.cc | 74 ++------ tiledb/common/thread_pool/thread_pool.h | 65 +++---- tiledb/sm/misc/cancelable_tasks.cc | 17 +- tiledb/sm/misc/cancelable_tasks.h | 9 +- 5 files changed, 74 insertions(+), 251 deletions(-) diff --git a/tiledb/common/thread_pool/test/unit_thread_pool.cc b/tiledb/common/thread_pool/test/unit_thread_pool.cc index 5220f9be8a9..52e0cb25e00 100644 --- a/tiledb/common/thread_pool/test/unit_thread_pool.cc +++ b/tiledb/common/thread_pool/test/unit_thread_pool.cc @@ -100,57 +100,13 @@ void wait_all( ThreadPool& pool, bool use_wait, std::vector& results) { if (use_wait) { for (auto& r : results) { - REQUIRE(pool.wait(r).ok()); + REQUIRE_NOTHROW(pool.wait(r)); } } else { - REQUIRE(pool.wait_all(results).ok()); + REQUIRE_NOTHROW(pool.wait_all(results)); } } -/** - * Use the wait or wait_all function to wait on all status. - * - * @return First failed status code or success. - */ -Status wait_all_status( - ThreadPool& pool, bool use_wait, std::vector& results) { - if (use_wait) { - Status ret; - for (auto& r : results) { - auto st = pool.wait(r); - if (ret.ok() && !st.ok()) { - ret = st; - } - } - - return ret; - } else { - return pool.wait_all(results); - } -} - -/** - * Use the wait or wait_all function to wait on all status. - * - * @return Number of successes. - */ -uint64_t wait_all_num_status( - ThreadPool& pool, bool use_wait, std::vector& results) { - int num_ok = 0; - if (use_wait) { - for (auto& r : results) { - num_ok += pool.wait(r).ok() ? 1 : 0; - } - } else { - std::vector statuses = pool.wait_all_status(results); - for (const auto& st : statuses) { - num_ok += st.ok() ? 1 : 0; - } - } - - return num_ok; -} - TEST_CASE("ThreadPool: Test empty", "[threadpool]") { for (int i = 0; i < 10; i++) { ThreadPool pool{4}; @@ -165,10 +121,7 @@ TEST_CASE("ThreadPool: Test single thread", "[threadpool]") { ThreadPool pool{1}; for (int i = 0; i < 100; i++) { - ThreadPool::Task task = pool.execute([&result]() { - result++; - return Status::Ok(); - }); + ThreadPool::Task task = pool.execute([&result]() { result++; }); REQUIRE(task.valid()); @@ -184,30 +137,12 @@ TEST_CASE("ThreadPool: Test multiple threads", "[threadpool]") { std::vector results; ThreadPool pool{4}; for (int i = 0; i < 100; i++) { - results.push_back(pool.execute([&result]() { - result++; - return Status::Ok(); - })); + results.push_back(pool.execute([&result]() { result++; })); } wait_all(pool, use_wait, results); REQUIRE(result == 100); } -TEST_CASE("ThreadPool: Test wait status", "[threadpool]") { - bool use_wait = GENERATE(true, false); - std::atomic result(0); - std::vector results; - ThreadPool pool{4}; - for (int i = 0; i < 100; i++) { - results.push_back(pool.execute([&result, i]() { - result++; - return i == 50 ? Status_Error("Generic error") : Status::Ok(); - })); - } - REQUIRE(wait_all_num_status(pool, use_wait, results) == 99); - REQUIRE(result == 100); -} - struct AtomicHolder { AtomicHolder(int val) : val_(val) { @@ -223,7 +158,6 @@ TEST_CASE("ThreadPool: Test no wait", "[threadpool]") { ThreadPool::Task task = pool.execute([result = ptr]() { result->val_++; std::this_thread::sleep_for(std::chrono::milliseconds(random_ms(1000))); - return Status::Ok(); }); REQUIRE(task.valid()); } @@ -234,7 +168,6 @@ TEST_CASE("ThreadPool: Test no wait", "[threadpool]") { TEST_CASE( "ThreadPool: Test pending task cancellation", "[threadpool][cancel]") { - bool use_wait = GENERATE(true, false); SECTION("- No cancellation callback") { ThreadPool pool{2}; @@ -247,17 +180,12 @@ TEST_CASE( tasks.push_back(cancelable_tasks.execute(&pool, [&result]() { std::this_thread::sleep_for(std::chrono::seconds(2)); result++; - return Status::Ok(); })); } // Because the thread pool has 2 threads, the first two will probably be // executing at this point, but some will still be queued. - cancelable_tasks.cancel_all_tasks(); - - // The result is the number of threads that returned Ok (were not - // cancelled). - REQUIRE(result == wait_all_num_status(pool, use_wait, tasks)); + REQUIRE_THROWS(cancelable_tasks.cancel_all_tasks()); } SECTION("- With cancellation callback") { @@ -272,18 +200,13 @@ TEST_CASE( [&result]() { std::this_thread::sleep_for(std::chrono::seconds(2)); result++; - return Status::Ok(); }, [&num_cancelled]() { num_cancelled++; })); } // Because the thread pool has 2 threads, the first two will probably be // executing at this point, but some will still be queued. - cancelable_tasks.cancel_all_tasks(); - - // The result is the number of threads that returned Ok (were not - // cancelled). - REQUIRE(result == wait_all_num_status(pool, use_wait, tasks)); + REQUIRE_THROWS(cancelable_tasks.cancel_all_tasks()); REQUIRE(num_cancelled == ((int64_t)tasks.size() - result)); } } @@ -311,12 +234,10 @@ TEST_CASE("ThreadPool: Test recursion, simplest case", "[threadpool]") { auto b = pool.execute([&result]() { std::this_thread::sleep_for(std::chrono::milliseconds(100)); ++result; - return Status::Ok(); }); REQUIRE(b.valid()); tasks.emplace_back(std::move(b)); wait_all(pool, use_wait, tasks); - return Status::Ok(); }); REQUIRE(a.valid()); tasks.emplace_back(std::move(a)); @@ -353,14 +274,12 @@ TEST_CASE("ThreadPool: Test recursion", "[threadpool]") { auto inner_task = pool.execute([&]() { std::this_thread::sleep_for(std::chrono::milliseconds(random_ms())); ++result; - return Status::Ok(); }); inner_tasks.emplace_back(std::move(inner_task)); } wait_all(pool, use_wait, inner_tasks); - return Status::Ok(); }); REQUIRE(task.valid()); @@ -383,11 +302,8 @@ TEST_CASE("ThreadPool: Test recursion", "[threadpool]") { if (--result == 0) { cv.notify_all(); } - return Status::Ok(); }); } - - return Status::Ok(); }); REQUIRE(task.valid()); @@ -445,21 +361,18 @@ TEST_CASE("ThreadPool: Test recursion, two pools", "[threadpool]") { std::this_thread::sleep_for( std::chrono::milliseconds(random_ms())); ++result; - return Status::Ok(); }); tasks_c.emplace_back(std::move(task_c)); } wait_all(pool_a, use_wait, tasks_c); - return Status::Ok(); }); tasks_b.emplace_back(std::move(task_b)); } wait_all(pool_b, use_wait, tasks_b); - return Status::Ok(); }); REQUIRE(task_a.valid()); @@ -487,21 +400,18 @@ TEST_CASE("ThreadPool: Test recursion, two pools", "[threadpool]") { std::unique_lock ul(cv_mutex); cv.notify_all(); } - return Status::Ok(); }); tasks_c.emplace_back(std::move(task_c)); } wait_all(pool_a, use_wait, tasks_c); - return Status::Ok(); }); tasks_b.emplace_back(std::move(task_b)); } wait_all(pool_b, use_wait, tasks_b); - return Status::Ok(); }); REQUIRE(task_a.valid()); @@ -533,14 +443,10 @@ TEST_CASE("ThreadPool: Test Exceptions", "[threadpool]") { if (tmp == 13) { throw(std::string("Unripe banana")); } - return Status::Ok(); })); } - REQUIRE( - wait_all_status(pool, use_wait, results).to_string() == - unripe_banana_status.to_string()); - REQUIRE(result == 207); + REQUIRE_THROWS_WITH(wait_all(pool, use_wait, results), "Unripe banana"); } SECTION("One tile error exception") { @@ -550,16 +456,14 @@ TEST_CASE("ThreadPool: Test Exceptions", "[threadpool]") { results.push_back(pool.execute([&result, &unbaked_potato_status]() { auto tmp = result++; if (tmp == 31) { - throw(unbaked_potato_status); + throw StatusException(unbaked_potato_status); } - return Status::Ok(); })); } - REQUIRE( - wait_all_status(pool, use_wait, results).to_string() == - unbaked_potato_status.to_string()); - REQUIRE(result == 207); + REQUIRE_THROWS_WITH( + wait_all(pool, use_wait, results), + Catch::Matchers::EndsWith("Unbaked potato")); } SECTION("Two exceptions") { @@ -574,16 +478,13 @@ TEST_CASE("ThreadPool: Test Exceptions", "[threadpool]") { if (tmp == 31) { throw(Status_TileError("Unbaked potato")); } - - return Status::Ok(); })); } - auto pool_status = wait_all_status(pool, use_wait, results); - REQUIRE( - ((pool_status.to_string() == unripe_banana_status.to_string()) || - (pool_status.to_string() == unbaked_potato_status.to_string()))); - REQUIRE(result == 207); + REQUIRE_THROWS_WITH( + wait_all(pool, use_wait, results), + Catch::Matchers::Equals("Unripe banana") || + Catch::Matchers::Equals("Unbaked potato")); } SECTION("Two exceptions reverse order") { @@ -598,16 +499,13 @@ TEST_CASE("ThreadPool: Test Exceptions", "[threadpool]") { if (tmp == 13) { throw(Status_TileError("Unbaked potato")); } - - return Status::Ok(); })); } - auto pool_status = wait_all_status(pool, use_wait, results); - REQUIRE( - ((pool_status.to_string() == unripe_banana_status.to_string()) || - (pool_status.to_string() == unbaked_potato_status.to_string()))); - REQUIRE(result == 207); + REQUIRE_THROWS_WITH( + wait_all(pool, use_wait, results), + Catch::Matchers::Equals("Unripe banana") || + Catch::Matchers::Equals("Unbaked potato")); } SECTION("Two exceptions strict order") { @@ -622,15 +520,13 @@ TEST_CASE("ThreadPool: Test Exceptions", "[threadpool]") { if (i == 31) { throw(Status_TileError("Unbaked potato")); } - - return Status::Ok(); })); } - REQUIRE( - wait_all_status(pool, use_wait, results).to_string() == - unripe_banana_status.to_string()); - REQUIRE(result == 207); + REQUIRE_THROWS_WITH( + wait_all(pool, use_wait, results), + Catch::Matchers::Equals("Unripe banana") || + Catch::Matchers::Equals("Unbaked potato")); } SECTION("Two exceptions strict reverse order") { @@ -645,14 +541,12 @@ TEST_CASE("ThreadPool: Test Exceptions", "[threadpool]") { if (i == 13) { throw(Status_TileError("Unbaked potato")); } - - return Status::Ok(); })); } - REQUIRE( - wait_all_status(pool, use_wait, results).to_string() == - unbaked_potato_status.to_string()); - REQUIRE(result == 207); + REQUIRE_THROWS_WITH( + wait_all(pool, use_wait, results), + Catch::Matchers::Equals("Unripe banana") || + Catch::Matchers::Equals("Unbaked potato")); } } diff --git a/tiledb/common/thread_pool/thread_pool.cc b/tiledb/common/thread_pool/thread_pool.cc index b026262a3c3..c410bc5ecc4 100644 --- a/tiledb/common/thread_pool/thread_pool.cc +++ b/tiledb/common/thread_pool/thread_pool.cc @@ -121,16 +121,6 @@ void ThreadPool::shutdown() { threads_.clear(); } -Status ThreadPool::wait_all(std::vector& tasks) { - auto statuses = wait_all_status(tasks); - for (auto& st : statuses) { - if (!st.ok()) { - return st; - } - } - return Status::Ok(); -} - // Return a vector of Status. If any task returns an error value or throws an // exception, we save an error code in the corresponding location in the Status // vector. All tasks are waited on before return. Multiple error statuses may @@ -138,13 +128,11 @@ Status ThreadPool::wait_all(std::vector& tasks) { // context is fully constructed (which will include logger). // Unfortunately, C++ does not have the notion of an aggregate exception, so we // don't throw in the case of errors/exceptions. -std::vector ThreadPool::wait_all_status(std::vector& tasks) { - std::vector statuses(tasks.size()); - +void ThreadPool::wait_all(std::vector& tasks) { std::queue pending_tasks; // Create queue of ids of all the pending tasks for processing - for (size_t i = 0; i < statuses.size(); ++i) { + for (size_t i = 0; i < tasks.size(); ++i) { pending_tasks.push(i); } @@ -155,33 +143,12 @@ std::vector ThreadPool::wait_all_status(std::vector& tasks) { auto& task = tasks[task_id]; if (!task.valid()) { - statuses[task_id] = Status_ThreadPoolError("Invalid task future"); - LOG_STATUS_NO_RETURN_VALUE(statuses[task_id]); + throw TaskException("Invalid task future"); } else if ( task.wait_for(std::chrono::milliseconds(0)) == std::future_status::ready) { - // Task is completed, get result, handling possible exceptions - - Status st = [&task] { - try { - return task.get(); - } catch (const std::exception& e) { - return Status_TaskError( - "Caught std::exception: " + std::string(e.what())); - } catch (const std::string& msg) { - return Status_TaskError("Caught msg: " + msg); - } catch (const Status& stat) { - return stat; - } catch (...) { - return Status_TaskError("Unknown exception"); - } - }(); - - if (!st.ok()) { - LOG_STATUS_NO_RETURN_VALUE(st); - } - statuses[task_id] = st; - + // Task is completed, throw possible exception + task.get(); } else { // If the task is not completed, try again later pending_tasks.push(task_id); @@ -201,39 +168,18 @@ std::vector ThreadPool::wait_all_status(std::vector& tasks) { } } } - - return statuses; } -Status ThreadPool::wait(Task& task) { +void ThreadPool::wait(Task& task) { while (true) { if (!task.valid()) { - return Status_ThreadPoolError("Invalid task future"); + throw TaskException("Invalid task future"); } else if ( task.wait_for(std::chrono::milliseconds(0)) == std::future_status::ready) { - // Task is completed, get result, handling possible exceptions - - Status st = [&task] { - try { - return task.get(); - } catch (const std::exception& e) { - return Status_TaskError( - "Caught std::exception: " + std::string(e.what())); - } catch (const std::string& msg) { - return Status_TaskError("Caught msg: " + msg); - } catch (const Status& stat) { - return stat; - } catch (...) { - return Status_TaskError("Unknown exception"); - } - }(); - - if (!st.ok()) { - LOG_STATUS_NO_RETURN_VALUE(st); - } - - return st; + // Task is completed, throw possible exception + task.get(); + return; } else { // In the meantime, try to do something useful to make progress (and avoid // deadlock) diff --git a/tiledb/common/thread_pool/thread_pool.h b/tiledb/common/thread_pool/thread_pool.h index e0f804c7278..146a5d4d628 100644 --- a/tiledb/common/thread_pool/thread_pool.h +++ b/tiledb/common/thread_pool/thread_pool.h @@ -35,19 +35,27 @@ #include "producer_consumer_queue.h" +#include #include #include #include "tiledb/common/common.h" #include "tiledb/common/logger_public.h" #include "tiledb/common/macros.h" -#include "tiledb/common/status.h" namespace tiledb::common { +/** Class for Task status exceptions. */ +class TaskException : public StatusException { + public: + explicit TaskException(const std::string& msg) + : StatusException("Task", msg) { + } +}; + class ThreadPool { public: - using Task = std::future; + using Task = std::future; /* ********************************* */ /* CONSTRUCTORS & DESTRUCTORS */ @@ -92,26 +100,19 @@ class ThreadPool { */ template - auto async(Fn&& f, Args&&... args) { + Task async(Fn&& f, Args&&... args) + requires std::same_as...>, void> + { if (concurrency_level_ == 0) { Task invalid_future; LOG_ERROR("Cannot execute task; thread pool uninitialized."); return invalid_future; } - using R = std::invoke_result_t, std::decay_t...>; - - auto task = make_shared>( - HERE(), - [f = std::forward(f), - args = std::make_tuple(std::forward(args)...)]() mutable { - return std::apply(std::move(f), std::move(args)); - }); - - std::future future = task->get_future(); - + auto task = make_shared>( + HERE(), std::bind(std::forward(f), std::forward(args)...)); + auto future = task->get_future(); task_queue_.push(task); - return future; } @@ -123,7 +124,9 @@ class ThreadPool { * @return std::future referring to the shared state created by this call */ template - auto execute(Fn&& f, Args&&... args) { + Task execute(Fn&& f, Args&&... args) + requires std::same_as...>, void> + { return async(std::forward(f), std::forward(args)...); } @@ -133,25 +136,11 @@ class ThreadPool { * waiting. * * @param tasks Task list to wait on. - * @return Status::Ok if all tasks returned Status::Ok, otherwise the first - * error status is returned - */ - Status wait_all(std::vector& tasks); - - /** - * Wait on all the given tasks to complete, returning a vector of their return - * Status. Exceptions caught while waiting are returned as Status_TaskError. - * Status are saved at the same index in the return vector as the - * corresponding task in the input vector. The status vector may contain more - * than one error Status. - * - * This function is safe to call recursively and may execute pending tasks - * with the calling thread while waiting. * - * @param tasks Task list to wait on - * @return Vector of each task's Status. + * @throws This function will throw the first exception thrown by one of the + * tasks. */ - std::vector wait_all_status(std::vector& tasks); + void wait_all(std::vector& tasks); /** * Wait on a single tasks to complete. This function is safe to call @@ -159,10 +148,10 @@ class ThreadPool { * waiting. * * @param task Task to wait on. - * @return Status::Ok if the task returned Status::Ok, otherwise the error - * status is returned + * + * @throws This function will throw the exception thrown by task. */ - Status wait(Task& task); + void wait(Task& task); /* ********************************* */ /* PRIVATE ATTRIBUTES */ @@ -177,8 +166,8 @@ class ThreadPool { /** Producer-consumer queue where functions to be executed are kept */ ProducerConsumerQueue< - shared_ptr>, - std::deque>>> + shared_ptr>, + std::deque>>> task_queue_; /** The worker threads */ diff --git a/tiledb/sm/misc/cancelable_tasks.cc b/tiledb/sm/misc/cancelable_tasks.cc index 960f98da05d..7ddfd2124ec 100644 --- a/tiledb/sm/misc/cancelable_tasks.cc +++ b/tiledb/sm/misc/cancelable_tasks.cc @@ -44,12 +44,10 @@ CancelableTasks::CancelableTasks() ThreadPool::Task CancelableTasks::execute( ThreadPool* const thread_pool, - std::function&& fn, + std::function&& fn, std::function&& on_cancel) { - std::function wrapped_fn = - std::bind(&CancelableTasks::fn_wrapper, this, fn, on_cancel); - - ThreadPool::Task task = thread_pool->execute(std::move(wrapped_fn)); + ThreadPool::Task task = + thread_pool->execute(&CancelableTasks::fn_wrapper, this, fn, on_cancel); if (task.valid()) { std::unique_lock lck(outstanding_tasks_mutex_); ++outstanding_tasks_; @@ -73,8 +71,8 @@ void CancelableTasks::cancel_all_tasks() { } } -Status CancelableTasks::fn_wrapper( - const std::function& fn, const std::function& on_cancel) { +void CancelableTasks::fn_wrapper( + const std::function& fn, const std::function& on_cancel) { std::unique_lock lck(outstanding_tasks_mutex_); if (should_cancel_) { if (on_cancel) { @@ -85,10 +83,10 @@ Status CancelableTasks::fn_wrapper( if (--outstanding_tasks_ == 0) { outstanding_tasks_cv_.notify_all(); } - return Status_Error("Task cancelled before execution."); + throw TaskException("Task cancelled before execution."); } else { lck.unlock(); - Status st = fn(); + fn(); lck.lock(); --outstanding_tasks_; // If 'should_cancel_' became true when the lock was released to execute @@ -96,7 +94,6 @@ Status CancelableTasks::fn_wrapper( if (should_cancel_ && outstanding_tasks_ == 0) { outstanding_tasks_cv_.notify_all(); } - return st; } } diff --git a/tiledb/sm/misc/cancelable_tasks.h b/tiledb/sm/misc/cancelable_tasks.h index fc38c39d65c..286c154b9bb 100644 --- a/tiledb/sm/misc/cancelable_tasks.h +++ b/tiledb/sm/misc/cancelable_tasks.h @@ -65,7 +65,7 @@ class CancelableTasks { */ ThreadPool::Task execute( ThreadPool* thread_pool, - std::function&& fn, + std::function&& fn, std::function&& on_cancel = nullptr); /** @@ -81,12 +81,9 @@ class CancelableTasks { * * @param function Task to be executed. * @param function Optional routine to execute on cancelation. - * @return Status The returned status from 'fn', or a non-OK status if tasks - * were cancelled. */ - Status fn_wrapper( - const std::function& fn, - const std::function& on_cancel); + void fn_wrapper( + const std::function& fn, const std::function& on_cancel); /** The number of outstanding tasks */ uint32_t outstanding_tasks_; From c6bc7afe504bec13e0c043709b7288cdb29968fd Mon Sep 17 00:00:00 2001 From: Theodore Tsirpanis Date: Thu, 13 Jun 2024 17:48:38 +0300 Subject: [PATCH 02/11] Remove `Status` from `parallel_functions`. --- tiledb/sm/misc/parallel_functions.h | 153 +++++++--------------------- 1 file changed, 38 insertions(+), 115 deletions(-) diff --git a/tiledb/sm/misc/parallel_functions.h b/tiledb/sm/misc/parallel_functions.h index 4cfc6ff141a..1a934d76e31 100644 --- a/tiledb/sm/misc/parallel_functions.h +++ b/tiledb/sm/misc/parallel_functions.h @@ -91,20 +91,20 @@ void parallel_sort( // Define a work routine that encapsulates steps #1 - #3 in the // algorithm. - std::function quick_sort; - quick_sort = [&](const uint64_t depth, IterT begin, IterT end) -> Status { + std::function quick_sort; + quick_sort = [&](const uint64_t depth, IterT begin, IterT end) { const size_t elements = std::distance(begin, end); // Stop the recursion if this subrange does not contain // any elements to sort. if (elements <= 1) { - return Status::Ok(); + return; } // If there are only two elements remaining, directly sort them. if (elements <= 2) { std::sort(begin, end, cmp); - return Status::Ok(); + return; } // If we have reached the target height of the call stack tree, @@ -114,7 +114,7 @@ void parallel_sort( // evenly distributed among them. if (depth + 1 == height) { std::sort(begin, end, cmp); - return Status::Ok(); + return; } // Step #1: Pick a pivot value in the range. @@ -136,63 +136,43 @@ void parallel_sort( // Step #3: Recursively sort the left and right partitions. std::vector tasks; if (begin != middle) { - std::function quick_sort_left = - std::bind(quick_sort, depth + 1, begin, middle); - ThreadPool::Task left_task = tp->execute(std::move(quick_sort_left)); - tasks.emplace_back(std::move(left_task)); + tasks.emplace_back(tp->execute(quick_sort, depth + 1, begin, middle)); } if (middle != end) { - std::function quick_sort_right = - std::bind(quick_sort, depth + 1, middle + 1, end); - ThreadPool::Task right_task = tp->execute(std::move(quick_sort_right)); - tasks.emplace_back(std::move(right_task)); + tasks.emplace_back(tp->execute(quick_sort, depth + 1, middle + 1, end)); } // Wait for the sorted partitions. - return tp->wait_all(tasks); + tp->wait_all(tasks); }; // Start the quicksort from the entire range. - throw_if_not_ok(quick_sort(0, begin, end)); + quick_sort(0, begin, end); } /** * Call the given function on each element in the given iterator range. * * @tparam IterT Iterator type - * @tparam FuncT Function type (returning Status). + * @tparam FuncT Function type. * @param tp The threadpool to use. * @param begin Beginning of range (inclusive). * @param end End of range (exclusive). * @param F Function to call on each item - * @return Status */ template -Status parallel_for( - ThreadPool* const tp, uint64_t begin, uint64_t end, const FuncT& F) { +void parallel_for( + ThreadPool* const tp, uint64_t begin, uint64_t end, const FuncT& F) + requires std::same_as, void> +{ assert(begin <= end); const uint64_t range_len = end - begin; if (range_len == 0) - return Status::Ok(); + return; assert(tp); - /* - * Mutex protects atomicity of `failed*` local variables together. The first - * subrange to fail determines the return or exception value. - */ - std::mutex failed_mutex; - /* - * If we were checking this variable inside either the main loop or the one in - * `execute_subrange`, then it would be better to use `atomic_bool` to lessen - * the lock overhead on the mutex. As it is, we do not prematurely stop any - * loop that is not the first to fail. - */ - bool failed = false; - optional failed_exception{nullopt}; - optional failed_status{nullopt}; - /* * Executes subrange [subrange_start, subrange_end) that exists within the * range [begin, end). @@ -203,43 +183,12 @@ Status parallel_for( * returns not OK, then this function returns that value. If a first function * to fail throws, then this function throws that value. */ - std::function execute_subrange = - [&failed, &failed_exception, &failed_status, &failed_mutex, &F]( - const uint64_t subrange_start, - const uint64_t subrange_end) -> Status { - for (uint64_t i = subrange_start; i < subrange_end; ++i) { - Status st; - try { - st = F(i); - if (st.ok()) { - continue; - } - std::lock_guard lock(failed_mutex); - if (!failed) { - failed_status = st; - failed = true; - return st; + auto execute_subrange = + [&F](const uint64_t subrange_start, const uint64_t subrange_end) { + for (uint64_t i = subrange_start; i < subrange_end; ++i) { + F(i); } - } catch (...) { - std::lock_guard lock(failed_mutex); - if (!failed) { - auto ce{std::current_exception()}; - failed_exception = ce; - failed = true; - std::rethrow_exception(ce); - } - } - /* - * If we reach this line, then either the status was not OK or `F` threw. - * Now you'd think that we'd do something other than continue the loop in - * this case, like `break` and end the function. Nope. That's not the - * legacy behavior of this function. Nor is checking failure status in the - * loop that kicks off this function. Regardless, we are leaving the - * behavior exactly the same for now. - */ - } - return Status{}; - }; + }; // Calculate the length of the subrange that each thread will // be responsible for. @@ -261,48 +210,38 @@ Status parallel_for( const uint64_t subrange_start = begin + fn_iter; const uint64_t subrange_end = begin + fn_iter + task_subrange_len; - std::function bound_fn = - std::bind(execute_subrange, subrange_start, subrange_end); - tasks.emplace_back(tp->execute(std::move(bound_fn))); + tasks.emplace_back( + tp->execute(execute_subrange, subrange_start, subrange_end)); fn_iter += task_subrange_len; } // Wait for all instances of `execute_subrange` to complete. - // This is ignoring the wait status as we use failed_exception for propagating - // the tasks exceptions. - (void)tp->wait_all(tasks); - - if (failed_exception.has_value()) { - std::rethrow_exception(failed_exception.value()); - } - if (failed_status.has_value()) { - return failed_status.value(); - } - return Status{}; // otherwise return OK + tp->wait_all(tasks); } /** * Call the given function on every pair (i, j) in the given i and j ranges, * possibly in parallel. * - * @tparam FuncT Function type (returning Status). + * @tparam FuncT Function type. * @param tp The threadpool to use. * @param i0 Inclusive start of outer (rows) range. * @param i1 Exclusive end of outer range. * @param j0 Inclusive start of inner (cols) range. * @param j1 Exclusive end of inner range. * @param F Function to call on each (i, j) pair. - * @return Status */ template -Status parallel_for_2d( +void parallel_for_2d( ThreadPool* const tp, uint64_t i0, uint64_t i1, uint64_t j0, uint64_t j1, - const FuncT& F) { + const FuncT& F) + requires std::same_as, void> +{ assert(i0 <= i1); assert(j0 <= j1); @@ -312,11 +251,7 @@ Status parallel_for_2d( const uint64_t range_len_j = j1 - j0; if (range_len_i == 0 || range_len_j == 0) - return Status::Ok(); - - bool failed = false; - Status return_st = Status::Ok(); - std::mutex return_st_mutex; + return; // Calculate the length of the subrange-i and subrange-j that // each thread will be responsible for. @@ -328,24 +263,16 @@ Status parallel_for_2d( // Executes subarray [begin_i, end_i) x [start_j, end_j) within the // array [i0, i1) x [j0, j1). - std::function - execute_subrange_ij = [&failed, &return_st, &return_st_mutex, &F]( - const uint64_t begin_i, - const uint64_t end_i, - const uint64_t start_j, - const uint64_t end_j) -> Status { + auto execute_subrange_ij = [&F]( + const uint64_t begin_i, + const uint64_t end_i, + const uint64_t start_j, + const uint64_t end_j) { for (uint64_t i = begin_i; i < end_i; ++i) { for (uint64_t j = start_j; j < end_j; ++j) { - const Status st = F(i, j); - if (!st.ok() && !failed) { - failed = true; - std::lock_guard lock(return_st_mutex); - return_st = st; - } + F(i, j); } } - - return Status::Ok(); }; // Calculate the subranges for each dimension, i and j. @@ -383,21 +310,17 @@ Status parallel_for_2d( tasks.reserve(concurrency_level * concurrency_level); for (const auto& subrange_i : subranges_i) { for (const auto& subrange_j : subranges_j) { - std::function bound_fn = std::bind( + tasks.emplace_back(tp->execute( execute_subrange_ij, subrange_i.first, subrange_i.second, subrange_j.first, - subrange_j.second); - tasks.emplace_back(tp->execute(std::move(bound_fn))); + subrange_j.second)); } } // Wait for all instances of `execute_subrange` to complete. - auto wait_status = tp->wait_all(tasks); - if (!wait_status.ok()) - return wait_status; - return return_st; + tp->wait_all(tasks); } } // namespace tiledb::sm From 0eadbe5f703e8f5628dd0cc3235ef63473c8b849 Mon Sep 17 00:00:00 2001 From: Theodore Tsirpanis Date: Thu, 13 Jun 2024 17:48:49 +0300 Subject: [PATCH 03/11] Update usages. --- tiledb/sm/array/array.cc | 44 +- tiledb/sm/array/array_directory.cc | 157 +++--- tiledb/sm/array/array_operations.cc | 68 +-- .../sm/consolidator/fragment_consolidator.cc | 31 +- .../fragment_meta_consolidator.cc | 38 +- tiledb/sm/filesystem/azure.cc | 34 +- tiledb/sm/filesystem/azure.h | 3 +- tiledb/sm/filesystem/gcs.cc | 44 +- tiledb/sm/filesystem/gcs.h | 6 +- tiledb/sm/filesystem/s3.cc | 156 +++--- .../sm/filesystem/s3_thread_pool_executor.cc | 6 +- tiledb/sm/filesystem/vfs.cc | 25 +- tiledb/sm/filter/filter_pipeline.cc | 12 +- tiledb/sm/fragment/fragment_info.cc | 20 +- tiledb/sm/fragment/fragment_metadata.cc | 278 +++++----- tiledb/sm/group/group.cc | 23 +- tiledb/sm/group/group_directory.cc | 17 +- .../array_dimension_label_queries.cc | 10 +- tiledb/sm/query/legacy/reader.cc | 157 +++--- tiledb/sm/query/legacy/reader.h | 6 +- tiledb/sm/query/query.cc | 5 +- tiledb/sm/query/readers/dense_reader.cc | 58 +- tiledb/sm/query/readers/dense_reader.h | 11 +- tiledb/sm/query/readers/filtered_data.h | 10 +- .../query/readers/ordered_dim_label_reader.cc | 28 +- tiledb/sm/query/readers/reader_base.cc | 89 ++- .../readers/sparse_global_order_reader.cc | 512 ++++++++---------- .../query/readers/sparse_index_reader_base.cc | 24 +- .../sparse_unordered_with_dups_reader.cc | 146 +++-- .../sm/query/writers/global_order_writer.cc | 33 +- tiledb/sm/query/writers/ordered_writer.cc | 89 ++- tiledb/sm/query/writers/unordered_writer.cc | 30 +- tiledb/sm/query/writers/writer_base.cc | 134 ++--- .../subarray/relevant_fragment_generator.cc | 10 +- tiledb/sm/subarray/subarray.cc | 168 +++--- 35 files changed, 1057 insertions(+), 1425 deletions(-) diff --git a/tiledb/sm/array/array.cc b/tiledb/sm/array/array.cc index d99845dad86..afb95b2f393 100644 --- a/tiledb/sm/array/array.cc +++ b/tiledb/sm/array/array.cc @@ -679,16 +679,14 @@ void Array::delete_fragments( // Delete fragments and commits auto vfs = &(resources.vfs()); - throw_if_not_ok(parallel_for( - &resources.compute_tp(), 0, fragment_uris.size(), [&](size_t i) { - throw_if_not_ok(vfs->remove_dir(fragment_uris[i].uri_)); - bool is_file = false; - throw_if_not_ok(vfs->is_file(commit_uris_to_delete[i], &is_file)); - if (is_file) { - throw_if_not_ok(vfs->remove_file(commit_uris_to_delete[i])); - } - return Status::Ok(); - })); + parallel_for(&resources.compute_tp(), 0, fragment_uris.size(), [&](size_t i) { + throw_if_not_ok(vfs->remove_dir(fragment_uris[i].uri_)); + bool is_file = false; + throw_if_not_ok(vfs->is_file(commit_uris_to_delete[i], &is_file)); + if (is_file) { + throw_if_not_ok(vfs->remove_file(commit_uris_to_delete[i])); + } + }); } void Array::delete_fragments( @@ -1711,7 +1709,7 @@ std::unordered_map Array::get_average_var_cell_sizes() // Load all metadata for tile var sizes among fragments. for (const auto& var_name : var_names) { - throw_if_not_ok(parallel_for( + parallel_for( &resources_.compute_tp(), 0, fragment_metadata.size(), @@ -1720,17 +1718,16 @@ std::unordered_map Array::get_average_var_cell_sizes() // evolution that do not exists in this fragment. const auto& schema = fragment_metadata[f]->array_schema(); if (!schema->is_field(var_name)) { - return Status::Ok(); + return; } fragment_metadata[f]->loaded_metadata()->load_tile_var_sizes( *encryption_key(), var_name); - return Status::Ok(); - })); + }); } // Now compute for each var size names, the average cell size. - throw_if_not_ok(parallel_for( + parallel_for( &resources_.compute_tp(), 0, var_names.size(), [&](const uint64_t n) { uint64_t total_size = 0; uint64_t cell_num = 0; @@ -1756,9 +1753,7 @@ std::unordered_map Array::get_average_var_cell_sizes() uint64_t average_cell_size = total_size / cell_num; ret[var_name] = std::max(average_cell_size, 1); - - return Status::Ok(); - })); + }); return ret; } @@ -1988,15 +1983,12 @@ void Array::do_load_metadata() { auto metadata_num = array_metadata_to_load.size(); std::vector> metadata_tiles(metadata_num); - throw_if_not_ok( - parallel_for(&resources_.compute_tp(), 0, metadata_num, [&](size_t m) { - const auto& uri = array_metadata_to_load[m].uri_; - - metadata_tiles[m] = GenericTileIO::load( - resources_, uri, 0, *encryption_key(), memory_tracker_); + parallel_for(&resources_.compute_tp(), 0, metadata_num, [&](size_t m) { + const auto& uri = array_metadata_to_load[m].uri_; - return Status::Ok(); - })); + metadata_tiles[m] = GenericTileIO::load( + resources_, uri, 0, *encryption_key(), memory_tracker_); + }); // Compute array metadata size for the statistics uint64_t meta_size = 0; diff --git a/tiledb/sm/array/array_directory.cc b/tiledb/sm/array/array_directory.cc index 444036d9f2e..11631b82f86 100644 --- a/tiledb/sm/array/array_directory.cc +++ b/tiledb/sm/array/array_directory.cc @@ -168,7 +168,7 @@ ArrayDirectory::load_all_array_schemas( auto schema_num = schema_uris.size(); schema_vector.resize(schema_num); - auto status = parallel_for( + parallel_for( &resources_.get().compute_tp(), 0, schema_num, [&](size_t schema_ith) { auto& schema_uri = schema_uris[schema_ith]; try { @@ -182,10 +182,7 @@ ArrayDirectory::load_all_array_schemas( // when Status gets removed from this module. throw ArrayDirectoryException(e.what()); } - - return Status::Ok(); }); - throw_if_not_ok(status); std::unordered_map> array_schemas; for (const auto& schema : schema_vector) { @@ -209,11 +206,10 @@ ArrayDirectory::load_enumerations_from_paths( std::vector> ret(enumeration_paths.size()); auto& tp = resources_.get().io_tp(); - throw_if_not_ok(parallel_for(&tp, 0, enumeration_paths.size(), [&](size_t i) { + parallel_for(&tp, 0, enumeration_paths.size(), [&](size_t i) { ret[i] = load_enumeration(enumeration_paths[i], encryption_key, memory_tracker); - return Status::Ok(); - })); + }); return ret; } @@ -317,17 +313,15 @@ void ArrayDirectory::delete_fragments_list( } // Delete fragments and commits - throw_if_not_ok(parallel_for( - &resources_.get().compute_tp(), 0, uris.size(), [&](size_t i) { - auto& vfs = resources_.get().vfs(); - throw_if_not_ok(vfs.remove_dir(uris[i])); - bool is_file = false; - throw_if_not_ok(vfs.is_file(commit_uris_to_delete[i], &is_file)); - if (is_file) { - throw_if_not_ok(vfs.remove_file(commit_uris_to_delete[i])); - } - return Status::Ok(); - })); + parallel_for(&resources_.get().compute_tp(), 0, uris.size(), [&](size_t i) { + auto& vfs = resources_.get().vfs(); + throw_if_not_ok(vfs.remove_dir(uris[i])); + bool is_file = false; + throw_if_not_ok(vfs.is_file(commit_uris_to_delete[i], &is_file)); + if (is_file) { + throw_if_not_ok(vfs.remove_file(commit_uris_to_delete[i])); + } + }); } Status ArrayDirectory::load() { @@ -342,16 +336,12 @@ Status ArrayDirectory::load() { // Some processing is also done here for things that don't depend on others. if (mode_ != ArrayDirectoryMode::SCHEMA_ONLY) { // List (in parallel) the root directory URIs - tasks.emplace_back(resources_.get().compute_tp().execute([&]() { - root_dir_uris = list_root_dir_uris(); - return Status::Ok(); - })); + tasks.emplace_back(resources_.get().compute_tp().execute( + [&]() { root_dir_uris = list_root_dir_uris(); })); // List (in parallel) the commits directory URIs - tasks.emplace_back(resources_.get().compute_tp().execute([&]() { - commits_dir_uris = list_commits_dir_uris(); - return Status::Ok(); - })); + tasks.emplace_back(resources_.get().compute_tp().execute( + [&]() { commits_dir_uris = list_commits_dir_uris(); })); // For commits mode, no need to load fragment/array metadata as they // are not used for commits consolidation/vacuuming. @@ -360,14 +350,11 @@ Status ArrayDirectory::load() { tasks.emplace_back(resources_.get().compute_tp().execute([&]() { fragment_meta_uris_v12_or_higher = list_fragment_metadata_dir_uris_v12_or_higher(); - return Status::Ok(); })); // Load (in parallel) the array metadata URIs - tasks.emplace_back(resources_.get().compute_tp().execute([&]() { - load_array_meta_uris(); - return Status::Ok(); - })); + tasks.emplace_back(resources_.get().compute_tp().execute( + [&]() { load_array_meta_uris(); })); } } @@ -375,14 +362,12 @@ Status ArrayDirectory::load() { // commits consolidation/vacuuming. if (mode_ != ArrayDirectoryMode::COMMITS) { // Load (in parallel) the array schema URIs - tasks.emplace_back(resources_.get().compute_tp().execute([&]() { - load_array_schema_uris(); - return Status::Ok(); - })); + tasks.emplace_back(resources_.get().compute_tp().execute( + [&]() { load_array_schema_uris(); })); } // Wait for all tasks to complete - RETURN_NOT_OK(resources_.get().compute_tp().wait_all(tasks)); + resources_.get().compute_tp().wait_all(tasks); if (mode_ != ArrayDirectoryMode::COMMITS) { // Add old array schema, if required. @@ -935,16 +920,14 @@ ArrayDirectory::compute_fragment_uris_v1_v11( // Get only the committed fragment uris std::vector is_fragment(array_dir_uris.size(), 0); auto& tp = resources_.get().compute_tp(); - auto status = parallel_for(&tp, 0, array_dir_uris.size(), [&](size_t i) { + parallel_for(&tp, 0, array_dir_uris.size(), [&](size_t i) { if (stdx::string::starts_with(array_dir_uris[i].last_path_part(), ".")) - return Status::Ok(); + return; int32_t flag; throw_if_not_ok(this->is_fragment( array_dir_uris[i], ok_uris, consolidated_commit_uris_set_, &flag)); is_fragment[i] = (uint8_t)flag; - return Status::Ok(); }); - RETURN_NOT_OK_TUPLE(status, nullopt); for (size_t i = 0; i < array_dir_uris.size(); ++i) { if (is_fragment[i]) { @@ -1018,32 +1001,29 @@ ArrayDirectory::compute_uris_to_vacuum( std::vector vac_file_bitmap(uris.size()); std::vector overlapping_vac_file_bitmap(uris.size()); std::vector non_vac_uri_bitmap(uris.size()); - throw_if_not_ok(parallel_for( - &resources_.get().compute_tp(), 0, uris.size(), [&](size_t i) { - auto& uri = uris[i]; - - // Get the start and end timestamp for this fragment - FragmentID fragment_id{uri}; - auto fragment_timestamp_range{fragment_id.timestamp_range()}; - if (is_vacuum_file(uri)) { - vac_file_bitmap[i] = 1; - if (timestamps_overlap( - fragment_timestamp_range, - !full_overlap_only && - consolidation_with_timestamps_supported(uri))) { - overlapping_vac_file_bitmap[i] = 1; - } - } else { - if (!timestamps_overlap( - fragment_timestamp_range, - !full_overlap_only && - consolidation_with_timestamps_supported(uri))) { - non_vac_uri_bitmap[i] = 1; - } - } + parallel_for(&resources_.get().compute_tp(), 0, uris.size(), [&](size_t i) { + auto& uri = uris[i]; - return Status::Ok(); - })); + // Get the start and end timestamp for this fragment + FragmentID fragment_id{uri}; + auto fragment_timestamp_range{fragment_id.timestamp_range()}; + if (is_vacuum_file(uri)) { + vac_file_bitmap[i] = 1; + if (timestamps_overlap( + fragment_timestamp_range, + !full_overlap_only && + consolidation_with_timestamps_supported(uri))) { + overlapping_vac_file_bitmap[i] = 1; + } + } else { + if (!timestamps_overlap( + fragment_timestamp_range, + !full_overlap_only && + consolidation_with_timestamps_supported(uri))) { + non_vac_uri_bitmap[i] = 1; + } + } + }); auto num_vac_files = std::accumulate(vac_file_bitmap.begin(), vac_file_bitmap.end(), 0); @@ -1076,7 +1056,7 @@ ArrayDirectory::compute_uris_to_vacuum( std::vector to_vacuum_vec(uris.size(), 0); std::vector to_vacuum_vac_files_vec(vac_files.size(), 0); auto& tp = resources_.get().compute_tp(); - auto status = parallel_for(&tp, 0, vac_files.size(), [&](size_t i) { + parallel_for(&tp, 0, vac_files.size(), [&](size_t i) { uint64_t size = 0; auto& vfs = resources_.get().vfs(); throw_if_not_ok(vfs.file_size(vac_files[i], &size)); @@ -1099,10 +1079,7 @@ ArrayDirectory::compute_uris_to_vacuum( } to_vacuum_vac_files_vec[i] = vacuum_vac_file; - - return Status::Ok(); }); - RETURN_NOT_OK_TUPLE(status, nullopt, nullopt); // Compute the fragment URIs to vacuum std::vector uris_to_vacuum; @@ -1144,30 +1121,28 @@ ArrayDirectory::compute_filtered_uris( std::vector overlaps_bitmap(uris.size()); std::vector> fragment_timestamp_ranges( uris.size()); - throw_if_not_ok(parallel_for( - &resources_.get().compute_tp(), 0, uris.size(), [&](size_t i) { - auto& uri = uris[i]; - std::string short_uri = uri.to_string().substr(base_uri_size); - if (to_ignore_set.count(short_uri.c_str()) != 0) { - return Status::Ok(); - } + parallel_for(&resources_.get().compute_tp(), 0, uris.size(), [&](size_t i) { + auto& uri = uris[i]; + std::string short_uri = uri.to_string().substr(base_uri_size); + if (to_ignore_set.count(short_uri.c_str()) != 0) { + return; + } - // Also ignore any vac uris - if (is_vacuum_file(uri)) { - return Status::Ok(); - } + // Also ignore any vac uris + if (is_vacuum_file(uri)) { + return; + } - // Get the start and end timestamp for this fragment - FragmentID fragment_id{uri}; - fragment_timestamp_ranges[i] = fragment_id.timestamp_range(); - if (timestamps_overlap( - fragment_timestamp_ranges[i], - !full_overlap_only && - consolidation_with_timestamps_supported(uri))) { - overlaps_bitmap[i] = 1; - } - return Status::Ok(); - })); + // Get the start and end timestamp for this fragment + FragmentID fragment_id{uri}; + fragment_timestamp_ranges[i] = fragment_id.timestamp_range(); + if (timestamps_overlap( + fragment_timestamp_ranges[i], + !full_overlap_only && + consolidation_with_timestamps_supported(uri))) { + overlaps_bitmap[i] = 1; + } + }); auto count = std::accumulate(overlaps_bitmap.begin(), overlaps_bitmap.end(), 0); diff --git a/tiledb/sm/array/array_operations.cc b/tiledb/sm/array/array_operations.cc index f221db80c63..9b2d2251d5b 100644 --- a/tiledb/sm/array/array_operations.cc +++ b/tiledb/sm/array/array_operations.cc @@ -74,48 +74,38 @@ load_delete_and_update_conditions( auto conditions = std::vector(locations.size()); auto update_values = std::vector>(locations.size()); - auto status = - parallel_for(&resources.compute_tp(), 0, locations.size(), [&](size_t i) { - // Get condition marker. - auto& uri = locations[i].uri(); + parallel_for(&resources.compute_tp(), 0, locations.size(), [&](size_t i) { + // Get condition marker. + auto& uri = locations[i].uri(); - // Read the condition from storage. - auto tile = GenericTileIO::load( - resources, - uri, - locations[i].offset(), - *(opened_array.encryption_key()), - resources.ephemeral_memory_tracker()); + // Read the condition from storage. + auto tile = GenericTileIO::load( + resources, + uri, + locations[i].offset(), + *(opened_array.encryption_key()), + resources.ephemeral_memory_tracker()); - if (tiledb::sm::utils::parse::ends_with( - locations[i].condition_marker(), - tiledb::sm::constants::delete_file_suffix)) { - conditions[i] = tiledb::sm::deletes_and_updates::serialization:: - deserialize_condition( - i, - locations[i].condition_marker(), - tile->data(), - tile->size()); - } else if (tiledb::sm::utils::parse::ends_with( - locations[i].condition_marker(), - tiledb::sm::constants::update_file_suffix)) { - auto&& [cond, uvs] = tiledb::sm::deletes_and_updates::serialization:: - deserialize_update_condition_and_values( - i, - locations[i].condition_marker(), - tile->data(), - tile->size()); - conditions[i] = std::move(cond); - update_values[i] = std::move(uvs); - } else { - throw ArrayOperationsException("Unknown condition marker extension"); - } + if (tiledb::sm::utils::parse::ends_with( + locations[i].condition_marker(), + tiledb::sm::constants::delete_file_suffix)) { + conditions[i] = + tiledb::sm::deletes_and_updates::serialization::deserialize_condition( + i, locations[i].condition_marker(), tile->data(), tile->size()); + } else if (tiledb::sm::utils::parse::ends_with( + locations[i].condition_marker(), + tiledb::sm::constants::update_file_suffix)) { + auto&& [cond, uvs] = tiledb::sm::deletes_and_updates::serialization:: + deserialize_update_condition_and_values( + i, locations[i].condition_marker(), tile->data(), tile->size()); + conditions[i] = std::move(cond); + update_values[i] = std::move(uvs); + } else { + throw ArrayOperationsException("Unknown condition marker extension"); + } - throw_if_not_ok( - conditions[i].check(opened_array.array_schema_latest())); - return Status::Ok(); - }); - throw_if_not_ok(status); + throw_if_not_ok(conditions[i].check(opened_array.array_schema_latest())); + }); return {conditions, update_values}; } diff --git a/tiledb/sm/consolidator/fragment_consolidator.cc b/tiledb/sm/consolidator/fragment_consolidator.cc index 6453cc4e074..966bca7586e 100644 --- a/tiledb/sm/consolidator/fragment_consolidator.cc +++ b/tiledb/sm/consolidator/fragment_consolidator.cc @@ -523,24 +523,21 @@ void FragmentConsolidator::vacuum(const char* array_name) { // Delete fragment directories auto& vfs = resources_.vfs(); auto& compute_tp = resources_.compute_tp(); - throw_if_not_ok(parallel_for( - &compute_tp, 0, fragment_uris_to_vacuum.size(), [&](size_t i) { - // Remove the commit file, if present. - auto commit_uri = array_dir.get_commit_uri(fragment_uris_to_vacuum[i]); - bool is_file = false; - throw_if_not_ok(vfs.is_file(commit_uri, &is_file)); - if (is_file) { - throw_if_not_ok(vfs.remove_file(commit_uri)); - } - - bool is_dir = false; - throw_if_not_ok(vfs.is_dir(fragment_uris_to_vacuum[i], &is_dir)); - if (is_dir) { - throw_if_not_ok(vfs.remove_dir(fragment_uris_to_vacuum[i])); - } + parallel_for(&compute_tp, 0, fragment_uris_to_vacuum.size(), [&](size_t i) { + // Remove the commit file, if present. + auto commit_uri = array_dir.get_commit_uri(fragment_uris_to_vacuum[i]); + bool is_file = false; + throw_if_not_ok(vfs.is_file(commit_uri, &is_file)); + if (is_file) { + throw_if_not_ok(vfs.remove_file(commit_uri)); + } - return Status::Ok(); - })); + bool is_dir = false; + throw_if_not_ok(vfs.is_dir(fragment_uris_to_vacuum[i], &is_dir)); + if (is_dir) { + throw_if_not_ok(vfs.remove_dir(fragment_uris_to_vacuum[i])); + } + }); // Delete the vacuum files. vfs.remove_files( diff --git a/tiledb/sm/consolidator/fragment_meta_consolidator.cc b/tiledb/sm/consolidator/fragment_meta_consolidator.cc index 3ab957f9a35..8199016306b 100644 --- a/tiledb/sm/consolidator/fragment_meta_consolidator.cc +++ b/tiledb/sm/consolidator/fragment_meta_consolidator.cc @@ -117,18 +117,14 @@ Status FragmentMetaConsolidator::consolidate( // Serialize all fragment metadata footers in parallel std::vector> tiles(meta.size()); - auto status = - parallel_for(&resources_.compute_tp(), 0, tiles.size(), [&](size_t i) { - SizeComputationSerializer size_computation_serializer; - meta[i]->write_footer(size_computation_serializer); - tiles[i] = WriterTile::from_generic( - size_computation_serializer.size(), consolidator_memory_tracker_); - Serializer serializer(tiles[i]->data(), tiles[i]->size()); - meta[i]->write_footer(serializer); - - return Status::Ok(); - }); - throw_if_not_ok(status); + parallel_for(&resources_.compute_tp(), 0, tiles.size(), [&](size_t i) { + SizeComputationSerializer size_computation_serializer; + meta[i]->write_footer(size_computation_serializer); + tiles[i] = WriterTile::from_generic( + size_computation_serializer.size(), consolidator_memory_tracker_); + Serializer serializer(tiles[i]->data(), tiles[i]->size()); + meta[i]->write_footer(serializer); + }); auto serialize_data = [&](Serializer& serializer, uint64_t offset) { // Write number of fragments @@ -202,16 +198,14 @@ void FragmentMetaConsolidator::vacuum(const char* array_name) { // Vacuum auto& vfs = resources_.vfs(); auto& compute_tp = resources_.compute_tp(); - throw_if_not_ok( - parallel_for(&compute_tp, 0, fragment_meta_uris.size(), [&](size_t i) { - auto& uri = fragment_meta_uris[i]; - FragmentID fragment_id{uri}; - auto timestamp_range{fragment_id.timestamp_range()}; - if (timestamp_range.second != t_latest) { - throw_if_not_ok(vfs.remove_file(uri)); - } - return Status::Ok(); - })); + parallel_for(&compute_tp, 0, fragment_meta_uris.size(), [&](size_t i) { + auto& uri = fragment_meta_uris[i]; + FragmentID fragment_id{uri}; + auto timestamp_range{fragment_id.timestamp_range()}; + if (timestamp_range.second != t_latest) { + throw_if_not_ok(vfs.remove_file(uri)); + } + }); } } // namespace tiledb::sm diff --git a/tiledb/sm/filesystem/azure.cc b/tiledb/sm/filesystem/azure.cc index 02b27bc6527..da3ff298aa8 100644 --- a/tiledb/sm/filesystem/azure.cc +++ b/tiledb/sm/filesystem/azure.cc @@ -838,11 +838,9 @@ Status Azure::remove_blob(const URI& uri) const { Status Azure::remove_dir(const URI& uri) const { std::vector paths; RETURN_NOT_OK(ls(uri, &paths, "")); - auto status = parallel_for(thread_pool_, 0, paths.size(), [&](size_t i) { + parallel_for(thread_pool_, 0, paths.size(), [&](size_t i) { throw_if_not_ok(remove_blob(URI(paths[i]))); - return Status::Ok(); }); - RETURN_NOT_OK(status); return Status::Ok(); } @@ -1054,10 +1052,7 @@ Status Azure::write_blocks( if (num_ops == 1) { const std::string block_id = state->next_block_id(); - const Status st = - upload_block(container_name, blob_path, buffer, length, block_id); - state->update_st(st); - return st; + upload_block(container_name, blob_path, buffer, length, block_id); } else { std::vector tasks; tasks.reserve(num_ops); @@ -1070,21 +1065,17 @@ Status Azure::write_blocks( const uint64_t thread_buffer_len = end - begin + 1; const std::string block_id = state->next_block_id(); - std::function upload_block_fn = std::bind( + tasks.emplace_back(thread_pool_->execute( &Azure::upload_block, this, container_name, blob_path, thread_buffer, thread_buffer_len, - block_id); - ThreadPool::Task task = thread_pool_->execute(std::move(upload_block_fn)); - tasks.emplace_back(std::move(task)); + block_id)); } - const Status st = thread_pool_->wait_all(tasks); - state->update_st(st); - return st; + thread_pool_->wait_all(tasks); } return Status::Ok(); @@ -1153,7 +1144,7 @@ LsObjects Azure::list_blobs_impl( } } -Status Azure::upload_block( +void Azure::upload_block( const std::string& container_name, const std::string& blob_path, const void* const buffer, @@ -1162,16 +1153,9 @@ Status Azure::upload_block( const auto& c = client(); ::Azure::Core::IO::MemoryBodyStream stream( static_cast(buffer), static_cast(length)); - try { - c.GetBlobContainerClient(container_name) - .GetBlockBlobClient(blob_path) - .StageBlock(block_id, stream); - } catch (const ::Azure::Storage::StorageException& e) { - return LOG_STATUS(Status_AzureError( - "Upload block failed on: " + blob_path + "; " + e.Message)); - } - - return Status::Ok(); + c.GetBlobContainerClient(container_name) + .GetBlockBlobClient(blob_path) + .StageBlock(block_id, stream); } Status Azure::parse_azure_uri( diff --git a/tiledb/sm/filesystem/azure.h b/tiledb/sm/filesystem/azure.h index f19e3779753..54e0ed62e11 100644 --- a/tiledb/sm/filesystem/azure.h +++ b/tiledb/sm/filesystem/azure.h @@ -745,9 +745,8 @@ class Azure { * @param length The length of `buffer`. * @param block_id A base64-encoded string that is unique to this block * within the blob. - * @return Status */ - Status upload_block( + void upload_block( const std::string& container_name, const std::string& blob_path, const void* const buffer, diff --git a/tiledb/sm/filesystem/gcs.cc b/tiledb/sm/filesystem/gcs.cc index 2f002144c2b..e0b3b5c19c9 100644 --- a/tiledb/sm/filesystem/gcs.cc +++ b/tiledb/sm/filesystem/gcs.cc @@ -436,11 +436,9 @@ Status GCS::remove_dir(const URI& uri) const { std::vector paths; RETURN_NOT_OK(ls(uri, &paths, "")); - auto status = parallel_for(thread_pool_, 0, paths.size(), [&](size_t i) { + parallel_for(thread_pool_, 0, paths.size(), [&](size_t i) { throw_if_not_ok(remove_object(URI(paths[i]))); - return Status::Ok(); }); - RETURN_NOT_OK(status); return Status::Ok(); } @@ -1036,11 +1034,8 @@ Status GCS::write_parts( if (num_ops == 1) { const std::string object_part_path = state->next_part_path(); - const Status st = - upload_part(bucket_name, object_part_path, buffer, length); - state->update_st(st); + upload_part(bucket_name, object_part_path, buffer, length); state_lck.unlock(); - return st; } else { std::vector tasks; tasks.reserve(num_ops); @@ -1053,27 +1048,23 @@ Status GCS::write_parts( const uint64_t thread_buffer_len = end - begin + 1; const std::string object_part_path = state->next_part_path(); - std::function upload_part_fn = std::bind( + ThreadPool::Task task = thread_pool_->execute( &GCS::upload_part, this, bucket_name, object_part_path, thread_buffer, thread_buffer_len); - ThreadPool::Task task = thread_pool_->execute(std::move(upload_part_fn)); tasks.emplace_back(std::move(task)); } - const Status st = thread_pool_->wait_all(tasks); - state->update_st(st); - state_lck.unlock(); - return st; + thread_pool_->wait_all(tasks); } return Status::Ok(); } -Status GCS::upload_part( +void GCS::upload_part( const std::string& bucket_name, const std::string& object_part_path, const void* const buffer, @@ -1088,12 +1079,10 @@ Status GCS::upload_part( if (!object_metadata.ok()) { const google::cloud::Status status = object_metadata.status(); - return LOG_STATUS(Status_GCSError(std::string( + throw GCSException(std::string( "Upload part failed on: " + object_part_path + " (" + status.message() + - ")"))); + ")")); } - - return Status::Ok(); } Status GCS::flush_object(const URI& uri) { @@ -1201,28 +1190,25 @@ void GCS::delete_parts( std::vector tasks; tasks.reserve(part_paths.size()); for (const auto& part_path : part_paths) { - std::function delete_part_fn = - std::bind(&GCS::delete_part, this, bucket_name, part_path); - ThreadPool::Task task = thread_pool_->execute(std::move(delete_part_fn)); - tasks.emplace_back(std::move(task)); + tasks.emplace_back( + thread_pool_->execute(&GCS::delete_part, this, bucket_name, part_path)); } - const Status st = thread_pool_->wait_all(tasks); - if (!st.ok()) { - LOG_STATUS_NO_RETURN_VALUE(st); + try { + thread_pool_->wait_all(tasks); + } catch (const std::exception& e) { + LOG_ERROR(e.what()); } } -Status GCS::delete_part( +void GCS::delete_part( const std::string& bucket_name, const std::string& part_path) { const google::cloud::Status status = client_->DeleteObject(bucket_name, part_path); if (!status.ok()) { - return Status_GCSError(std::string( + throw GCSException(std::string( "Delete part failed on: " + part_path + " (" + status.message() + ")")); } - - return Status::Ok(); } void GCS::finish_multi_part_upload(const URI& uri) { diff --git a/tiledb/sm/filesystem/gcs.h b/tiledb/sm/filesystem/gcs.h index 9df5fefa4f6..8a889a6b7cf 100644 --- a/tiledb/sm/filesystem/gcs.h +++ b/tiledb/sm/filesystem/gcs.h @@ -721,9 +721,8 @@ class GCS { * @param object_part_path The object's part path. * @param length The length of `buffer`. * @param part_id A unique integer identifier for this part. - * @return Status */ - Status upload_part( + void upload_part( const std::string& bucket_name, const std::string& object_part_path, const void* const buffer, @@ -746,9 +745,8 @@ class GCS { * * @param bucket_name The object's bucket name. * @param part_path The object part's path to delete. - * @return Status */ - Status delete_part( + void delete_part( const std::string& bucket_name, const std::string& part_path); /** diff --git a/tiledb/sm/filesystem/s3.cc b/tiledb/sm/filesystem/s3.cc index 3c8a5fe3987..947b87ab8df 100644 --- a/tiledb/sm/filesystem/s3.cc +++ b/tiledb/sm/filesystem/s3.cc @@ -497,11 +497,9 @@ void S3::remove_dir(const URI& uri) const { return; } - throw_if_not_ok( - parallel_for(vfs_thread_pool_, 0, paths.size(), [&](size_t i) { - throw_if_not_ok(remove_object(URI(paths[i]))); - return Status::Ok(); - })); + parallel_for(vfs_thread_pool_, 0, paths.size(), [&](size_t i) { + throw_if_not_ok(remove_object(URI(paths[i]))); + }); // Minio changed their delete behavior when an object masks another object // with the same prefix. Previously, minio would delete any object with @@ -518,11 +516,9 @@ void S3::remove_dir(const URI& uri) const { } // Delete the uncovered object prefixes. - throw_if_not_ok( - parallel_for(vfs_thread_pool_, 0, paths.size(), [&](size_t i) { - throw_if_not_ok(remove_object(URI(paths[i]))); - return Status::Ok(); - })); + parallel_for(vfs_thread_pool_, 0, paths.size(), [&](size_t i) { + throw_if_not_ok(remove_object(URI(paths[i]))); + }); } void S3::touch(const URI& uri) const { @@ -662,41 +658,37 @@ Status S3::disconnect() { for (auto& kv : multipart_upload_states_) states.emplace_back(&kv.second); - auto status = - parallel_for(vfs_thread_pool_, 0, states.size(), [&](uint64_t i) { - const MultiPartUploadState* state = states[i]; - // Lock multipart state - std::unique_lock state_lck(state->mtx); - - if (state->st.ok()) { - Aws::S3::Model::CompleteMultipartUploadRequest complete_request = - make_multipart_complete_request(*state); - auto outcome = client_->CompleteMultipartUpload(complete_request); - if (!outcome.IsSuccess()) { - const Status st = LOG_STATUS(Status_S3Error( - std::string("Failed to disconnect and flush S3 objects. ") + - outcome_error_message(outcome))); - if (!st.ok()) { - ret_st = st; - } - } - } else { - Aws::S3::Model::AbortMultipartUploadRequest abort_request = - make_multipart_abort_request(*state); - auto outcome = client_->AbortMultipartUpload(abort_request); - if (!outcome.IsSuccess()) { - ret_st = LOG_STATUS(Status_S3Error( - std::string("Failed to disconnect and flush S3 objects. ") + - outcome_error_message(outcome))); - } else { - ret_st = LOG_STATUS(Status_S3Error( - std::string("Failed to disconnect and flush S3 objects. "))); - } + parallel_for(vfs_thread_pool_, 0, states.size(), [&](uint64_t i) { + const MultiPartUploadState* state = states[i]; + // Lock multipart state + std::unique_lock state_lck(state->mtx); + + if (state->st.ok()) { + Aws::S3::Model::CompleteMultipartUploadRequest complete_request = + make_multipart_complete_request(*state); + auto outcome = client_->CompleteMultipartUpload(complete_request); + if (!outcome.IsSuccess()) { + const Status st = LOG_STATUS(Status_S3Error( + std::string("Failed to disconnect and flush S3 objects. ") + + outcome_error_message(outcome))); + if (!st.ok()) { + ret_st = st; } - return Status::Ok(); - }); - - RETURN_NOT_OK(status); + } + } else { + Aws::S3::Model::AbortMultipartUploadRequest abort_request = + make_multipart_abort_request(*state); + auto outcome = client_->AbortMultipartUpload(abort_request); + if (!outcome.IsSuccess()) { + ret_st = LOG_STATUS(Status_S3Error( + std::string("Failed to disconnect and flush S3 objects. ") + + outcome_error_message(outcome))); + } else { + ret_st = LOG_STATUS(Status_S3Error( + std::string("Failed to disconnect and flush S3 objects. "))); + } + } + }); } unique_rl.unlock(); @@ -815,7 +807,7 @@ void S3::finalize_and_flush_object(const URI& uri) { } std::vector merged(sum_sizes); - throw_if_not_ok(parallel_for( + parallel_for( vfs_thread_pool_, 0, intermediate_chunks.size(), [&](size_t i) { uint64_t length_returned; throw_if_not_ok(read_impl( @@ -825,8 +817,7 @@ void S3::finalize_and_flush_object(const URI& uri) { intermediate_chunks[i].size, 0, &length_returned)); - return Status::Ok(); - })); + }); const int part_num = state.part_number++; auto ctx = make_upload_part_req( @@ -861,11 +852,9 @@ void S3::finalize_and_flush_object(const URI& uri) { } // Remove intermediate chunk files if any - throw_if_not_ok(parallel_for( - vfs_thread_pool_, 0, intermediate_chunks.size(), [&](size_t i) { - throw_if_not_ok(remove_object(URI(intermediate_chunks[i].uri))); - return Status::Ok(); - })); + parallel_for(vfs_thread_pool_, 0, intermediate_chunks.size(), [&](size_t i) { + throw_if_not_ok(remove_object(URI(intermediate_chunks[i].uri))); + }); // Remove the multipart upload state entry UniqueWriteLock unique_wl(&multipart_upload_rwlock_); @@ -1217,18 +1206,16 @@ void S3::global_order_write( // Read all previously written chunks into one contiguous buffer and add the // current data at the end. std::vector merged(sum_sizes); - throw_if_not_ok(parallel_for( - vfs_thread_pool_, 0, intermediate_chunks.size(), [&](size_t i) { - uint64_t length_returned; - throw_if_not_ok(read_impl( - URI(intermediate_chunks[i].uri), - 0, - merged.data() + offsets[i], - intermediate_chunks[i].size, - 0, - &length_returned)); - return Status::Ok(); - })); + parallel_for(vfs_thread_pool_, 0, intermediate_chunks.size(), [&](size_t i) { + uint64_t length_returned; + throw_if_not_ok(read_impl( + URI(intermediate_chunks[i].uri), + 0, + merged.data() + offsets[i], + intermediate_chunks[i].size, + 0, + &length_returned)); + }); std::memcpy(merged.data() + offsets.back(), buffer, length); // Issue one more multiple part uploads depending on the cumulative size of @@ -1256,22 +1243,19 @@ void S3::global_order_write( ctx_vec.resize(num_ops); const uint64_t bytes_per_op = s3_params_.multipart_part_size_; const int part_num_base = state.part_number; - throw_if_not_ok( - parallel_for(vfs_thread_pool_, 0, num_ops, [&](uint64_t op_idx) { - uint64_t begin = op_idx * bytes_per_op, - end = std::min( - (op_idx + 1) * bytes_per_op - 1, - uint64_t(merged.size() - 1)); - uint64_t thread_nbytes = end - begin + 1; - auto thread_buffer = static_cast( - static_cast(merged.data())) + - begin; - int part_num = static_cast(part_num_base + op_idx); - ctx_vec[op_idx] = make_upload_part_req( - aws_uri, thread_buffer, thread_nbytes, upload_id, part_num); - - return Status::Ok(); - })); + parallel_for(vfs_thread_pool_, 0, num_ops, [&](uint64_t op_idx) { + uint64_t begin = op_idx * bytes_per_op, + end = std::min( + (op_idx + 1) * bytes_per_op - 1, + uint64_t(merged.size() - 1)); + uint64_t thread_nbytes = end - begin + 1; + auto thread_buffer = + static_cast(static_cast(merged.data())) + + begin; + int part_num = static_cast(part_num_base + op_idx); + ctx_vec[op_idx] = make_upload_part_req( + aws_uri, thread_buffer, thread_nbytes, upload_id, part_num); + }); state.part_number += num_ops; for (auto& ctx : ctx_vec) { @@ -1282,11 +1266,9 @@ void S3::global_order_write( } } - throw_if_not_ok(parallel_for( - vfs_thread_pool_, 0, intermediate_chunks.size(), [&](size_t i) { - throw_if_not_ok(remove_object(URI(intermediate_chunks[i].uri))); - return Status::Ok(); - })); + parallel_for(vfs_thread_pool_, 0, intermediate_chunks.size(), [&](size_t i) { + throw_if_not_ok(remove_object(URI(intermediate_chunks[i].uri))); + }); intermediate_chunks.clear(); } @@ -1919,7 +1901,7 @@ Status S3::write_multipart( ctx_vec.resize(num_ops); const uint64_t bytes_per_op = s3_params_.multipart_part_size_; const int part_num_base = state->part_number; - auto status = parallel_for(vfs_thread_pool_, 0, num_ops, [&](uint64_t i) { + parallel_for(vfs_thread_pool_, 0, num_ops, [&](uint64_t i) { uint64_t begin = i * bytes_per_op, end = std::min((i + 1) * bytes_per_op - 1, length - 1); uint64_t thread_nbytes = end - begin + 1; @@ -1927,8 +1909,6 @@ Status S3::write_multipart( int part_num = static_cast(part_num_base + i); ctx_vec[i] = make_upload_part_req( aws_uri, thread_buffer, thread_nbytes, upload_id, part_num); - - return Status::Ok(); }); state->part_number += num_ops; state_lck.unlock(); diff --git a/tiledb/sm/filesystem/s3_thread_pool_executor.cc b/tiledb/sm/filesystem/s3_thread_pool_executor.cc index 566ac31d3c4..871c983d27d 100644 --- a/tiledb/sm/filesystem/s3_thread_pool_executor.cc +++ b/tiledb/sm/filesystem/s3_thread_pool_executor.cc @@ -67,14 +67,12 @@ void S3ThreadPoolExecutor::Stop() { } bool S3ThreadPoolExecutor::SubmitToThread(std::function&& fn) { - auto wrapped_fn = [this, fn]() -> Status { + auto wrapped_fn = [this, fn]() { fn(); std::unique_lock lock_guard(lock_); if (--outstanding_tasks_ == 0) cv_.notify_all(); - - return Status::Ok(); }; std::unique_lock lock_guard(lock_); @@ -84,7 +82,7 @@ bool S3ThreadPoolExecutor::SubmitToThread(std::function&& fn) { ++outstanding_tasks_; lock_guard.unlock(); - ThreadPool::Task task = thread_pool_->execute(wrapped_fn); + ThreadPool::Task task = thread_pool_->execute(std::move(wrapped_fn)); return task.valid(); } diff --git a/tiledb/sm/filesystem/vfs.cc b/tiledb/sm/filesystem/vfs.cc index e32e036f23a..5c393e162e0 100644 --- a/tiledb/sm/filesystem/vfs.cc +++ b/tiledb/sm/filesystem/vfs.cc @@ -448,14 +448,13 @@ void VFS::remove_dir_if_empty(const URI& uri) const { void VFS::remove_dirs( ThreadPool* compute_tp, const std::vector& uris) const { - throw_if_not_ok(parallel_for(compute_tp, 0, uris.size(), [&](size_t i) { + parallel_for(compute_tp, 0, uris.size(), [&](size_t i) { bool is_dir; throw_if_not_ok(this->is_dir(uris[i], &is_dir)); if (is_dir) { throw_if_not_ok(remove_dir(uris[i])); } - return Status::Ok(); - })); + }); } Status VFS::remove_file(const URI& uri) const { @@ -504,18 +503,16 @@ Status VFS::remove_file(const URI& uri) const { void VFS::remove_files( ThreadPool* compute_tp, const std::vector& uris) const { - throw_if_not_ok(parallel_for(compute_tp, 0, uris.size(), [&](size_t i) { + parallel_for(compute_tp, 0, uris.size(), [&](size_t i) { throw_if_not_ok(remove_file(uris[i])); - return Status::Ok(); - })); + }); } void VFS::remove_files( ThreadPool* compute_tp, const std::vector& uris) const { - throw_if_not_ok(parallel_for(compute_tp, 0, uris.size(), [&](size_t i) { + parallel_for(compute_tp, 0, uris.size(), [&](size_t i) { throw_if_not_ok(remove_file(uris[i].uri_)); - return Status::Ok(); - })); + }); } Status VFS::max_parallel_ops(const URI& uri, uint64_t* ops) const { @@ -1139,14 +1136,8 @@ Status VFS::read( }); results.push_back(std::move(task)); } - Status st = io_tp_->wait_all(results); - if (!st.ok()) { - std::stringstream errmsg; - errmsg << "VFS parallel read error '" << uri.to_string() << "'; " - << st.message(); - return Status_VFSError(errmsg.str()); - } - return st; + io_tp_->wait_all(results); + return Status::Ok(); } } diff --git a/tiledb/sm/filter/filter_pipeline.cc b/tiledb/sm/filter/filter_pipeline.cc index 158d9a6fa72..17db72f04fc 100644 --- a/tiledb/sm/filter/filter_pipeline.cc +++ b/tiledb/sm/filter/filter_pipeline.cc @@ -238,7 +238,7 @@ Status FilterPipeline::filter_chunks_forward( nchunks); // Run each chunk through the entire pipeline. - auto status = parallel_for(compute_tp, 0, nchunks, [&](uint64_t i) { + parallel_for(compute_tp, 0, nchunks, [&](uint64_t i) { // TODO(ttd): can we instead allocate one FilterStorage per thread? // or make it threadsafe? FilterStorage storage; @@ -252,7 +252,7 @@ Status FilterPipeline::filter_chunks_forward( i == nchunks - 1 ? last_buffer_size : var_sizes ? chunk_offsets[i + 1] - chunk_offsets[i] : chunk_size; - RETURN_NOT_OK(input_data.init(chunk_buffer, chunk_buffer_size)); + throw_if_not_ok(input_data.init(chunk_buffer, chunk_buffer_size)); // Apply the filters sequentially. for (auto it = filters_.begin(), ite = filters_.end(); it != ite; ++it) { @@ -297,11 +297,8 @@ Status FilterPipeline::filter_chunks_forward( throw_if_not_ok(io_input.second.swap(input_data)); throw_if_not_ok(io_output.first.swap(output_metadata)); throw_if_not_ok(io_output.second.swap(output_data)); - return Status::Ok(); }); - RETURN_NOT_OK(status); - uint64_t total_processed_size = 0; std::vector var_chunk_sizes(final_stage_io.size()); uint64_t offset = sizeof(uint64_t); @@ -336,7 +333,7 @@ Status FilterPipeline::filter_chunks_forward( memcpy(output.data(), &nchunks, sizeof(uint64_t)); // Concatenate all processed chunks into the final output buffer. - status = parallel_for(compute_tp, 0, final_stage_io.size(), [&](uint64_t i) { + parallel_for(compute_tp, 0, final_stage_io.size(), [&](uint64_t i) { auto& final_stage_output_metadata = final_stage_io[i].first.first; auto& final_stage_output_data = final_stage_io[i].first.second; auto filtered_size = (uint32_t)final_stage_output_data.size(); @@ -363,11 +360,8 @@ Status FilterPipeline::filter_chunks_forward( dest_offset += metadata_size; // Write the chunk data throw_if_not_ok(final_stage_output_data.copy_to((char*)dest + dest_offset)); - return Status::Ok(); }); - RETURN_NOT_OK(status); - return Status::Ok(); } diff --git a/tiledb/sm/fragment/fragment_info.cc b/tiledb/sm/fragment/fragment_info.cc index 08aa78dbf31..1460ec3c763 100644 --- a/tiledb/sm/fragment/fragment_info.cc +++ b/tiledb/sm/fragment/fragment_info.cc @@ -844,7 +844,7 @@ Status FragmentInfo::load(const ArrayDirectory& array_dir) { // Get fragment sizes std::vector sizes(fragment_num, 0); - throw_if_not_ok(parallel_for( + parallel_for( &resources_->compute_tp(), 0, fragment_num, @@ -860,9 +860,7 @@ Status FragmentInfo::load(const ArrayDirectory& array_dir) { if (preload_rtrees & !meta->dense()) { meta->loaded_metadata()->load_rtree(enc_key_); } - - return Status::Ok(); - })); + }); // Clear single fragment info vec and anterior range single_fragment_info_vec_.clear(); @@ -1000,14 +998,12 @@ FragmentInfo::load_array_schemas_and_fragment_metadata( std::vector> fragment_metadata_tiles(meta_uris.size()); std::vector>> offsets_vectors( meta_uris.size()); - throw_if_not_ok( - parallel_for(&resources.compute_tp(), 0, meta_uris.size(), [&](size_t i) { - auto&& [tile_opt, offsets] = load_consolidated_fragment_meta( - resources, meta_uris[i], enc_key, memory_tracker); - fragment_metadata_tiles[i] = tile_opt; - offsets_vectors[i] = std::move(offsets); - return Status::Ok(); - })); + parallel_for(&resources.compute_tp(), 0, meta_uris.size(), [&](size_t i) { + auto&& [tile_opt, offsets] = load_consolidated_fragment_meta( + resources, meta_uris[i], enc_key, memory_tracker); + fragment_metadata_tiles[i] = tile_opt; + offsets_vectors[i] = std::move(offsets); + }); // Get the unique fragment metadatas into a map. std::unordered_map> offsets; diff --git a/tiledb/sm/fragment/fragment_metadata.cc b/tiledb/sm/fragment/fragment_metadata.cc index 236cb1e8de4..5cad2c5dad9 100644 --- a/tiledb/sm/fragment/fragment_metadata.cc +++ b/tiledb/sm/fragment/fragment_metadata.cc @@ -373,96 +373,93 @@ void FragmentMetadata::compute_fragment_min_max_sum_null_count() { } // Process all attributes in parallel. - throw_if_not_ok(parallel_for( - &resources_->compute_tp(), 0, idx_map_.size(), [&](uint64_t n) { - // For easy reference. - const auto& name = names[n]; - const auto& idx = idx_map_[name]; - const auto var_size = array_schema_->var_size(name); - const auto type = array_schema_->type(name); - - // Compute null count. - loaded_metadata_ptr_->fragment_null_counts()[idx] = std::accumulate( - loaded_metadata_ptr_->tile_null_counts()[idx].begin(), - loaded_metadata_ptr_->tile_null_counts()[idx].end(), - 0); - - if (var_size) { - min_max_var(name); - } else { - // Switch depending on datatype. - switch (type) { - case Datatype::INT8: - compute_fragment_min_max_sum(name); - break; - case Datatype::INT16: - compute_fragment_min_max_sum(name); - break; - case Datatype::INT32: - compute_fragment_min_max_sum(name); - break; - case Datatype::INT64: - compute_fragment_min_max_sum(name); - break; - case Datatype::BOOL: - case Datatype::UINT8: - compute_fragment_min_max_sum(name); - break; - case Datatype::UINT16: - compute_fragment_min_max_sum(name); - break; - case Datatype::UINT32: - compute_fragment_min_max_sum(name); - break; - case Datatype::UINT64: - compute_fragment_min_max_sum(name); - break; - case Datatype::FLOAT32: - compute_fragment_min_max_sum(name); - break; - case Datatype::FLOAT64: - compute_fragment_min_max_sum(name); - break; - case Datatype::DATETIME_YEAR: - case Datatype::DATETIME_MONTH: - case Datatype::DATETIME_WEEK: - case Datatype::DATETIME_DAY: - case Datatype::DATETIME_HR: - case Datatype::DATETIME_MIN: - case Datatype::DATETIME_SEC: - case Datatype::DATETIME_MS: - case Datatype::DATETIME_US: - case Datatype::DATETIME_NS: - case Datatype::DATETIME_PS: - case Datatype::DATETIME_FS: - case Datatype::DATETIME_AS: - case Datatype::TIME_HR: - case Datatype::TIME_MIN: - case Datatype::TIME_SEC: - case Datatype::TIME_MS: - case Datatype::TIME_US: - case Datatype::TIME_NS: - case Datatype::TIME_PS: - case Datatype::TIME_FS: - case Datatype::TIME_AS: - compute_fragment_min_max_sum(name); - break; - case Datatype::STRING_ASCII: - case Datatype::CHAR: - compute_fragment_min_max_sum(name); - break; - case Datatype::BLOB: - case Datatype::GEOM_WKB: - case Datatype::GEOM_WKT: - compute_fragment_min_max_sum(name); - break; - default: - break; - } - } + parallel_for(&resources_->compute_tp(), 0, idx_map_.size(), [&](uint64_t n) { + // For easy reference. + const auto& name = names[n]; + const auto& idx = idx_map_[name]; + const auto var_size = array_schema_->var_size(name); + const auto type = array_schema_->type(name); + + // Compute null count. + loaded_metadata_ptr_->fragment_null_counts()[idx] = std::accumulate( + loaded_metadata_ptr_->tile_null_counts()[idx].begin(), + loaded_metadata_ptr_->tile_null_counts()[idx].end(), + 0); - return Status::Ok(); - })); + if (var_size) { + min_max_var(name); + } else { + // Switch depending on datatype. + switch (type) { + case Datatype::INT8: + compute_fragment_min_max_sum(name); + break; + case Datatype::INT16: + compute_fragment_min_max_sum(name); + break; + case Datatype::INT32: + compute_fragment_min_max_sum(name); + break; + case Datatype::INT64: + compute_fragment_min_max_sum(name); + break; + case Datatype::BOOL: + case Datatype::UINT8: + compute_fragment_min_max_sum(name); + break; + case Datatype::UINT16: + compute_fragment_min_max_sum(name); + break; + case Datatype::UINT32: + compute_fragment_min_max_sum(name); + break; + case Datatype::UINT64: + compute_fragment_min_max_sum(name); + break; + case Datatype::FLOAT32: + compute_fragment_min_max_sum(name); + break; + case Datatype::FLOAT64: + compute_fragment_min_max_sum(name); + break; + case Datatype::DATETIME_YEAR: + case Datatype::DATETIME_MONTH: + case Datatype::DATETIME_WEEK: + case Datatype::DATETIME_DAY: + case Datatype::DATETIME_HR: + case Datatype::DATETIME_MIN: + case Datatype::DATETIME_SEC: + case Datatype::DATETIME_MS: + case Datatype::DATETIME_US: + case Datatype::DATETIME_NS: + case Datatype::DATETIME_PS: + case Datatype::DATETIME_FS: + case Datatype::DATETIME_AS: + case Datatype::TIME_HR: + case Datatype::TIME_MIN: + case Datatype::TIME_SEC: + case Datatype::TIME_MS: + case Datatype::TIME_US: + case Datatype::TIME_NS: + case Datatype::TIME_PS: + case Datatype::TIME_FS: + case Datatype::TIME_AS: + compute_fragment_min_max_sum(name); + break; + case Datatype::STRING_ASCII: + case Datatype::CHAR: + compute_fragment_min_max_sum(name); + break; + case Datatype::BLOB: + case Datatype::GEOM_WKB: + case Datatype::GEOM_WKT: + compute_fragment_min_max_sum(name); + break; + default: + break; + } + } + }); } void FragmentMetadata::set_array_schema( @@ -757,63 +754,60 @@ std::vector> FragmentMetadata::load( auto fragment_num = fragments_to_load.size(); std::vector> fragment_metadata; fragment_metadata.resize(fragment_num); - auto status = - parallel_for(&resources.compute_tp(), 0, fragment_num, [&](size_t f) { - const auto& sf = fragments_to_load[f]; - URI coords_uri = - sf.uri_.join_path(constants::coords + constants::file_suffix); - - // Note that the fragment metadata version is >= the array schema - // version. Therefore, the check below is defensive and will always - // ensure backwards compatibility. - shared_ptr metadata; - FragmentID fragment_id{sf.uri_}; - if (fragment_id.array_format_version() <= 2) { - bool sparse; - throw_if_not_ok(resources.vfs().is_file(coords_uri, &sparse)); - metadata = make_shared( - HERE(), - &resources, - array_schema_latest, - sf.uri_, - sf.timestamp_range_, - memory_tracker, - !sparse); - } else { - // Fragment format version > 2 - metadata = make_shared( - HERE(), - &resources, - array_schema_latest, - sf.uri_, - sf.timestamp_range_, - memory_tracker); - } + parallel_for(&resources.compute_tp(), 0, fragment_num, [&](size_t f) { + const auto& sf = fragments_to_load[f]; + URI coords_uri = + sf.uri_.join_path(constants::coords + constants::file_suffix); + + // Note that the fragment metadata version is >= the array schema + // version. Therefore, the check below is defensive and will always + // ensure backwards compatibility. + shared_ptr metadata; + FragmentID fragment_id{sf.uri_}; + if (fragment_id.array_format_version() <= 2) { + bool sparse; + throw_if_not_ok(resources.vfs().is_file(coords_uri, &sparse)); + metadata = make_shared( + HERE(), + &resources, + array_schema_latest, + sf.uri_, + sf.timestamp_range_, + memory_tracker, + !sparse); + } else { + // Fragment format version > 2 + metadata = make_shared( + HERE(), + &resources, + array_schema_latest, + sf.uri_, + sf.timestamp_range_, + memory_tracker); + } - // Potentially find the basic fragment metadata in the consolidated - // metadata buffer - Tile* fragment_metadata_tile = nullptr; - uint64_t offset = 0; + // Potentially find the basic fragment metadata in the consolidated + // metadata buffer + Tile* fragment_metadata_tile = nullptr; + uint64_t offset = 0; - auto it = offsets.end(); - if (metadata->format_version() >= 9) { - it = offsets.find(fragment_id.name()); - } else { - it = offsets.find(sf.uri_.to_string()); - } - if (it != offsets.end()) { - fragment_metadata_tile = it->second.first; - offset = it->second.second; - } + auto it = offsets.end(); + if (metadata->format_version() >= 9) { + it = offsets.find(fragment_id.name()); + } else { + it = offsets.find(sf.uri_.to_string()); + } + if (it != offsets.end()) { + fragment_metadata_tile = it->second.first; + offset = it->second.second; + } - // Load fragment metadata - metadata->load( - encryption_key, fragment_metadata_tile, offset, array_schemas_all); + // Load fragment metadata + metadata->load( + encryption_key, fragment_metadata_tile, offset, array_schemas_all); - fragment_metadata[f] = metadata; - return Status::Ok(); - }); - throw_if_not_ok(status); + fragment_metadata[f] = metadata; + }); return fragment_metadata; } diff --git a/tiledb/sm/group/group.cc b/tiledb/sm/group/group.cc index aa8eed1fbf4..37af8645e74 100644 --- a/tiledb/sm/group/group.cc +++ b/tiledb/sm/group/group.cc @@ -845,19 +845,16 @@ void Group::load_metadata_from_storage( auto metadata_num = group_metadata_to_load.size(); // TBD: Might use DynamicArray when it is more capable. std::vector> metadata_tiles(metadata_num); - throw_if_not_ok( - parallel_for(&resources_.compute_tp(), 0, metadata_num, [&](size_t m) { - const auto& uri = group_metadata_to_load[m].uri_; - - metadata_tiles[m] = GenericTileIO::load( - resources_, - uri, - 0, - encryption_key, - resources_.ephemeral_memory_tracker()); - - return Status::Ok(); - })); + parallel_for(&resources_.compute_tp(), 0, metadata_num, [&](size_t m) { + const auto& uri = group_metadata_to_load[m].uri_; + + metadata_tiles[m] = GenericTileIO::load( + resources_, + uri, + 0, + encryption_key, + resources_.ephemeral_memory_tracker()); + }); // Compute array metadata size for the statistics uint64_t meta_size = 0; diff --git a/tiledb/sm/group/group_directory.cc b/tiledb/sm/group/group_directory.cc index 3881c68c32f..297f00ddc86 100644 --- a/tiledb/sm/group/group_directory.cc +++ b/tiledb/sm/group/group_directory.cc @@ -134,19 +134,19 @@ Status GroupDirectory::load() { auto&& [st, uris] = list_root_dir_uris(); throw_if_not_ok(st); - root_dir_uris = std::move(uris.value()); - - return Status::Ok(); + root_dir_uris = std::move(uris).value(); })); // Load (in parallel) the group metadata URIs - tasks.emplace_back(tp_.execute([&]() { return load_group_meta_uris(); })); + tasks.emplace_back( + tp_.execute([&]() { throw_if_not_ok(load_group_meta_uris()); })); // Load (in paralell) the group details URIs - tasks.emplace_back(tp_.execute([&] { return load_group_detail_uris(); })); + tasks.emplace_back( + tp_.execute([&] { throw_if_not_ok(load_group_detail_uris()); })); // Wait for all tasks to complete - throw_if_not_ok(tp_.wait_all(tasks)); + tp_.wait_all(tasks); // Error check bool is_group = false; @@ -268,7 +268,7 @@ GroupDirectory::compute_uris_to_vacuum(const std::vector& uris) const { // Also determine which vac files to vacuum std::vector to_vacuum_vec(uris.size(), 0); std::vector to_vacuum_vac_files_vec(vac_files.size(), 0); - auto status = parallel_for(&tp_, 0, vac_files.size(), [&](size_t i) { + parallel_for(&tp_, 0, vac_files.size(), [&](size_t i) { uint64_t size = 0; throw_if_not_ok(vfs_.file_size(vac_files[i], &size)); std::string names; @@ -288,10 +288,7 @@ GroupDirectory::compute_uris_to_vacuum(const std::vector& uris) const { } to_vacuum_vac_files_vec[i] = vacuum_vac_file; - - return Status::Ok(); }); - RETURN_NOT_OK_TUPLE(status, nullopt, nullopt); // Compute the fragment URIs to vacuum std::vector uris_to_vacuum; diff --git a/tiledb/sm/query/dimension_label/array_dimension_label_queries.cc b/tiledb/sm/query/dimension_label/array_dimension_label_queries.cc index 4f961970d5a..c8b36d35b50 100644 --- a/tiledb/sm/query/dimension_label/array_dimension_label_queries.cc +++ b/tiledb/sm/query/dimension_label/array_dimension_label_queries.cc @@ -151,7 +151,7 @@ std::vector ArrayDimensionLabelQueries::get_data_query( } void ArrayDimensionLabelQueries::process_data_queries() { - throw_if_not_ok(parallel_for( + parallel_for( &resources_.compute_tp(), 0, data_queries_.size(), @@ -160,18 +160,17 @@ void ArrayDimensionLabelQueries::process_data_queries() { try { query->init(); throw_if_not_ok(query->process()); - return Status::Ok(); } catch (const StatusException& err) { throw DimensionLabelQueryException( "Failed to process data query for label '" + query->dim_label_name() + "'. " + err.what()); } - })); + }); } void ArrayDimensionLabelQueries::process_range_queries(Query* parent_query) { // Process queries and update the subarray. - throw_if_not_ok(parallel_for( + parallel_for( &resources_.compute_tp(), 0, label_range_queries_by_dim_idx_.size(), @@ -201,13 +200,12 @@ void ArrayDimensionLabelQueries::process_range_queries(Query* parent_query) { parent_query->add_index_ranges_from_label( dim_idx, is_point_ranges, range_data, count); } - return Status::Ok(); } catch (const StatusException& err) { throw DimensionLabelQueryException( "Failed to process and update index ranges for label '" + range_query->dim_label_name() + "'. " + err.what()); } - })); + }); // Mark the range query as completed. range_query_status_ = QueryStatus::COMPLETED; diff --git a/tiledb/sm/query/legacy/reader.cc b/tiledb/sm/query/legacy/reader.cc index 78b21e07898..b6d05b8175c 100644 --- a/tiledb/sm/query/legacy/reader.cc +++ b/tiledb/sm/query/legacy/reader.cc @@ -571,33 +571,24 @@ Status Reader::compute_range_result_coords( } } - auto status = - parallel_for(&resources_.compute_tp(), 0, range_num, [&](uint64_t r) { - // Compute overlapping coordinates per range - throw_if_not_ok(compute_range_result_coords( - subarray, - r, - result_tile_map, - result_tiles, - range_result_coords[r])); - - // Dedup unless there is a single fragment or array schema allows - // duplicates - if (!single_fragment[r] && !allows_dups) { - throw_if_not_ok(sort_result_coords( - range_result_coords[r].begin(), - range_result_coords[r].end(), - range_result_coords[r].size(), - sort_layout)); - throw_if_cancelled(); - throw_if_not_ok(dedup_result_coords(range_result_coords[r])); - throw_if_cancelled(); - } - - return Status::Ok(); - }); - - RETURN_NOT_OK(status); + parallel_for(&resources_.compute_tp(), 0, range_num, [&](uint64_t r) { + // Compute overlapping coordinates per range + throw_if_not_ok(compute_range_result_coords( + subarray, r, result_tile_map, result_tiles, range_result_coords[r])); + + // Dedup unless there is a single fragment or array schema allows + // duplicates + if (!single_fragment[r] && !allows_dups) { + throw_if_not_ok(sort_result_coords( + range_result_coords[r].begin(), + range_result_coords[r].end(), + range_result_coords[r].size(), + sort_layout)); + throw_if_cancelled(); + throw_if_not_ok(dedup_result_coords(range_result_coords[r])); + throw_if_cancelled(); + } + }); return Status::Ok(); } @@ -671,18 +662,15 @@ Status Reader::compute_range_result_coords( // Gather result range coordinates per fragment auto fragment_num = fragment_metadata_.size(); std::vector> range_result_coords_vec(fragment_num); - auto status = - parallel_for(&resources_.compute_tp(), 0, fragment_num, [&](uint32_t f) { - throw_if_not_ok(compute_range_result_coords( - subarray, - range_idx, - f, - result_tile_map, - result_tiles, - range_result_coords_vec[f])); - return Status::Ok(); - }); - RETURN_NOT_OK(status); + parallel_for(&resources_.compute_tp(), 0, fragment_num, [&](uint32_t f) { + throw_if_not_ok(compute_range_result_coords( + subarray, + range_idx, + f, + result_tile_map, + result_tiles, + range_result_coords_vec[f])); + }); // Consolidate the result coordinates in the single result vector for (const auto& vec : range_result_coords_vec) { @@ -959,22 +947,19 @@ Status Reader::copy_fixed_cells( } // Copy result cell slabs in parallel. - std::function copy_fn = std::bind( - &Reader::copy_partitioned_fixed_cells, - this, - std::placeholders::_1, - &name, - stride, - result_cell_slabs, - &cs_offsets, - fixed_cs_partitions); - auto status = parallel_for( + parallel_for( &resources_.compute_tp(), 0, fixed_cs_partitions->size(), - std::move(copy_fn)); - - RETURN_NOT_OK(status); + std::bind( + &Reader::copy_partitioned_fixed_cells, + this, + std::placeholders::_1, + &name, + stride, + result_cell_slabs, + &cs_offsets, + fixed_cs_partitions)); // Update buffer offsets *(buffers_[name].buffer_size_) = buffer_offset; @@ -1012,7 +997,7 @@ void Reader::compute_fixed_cs_partitions( } } -Status Reader::copy_partitioned_fixed_cells( +void Reader::copy_partitioned_fixed_cells( const size_t partition_idx, const std::string* const name, const uint64_t stride, @@ -1087,20 +1072,20 @@ Status Reader::copy_partitioned_fixed_cells( if (stride == UINT64_MAX) { if (!nullable) - RETURN_NOT_OK(cs.tile_->read( + throw_if_not_ok(cs.tile_->read( *name, buffer, offset, cs.start_, cs_length, timestamp)); else - RETURN_NOT_OK(cs.tile_->read_nullable( + throw_if_not_ok(cs.tile_->read_nullable( *name, buffer, offset, cs.start_, cs_length, buffer_validity)); } else { auto cell_offset = offset; auto start = cs.start_; for (uint64_t j = 0; j < cs_length; ++j) { if (!nullable) - RETURN_NOT_OK(cs.tile_->read( + throw_if_not_ok(cs.tile_->read( *name, buffer, cell_offset, start, 1, timestamp)); else - RETURN_NOT_OK(cs.tile_->read_nullable( + throw_if_not_ok(cs.tile_->read_nullable( *name, buffer, cell_offset, start, 1, buffer_validity)); cell_offset += cell_size; start += stride; @@ -1108,8 +1093,6 @@ Status Reader::copy_partitioned_fixed_cells( } } } - - return Status::Ok(); } Status Reader::copy_var_cells( @@ -1148,20 +1131,20 @@ Status Reader::copy_var_cells( } // Copy result cell slabs in parallel - std::function copy_fn = std::bind( - &Reader::copy_partitioned_var_cells, - this, - std::placeholders::_1, - &name, - stride, - result_cell_slabs, - &offset_offsets_per_cs, - &var_offsets_per_cs, - var_cs_partitions); - auto status = parallel_for( - &resources_.compute_tp(), 0, var_cs_partitions->size(), copy_fn); - - RETURN_NOT_OK(status); + parallel_for( + &resources_.compute_tp(), + 0, + var_cs_partitions->size(), + std::bind( + &Reader::copy_partitioned_var_cells, + this, + std::placeholders::_1, + &name, + stride, + result_cell_slabs, + &offset_offsets_per_cs, + &var_offsets_per_cs, + var_cs_partitions)); // Update buffer offsets *(buffers_[name].buffer_size_) = total_offset_size; @@ -1315,7 +1298,7 @@ Status Reader::compute_var_cell_destinations( return Status::Ok(); } -Status Reader::copy_partitioned_var_cells( +void Reader::copy_partitioned_var_cells( const size_t partition_idx, const std::string* const name, uint64_t stride, @@ -1412,8 +1395,6 @@ Status Reader::copy_partitioned_var_cells( arr_offset += cs_length; } - - return Status::Ok(); } Status Reader::process_tiles( @@ -2178,20 +2159,16 @@ Status Reader::calculate_hilbert_values( auto coords_num = (uint64_t)hilbert_values->size(); // Calculate Hilbert values in parallel - auto status = - parallel_for(&resources_.compute_tp(), 0, coords_num, [&](uint64_t c) { - std::vector coords(dim_num); - for (uint32_t d = 0; d < dim_num; ++d) { - auto dim{array_schema_.dimension_ptr(d)}; - coords[d] = hilbert_order::map_to_uint64( - *dim, *(iter_begin + c), d, bits, max_bucket_val); - } - (*hilbert_values)[c] = - std::pair(h.coords_to_hilbert(&coords[0]), c); - return Status::Ok(); - }); - - RETURN_NOT_OK_ELSE(status, throw_if_not_ok(logger_->status(status))); + parallel_for(&resources_.compute_tp(), 0, coords_num, [&](uint64_t c) { + std::vector coords(dim_num); + for (uint32_t d = 0; d < dim_num; ++d) { + auto dim{array_schema_.dimension_ptr(d)}; + coords[d] = hilbert_order::map_to_uint64( + *dim, *(iter_begin + c), d, bits, max_bucket_val); + } + (*hilbert_values)[c] = + std::pair(h.coords_to_hilbert(&coords[0]), c); + }); return Status::Ok(); } diff --git a/tiledb/sm/query/legacy/reader.h b/tiledb/sm/query/legacy/reader.h index 43e55f2123e..0d9b3b67417 100644 --- a/tiledb/sm/query/legacy/reader.h +++ b/tiledb/sm/query/legacy/reader.h @@ -486,9 +486,8 @@ class Reader : public ReaderBase, public IQueryStrategy { * @param result_cell_slabs The result cell slabs to copy cells for. * @param cs_offsets The cell slab offsets. * @param cs_partitions The cell slab partitions to operate on. - * @return Status */ - Status copy_partitioned_fixed_cells( + void copy_partitioned_fixed_cells( size_t partition_idx, const std::string* name, uint64_t stride, @@ -578,9 +577,8 @@ class Reader : public ReaderBase, public IQueryStrategy { * @param var_offsets_per_cs Maps each cell slab to its offset * for its variable-length data. * @param cs_partitions The cell slab partitions to operate on. - * @return Status */ - Status copy_partitioned_var_cells( + void copy_partitioned_var_cells( size_t partition_idx, const std::string* name, uint64_t stride, diff --git a/tiledb/sm/query/query.cc b/tiledb/sm/query/query.cc index 8fd1189db3d..b671ca4e391 100644 --- a/tiledb/sm/query/query.cc +++ b/tiledb/sm/query/query.cc @@ -833,11 +833,10 @@ Status Query::process() { enmr_names.emplace_back(enmr_name); } - throw_if_not_ok(parallel_for( + parallel_for( &resources_.compute_tp(), 0, enmr_names.size(), [&](const uint64_t i) { array_->get_enumeration(enmr_names[i]); - return Status::Ok(); - })); + }); condition_->rewrite_enumeration_conditions(array_schema()); } diff --git a/tiledb/sm/query/readers/dense_reader.cc b/tiledb/sm/query/readers/dense_reader.cc index fc0d678d012..ae8a3e177d7 100644 --- a/tiledb/sm/query/readers/dense_reader.cc +++ b/tiledb/sm/query/readers/dense_reader.cc @@ -249,13 +249,11 @@ Status DenseReader::dense_read() { std::vector tiles_cell_num(tile_coords.size()); { auto timer_se = stats_->start_timer("compute_tiles_cell_num"); - auto status = parallel_for( + parallel_for( &resources_.compute_tp(), 0, tile_coords.size(), [&](uint64_t t) { tiles_cell_num[t] = subarray.tile_cell_num((const DimType*)&tile_coords[t][0]); - return Status::Ok(); }); - RETURN_NOT_OK(status); } // Compute tile offsets for global order or range info for row/col major. @@ -429,7 +427,7 @@ Status DenseReader::dense_read() { // prevent using too much memory when the budget is small and doesn't allow // to process more than one batch at a time. if (wait_compute_task_before_read && compute_task.valid()) { - throw_if_not_ok(resources_.compute_tp().wait(compute_task)); + resources_.compute_tp().wait(compute_task); } // Apply the query condition. @@ -475,7 +473,7 @@ Status DenseReader::dense_read() { // is to prevent using too much memory when the budget is small and // doesn't allow to process more than one batch at a time. if (wait_compute_task_before_read && compute_task.valid()) { - throw_if_not_ok(resources_.compute_tp().wait(compute_task)); + resources_.compute_tp().wait(compute_task); } // Read and unfilter tiles. @@ -486,7 +484,7 @@ Status DenseReader::dense_read() { } if (compute_task.valid()) { - throw_if_not_ok(resources_.compute_tp().wait(compute_task)); + resources_.compute_tp().wait(compute_task); if (read_state_.overflowed_) { return Status::Ok(); } @@ -504,7 +502,7 @@ Status DenseReader::dense_read() { result_tiles]() mutable { if (!dense_dim) { // Unfilter tiles. - RETURN_NOT_OK(unfilter_tiles(name, validity_only, result_tiles)); + throw_if_not_ok(unfilter_tiles(name, validity_only, result_tiles)); // The filtered data is no longer required, release it. filtered_data.reset(); @@ -513,7 +511,7 @@ Status DenseReader::dense_read() { if (buffers_.count(name) != 0) { // Copy attribute data to users buffers. auto& var_buffer_size = var_buffer_sizes[name]; - auto status = copy_attribute( + throw_if_not_ok(copy_attribute( name, tile_extents, subarray, @@ -524,13 +522,13 @@ Status DenseReader::dense_read() { range_info, iteration_tile_data, qc_result, - num_range_threads); - RETURN_CANCEL_OR_ERROR(status); + num_range_threads)); + throw_if_cancelled(); } } if (aggregates_.count(name) != 0) { - auto status = process_aggregates( + throw_if_not_ok(process_aggregates( name, tile_extents, subarray, @@ -539,15 +537,13 @@ Status DenseReader::dense_read() { range_info, iteration_tile_data, qc_result, - num_range_threads); - RETURN_CANCEL_OR_ERROR(status); + num_range_threads)); + throw_if_cancelled(); } if (!dense_dim) { clear_tiles(name, result_tiles); } - - return Status::Ok(); }); } @@ -575,7 +571,7 @@ Status DenseReader::dense_read() { } if (compute_task.valid()) { - throw_if_not_ok(resources_.compute_tp().wait(compute_task)); + resources_.compute_tp().wait(compute_task); } // For `qc_coords_mode` just fill in the coordinates and skip attribute @@ -1068,7 +1064,7 @@ Status DenseReader::apply_query_condition( NameToLoad::from_string_vec(qc_names), result_tiles)); if (compute_task.valid()) { - throw_if_not_ok(resources_.compute_tp().wait(compute_task)); + resources_.compute_tp().wait(compute_task); } compute_task = resources_.compute_tp().execute([&, @@ -1086,7 +1082,7 @@ Status DenseReader::apply_query_condition( // Unfilter tiles. for (auto& name : qc_names) { - RETURN_NOT_OK(unfilter_tiles(name, false, result_tiles)); + throw_if_not_ok(unfilter_tiles(name, false, result_tiles)); } // The filtered data is no longer required, release it. @@ -1097,7 +1093,7 @@ Status DenseReader::apply_query_condition( } // Process all tiles in parallel. - auto status = parallel_for_2d( + parallel_for_2d( &resources_.compute_tp(), iteration_tile_data->t_start(), iteration_tile_data->t_end(), @@ -1184,10 +1180,7 @@ Status DenseReader::apply_query_condition( ++iter; } - - return Status::Ok(); }); - throw_if_not_ok(status); // For `qc_coords_mode` just fill in the coordinates and skip // attribute @@ -1197,8 +1190,6 @@ Status DenseReader::apply_query_condition( clear_tiles(name, result_tiles); } } - - return Status::Ok(); }); } @@ -1276,7 +1267,7 @@ Status DenseReader::copy_attribute( // Process offsets. { auto timer_se = stats_->start_timer("copy_offset_tiles"); - auto status = parallel_for_2d( + parallel_for_2d( &resources_.compute_tp(), iteration_tile_data->t_start(), iteration_tile_data->t_end(), @@ -1301,9 +1292,7 @@ Status DenseReader::copy_attribute( qc_result, range_thread_idx, num_range_threads)); - return Status::Ok(); }); - RETURN_NOT_OK(status); } { @@ -1337,7 +1326,7 @@ Status DenseReader::copy_attribute( { auto timer_se = stats_->start_timer("copy_var_tiles"); // Process var data in parallel. - auto status = parallel_for_2d( + parallel_for_2d( &resources_.compute_tp(), iteration_tile_data->t_start(), iteration_tile_data->t_end(), @@ -1362,10 +1351,7 @@ Status DenseReader::copy_attribute( var_buffer_size, range_thread_idx, num_range_threads)); - - return Status::Ok(); }); - RETURN_NOT_OK(status); } } else { // Make sure the user fixed buffer is big enough. @@ -1379,7 +1365,7 @@ Status DenseReader::copy_attribute( { auto timer_se = stats_->start_timer("copy_fixed_tiles"); // Process values in parallel. - auto status = parallel_for_2d( + parallel_for_2d( &resources_.compute_tp(), iteration_tile_data->t_start(), iteration_tile_data->t_end(), @@ -1402,10 +1388,7 @@ Status DenseReader::copy_attribute( qc_result, range_thread_idx, num_range_threads)); - - return Status::Ok(); }); - RETURN_NOT_OK(status); } // Set the output size for the fixed buffer. @@ -1478,7 +1461,7 @@ Status DenseReader::process_aggregates( const auto global_order = layout_ == Layout::GLOBAL_ORDER; // Process values in parallel. - auto status = parallel_for_2d( + parallel_for_2d( &resources_.compute_tp(), iteration_tile_data->t_start(), iteration_tile_data->t_end(), @@ -1513,10 +1496,7 @@ Status DenseReader::process_aggregates( range_thread_idx, num_range_threads)); } - - return Status::Ok(); }); - RETURN_NOT_OK(status); return Status::Ok(); } diff --git a/tiledb/sm/query/readers/dense_reader.h b/tiledb/sm/query/readers/dense_reader.h index 2434839157e..dc8b2e502d2 100644 --- a/tiledb/sm/query/readers/dense_reader.h +++ b/tiledb/sm/query/readers/dense_reader.h @@ -80,13 +80,10 @@ class DenseReader : public ReaderBase, public IQueryStrategy { memory_tracker_->get_resource(MemoryType::DENSE_TILE_SUBARRAY)) , result_space_tiles_(std::move(result_space_tiles)) { auto& tile_coords = subarray.tile_coords(); - throw_if_not_ok( - parallel_for(&compute_tp, 0, tile_subarrays_.size(), [&](uint64_t t) { - subarray.crop_to_tile( - tile_subarrays_[t], - (const DimType*)&tile_coords[t + t_start][0]); - return Status::Ok(); - })); + parallel_for(&compute_tp, 0, tile_subarrays_.size(), [&](uint64_t t) { + subarray.crop_to_tile( + tile_subarrays_[t], (const DimType*)&tile_coords[t + t_start][0]); + }); }; DISABLE_COPY_AND_COPY_ASSIGN(IterationTileData); diff --git a/tiledb/sm/query/readers/filtered_data.h b/tiledb/sm/query/readers/filtered_data.h index 455fc71a2b0..f34fb630f7f 100644 --- a/tiledb/sm/query/readers/filtered_data.h +++ b/tiledb/sm/query/readers/filtered_data.h @@ -394,11 +394,11 @@ class FilteredData { auto data{block.data()}; auto size{block.size()}; URI uri{file_uri(fragment_metadata_[block.frag_idx()].get(), type)}; - auto task = resources_.io_tp().execute([this, offset, data, size, uri]() { - throw_if_not_ok(resources_.vfs().read(uri, offset, data, size, false)); - return Status::Ok(); - }); - read_tasks_.push_back(std::move(task)); + read_tasks_.push_back( + resources_.io_tp().execute([this, offset, data, size, uri]() { + throw_if_not_ok( + resources_.vfs().read(uri, offset, data, size, false)); + })); } /** @return Data blocks corresponding to the tile type. */ diff --git a/tiledb/sm/query/readers/ordered_dim_label_reader.cc b/tiledb/sm/query/readers/ordered_dim_label_reader.cc index e26a3855d3e..d0646608973 100644 --- a/tiledb/sm/query/readers/ordered_dim_label_reader.cc +++ b/tiledb/sm/query/readers/ordered_dim_label_reader.cc @@ -245,11 +245,9 @@ void OrderedDimLabelReader::label_read() { read_and_unfilter_attribute_tiles({label_name_}, result_tiles)); // Compute/copy results. - throw_if_not_ok( - parallel_for(&resources_.compute_tp(), 0, max_range, [&](uint64_t r) { - compute_and_copy_range_indexes(buffer_offset, r); - return Status::Ok(); - })); + parallel_for(&resources_.compute_tp(), 0, max_range, [&](uint64_t r) { + compute_and_copy_range_indexes(buffer_offset, r); + }); // Truncate ranges_ for the next iteration. for (auto& rt_map : result_tiles_) { @@ -286,7 +284,7 @@ void OrderedDimLabelReader::compute_array_tile_indexes_for_ranges() { for (uint64_t r = 0; r < ranges_.size(); r++) { per_range_array_tile_indexes[r].resize(fragment_metadata_.size()); } - throw_if_not_ok(parallel_for_2d( + parallel_for_2d( &resources_.compute_tp(), 0, fragment_metadata_.size(), @@ -295,18 +293,15 @@ void OrderedDimLabelReader::compute_array_tile_indexes_for_ranges() { [&](uint64_t f, uint64_t r) { per_range_array_tile_indexes[r][f] = get_array_tile_indexes_for_range(f, r); - return Status::Ok(); - })); + }); // Compute the tile indexes (min/max) that can potentially contain the label // value for each range start/end. per_range_array_tile_indexes_.resize(ranges_.size()); - throw_if_not_ok(parallel_for( - &resources_.compute_tp(), 0, ranges_.size(), [&](uint64_t r) { - per_range_array_tile_indexes_[r] = RangeTileIndexes( - tile_idx_min, tile_idx_max, per_range_array_tile_indexes[r]); - return Status::Ok(); - })); + parallel_for(&resources_.compute_tp(), 0, ranges_.size(), [&](uint64_t r) { + per_range_array_tile_indexes_[r] = RangeTileIndexes( + tile_idx_min, tile_idx_max, per_range_array_tile_indexes[r]); + }); } void OrderedDimLabelReader::load_label_min_max_values() { @@ -314,7 +309,7 @@ void OrderedDimLabelReader::load_label_min_max_values() { const auto encryption_key = array_->encryption_key(); // Load min/max data for all fragments. - throw_if_not_ok(parallel_for( + parallel_for( &resources_.compute_tp(), 0, fragment_metadata_.size(), @@ -325,8 +320,7 @@ void OrderedDimLabelReader::load_label_min_max_values() { *encryption_key, names); fragment->loaded_metadata()->load_tile_max_values( *encryption_key, names); - return Status::Ok(); - })); + }); } template diff --git a/tiledb/sm/query/readers/reader_base.cc b/tiledb/sm/query/readers/reader_base.cc index 083d202fa15..50f9ab22ce0 100644 --- a/tiledb/sm/query/readers/reader_base.cc +++ b/tiledb/sm/query/readers/reader_base.cc @@ -501,7 +501,7 @@ void ReaderBase::load_tile_offsets( auto timer_se = stats_->start_timer("load_tile_offsets"); const auto encryption_key = array_->encryption_key(); - throw_if_not_ok(parallel_for( + parallel_for( &resources_.compute_tp(), 0, relevant_fragments.size(), @@ -522,8 +522,7 @@ void ReaderBase::load_tile_offsets( fragment->loaded_metadata()->load_tile_offsets( *encryption_key, filtered_names); - return Status::Ok(); - })); + }); } void ReaderBase::load_tile_var_sizes( @@ -532,7 +531,7 @@ void ReaderBase::load_tile_var_sizes( auto timer_se = stats_->start_timer("load_tile_var_sizes"); const auto encryption_key = array_->encryption_key(); - throw_if_not_ok(parallel_for( + parallel_for( &resources_.compute_tp(), 0, relevant_fragments.size(), @@ -556,9 +555,7 @@ void ReaderBase::load_tile_var_sizes( fragment->loaded_metadata()->load_tile_var_sizes( *encryption_key, name); } - - return Status::Ok(); - })); + }); } void ReaderBase::load_tile_metadata( @@ -567,7 +564,7 @@ void ReaderBase::load_tile_metadata( auto timer_se = stats_->start_timer("load_tile_metadata"); const auto encryption_key = array_->encryption_key(); - throw_if_not_ok(parallel_for( + parallel_for( &resources_.compute_tp(), 0, relevant_fragments.size(), @@ -598,9 +595,7 @@ void ReaderBase::load_tile_metadata( *encryption_key, to_load); fragment->loaded_metadata()->load_tile_null_count_values( *encryption_key, to_load); - - return Status::Ok(); - })); + }); } void ReaderBase::load_processed_conditions() { @@ -608,7 +603,7 @@ void ReaderBase::load_processed_conditions() { const auto encryption_key = array_->encryption_key(); // Load all fragments in parallel. - throw_if_not_ok(parallel_for( + parallel_for( &resources_.compute_tp(), 0, fragment_metadata_.size(), @@ -619,9 +614,7 @@ void ReaderBase::load_processed_conditions() { fragment->loaded_metadata()->load_processed_conditions( *encryption_key); } - - return Status::Ok(); - })); + }); } Status ReaderBase::read_and_unfilter_attribute_tiles( @@ -773,10 +766,7 @@ std::list ReaderBase::read_tiles( stats_->add_counter("num_tiles_read", num_tiles_read); // Wait for the read tasks to finish. - auto statuses{resources_.io_tp().wait_all_status(read_tasks)}; - for (const auto& st : statuses) { - throw_if_not_ok(st); - } + resources_.io_tp().wait_all(read_tasks); return filtered_data; } @@ -935,31 +925,28 @@ Status ReaderBase::unfilter_tiles( std::vector unfiltered_tile_validity_size(num_tiles); // Pre-compute chunk offsets. - auto status = parallel_for( - &resources_.compute_tp(), 0, num_tiles, [&, this](uint64_t i) { - auto&& [st, tile_size, tile_var_size, tile_validity_size] = - load_tile_chunk_data( - name, - validity_only, - result_tiles[i], - var_size, - nullable, - tiles_chunk_data[i], - tiles_chunk_var_data[i], - tiles_chunk_validity_data[i]); - throw_if_not_ok(st); - unfiltered_tile_size[i] = tile_size.value(); - unfiltered_tile_var_size[i] = tile_var_size.value(); - unfiltered_tile_validity_size[i] = tile_validity_size.value(); - return Status::Ok(); - }); - RETURN_NOT_OK_ELSE(status, throw_if_not_ok(logger_->status(status))); + parallel_for(&resources_.compute_tp(), 0, num_tiles, [&, this](uint64_t i) { + auto&& [st, tile_size, tile_var_size, tile_validity_size] = + load_tile_chunk_data( + name, + validity_only, + result_tiles[i], + var_size, + nullable, + tiles_chunk_data[i], + tiles_chunk_var_data[i], + tiles_chunk_validity_data[i]); + throw_if_not_ok(st); + unfiltered_tile_size[i] = tile_size.value(); + unfiltered_tile_var_size[i] = tile_var_size.value(); + unfiltered_tile_validity_size[i] = tile_validity_size.value(); + }); if (tiles_chunk_data.empty()) return Status::Ok(); // Unfilter all tiles/chunks in parallel using the precomputed offsets. - status = parallel_for_2d( + parallel_for_2d( &resources_.compute_tp(), 0, num_tiles, @@ -977,9 +964,7 @@ Status ReaderBase::unfilter_tiles( tiles_chunk_data[i], tiles_chunk_var_data[i], tiles_chunk_validity_data[i])); - return Status::Ok(); }); - RETURN_CANCEL_OR_ERROR(status); // Perform required post-processing of unfiltered tiles for (size_t i = 0; i < num_tiles; i++) { @@ -1221,16 +1206,14 @@ ReaderBase::cache_dimension_label_data() { auto tile_extent{index_dim->tile_extent().rvalue_as()}; std::vector non_empty_domains(fragment_metadata_.size()); std::vector frag_first_array_tile_idx(fragment_metadata_.size()); - throw_if_not_ok(parallel_for( + parallel_for( &resources_.compute_tp(), 0, fragment_metadata_.size(), [&](unsigned f) { non_empty_domains[f] = fragment_metadata_[f]->non_empty_domain()[0].data(); auto ned = static_cast(non_empty_domains[f]); frag_first_array_tile_idx[f] = index_dim->tile_idx(ned[0], dim_dom[0], tile_extent); - - return Status::Ok(); - })); + }); // Compute the array non empty domain. IndexType min = std::numeric_limits::max(); @@ -1268,15 +1251,13 @@ void ReaderBase::validate_attribute_order( // See if some values will already be processed by previous fragments. AttributeOrderValidator validator( attribute_name, fragment_metadata_.size(), query_memory_tracker_); - throw_if_not_ok(parallel_for( + parallel_for( &resources_.compute_tp(), 0, fragment_metadata_.size(), [&](uint64_t f) { validator.find_fragments_to_check( array_min_idx, array_max_idx, f, non_empty_domains); + }); - return Status::Ok(); - })); - - throw_if_not_ok(parallel_for( + parallel_for( &resources_.compute_tp(), 0, fragment_metadata_.size(), [&](int64_t f) { validator.validate_without_loading_tiles( index_dim, @@ -1285,8 +1266,7 @@ void ReaderBase::validate_attribute_order( non_empty_domains, fragment_metadata_, frag_first_array_tile_idx); - return Status::Ok(); - })); + }); // If we need tiles to finish order validation, load them, then finish the // validation. @@ -1297,7 +1277,7 @@ void ReaderBase::validate_attribute_order( read_and_unfilter_attribute_tiles({attribute_name}, tiles_to_load)); // Validate bounds not validated using tile data. - throw_if_not_ok(parallel_for( + parallel_for( &resources_.compute_tp(), 0, fragment_metadata_.size(), @@ -1309,8 +1289,7 @@ void ReaderBase::validate_attribute_order( non_empty_domains, fragment_metadata_, frag_first_array_tile_idx); - return Status::Ok(); - })); + }); } } diff --git a/tiledb/sm/query/readers/sparse_global_order_reader.cc b/tiledb/sm/query/readers/sparse_global_order_reader.cc index de157a8bf88..9594cafca83 100644 --- a/tiledb/sm/query/readers/sparse_global_order_reader.cc +++ b/tiledb/sm/query/readers/sparse_global_order_reader.cc @@ -359,101 +359,95 @@ SparseGlobalOrderReader::create_result_tiles( // Create result tiles. if (subarray_.is_set()) { // Load as many tiles as the memory budget allows. - throw_if_not_ok(parallel_for( - &resources_.compute_tp(), 0, fragment_num, [&](uint64_t f) { - uint64_t t = 0; - auto& tile_ranges = tmp_read_state_.tile_ranges(f); - while (!tile_ranges.empty()) { - auto& range = tile_ranges.back(); - for (t = range.first; t <= range.second; t++) { - auto budget_exceeded = add_result_tile( - dim_num, - per_fragment_memory_, - f, - t, - *fragment_metadata_[f], - result_tiles); - - if (budget_exceeded) { - logger_->debug( - "Budget exceeded adding result tiles, fragment {0}, tile " - "{1}", - f, - t); - - if (result_tiles[f].empty()) { - auto tiles_size = get_coord_tiles_size(dim_num, f, t); - throw SparseGlobalOrderReaderException( - "Cannot load a single tile for fragment, increase " - "memory " - "budget, tile size : " + - std::to_string(tiles_size) + ", per fragment memory " + - std::to_string(per_fragment_memory_) + ", total budget " + - std::to_string(memory_budget_.total_budget()) + - ", num fragments to process " + - std::to_string(num_fragments_to_process)); - } - return Status::Ok(); - } - - range.first++; + parallel_for(&resources_.compute_tp(), 0, fragment_num, [&](uint64_t f) { + uint64_t t = 0; + auto& tile_ranges = tmp_read_state_.tile_ranges(f); + while (!tile_ranges.empty()) { + auto& range = tile_ranges.back(); + for (t = range.first; t <= range.second; t++) { + auto budget_exceeded = add_result_tile( + dim_num, + per_fragment_memory_, + f, + t, + *fragment_metadata_[f], + result_tiles); + + if (budget_exceeded) { + logger_->debug( + "Budget exceeded adding result tiles, fragment {0}, tile " + "{1}", + f, + t); + + if (result_tiles[f].empty()) { + auto tiles_size = get_coord_tiles_size(dim_num, f, t); + throw SparseGlobalOrderReaderException( + "Cannot load a single tile for fragment, increase " + "memory " + "budget, tile size : " + + std::to_string(tiles_size) + ", per fragment memory " + + std::to_string(per_fragment_memory_) + ", total budget " + + std::to_string(memory_budget_.total_budget()) + + ", num fragments to process " + + std::to_string(num_fragments_to_process)); } - - tmp_read_state_.remove_tile_range(f); + return; } - tmp_read_state_.set_all_tiles_loaded(f); + range.first++; + } + + tmp_read_state_.remove_tile_range(f); + } - return Status::Ok(); - })); + tmp_read_state_.set_all_tiles_loaded(f); + }); } else { // Load as many tiles as the memory budget allows. - throw_if_not_ok(parallel_for( - &resources_.compute_tp(), 0, fragment_num, [&](uint64_t f) { - uint64_t t = 0; - auto tile_num = fragment_metadata_[f]->tile_num(); - - // Figure out the start index. - auto start = read_state_.frag_idx()[f].tile_idx_; - if (!result_tiles[f].empty()) { - start = std::max(start, result_tiles[f].back().tile_idx() + 1); - } + parallel_for(&resources_.compute_tp(), 0, fragment_num, [&](uint64_t f) { + uint64_t t = 0; + auto tile_num = fragment_metadata_[f]->tile_num(); + + // Figure out the start index. + auto start = read_state_.frag_idx()[f].tile_idx_; + if (!result_tiles[f].empty()) { + start = std::max(start, result_tiles[f].back().tile_idx() + 1); + } - for (t = start; t < tile_num; t++) { - auto budget_exceeded = add_result_tile( - dim_num, - per_fragment_memory_, - f, - t, - *fragment_metadata_[f], - result_tiles); - - if (budget_exceeded) { - logger_->debug( - "Budget exceeded adding result tiles, fragment {0}, tile " - "{1}", - f, - t); - - if (result_tiles[f].empty()) { - auto tiles_size = get_coord_tiles_size(dim_num, f, t); - return logger_->status(Status_SparseGlobalOrderReaderError( - "Cannot load a single tile for fragment, increase memory " - "budget, tile size : " + - std::to_string(tiles_size) + ", per fragment memory " + - std::to_string(per_fragment_memory_) + ", total budget " + - std::to_string(memory_budget_.total_budget()) + - ", num fragments to process " + - std::to_string(num_fragments_to_process))); - } - return Status::Ok(); - } + for (t = start; t < tile_num; t++) { + auto budget_exceeded = add_result_tile( + dim_num, + per_fragment_memory_, + f, + t, + *fragment_metadata_[f], + result_tiles); + + if (budget_exceeded) { + logger_->debug( + "Budget exceeded adding result tiles, fragment {0}, tile " + "{1}", + f, + t); + + if (result_tiles[f].empty()) { + auto tiles_size = get_coord_tiles_size(dim_num, f, t); + throw SparseGlobalOrderReaderException( + "Cannot load a single tile for fragment, increase memory " + "budget, tile size : " + + std::to_string(tiles_size) + ", per fragment memory " + + std::to_string(per_fragment_memory_) + ", total budget " + + std::to_string(memory_budget_.total_budget()) + + ", num fragments to process " + + std::to_string(num_fragments_to_process)); } + return; + } + } - tmp_read_state_.set_all_tiles_loaded(f); - - return Status::Ok(); - })); + tmp_read_state_.set_all_tiles_loaded(f); + }); } bool done_adding_result_tiles = tmp_read_state_.done_adding_result_tiles(); @@ -488,20 +482,17 @@ void SparseGlobalOrderReader::clean_tile_list( std::vector& result_tiles) { // Clear result tiles that are not necessary anymore. auto fragment_num = fragment_metadata_.size(); - throw_if_not_ok( - parallel_for(&resources_.compute_tp(), 0, fragment_num, [&](uint64_t f) { - auto it = result_tiles[f].begin(); - while (it != result_tiles[f].end()) { - if (it->result_num() == 0) { - tmp_read_state_.add_ignored_tile(*it); - remove_result_tile(f, it++, result_tiles); - } else { - it++; - } - } - - return Status::Ok(); - })); + parallel_for(&resources_.compute_tp(), 0, fragment_num, [&](uint64_t f) { + auto it = result_tiles[f].begin(); + while (it != result_tiles[f].end()) { + if (it->result_num() == 0) { + tmp_read_state_.add_ignored_tile(*it); + remove_result_tile(f, it++, result_tiles); + } else { + it++; + } + } + }); } template @@ -516,7 +507,7 @@ void SparseGlobalOrderReader::dedup_tiles_with_timestamps( auto timer_se = stats_->start_timer("dedup_tiles_with_timestamps"); // Process all tiles in parallel. - throw_if_not_ok(parallel_for( + parallel_for( &resources_.compute_tp(), 0, result_tiles.size(), [&](uint64_t t) { const auto f = result_tiles[t]->frag_idx(); if (fragment_metadata_[f]->has_timestamps()) { @@ -572,9 +563,7 @@ void SparseGlobalOrderReader::dedup_tiles_with_timestamps( // Count new number of cells in the bitmap. rt->count_cells(); } - - return Status::Ok(); - })); + }); logger_->debug("Done processing fragments with timestamps"); } @@ -592,64 +581,60 @@ void SparseGlobalOrderReader::dedup_fragments_with_timestamps( // Run all fragments in parallel. auto fragment_num = fragment_metadata_.size(); - throw_if_not_ok( - parallel_for(&resources_.compute_tp(), 0, fragment_num, [&](uint64_t f) { - // Run only for fragments with timestamps. - if (fragment_metadata_[f]->has_timestamps()) { - // Process all tiles. - auto it = result_tiles[f].begin(); - while (it != result_tiles[f].end()) { - // Compare the current tile to the next. - auto next_tile = it; - next_tile++; - if (next_tile == result_tiles[f].end()) { - // No more tiles, save the last cell for this fragment for later - // processing. - last_cells_[f] = - FragIdx(it->tile_idx(), it->last_cell_in_bitmap()); - it++; + parallel_for(&resources_.compute_tp(), 0, fragment_num, [&](uint64_t f) { + // Run only for fragments with timestamps. + if (fragment_metadata_[f]->has_timestamps()) { + // Process all tiles. + auto it = result_tiles[f].begin(); + while (it != result_tiles[f].end()) { + // Compare the current tile to the next. + auto next_tile = it; + next_tile++; + if (next_tile == result_tiles[f].end()) { + // No more tiles, save the last cell for this fragment for later + // processing. + last_cells_[f] = FragIdx(it->tile_idx(), it->last_cell_in_bitmap()); + it++; + } else { + // Compare the last tile from current to the first from next. + auto last = it->last_cell_in_bitmap(); + auto first = next_tile->first_cell_in_bitmap(); + if (!it->same_coords(*next_tile, last, first)) { + // Not the same coords, move to the next tile. + it++; + } else { + // Same coords, compare timestamps. + if (it->timestamp(last) > next_tile->timestamp(first)) { + // Remove the cell in the next tile. + if (next_tile->result_num() == 1) { + // Only one cell in the bitmap, delete next tile. + // Stay on this tile as we will compare to the new next. + tmp_read_state_.add_ignored_tile(*next_tile); + remove_result_tile(f, next_tile, result_tiles); + } else { + // Remove the cell in the bitmap and move to the next tile. + next_tile->clear_cell(first); + it++; + } } else { - // Compare the last tile from current to the first from next. - auto last = it->last_cell_in_bitmap(); - auto first = next_tile->first_cell_in_bitmap(); - if (!it->same_coords(*next_tile, last, first)) { - // Not the same coords, move to the next tile. + // Remove the cell in the current tile. + if (next_tile->result_num() == 1) { + // Only one cell in the bitmap, delete current tile. + auto to_delete = it; it++; + tmp_read_state_.add_ignored_tile(*to_delete); + remove_result_tile(f, to_delete, result_tiles); } else { - // Same coords, compare timestamps. - if (it->timestamp(last) > next_tile->timestamp(first)) { - // Remove the cell in the next tile. - if (next_tile->result_num() == 1) { - // Only one cell in the bitmap, delete next tile. - // Stay on this tile as we will compare to the new next. - tmp_read_state_.add_ignored_tile(*next_tile); - remove_result_tile(f, next_tile, result_tiles); - } else { - // Remove the cell in the bitmap and move to the next tile. - next_tile->clear_cell(first); - it++; - } - } else { - // Remove the cell in the current tile. - if (next_tile->result_num() == 1) { - // Only one cell in the bitmap, delete current tile. - auto to_delete = it; - it++; - tmp_read_state_.add_ignored_tile(*to_delete); - remove_result_tile(f, to_delete, result_tiles); - } else { - // Remove the cell in the bitmap and move to the next tile. - it->clear_cell(last); - it++; - } - } + // Remove the cell in the bitmap and move to the next tile. + it->clear_cell(last); + it++; } } } } - - return Status::Ok(); - })); + } + } + }); } template @@ -842,7 +827,7 @@ void SparseGlobalOrderReader::compute_hilbert_values( auto max_bucket_val = ((uint64_t)1 << bits) - 1; // Parallelize on tiles. - throw_if_not_ok(parallel_for( + parallel_for( &resources_.compute_tp(), 0, result_tiles.size(), [&](uint64_t t) { auto tile = static_cast*>(result_tiles[t]); @@ -866,9 +851,7 @@ void SparseGlobalOrderReader::compute_hilbert_values( tile->set_hilbert_value(rc.pos_, h.coords_to_hilbert(&coords[0])); } } - - return Status::Ok(); - })); + }); } template @@ -922,7 +905,7 @@ SparseGlobalOrderReader::merge_result_cell_slabs( // For all fragments, get the first tile in the sorting queue. std::vector to_delete; - throw_if_not_ok(parallel_for( + parallel_for( &resources_.compute_tp(), 0, result_tiles.size(), [&](uint64_t f) { if (result_tiles[f].size() > 0) { // Initialize the iterator for this fragment. @@ -941,9 +924,7 @@ SparseGlobalOrderReader::merge_result_cell_slabs( need_more_tiles |= res; } } - - return Status::Ok(); - })); + }); const bool non_overlapping_ranges = std::is_same::value; @@ -1169,7 +1150,7 @@ void SparseGlobalOrderReader::copy_offsets_tiles( auto timer_se = stats_->start_timer("copy_offsets_tiles"); // Process all tiles/cells in parallel. - throw_if_not_ok(parallel_for_2d( + parallel_for_2d( &resources_.compute_tp(), 0, result_cell_slabs.size(), @@ -1190,7 +1171,7 @@ void SparseGlobalOrderReader::copy_offsets_tiles( rcs.length_, cell_offsets[i]); if (skip_copy) { - return Status::Ok(); + return; } // Get source buffers. @@ -1254,9 +1235,7 @@ void SparseGlobalOrderReader::copy_offsets_tiles( } } } - - return Status::Ok(); - })); + }); } template @@ -1275,7 +1254,7 @@ void SparseGlobalOrderReader::copy_var_data_tiles( auto var_data_buffer = static_cast(query_buffer.buffer_var_); // Process all tiles/cells in parallel. - throw_if_not_ok(parallel_for_2d( + parallel_for_2d( &resources_.compute_tp(), 0, result_cell_slabs.size(), @@ -1296,7 +1275,7 @@ void SparseGlobalOrderReader::copy_var_data_tiles( cell_offsets[i]); (void)dest_cell_offset; if (skip_copy) { - return Status::Ok(); + return; } if (max_pos != min_pos) { @@ -1327,9 +1306,7 @@ void SparseGlobalOrderReader::copy_var_data_tiles( size); } } - - return Status::Ok(); - })); + }); } template @@ -1346,7 +1323,7 @@ void SparseGlobalOrderReader::copy_fixed_data_tiles( auto timer_se = stats_->start_timer("copy_fixed_data_tiles"); // Process all tiles/cells in parallel. - throw_if_not_ok(parallel_for_2d( + parallel_for_2d( &resources_.compute_tp(), 0, result_cell_slabs.size(), @@ -1367,7 +1344,7 @@ void SparseGlobalOrderReader::copy_fixed_data_tiles( rcs.length_, cell_offsets[i]); if (skip_copy) { - return Status::Ok(); + return; } // Get source buffers. @@ -1432,9 +1409,7 @@ void SparseGlobalOrderReader::copy_fixed_data_tiles( } } } - - return Status::Ok(); - })); + }); } template @@ -1446,7 +1421,7 @@ void SparseGlobalOrderReader::copy_timestamps_tiles( auto timer_se = stats_->start_timer("copy_timestamps_tiles"); // Process all tiles/cells in parallel. - throw_if_not_ok(parallel_for_2d( + parallel_for_2d( &resources_.compute_tp(), 0, result_cell_slabs.size(), @@ -1468,7 +1443,7 @@ void SparseGlobalOrderReader::copy_timestamps_tiles( rcs.length_, cell_offsets[i]); if (skip_copy) { - return Status::Ok(); + return; } // Get dest buffer. @@ -1492,9 +1467,7 @@ void SparseGlobalOrderReader::copy_timestamps_tiles( buffer++; } } - - return Status::Ok(); - })); + }); } template @@ -1513,7 +1486,7 @@ void SparseGlobalOrderReader::copy_delete_meta_tiles( } // Process all tiles/cells in parallel. - throw_if_not_ok(parallel_for_2d( + parallel_for_2d( &resources_.compute_tp(), 0, result_cell_slabs.size(), @@ -1534,7 +1507,7 @@ void SparseGlobalOrderReader::copy_delete_meta_tiles( rcs.length_, cell_offsets[i]); if (skip_copy) { - return Status::Ok(); + return; } // Get dest buffers. @@ -1607,9 +1580,7 @@ void SparseGlobalOrderReader::copy_delete_meta_tiles( buffer_condition_indexes++; } } - - return Status::Ok(); - })); + }); } template @@ -1623,90 +1594,86 @@ SparseGlobalOrderReader::respect_copy_memory_budget( uint64_t max_cs_idx = result_cell_slabs.size(); std::mutex max_cs_idx_mtx; std::vector total_mem_usage_per_attr(names.size()); - throw_if_not_ok( - parallel_for(&resources_.compute_tp(), 0, names.size(), [&](uint64_t i) { - // For easy reference. - const auto& name = names[i]; - const bool agg_only = aggregate_only(name); - const auto var_sized = array_schema_.var_size(name); - uint64_t* mem_usage = &total_mem_usage_per_attr[i]; - const bool is_timestamps = name == constants::timestamps || - name == constants::delete_timestamps; - - // Keep track of tiles already accounted for. - std:: - unordered_set, utils::hash::pair_hash> - accounted_tiles; - - // For dimensions or query condition fields, tiles are already all - // loaded in memory. - if (array_schema_.is_dim(name) || - qc_loaded_attr_names_set_.count(name) != 0 || is_timestamps || - name == constants::count_of_rows) { - return Status::Ok(); - } - - // Get the size for all tiles. - uint64_t idx = 0; - for (; idx < max_cs_idx; idx++) { - auto rcs = result_cell_slabs[idx]; - if (rcs.length_ == 0) { - continue; - } - - // Skip this tile if it's aggregate only and we can aggregate it with - // the fragment metadata only. - if (agg_only && can_aggregate_tile_with_frag_md(rcs)) { - continue; - } + parallel_for(&resources_.compute_tp(), 0, names.size(), [&](uint64_t i) { + // For easy reference. + const auto& name = names[i]; + const bool agg_only = aggregate_only(name); + const auto var_sized = array_schema_.var_size(name); + uint64_t* mem_usage = &total_mem_usage_per_attr[i]; + const bool is_timestamps = + name == constants::timestamps || name == constants::delete_timestamps; + + // Keep track of tiles already accounted for. + std::unordered_set, utils::hash::pair_hash> + accounted_tiles; + + // For dimensions or query condition fields, tiles are already all + // loaded in memory. + if (array_schema_.is_dim(name) || + qc_loaded_attr_names_set_.count(name) != 0 || is_timestamps || + name == constants::count_of_rows) { + return; + } - auto rt = static_cast*>(rcs.tile_); - const auto f = rt->frag_idx(); - const auto t = rt->tile_idx(); - auto id = std::pair(f, t); + // Get the size for all tiles. + uint64_t idx = 0; + for (; idx < max_cs_idx; idx++) { + auto rcs = result_cell_slabs[idx]; + if (rcs.length_ == 0) { + continue; + } - if (accounted_tiles.count(id) == 0) { - accounted_tiles.emplace(id); + // Skip this tile if it's aggregate only and we can aggregate it with + // the fragment metadata only. + if (agg_only && can_aggregate_tile_with_frag_md(rcs)) { + continue; + } - // Skip for delete condition name if the fragment doesn't have - // delete metadata. - if (name == constants::delete_condition_index && - !fragment_metadata_[f]->has_delete_meta()) { - continue; - } + auto rt = static_cast*>(rcs.tile_); + const auto f = rt->frag_idx(); + const auto t = rt->tile_idx(); + auto id = std::pair(f, t); - // Skip for fields added in schema evolution. - if (!fragment_metadata_[f]->array_schema()->is_field(name)) { - continue; - } + if (accounted_tiles.count(id) == 0) { + accounted_tiles.emplace(id); - // Size of the tile in memory. - auto tile_size = get_attribute_tile_size(name, f, t); + // Skip for delete condition name if the fragment doesn't have + // delete metadata. + if (name == constants::delete_condition_index && + !fragment_metadata_[f]->has_delete_meta()) { + continue; + } - // Account for the pointers to the var data that is created in - // copy_tiles for var sized attributes. - if (var_sized) { - tile_size += sizeof(void*) * rt->result_num(); - } + // Skip for fields added in schema evolution. + if (!fragment_metadata_[f]->array_schema()->is_field(name)) { + continue; + } - // Stop when we reach the budget. - if (*mem_usage + tile_size > memory_budget) { - break; - } + // Size of the tile in memory. + auto tile_size = get_attribute_tile_size(name, f, t); - // Adjust memory usage. - *mem_usage += tile_size; - } + // Account for the pointers to the var data that is created in + // copy_tiles for var sized attributes. + if (var_sized) { + tile_size += sizeof(void*) * rt->result_num(); } - // Save the minimum result tile index that we saw for all attributes. - { - std::unique_lock ul(max_cs_idx_mtx); - max_cs_idx = std::min(idx, max_cs_idx); + // Stop when we reach the budget. + if (*mem_usage + tile_size > memory_budget) { + break; } - return Status::Ok(); - })); + // Adjust memory usage. + *mem_usage += tile_size; + } + } + + // Save the minimum result tile index that we saw for all attributes. + { + std::unique_lock ul(max_cs_idx_mtx); + max_cs_idx = std::min(idx, max_cs_idx); + } + }); if (max_cs_idx == 0) { throw SparseGlobalOrderReaderException( @@ -2118,7 +2085,7 @@ void SparseGlobalOrderReader::process_aggregates( } // Process all tiles/cells in parallel. - throw_if_not_ok(parallel_for_2d( + parallel_for_2d( &resources_.compute_tp(), 0, result_cell_slabs.size(), @@ -2146,7 +2113,7 @@ void SparseGlobalOrderReader::process_aggregates( rcs.length_, cell_offsets[i]); if (skip_aggregate) { - return Status::Ok(); + return; } // Compute aggregate. @@ -2162,9 +2129,7 @@ void SparseGlobalOrderReader::process_aggregates( aggregate->aggregate_data(aggregate_buffer); } } - - return Status::Ok(); - })); + }); } template @@ -2194,16 +2159,13 @@ void SparseGlobalOrderReader::end_iteration( auto fragment_num = fragment_metadata_.size(); // Clear fully processed tiles in each fragments. - throw_if_not_ok( - parallel_for(&resources_.compute_tp(), 0, fragment_num, [&](uint64_t f) { - while (!result_tiles[f].empty() && - result_tiles[f].front().tile_idx() < - read_state_.frag_idx()[f].tile_idx_) { - remove_result_tile(f, result_tiles[f].begin(), result_tiles); - } - - return Status::Ok(); - })); + parallel_for(&resources_.compute_tp(), 0, fragment_num, [&](uint64_t f) { + while (!result_tiles[f].empty() && + result_tiles[f].front().tile_idx() < + read_state_.frag_idx()[f].tile_idx_) { + remove_result_tile(f, result_tiles[f].begin(), result_tiles); + } + }); if (!incomplete()) { assert(memory_used_for_coords_total_ == 0); diff --git a/tiledb/sm/query/readers/sparse_index_reader_base.cc b/tiledb/sm/query/readers/sparse_index_reader_base.cc index 1956529a140..d5113d5ef3c 100644 --- a/tiledb/sm/query/readers/sparse_index_reader_base.cc +++ b/tiledb/sm/query/readers/sparse_index_reader_base.cc @@ -473,16 +473,15 @@ void SparseIndexReaderBase::compute_tile_bitmaps( // are going to run multiple range threads. if (num_range_threads != 1) { // Resize bitmaps to process for each tiles in parallel. - throw_if_not_ok(parallel_for( + parallel_for( &resources_.compute_tp(), 0, result_tiles.size(), [&](uint64_t t) { static_cast*>(result_tiles[t]) ->alloc_bitmap(); - return Status::Ok(); - })); + }); } // Process all tiles/cells in parallel. - throw_if_not_ok(parallel_for_2d( + parallel_for_2d( &resources_.compute_tp(), 0, result_tiles.size(), @@ -503,7 +502,7 @@ void SparseIndexReaderBase::compute_tile_bitmaps( // Prevent processing past the end of the cells in case there are more // threads than cells. if (range_thread_idx > cell_num - 1) { - return Status::Ok(); + return; } // Get the MBR for this tile. @@ -572,20 +571,17 @@ void SparseIndexReaderBase::compute_tile_bitmaps( if (num_range_threads == 1) { rt->count_cells(); } - - return Status::Ok(); - })); + }); // For multiple range threads, bitmap cell count is done in a separate // parallel for. if (num_range_threads != 1) { // Compute number of cells in each bitmaps in parallel. - throw_if_not_ok(parallel_for( + parallel_for( &resources_.compute_tp(), 0, result_tiles.size(), [&](uint64_t t) { static_cast*>(result_tiles[t]) ->count_cells(); - return Status::Ok(); - })); + }); } logger_->debug("Done computing tile bitmaps"); @@ -599,7 +595,7 @@ void SparseIndexReaderBase::apply_query_condition( if (condition_.has_value() || !delete_and_update_conditions_.empty() || use_timestamps_) { // Process all tiles in parallel. - throw_if_not_ok(parallel_for( + parallel_for( &resources_.compute_tp(), 0, result_tiles.size(), [&](uint64_t t) { // For easy reference. auto rt = static_cast(result_tiles[t]); @@ -709,9 +705,7 @@ void SparseIndexReaderBase::apply_query_condition( } } } - - return Status::Ok(); - })); + }); } logger_->debug("Done applying query condition"); diff --git a/tiledb/sm/query/readers/sparse_unordered_with_dups_reader.cc b/tiledb/sm/query/readers/sparse_unordered_with_dups_reader.cc index e077fb6849d..9ef54a16ddf 100644 --- a/tiledb/sm/query/readers/sparse_unordered_with_dups_reader.cc +++ b/tiledb/sm/query/readers/sparse_unordered_with_dups_reader.cc @@ -758,7 +758,7 @@ void SparseUnorderedWithDupsReader::copy_offsets_tiles( auto val_buffer = query_buffer.validity_vector_.buffer(); // Process all tiles/cells in parallel. - throw_if_not_ok(parallel_for_2d( + parallel_for_2d( &resources_.compute_tp(), 0, result_tiles.size(), @@ -793,7 +793,7 @@ void SparseUnorderedWithDupsReader::copy_offsets_tiles( cell_offsets[i], rt); if (skip_copy) { - return Status::Ok(); + return; } // Copy tile. @@ -807,9 +807,7 @@ void SparseUnorderedWithDupsReader::copy_offsets_tiles( buffer + dest_cell_offset, val_buffer + dest_cell_offset, &var_data[dest_cell_offset - cell_offsets[0]]); - - return Status::Ok(); - })); + }); } /** Copy Var data. */ @@ -864,7 +862,7 @@ void SparseUnorderedWithDupsReader::copy_var_data_tiles( auto var_data_buffer = (uint8_t*)query_buffer.buffer_var_; // Process all tiles/cells in parallel. - throw_if_not_ok(parallel_for_2d( + parallel_for_2d( &resources_.compute_tp(), 0, result_tiles.size(), @@ -884,7 +882,7 @@ void SparseUnorderedWithDupsReader::copy_var_data_tiles( cell_offsets[i], nullptr); if (skip_copy) { - return Status::Ok(); + return; } copy_var_data_tile( @@ -897,9 +895,7 @@ void SparseUnorderedWithDupsReader::copy_var_data_tiles( (const void**)var_data.data(), offsets_buffer + dest_cell_offset, var_data_buffer); - - return Status::Ok(); - })); + }); } /** Copy fixed data with a result count bitmap. */ @@ -1236,7 +1232,7 @@ void SparseUnorderedWithDupsReader::copy_fixed_data_tiles( auto val_buffer = query_buffer.validity_vector_.buffer(); // Process all tiles/cells in parallel. - throw_if_not_ok(parallel_for_2d( + parallel_for_2d( &resources_.compute_tp(), 0, result_tiles.size(), @@ -1271,7 +1267,7 @@ void SparseUnorderedWithDupsReader::copy_fixed_data_tiles( cell_offsets[i], rt); if (skip_copy) { - return Status::Ok(); + return; } // Copy tile. @@ -1294,9 +1290,7 @@ void SparseUnorderedWithDupsReader::copy_fixed_data_tiles( buffer + dest_cell_offset * cell_size, val_buffer + dest_cell_offset); } - - return Status::Ok(); - })); + }); } template @@ -1440,73 +1434,69 @@ SparseUnorderedWithDupsReader::respect_copy_memory_budget( uint64_t max_rt_idx = result_tiles.size(); std::mutex max_rt_idx_mtx; std::vector total_mem_usage_per_attr(names.size()); - throw_if_not_ok( - parallel_for(&resources_.compute_tp(), 0, names.size(), [&](uint64_t i) { - // For easy reference. - const auto& name = names[i]; - const bool agg_only = aggregate_only(name); - const auto var_sized = array_schema_.var_size(name); - auto mem_usage = &total_mem_usage_per_attr[i]; - const bool is_timestamps = name == constants::timestamps || - name == constants::delete_timestamps; - - // For dimensions, when we have a subarray, tiles are already all - // loaded in memory. - if ((include_coords_ && array_schema_.is_dim(name)) || - qc_loaded_attr_names_set_.count(name) != 0 || is_timestamps || - name == constants::count_of_rows) { - return Status::Ok(); - } + parallel_for(&resources_.compute_tp(), 0, names.size(), [&](uint64_t i) { + // For easy reference. + const auto& name = names[i]; + const bool agg_only = aggregate_only(name); + const auto var_sized = array_schema_.var_size(name); + auto mem_usage = &total_mem_usage_per_attr[i]; + const bool is_timestamps = + name == constants::timestamps || name == constants::delete_timestamps; + + // For dimensions, when we have a subarray, tiles are already all + // loaded in memory. + if ((include_coords_ && array_schema_.is_dim(name)) || + qc_loaded_attr_names_set_.count(name) != 0 || is_timestamps || + name == constants::count_of_rows) { + return; + } - // Get the size for all tiles. - uint64_t idx = 0; - for (; idx < max_rt_idx; idx++) { - // Size of the tile in memory. - auto rt = static_cast*>( - result_tiles[idx]); - - // Skip this tile if it's aggregate only and we can aggregate it with - // the fragment metadata only. - if (agg_only && can_aggregate_tile_with_frag_md(rt)) { - continue; - } + // Get the size for all tiles. + uint64_t idx = 0; + for (; idx < max_rt_idx; idx++) { + // Size of the tile in memory. + auto rt = static_cast*>( + result_tiles[idx]); - // Skip for fields added in schema evolution. - if (!fragment_metadata_[rt->frag_idx()]->array_schema()->is_field( - name)) { - continue; - } + // Skip this tile if it's aggregate only and we can aggregate it with + // the fragment metadata only. + if (agg_only && can_aggregate_tile_with_frag_md(rt)) { + continue; + } - auto tile_size = - get_attribute_tile_size(name, rt->frag_idx(), rt->tile_idx()); + // Skip for fields added in schema evolution. + if (!fragment_metadata_[rt->frag_idx()]->array_schema()->is_field(name)) { + continue; + } - // Account for the pointers to the var data that is created in - // copy_tiles for var sized attributes. - if (var_sized) { - tile_size += sizeof(void*) * rt->result_num(); - } + auto tile_size = + get_attribute_tile_size(name, rt->frag_idx(), rt->tile_idx()); - // Stop when we reach the upper limit. - if (*mem_usage + tile_size > upper_memory_limit) { - // We can allow the first tile to go above the upper limit if it - // fits the available memory. - if (idx != 0 || tile_size > available_memory()) { - break; - } - } + // Account for the pointers to the var data that is created in + // copy_tiles for var sized attributes. + if (var_sized) { + tile_size += sizeof(void*) * rt->result_num(); + } - // Adjust memory usage. - *mem_usage += tile_size; + // Stop when we reach the upper limit. + if (*mem_usage + tile_size > upper_memory_limit) { + // We can allow the first tile to go above the upper limit if it + // fits the available memory. + if (idx != 0 || tile_size > available_memory()) { + break; } + } - // Save the minimum result tile index that we saw for all attributes. - { - std::unique_lock ul(max_rt_idx_mtx); - max_rt_idx = std::min(idx, max_rt_idx); - } + // Adjust memory usage. + *mem_usage += tile_size; + } - return Status::Ok(); - })); + // Save the minimum result tile index that we saw for all attributes. + { + std::unique_lock ul(max_rt_idx_mtx); + max_rt_idx = std::min(idx, max_rt_idx); + } + }); if (max_rt_idx == 0) { throw SparseUnorderedWithDupsReaderException( @@ -1919,7 +1909,7 @@ void SparseUnorderedWithDupsReader::process_aggregates( const bool count_bitmap = std::is_same::value; // Process all tiles/cells in parallel. - throw_if_not_ok(parallel_for_2d( + parallel_for_2d( &resources_.compute_tp(), 0, result_tiles.size(), @@ -1932,7 +1922,7 @@ void SparseUnorderedWithDupsReader::process_aggregates( // The first tile might have already been processed by the last // computation. We only process a tile the first time. if (i == 0 && read_state_.frag_idx()[rt->frag_idx()].cell_idx_ != 0) { - return Status::Ok(); + return; } if (can_aggregate_tile_with_frag_md(rt)) { @@ -1956,7 +1946,7 @@ void SparseUnorderedWithDupsReader::process_aggregates( cell_offsets[i], nullptr); if (skip_aggregate) { - return Status::Ok(); + return; } // Compute aggregate. @@ -1973,9 +1963,7 @@ void SparseUnorderedWithDupsReader::process_aggregates( aggregate->aggregate_data(aggregate_buffer); } } - - return Status::Ok(); - })); + }); } template diff --git a/tiledb/sm/query/writers/global_order_writer.cc b/tiledb/sm/query/writers/global_order_writer.cc index fccf6754494..b94bd2c7514 100644 --- a/tiledb/sm/query/writers/global_order_writer.cc +++ b/tiledb/sm/query/writers/global_order_writer.cc @@ -322,7 +322,7 @@ Status GlobalOrderWriter::check_coord_dups() const { buffs_var_sizes[d] = buffers_.find(dim_name)->second.buffer_var_size_; } - auto status = parallel_for( + parallel_for( &resources_.compute_tp(), 1, coords_info_.coords_num_, [&](uint64_t i) { // Check for duplicate in adjacent cells bool found_dup = true; @@ -366,14 +366,10 @@ Status GlobalOrderWriter::check_coord_dups() const { std::stringstream ss; ss << "Duplicate coordinates " << coords_to_str(i); ss << " are not allowed"; - return Status_WriterError(ss.str()); + throw GlobalOrderWriterException(ss.str()); } - - return Status::Ok(); }); - RETURN_NOT_OK(status); - return Status::Ok(); } @@ -419,7 +415,7 @@ Status GlobalOrderWriter::check_global_order() const { } // Check if all coordinates are in global order in parallel - auto status = parallel_for( + parallel_for( &resources_.compute_tp(), 0, coords_info_.coords_num_ - 1, @@ -436,13 +432,10 @@ Status GlobalOrderWriter::check_global_order() const { ss << " in the global order"; if (tile_cmp > 0) ss << " due to writes across tiles"; - return Status_WriterError(ss.str()); + throw GlobalOrderWriterException(ss.str()); } - return Status::Ok(); }); - RETURN_NOT_OK(status); - // Save the last cell's coordinates. auto last_cell_coords{ domain_buffs.domain_ref_at(domain, coords_info_.coords_num_ - 1)}; @@ -470,7 +463,7 @@ Status GlobalOrderWriter::check_global_order_hilbert() const { } // Check if all coordinates are in hilbert order in parallel - auto status = parallel_for( + parallel_for( &resources_.compute_tp(), 0, coords_info_.coords_num_ - 1, @@ -480,13 +473,10 @@ Status GlobalOrderWriter::check_global_order_hilbert() const { ss << "Write failed; Coordinates " << coords_to_str(i); ss << " succeed " << coords_to_str(i + 1); ss << " in the hilbert order"; - return Status_WriterError(ss.str()); + throw GlobalOrderWriterException(ss.str()); } - return Status::Ok(); }); - RETURN_NOT_OK(status); - // Save the last hilbert value global_write_state_->last_hilbert_value_ = hilbert_values[coords_info_.coords_num_ - 1]; @@ -568,7 +558,7 @@ Status GlobalOrderWriter::compute_coord_dups( } std::mutex mtx; - auto status = parallel_for( + parallel_for( &resources_.compute_tp(), 1, coords_info_.coords_num_, [&](uint64_t i) { // Check for duplicate in adjacent cells bool found_dup = true; @@ -612,12 +602,8 @@ Status GlobalOrderWriter::compute_coord_dups( std::lock_guard lock(mtx); coord_dups->insert(i); } - - return Status::Ok(); }); - RETURN_NOT_OK(status); - return Status::Ok(); } @@ -865,17 +851,14 @@ Status GlobalOrderWriter::prepare_full_tiles( } auto num = buffers_.size(); - auto status = parallel_for(&resources_.compute_tp(), 0, num, [&](uint64_t i) { + parallel_for(&resources_.compute_tp(), 0, num, [&](uint64_t i) { auto buff_it = buffers_.begin(); std::advance(buff_it, i); const auto& name = buff_it->first; throw_if_not_ok(prepare_full_tiles(name, coord_dups, &tiles->at(name))); throw_if_cancelled(); - return Status::Ok(); }); - RETURN_NOT_OK(status); - return Status::Ok(); } diff --git a/tiledb/sm/query/writers/ordered_writer.cc b/tiledb/sm/query/writers/ordered_writer.cc index ed20b1f35d4..18973be073e 100644 --- a/tiledb/sm/query/writers/ordered_writer.cc +++ b/tiledb/sm/query/writers/ordered_writer.cc @@ -205,26 +205,26 @@ Status OrderedWriter::ordered_write() { } if (attr_num > tile_num) { // Parallelize over attributes - RETURN_NOT_OK(parallel_for(compute_tp, 0, attr_num, [&](uint64_t i) { + parallel_for(compute_tp, 0, attr_num, [&](uint64_t i) { auto buff_it = buffers_.begin(); std::advance(buff_it, i); const auto& attr = buff_it->first; auto& attr_tile_batches = tiles.at(attr); - return prepare_filter_and_write_tiles( - attr, attr_tile_batches, frag_meta, &dense_tiler, 1); - })); + throw_if_not_ok(prepare_filter_and_write_tiles( + attr, attr_tile_batches, frag_meta, &dense_tiler, 1)); + }); } else { // Parallelize over tiles for (const auto& buff : buffers_) { const auto& attr = buff.first; auto& attr_tile_batches = tiles.at(attr); - RETURN_NOT_OK(prepare_filter_and_write_tiles( + throw_if_not_ok(prepare_filter_and_write_tiles( attr, attr_tile_batches, frag_meta, &dense_tiler, thread_num)); } } // Fix the tile metadata for var size attributes. if (attr_num > tile_num) { // Parallelize over attributes - RETURN_NOT_OK(parallel_for(compute_tp, 0, attr_num, [&](uint64_t i) { + parallel_for(compute_tp, 0, attr_num, [&](uint64_t i) { auto buff_it = buffers_.begin(); std::advance(buff_it, i); const auto& attr = buff_it->first; @@ -242,8 +242,7 @@ Status OrderedWriter::ordered_write() { } } } - return Status::Ok(); - })); + }); } else { // Parallelize over tiles for (const auto& buff : buffers_) { const auto& attr = buff.first; @@ -252,18 +251,16 @@ Status OrderedWriter::ordered_write() { if (has_min_max_metadata(attr, var_size) && array_schema_.var_size(attr)) { frag_meta->convert_tile_min_max_var_sizes_to_offsets(attr); - RETURN_NOT_OK(parallel_for( - compute_tp, 0, attr_tile_batches.size(), [&](uint64_t b) { - const auto& attr = buff.first; - auto& batch = tiles.at(attr)[b]; - auto idx = b * thread_num; - for (auto& tile : batch) { - frag_meta->set_tile_min_var(attr, idx, tile.min()); - frag_meta->set_tile_max_var(attr, idx, tile.max()); - idx++; - } - return Status::Ok(); - })); + parallel_for(compute_tp, 0, attr_tile_batches.size(), [&](uint64_t b) { + const auto& attr = buff.first; + auto& batch = tiles.at(attr)[b]; + auto idx = b * thread_num; + for (auto& tile : batch) { + frag_meta->set_tile_min_var(attr, idx, tile.min()); + frag_meta->set_tile_max_var(attr, idx, tile.max()); + idx++; + } + }); } } } @@ -331,40 +328,36 @@ Status OrderedWriter::prepare_filter_and_write_tiles( { auto timer_se = stats_->start_timer("prepare_and_filter_tiles"); - auto st = parallel_for( - &resources_.compute_tp(), 0, batch_size, [&](uint64_t i) { - // Prepare and filter tiles - auto& writer_tile = tile_batches[b][i]; - throw_if_not_ok( - dense_tiler->get_tile(frag_tile_id + i, name, writer_tile)); - - if (!var) { - throw_if_not_ok(filter_tile( - name, &writer_tile.fixed_tile(), nullptr, false, false)); - } else { - auto offset_tile = &writer_tile.offset_tile(); - throw_if_not_ok(filter_tile( - name, &writer_tile.var_tile(), offset_tile, false, false)); - throw_if_not_ok( - filter_tile(name, offset_tile, nullptr, true, false)); - } - if (nullable) { - throw_if_not_ok(filter_tile( - name, &writer_tile.validity_tile(), nullptr, false, true)); - } - return Status::Ok(); - }); - RETURN_NOT_OK(st); + parallel_for(&resources_.compute_tp(), 0, batch_size, [&](uint64_t i) { + // Prepare and filter tiles + auto& writer_tile = tile_batches[b][i]; + throw_if_not_ok( + dense_tiler->get_tile(frag_tile_id + i, name, writer_tile)); + + if (!var) { + throw_if_not_ok(filter_tile( + name, &writer_tile.fixed_tile(), nullptr, false, false)); + } else { + auto offset_tile = &writer_tile.offset_tile(); + throw_if_not_ok(filter_tile( + name, &writer_tile.var_tile(), offset_tile, false, false)); + throw_if_not_ok(filter_tile(name, offset_tile, nullptr, true, false)); + } + if (nullable) { + throw_if_not_ok(filter_tile( + name, &writer_tile.validity_tile(), nullptr, false, true)); + } + }); } if (write_task.has_value()) { write_task->wait(); - RETURN_NOT_OK(write_task->get()); + write_task->get(); } write_task = resources_.io_tp().execute([&, b, frag_tile_id]() { close_files = (b == batch_num - 1); - RETURN_NOT_OK(write_tiles( + throw_if_not_ok(write_tiles( 0, tile_batches[b].size(), name, @@ -372,8 +365,6 @@ Status OrderedWriter::prepare_filter_and_write_tiles( frag_tile_id, &tile_batches[b], close_files)); - - return Status::Ok(); }); frag_tile_id += batch_size; @@ -381,7 +372,7 @@ Status OrderedWriter::prepare_filter_and_write_tiles( if (write_task.has_value()) { write_task->wait(); - RETURN_NOT_OK(write_task->get()); + write_task->get(); } return Status::Ok(); diff --git a/tiledb/sm/query/writers/unordered_writer.cc b/tiledb/sm/query/writers/unordered_writer.cc index e30273c6eab..5f752d9486c 100644 --- a/tiledb/sm/query/writers/unordered_writer.cc +++ b/tiledb/sm/query/writers/unordered_writer.cc @@ -214,7 +214,7 @@ Status UnorderedWriter::check_coord_dups() const { buffs_var_sizes[d] = buffers_.find(dim_name)->second.buffer_var_size_; } - auto status = parallel_for( + parallel_for( &resources_.compute_tp(), 1, coords_info_.coords_num_, [&](uint64_t i) { // Check for duplicate in adjacent cells bool found_dup = true; @@ -265,14 +265,10 @@ Status UnorderedWriter::check_coord_dups() const { std::stringstream ss; ss << "Duplicate coordinates " << coords_to_str(cell_pos_[i]); ss << " are not allowed"; - return Status_WriterError(ss.str()); + throw UnorderWriterException(ss.str()); } - - return Status::Ok(); }); - RETURN_NOT_OK_ELSE(status, logger_->error(status.message())); - return Status::Ok(); } @@ -304,7 +300,7 @@ Status UnorderedWriter::compute_coord_dups() { } std::mutex mtx; - auto status = parallel_for( + parallel_for( &resources_.compute_tp(), 1, coords_info_.coords_num_, [&](uint64_t i) { // Check for duplicate in adjacent cells bool found_dup = true; @@ -355,12 +351,8 @@ Status UnorderedWriter::compute_coord_dups() { std::lock_guard lock(mtx); coord_dups_.insert(cell_pos_[i]); } - - return Status::Ok(); }); - RETURN_NOT_OK(status); - return Status::Ok(); } @@ -381,16 +373,12 @@ Status UnorderedWriter::prepare_tiles( } // Prepare tiles for all attributes and coordinates - auto status = - parallel_for(&resources_.compute_tp(), 0, tiles->size(), [&](uint64_t i) { - auto tiles_it = tiles->begin(); - std::advance(tiles_it, i); - throw_if_not_ok(prepare_tiles(tiles_it->first, &(tiles_it->second))); - throw_if_cancelled(); - return Status::Ok(); - }); - - RETURN_NOT_OK(status); + parallel_for(&resources_.compute_tp(), 0, tiles->size(), [&](uint64_t i) { + auto tiles_it = tiles->begin(); + std::advance(tiles_it, i); + throw_if_not_ok(prepare_tiles(tiles_it->first, &(tiles_it->second))); + throw_if_cancelled(); + }); return Status::Ok(); } diff --git a/tiledb/sm/query/writers/writer_base.cc b/tiledb/sm/query/writers/writer_base.cc index b6ff5ffddfd..870ed8296f8 100644 --- a/tiledb/sm/query/writers/writer_base.cc +++ b/tiledb/sm/query/writers/writer_base.cc @@ -325,7 +325,7 @@ Status WriterBase::calculate_hilbert_values( // Calculate Hilbert values in parallel assert(hilbert_values.size() >= coords_info_.coords_num_); - auto status = parallel_for( + parallel_for( &resources_.compute_tp(), 0, coords_info_.coords_num_, [&](uint64_t c) { std::vector coords(dim_num); for (uint32_t d = 0; d < dim_num; ++d) { @@ -334,12 +334,8 @@ Status WriterBase::calculate_hilbert_values( *dim, domain_buffers[d], c, bits, max_bucket_val); } hilbert_values[c] = h.coords_to_hilbert(&coords[0]); - - return Status::Ok(); }); - RETURN_NOT_OK_ELSE(status, logger_->error(status.message())); - return Status::Ok(); } @@ -416,7 +412,7 @@ Status WriterBase::check_coord_oob() const { } // Check if all coordinates fall in the domain in parallel - auto status = parallel_for_2d( + parallel_for_2d( &resources_.compute_tp(), 0, coords_info_.coords_num_, @@ -425,12 +421,10 @@ Status WriterBase::check_coord_oob() const { [&](uint64_t c, unsigned d) { auto dim{array_schema_.dimension_ptr(d)}; if (datatype_is_string(dim->type())) - return Status::Ok(); - return dim->oob(buffs[d] + c * coord_sizes[d]); + return; + throw_if_not_ok(dim->oob(buffs[d] + c * coord_sizes[d])); }); - RETURN_NOT_OK(status); - // Success return Status::Ok(); } @@ -598,18 +592,14 @@ Status WriterBase::close_files(shared_ptr meta) const { } } - auto status = - parallel_for(&resources_.io_tp(), 0, file_uris.size(), [&](uint64_t i) { - const auto& file_uri = file_uris[i]; - if (layout_ == Layout::GLOBAL_ORDER && remote_query()) { - resources_.vfs().finalize_and_close_file(file_uri); - } else { - throw_if_not_ok(resources_.vfs().close_file(file_uri)); - } - return Status::Ok(); - }); - - throw_if_not_ok(status); + parallel_for(&resources_.io_tp(), 0, file_uris.size(), [&](uint64_t i) { + const auto& file_uri = file_uris[i]; + if (layout_ == Layout::GLOBAL_ORDER && remote_query()) { + resources_.vfs().finalize_and_close_file(file_uri); + } else { + throw_if_not_ok(resources_.vfs().close_file(file_uri)); + } + }); return Status::Ok(); } @@ -636,25 +626,21 @@ std::vector WriterBase::compute_mbrs( // Compute MBRs std::vector mbrs(tile_num); - auto status = - parallel_for(&resources_.compute_tp(), 0, tile_num, [&](uint64_t i) { - mbrs[i].resize(dim_num); - std::vector data(dim_num); - for (unsigned d = 0; d < dim_num; ++d) { - auto dim{array_schema_.dimension_ptr(d)}; - const auto& dim_name = dim->name(); - auto tiles_it = tiles.find(dim_name); - assert(tiles_it != tiles.end()); - mbrs[i][d] = dim->var_size() ? - dim->compute_mbr_var( - tiles_it->second[i].offset_tile(), - tiles_it->second[i].var_tile()) : - dim->compute_mbr(tiles_it->second[i].fixed_tile()); - } - - return Status::Ok(); - }); - throw_if_not_ok(status); + parallel_for(&resources_.compute_tp(), 0, tile_num, [&](uint64_t i) { + mbrs[i].resize(dim_num); + std::vector data(dim_num); + for (unsigned d = 0; d < dim_num; ++d) { + auto dim{array_schema_.dimension_ptr(d)}; + const auto& dim_name = dim->name(); + auto tiles_it = tiles.find(dim_name); + assert(tiles_it != tiles.end()); + mbrs[i][d] = dim->var_size() ? + dim->compute_mbr_var( + tiles_it->second[i].offset_tile(), + tiles_it->second[i].var_tile()) : + dim->compute_mbr(tiles_it->second[i].fixed_tile()); + } + }); return mbrs; } @@ -675,12 +661,10 @@ void WriterBase::set_coords_metadata( return; } - auto status = parallel_for( + parallel_for( &resources_.compute_tp(), start_tile_idx, end_tile_idx, [&](uint64_t i) { meta->set_mbr(i - start_tile_idx, mbrs[i]); - return Status::Ok(); }); - throw_if_not_ok(status); // Set last tile cell number auto dim_0{array_schema_.dimension_ptr(0)}; @@ -696,7 +680,7 @@ Status WriterBase::compute_tiles_metadata( // Parallelize over attributes? if (tiles.size() > tile_num) { - auto st = parallel_for(compute_tp, 0, tiles.size(), [&](uint64_t i) { + parallel_for(compute_tp, 0, tiles.size(), [&](uint64_t i) { auto tiles_it = tiles.begin(); std::advance(tiles_it, i); const auto& attr = tiles_it->first; @@ -712,10 +696,7 @@ Status WriterBase::compute_tiles_metadata( md_generator.process_full_tile(tile); md_generator.set_tile_metadata(tile); } - - return Status::Ok(); }); - RETURN_NOT_OK(st); } else { // Parallelize over tiles for (auto& tile_vec : tiles) { const auto& attr = tile_vec.first; @@ -725,15 +706,12 @@ Status WriterBase::compute_tiles_metadata( const auto var_size = array_schema_.var_size(attr); const auto cell_size = array_schema_.cell_size(attr); const auto cell_val_num = array_schema_.cell_val_num(attr); - auto st = parallel_for(compute_tp, 0, tile_num, [&](uint64_t t) { + parallel_for(compute_tp, 0, tile_num, [&](uint64_t t) { TileMetadataGenerator md_generator( type, is_dim, var_size, cell_size, cell_val_num); md_generator.process_full_tile(attr_tiles[t]); md_generator.set_tile_metadata(attr_tiles[t]); - - return Status::Ok(); }); - RETURN_NOT_OK(st); } } @@ -797,16 +775,13 @@ Status WriterBase::create_fragment( Status WriterBase::filter_tiles( tdb::pmr::unordered_map* tiles) { auto timer_se = stats_->start_timer("filter_tiles"); - auto status = - parallel_for(&resources_.compute_tp(), 0, tiles->size(), [&](uint64_t i) { - auto tiles_it = tiles->begin(); - std::advance(tiles_it, i); - throw_if_not_ok(filter_tiles(tiles_it->first, &tiles_it->second)); - throw_if_cancelled(); - return Status::Ok(); - }); + parallel_for(&resources_.compute_tp(), 0, tiles->size(), [&](uint64_t i) { + auto tiles_it = tiles->begin(); + std::advance(tiles_it, i); + throw_if_not_ok(filter_tiles(tiles_it->first, &tiles_it->second)); + throw_if_cancelled(); + }); - RETURN_NOT_OK(status); return Status::Ok(); } @@ -834,26 +809,19 @@ Status WriterBase::filter_tiles( } // For fixed size, process everything, for var size, everything minus offsets. - auto status = - parallel_for(&resources_.compute_tp(), 0, args.size(), [&](uint64_t i) { - const auto& [tile, offset_tile, contains_offsets, is_nullable] = - args[i]; - throw_if_not_ok(filter_tile( - name, tile, offset_tile, contains_offsets, is_nullable)); - return Status::Ok(); - }); - RETURN_NOT_OK(status); + parallel_for(&resources_.compute_tp(), 0, args.size(), [&](uint64_t i) { + const auto& [tile, offset_tile, contains_offsets, is_nullable] = args[i]; + throw_if_not_ok( + filter_tile(name, tile, offset_tile, contains_offsets, is_nullable)); + }); // Process offsets for var size. if (var_size) { - auto status = parallel_for( - &resources_.compute_tp(), 0, tiles->size(), [&](uint64_t i) { - auto& tile = (*tiles)[i]; - throw_if_not_ok( - filter_tile(name, &tile.offset_tile(), nullptr, true, false)); - return Status::Ok(); - }); - RETURN_NOT_OK(status); + parallel_for(&resources_.compute_tp(), 0, tiles->size(), [&](uint64_t i) { + auto& tile = (*tiles)[i]; + throw_if_not_ok( + filter_tile(name, &tile.offset_tile(), nullptr, true, false)); + }); } return Status::Ok(); @@ -1044,8 +1012,9 @@ Status WriterBase::write_tiles( tasks.push_back(resources_.io_tp().execute([&, this]() { auto& attr = it.first; auto& tiles = it.second; - RETURN_CANCEL_OR_ERROR(write_tiles( + throw_if_not_ok(write_tiles( start_tile_idx, end_tile_idx, attr, frag_meta, 0, &tiles)); + throw_if_cancelled(); // Fix var size attributes metadata. const auto var_size = array_schema_.var_size(attr); @@ -1060,14 +1029,11 @@ Status WriterBase::write_tiles( attr, idx - start_tile_idx, tiles[idx].max()); } } - return Status::Ok(); })); } - // Wait for writes and check all statuses - auto statuses = resources_.io_tp().wait_all_status(tasks); - for (auto& st : statuses) - RETURN_NOT_OK(st); + // Wait for writes + resources_.io_tp().wait_all(tasks); return Status::Ok(); } diff --git a/tiledb/sm/subarray/relevant_fragment_generator.cc b/tiledb/sm/subarray/relevant_fragment_generator.cc index a4b56a20ee1..e188eb12952 100644 --- a/tiledb/sm/subarray/relevant_fragment_generator.cc +++ b/tiledb/sm/subarray/relevant_fragment_generator.cc @@ -101,7 +101,7 @@ RelevantFragments RelevantFragmentGenerator::compute_relevant_fragments( const auto meta = array_->fragment_metadata(); // Populate the fragment bytemap for each dimension in parallel. - throw_if_not_ok(parallel_for_2d( + parallel_for_2d( compute_tp, 0, dim_num, @@ -109,13 +109,13 @@ RelevantFragments RelevantFragmentGenerator::compute_relevant_fragments( fragment_num, [&](const uint32_t d, const uint64_t f) { if (subarray_.is_default(d)) { - return Status::Ok(); + return; } // We're done when we have already determined fragment `f` to // be relevant for this dimension. if (fragment_bytemaps_[d][f] == 1) { - return Status::Ok(); + return; } auto dim{array_->array_schema_latest().dimension_ptr(d)}; @@ -132,9 +132,7 @@ RelevantFragments RelevantFragmentGenerator::compute_relevant_fragments( break; } } - - return Status::Ok(); - })); + }); // Recalculate relevant fragments. return RelevantFragments(dim_num, fragment_num, fragment_bytemaps_); diff --git a/tiledb/sm/subarray/subarray.cc b/tiledb/sm/subarray/subarray.cc index 87fd27e5d2a..81705bc07f5 100644 --- a/tiledb/sm/subarray/subarray.cc +++ b/tiledb/sm/subarray/subarray.cc @@ -1513,7 +1513,7 @@ void Subarray::compute_relevant_fragment_est_result_sizes( auto all_dims_fixed = array_schema.domain().all_dims_fixed(); auto num_threads = compute_tp->concurrency_level(); auto ranges_per_thread = (uint64_t)std::ceil((double)range_num / num_threads); - auto status = parallel_for(compute_tp, 0, num_threads, [&](uint64_t t) { + parallel_for(compute_tp, 0, num_threads, [&](uint64_t t) { auto r_start = range_start + t * ranges_per_thread; auto r_end = std::min(range_start + (t + 1) * ranges_per_thread - 1, range_end); @@ -1554,9 +1554,7 @@ void Subarray::compute_relevant_fragment_est_result_sizes( // Global order - noop } } - return Status::Ok(); }); - throw_if_not_ok(status); // Compute the mem sizes vector mem_sizes->resize(range_num); @@ -1635,23 +1633,21 @@ void Subarray::sort_and_merge_ranges(ThreadPool* const compute_tp) { // Sort and conditionally merge ranges auto timer = stats_->start_timer("sort_and_merge_ranges"); - throw_if_not_ok(parallel_for( + parallel_for( compute_tp, 0, array_->array_schema_latest().dim_num(), [&](uint64_t dim_idx) { range_subset_[dim_idx].sort_and_merge_ranges( compute_tp, merge_overlapping_ranges_); - return Status::Ok(); - })); - ranges_sorted_ = true; + }); } bool Subarray::non_overlapping_ranges(ThreadPool* compute_tp) { sort_and_merge_ranges(compute_tp); std::atomic non_overlapping_ranges = true; - auto st = parallel_for( + parallel_for( compute_tp, 0, array_->array_schema_latest().dim_num(), @@ -1660,9 +1656,7 @@ bool Subarray::non_overlapping_ranges(ThreadPool* compute_tp) { if (!nor) { non_overlapping_ranges = false; } - return Status::Ok(); }); - throw_if_not_ok(st); return non_overlapping_ranges; } @@ -2240,85 +2234,77 @@ void Subarray::precompute_all_ranges_tile_overlap( all_threads_tile_bitmaps(static_cast(num_threads)); // Run all fragments in parallel. - auto status = - parallel_for(compute_tp, 0, relevant_fragments_.size(), [&](uint64_t i) { - const auto f = relevant_fragments_[i]; - auto tile_bitmaps_resource_guard = - ResourceGuard(all_threads_tile_bitmaps); - auto tile_bitmaps = tile_bitmaps_resource_guard.get(); - - // Make sure all bitmaps have the correct size. - if (tile_bitmaps.size() == 0) { - tile_bitmaps.resize(dim_num); - for (unsigned d = 0; d < dim_num; d++) - tile_bitmaps[d].resize(meta[f]->tile_num()); - } else { - uint64_t memset_length = - std::min((uint64_t)tile_bitmaps[0].size(), meta[f]->tile_num()); - for (unsigned d = 0; d < dim_num; d++) { - // TODO we might be able to skip the memset if - // tile_bitmaps.capacity() <= meta[f]->tile_num(). - memset(tile_bitmaps[d].data(), 0, memset_length * sizeof(uint8_t)); - tile_bitmaps[d].resize(meta[f]->tile_num()); - } - } - - for (unsigned d = 0; d < dim_num; d++) { - if (is_default_[d]) { - continue; - } - - // Run all ranges in parallel. - const uint64_t range_num = range_subset_[d].num_ranges(); - - // Compute tile bitmaps for this fragment. - const auto ranges_per_thread = - (uint64_t)std::ceil((double)range_num / num_threads); - const auto status_ranges = - parallel_for(compute_tp, 0, num_threads, [&](uint64_t t) { - const auto r_start = t * ranges_per_thread; - const auto r_end = - std::min((t + 1) * ranges_per_thread - 1, range_num - 1); - for (uint64_t r = r_start; r <= r_end; ++r) { - meta[f]->loaded_metadata()->compute_tile_bitmap( - range_subset_[d][r], d, &tile_bitmaps[d]); - } - return Status::Ok(); - }); - RETURN_NOT_OK(status_ranges); - } + parallel_for(compute_tp, 0, relevant_fragments_.size(), [&](uint64_t i) { + const auto f = relevant_fragments_[i]; + auto tile_bitmaps_resource_guard = ResourceGuard(all_threads_tile_bitmaps); + auto tile_bitmaps = tile_bitmaps_resource_guard.get(); + + // Make sure all bitmaps have the correct size. + if (tile_bitmaps.size() == 0) { + tile_bitmaps.resize(dim_num); + for (unsigned d = 0; d < dim_num; d++) + tile_bitmaps[d].resize(meta[f]->tile_num()); + } else { + uint64_t memset_length = + std::min((uint64_t)tile_bitmaps[0].size(), meta[f]->tile_num()); + for (unsigned d = 0; d < dim_num; d++) { + // TODO we might be able to skip the memset if + // tile_bitmaps.capacity() <= meta[f]->tile_num(). + memset(tile_bitmaps[d].data(), 0, memset_length * sizeof(uint8_t)); + tile_bitmaps[d].resize(meta[f]->tile_num()); + } + } - // Go through the bitmaps in reverse, whenever there is a "hole" in tile - // contiguity, push a new result tile range. - uint64_t end = tile_bitmaps[0].size() - 1; - uint64_t length = 0; - int64_t min = static_cast(frag_tile_idx[f].tile_idx_); - for (int64_t t = tile_bitmaps[0].size() - 1; t >= min; t--) { - bool comb = true; - for (unsigned d = 0; d < dim_num; d++) { - comb &= is_default_[d] || (bool)tile_bitmaps[d][t]; - } + for (unsigned d = 0; d < dim_num; d++) { + if (is_default_[d]) { + continue; + } - if (!comb) { - if (length != 0) { - tile_ranges->add_tile_range(f, end + 1 - length, end); - length = 0; - } + // Run all ranges in parallel. + const uint64_t range_num = range_subset_[d].num_ranges(); - end = t - 1; - } else { - length++; - } + // Compute tile bitmaps for this fragment. + const auto ranges_per_thread = + (uint64_t)std::ceil((double)range_num / num_threads); + parallel_for(compute_tp, 0, num_threads, [&](uint64_t t) { + const auto r_start = t * ranges_per_thread; + const auto r_end = + std::min((t + 1) * ranges_per_thread - 1, range_num - 1); + for (uint64_t r = r_start; r <= r_end; ++r) { + meta[f]->loaded_metadata()->compute_tile_bitmap( + range_subset_[d][r], d, &tile_bitmaps[d]); } + }); + } - // Push the last result tile range. + // Go through the bitmaps in reverse, whenever there is a "hole" in tile + // contiguity, push a new result tile range. + uint64_t end = tile_bitmaps[0].size() - 1; + uint64_t length = 0; + int64_t min = static_cast(frag_tile_idx[f].tile_idx_); + for (int64_t t = tile_bitmaps[0].size() - 1; t >= min; t--) { + bool comb = true; + for (unsigned d = 0; d < dim_num; d++) { + comb &= is_default_[d] || (bool)tile_bitmaps[d][t]; + } + + if (!comb) { if (length != 0) { tile_ranges->add_tile_range(f, end + 1 - length, end); + length = 0; } - return Status::Ok(); - }); - throw_if_not_ok(status); + end = t - 1; + } else { + length++; + } + } + + // Push the last result tile range. + if (length != 0) { + tile_ranges->add_tile_range(f, end + 1 - length, end); + } + }); tile_ranges->done_adding_tile_ranges(); } @@ -2534,13 +2520,10 @@ void Subarray::load_relevant_fragment_rtrees(ThreadPool* compute_tp) const { auto meta = array_->fragment_metadata(); auto encryption_key = array_->encryption_key(); - auto status = - parallel_for(compute_tp, 0, relevant_fragments_.size(), [&](uint64_t f) { - meta[relevant_fragments_[f]]->loaded_metadata()->load_rtree( - *encryption_key); - return Status::Ok(); - }); - throw_if_not_ok(status); + parallel_for(compute_tp, 0, relevant_fragments_.size(), [&](uint64_t f) { + meta[relevant_fragments_[f]]->loaded_metadata()->load_rtree( + *encryption_key); + }); } void Subarray::compute_relevant_fragment_tile_overlap( @@ -2559,7 +2542,7 @@ void Subarray::compute_relevant_fragment_tile_overlap( const auto ranges_per_thread = (uint64_t)std::ceil((double)fn_ctx->range_len_ / num_threads); - auto status = parallel_for_2d( + parallel_for_2d( compute_tp, 0, relevant_fragments_.size(), @@ -2585,10 +2568,7 @@ void Subarray::compute_relevant_fragment_tile_overlap( range, is_default_, tile_overlap->at(frag_idx, r)); } } - - return Status::Ok(); }); - throw_if_not_ok(status); } void Subarray::load_relevant_fragment_tile_var_sizes( @@ -2612,20 +2592,18 @@ void Subarray::load_relevant_fragment_tile_var_sizes( // Load all metadata for tile var sizes among fragments. for (const auto& var_name : var_names) { - const auto status = parallel_for( + parallel_for( compute_tp, 0, relevant_fragments_.size(), [&](const size_t i) { auto f = relevant_fragments_[i]; // Gracefully skip loading tile sizes for attributes added in schema // evolution that do not exists in this fragment const auto& schema = meta[f]->array_schema(); if (!schema->is_field(var_name)) { - return Status::Ok(); + return; } meta[f]->loaded_metadata()->load_tile_var_sizes( *encryption_key, var_name); - return Status::Ok(); }); - throw_if_not_ok(status); } } From 4b18f6c701fe8f0baf79116dd0da8bbed8dba8cb Mon Sep 17 00:00:00 2001 From: Theodore Tsirpanis Date: Thu, 13 Jun 2024 18:56:56 +0300 Subject: [PATCH 04/11] Remove status from some trivial functions in array and group directory. --- tiledb/sm/array/array_directory.cc | 77 +++++++------------ tiledb/sm/array/array_directory.h | 36 ++++----- tiledb/sm/group/group_directory.cc | 70 +++++------------ tiledb/sm/group/group_directory.h | 23 +++--- tiledb/sm/query/legacy/reader.cc | 5 +- .../query/readers/sparse_index_reader_base.cc | 3 +- 6 files changed, 77 insertions(+), 137 deletions(-) diff --git a/tiledb/sm/array/array_directory.cc b/tiledb/sm/array/array_directory.cc index 11631b82f86..81ce5deb6e4 100644 --- a/tiledb/sm/array/array_directory.cc +++ b/tiledb/sm/array/array_directory.cc @@ -410,22 +410,18 @@ Status ArrayDirectory::load() { consolidated_commit_uris_set_); } else { // Process root dir. - auto&& [st2, fragment_uris_v1_v11] = - load_root_dir_uris_v1_v11(root_dir_uris); - RETURN_NOT_OK(st2); + auto fragment_uris_v1_v11 = load_root_dir_uris_v1_v11(root_dir_uris); // Process commit dir. - auto&& [st3, fragment_uris_v12_or_higher] = - load_commits_dir_uris_v12_or_higher( - commits_dir_uris, consolidated_commit_uris.value()); - RETURN_NOT_OK(st3); + auto fragment_uris_v12_or_higher = load_commits_dir_uris_v12_or_higher( + commits_dir_uris, consolidated_commit_uris.value()); // Append the two fragment URI vectors together - unfiltered_fragment_uris_ = std::move(fragment_uris_v1_v11.value()); + unfiltered_fragment_uris_ = std::move(fragment_uris_v1_v11); unfiltered_fragment_uris_.insert( unfiltered_fragment_uris_.end(), - fragment_uris_v12_or_higher.value().begin(), - fragment_uris_v12_or_higher.value().end()); + fragment_uris_v12_or_higher.begin(), + fragment_uris_v12_or_higher.end()); // Merge the fragment meta URIs. std::copy( @@ -451,18 +447,15 @@ Status ArrayDirectory::load() { const ArrayDirectory::FilteredFragmentUris ArrayDirectory::filtered_fragment_uris(const bool full_overlap_only) const { // Compute fragment URIs and the vacuum file URIs to vacuum - auto&& [st, fragment_uris_to_vacuum, fragment_vac_uris_to_vacuum] = + auto&& [fragment_uris_to_vacuum, fragment_vac_uris_to_vacuum] = compute_uris_to_vacuum(full_overlap_only, unfiltered_fragment_uris_); - if (!st.ok()) { - throw std::logic_error(st.message()); - } // Compute commit URIs to vacuum, which only need to be done for fragment // vacuuming mode. std::vector commit_uris_to_vacuum; std::vector commit_uris_to_ignore; if (mode_ == ArrayDirectoryMode::VACUUM_FRAGMENTS) { - for (auto& uri : fragment_uris_to_vacuum.value()) { + for (auto& uri : fragment_uris_to_vacuum) { auto commit_uri = get_commit_uri(uri); if (consolidated_commit_uris_set_.count(commit_uri.c_str()) == 0) { commit_uris_to_vacuum.emplace_back(commit_uri); @@ -473,20 +466,15 @@ ArrayDirectory::filtered_fragment_uris(const bool full_overlap_only) const { } // Compute filtered fragment URIs - auto&& [st2, fragment_filtered_uris] = compute_filtered_uris( - full_overlap_only, - unfiltered_fragment_uris_, - fragment_uris_to_vacuum.value()); - if (!st2.ok()) { - throw std::logic_error(st.message()); - } + auto fragment_filtered_uris = compute_filtered_uris( + full_overlap_only, unfiltered_fragment_uris_, fragment_uris_to_vacuum); return FilteredFragmentUris( - std::move(fragment_uris_to_vacuum.value()), + std::move(fragment_uris_to_vacuum), std::move(commit_uris_to_vacuum), std::move(commit_uris_to_ignore), - std::move(fragment_vac_uris_to_vacuum.value()), - std::move(fragment_filtered_uris.value())); + std::move(fragment_vac_uris_to_vacuum), + std::move(fragment_filtered_uris)); } const std::vector& ArrayDirectory::fragment_meta_uris() const { @@ -610,16 +598,14 @@ std::vector ArrayDirectory::list_root_dir_uris() { return ls(uri_); } -tuple>> -ArrayDirectory::load_root_dir_uris_v1_v11( +std::vector ArrayDirectory::load_root_dir_uris_v1_v11( const std::vector& root_dir_uris) { // Compute the fragment URIs - auto&& [st1, fragment_uris] = compute_fragment_uris_v1_v11(root_dir_uris); - RETURN_NOT_OK_TUPLE(st1, nullopt); + auto fragment_uris = compute_fragment_uris_v1_v11(root_dir_uris); fragment_meta_uris_ = compute_fragment_meta_uris(root_dir_uris); - return {Status::Ok(), fragment_uris.value()}; + return fragment_uris; } std::vector ArrayDirectory::list_commits_dir_uris() { @@ -628,8 +614,7 @@ std::vector ArrayDirectory::list_commits_dir_uris() { return ls(uri_.join_path(constants::array_commits_dir_name)); } -tuple>> -ArrayDirectory::load_commits_dir_uris_v12_or_higher( +std::vector ArrayDirectory::load_commits_dir_uris_v12_or_higher( const std::vector& commits_dir_uris, const std::vector& consolidated_uris) { std::vector fragment_uris; @@ -682,7 +667,7 @@ ArrayDirectory::load_commits_dir_uris_v12_or_higher( } } - return {Status::Ok(), fragment_uris}; + return fragment_uris; } std::vector @@ -827,18 +812,12 @@ void ArrayDirectory::load_array_meta_uris() { } // Compute array metadata URIs and vacuum URIs to vacuum. */ - auto&& [st1, array_meta_uris_to_vacuum, array_meta_vac_uris_to_vacuum] = + std::tie(array_meta_uris_to_vacuum_, array_meta_vac_uris_to_vacuum_) = compute_uris_to_vacuum(true, array_meta_dir_uris); - throw_if_not_ok(st1); - array_meta_uris_to_vacuum_ = std::move(array_meta_uris_to_vacuum.value()); - array_meta_vac_uris_to_vacuum_ = - std::move(array_meta_vac_uris_to_vacuum.value()); // Compute filtered array metadata URIs - auto&& [st2, array_meta_filtered_uris] = compute_filtered_uris( + array_meta_uris_ = compute_filtered_uris( true, array_meta_dir_uris, array_meta_uris_to_vacuum_); - throw_if_not_ok(st2); - array_meta_uris_ = std::move(array_meta_filtered_uris.value()); array_meta_dir_uris.clear(); } @@ -901,8 +880,7 @@ void ArrayDirectory::load_commits_uris_to_consolidate( } } -tuple>> -ArrayDirectory::compute_fragment_uris_v1_v11( +std::vector ArrayDirectory::compute_fragment_uris_v1_v11( const std::vector& array_dir_uris) const { std::vector fragment_uris; @@ -937,7 +915,7 @@ ArrayDirectory::compute_fragment_uris_v1_v11( } } - return {Status::Ok(), fragment_uris}; + return fragment_uris; } std::vector ArrayDirectory::compute_fragment_meta_uris( @@ -994,7 +972,7 @@ bool ArrayDirectory::timestamps_overlap( } } -tuple>, optional>> +tuple, std::vector> ArrayDirectory::compute_uris_to_vacuum( const bool full_overlap_only, const std::vector& uris) const { // Get vacuum URIs @@ -1095,11 +1073,10 @@ ArrayDirectory::compute_uris_to_vacuum( vac_uris_to_vacuum.emplace_back(vac_files[i]); } - return {Status::Ok(), uris_to_vacuum, vac_uris_to_vacuum}; + return {uris_to_vacuum, vac_uris_to_vacuum}; } -tuple>> -ArrayDirectory::compute_filtered_uris( +std::vector ArrayDirectory::compute_filtered_uris( const bool full_overlap_only, const std::vector& uris, const std::vector& to_ignore) const { @@ -1107,7 +1084,7 @@ ArrayDirectory::compute_filtered_uris( // Do nothing if there are not enough URIs if (uris.empty()) { - return {Status::Ok(), filtered_uris}; + return filtered_uris; } // Get the URIs that must be ignored @@ -1156,7 +1133,7 @@ ArrayDirectory::compute_filtered_uris( // Sort the names based on the timestamps std::sort(filtered_uris.begin(), filtered_uris.end()); - return {Status::Ok(), filtered_uris}; + return filtered_uris; } Status ArrayDirectory::compute_array_schema_uris( diff --git a/tiledb/sm/array/array_directory.h b/tiledb/sm/array/array_directory.h index 3016109a8b8..4e1444264df 100644 --- a/tiledb/sm/array/array_directory.h +++ b/tiledb/sm/array/array_directory.h @@ -323,7 +323,7 @@ class ArrayDirectory { * * @param array_schema_uri The URI path of the array schema. * @param encryption_key The encryption key to use. - * @return Status, the loaded array schema + * @return the loaded array schema */ static shared_ptr load_array_schema_from_uri( ContextResources& resources, @@ -348,7 +348,7 @@ class ArrayDirectory { * @param array_dir The ArrayDirectory object used to retrieve the * various URIs in the array directory. * @param encryption_key The encryption key to use. - * @return Status, a new ArraySchema + * @return a new ArraySchema */ shared_ptr load_array_schema_latest( const EncryptionKey& encryption_key, @@ -361,10 +361,8 @@ class ArrayDirectory { * @param array_dir The ArrayDirectory object used to retrieve the * various URIs in the array directory. * @param encryption_key The encryption key to use. - * @return tuple of Status, latest array schema and all array schemas. - * Status Ok on success, else error - * ArraySchema The latest array schema. - * ArraySchemaMap Map of all array schemas loaded, keyed by name + * @return tuple of latest array schema and all array schemas loaded, keyed by + * name. */ tuple< shared_ptr, @@ -377,10 +375,7 @@ class ArrayDirectory { * Loads all schemas of an array from persistent storage into memory. * * @param encryption_key The encryption key to use. - * @return tuple of Status and optional unordered map. If Status is an error - * the unordered_map will be nullopt - * Status Ok on success, else error - * ArraySchemaMap Map of all array schemas found keyed by name + * @return Map of all array schemas found keyed by name */ std::unordered_map> load_all_array_schemas( @@ -663,9 +658,9 @@ class ArrayDirectory { /** * Loads the root directory uris for v1 to v11. * - * @return Status, vector of fragment URIs. + * @return vector of fragment URIs. */ - tuple>> load_root_dir_uris_v1_v11( + std::vector load_root_dir_uris_v1_v11( const std::vector& root_dir_uris); /** @@ -678,9 +673,9 @@ class ArrayDirectory { /** * Loads the commit directory uris for v12 or higher. * - * @return Status, vector of fragment URIs. + * @return vector of fragment URIs. */ - tuple>> load_commits_dir_uris_v12_or_higher( + std::vector load_commits_dir_uris_v12_or_higher( const std::vector& commits_dir_uris, const std::vector& consolidated_uris); @@ -722,7 +717,7 @@ class ArrayDirectory { * Computes the fragment URIs from the input array directory URIs, for * versions 1 to 11. */ - tuple>> compute_fragment_uris_v1_v11( + std::vector compute_fragment_uris_v1_v11( const std::vector& array_dir_uris) const; /** @@ -736,11 +731,10 @@ class ArrayDirectory { * * @param full_overlap_only Only enable full overlap. * @param uris The URIs to calculate the URIs to vacuum from. - * @return Status, a vector of the URIs to vacuum, a vector of - * the vac file URIs to vacuum. + * @return a vector of the URIs to vacuum, a vector of the vac file URIs to + * vacuum. */ - tuple>, optional>> - compute_uris_to_vacuum( + tuple, std::vector> compute_uris_to_vacuum( const bool full_overlap_only, const std::vector& uris) const; /** @@ -750,9 +744,9 @@ class ArrayDirectory { * @param full_overlap_only Only enable full overlap. * @param uris The URIs to filter. * @param to_ignore The URIs to ignore (because they are vacuumed). - * @return Status, vector of filtered timestamped URIs. + * @return vector of filtered timestamped URIs. */ - tuple>> compute_filtered_uris( + std::vector compute_filtered_uris( const bool full_overlap_only, const std::vector& uris, const std::vector& to_ignore) const; diff --git a/tiledb/sm/group/group_directory.cc b/tiledb/sm/group/group_directory.cc index 297f00ddc86..bd9781e7c3d 100644 --- a/tiledb/sm/group/group_directory.cc +++ b/tiledb/sm/group/group_directory.cc @@ -70,10 +70,7 @@ GroupDirectory::GroupDirectory( , timestamp_start_(timestamp_start) , timestamp_end_(timestamp_end) , loaded_(false) { - auto st = load(); - if (!st.ok()) { - throw GroupDirectoryException(st.message()); - } + load(); } /* ********************************* */ @@ -121,7 +118,7 @@ const std::vector& GroupDirectory::group_detail_uris() const { return group_detail_uris_; } -Status GroupDirectory::load() { +void GroupDirectory::load() { assert(!loaded_); std::vector tasks; std::vector root_dir_uris; @@ -130,20 +127,15 @@ Status GroupDirectory::load() { // Lists all directories in parallel. Skipping for schema only. // Some processing is also done here for things that don't depend on others. // List (in parallel) the root directory URIs - tasks.emplace_back(tp_.execute([&]() { - auto&& [st, uris] = list_root_dir_uris(); - throw_if_not_ok(st); - - root_dir_uris = std::move(uris).value(); - })); + tasks.emplace_back( + tp_.execute([&]() { root_dir_uris = list_root_dir_uris(); })); // Load (in parallel) the group metadata URIs - tasks.emplace_back( - tp_.execute([&]() { throw_if_not_ok(load_group_meta_uris()); })); + tasks.emplace_back(tp_.execute(&GroupDirectory::load_group_meta_uris, this)); - // Load (in paralell) the group details URIs + // Load (in parallel) the group details URIs tasks.emplace_back( - tp_.execute([&] { throw_if_not_ok(load_group_detail_uris()); })); + tp_.execute(&GroupDirectory::load_group_detail_uris, this)); // Wait for all tasks to complete tp_.wait_all(tasks); @@ -167,8 +159,6 @@ Status GroupDirectory::load() { // The URI manager has been loaded successfully loaded_ = true; - - return Status::Ok(); } bool GroupDirectory::loaded() const { @@ -179,69 +169,52 @@ bool GroupDirectory::loaded() const { /* PRIVATE METHODS */ /* ********************************* */ -tuple>> GroupDirectory::list_root_dir_uris() { +std::vector GroupDirectory::list_root_dir_uris() { // List the group directory URIs std::vector group_dir_uris; - RETURN_NOT_OK_TUPLE(vfs_.ls(uri_, &group_dir_uris), nullopt); + throw_if_not_ok(vfs_.ls(uri_, &group_dir_uris)); - return {Status::Ok(), group_dir_uris}; + return group_dir_uris; } -Status GroupDirectory::load_group_meta_uris() { +void GroupDirectory::load_group_meta_uris() { // Load the URIs in the group metadata directory std::vector group_meta_dir_uris; auto group_meta_uri = uri_.join_path(constants::group_metadata_dir_name); throw_if_not_ok(vfs_.ls(group_meta_uri, &group_meta_dir_uris)); // Compute and group metadata URIs and the vacuum file URIs to vacuum. - auto&& [st1, group_meta_uris_to_vacuum, group_meta_vac_uris_to_vacuum] = + std::tie(group_meta_uris_to_vacuum_, group_meta_vac_uris_to_vacuum_) = compute_uris_to_vacuum(group_meta_dir_uris); - RETURN_NOT_OK(st1); - group_meta_uris_to_vacuum_ = std::move(group_meta_uris_to_vacuum.value()); - group_meta_vac_uris_to_vacuum_ = - std::move(group_meta_vac_uris_to_vacuum.value()); // Compute filtered group metadata URIs - auto&& [st2, group_meta_filtered_uris] = + group_meta_uris_ = compute_filtered_uris(group_meta_dir_uris, group_meta_uris_to_vacuum_); - RETURN_NOT_OK(st2); - group_meta_uris_ = std::move(group_meta_filtered_uris.value()); group_meta_dir_uris.clear(); - - return Status::Ok(); } -Status GroupDirectory::load_group_detail_uris() { +void GroupDirectory::load_group_detail_uris() { // Load the URIs in the group details directory std::vector group_detail_dir_uris; auto group_detail_uri = uri_.join_path(constants::group_detail_dir_name); throw_if_not_ok(vfs_.ls(group_detail_uri, &group_detail_dir_uris)); // Compute and group details URIs and the vacuum file URIs to vacuum. - auto&& [st1, group_detail_uris_to_vacuum, group_detail_vac_uris_to_vacuum] = + std::tie(group_detail_uris_to_vacuum_, group_detail_vac_uris_to_vacuum_) = compute_uris_to_vacuum(group_detail_dir_uris); - throw_if_not_ok(st1); - group_detail_uris_to_vacuum_ = std::move(group_detail_uris_to_vacuum.value()); - group_detail_vac_uris_to_vacuum_ = - std::move(group_detail_vac_uris_to_vacuum.value()); // Compute filtered group details URIs - auto&& [st2, group_detail_filtered_uris] = compute_filtered_uris( + group_detail_uris_ = compute_filtered_uris( group_detail_dir_uris, group_detail_uris_to_vacuum_); - throw_if_not_ok(st2); - group_detail_uris_ = std::move(group_detail_filtered_uris.value()); - group_detail_dir_uris.clear(); // Set the latest array schema URI if (!group_detail_uris_.empty()) { latest_group_details_uri_ = group_detail_uris_.back().uri_; assert(!latest_group_details_uri_.is_invalid()); } - - return Status::Ok(); } -tuple>, optional>> +tuple, std::vector> GroupDirectory::compute_uris_to_vacuum(const std::vector& uris) const { // Get vacuum URIs std::vector vac_files; @@ -304,17 +277,16 @@ GroupDirectory::compute_uris_to_vacuum(const std::vector& uris) const { vac_uris_to_vacuum.emplace_back(vac_files[i]); } - return {Status::Ok(), uris_to_vacuum, vac_uris_to_vacuum}; + return {uris_to_vacuum, vac_uris_to_vacuum}; } -tuple>> -GroupDirectory::compute_filtered_uris( +std::vector GroupDirectory::compute_filtered_uris( const std::vector& uris, const std::vector& to_ignore) const { std::vector filtered_uris; // Do nothing if there are not enough URIs if (uris.empty()) { - return {Status::Ok(), filtered_uris}; + return filtered_uris; } // Get the URIs that must be ignored @@ -346,7 +318,7 @@ GroupDirectory::compute_filtered_uris( // Sort the names based on the timestamps std::sort(filtered_uris.begin(), filtered_uris.end()); - return {Status::Ok(), filtered_uris}; + return filtered_uris; } bool GroupDirectory::is_vacuum_file(const URI& uri) const { diff --git a/tiledb/sm/group/group_directory.h b/tiledb/sm/group/group_directory.h index be5aec059d0..a3ec6fa79ce 100644 --- a/tiledb/sm/group/group_directory.h +++ b/tiledb/sm/group/group_directory.h @@ -33,7 +33,6 @@ #ifndef TILEDB_GROUP_DIRECTORY_H #define TILEDB_GROUP_DIRECTORY_H -#include "tiledb/common/status.h" #include "tiledb/common/thread_pool/thread_pool.h" #include "tiledb/sm/filesystem/uri.h" #include "tiledb/sm/filesystem/vfs.h" @@ -202,30 +201,30 @@ class GroupDirectory { /* ********************************* */ /** Loads the URIs from the various group subdirectories. */ - Status load(); + void load(); /** * List the root directory uris for v1 to v11. * - * @return Status, vector of URIs. + * @return vector of URIs. */ - tuple>> list_root_dir_uris(); + std::vector list_root_dir_uris(); /** Loads the group metadata URIs. */ - Status load_group_meta_uris(); + void load_group_meta_uris(); /** Loads the group details URIs. */ - Status load_group_detail_uris(); + void load_group_detail_uris(); /** * Computes the fragment URIs and vacuum URIs to vacuum. * * @param uris The URIs to calculate the URIs to vacuum from. - * @return Status, a vector of the URIs to vacuum, a vector of - * the vac file URIs to vacuum. + * @return a vector of the URIs to vacuum, a vector of the vac file URIs to + * vacuum. */ - tuple>, optional>> - compute_uris_to_vacuum(const std::vector& uris) const; + tuple, std::vector> compute_uris_to_vacuum( + const std::vector& uris) const; /** * Computes the filtered URIs based on the input, which fall @@ -233,9 +232,9 @@ class GroupDirectory { * * @param uris The URIs to filter. * @param to_ignore The URIs to ignore (because they are vacuumed). - * @return Status, vector of filtered timestamped URIs. + * @return vector of filtered timestamped URIs. */ - tuple>> compute_filtered_uris( + std::vector compute_filtered_uris( const std::vector& uris, const std::vector& to_ignore) const; /** Returns true if the input URI is a vacuum file. */ diff --git a/tiledb/sm/query/legacy/reader.cc b/tiledb/sm/query/legacy/reader.cc index b6d05b8175c..41914044e47 100644 --- a/tiledb/sm/query/legacy/reader.cc +++ b/tiledb/sm/query/legacy/reader.cc @@ -330,9 +330,8 @@ Status Reader::load_initial_data() { } // Load delete conditions. - auto&& [conditions, update_values] = - load_delete_and_update_conditions(resources_, *array_.get()); - delete_and_update_conditions_ = conditions; + std::tie(delete_and_update_conditions_, std::ignore) = + load_delete_and_update_conditions(resources_, *array_); // Set timestamps variables user_requested_timestamps_ = buffers_.count(constants::timestamps) != 0 || diff --git a/tiledb/sm/query/readers/sparse_index_reader_base.cc b/tiledb/sm/query/readers/sparse_index_reader_base.cc index d5113d5ef3c..a81c0106f13 100644 --- a/tiledb/sm/query/readers/sparse_index_reader_base.cc +++ b/tiledb/sm/query/readers/sparse_index_reader_base.cc @@ -217,9 +217,8 @@ Status SparseIndexReaderBase::load_initial_data() { const auto dim_num = array_schema_.dim_num(); // Load delete conditions. - auto&& [conditions, update_values] = + std::tie(delete_and_update_conditions_, std::ignore) = load_delete_and_update_conditions(resources_, *array_.get()); - delete_and_update_conditions_ = conditions; bool make_timestamped_conditions = need_timestamped_conditions(); if (make_timestamped_conditions) { From dc139ac8e173dc66256042da1257490ac47e8c97 Mon Sep 17 00:00:00 2001 From: Theodore Tsirpanis Date: Thu, 13 Jun 2024 19:37:54 +0300 Subject: [PATCH 05/11] Fix tests. --- tiledb/common/thread_pool/test/unit_thread_pool.cc | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/tiledb/common/thread_pool/test/unit_thread_pool.cc b/tiledb/common/thread_pool/test/unit_thread_pool.cc index 52e0cb25e00..d3a51ff0057 100644 --- a/tiledb/common/thread_pool/test/unit_thread_pool.cc +++ b/tiledb/common/thread_pool/test/unit_thread_pool.cc @@ -98,12 +98,14 @@ size_t random_ms(size_t max = 3) { */ void wait_all( ThreadPool& pool, bool use_wait, std::vector& results) { + // Do not use REQUIRE_NOTHROW here; it will invoke more test code and Catch2's + // checks are not reentrant. if (use_wait) { for (auto& r : results) { - REQUIRE_NOTHROW(pool.wait(r)); + pool.wait(r); } } else { - REQUIRE_NOTHROW(pool.wait_all(results)); + pool.wait_all(results); } } @@ -185,7 +187,7 @@ TEST_CASE( // Because the thread pool has 2 threads, the first two will probably be // executing at this point, but some will still be queued. - REQUIRE_THROWS(cancelable_tasks.cancel_all_tasks()); + REQUIRE_NOTHROW(cancelable_tasks.cancel_all_tasks()); } SECTION("- With cancellation callback") { @@ -206,7 +208,7 @@ TEST_CASE( // Because the thread pool has 2 threads, the first two will probably be // executing at this point, but some will still be queued. - REQUIRE_THROWS(cancelable_tasks.cancel_all_tasks()); + REQUIRE_NOTHROW(cancelable_tasks.cancel_all_tasks()); REQUIRE(num_cancelled == ((int64_t)tasks.size() - result)); } } From 4118efa0f31ce574513cf5f6bb309b9d5ffd7fb6 Mon Sep 17 00:00:00 2001 From: Theodore Tsirpanis Date: Thu, 13 Jun 2024 21:38:12 +0300 Subject: [PATCH 06/11] Add tags to the thread pool tests. --- .../thread_pool/test/unit_thread_pool.cc | 20 +++++++++++-------- 1 file changed, 12 insertions(+), 8 deletions(-) diff --git a/tiledb/common/thread_pool/test/unit_thread_pool.cc b/tiledb/common/thread_pool/test/unit_thread_pool.cc index d3a51ff0057..cefa90b49f5 100644 --- a/tiledb/common/thread_pool/test/unit_thread_pool.cc +++ b/tiledb/common/thread_pool/test/unit_thread_pool.cc @@ -109,13 +109,13 @@ void wait_all( } } -TEST_CASE("ThreadPool: Test empty", "[threadpool]") { +TEST_CASE("ThreadPool: Test empty", "[threadpool][empty]") { for (int i = 0; i < 10; i++) { ThreadPool pool{4}; } } -TEST_CASE("ThreadPool: Test single thread", "[threadpool]") { +TEST_CASE("ThreadPool: Test single thread", "[threadpool][single-thread]") { bool use_wait = GENERATE(true, false); std::atomic result = 0; // needs to be atomic b/c scavenging thread can // run in addition to thread pool @@ -133,7 +133,8 @@ TEST_CASE("ThreadPool: Test single thread", "[threadpool]") { REQUIRE(result == 100); } -TEST_CASE("ThreadPool: Test multiple threads", "[threadpool]") { +TEST_CASE( + "ThreadPool: Test multiple threads", "[threadpool][multiple-threads]") { bool use_wait = GENERATE(true, false); std::atomic result(0); std::vector results; @@ -152,7 +153,7 @@ struct AtomicHolder { std::atomic val_; }; -TEST_CASE("ThreadPool: Test no wait", "[threadpool]") { +TEST_CASE("ThreadPool: Test no wait", "[threadpool][no-wait]") { { ThreadPool pool{4}; auto ptr = tdb::make_shared(HERE(), 0); @@ -224,7 +225,8 @@ TEST_CASE( // } // } -TEST_CASE("ThreadPool: Test recursion, simplest case", "[threadpool]") { +TEST_CASE( + "ThreadPool: Test recursion, simplest case", "[threadpool][recursion]") { bool use_wait = GENERATE(true, false); ThreadPool pool{1}; @@ -247,7 +249,7 @@ TEST_CASE("ThreadPool: Test recursion, simplest case", "[threadpool]") { REQUIRE(result == 1); } -TEST_CASE("ThreadPool: Test recursion", "[threadpool]") { +TEST_CASE("ThreadPool: Test recursion", "[threadpool][recursion]") { bool use_wait = GENERATE(true, false); size_t num_threads = 0; SECTION("- One thread") { @@ -320,7 +322,9 @@ TEST_CASE("ThreadPool: Test recursion", "[threadpool]") { cv.wait(ul); } -TEST_CASE("ThreadPool: Test recursion, two pools", "[threadpool]") { +TEST_CASE( + "ThreadPool: Test recursion, two pools", + "[threadpool][recursion][two-pools]") { bool use_wait = GENERATE(true, false); size_t num_threads = 0; @@ -428,7 +432,7 @@ TEST_CASE("ThreadPool: Test recursion, two pools", "[threadpool]") { } } -TEST_CASE("ThreadPool: Test Exceptions", "[threadpool]") { +TEST_CASE("ThreadPool: Test Exceptions", "[threadpool][exceptions]") { bool use_wait = GENERATE(true, false); std::atomic result(0); ThreadPool pool{7}; From dcc581477aee12c87ff19996de936ecc91495ca5 Mon Sep 17 00:00:00 2001 From: Theodore Tsirpanis Date: Thu, 13 Jun 2024 21:38:34 +0300 Subject: [PATCH 07/11] Do not test throwing things other than exceptions from the thread pool. --- .../thread_pool/test/unit_thread_pool.cc | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/tiledb/common/thread_pool/test/unit_thread_pool.cc b/tiledb/common/thread_pool/test/unit_thread_pool.cc index cefa90b49f5..804ca80e30f 100644 --- a/tiledb/common/thread_pool/test/unit_thread_pool.cc +++ b/tiledb/common/thread_pool/test/unit_thread_pool.cc @@ -447,7 +447,7 @@ TEST_CASE("ThreadPool: Test Exceptions", "[threadpool][exceptions]") { results.push_back(pool.execute([&result]() { auto tmp = result++; if (tmp == 13) { - throw(std::string("Unripe banana")); + throw std::exception("Unripe banana"); } })); } @@ -479,10 +479,10 @@ TEST_CASE("ThreadPool: Test Exceptions", "[threadpool][exceptions]") { results.push_back(pool.execute([&result]() { auto tmp = result++; if (tmp == 13) { - throw(std::string("Unripe banana")); + throw std::string("Unripe banana"); } if (tmp == 31) { - throw(Status_TileError("Unbaked potato")); + throw std::exception("Unbaked potato"); } })); } @@ -500,10 +500,10 @@ TEST_CASE("ThreadPool: Test Exceptions", "[threadpool][exceptions]") { results.push_back(pool.execute([&result]() { auto tmp = result++; if (tmp == 31) { - throw(std::string("Unripe banana")); + throw std::string("Unripe banana"); } if (tmp == 13) { - throw(Status_TileError("Unbaked potato")); + throw std::exception("Unbaked potato"); } })); } @@ -521,10 +521,10 @@ TEST_CASE("ThreadPool: Test Exceptions", "[threadpool][exceptions]") { results.push_back(pool.execute([i, &result]() { result++; if (i == 13) { - throw(std::string("Unripe banana")); + throw std::exception("Unripe banana"); } if (i == 31) { - throw(Status_TileError("Unbaked potato")); + throw std::exception("Unbaked potato"); } })); } @@ -542,10 +542,10 @@ TEST_CASE("ThreadPool: Test Exceptions", "[threadpool][exceptions]") { results.push_back(pool.execute([i, &result]() { ++result; if (i == 31) { - throw(std::string("Unripe banana")); + throw(std::exception("Unripe banana")); } if (i == 13) { - throw(Status_TileError("Unbaked potato")); + throw std::exception("Unbaked potato"); } })); } From fcc7ca83d0c08df045e6a9741d7bb57981d5c9f4 Mon Sep 17 00:00:00 2001 From: Theodore Tsirpanis Date: Thu, 13 Jun 2024 21:40:57 +0300 Subject: [PATCH 08/11] Fix compile error. --- tiledb/sm/filesystem/vfs.cc | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tiledb/sm/filesystem/vfs.cc b/tiledb/sm/filesystem/vfs.cc index 5c393e162e0..84fa60ab38f 100644 --- a/tiledb/sm/filesystem/vfs.cc +++ b/tiledb/sm/filesystem/vfs.cc @@ -1127,12 +1127,12 @@ Status VFS::read( thread_buffer, thread_nbytes, use_read_ahead]() { - return read_impl( + throw_if_not_ok(read_impl( uri, thread_offset, thread_buffer, thread_nbytes, - use_read_ahead); + use_read_ahead)); }); results.push_back(std::move(task)); } From 3e72cf9c3e0d70d8052f9472b81ddf47e6f588a2 Mon Sep 17 00:00:00 2001 From: Theodore Tsirpanis Date: Mon, 16 Sep 2024 18:50:42 +0300 Subject: [PATCH 09/11] Remove unnecessary header. --- .../thread_pool/test/unit_thread_pool.cc | 2 -- .../thread_pool/test/unit_thread_pool.h | 31 ------------------- 2 files changed, 33 deletions(-) delete mode 100644 tiledb/common/thread_pool/test/unit_thread_pool.h diff --git a/tiledb/common/thread_pool/test/unit_thread_pool.cc b/tiledb/common/thread_pool/test/unit_thread_pool.cc index 804ca80e30f..9bd965e59cd 100644 --- a/tiledb/common/thread_pool/test/unit_thread_pool.cc +++ b/tiledb/common/thread_pool/test/unit_thread_pool.cc @@ -30,8 +30,6 @@ * Tests the `ThreadPool` class. */ -#include "unit_thread_pool.h" - #include #include #include diff --git a/tiledb/common/thread_pool/test/unit_thread_pool.h b/tiledb/common/thread_pool/test/unit_thread_pool.h deleted file mode 100644 index 908a8a295e4..00000000000 --- a/tiledb/common/thread_pool/test/unit_thread_pool.h +++ /dev/null @@ -1,31 +0,0 @@ -/** - * @file tiledb/common/thread_pool/test/unit_thread_pool.h - * - * @section LICENSE - * - * The MIT License - * - * @copyright Copyright (c) 2021 TileDB, Inc. - * - * Permission is hereby granted, free of charge, to any person obtaining a copy - * of this software and associated documentation files (the "Software"), to deal - * in the Software without restriction, including without limitation the rights - * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell - * copies of the Software, and to permit persons to whom the Software is - * furnished to do so, subject to the following conditions: - * - * The above copyright notice and this permission notice shall be included in - * all copies or substantial portions of the Software. - * - * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR - * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, - * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE - * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER - * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, - * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN - * THE SOFTWARE. - * - * @section DESCRIPTION - */ - -#include From f8d0e610d9cd4acfdcc69616369ac682dec52f62 Mon Sep 17 00:00:00 2001 From: Theodore Tsirpanis Date: Mon, 16 Sep 2024 18:51:08 +0300 Subject: [PATCH 10/11] Throw `runtime_error` instead of `exception`. --- .../common/thread_pool/test/unit_thread_pool.cc | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/tiledb/common/thread_pool/test/unit_thread_pool.cc b/tiledb/common/thread_pool/test/unit_thread_pool.cc index 9bd965e59cd..18912d010f7 100644 --- a/tiledb/common/thread_pool/test/unit_thread_pool.cc +++ b/tiledb/common/thread_pool/test/unit_thread_pool.cc @@ -36,6 +36,7 @@ #include #include #include +#include #include #include "tiledb/common/thread_pool/thread_pool.h" @@ -445,7 +446,7 @@ TEST_CASE("ThreadPool: Test Exceptions", "[threadpool][exceptions]") { results.push_back(pool.execute([&result]() { auto tmp = result++; if (tmp == 13) { - throw std::exception("Unripe banana"); + throw std::runtime_error("Unripe banana"); } })); } @@ -480,7 +481,7 @@ TEST_CASE("ThreadPool: Test Exceptions", "[threadpool][exceptions]") { throw std::string("Unripe banana"); } if (tmp == 31) { - throw std::exception("Unbaked potato"); + throw std::runtime_error("Unbaked potato"); } })); } @@ -501,7 +502,7 @@ TEST_CASE("ThreadPool: Test Exceptions", "[threadpool][exceptions]") { throw std::string("Unripe banana"); } if (tmp == 13) { - throw std::exception("Unbaked potato"); + throw std::runtime_error("Unbaked potato"); } })); } @@ -519,10 +520,10 @@ TEST_CASE("ThreadPool: Test Exceptions", "[threadpool][exceptions]") { results.push_back(pool.execute([i, &result]() { result++; if (i == 13) { - throw std::exception("Unripe banana"); + throw std::runtime_error("Unripe banana"); } if (i == 31) { - throw std::exception("Unbaked potato"); + throw std::runtime_error("Unbaked potato"); } })); } @@ -540,10 +541,10 @@ TEST_CASE("ThreadPool: Test Exceptions", "[threadpool][exceptions]") { results.push_back(pool.execute([i, &result]() { ++result; if (i == 31) { - throw(std::exception("Unripe banana")); + throw(std::runtime_error("Unripe banana")); } if (i == 13) { - throw std::exception("Unbaked potato"); + throw std::runtime_error("Unbaked potato"); } })); } From 34189c261f2984510a17ba84e6cea75c8e675719 Mon Sep 17 00:00:00 2001 From: Theodore Tsirpanis Date: Mon, 16 Sep 2024 20:08:20 +0300 Subject: [PATCH 11/11] Fix error. --- tiledb/sm/query/readers/reader_base.cc | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/tiledb/sm/query/readers/reader_base.cc b/tiledb/sm/query/readers/reader_base.cc index 50f9ab22ce0..868a3af0b0e 100644 --- a/tiledb/sm/query/readers/reader_base.cc +++ b/tiledb/sm/query/readers/reader_base.cc @@ -156,7 +156,7 @@ std::vector ReaderBase::tile_offset_sizes() { // Compute the size of tile offsets per fragments. const auto relevant_fragments = subarray_.relevant_fragments(); - throw_if_not_ok(parallel_for( + parallel_for( &resources_.compute_tp(), 0, relevant_fragments.size(), [&](uint64_t i) { // For easy reference. auto frag_idx = relevant_fragments[i]; @@ -269,8 +269,7 @@ std::vector ReaderBase::tile_offset_sizes() { unsigned offsets_init_size = num_fields * 4 * 32; ret[frag_idx] = offsets_size + offsets_init_size; - return Status::Ok(); - })); + }); return ret; }