From bee5684c70187fda8b93b884e7c209f3d978179a Mon Sep 17 00:00:00 2001 From: jdolence Date: Mon, 11 Dec 2023 08:49:01 -0700 Subject: [PATCH 01/31] trying to use new tasks --- src/CMakeLists.txt | 9 +- src/basic_types.hpp | 2 +- src/bvals/comms/boundary_communication.cpp | 8 +- src/bvals/comms/bvals_in_one.hpp | 4 +- src/defs.hpp | 2 +- src/driver/driver.hpp | 2 +- src/driver/multistage.hpp | 2 +- src/parthenon/driver.hpp | 4 +- src/solvers/bicgstab_solver.hpp | 4 +- src/solvers/mg_solver.hpp | 4 +- src/tasks/tasks.hpp | 490 +++++++++++++++++++++ src/tasks/thread_pool.hpp | 122 +++++ tst/unit/test_tasklist.cpp | 2 +- 13 files changed, 631 insertions(+), 24 deletions(-) create mode 100644 src/tasks/tasks.hpp create mode 100644 src/tasks/thread_pool.hpp diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index aaeabd8a7dd7..0dd7594f4ed8 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -213,10 +213,11 @@ add_library(parthenon solvers/mg_solver.hpp solvers/solver_utils.hpp - tasks/task_id.cpp - tasks/task_id.hpp - tasks/task_list.hpp - tasks/task_types.hpp + tasks/tasks.hpp + #tasks/task_id.cpp + #tasks/task_id.hpp + #tasks/task_list.hpp + #tasks/task_types.hpp time_integration/butcher_integrator.cpp time_integration/low_storage_integrator.cpp diff --git a/src/basic_types.hpp b/src/basic_types.hpp index 9898140f82b4..4f827b1c05e0 100644 --- a/src/basic_types.hpp +++ b/src/basic_types.hpp @@ -45,7 +45,7 @@ using Real = double; // X3DIR z, phi, etc... enum CoordinateDirection { NODIR = -1, X0DIR = 0, X1DIR = 1, X2DIR = 2, X3DIR = 3 }; enum class BlockLocation { Left = 0, Center = 1, Right = 2 }; -enum class TaskStatus { fail, complete, incomplete, iterate, skip, waiting }; +enum class TaskStatus { complete, incomplete, iterate }; enum class AmrTag : int { derefine = -1, same = 0, refine = 1 }; enum class RefinementOp_t { Prolongation, Restriction, None }; diff --git a/src/bvals/comms/boundary_communication.cpp b/src/bvals/comms/boundary_communication.cpp index 8e97e9915c74..6b16510b58c2 100644 --- a/src/bvals/comms/boundary_communication.cpp +++ b/src/bvals/comms/boundary_communication.cpp @@ -34,8 +34,8 @@ #include "mesh/mesh_refinement.hpp" #include "mesh/meshblock.hpp" #include "prolong_restrict/prolong_restrict.hpp" -#include "tasks/task_id.hpp" -#include "tasks/task_list.hpp" + +#include "tasks/tasks.hpp" #include "utils/error_checking.hpp" #include "utils/loop_utils.hpp" @@ -424,11 +424,7 @@ TaskID AddBoundaryExchangeTasks(TaskID dependency, TL_t &tl, } template TaskID AddBoundaryExchangeTasks( TaskID, TaskList &, std::shared_ptr> &, bool); -template TaskID AddBoundaryExchangeTasks( - TaskID, IterativeTasks &, std::shared_ptr> &, bool); template TaskID AddBoundaryExchangeTasks( TaskID, TaskList &, std::shared_ptr> &, bool); -template TaskID AddBoundaryExchangeTasks( - TaskID, IterativeTasks &, std::shared_ptr> &, bool); } // namespace parthenon diff --git a/src/bvals/comms/bvals_in_one.hpp b/src/bvals/comms/bvals_in_one.hpp index dbc48efd821b..05a95276a114 100644 --- a/src/bvals/comms/bvals_in_one.hpp +++ b/src/bvals/comms/bvals_in_one.hpp @@ -25,8 +25,8 @@ #include "basic_types.hpp" #include "bvals/bvals_interfaces.hpp" #include "coordinates/coordinates.hpp" -#include "tasks/task_id.hpp" -#include "tasks/task_list.hpp" + +#include "tasks/tasks.hpp" #include "utils/object_pool.hpp" namespace parthenon { diff --git a/src/defs.hpp b/src/defs.hpp index ace72628f035..47e8c28b6b78 100644 --- a/src/defs.hpp +++ b/src/defs.hpp @@ -107,7 +107,7 @@ struct RegionSize { // TODO(felker): C++ Core Guidelines Enum.5: Don’t use ALL_CAPS for enumerators // (avoid clashes with preprocessor macros). Enumerated type definitions in this file and: // io_wrapper.hpp, bvals.hpp, field_diffusion.hpp, -// task_list.hpp, ??? +// tasks.hpp, ??? // identifiers for all 6 faces of a MeshBlock constexpr int BOUNDARY_NFACES = 6; diff --git a/src/driver/driver.hpp b/src/driver/driver.hpp index 96937177312b..929ea19c3f2b 100644 --- a/src/driver/driver.hpp +++ b/src/driver/driver.hpp @@ -27,7 +27,7 @@ #include "mesh/mesh.hpp" #include "outputs/outputs.hpp" #include "parameter_input.hpp" -#include "tasks/task_list.hpp" +#include "tasks/tasks.hpp" namespace parthenon { diff --git a/src/driver/multistage.hpp b/src/driver/multistage.hpp index 07748ae3564c..8bec06950c03 100644 --- a/src/driver/multistage.hpp +++ b/src/driver/multistage.hpp @@ -22,7 +22,7 @@ #include "driver/driver.hpp" #include "mesh/mesh.hpp" #include "parameter_input.hpp" -#include "tasks/task_list.hpp" +#include "tasks/tasks.hpp" #include "time_integration/staged_integrator.hpp" namespace parthenon { diff --git a/src/parthenon/driver.hpp b/src/parthenon/driver.hpp index 60dd5f3bd7b5..aac6244a3cf1 100644 --- a/src/parthenon/driver.hpp +++ b/src/parthenon/driver.hpp @@ -26,9 +26,7 @@ #include #include #include -#include -#include -#include +#include #include #include #include diff --git a/src/solvers/bicgstab_solver.hpp b/src/solvers/bicgstab_solver.hpp index f285646d2923..90fefeb55ad7 100644 --- a/src/solvers/bicgstab_solver.hpp +++ b/src/solvers/bicgstab_solver.hpp @@ -24,8 +24,8 @@ #include "kokkos_abstraction.hpp" #include "solvers/mg_solver.hpp" #include "solvers/solver_utils.hpp" -#include "tasks/task_id.hpp" -#include "tasks/task_list.hpp" + +#include "tasks/tasks.hpp" namespace parthenon { diff --git a/src/solvers/mg_solver.hpp b/src/solvers/mg_solver.hpp index 074663983099..46255b329075 100644 --- a/src/solvers/mg_solver.hpp +++ b/src/solvers/mg_solver.hpp @@ -23,8 +23,8 @@ #include "interface/state_descriptor.hpp" #include "kokkos_abstraction.hpp" #include "solvers/solver_utils.hpp" -#include "tasks/task_id.hpp" -#include "tasks/task_list.hpp" + +#include "tasks/tasks.hpp" namespace parthenon { diff --git a/src/tasks/tasks.hpp b/src/tasks/tasks.hpp new file mode 100644 index 000000000000..95c99991e53d --- /dev/null +++ b/src/tasks/tasks.hpp @@ -0,0 +1,490 @@ +//======================================================================================== +// (C) (or copyright) 2023. Triad National Security, LLC. All rights reserved. +// +// This program was produced under U.S. Government contract 89233218CNA000001 for Los +// Alamos National Laboratory (LANL), which is operated by Triad National Security, LLC +// for the U.S. Department of Energy/National Nuclear Security Administration. All rights +// in the program are reserved by Triad National Security, LLC, and the U.S. Department +// of Energy/National Nuclear Security Administration. The Government is granted for +// itself and others acting on its behalf a nonexclusive, paid-up, irrevocable worldwide +// license in this material to reproduce, prepare derivative works, distribute copies to +// the public, perform publicly and display publicly, and to permit others to do so. +//======================================================================================== + +#ifndef TASKS_TASKS_HPP_ +#define TASKS_TASKS_HPP_ + +#include + +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include "thread_pool.hpp" + +namespace parthenon { + +enum class TaskListStatus {complete}; // doesn't feel like we need this... +enum class TaskType {normal, completion}; + +class TaskQualifier { + public: + using qualifier_t = uint64_t; + TaskQualifier() = delete; + TaskQualifier(const qualifier_t n) : flags(n) {} + + static inline constexpr qualifier_t normal{0}; + static inline constexpr qualifier_t local_sync{1 << 0}; + static inline constexpr qualifier_t global_sync{1 << 1}; + static inline constexpr qualifier_t completion{1 << 2}; + static inline constexpr qualifier_t once_per_region{1 << 3}; + + bool LocalSync() const { return flags & local_sync; } + bool GlobalSync() const { return flags & global_sync; } + bool Completion() const { return flags & completion; } + bool Once() const { return flags & once_per_region; } + bool Valid() const { + if (LocalSync() && GlobalSync()) return false; + return true; + } + + private: + qualifier_t flags; +}; + +// forward declare Task for TaskID +class Task; +class TaskID { + public: + TaskID() : task(nullptr) {} + // pointers to Task are implicitly convertible to TaskID + TaskID(Task *t) : task(t) {} + + TaskID operator|(const TaskID &other) const { + // calling this operator means you're building a TaskID to hold a dependency + TaskID result; + if (task) { + result.dep.push_back(task); + } else { + result.dep.insert(result.dep.end(), dep.begin(), dep.end()); + } + if (other.task) { + result.dep.push_back(other.task); + } else { + result.dep.insert(result.dep.end(), other.dep.begin(), other.dep.end()); + } + return result; + } + + const std::vector &GetIDs() const { + return std::cref(dep); + } + + bool empty() const { + return (!task && dep.size() == 0); + } + Task *GetTask() { + return task; + } + private: + Task *task; + std::vector dep; +}; + +class Task { + public: + template + Task(TID&& dep, const std::function &func, + std::pair limits = {1,1}) + : f(func), exec_limits(limits) { + if (dep.GetIDs().size() == 0 && dep.GetTask()) { + dependencies.insert(dep.GetTask()); + } else { + for (auto &d : dep.GetIDs()) { + dependencies.insert(d); + } + } + //printf("dependencies are: "); + //for (auto &d : dependencies) { + //printf("%p ", d); + //} + //printf("\n"); + // always add "this" to repeat task if it's incomplete + dependent[static_cast(TaskStatus::incomplete)].push_back(this); + } + + TaskStatus operator()() { + auto status = f(); + if (status == TaskStatus::incomplete) { + printf("Got an incomplete task\n"); + } + if (task_type == TaskType::completion) { + // keep track of how many times it's been called + num_calls += (status == TaskStatus::iterate || status == TaskStatus::complete); + // enforce minimum number of iterations + if (num_calls < exec_limits.first && status == TaskStatus::complete) + status = TaskStatus::iterate; + // enforce maximum number of iterations + if (num_calls == exec_limits.second) status = TaskStatus::complete; + } + // save the status in the Task object + SetStatus(status); + return status; + } + TaskID GetID() { return this; } + bool ready() { + // check that no dependency is incomplete + bool go = true; + for (auto &dep : dependencies) { + go = go && (dep->GetStatus() != TaskStatus::incomplete); + } + return go; + } + void AddDependency(Task *t) { dependencies.insert(t); } + std::unordered_set& GetDependencies() { return dependencies; } + void AddDependent(Task *t, TaskStatus status) { + dependent[static_cast(status)].push_back(t); + } + std::vector &GetDependent(TaskStatus status = TaskStatus::complete) { + return dependent[static_cast(status)]; + } + void SetType(TaskType type) { task_type = type; } + TaskType GetType() { return task_type; } + void SetStatus(TaskStatus status) { + std::lock_guard lock(mutex); + task_status = status; + } + TaskStatus GetStatus() { + std::lock_guard lock(mutex); + return task_status; + } + void reset_iteration() { num_calls = 0; } + + private: + std::function f; + std::array, 3> dependent; + std::unordered_set dependencies; + std::pair exec_limits; + TaskType task_type = TaskType::normal; + int num_calls = 0; + TaskStatus task_status = TaskStatus::incomplete; + std::mutex mutex; +}; + +class TaskRegion; +class TaskList { + friend class TaskRegion; + public: + TaskList() : TaskList(TaskID(), {1,1}) {} + explicit TaskList(const TaskID &dep, std::pair limits) + : dependency(dep), exec_limits(limits) { + // make a trivial first_task after which others will get launched + // simplifies logic for iteration and startup + tasks.push_back(std::make_shared(dependency, [&tasks = tasks]() { + for (auto &t : tasks) { + t->SetStatus(TaskStatus::incomplete); + } + return TaskStatus::complete; + }, exec_limits)); + first_task = tasks.back().get(); + // connect list dependencies to this list's first_task + for (auto t : first_task->GetDependencies()) { + t->AddDependent(first_task, TaskStatus::complete); + } + + // make a trivial last_task that tasks dependent on this list's execution + // can depend on. Also simplifies exiting completed iterations + tasks.push_back(std::make_shared(TaskID(), [&completion_tasks = completion_tasks]() { + for (auto t : completion_tasks) { + t->reset_iteration(); + } + return TaskStatus::complete; + }, exec_limits)); + last_task = tasks.back().get(); + + } + + + template + TaskID AddTask(TaskID dep, Args &&... args) { + return AddTask(TaskQualifier::local_sync, dep, std::forward(args)...); + } + + template + TaskID AddTask(const TaskQualifier tq, TaskID dep, Args&&... args) { + assert(tq.Valid()); + + // user-space tasks always depend on something. if no dependencies are given, + // make the task dependent on the list's first_task + if (dep.empty()) dep = TaskID(first_task); + + if (!tq.Once() || (tq.Once() && unique_id == 0)) { + AddUserTask(dep, std::forward(args)...); + } else { + tasks.push_back(std::make_shared(dep, [=]() { + return TaskStatus::complete; + }, exec_limits)); + } + + Task *my_task = tasks.back().get(); + TaskID id(my_task); + + if (tq.LocalSync() || tq.GlobalSync() || tq.Once()) { + regional_tasks.push_back(my_task); + } + + if (tq.GlobalSync()) { + // make status, request, and comm for this global task + global_status.push_back(std::make_shared(0)); + global_request.push_back(std::make_shared(MPI_REQUEST_NULL)); + // be careful about the custom deleter so it doesn't call + // an MPI function after Finalize + global_comm.emplace_back(new MPI_Comm, [&](MPI_Comm *d) { + int finalized; + MPI_Finalized(&finalized); + if (!finalized) + MPI_Comm_free(d); + }); + // we need another communicator to support multiple in flight non-blocking + // collectives where we can't guarantee calling order across ranks + MPI_Comm_dup(MPI_COMM_WORLD, global_comm.back().get()); + TaskID start; + // only call MPI once per region, on the list with unique_id = 0 + if (unique_id == 0) { + // add a task that starts the Iallreduce on the task statuses + tasks.push_back(std::make_shared(id, [my_task, &stat = *global_status.back(), + &req = *global_request.back(), + &comm = *global_comm.back()]() { + // jump through a couple hoops to figure out statuses of all instances of my_task + // accross all lists in the enclosing TaskRegion + auto dependent = my_task->GetDependent(TaskStatus::complete); + assert(dependent.size() == 1); + auto mytask = *dependent.begin(); + stat = 0; + for (auto dep : mytask->GetDependencies()) { + stat = std::max(stat, static_cast(dep->GetStatus())); + } + MPI_Iallreduce(MPI_IN_PLACE, &stat, 1, MPI_INT, MPI_MAX, comm, &req); + return TaskStatus::complete; + }, exec_limits)); + start = TaskID(tasks.back().get()); + // add a task that tests for completion of the Iallreduces of statuses + tasks.push_back(std::make_shared(start, [&stat = *global_status.back(), + &req = *global_request.back()]() { + int check; + MPI_Test(&req, &check, MPI_STATUS_IGNORE); + if (check) { + return static_cast(stat); + } + return TaskStatus::incomplete; + }, exec_limits)); + } else { // unique_id != 0 + // just add empty tasks + tasks.push_back(std::make_shared(id, [&]() { return TaskStatus::complete; }, exec_limits)); + start = TaskID(tasks.back().get()); + tasks.push_back(std::make_shared(start, [&]() { return TaskStatus::complete; }, exec_limits)); + } + // reset id so it now points at the task that finishes the Iallreduce + id = TaskID(tasks.back().get()); + // make the task that starts the Iallreduce point at the one that finishes it + start.GetTask()->AddDependent(id.GetTask(), TaskStatus::complete); + // for any status != incomplete, my_task should point at the mpi reduction + my_task->AddDependent(start.GetTask(), TaskStatus::complete); + my_task->AddDependent(start.GetTask(), TaskStatus::iterate); + // make the finish Iallreduce task finish on all lists before moving on + regional_tasks.push_back(id.GetTask()); + } + + // connect completion tasks to last_task + if (tq.Completion()) { + auto t = id.GetTask(); + t->SetType(TaskType::completion); + t->AddDependent(last_task, TaskStatus::complete); + completion_tasks.push_back(t); + } + + // make connections so tasks point to this task to run next + for (auto d : my_task->GetDependencies()) { + if (d->GetType() == TaskType::completion) { + d->AddDependent(my_task, TaskStatus::iterate); + } else { + d->AddDependent(my_task, TaskStatus::complete); + } + } + return id; + } + + template + std::pair AddSublist(TID&& dep, std::pair minmax_iters = {1,1}) { + sublists.push_back(std::make_shared(dep, minmax_iters)); + auto &tl = *sublists.back(); + tl.SetID(unique_id); + return std::make_pair(std::ref(tl), TaskID(tl.last_task)); + } + + private: + TaskID dependency; + std::pair exec_limits; + // put these in shared_ptrs so copying TaskList works as expected + std::vector> tasks; + std::vector> sublists; + std::vector> global_status; + std::vector> global_request; + std::vector> global_comm; + // vectors are fine for these + std::vector regional_tasks; + std::vector global_tasks; + std::vector completion_tasks; + // special startup and takedown tasks auto added to lists + Task *first_task; + Task *last_task; + // a unique id to support tasks that should only get executed once per region + int unique_id; + + Task *GetStartupTask() { return first_task; } + size_t NumRegional() const { return regional_tasks.size(); } + Task *Regional(const int i) { return regional_tasks[i]; } + void SetID(const int id) { unique_id = id; } + + void ConnectIteration() { + if (completion_tasks.size() != 0) { + auto last = completion_tasks.back(); + last->AddDependent(first_task, TaskStatus::iterate); + } + for (auto &tl : sublists) tl->ConnectIteration(); + } + + template + void AddUserTask(TaskID &dep, TaskStatus (T::*func)(Args1...), U *obj, Args2 &&... args) { + tasks.push_back(std::make_shared(dep, [=]() mutable -> TaskStatus { + return (obj->*func)(std::forward(args)...); + }, exec_limits)); + } + + template + void AddUserTask(TaskID &dep, F&& func, Args &&... args) { + tasks.push_back(std::make_shared(dep, [=, func = std::forward(func)]() mutable -> TaskStatus { + return func(std::forward(args)...); + }, exec_limits)); + } +}; + +class TaskRegion { + public: + TaskRegion() = delete; + TaskRegion(const int num_lists) + : task_lists(num_lists) { + for (int i = 0; i < num_lists; i++) task_lists[i].SetID(i); + } + + void Execute(ThreadPool &pool) { + // first, if needed, finish building the graph + if (!graph_built) BuildGraph(); + + // declare this so it can call itself + std::function ProcessTask; + ProcessTask = [&pool, &ProcessTask](Task *task) -> TaskStatus { + auto status = task->operator()(); + auto next_up = task->GetDependent(status); + for (auto t : next_up) { + if (t->ready()) { + pool.enqueue([t, &ProcessTask]() { + return ProcessTask(t); + }); + } + } + return status; + }; + + // now enqueue the "first_task" for all task lists + for (auto &tl : task_lists) { + auto t = tl.GetStartupTask(); + pool.enqueue([t, &ProcessTask]() { + return ProcessTask(t); + }); + } + + // then wait until everything is done + pool.wait(); + } + + TaskList& operator[](const int i) { + return task_lists[i]; + } + + size_t size() const { + return task_lists.size(); + } + + private: + std::vector task_lists; + bool graph_built = false; + + void BuildGraph() { + // first handle regional dependencies + const auto num_lists = task_lists.size(); + const auto num_regional = task_lists.front().NumRegional(); + std::vector tasks(num_lists); + for (int i = 0; i < num_regional; i++) { + for (int j = 0; j < num_lists; j++) { + tasks[j] = task_lists[j].Regional(i); + } + std::vector> reg_dep; + for (int j = 0; j < num_lists; j++) { + reg_dep.push_back(std::vector()); + for (auto t : tasks[j]->GetDependent(TaskStatus::complete)) { + reg_dep[j].push_back(t); + } + } + for (int j = 0; j < num_lists; j++) { + for (auto t : reg_dep[j]) { + for (int k = 0; k < num_lists; k++) { + if (j == k) continue; + t->AddDependency(tasks[k]); + tasks[k]->AddDependent(t, TaskStatus::complete); + } + } + } + + } + + // now hook up iterations + for (auto &tl : task_lists) { + tl.ConnectIteration(); + } + + graph_built = true; + } +}; + +class TaskCollection { + public: + TaskCollection() = default; + + TaskRegion &AddRegion(const int num_lists) { + regions.emplace_back(num_lists); + return regions.back(); + } + TaskListStatus Execute() { + ThreadPool pool(1); + return Execute(pool); + } + TaskListStatus Execute(ThreadPool &pool) { + for (auto ®ion : regions) { + region.Execute(pool); + } + return TaskListStatus::complete; + } + private: + std::list regions; +}; + +} // namespace parthenon + +#endif // TASKS_TASKS_HPP_ diff --git a/src/tasks/thread_pool.hpp b/src/tasks/thread_pool.hpp new file mode 100644 index 000000000000..fe7cb2e7019c --- /dev/null +++ b/src/tasks/thread_pool.hpp @@ -0,0 +1,122 @@ + +#include +#include +#include +#include +#include +#include +#include + +template +class Queue { + public: + Queue(const int num_workers) : nworkers(num_workers), nwaiting(0) {} + void push(T q) { + std::lock_guard lock(mutex); + queue.push(q); + cv.notify_one(); + } + bool pop(T &q) { + std::unique_lock lock(mutex); + if (queue.empty()) { + nwaiting++; + if (nwaiting == nworkers && started) { + complete = true; + complete_cv.notify_all(); + } + cv.wait(lock, [this]() { return exit || !queue.empty(); }); + nwaiting--; + } + started = true; + if (exit && complete && queue.empty()) return true; + if (!queue.empty()) { + q = queue.front(); + queue.pop(); + } + return false; + } + void signal_kill() { + std::lock_guard lock(mutex); + std::queue().swap(queue); + complete = true; + exit = true; + cv.notify_all(); + } + void signal_exit_when_finished() { + std::lock_guard lock(mutex); + exit = true; + complete = true; + cv.notify_all(); + } + void wait_for_complete() { + std::unique_lock lock(mutex); + if (complete) { + complete = false; + return; + } + complete_cv.wait(lock, [this]() { return complete; }); + complete = false; + } + + private: + const int nworkers; + int nwaiting; + std::queue queue; + std::mutex mutex; + std::condition_variable cv; + std::condition_variable complete_cv; + bool complete = false; + bool exit = false; + bool started = false; + +}; + +class ThreadPool { + public: + explicit ThreadPool(const int numthreads = std::thread::hardware_concurrency()) + : nthreads(numthreads), queue(nthreads) { + for (int i = 0; i < nthreads; i++) { + auto worker = [&]() { + while (true) { + std::function f; + auto stop = queue.pop(f); + if (stop) break; + if (f) f(); + } + }; + threads.emplace_back(worker); + } + } + ~ThreadPool() { + queue.signal_exit_when_finished(); + for (auto &t : threads) { + t.join(); + } + } + + void wait() { + queue.wait_for_complete(); + } + + void kill() { + queue.signal_kill(); + } + + template + std::future::type> enqueue(F &&f, Args &&... args) { + using return_t = typename std::result_of::type; + auto task = std::make_shared>([=, func = std::forward(f)] { + return func(std::forward(args)...); + }); + std::future result = task->get_future(); + queue.push([task]() { + (*task)(); + }); + return result; + } + + private: + const int nthreads; + std::vector threads; + Queue> queue; +}; \ No newline at end of file diff --git a/tst/unit/test_tasklist.cpp b/tst/unit/test_tasklist.cpp index f06ce49c3e99..1790a4eb0ad0 100644 --- a/tst/unit/test_tasklist.cpp +++ b/tst/unit/test_tasklist.cpp @@ -19,7 +19,7 @@ // Internal Includes #include "basic_types.hpp" -#include "tasks/task_list.hpp" +#include "tasks/tasks.hpp" using parthenon::TaskID; using parthenon::TaskList; From 90f3e59f0d7147143c27825daa8a02987a15ee61 Mon Sep 17 00:00:00 2001 From: jdolence Date: Thu, 14 Dec 2023 12:51:25 -0700 Subject: [PATCH 02/31] remove debugging --- src/tasks/tasks.hpp | 10 +--------- 1 file changed, 1 insertion(+), 9 deletions(-) diff --git a/src/tasks/tasks.hpp b/src/tasks/tasks.hpp index 95c99991e53d..5e6cf14116bd 100644 --- a/src/tasks/tasks.hpp +++ b/src/tasks/tasks.hpp @@ -110,20 +110,12 @@ class Task { dependencies.insert(d); } } - //printf("dependencies are: "); - //for (auto &d : dependencies) { - //printf("%p ", d); - //} - //printf("\n"); // always add "this" to repeat task if it's incomplete dependent[static_cast(TaskStatus::incomplete)].push_back(this); } TaskStatus operator()() { auto status = f(); - if (status == TaskStatus::incomplete) { - printf("Got an incomplete task\n"); - } if (task_type == TaskType::completion) { // keep track of how many times it's been called num_calls += (status == TaskStatus::iterate || status == TaskStatus::complete); @@ -213,7 +205,7 @@ class TaskList { template TaskID AddTask(TaskID dep, Args &&... args) { - return AddTask(TaskQualifier::local_sync, dep, std::forward(args)...); + return AddTask(TaskQualifier::normal, dep, std::forward(args)...); } template From 92564e17a13824384c9bfcf8289ab77fcdc9db77 Mon Sep 17 00:00:00 2001 From: jdolence Date: Thu, 14 Dec 2023 12:55:03 -0700 Subject: [PATCH 03/31] formatting --- src/tasks/tasks.hpp | 191 +++++++++++++++++++------------------- src/tasks/thread_pool.hpp | 22 ++--- 2 files changed, 105 insertions(+), 108 deletions(-) diff --git a/src/tasks/tasks.hpp b/src/tasks/tasks.hpp index 5e6cf14116bd..882c6e1b919f 100644 --- a/src/tasks/tasks.hpp +++ b/src/tasks/tasks.hpp @@ -21,17 +21,17 @@ #include #include #include -#include #include +#include #include -#include #include "thread_pool.hpp" +#include namespace parthenon { -enum class TaskListStatus {complete}; // doesn't feel like we need this... -enum class TaskType {normal, completion}; +enum class TaskListStatus { complete }; // doesn't feel like we need this... +enum class TaskType { normal, completion }; class TaskQualifier { public: @@ -82,16 +82,11 @@ class TaskID { return result; } - const std::vector &GetIDs() const { - return std::cref(dep); - } + const std::vector &GetIDs() const { return std::cref(dep); } + + bool empty() const { return (!task && dep.size() == 0); } + Task *GetTask() { return task; } - bool empty() const { - return (!task && dep.size() == 0); - } - Task *GetTask() { - return task; - } private: Task *task; std::vector dep; @@ -100,9 +95,9 @@ class TaskID { class Task { public: template - Task(TID&& dep, const std::function &func, - std::pair limits = {1,1}) - : f(func), exec_limits(limits) { + Task(TID &&dep, const std::function &func, + std::pair limits = {1, 1}) + : f(func), exec_limits(limits) { if (dep.GetIDs().size() == 0 && dep.GetTask()) { dependencies.insert(dep.GetTask()); } else { @@ -139,7 +134,7 @@ class Task { return go; } void AddDependency(Task *t) { dependencies.insert(t); } - std::unordered_set& GetDependencies() { return dependencies; } + std::unordered_set &GetDependencies() { return dependencies; } void AddDependent(Task *t, TaskStatus status) { dependent[static_cast(status)].push_back(t); } @@ -171,19 +166,23 @@ class Task { class TaskRegion; class TaskList { - friend class TaskRegion; + friend class TaskRegion; + public: - TaskList() : TaskList(TaskID(), {1,1}) {} + TaskList() : TaskList(TaskID(), {1, 1}) {} explicit TaskList(const TaskID &dep, std::pair limits) - : dependency(dep), exec_limits(limits) { + : dependency(dep), exec_limits(limits) { // make a trivial first_task after which others will get launched // simplifies logic for iteration and startup - tasks.push_back(std::make_shared(dependency, [&tasks = tasks]() { - for (auto &t : tasks) { - t->SetStatus(TaskStatus::incomplete); - } - return TaskStatus::complete; - }, exec_limits)); + tasks.push_back(std::make_shared( + dependency, + [&tasks = tasks]() { + for (auto &t : tasks) { + t->SetStatus(TaskStatus::incomplete); + } + return TaskStatus::complete; + }, + exec_limits)); first_task = tasks.back().get(); // connect list dependencies to this list's first_task for (auto t : first_task->GetDependencies()) { @@ -192,24 +191,25 @@ class TaskList { // make a trivial last_task that tasks dependent on this list's execution // can depend on. Also simplifies exiting completed iterations - tasks.push_back(std::make_shared(TaskID(), [&completion_tasks = completion_tasks]() { - for (auto t : completion_tasks) { - t->reset_iteration(); - } - return TaskStatus::complete; - }, exec_limits)); + tasks.push_back(std::make_shared( + TaskID(), + [&completion_tasks = completion_tasks]() { + for (auto t : completion_tasks) { + t->reset_iteration(); + } + return TaskStatus::complete; + }, + exec_limits)); last_task = tasks.back().get(); - } - template - TaskID AddTask(TaskID dep, Args &&... args) { + TaskID AddTask(TaskID dep, Args &&...args) { return AddTask(TaskQualifier::normal, dep, std::forward(args)...); } template - TaskID AddTask(const TaskQualifier tq, TaskID dep, Args&&... args) { + TaskID AddTask(const TaskQualifier tq, TaskID dep, Args &&...args) { assert(tq.Valid()); // user-space tasks always depend on something. if no dependencies are given, @@ -219,9 +219,8 @@ class TaskList { if (!tq.Once() || (tq.Once() && unique_id == 0)) { AddUserTask(dep, std::forward(args)...); } else { - tasks.push_back(std::make_shared(dep, [=]() { - return TaskStatus::complete; - }, exec_limits)); + tasks.push_back(std::make_shared( + dep, [=]() { return TaskStatus::complete; }, exec_limits)); } Task *my_task = tasks.back().get(); @@ -240,8 +239,7 @@ class TaskList { global_comm.emplace_back(new MPI_Comm, [&](MPI_Comm *d) { int finalized; MPI_Finalized(&finalized); - if (!finalized) - MPI_Comm_free(d); + if (!finalized) MPI_Comm_free(d); }); // we need another communicator to support multiple in flight non-blocking // collectives where we can't guarantee calling order across ranks @@ -250,37 +248,43 @@ class TaskList { // only call MPI once per region, on the list with unique_id = 0 if (unique_id == 0) { // add a task that starts the Iallreduce on the task statuses - tasks.push_back(std::make_shared(id, [my_task, &stat = *global_status.back(), - &req = *global_request.back(), - &comm = *global_comm.back()]() { - // jump through a couple hoops to figure out statuses of all instances of my_task - // accross all lists in the enclosing TaskRegion - auto dependent = my_task->GetDependent(TaskStatus::complete); - assert(dependent.size() == 1); - auto mytask = *dependent.begin(); - stat = 0; - for (auto dep : mytask->GetDependencies()) { - stat = std::max(stat, static_cast(dep->GetStatus())); - } - MPI_Iallreduce(MPI_IN_PLACE, &stat, 1, MPI_INT, MPI_MAX, comm, &req); - return TaskStatus::complete; - }, exec_limits)); + tasks.push_back(std::make_shared( + id, + [my_task, &stat = *global_status.back(), &req = *global_request.back(), + &comm = *global_comm.back()]() { + // jump through a couple hoops to figure out statuses of all instances of + // my_task accross all lists in the enclosing TaskRegion + auto dependent = my_task->GetDependent(TaskStatus::complete); + assert(dependent.size() == 1); + auto mytask = *dependent.begin(); + stat = 0; + for (auto dep : mytask->GetDependencies()) { + stat = std::max(stat, static_cast(dep->GetStatus())); + } + MPI_Iallreduce(MPI_IN_PLACE, &stat, 1, MPI_INT, MPI_MAX, comm, &req); + return TaskStatus::complete; + }, + exec_limits)); start = TaskID(tasks.back().get()); // add a task that tests for completion of the Iallreduces of statuses - tasks.push_back(std::make_shared(start, [&stat = *global_status.back(), - &req = *global_request.back()]() { - int check; - MPI_Test(&req, &check, MPI_STATUS_IGNORE); - if (check) { - return static_cast(stat); - } - return TaskStatus::incomplete; - }, exec_limits)); + tasks.push_back(std::make_shared( + start, + [&stat = *global_status.back(), &req = *global_request.back()]() { + int check; + MPI_Test(&req, &check, MPI_STATUS_IGNORE); + if (check) { + return static_cast(stat); + } + return TaskStatus::incomplete; + }, + exec_limits)); } else { // unique_id != 0 // just add empty tasks - tasks.push_back(std::make_shared(id, [&]() { return TaskStatus::complete; }, exec_limits)); + tasks.push_back(std::make_shared( + id, [&]() { return TaskStatus::complete; }, exec_limits)); start = TaskID(tasks.back().get()); - tasks.push_back(std::make_shared(start, [&]() { return TaskStatus::complete; }, exec_limits)); + tasks.push_back(std::make_shared( + start, [&]() { return TaskStatus::complete; }, exec_limits)); } // reset id so it now points at the task that finishes the Iallreduce id = TaskID(tasks.back().get()); @@ -313,7 +317,8 @@ class TaskList { } template - std::pair AddSublist(TID&& dep, std::pair minmax_iters = {1,1}) { + std::pair AddSublist(TID &&dep, + std::pair minmax_iters = {1, 1}) { sublists.push_back(std::make_shared(dep, minmax_iters)); auto &tl = *sublists.back(); tl.SetID(unique_id); @@ -349,30 +354,38 @@ class TaskList { auto last = completion_tasks.back(); last->AddDependent(first_task, TaskStatus::iterate); } - for (auto &tl : sublists) tl->ConnectIteration(); + for (auto &tl : sublists) + tl->ConnectIteration(); } template - void AddUserTask(TaskID &dep, TaskStatus (T::*func)(Args1...), U *obj, Args2 &&... args) { - tasks.push_back(std::make_shared(dep, [=]() mutable -> TaskStatus { - return (obj->*func)(std::forward(args)...); - }, exec_limits)); + void AddUserTask(TaskID &dep, TaskStatus (T::*func)(Args1...), U *obj, + Args2 &&...args) { + tasks.push_back(std::make_shared( + dep, + [=]() mutable -> TaskStatus { + return (obj->*func)(std::forward(args)...); + }, + exec_limits)); } template - void AddUserTask(TaskID &dep, F&& func, Args &&... args) { - tasks.push_back(std::make_shared(dep, [=, func = std::forward(func)]() mutable -> TaskStatus { - return func(std::forward(args)...); - }, exec_limits)); + void AddUserTask(TaskID &dep, F &&func, Args &&...args) { + tasks.push_back(std::make_shared( + dep, + [=, func = std::forward(func)]() mutable -> TaskStatus { + return func(std::forward(args)...); + }, + exec_limits)); } }; class TaskRegion { public: TaskRegion() = delete; - TaskRegion(const int num_lists) - : task_lists(num_lists) { - for (int i = 0; i < num_lists; i++) task_lists[i].SetID(i); + TaskRegion(const int num_lists) : task_lists(num_lists) { + for (int i = 0; i < num_lists; i++) + task_lists[i].SetID(i); } void Execute(ThreadPool &pool) { @@ -386,9 +399,7 @@ class TaskRegion { auto next_up = task->GetDependent(status); for (auto t : next_up) { if (t->ready()) { - pool.enqueue([t, &ProcessTask]() { - return ProcessTask(t); - }); + pool.enqueue([t, &ProcessTask]() { return ProcessTask(t); }); } } return status; @@ -397,22 +408,16 @@ class TaskRegion { // now enqueue the "first_task" for all task lists for (auto &tl : task_lists) { auto t = tl.GetStartupTask(); - pool.enqueue([t, &ProcessTask]() { - return ProcessTask(t); - }); + pool.enqueue([t, &ProcessTask]() { return ProcessTask(t); }); } // then wait until everything is done pool.wait(); } - TaskList& operator[](const int i) { - return task_lists[i]; - } + TaskList &operator[](const int i) { return task_lists[i]; } - size_t size() const { - return task_lists.size(); - } + size_t size() const { return task_lists.size(); } private: std::vector task_lists; @@ -443,7 +448,6 @@ class TaskRegion { } } } - } // now hook up iterations @@ -473,6 +477,7 @@ class TaskCollection { } return TaskListStatus::complete; } + private: std::list regions; }; diff --git a/src/tasks/thread_pool.hpp b/src/tasks/thread_pool.hpp index fe7cb2e7019c..aaaf63894e25 100644 --- a/src/tasks/thread_pool.hpp +++ b/src/tasks/thread_pool.hpp @@ -68,13 +68,12 @@ class Queue { bool complete = false; bool exit = false; bool started = false; - }; class ThreadPool { public: explicit ThreadPool(const int numthreads = std::thread::hardware_concurrency()) - : nthreads(numthreads), queue(nthreads) { + : nthreads(numthreads), queue(nthreads) { for (int i = 0; i < nthreads; i++) { auto worker = [&]() { while (true) { @@ -94,24 +93,17 @@ class ThreadPool { } } - void wait() { - queue.wait_for_complete(); - } + void wait() { queue.wait_for_complete(); } - void kill() { - queue.signal_kill(); - } + void kill() { queue.signal_kill(); } template - std::future::type> enqueue(F &&f, Args &&... args) { + std::future::type> enqueue(F &&f, Args &&...args) { using return_t = typename std::result_of::type; - auto task = std::make_shared>([=, func = std::forward(f)] { - return func(std::forward(args)...); - }); + auto task = std::make_shared>( + [=, func = std::forward(f)] { return func(std::forward(args)...); }); std::future result = task->get_future(); - queue.push([task]() { - (*task)(); - }); + queue.push([task]() { (*task)(); }); return result; } From 6fde57d85b158abfe91afb810d80a45b0688ad88 Mon Sep 17 00:00:00 2001 From: jdolence Date: Thu, 14 Dec 2023 13:23:35 -0700 Subject: [PATCH 04/31] remove raw mpi.hpp include --- src/tasks/tasks.hpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/tasks/tasks.hpp b/src/tasks/tasks.hpp index 882c6e1b919f..918c799e4313 100644 --- a/src/tasks/tasks.hpp +++ b/src/tasks/tasks.hpp @@ -14,8 +14,6 @@ #ifndef TASKS_TASKS_HPP_ #define TASKS_TASKS_HPP_ -#include - #include #include #include From 2320c0e47bd2d5c3ac36d78be11454537c3a6e7c Mon Sep 17 00:00:00 2001 From: jdolence Date: Thu, 14 Dec 2023 13:26:42 -0700 Subject: [PATCH 05/31] style --- src/tasks/thread_pool.hpp | 23 ++++++++++++++++++++++- 1 file changed, 22 insertions(+), 1 deletion(-) diff --git a/src/tasks/thread_pool.hpp b/src/tasks/thread_pool.hpp index aaaf63894e25..8254efd7d44a 100644 --- a/src/tasks/thread_pool.hpp +++ b/src/tasks/thread_pool.hpp @@ -1,3 +1,20 @@ +//======================================================================================== +// (C) (or copyright) 2023. Triad National Security, LLC. All rights reserved. +// +// This program was produced under U.S. Government contract 89233218CNA000001 for Los +// Alamos National Laboratory (LANL), which is operated by Triad National Security, LLC +// for the U.S. Department of Energy/National Nuclear Security Administration. All rights +// in the program are reserved by Triad National Security, LLC, and the U.S. Department +// of Energy/National Nuclear Security Administration. The Government is granted for +// itself and others acting on its behalf a nonexclusive, paid-up, irrevocable worldwide +// license in this material to reproduce, prepare derivative works, distribute copies to +// the public, perform publicly and display publicly, and to permit others to do so. +//======================================================================================== + +#ifndef TASKS_THREAD_POOL_HPP_ +#define TASKS_THREAD_POOL_HPP_ + +namespace parthenon { #include #include @@ -111,4 +128,8 @@ class ThreadPool { const int nthreads; std::vector threads; Queue> queue; -}; \ No newline at end of file +}; + +} // namespace parthenon + +#endif // TASKS_THREAD_POOL_HPP_ From 95818bab60d4bbd311b97a962b70e9702af14e37 Mon Sep 17 00:00:00 2001 From: jdolence Date: Thu, 14 Dec 2023 13:45:54 -0700 Subject: [PATCH 06/31] more style --- src/tasks/tasks.hpp | 10 ++++++---- tst/style/cpplint.py | 8 ++++---- 2 files changed, 10 insertions(+), 8 deletions(-) diff --git a/src/tasks/tasks.hpp b/src/tasks/tasks.hpp index 918c799e4313..c3572c547785 100644 --- a/src/tasks/tasks.hpp +++ b/src/tasks/tasks.hpp @@ -14,6 +14,7 @@ #ifndef TASKS_TASKS_HPP_ #define TASKS_TASKS_HPP_ +#include #include #include #include @@ -21,10 +22,11 @@ #include #include #include +#include #include -#include "thread_pool.hpp" #include +#include "thread_pool.hpp" namespace parthenon { @@ -35,7 +37,7 @@ class TaskQualifier { public: using qualifier_t = uint64_t; TaskQualifier() = delete; - TaskQualifier(const qualifier_t n) : flags(n) {} + TaskQualifier(const qualifier_t n) : flags(n) {} // NOLINT(runtime/explicit) static inline constexpr qualifier_t normal{0}; static inline constexpr qualifier_t local_sync{1 << 0}; @@ -62,7 +64,7 @@ class TaskID { public: TaskID() : task(nullptr) {} // pointers to Task are implicitly convertible to TaskID - TaskID(Task *t) : task(t) {} + TaskID(Task *t) : task(t) {} // NOLINT(runtime/explicit) TaskID operator|(const TaskID &other) const { // calling this operator means you're building a TaskID to hold a dependency @@ -381,7 +383,7 @@ class TaskList { class TaskRegion { public: TaskRegion() = delete; - TaskRegion(const int num_lists) : task_lists(num_lists) { + explicit TaskRegion(const int num_lists) : task_lists(num_lists) { for (int i = 0; i < num_lists; i++) task_lists[i].SetID(i); } diff --git a/tst/style/cpplint.py b/tst/style/cpplint.py index 4df4a7d26033..c2c402f46295 100755 --- a/tst/style/cpplint.py +++ b/tst/style/cpplint.py @@ -7026,11 +7026,11 @@ def FlagCxx11Features(filename, clean_lines, linenum, error): # Flag unapproved C++11 headers. if include and include.group(1) in ( "cfenv", - "condition_variable", + # "condition_variable", "fenv.h", - "future", - "mutex", - "thread", + # "future", + # "mutex", + # "thread", # "chrono", "ratio", # "regex", From d602a354c05045207cc8d7d013d0d19d91deb3e7 Mon Sep 17 00:00:00 2001 From: jdolence Date: Thu, 14 Dec 2023 13:46:58 -0700 Subject: [PATCH 07/31] and more style --- src/tasks/thread_pool.hpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/tasks/thread_pool.hpp b/src/tasks/thread_pool.hpp index 8254efd7d44a..c7154a20b307 100644 --- a/src/tasks/thread_pool.hpp +++ b/src/tasks/thread_pool.hpp @@ -27,7 +27,7 @@ namespace parthenon { template class Queue { public: - Queue(const int num_workers) : nworkers(num_workers), nwaiting(0) {} + explicit Queue(const int num_workers) : nworkers(num_workers), nwaiting(0) {} void push(T q) { std::lock_guard lock(mutex); queue.push(q); From 10a67f14c5821a596fc5679d31aef32f8a456ec4 Mon Sep 17 00:00:00 2001 From: jdolence Date: Thu, 14 Dec 2023 13:48:30 -0700 Subject: [PATCH 08/31] ok thats enough --- src/tasks/tasks.hpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/tasks/tasks.hpp b/src/tasks/tasks.hpp index c3572c547785..670c112c4e22 100644 --- a/src/tasks/tasks.hpp +++ b/src/tasks/tasks.hpp @@ -26,6 +26,7 @@ #include #include + #include "thread_pool.hpp" namespace parthenon { From 23803d0a6caa296fa7de42211db837a5713c9b76 Mon Sep 17 00:00:00 2001 From: jdolence Date: Thu, 14 Dec 2023 13:49:53 -0700 Subject: [PATCH 09/31] actually remove the old task stuff --- src/CMakeLists.txt | 5 +- src/tasks/task_id.cpp | 157 ------------ src/tasks/task_id.hpp | 51 ---- src/tasks/task_list.hpp | 538 --------------------------------------- src/tasks/task_types.hpp | 102 -------- 5 files changed, 1 insertion(+), 852 deletions(-) delete mode 100644 src/tasks/task_id.cpp delete mode 100644 src/tasks/task_id.hpp delete mode 100644 src/tasks/task_list.hpp delete mode 100644 src/tasks/task_types.hpp diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 0dd7594f4ed8..077d05f6a117 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -214,10 +214,7 @@ add_library(parthenon solvers/solver_utils.hpp tasks/tasks.hpp - #tasks/task_id.cpp - #tasks/task_id.hpp - #tasks/task_list.hpp - #tasks/task_types.hpp + tasks/thread_pool.hpp time_integration/butcher_integrator.cpp time_integration/low_storage_integrator.cpp diff --git a/src/tasks/task_id.cpp b/src/tasks/task_id.cpp deleted file mode 100644 index 70bacbea8f36..000000000000 --- a/src/tasks/task_id.cpp +++ /dev/null @@ -1,157 +0,0 @@ -//======================================================================================== -// Athena++ astrophysical MHD code -// Copyright(C) 2014 James M. Stone and other code contributors -// Licensed under the 3-clause BSD License, see LICENSE file for details -//======================================================================================== -// (C) (or copyright) 2020. Triad National Security, LLC. All rights reserved. -// -// This program was produced under U.S. Government contract 89233218CNA000001 for Los -// Alamos National Laboratory (LANL), which is operated by Triad National Security, LLC -// for the U.S. Department of Energy/National Nuclear Security Administration. All rights -// in the program are reserved by Triad National Security, LLC, and the U.S. Department -// of Energy/National Nuclear Security Administration. The Government is granted for -// itself and others acting on its behalf a nonexclusive, paid-up, irrevocable worldwide -// license in this material to reproduce, prepare derivative works, distribute copies to -// the public, perform publicly and display publicly, and to permit others to do so. -//======================================================================================== -//! \file tasks.cpp -// \brief implementation of the TaskID class - -#include "tasks/task_id.hpp" - -#include -#include -#include -#include -#include - -namespace parthenon { - -// TaskID constructor. Default id = 0. -TaskID::TaskID(int id) { Set(id); } - -void TaskID::Set(int id) { - if (id < 0) throw std::invalid_argument("TaskID requires integer arguments >= 0"); - if (id == 0) { - bitblocks.resize(1); - return; - } - id--; - const int n_myblocks = id / BITBLOCK + 1; - // grow if necessary. never shrink - if (n_myblocks > bitblocks.size()) bitblocks.resize(n_myblocks); - bitblocks[n_myblocks - 1].set(id % BITBLOCK); -} - -void TaskID::clear() { - for (auto &bset : bitblocks) { - bset.reset(); - } -} - -bool TaskID::CheckDependencies(const TaskID &rhs) const { - const int n_myblocks = bitblocks.size(); - const int n_srcblocks = rhs.bitblocks.size(); - if (n_myblocks == n_srcblocks) { - for (int i = 0; i < n_myblocks; i++) { - if ((bitblocks[i] & rhs.bitblocks[i]) != rhs.bitblocks[i]) return false; - } - } else if (n_myblocks > n_srcblocks) { - for (int i = 0; i < n_srcblocks; i++) { - if ((bitblocks[i] & rhs.bitblocks[i]) != rhs.bitblocks[i]) return false; - } - } else { - for (int i = 0; i < n_myblocks; i++) { - if ((bitblocks[i] & rhs.bitblocks[i]) != rhs.bitblocks[i]) return false; - } - for (int i = n_myblocks; i < n_srcblocks; i++) { - if (rhs.bitblocks[i].any()) return false; - } - } - return true; -} - -void TaskID::SetFinished(const TaskID &rhs) { - const int n_myblocks = bitblocks.size(); - const int n_srcblocks = rhs.bitblocks.size(); - if (n_myblocks == n_srcblocks) { - for (int i = 0; i < n_myblocks; i++) { - bitblocks[i] ^= rhs.bitblocks[i]; - } - } else if (n_myblocks > n_srcblocks) { - for (int i = 0; i < n_srcblocks; i++) { - bitblocks[i] ^= rhs.bitblocks[i]; - } - } else { - for (int i = 0; i < n_myblocks; i++) { - bitblocks[i] ^= rhs.bitblocks[i]; - } - for (int i = n_myblocks; i < n_srcblocks; i++) { - bitblocks.push_back(rhs.bitblocks[i]); - } - } -} - -bool TaskID::operator==(const TaskID &rhs) const { - const int n_myblocks = bitblocks.size(); - const int n_srcblocks = rhs.bitblocks.size(); - if (n_myblocks == n_srcblocks) { - for (int i = 0; i < n_myblocks; i++) { - if (bitblocks[i] != rhs.bitblocks[i]) return false; - } - } else if (n_myblocks > n_srcblocks) { - for (int i = 0; i < n_srcblocks; i++) { - if (bitblocks[i] != rhs.bitblocks[i]) return false; - } - for (int i = n_srcblocks; i < n_myblocks; i++) { - if (bitblocks[i].any()) return false; - } - } else { - for (int i = 0; i < n_myblocks; i++) { - if (bitblocks[i] != rhs.bitblocks[i]) return false; - } - for (int i = n_myblocks; i < n_srcblocks; i++) { - if (rhs.bitblocks[i].any()) return false; - } - } - return true; -} - -bool TaskID::operator!=(const TaskID &rhs) const { return !operator==(rhs); } - -TaskID TaskID::operator|(const TaskID &rhs) const { - TaskID res; - const int n_myblocks = bitblocks.size(); - const int n_srcblocks = rhs.bitblocks.size(); - res.bitblocks.resize(std::max(n_myblocks, n_srcblocks)); - if (n_myblocks == n_srcblocks) { - for (int i = 0; i < n_myblocks; i++) { - res.bitblocks[i] = bitblocks[i] | rhs.bitblocks[i]; - } - } else if (n_myblocks > n_srcblocks) { - for (int i = 0; i < n_srcblocks; i++) { - res.bitblocks[i] = bitblocks[i] | rhs.bitblocks[i]; - } - for (int i = n_srcblocks; i < n_myblocks; i++) { - res.bitblocks[i] = bitblocks[i]; - } - } else { - for (int i = 0; i < n_myblocks; i++) { - res.bitblocks[i] = bitblocks[i] | rhs.bitblocks[i]; - } - for (int i = n_myblocks; i < n_srcblocks; i++) { - res.bitblocks[i] = rhs.bitblocks[i]; - } - } - return res; -} - -std::string TaskID::to_string() const { - std::string bs; - for (int i = bitblocks.size() - 1; i >= 0; i--) { - bs += bitblocks[i].to_string(); - } - return bs; -} - -} // namespace parthenon diff --git a/src/tasks/task_id.hpp b/src/tasks/task_id.hpp deleted file mode 100644 index 54043001af66..000000000000 --- a/src/tasks/task_id.hpp +++ /dev/null @@ -1,51 +0,0 @@ -//======================================================================================== -// (C) (or copyright) 2020. Triad National Security, LLC. All rights reserved. -// -// This program was produced under U.S. Government contract 89233218CNA000001 for Los -// Alamos National Laboratory (LANL), which is operated by Triad National Security, LLC -// for the U.S. Department of Energy/National Nuclear Security Administration. All rights -// in the program are reserved by Triad National Security, LLC, and the U.S. Department -// of Energy/National Nuclear Security Administration. The Government is granted for -// itself and others acting on its behalf a nonexclusive, paid-up, irrevocable worldwide -// license in this material to reproduce, prepare derivative works, distribute copies to -// the public, perform publicly and display publicly, and to permit others to do so. -//======================================================================================== - -#ifndef TASKS_TASK_ID_HPP_ -#define TASKS_TASK_ID_HPP_ - -#include -#include -#include - -#include "basic_types.hpp" - -namespace parthenon { - -//---------------------------------------------------------------------------------------- -//! \class TaskID -// \brief generalization of bit fields for Task IDs, status, and dependencies. - -#define BITBLOCK 16 - -class TaskID { - public: - TaskID() { Set(0); } - explicit TaskID(int id); - - void Set(int id); - void clear(); - bool CheckDependencies(const TaskID &rhs) const; - void SetFinished(const TaskID &rhs); - bool operator==(const TaskID &rhs) const; - bool operator!=(const TaskID &rhs) const; - TaskID operator|(const TaskID &rhs) const; - std::string to_string() const; - - private: - std::vector> bitblocks; -}; - -} // namespace parthenon - -#endif // TASKS_TASK_ID_HPP_ diff --git a/src/tasks/task_list.hpp b/src/tasks/task_list.hpp deleted file mode 100644 index 322d1a788d70..000000000000 --- a/src/tasks/task_list.hpp +++ /dev/null @@ -1,538 +0,0 @@ -//======================================================================================== -// (C) (or copyright) 2023. Triad National Security, LLC. All rights reserved. -// -// This program was produced under U.S. Government contract 89233218CNA000001 for Los -// Alamos National Laboratory (LANL), which is operated by Triad National Security, LLC -// for the U.S. Department of Energy/National Nuclear Security Administration. All rights -// in the program are reserved by Triad National Security, LLC, and the U.S. Department -// of Energy/National Nuclear Security Administration. The Government is granted for -// itself and others acting on its behalf a nonexclusive, paid-up, irrevocable worldwide -// license in this material to reproduce, prepare derivative works, distribute copies to -// the public, perform publicly and display publicly, and to permit others to do so. -//======================================================================================== - -#ifndef TASKS_TASK_LIST_HPP_ -#define TASKS_TASK_LIST_HPP_ - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include "basic_types.hpp" -#include "task_id.hpp" -#include "task_types.hpp" -#include "utils/error_checking.hpp" -#include "utils/reductions.hpp" - -namespace parthenon { - -enum class TaskListStatus { running, stuck, complete, nothing_to_do }; - -class TaskList; -namespace task_list_impl { -TaskID AddTaskHelper(TaskList *, Task); -} // namespace task_list_impl - -class IterativeTasks { - public: - IterativeTasks() = default; - IterativeTasks(TaskList *tl, int key) : tl_(tl), key_(key) { - max_iterations_ = std::numeric_limits::max(); - } - - // overload to add member functions of class T to task list - // NOTE: we must capture the object pointer - template - TaskID AddTask(TaskID const &dep, TaskStatus (T::*func)(Args1...), U *obj, - Args2 &&...args) { - return this->AddTask_(TaskType::iterative, 1, dep, [=]() mutable -> TaskStatus { - return (obj->*func)(std::forward(args)...); - }); - } - - template - TaskID AddTask(TaskID const &dep, T &&func, Args &&...args) { - return AddTask_(TaskType::iterative, 1, dep, std::forward(func), - std::forward(args)...); - } - - template - TaskID SetCompletionTask(TaskID const &dep, TaskStatus (T::*func)(Args...), U *obj, - Args &&...args) { - return AddTask_(TaskType::completion_criteria, check_interval_, dep, - [=]() mutable -> TaskStatus { - return (obj->*func)(std::forward(args)...); - }); - } - - template - TaskID SetCompletionTask(TaskID const &dep, T &&func, Args &&...args) { - return AddTask_(TaskType::completion_criteria, check_interval_, dep, - std::forward(func), std::forward(args)...); - } - - void SetMaxIterations(const int max) { - assert(max > 0); - max_iterations_ = max; - } - void SetCheckInterval(const int chk) { - assert(chk > 0); - check_interval_ = chk; - } - void SetFailWithMaxIterations(const bool flag) { throw_with_max_iters_ = flag; } - void SetWarnWithMaxIterations(const bool flag) { warn_with_max_iters_ = flag; } - bool ShouldThrowWithMax() const { return throw_with_max_iters_; } - bool ShouldWarnWithMax() const { return warn_with_max_iters_; } - int GetMaxIterations() const { return max_iterations_; } - int GetIterationCount() const { return count_; } - void IncrementCount() { count_++; } - void ResetCount() { count_ = 0; } - void PrintList() { std::cout << "tl_ = " << tl_ << std::endl; } - - private: - template - TaskID AddTask_(const TaskType &type, const int interval, TaskID const &dep, F &&func, - Args &&...args) { - TaskID id(0); - id = task_list_impl::AddTaskHelper( - tl_, Task( - id, dep, - [=, func = std::forward(func)]() mutable -> TaskStatus { - return func(std::forward(args)...); - }, - type, key_)); - return id; - } - TaskList *tl_; - int key_; - int max_iterations_; - unsigned int count_ = 0; - int check_interval_ = 1; - bool throw_with_max_iters_ = false; - bool warn_with_max_iters_ = true; -}; - -class TaskList { - public: - TaskList() = default; - bool IsComplete() { return task_list_.empty(); } - int Size() { return task_list_.size(); } - void MarkRegional(const TaskID &id) { - for (auto &task : task_list_) { - if (task.GetID() == id) { - task.SetRegional(); - break; - } - } - } - void MarkTaskComplete(const TaskID &id) { tasks_completed_.SetFinished(id); } - bool CheckDependencies(const TaskID &id) const { - return tasks_completed_.CheckDependencies(id); - } - bool CheckTaskRan(TaskID id) const { - for (auto &task : task_list_) { - if (task.GetID() == id) { - return (task.GetStatus() != TaskStatus::incomplete && - task.GetStatus() != TaskStatus::skip && - task.GetStatus() != TaskStatus::waiting); - } - } - return false; - } - bool CheckStatus(const TaskID &id, TaskStatus status) const { - for (auto &task : task_list_) { - if (task.GetID() == id) return (task.GetStatus() == status); - } - return true; - } - bool CheckTaskCompletion(const TaskID &id) const { - return CheckStatus(id, TaskStatus::complete); - } - void ClearComplete() { - auto task = task_list_.begin(); - while (task != task_list_.end()) { - if (task->GetStatus() == TaskStatus::complete && - task->GetType() != TaskType::iterative && - task->GetType() != TaskType::completion_criteria && !task->IsRegional()) { - task = task_list_.erase(task); - } else { - ++task; - } - } - std::set completed_iters; - for (auto &tsk : task_list_) { - if (tsk.GetType() == TaskType::completion_criteria && - tsk.GetStatus() == TaskStatus::complete && !tsk.IsRegional()) { - completed_iters.insert(tsk.GetKey()); - } - } - for (const auto &key : completed_iters) { - ClearIteration(key); - } - } - void ClearIteration(const int key) { - auto task = task_list_.begin(); - while (task != task_list_.end()) { - if (task->GetKey() == key) { - task = task_list_.erase(task); - } else { - ++task; - } - } - iter_tasks[key].ResetCount(); - } - void ResetIteration(const int key) { - PARTHENON_REQUIRE_THROWS(key < iter_tasks.size(), "Invalid iteration key"); - iter_tasks[key].IncrementCount(); - if (iter_tasks[key].GetIterationCount() == iter_tasks[key].GetMaxIterations()) { - if (iter_tasks[key].ShouldThrowWithMax()) { - PARTHENON_THROW("Iteration " + iter_labels[key] + - " reached maximum allowed cycles without convergence."); - } - if (iter_tasks[key].ShouldWarnWithMax()) { - PARTHENON_WARN("Iteration " + iter_labels[key] + - " reached maximum allowed cycles without convergence."); - } - for (auto &task : task_list_) { - if (task.GetKey() == key && task.GetType() == TaskType::completion_criteria) { - MarkTaskComplete(task.GetID()); - } - } - ClearIteration(key); - return; - } - for (auto &task : task_list_) { - if (task.GetKey() == key) { - if (CheckDependencies(task.GetID())) { - MarkTaskComplete(task.GetID()); - } - task.SetStatus(TaskStatus::incomplete); - } - } - } - void ResetIfNeeded(const TaskID &id) { - for (auto &task : task_list_) { - if (task.GetID() == id) { - if (task.GetType() == TaskType::completion_criteria) { - ResetIteration(task.GetKey()); - } - break; - } - } - } - bool CompleteIfNeeded(const TaskID &id) { - MarkTaskComplete(id); - auto task = task_list_.begin(); - while (task != task_list_.end()) { - if (task->GetID() == id) { - if (task->GetType() == TaskType::completion_criteria) { - ClearIteration(task->GetKey()); - return true; - } else if (task->GetType() == TaskType::single) { - task = task_list_.erase(task); - } else { - task->SetStatus(TaskStatus::waiting); - } - break; - } else { - ++task; - } - } - return false; - } - void DoAvailable() { - auto task = task_list_.begin(); - while (task != task_list_.end()) { - // first skip task if it's complete. Possible for iterative tasks - if (task->GetStatus() != TaskStatus::incomplete) { - ++task; - continue; - } - auto dep = task->GetDependency(); - if (CheckDependencies(dep)) { - (*task)(); - if (task->GetStatus() == TaskStatus::complete && !task->IsRegional()) { - MarkTaskComplete(task->GetID()); - } else if (task->GetStatus() == TaskStatus::skip && - task->GetType() == TaskType::completion_criteria) { - ResetIteration(task->GetKey()); - } else if (task->GetStatus() == TaskStatus::iterate && !task->IsRegional()) { - ResetIteration(task->GetKey()); - } - } - ++task; - } - ClearComplete(); - } - bool Validate() const { - std::set iters; - for (auto &task : task_list_) { - if (task.GetType() == TaskType::iterative) iters.insert(task.GetKey()); - } - int num_iters = iters.size(); - int found = 0; - for (auto &iter : iters) { - for (auto &task : task_list_) { - if (task.GetType() == TaskType::completion_criteria && task.GetKey() == iter) { - found++; - break; - } - } - } - bool valid = (found == num_iters); - PARTHENON_REQUIRE_THROWS( - valid, - "Task list validation found iterative tasks without a completion criteria"); - return valid; - } - - TaskID AddTask(Task &tsk) { - TaskID id(tasks_added_ + 1); - tsk.SetID(id); - task_list_.push_back(std::move(tsk)); - tasks_added_++; - return id; - } - - // overload to add member functions of class T to task list - // NOTE: we must capture the object pointer - template - TaskID AddTask(TaskID const &dep, TaskStatus (T::*func)(Args1...), U *obj, - Args2 &&...args) { - return this->AddTask(dep, [=]() mutable -> TaskStatus { - return (obj->*func)(std::forward(args)...); - }); - } - - template - TaskID AddTask(TaskID const &dep, F &&func, Args &&...args) { - TaskID id(tasks_added_ + 1); - task_list_.push_back( - Task(id, dep, [=, func = std::forward(func)]() mutable -> TaskStatus { - return func(std::forward(args)...); - })); - tasks_added_++; - return id; - } - - IterativeTasks &AddIteration(const std::string &label) { - int key = iter_tasks.size(); - iter_tasks[key] = IterativeTasks(this, key); - iter_labels[key] = label; - return iter_tasks[key]; - } - - void Print() { - int i = 0; - std::cout << "TaskList::Print():" << std::endl; - for (auto &t : task_list_) { - std::cout << " " << i << " " << t.GetID().to_string() << " " - << t.GetDependency().to_string() << " " << tasks_completed_.to_string() - << " " << (t.GetStatus() == TaskStatus::incomplete) - << (t.GetStatus() == TaskStatus::complete) - << (t.GetStatus() == TaskStatus::skip) - << (t.GetStatus() == TaskStatus::iterate) - << (t.GetStatus() == TaskStatus::fail) << std::endl; - - i++; - } - } - - protected: - std::map iter_tasks; - std::map iter_labels; - std::list task_list_; - int tasks_added_ = 0; - TaskID tasks_completed_; -}; - -namespace task_list_impl { -// helper function to avoid having to call a member function of TaskList from -// IterativeTasks before TaskList has been defined -inline TaskID AddTaskHelper(TaskList *tl, Task tsk) { return tl->AddTask(tsk); } -} // namespace task_list_impl - -class RegionCounter { - public: - explicit RegionCounter(const std::string &base) : base_(base), cnt_(0) {} - std::string ID() { return base_ + std::to_string(cnt_++); } - - private: - const std::string base_; - int cnt_; -}; - -class TaskRegion { - public: - explicit TaskRegion(const int size) : lists(size) {} - void AddRegionalDependencies(const int reg_dep_id, const int list_index, - const TaskID &id) { - AddRegionalDependencies(std::to_string(reg_dep_id), list_index, id); - } - void AddRegionalDependencies(const std::string ®_dep_id, const int list_index, - const TaskID &id) { - AddDependencies(reg_dep_id, list_index, id); - global[reg_dep_id] = false; - } - void AddGlobalDependencies(const int reg_dep_id, const int list_index, - const TaskID &id) { - AddGlobalDependencies(std::to_string(reg_dep_id), list_index, id); - } - void AddGlobalDependencies(const std::string ®_dep_id, const int list_index, - const TaskID &id) { - AddDependencies(reg_dep_id, list_index, id); - global[reg_dep_id] = true; - } - - TaskList &operator[](int i) { return lists[i]; } - - int size() const { return lists.size(); } - - bool Execute() { - for (auto i = 0; i < lists.size(); ++i) { - if (!lists[i].IsComplete()) { - lists[i].DoAvailable(); - } - } - return CheckAndUpdate(); - } - - bool CheckAndUpdate() { - auto it = id_for_reg.begin(); - while (it != id_for_reg.end()) { - auto ®_id = it->first; - bool check = false; - if (HasRun(reg_id) && !all_done[reg_id].active) { - all_done[reg_id].val = IsComplete(reg_id); - if (global[reg_id]) { - all_done[reg_id].StartReduce(MPI_MIN); - } else { - check = true; - } - } - if (global[reg_id] && all_done[reg_id].active) { - auto status = all_done[reg_id].CheckReduce(); - if (status == TaskStatus::complete) { - check = true; - } - } - if (check) { - if (all_done[reg_id].val) { - bool clear = false; - for (auto &lst : it->second) { - clear = lists[lst.first].CompleteIfNeeded(lst.second); - } - if (clear) { - all_done.erase(reg_id); - global.erase(reg_id); - it = id_for_reg.erase(it); - } else { - ++it; - } - } else { - for (auto &lst : it->second) { - lists[lst.first].ResetIfNeeded(lst.second); - } - all_done[reg_id].val = 0; - ++it; - } - } else { - ++it; - } - } - int complete_cnt = 0; - const int num_lists = size(); - for (auto i = 0; i < num_lists; ++i) { - if (lists[i].IsComplete()) complete_cnt++; - } - return (complete_cnt == num_lists); - } - - bool Validate() const { - for (auto &list : lists) { - if (!list.Validate()) return false; - } - return true; - } - - private: - void AddDependencies(const std::string &label, const int list_id, const TaskID &tid) { - id_for_reg[label][list_id] = tid; - lists[list_id].MarkRegional(tid); - all_done[label].val = 0; - } - bool HasRun(const std::string ®_id) { - auto &lvec = id_for_reg[reg_id]; - int n_to_run = lvec.size(); - int n_ran = 0; - for (auto &pair : lvec) { - int list_index = pair.first; - TaskID id = pair.second; - if (lists[list_index].CheckTaskRan(id)) { - n_ran++; - } - } - return n_ran == n_to_run; - } - bool IsComplete(const std::string ®_id) { - auto &lvec = id_for_reg[reg_id]; - int n_to_finish = lvec.size(); - int n_finished = 0; - for (auto &pair : lvec) { - int list_index = pair.first; - TaskID id = pair.second; - if (lists[list_index].CheckTaskCompletion(id)) { - n_finished++; - } - } - return n_finished == n_to_finish; - } - - std::unordered_map> id_for_reg; - std::vector lists; - std::unordered_map> all_done; - std::unordered_map global; -}; - -class TaskCollection { - public: - TaskCollection() = default; - TaskRegion &AddRegion(const int num_lists) { - regions.push_back(TaskRegion(num_lists)); - return regions.back(); - } - TaskListStatus Execute() { - assert(Validate()); - for (auto ®ion : regions) { - bool complete = false; - while (!complete) { - complete = region.Execute(); - } - } - return TaskListStatus::complete; - } - - private: - bool Validate() const { - for (auto ®ion : regions) { - if (!region.Validate()) return false; - } - return true; - } - - std::vector regions; -}; - -} // namespace parthenon - -#endif // TASKS_TASK_LIST_HPP_ diff --git a/src/tasks/task_types.hpp b/src/tasks/task_types.hpp deleted file mode 100644 index c3475784e50a..000000000000 --- a/src/tasks/task_types.hpp +++ /dev/null @@ -1,102 +0,0 @@ -//======================================================================================== -// (C) (or copyright) 2021. Triad National Security, LLC. All rights reserved. -// -// This program was produced under U.S. Government contract 89233218CNA000001 for Los -// Alamos National Laboratory (LANL), which is operated by Triad National Security, LLC -// for the U.S. Department of Energy/National Nuclear Security Administration. All rights -// in the program are reserved by Triad National Security, LLC, and the U.S. Department -// of Energy/National Nuclear Security Administration. The Government is granted for -// itself and others acting on its behalf a nonexclusive, paid-up, irrevocable worldwide -// license in this material to reproduce, prepare derivative works, distribute copies to -// the public, perform publicly and display publicly, and to permit others to do so. -//======================================================================================== - -#ifndef TASKS_TASK_TYPES_HPP_ -#define TASKS_TASK_TYPES_HPP_ - -#include // NOLINT [build/c++11] -#include -#include -#include -#include - -#include "basic_types.hpp" -#include "globals.hpp" - -namespace parthenon { - -enum class TaskType { single, iterative, completion_criteria }; - -class Task { - public: - Task(const TaskID &id, const TaskID &dep, std::function func) - : myid_(id), dep_(dep), type_(TaskType::single), key_(-1), func_(std::move(func)), - interval_(1) {} - Task(const TaskID &id, const TaskID &dep, std::function func, - const TaskType &type, const int key) - : myid_(id), dep_(dep), type_(type), key_(key), func_(std::move(func)), - interval_(1) { - assert(key_ >= 0); - assert(type_ != TaskType::single); - } - Task(const TaskID &id, const TaskID &dep, std::function func, - const TaskType &type, const int key, const int interval) - : myid_(id), dep_(dep), type_(type), key_(key), func_(std::move(func)), - interval_(interval) { - assert(key_ >= 0); - assert(type_ != TaskType::single); - assert(interval_ > 0); - } - void operator()() { - if (calls_ == 0) { - // on first call, set start time - start_time_ = std::chrono::high_resolution_clock::now(); - } - - calls_++; - if (calls_ % interval_ == 0) { - // set total runtime of current task, must go into Global namespace because - // functions called by the task functor don't have access to the task itself and - // they may want to check if the task has been running for too long indicating that - // it got stuck in an infinite loop - Globals::current_task_runtime_sec = - std::chrono::duration_cast( - std::chrono::high_resolution_clock::now() - start_time_) - .count() * - 1e-9; - status_ = func_(); - Globals::current_task_runtime_sec = 0.0; - } else { - status_ = TaskStatus::skip; - } - } - void SetID(const TaskID &id) { myid_ = id; } - TaskID GetID() const { return myid_; } - TaskID GetDependency() const { return dep_; } - TaskStatus GetStatus() const { return status_; } - void SetStatus(const TaskStatus &status) { status_ = status; } - TaskType GetType() const { return type_; } - int GetKey() const { return key_; } - void SetRegional() { regional_ = true; } - bool IsRegional() const { return regional_; } - - private: - TaskID myid_; - const TaskID dep_; - const TaskType type_; - const int key_; - TaskStatus status_ = TaskStatus::incomplete; - bool regional_ = false; - bool lb_time_ = false; - std::function func_; - int calls_ = 0; - const int interval_; - - // this is used to record the start time of the task so that we can check for how long - // the task been running and detect potential hangs, infinite loops, etc. - std::chrono::high_resolution_clock::time_point start_time_; -}; - -} // namespace parthenon - -#endif // TASKS_TASK_TYPES_HPP_ From a4db040ec0d58458b20933ce80588c3d23b47e42 Mon Sep 17 00:00:00 2001 From: jdolence Date: Thu, 14 Dec 2023 13:51:16 -0700 Subject: [PATCH 10/31] formatting --- src/tasks/tasks.hpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/tasks/tasks.hpp b/src/tasks/tasks.hpp index 670c112c4e22..8395c122c2e1 100644 --- a/src/tasks/tasks.hpp +++ b/src/tasks/tasks.hpp @@ -38,7 +38,7 @@ class TaskQualifier { public: using qualifier_t = uint64_t; TaskQualifier() = delete; - TaskQualifier(const qualifier_t n) : flags(n) {} // NOLINT(runtime/explicit) + TaskQualifier(const qualifier_t n) : flags(n) {} // NOLINT(runtime/explicit) static inline constexpr qualifier_t normal{0}; static inline constexpr qualifier_t local_sync{1 << 0}; @@ -65,7 +65,7 @@ class TaskID { public: TaskID() : task(nullptr) {} // pointers to Task are implicitly convertible to TaskID - TaskID(Task *t) : task(t) {} // NOLINT(runtime/explicit) + TaskID(Task *t) : task(t) {} // NOLINT(runtime/explicit) TaskID operator|(const TaskID &other) const { // calling this operator means you're building a TaskID to hold a dependency From 8b7d42ab00af5103bbaba68d20ba933bbf914a9d Mon Sep 17 00:00:00 2001 From: jdolence Date: Thu, 14 Dec 2023 13:53:09 -0700 Subject: [PATCH 11/31] maybe last style commit... --- src/tasks/thread_pool.hpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/tasks/thread_pool.hpp b/src/tasks/thread_pool.hpp index c7154a20b307..48e4909d91eb 100644 --- a/src/tasks/thread_pool.hpp +++ b/src/tasks/thread_pool.hpp @@ -19,9 +19,11 @@ namespace parthenon { #include #include #include +#include #include #include #include +#include #include template From 52f0d5a11610d723637fa993858707d51082ddab Mon Sep 17 00:00:00 2001 From: jdolence Date: Thu, 14 Dec 2023 13:58:42 -0700 Subject: [PATCH 12/31] oops, includes inside parthenon namespace --- src/tasks/thread_pool.hpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/tasks/thread_pool.hpp b/src/tasks/thread_pool.hpp index 48e4909d91eb..51800e4487b5 100644 --- a/src/tasks/thread_pool.hpp +++ b/src/tasks/thread_pool.hpp @@ -14,8 +14,6 @@ #ifndef TASKS_THREAD_POOL_HPP_ #define TASKS_THREAD_POOL_HPP_ -namespace parthenon { - #include #include #include @@ -26,6 +24,8 @@ namespace parthenon { #include #include +namespace parthenon { + template class Queue { public: From e6eb2e3e84351343b616632837378913d4a8a416 Mon Sep 17 00:00:00 2001 From: jdolence Date: Thu, 14 Dec 2023 14:10:23 -0700 Subject: [PATCH 13/31] update TaskID unit test --- tst/unit/test_taskid.cpp | 36 +++++++++++------------------------- 1 file changed, 11 insertions(+), 25 deletions(-) diff --git a/tst/unit/test_taskid.cpp b/tst/unit/test_taskid.cpp index 0844226ad48d..b1cca6fc9085 100644 --- a/tst/unit/test_taskid.cpp +++ b/tst/unit/test_taskid.cpp @@ -19,34 +19,20 @@ #include -#include "tasks/task_id.hpp" +#include "tasks/tasks.hpp" +using parthenon::Task; using parthenon::TaskID; -TEST_CASE("Just check everything", "[CheckDependencies][SetFinished][equal][or]") { +TEST_CASE("Just check everything", "[|][GetIDs][empty]") { GIVEN("Some TaskIDs") { - TaskID a(1); - TaskID b(2); - TaskID c(BITBLOCK + 1); // make sure we get a task with more than one block - TaskID complete; - - TaskID ac = (a | c); - bool should_be_false = ac.CheckDependencies(b); - bool should_be_truea = ac.CheckDependencies(a); - bool should_be_truec = ac.CheckDependencies(c); - TaskID abc = (a | b | c); - complete.SetFinished(abc); - bool equal_true = (complete == abc); - bool equal_false = (complete == ac); - - REQUIRE(should_be_false == false); - REQUIRE(should_be_truea == true); - REQUIRE(should_be_truec == true); - REQUIRE(equal_true == true); - REQUIRE(equal_false == false); - - WHEN("a negative number is passed") { - REQUIRE_THROWS_AS(a.Set(-1), std::invalid_argument); - } + Task ta,tb; + TaskID a(&ta); + TaskID b(&tb); + TaskID c = a | b; + TaskID none; + + REQUIRE(none.empty() == true); + REQUIRE(c.GetIDs().size() == 2); } } From ce7a6bb0a1b77e60d761d93be40b0c390efbeff3 Mon Sep 17 00:00:00 2001 From: jdolence Date: Thu, 14 Dec 2023 14:20:26 -0700 Subject: [PATCH 14/31] missing header --- src/tasks/tasks.hpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/tasks/tasks.hpp b/src/tasks/tasks.hpp index 8395c122c2e1..95ffdf5b60bc 100644 --- a/src/tasks/tasks.hpp +++ b/src/tasks/tasks.hpp @@ -25,6 +25,7 @@ #include #include +#include #include #include "thread_pool.hpp" From 1ddc2e0ff44a981b3bb312098681e539bb42f7f1 Mon Sep 17 00:00:00 2001 From: jdolence Date: Thu, 14 Dec 2023 17:01:48 -0700 Subject: [PATCH 15/31] port the poisson examples --- example/poisson/poisson_driver.cpp | 234 +++++++++++-------------- example/poisson_gmg/parthinput.poisson | 8 +- example/poisson_gmg/poisson_driver.cpp | 8 +- src/solvers/bicgstab_solver.hpp | 107 +++++------ src/solvers/mg_solver.hpp | 68 +++---- src/solvers/solver_utils.hpp | 38 ++-- src/tasks/tasks.hpp | 1 + tst/unit/test_taskid.cpp | 4 +- 8 files changed, 201 insertions(+), 267 deletions(-) diff --git a/example/poisson/poisson_driver.cpp b/example/poisson/poisson_driver.cpp index a94f6874ab70..3077153a18c2 100644 --- a/example/poisson/poisson_driver.cpp +++ b/example/poisson/poisson_driver.cpp @@ -81,101 +81,83 @@ TaskCollection PoissonDriver::MakeTaskCollection(BlockList_t &blocks) { //--- Demo a few reductions // pass a pointer to the variable being reduced into - auto loc_red = tl.AddTask(none, poisson_package::SumMass>, md.get(), - &total_mass.val); - // make it a regional dependency so dependent tasks can't execute until all lists do - // this - solver_region.AddRegionalDependencies(reg_dep_id, i, loc_red); - reg_dep_id++; + auto loc_red = + tl.AddTask(TaskQualifier::local_sync, none, + poisson_package::SumMass>, md.get(), &total_mass.val); auto rank_red = tl.AddTask( - none, + TaskQualifier::local_sync, none, [](int *max_rank) { *max_rank = std::max(*max_rank, Globals::my_rank); return TaskStatus::complete; }, &max_rank.val); - solver_region.AddRegionalDependencies(reg_dep_id, i, rank_red); - reg_dep_id++; // start a non-blocking MPI_Iallreduce auto start_global_reduce = - (i == 0 ? tl.AddTask(loc_red, &AllReduce::StartReduce, &total_mass, MPI_SUM) - : none); + tl.AddTask(TaskQualifier::once_per_region, loc_red, &AllReduce::StartReduce, + &total_mass, MPI_SUM); - auto start_rank_reduce = - (i == 0 ? tl.AddTask(rank_red, &Reduce::StartReduce, &max_rank, 0, MPI_MAX) - : none); + auto start_rank_reduce = tl.AddTask(TaskQualifier::once_per_region, rank_red, + &Reduce::StartReduce, &max_rank, 0, MPI_MAX); // test the reduction until it completes auto finish_global_reduce = - tl.AddTask(start_global_reduce, &AllReduce::CheckReduce, &total_mass); - solver_region.AddRegionalDependencies(reg_dep_id, i, finish_global_reduce); - reg_dep_id++; + tl.AddTask(TaskQualifier::local_sync | TaskQualifier::once_per_region, + start_global_reduce, &AllReduce::CheckReduce, &total_mass); auto finish_rank_reduce = - tl.AddTask(start_rank_reduce, &Reduce::CheckReduce, &max_rank); - solver_region.AddRegionalDependencies(reg_dep_id, i, finish_rank_reduce); - reg_dep_id++; + tl.AddTask(TaskQualifier::local_sync | TaskQualifier::once_per_region, + start_rank_reduce, &Reduce::CheckReduce, &max_rank); // notice how we must always pass a pointer to the reduction value // since tasks capture args by value, this would print zero if we just passed in // the val since the tasks that compute the value haven't actually executed yet - auto report_mass = (i == 0 && Globals::my_rank == 0 - ? tl.AddTask( - finish_global_reduce, - [](Real *mass) { - std::cout << "Total mass = " << *mass << std::endl; - return TaskStatus::complete; - }, - &total_mass.val) - : none); - auto report_rank = (i == 0 && Globals::my_rank == 0 - ? tl.AddTask( - finish_rank_reduce, - [](int *max_rank) { - std::cout << "Max rank = " << *max_rank << std::endl; - return TaskStatus::complete; - }, - &max_rank.val) - : none); + auto report_mass = tl.AddTask( + TaskQualifier::once_per_region, finish_global_reduce, + [](Real *mass) { + if (Globals::my_rank == 0) std::cout << "Total mass = " << *mass << std::endl; + return TaskStatus::complete; + }, + &total_mass.val); + auto report_rank = tl.AddTask( + TaskQualifier::once_per_region, finish_rank_reduce, + [](int *max_rank) { + if (Globals::my_rank == 0) std::cout << "Max rank = " << *max_rank << std::endl; + return TaskStatus::complete; + }, + &max_rank.val); //--- Begining of tasks related to solving the Poisson eq. auto mat_elem = tl.AddTask(none, poisson_package::SetMatrixElements>, md.get()); - auto &solver = tl.AddIteration("poisson solver"); - solver.SetMaxIterations(max_iters); - solver.SetCheckInterval(check_interval); - solver.SetFailWithMaxIterations(fail_flag); - solver.SetWarnWithMaxIterations(warn_flag); + auto [solver, solver_id] = tl.AddSublist(mat_elem, {1, max_iters}); auto start_recv = solver.AddTask(none, parthenon::StartReceiveBoundaryBuffers, md); - auto update = solver.AddTask(mat_elem, poisson_package::UpdatePhi>, + auto update = solver.AddTask(none, poisson_package::UpdatePhi>, md.get(), mdelta.get()); - auto norm = solver.AddTask(update, poisson_package::SumDeltaPhi>, - mdelta.get(), &update_norm.val); - solver_region.AddRegionalDependencies(reg_dep_id, i, norm); - reg_dep_id++; - auto start_reduce_norm = (i == 0 ? solver.AddTask(norm, &AllReduce::StartReduce, - &update_norm, MPI_SUM) - : none); + auto norm = solver.AddTask(TaskQualifier::local_sync, update, + poisson_package::SumDeltaPhi>, mdelta.get(), + &update_norm.val); + auto start_reduce_norm = + solver.AddTask(TaskQualifier::once_per_region, norm, + &AllReduce::StartReduce, &update_norm, MPI_SUM); auto finish_reduce_norm = - solver.AddTask(start_reduce_norm, &AllReduce::CheckReduce, &update_norm); - auto report_norm = (i == 0 ? solver.AddTask( - finish_reduce_norm, - [](Real *norm) { - if (Globals::my_rank == 0) { - std::cout << "Update norm = " << *norm - << std::endl; - } - *norm = 0.0; - return TaskStatus::complete; - }, - &update_norm.val) - : none); + solver.AddTask(TaskQualifier::once_per_region, start_reduce_norm, + &AllReduce::CheckReduce, &update_norm); + auto report_norm = solver.AddTask( + TaskQualifier::once_per_region, finish_reduce_norm, + [](Real *norm) { + if (Globals::my_rank == 0) { + std::cout << "Update norm = " << *norm << std::endl; + } + *norm = 0.0; + return TaskStatus::complete; + }, + &update_norm.val); auto send = solver.AddTask(update, SendBoundaryBuffers, md); @@ -183,24 +165,18 @@ TaskCollection PoissonDriver::MakeTaskCollection(BlockList_t &blocks) { auto setb = solver.AddTask(recv | update, SetBoundaries, md); - auto check = solver.SetCompletionTask( - send | setb | report_norm, poisson_package::CheckConvergence>, - md.get(), mdelta.get()); - // mark task so that dependent tasks (below) won't execute - // until all task lists have completed it - solver_region.AddRegionalDependencies(reg_dep_id, i, check); - reg_dep_id++; - - auto print = none; - if (i == 0) { // only print once - print = tl.AddTask(check, poisson_package::PrintComplete); - } + auto check = solver.AddTask( + TaskQualifier::completion | TaskQualifier::global_sync, send | setb | report_norm, + poisson_package::CheckConvergence>, md.get(), mdelta.get()); + + auto print = tl.AddTask(TaskQualifier::once_per_region, solver_id, + poisson_package::PrintComplete); //--- End of tasks related to solving the Poisson eq // do a vector reduction (everything below here), just for fun // first fill it in auto fill_vec = tl.AddTask( - none, + TaskQualifier::local_sync, none, [](std::vector *vec) { auto &v = *vec; for (int n = 0; n < v.size(); n++) @@ -208,72 +184,64 @@ TaskCollection PoissonDriver::MakeTaskCollection(BlockList_t &blocks) { return TaskStatus::complete; }, &vec_reduce.val); - solver_region.AddRegionalDependencies(reg_dep_id, i, fill_vec); - reg_dep_id++; TaskID start_vec_reduce = - (i == 0 ? tl.AddTask(fill_vec, &AllReduce>::StartReduce, - &vec_reduce, MPI_SUM) - : none); + tl.AddTask(TaskQualifier::once_per_region, fill_vec, + &AllReduce>::StartReduce, &vec_reduce, MPI_SUM); // test the reduction until it completes TaskID finish_vec_reduce = tl.AddTask( - start_vec_reduce, &AllReduce>::CheckReduce, &vec_reduce); - solver_region.AddRegionalDependencies(reg_dep_id, i, finish_vec_reduce); - reg_dep_id++; - - auto report_vec = (i == 0 && Globals::my_rank == 0 - ? tl.AddTask( - finish_vec_reduce, - [num_partitions](std::vector *vec) { - auto &v = *vec; - std::cout << "Vec reduction: "; - for (int n = 0; n < v.size(); n++) { - std::cout << v[n] << " "; - } - std::cout << std::endl; - std::cout << "Should be: "; - for (int n = 0; n < v.size(); n++) { - std::cout << n * num_partitions * Globals::nranks - << " "; - } - std::cout << std::endl; - return TaskStatus::complete; - }, - &vec_reduce.val) - : none); + TaskQualifier::once_per_region | TaskQualifier::local_sync, start_vec_reduce, + &AllReduce>::CheckReduce, &vec_reduce); + + auto report_vec = tl.AddTask( + TaskQualifier::once_per_region, finish_vec_reduce, + [num_partitions](std::vector *vec) { + if (Globals::my_rank == 0) { + auto &v = *vec; + std::cout << "Vec reduction: "; + for (int n = 0; n < v.size(); n++) { + std::cout << v[n] << " "; + } + std::cout << std::endl; + std::cout << "Should be: "; + for (int n = 0; n < v.size(); n++) { + std::cout << n * num_partitions * Globals::nranks << " "; + } + std::cout << std::endl; + } + return TaskStatus::complete; + }, + &vec_reduce.val); // And lets do a view reduce too just for fun // The views are filled in the package TaskID start_view_reduce = - (i == 0 ? tl.AddTask(none, &AllReduce>::StartReduce, - pview_reduce, MPI_SUM) - : none); + tl.AddTask(TaskQualifier::once_per_region, none, + &AllReduce>::StartReduce, pview_reduce, MPI_SUM); // test the reduction until it completes TaskID finish_view_reduce = tl.AddTask( - start_view_reduce, &AllReduce>::CheckReduce, pview_reduce); - solver_region.AddRegionalDependencies(reg_dep_id, i, finish_view_reduce); - reg_dep_id++; - - auto report_view = (i == 0 && Globals::my_rank == 0 - ? tl.AddTask( - finish_view_reduce, - [num_partitions](HostArray1D *view) { - auto &v = *view; - std::cout << "View reduction: "; - for (int n = 0; n < v.size(); n++) { - std::cout << v(n) << " "; - } - std::cout << std::endl; - std::cout << "Should be: "; - for (int n = 0; n < v.size(); n++) { - std::cout << n * num_partitions * Globals::nranks - << " "; - } - std::cout << std::endl; - return TaskStatus::complete; - }, - &(pview_reduce->val)) - : none); + TaskQualifier::once_per_region | TaskQualifier::local_sync, start_view_reduce, + &AllReduce>::CheckReduce, pview_reduce); + + auto report_view = tl.AddTask( + TaskQualifier::once_per_region, finish_view_reduce, + [num_partitions](HostArray1D *view) { + if (Globals::my_rank == 0) { + auto &v = *view; + std::cout << "View reduction: "; + for (int n = 0; n < v.size(); n++) { + std::cout << v(n) << " "; + } + std::cout << std::endl; + std::cout << "Should be: "; + for (int n = 0; n < v.size(); n++) { + std::cout << n * num_partitions * Globals::nranks << " "; + } + std::cout << std::endl; + } + return TaskStatus::complete; + }, + &(pview_reduce->val)); } return tc; diff --git a/example/poisson_gmg/parthinput.poisson b/example/poisson_gmg/parthinput.poisson index 7ec0878059ec..57f5febf871b 100644 --- a/example/poisson_gmg/parthinput.poisson +++ b/example/poisson_gmg/parthinput.poisson @@ -25,14 +25,14 @@ multigrid = true nx1 = 64 x1min = -1.0 x1max = 1.0 -ix1_bc = user -ox1_bc = user +ix1_bc = outflow +ox1_bc = outflow nx2 = 64 x2min = -1.0 x2max = 1.0 -ix2_bc = user -ox2_bc = user +ix2_bc = outflow +ox2_bc = outflow nx3 = 1 x3min = 0.0 diff --git a/example/poisson_gmg/poisson_driver.cpp b/example/poisson_gmg/poisson_driver.cpp index 784653237413..656dd3fcac87 100644 --- a/example/poisson_gmg/poisson_driver.cpp +++ b/example/poisson_gmg/poisson_driver.cpp @@ -99,11 +99,10 @@ TaskCollection PoissonDriver::MakeTaskCollection(BlockList_t &blocks) { auto zero_u = tl.AddTask(get_rhs, solvers::utils::SetToZero, md); auto solve = zero_u; - auto &itl = tl.AddIteration("Solver"); if (solver == "BiCGSTAB") { - solve = bicgstab_solver->AddTasks(tl, itl, zero_u, i, pmesh, region, reg_dep_id); + solve = bicgstab_solver->AddTasks(tl, zero_u, pmesh, i); } else if (solver == "MG") { - solve = mg_solver->AddTasks(tl, itl, zero_u, i, pmesh, region, reg_dep_id); + solve = mg_solver->AddTasks(tl, zero_u, pmesh, i); } else { PARTHENON_FAIL("Unknown solver type."); } @@ -113,8 +112,7 @@ TaskCollection PoissonDriver::MakeTaskCollection(BlockList_t &blocks) { if (use_exact_rhs) { auto diff = tl.AddTask(solve, solvers::utils::AddFieldsAndStore, md, 1.0, -1.0); - auto get_err = - solvers::utils::DotProduct(diff, region, tl, i, reg_dep_id, &err, md); + auto get_err = solvers::utils::DotProduct(diff, tl, &err, md); tl.AddTask( get_err, [](PoissonDriver *driver, int partition) { diff --git a/src/solvers/bicgstab_solver.hpp b/src/solvers/bicgstab_solver.hpp index 90fefeb55ad7..fa3a9837185e 100644 --- a/src/solvers/bicgstab_solver.hpp +++ b/src/solvers/bicgstab_solver.hpp @@ -78,10 +78,10 @@ class BiCGSTABSolver { pkg->AddField(p::name(), m_no_ghost); } - TaskID AddTasks(TaskList &tl, IterativeTasks &itl, TaskID dependence, int i, - Mesh *pmesh, TaskRegion ®ion, int ®_dep_id) { + TaskID AddTasks(TaskList &tl, TaskID dependence, Mesh *pmesh, const int partition) { using namespace utils; - auto &md = pmesh->mesh_data.GetOrAdd("base", i); + TaskID none; + auto &md = pmesh->mesh_data.GetOrAdd("base", partition); iter_counter = 0; // Initialization: x <- 0, r <- rhs, rhat0 <- rhs, @@ -91,12 +91,11 @@ class BiCGSTABSolver { auto copy_r = tl.AddTask(dependence, CopyData, md); auto copy_p = tl.AddTask(dependence, CopyData, md); auto copy_rhat0 = tl.AddTask(dependence, CopyData, md); - auto get_rhat0r_init = - DotProduct(dependence, region, tl, i, reg_dep_id, &rhat0r, md); + auto get_rhat0r_init = DotProduct(dependence, tl, &rhat0r, md); auto initialize = tl.AddTask( + TaskQualifier::once_per_region | TaskQualifier::local_sync, zero_x | zero_u_init | copy_r | copy_p | copy_rhat0 | get_rhat0r_init, - [](BiCGSTABSolver *solver, int partition) { - if (partition != 0) return TaskStatus::complete; + [](BiCGSTABSolver *solver) { solver->rhat0r_old = solver->rhat0r.val; solver->rhat0r.val = 0.0; solver->rhat0v.val = 0.0; @@ -105,28 +104,25 @@ class BiCGSTABSolver { solver->residual.val = 0.0; return TaskStatus::complete; }, - this, i); - region.AddRegionalDependencies(reg_dep_id, i, initialize); - reg_dep_id++; - if (i == 0) { - tl.AddTask(dependence, [&]() { - if (Globals::my_rank == 0) - printf("# [0] v-cycle\n# [1] rms-residual\n# [2] rms-error\n"); - return TaskStatus::complete; - }); - } + this); + tl.AddTask(TaskQualifier::once_per_region, dependence, [&]() { + if (Globals::my_rank == 0) + printf("# [0] v-cycle\n# [1] rms-residual\n# [2] rms-error\n"); + return TaskStatus::complete; + }); // BEGIN ITERATIVE TASKS + auto [itl, solver_id] = tl.AddSublist(initialize, {1, params_.max_iters}); // 1. u <- M p - auto precon1 = initialize; + auto precon1 = none; if (params_.precondition) { auto set_rhs = itl.AddTask(precon1, CopyData, md); auto zero_u = itl.AddTask(precon1, SetToZero, md); - precon1 = preconditioner.AddLinearOperatorTasks(region, itl, set_rhs | zero_u, i, - reg_dep_id, pmesh); + precon1 = + preconditioner.AddLinearOperatorTasks(itl, set_rhs | zero_u, partition, pmesh); } else { - precon1 = itl.AddTask(initialize, CopyData, md); + precon1 = itl.AddTask(none, CopyData, md); } // 2. v <- A u @@ -134,8 +130,7 @@ class BiCGSTABSolver { auto get_v = eqs_.template Ax(itl, comm, md); // 3. rhat0v <- (rhat0, v) - auto get_rhat0v = - DotProduct(get_v, region, itl, i, reg_dep_id, &rhat0v, md); + auto get_rhat0v = DotProduct(get_v, itl, &rhat0v, md); // 4. h <- x + alpha u (alpha = rhat0r_old / rhat0v) auto correct_h = itl.AddTask( @@ -156,26 +151,25 @@ class BiCGSTABSolver { this, md); // Check and print out residual - auto get_res = DotProduct(correct_s, region, itl, i, reg_dep_id, &residual, md); + auto get_res = DotProduct(correct_s, itl, &residual, md); auto print = itl.AddTask( - get_res, - [&](BiCGSTABSolver *solver, Mesh *pmesh, int partition) { - if (partition != 0) return TaskStatus::complete; + TaskQualifier::once_per_region, get_res, + [&](BiCGSTABSolver *solver, Mesh *pmesh) { Real rms_res = std::sqrt(solver->residual.val / pmesh->GetTotalCells()); if (Globals::my_rank == 0) printf("%i %e\n", solver->iter_counter * 2 + 1, rms_res); return TaskStatus::complete; }, - this, pmesh, i); + this, pmesh); // 6. u <- M s auto precon2 = correct_s; if (params_.precondition) { auto set_rhs = itl.AddTask(precon2, CopyData, md); auto zero_u = itl.AddTask(precon2, SetToZero, md); - precon2 = preconditioner.AddLinearOperatorTasks(region, itl, set_rhs | zero_u, i, - reg_dep_id, pmesh); + precon2 = + preconditioner.AddLinearOperatorTasks(itl, set_rhs | zero_u, partition, pmesh); } else { precon2 = itl.AddTask(precon2, CopyData, md); } @@ -185,12 +179,12 @@ class BiCGSTABSolver { auto get_t = eqs_.template Ax(itl, pre_t_comm, md); // 8. omega <- (t,s) / (t,t) - auto get_ts = DotProduct(get_t, region, itl, i, reg_dep_id, &ts, md); - auto get_tt = DotProduct(get_t, region, itl, i, reg_dep_id, &tt, md); + auto get_ts = DotProduct(get_t, itl, &ts, md); + auto get_tt = DotProduct(get_t, itl, &tt, md); // 9. x <- h + omega u auto correct_x = itl.AddTask( - get_tt | get_ts, + TaskQualifier::local_sync, get_tt | get_ts, [](BiCGSTABSolver *solver, std::shared_ptr> &md) { Real omega = solver->ts.val / solver->tt.val; return AddFieldsAndStore(md, 1.0, omega); @@ -207,29 +201,25 @@ class BiCGSTABSolver { this, md); // Check and print out residual - auto get_res2 = - DotProduct(correct_r, region, itl, i, reg_dep_id, &residual, md); - - if (i == 0) { - get_res2 = itl.AddTask( - get_res2, - [&](BiCGSTABSolver *solver, Mesh *pmesh) { - Real rms_err = std::sqrt(solver->residual.val / pmesh->GetTotalCells()); - if (Globals::my_rank == 0) - printf("%i %e\n", solver->iter_counter * 2 + 2, rms_err); - return TaskStatus::complete; - }, - this, pmesh); - } + auto get_res2 = DotProduct(correct_r, itl, &residual, md); + + get_res2 = itl.AddTask( + TaskQualifier::once_per_region, get_res2, + [&](BiCGSTABSolver *solver, Mesh *pmesh) { + Real rms_err = std::sqrt(solver->residual.val / pmesh->GetTotalCells()); + if (Globals::my_rank == 0) + printf("%i %e\n", solver->iter_counter * 2 + 2, rms_err); + return TaskStatus::complete; + }, + this, pmesh); // 11. rhat0r <- (rhat0, r) - auto get_rhat0r = - DotProduct(correct_r, region, itl, i, reg_dep_id, &rhat0r, md); + auto get_rhat0r = DotProduct(correct_r, itl, &rhat0r, md); // 12. beta <- rhat0r / rhat0r_old * alpha / omega // 13. p <- r + beta * (p - omega * v) auto update_p = itl.AddTask( - get_rhat0r | get_res2, + TaskQualifier::local_sync, get_rhat0r | get_res2, [](BiCGSTABSolver *solver, std::shared_ptr> &md) { Real alpha = solver->rhat0r_old / solver->rhat0v.val; Real omega = solver->ts.val / solver->tt.val; @@ -241,15 +231,14 @@ class BiCGSTABSolver { this, md); // 14. rhat0r_old <- rhat0r, zero all reductions - region.AddRegionalDependencies(reg_dep_id, i, update_p | correct_x); - auto check = itl.SetCompletionTask( + auto check = itl.AddTask( + TaskQualifier::completion | TaskQualifier::once_per_region | + TaskQualifier::global_sync, update_p | correct_x, - [](BiCGSTABSolver *solver, Mesh *pmesh, int partition, int max_iter, - Real res_tol) { - if (partition != 0) return TaskStatus::complete; + [](BiCGSTABSolver *solver, Mesh *pmesh, Real res_tol) { solver->iter_counter++; Real rms_res = std::sqrt(solver->residual.val / pmesh->GetTotalCells()); - if (rms_res < res_tol || solver->iter_counter >= max_iter) { + if (rms_res < res_tol) { solver->final_residual = rms_res; solver->final_iteration = solver->iter_counter; return TaskStatus::complete; @@ -262,11 +251,9 @@ class BiCGSTABSolver { solver->residual.val = 0.0; return TaskStatus::iterate; }, - this, pmesh, i, params_.max_iters, params_.residual_tolerance); - region.AddGlobalDependencies(reg_dep_id, i, check); - reg_dep_id++; + this, pmesh, params_.residual_tolerance); - return check; + return solver_id; } Real GetSquaredResidualSum() const { return residual.val; } diff --git a/src/solvers/mg_solver.hpp b/src/solvers/mg_solver.hpp index 46255b329075..e86c9aaf127b 100644 --- a/src/solvers/mg_solver.hpp +++ b/src/solvers/mg_solver.hpp @@ -82,61 +82,55 @@ class MGSolver { pkg->AddField(D::name(), mu0); } - TaskID AddTasks(TaskList & /*tl*/, IterativeTasks &itl, TaskID dependence, - int partition, Mesh *pmesh, TaskRegion ®ion, int ®_dep_id) { + TaskID AddTasks(TaskList &tl, TaskID dependence, Mesh *pmesh, const int partition) { using namespace utils; + TaskID none; + auto [itl, solve_id] = tl.AddSublist(dependence, {1, this->params_.max_iters}); iter_counter = 0; itl.AddTask( - dependence, - [](int partition, int *iter_counter) { - if (partition != 0 || *iter_counter > 0 || Globals::my_rank != 0) - return TaskStatus::complete; + TaskQualifier::once_per_region, none, + [](int *iter_counter) { + if (*iter_counter > 0 || Globals::my_rank != 0) return TaskStatus::complete; printf("# [0] v-cycle\n# [1] rms-residual\n# [2] rms-error\n"); return TaskStatus::complete; }, - partition, &iter_counter); - auto mg_finest = - AddLinearOperatorTasks(region, itl, dependence, partition, reg_dep_id, pmesh); + &iter_counter); + auto mg_finest = AddLinearOperatorTasks(itl, none, partition, pmesh); auto &md = pmesh->mesh_data.GetOrAdd("base", partition); auto calc_pointwise_res = eqs_.template Ax(itl, mg_finest, md); calc_pointwise_res = itl.AddTask( calc_pointwise_res, AddFieldsAndStoreInteriorSelect, md, 1.0, -1.0, false); - auto get_res = DotProduct(calc_pointwise_res, region, itl, - partition, reg_dep_id, &residual, md); + auto get_res = DotProduct(calc_pointwise_res, itl, &residual, md); - auto check = itl.SetCompletionTask( + auto check = itl.AddTask( + TaskQualifier::once_per_region | TaskQualifier::completion | + TaskQualifier::global_sync, get_res, - [](MGSolver *solver, int part, Mesh *pmesh) { - if (part != 0) return TaskStatus::complete; + [](MGSolver *solver, Mesh *pmesh) { solver->iter_counter++; Real rms_res = std::sqrt(solver->residual.val / pmesh->GetTotalCells()); if (Globals::my_rank == 0) printf("%i %e\n", solver->iter_counter, rms_res); - if (rms_res > solver->params_.residual_tolerance && - solver->iter_counter < solver->params_.max_iters) - return TaskStatus::iterate; + if (rms_res > solver->params_.residual_tolerance) return TaskStatus::iterate; solver->final_residual = rms_res; solver->final_iteration = solver->iter_counter; return TaskStatus::complete; }, - this, partition, pmesh); - region.AddGlobalDependencies(reg_dep_id, partition, check); - reg_dep_id++; + this, pmesh); - return check; + return solve_id; } - template - TaskID AddLinearOperatorTasks(TaskRegion ®ion, TL_t &tl, TaskID dependence, - int partition, int ®_dep_id, Mesh *pmesh) { + TaskID AddLinearOperatorTasks(TaskList &tl, TaskID dependence, int partition, + Mesh *pmesh) { using namespace utils; iter_counter = 0; int min_level = 0; int max_level = pmesh->GetGMGMaxLevel(); - return AddMultiGridTasksPartitionLevel(region, tl, dependence, partition, reg_dep_id, - max_level, min_level, max_level, pmesh); + return AddMultiGridTasksPartitionLevel(tl, dependence, partition, max_level, + min_level, max_level, pmesh); } Real GetSquaredResidualSum() const { return residual.val; } @@ -244,10 +238,9 @@ class MGSolver { return tl.AddTask(jacobi3, CopyData, md); } - template - TaskID AddMultiGridTasksPartitionLevel(TaskRegion ®ion, TL_t &tl, TaskID dependence, - int partition, int ®_dep_id, int level, - int min_level, int max_level, Mesh *pmesh) { + TaskID AddMultiGridTasksPartitionLevel(TaskList &tl, TaskID dependence, int partition, + int level, int min_level, int max_level, + Mesh *pmesh) { using namespace utils; auto smoother = params_.smoother; bool do_FAS = params_.do_FAS; @@ -278,10 +271,8 @@ class MGSolver { // Fill fields with restricted values auto recv_from_finer = tl.AddTask(dependence, ReceiveBoundBufs, md); - set_from_finer = - tl.AddTask(recv_from_finer, SetBounds, md); - region.AddRegionalDependencies(reg_dep_id, partition, set_from_finer); - reg_dep_id++; + set_from_finer = tl.AddTask( // TaskQualifier::local_sync, // is this required? + recv_from_finer, SetBounds, md); // 1. Copy residual from dual purpose communication field to the rhs, should be // actual RHS for finest level auto copy_u = tl.AddTask(set_from_finer, CopyData, md); @@ -327,19 +318,16 @@ class MGSolver { auto communicate_to_coarse = tl.AddTask(residual, SendBoundBufs, md); - auto coarser = AddMultiGridTasksPartitionLevel(region, tl, communicate_to_coarse, - partition, reg_dep_id, level - 1, - min_level, max_level, pmesh); + auto coarser = AddMultiGridTasksPartitionLevel( + tl, communicate_to_coarse, partition, level - 1, min_level, max_level, pmesh); // 6. Receive error field into communication field and prolongate auto recv_from_coarser = tl.AddTask(coarser, ReceiveBoundBufs, md); auto set_from_coarser = tl.AddTask(recv_from_coarser, SetBounds, md); - auto prolongate = tl.AddTask( + auto prolongate = tl.AddTask( // TaskQualifier::local_sync, // is this required? set_from_coarser, ProlongateBounds, md); - region.AddRegionalDependencies(reg_dep_id, partition, prolongate); - reg_dep_id++; // 7. Correct solution on this level with res_err field and store in // communication field diff --git a/src/solvers/solver_utils.hpp b/src/solvers/solver_utils.hpp index 38ab9cf17889..76f77ec7a298 100644 --- a/src/solvers/solver_utils.hpp +++ b/src/solvers/solver_utils.hpp @@ -272,32 +272,24 @@ TaskStatus DotProductLocal(const std::shared_ptr> &md, return TaskStatus::complete; } -template -TaskID DotProduct(TaskID dependency_in, TaskRegion ®ion, TL_t &tl, int partition, - int ®_dep_id, AllReduce *adotb, +template +TaskID DotProduct(TaskID dependency_in, TaskList &tl, AllReduce *adotb, const std::shared_ptr> &md) { using namespace impl; - auto zero_adotb = (partition == 0 ? tl.AddTask( - dependency_in, - [](AllReduce *r) { - r->val = 0.0; - return TaskStatus::complete; - }, - adotb) - : dependency_in); - region.AddRegionalDependencies(reg_dep_id, partition, zero_adotb); - reg_dep_id++; - auto get_adotb = tl.AddTask(zero_adotb, DotProductLocal, md, adotb); - region.AddRegionalDependencies(reg_dep_id, partition, get_adotb); - reg_dep_id++; - auto start_global_adotb = - (partition == 0 - ? tl.AddTask(get_adotb, &AllReduce::StartReduce, adotb, MPI_SUM) - : get_adotb); + auto zero_adotb = tl.AddTask( + TaskQualifier::once_per_region | TaskQualifier::local_sync, dependency_in, + [](AllReduce *r) { + r->val = 0.0; + return TaskStatus::complete; + }, + adotb); + auto get_adotb = tl.AddTask(TaskQualifier::local_sync, zero_adotb, + DotProductLocal, md, adotb); + auto start_global_adotb = tl.AddTask(TaskQualifier::once_per_region, get_adotb, + &AllReduce::StartReduce, adotb, MPI_SUM); auto finish_global_adotb = - tl.AddTask(start_global_adotb, &AllReduce::CheckReduce, adotb); - region.AddRegionalDependencies(reg_dep_id, partition, finish_global_adotb); - reg_dep_id++; + tl.AddTask(TaskQualifier::once_per_region | TaskQualifier::local_sync, + start_global_adotb, &AllReduce::CheckReduce, adotb); return finish_global_adotb; } diff --git a/src/tasks/tasks.hpp b/src/tasks/tasks.hpp index 95ffdf5b60bc..9cc5c7c38396 100644 --- a/src/tasks/tasks.hpp +++ b/src/tasks/tasks.hpp @@ -96,6 +96,7 @@ class TaskID { class Task { public: + Task() = default; template Task(TID &&dep, const std::function &func, std::pair limits = {1, 1}) diff --git a/tst/unit/test_taskid.cpp b/tst/unit/test_taskid.cpp index b1cca6fc9085..14dcc09b500a 100644 --- a/tst/unit/test_taskid.cpp +++ b/tst/unit/test_taskid.cpp @@ -24,9 +24,9 @@ using parthenon::Task; using parthenon::TaskID; -TEST_CASE("Just check everything", "[|][GetIDs][empty]") { +TEST_CASE("Just check everything", "[GetIDs][empty]") { GIVEN("Some TaskIDs") { - Task ta,tb; + Task ta, tb; TaskID a(&ta); TaskID b(&tb); TaskID c = a | b; From 0bd54cf6fa839ea3150aa92d53c7fa6aff7d2960 Mon Sep 17 00:00:00 2001 From: jdolence Date: Thu, 14 Dec 2023 17:21:03 -0700 Subject: [PATCH 16/31] try to fix serial builds --- src/tasks/tasks.hpp | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/src/tasks/tasks.hpp b/src/tasks/tasks.hpp index 9cc5c7c38396..c4df73777d17 100644 --- a/src/tasks/tasks.hpp +++ b/src/tasks/tasks.hpp @@ -234,6 +234,8 @@ class TaskList { } if (tq.GlobalSync()) { + bool do_mpi = false; +#ifdef MPI_PARALLEL // make status, request, and comm for this global task global_status.push_back(std::make_shared(0)); global_request.push_back(std::make_shared(MPI_REQUEST_NULL)); @@ -247,9 +249,12 @@ class TaskList { // we need another communicator to support multiple in flight non-blocking // collectives where we can't guarantee calling order across ranks MPI_Comm_dup(MPI_COMM_WORLD, global_comm.back().get()); + do_mpi = true; +#endif // MPI_PARALLEL TaskID start; // only call MPI once per region, on the list with unique_id = 0 - if (unique_id == 0) { + if (unique_id == 0 && do_mpi) { +#ifdef MPI_PARALLEL // add a task that starts the Iallreduce on the task statuses tasks.push_back(std::make_shared( id, @@ -281,6 +286,7 @@ class TaskList { return TaskStatus::incomplete; }, exec_limits)); +#endif // MPI_PARALLEL } else { // unique_id != 0 // just add empty tasks tasks.push_back(std::make_shared( @@ -334,9 +340,11 @@ class TaskList { // put these in shared_ptrs so copying TaskList works as expected std::vector> tasks; std::vector> sublists; +#ifdef MPI_PARALLEL std::vector> global_status; std::vector> global_request; std::vector> global_comm; +#endif // MPI_PARALLEL // vectors are fine for these std::vector regional_tasks; std::vector global_tasks; From 6082812e127df821f7f3d508a8450686c11a3562 Mon Sep 17 00:00:00 2001 From: Josh Dolence Date: Thu, 14 Dec 2023 17:45:00 -0700 Subject: [PATCH 17/31] clean up branching in `|` operator of TaskID Co-authored-by: Jonah Miller --- src/tasks/tasks.hpp | 14 ++++---------- 1 file changed, 4 insertions(+), 10 deletions(-) diff --git a/src/tasks/tasks.hpp b/src/tasks/tasks.hpp index c4df73777d17..85f88e8229eb 100644 --- a/src/tasks/tasks.hpp +++ b/src/tasks/tasks.hpp @@ -71,16 +71,10 @@ class TaskID { TaskID operator|(const TaskID &other) const { // calling this operator means you're building a TaskID to hold a dependency TaskID result; - if (task) { - result.dep.push_back(task); - } else { - result.dep.insert(result.dep.end(), dep.begin(), dep.end()); - } - if (other.task) { - result.dep.push_back(other.task); - } else { - result.dep.insert(result.dep.end(), other.dep.begin(), other.dep.end()); - } + if (task != nullptr) result.dep.push_back(task); + result.dep.insert(result.dep.end(), dep.begin(), dep.end()); + if (other.task != nullptr) result.dep.push_back(other.task); + result.dep.insert(result.dep.end(), other.dep.begin(), other.dep.end()); return result; } From 07ae71a5bfacdbd19227b7e1b610a90bd6deace9 Mon Sep 17 00:00:00 2001 From: jdolence Date: Thu, 14 Dec 2023 17:56:07 -0700 Subject: [PATCH 18/31] rename Queue ThreadQueue --- src/tasks/thread_pool.hpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/tasks/thread_pool.hpp b/src/tasks/thread_pool.hpp index 51800e4487b5..187325f0df7c 100644 --- a/src/tasks/thread_pool.hpp +++ b/src/tasks/thread_pool.hpp @@ -27,9 +27,9 @@ namespace parthenon { template -class Queue { +class ThreadQueue { public: - explicit Queue(const int num_workers) : nworkers(num_workers), nwaiting(0) {} + explicit ThreadQueue(const int num_workers) : nworkers(num_workers), nwaiting(0) {} void push(T q) { std::lock_guard lock(mutex); queue.push(q); @@ -129,7 +129,7 @@ class ThreadPool { private: const int nthreads; std::vector threads; - Queue> queue; + ThreadQueue> queue; }; } // namespace parthenon From c1dbcb3eec6488ae33cb47d7a7f85593883c72f6 Mon Sep 17 00:00:00 2001 From: jdolence Date: Thu, 14 Dec 2023 17:56:32 -0700 Subject: [PATCH 19/31] formatting --- src/tasks/tasks.hpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/tasks/tasks.hpp b/src/tasks/tasks.hpp index 85f88e8229eb..1425f65c40ce 100644 --- a/src/tasks/tasks.hpp +++ b/src/tasks/tasks.hpp @@ -280,7 +280,7 @@ class TaskList { return TaskStatus::incomplete; }, exec_limits)); -#endif // MPI_PARALLEL +#endif // MPI_PARALLEL } else { // unique_id != 0 // just add empty tasks tasks.push_back(std::make_shared( From fbbe02abb114280b4288908a97991ae39cacc1e2 Mon Sep 17 00:00:00 2001 From: jdolence Date: Fri, 15 Dec 2023 15:04:08 -0700 Subject: [PATCH 20/31] try to fix builds with threads --- CMakeLists.txt | 3 +++ src/CMakeLists.txt | 2 +- src/tasks/tasks.hpp | 2 +- 3 files changed, 5 insertions(+), 2 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 994399d4bdc0..0f6b0695c5e9 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -116,6 +116,9 @@ endif() list(APPEND CMAKE_MODULE_PATH "${CMAKE_CURRENT_SOURCE_DIR}/cmake") find_package(Filesystem REQUIRED COMPONENTS Experimental Final) +# Require threading for tasks +find_package(Threads) + set(ENABLE_MPI OFF) set(NUM_MPI_PROC_TESTING "4" CACHE STRING "Number of mpi processors to use when running tests with MPI") if (NOT PARTHENON_DISABLE_MPI) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 077d05f6a117..8c56edf90563 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -307,7 +307,7 @@ if (Kokkos_ENABLE_CUDA AND NOT CMAKE_CXX_COMPILER_ID STREQUAL "Clang") target_compile_options(parthenon PUBLIC --expt-relaxed-constexpr) endif() -target_link_libraries(parthenon PUBLIC Kokkos::kokkos) +target_link_libraries(parthenon PUBLIC Kokkos::kokkos Threads::Threads) if (PARTHENON_ENABLE_ASCENT) if (ENABLE_MPI) diff --git a/src/tasks/tasks.hpp b/src/tasks/tasks.hpp index 1425f65c40ce..921ccb06c0ad 100644 --- a/src/tasks/tasks.hpp +++ b/src/tasks/tasks.hpp @@ -84,7 +84,7 @@ class TaskID { Task *GetTask() { return task; } private: - Task *task; + Task *task = nullptr; std::vector dep; }; From d39a31aa96fc1bb44c1bebf58fd92e18cfa75978 Mon Sep 17 00:00:00 2001 From: jdolence Date: Mon, 18 Dec 2023 10:23:40 -0700 Subject: [PATCH 21/31] update tasking docs --- doc/sphinx/src/tasks.rst | 207 ++++++++++++++++++++++----------------- src/tasks/tasks.hpp | 9 +- 2 files changed, 123 insertions(+), 93 deletions(-) diff --git a/doc/sphinx/src/tasks.rst b/doc/sphinx/src/tasks.rst index d4c0b361b7f9..14e2390d8763 100644 --- a/doc/sphinx/src/tasks.rst +++ b/doc/sphinx/src/tasks.rst @@ -3,85 +3,81 @@ Tasks ===== +Parthenon's tasking infrastructure is how downstream applications describe +and execute their work. Tasks are organized into a hierarchy of objects. +``TaskCollection``s have one or more ``TaskRegion``s, ``TaskRegion``s have +one or more ``TaskList``s, and ``TaskList``s can have one or more sublists +(that are themselves ``TaskList``s). + +Task +---- + +Though downstream codes never have to interact with the ``Task`` object directly, +it's useful to describe nonetheless. A ``Task`` object is essentially a functor +that stores the necessary data to invoke a downstream code's functions with +the desired arguments. Importantly, however, it also stores information that +relates itself to other tasks, namely the tasks that must be complete before +it should execute and the tasks that may be available to run after it completes. +In other words, ``Task``s are nodes in a directed (possibly cyclic) graph, and +include the edges that connect to it and emerge from it. + TaskList -------- -The ``TaskList`` class implements methods to build and execute a set of -tasks with associated dependencies. The class implements a few public -facing member functions that provide useful functionality for downstream -apps: - -AddTask -~~~~~~~ - -``AddTask`` is a templated variadic function that takes the task -function to be executed, the task dependencies (see ``TaskID`` below), -and the arguments to the task function as it’s arguments. All arguments -are captured by value in a lambda for later execution. - -When adding functions that are non-static class member functions, a -slightly different interface is required. The first argument should be -the class-name-scoped name of the function. For example, for a function -named ``DoSomething`` in class ``SomeClass``, the first argument would -be ``&SomeClass::DoSomething``. The second argument should be a pointer -to the object that should invoke this member function. Finally, the -dependencies and function arguments should be provided as described -above. - -Examples of both ``AddTask`` calls can be found in the advection example -`here `__. - -AddIteration -~~~~~~~~~~~~ - -``AddIteration`` provides a means of grouping a set of tasks together -that will be executed repeatedly until stopping criteria are satisfied. -``AddIteration`` returns an ``IterativeTasks`` object which provides -overloaded ``AddTask`` functions as described above, but internally -handles the bookkeeping necessary to maintain the association of all the -tasks associated with the iterative process. A special function -``SetCompletionTask``, which behaves identically to ``AddTask``, allows -a task to be defined that evaluates the stopping criteria. The maximum -number of iterations can be controlled through the ``SetMaxIterations`` -member function and the number of iterations between evaluating the -stopping criteria can be set with the ``SetCheckInterval`` function. - -DoAvailable -~~~~~~~~~~~ - -``DoAvailable`` loops over the task list once, executing all tasks whose -dependencies are satisfied. Completed tasks are removed from the task -list. - -TaskID ------- - -The ``TaskID`` class implements methods that allow Parthenon to keep -track of tasks, their dependencies, and what remains to be completed. -The main way application code will interact with this object is as a -returned object from ``TaskList::AddTask`` and as an argument to -subsequent calls to ``TaskList::AddTask`` as a dependency for other -tasks. When used as a dependency, ``TaskID`` objects can be combined -with the bitwise or operator (``|``) to specify multiple dependencies. +The ``TaskList`` class stores a vector of all the tasks and sublists (a nested +``TaskList``) added to it. Additionally, it stores various bookkeeping +information that facilitate more advanced features described below. Adding +tasks and sublists are the only way to interact with ``TaskList`` objects. + +The basic call to ``AddTask`` takes the task's dependencies, the function to be +executed, and the arguments to the function as its arguments. ``AddTask`` returns +a ``TaskID`` object that can be used in subsequent calls to ``AddTask`` as a +dependency either on its own or combined with other ``TaskID``s via the ``|`` +operator. An overload of ``AddTask`` takes a ``TaskQualifier`` object as the +first argument which specifies certain special, non-default behaviors. These +will be described below. Note that the default constructor of ``TaskID`` produces +a special object that when passed into ``AddTask`` signifies that the task has +no dependencies. + +The ``AddSublist`` function adds a nested ``TaskList`` to the ``TaskList`` on +which its called. The principle use case for this is to add iterative cycles +to the graph, allowing one to execute a series of tasks repeatedly until some +criteria are satisfied. The call takes as arguments the dependencies (via +``TaskID``s combined with ``|``) that must be complete before the sublist +exectues and, optionally, a ``std::pair`` specifying the minimum +and maximum number of times the sublist should execute. Passing something like +``{min_iters, max_iters}`` as the second argument should suffice. ``AddSublist`` +returns a ``std::pair`` which is conveniently accessed via +a structured binding, e.g. +.. code:: cpp + TaskID none; + auto [child_list, child_list_id] = parent_list.AddSublist(dependencies, {1,3}); + auto task_id = child_list.AddTask(none, SomeFunction, arg1, arg2); +In the above example, passing ``none`` as the dependency for the task added to +``child_list`` does not imply that this task can execute at any time since +``child_list`` itself has dependencies that must be satisfied before any of its +tasks can be invoked. TaskRegion ---------- -``TaskRegion`` is a lightweight class that wraps -``std::vector``, providing a little extra functionality. -During task execution (described below), all task lists in a -``TaskRegion`` can be operated on concurrently. For example, a -``TaskRegion`` can be used to construct independent task lists for each -``MeshBlock``. Occasionally, it is useful to have a task not be -considered complete until that task completes in all lists of a region. -For example, a global iterative solver cannot be considered complete -until the stopping criteria are satisfied everywhere, which may require -evaluating those criteria in tasks that live in different lists within a -region. An example of this use case is -shown `here `__. The mechanism -to mark a task so that dependent tasks will wait until all lists have -completed it is to call ``AddRegionalDependencies``, as shown in the -Poisson example. +Under the hood, a ``TaskRegion`` is a directed, possibly cyclic graph. The graph +is built up incrementally as tasks are added to the ``TaskList``s within the +``TaskRegion``, and it's construction is completed upon the first time it's +executed. ``TaskRegion``s can have one or more ``TaskList``s. The primary reason +for this is to allow flexibility in how work is broken up into tasks (and +eventually kernels). A region with many lists will produce many small +tasks/kernels, but may expose more asynchrony (e.g. MPI communication). A region +with fewer lists will produce more work per kernel (which may be good for GPUs, +for example), but may limit asynchrony. Typically, each list is tied to a unique +partition of the mesh blocks owned by a rank. ``TaskRegion`` only provides a few +public facing functions: +- ``TaskListStatus Execute(ThreadPool &pool)``: ``TaskRegion``s can be executed, requiring a +``ThreadPool`` be provided by the caller. In practice, ``Execute`` is usually +called from the ``Execute`` member function of ``TaskCollection``. +- ``TaskList& operator[](const int i)``: return a reference to the ``i``th +``TaskList`` in the region. +- ``size_t size()``: return the number of ``TaskList``s in the region. TaskCollection -------------- @@ -120,21 +116,52 @@ is shown below. .. figure:: figs/TaskDiagram.png :alt: Task Diagram -``TaskCollection`` provides two member functions, ``AddRegion`` and -``Execute``. - -AddRegion -~~~~~~~~~ - -``AddRegion`` simply adds a new ``TaskRegion`` to the back of the -collection and returns it as a reference. The integer argument -determines how many task lists make up the region. - -Execute -~~~~~~~ - -Calling the ``Execute`` method on the ``TaskCollection`` executes all -the tasks that have been added to the collection, processing each -``TaskRegion`` in the order they were added, and allowing tasks in -different ``TaskList``\ s but the same ``TaskRegion`` to be executed -concurrently. +``TaskCollection`` provides a few +public-facing functions: +- ``TaskRegion& AddRegion(const int num_lists)``: Add and return a reference to +a new ``TaskRegion`` with the specified number of ``TaskList``s. +- ``TaskListStatus Execute(ThreadPool &pool)``: Execute all regions in the +collection. Regions are executed completely, in the order they were added, +before moving on to the next region. Task execution will take advantage of +the provided ``ThreadPool`` to (possibly) execute tasks across ``TaskList``s +in each region concurrently. +- ``TaskListStatus Execute()``: Same as above, but execution will use an +internally generated ``ThreadPool`` with a single thread. + +NOTE: Work remains to make the rest of +Parthenon thread-safe, so it is currently not recommended to use a ``ThreadPool`` +with more than one thread. + +TaskQualifier +------------- + +``TaskQualifier``s provide a mechanism for downstream codes to alter the default +behavior of specific tasks in certain ways. The qualifiers are described below: +- ``TaskQualifier::local_sync``: Tasks marked with ``local_sync`` synchronize across +lists in a region on a given MPI rank. Tasks that depend on a ``local_sync`` +marked task gain dependencies from the corresponding task on all lists within +a region. A typical use for this qualifier is to do a rank-local reduction, for +example before initiating a global MPI reduction (which should be done only once +per rank, not once per ``TaskList``). Note that Parthenon links tasks across +lists in the order they are added to each list, i.e. the ``n``th ``local_sync`` task +in a list is assumed to be associated with the ``n``th ``local_sync`` task in all +lists in the region. +- ``TaskQualifier::global_sync``: Tasks marked with ``global_sync`` implicitly have +the same semantics as ``local_sync``, but additionally do a global reduction on the +``TaskStatus`` to determine if/when execution can proceed on to dependent tasks. +- ``TaskQualifier::completion``: Tasks marked with ``completion`` can lead to exiting +execution of the owning ``TaskList``. If these tasks return ``TaskStatus::complete`` +and the minimum number of iterations of the list have been completed, the remainder +of the task list will be skipped (or the iteration stopped). Returning +``TaskList::iterate`` leads to continued execution/iteration, unless the maximum +number of iterations has been reached. +- ``TaskQualifier::once_per_region``: Tasks with the ``once_per_region`` qualifier +will only execute once (per iteration, if relevant) regardless of the number of +``TaskList``s in the region. This can be useful when, for example, doing MPI +reductions, printing out some rank-wide state, or calling a ``completion`` task +that depends on some global condition where all lists would evaluate identical code. + +``TaskQualifier``s can be combined via the ``|`` operator and all combinations are +supported. For example, you might mark a task ``global_sync | completion | once_per_region`` +if it were a task to determine whether an iteration should continue that depended +on some previously reduced quantity. diff --git a/src/tasks/tasks.hpp b/src/tasks/tasks.hpp index 921ccb06c0ad..08da684779f3 100644 --- a/src/tasks/tasks.hpp +++ b/src/tasks/tasks.hpp @@ -52,7 +52,6 @@ class TaskQualifier { bool Completion() const { return flags & completion; } bool Once() const { return flags & once_per_region; } bool Valid() const { - if (LocalSync() && GlobalSync()) return false; return true; } @@ -393,7 +392,7 @@ class TaskRegion { task_lists[i].SetID(i); } - void Execute(ThreadPool &pool) { + TaskListStatus Execute(ThreadPool &pool) { // first, if needed, finish building the graph if (!graph_built) BuildGraph(); @@ -418,6 +417,8 @@ class TaskRegion { // then wait until everything is done pool.wait(); + + return TaskListStatus::complete; } TaskList &operator[](const int i) { return task_lists[i]; } @@ -477,8 +478,10 @@ class TaskCollection { return Execute(pool); } TaskListStatus Execute(ThreadPool &pool) { + TaskListStatus status; for (auto ®ion : regions) { - region.Execute(pool); + status = region.Execute(pool); + if (status != TaskListStatus::complete) return status; } return TaskListStatus::complete; } From b074ee68301e76bbe1f7385a5be338a371ef57b8 Mon Sep 17 00:00:00 2001 From: jdolence Date: Mon, 18 Dec 2023 12:12:25 -0700 Subject: [PATCH 22/31] formatting and update changelog --- CHANGELOG.md | 2 ++ src/tasks/tasks.hpp | 4 +--- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 1831ee183089..13041501e2e6 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -3,6 +3,7 @@ ## Current develop ### Added (new features/APIs/variables/...) +- [[PR 987]](https://github.com/parthenon-hpc-lab/parthenon/pull/987) New tasking infrastructure and capabilities - [[PR 983]](https://github.com/parthenon-hpc-lab/parthenon/pull/983) Add Contains to SparsePack - [[PR 968]](https://github.com/parthenon-hpc-lab/parthenon/pull/968) Add per package registration of boundary conditions - [[PR 948]](https://github.com/parthenon-hpc-lab/parthenon/pull/948) Add solver interface and update Poisson geometric multi-grid example @@ -19,6 +20,7 @@ ### Removed (removing behavior/API/varaibles/...) ### Incompatibilities (i.e. breaking changes) +- [[PR 987]](https://github.com/parthenon-hpc-lab/parthenon/pull/987) Change the API for what was IterativeTasks - [[PR 974]](https://github.com/parthenon-hpc-lab/parthenon/pull/974) Change GetParentPointer to always return T* diff --git a/src/tasks/tasks.hpp b/src/tasks/tasks.hpp index 08da684779f3..d4e7d50b74e2 100644 --- a/src/tasks/tasks.hpp +++ b/src/tasks/tasks.hpp @@ -51,9 +51,7 @@ class TaskQualifier { bool GlobalSync() const { return flags & global_sync; } bool Completion() const { return flags & completion; } bool Once() const { return flags & once_per_region; } - bool Valid() const { - return true; - } + bool Valid() const { return true; } private: qualifier_t flags; From 829e047ab5148b28735ade533e78cb180109bf2e Mon Sep 17 00:00:00 2001 From: jdolence Date: Tue, 9 Jan 2024 11:19:52 -0700 Subject: [PATCH 23/31] address review comments --- doc/sphinx/src/tasks.rst | 16 ++++++------ src/bvals/comms/boundary_communication.cpp | 14 ++++++----- src/bvals/comms/bvals_in_one.hpp | 4 +-- src/tasks/tasks.hpp | 29 ++++++++++++++-------- src/tasks/thread_pool.hpp | 2 ++ 5 files changed, 40 insertions(+), 25 deletions(-) diff --git a/doc/sphinx/src/tasks.rst b/doc/sphinx/src/tasks.rst index 14e2390d8763..2fa42142ca3b 100644 --- a/doc/sphinx/src/tasks.rst +++ b/doc/sphinx/src/tasks.rst @@ -33,11 +33,13 @@ The basic call to ``AddTask`` takes the task's dependencies, the function to be executed, and the arguments to the function as its arguments. ``AddTask`` returns a ``TaskID`` object that can be used in subsequent calls to ``AddTask`` as a dependency either on its own or combined with other ``TaskID``s via the ``|`` -operator. An overload of ``AddTask`` takes a ``TaskQualifier`` object as the -first argument which specifies certain special, non-default behaviors. These -will be described below. Note that the default constructor of ``TaskID`` produces -a special object that when passed into ``AddTask`` signifies that the task has -no dependencies. +operator. Use of the ``|`` operator is historical and perhaps a bit misleading as +it really acts as a logical and -- that is, all tasks combined with ``|`` must be +complete before the dependencies are satisfied. An overload of ``AddTask`` takes +a ``TaskQualifier`` object as the first argument which specifies certain special, +non-default behaviors. These will be described below. Note that the default +constructor of ``TaskID`` produces a special object that when passed into +``AddTask`` signifies that the task has no dependencies. The ``AddSublist`` function adds a nested ``TaskList`` to the ``TaskList`` on which its called. The principle use case for this is to add iterative cycles @@ -129,8 +131,8 @@ in each region concurrently. internally generated ``ThreadPool`` with a single thread. NOTE: Work remains to make the rest of -Parthenon thread-safe, so it is currently not recommended to use a ``ThreadPool`` -with more than one thread. +Parthenon thread-safe, so it is currently required to use a ``ThreadPool`` +with one thread. TaskQualifier ------------- diff --git a/src/bvals/comms/boundary_communication.cpp b/src/bvals/comms/boundary_communication.cpp index 6b16510b58c2..4267a3a5a4e5 100644 --- a/src/bvals/comms/boundary_communication.cpp +++ b/src/bvals/comms/boundary_communication.cpp @@ -381,8 +381,8 @@ template TaskStatus ProlongateBounds(std::shared_ptr> &); // Adds all relevant boundary communication to a single task list -template -TaskID AddBoundaryExchangeTasks(TaskID dependency, TL_t &tl, +template +TaskID AddBoundaryExchangeTasks(TaskID dependency, TaskList &tl, std::shared_ptr> &md, bool multilevel) { // TODO(LFR): Splitting up the boundary tasks while doing prolongation can cause some // possible issues for sparse fields. In particular, the order in which @@ -422,9 +422,11 @@ TaskID AddBoundaryExchangeTasks(TaskID dependency, TL_t &tl, return fbound; } -template TaskID AddBoundaryExchangeTasks( - TaskID, TaskList &, std::shared_ptr> &, bool); +template TaskID +AddBoundaryExchangeTasks(TaskID, TaskList &, + std::shared_ptr> &, bool); -template TaskID AddBoundaryExchangeTasks( - TaskID, TaskList &, std::shared_ptr> &, bool); +template TaskID +AddBoundaryExchangeTasks(TaskID, TaskList &, + std::shared_ptr> &, bool); } // namespace parthenon diff --git a/src/bvals/comms/bvals_in_one.hpp b/src/bvals/comms/bvals_in_one.hpp index 05a95276a114..22b637a48a74 100644 --- a/src/bvals/comms/bvals_in_one.hpp +++ b/src/bvals/comms/bvals_in_one.hpp @@ -72,8 +72,8 @@ TaskStatus ReceiveFluxCorrections(std::shared_ptr> &md); TaskStatus SetFluxCorrections(std::shared_ptr> &md); // Adds all relevant boundary communication to a single task list -template -TaskID AddBoundaryExchangeTasks(TaskID dependency, TL_t &tl, +template +TaskID AddBoundaryExchangeTasks(TaskID dependency, TaskList &tl, std::shared_ptr> &md, bool multilevel); // These tasks should not be called in down stream code diff --git a/src/tasks/tasks.hpp b/src/tasks/tasks.hpp index d4e7d50b74e2..6aa8bedf2b4a 100644 --- a/src/tasks/tasks.hpp +++ b/src/tasks/tasks.hpp @@ -10,7 +10,6 @@ // license in this material to reproduce, prepare derivative works, distribute copies to // the public, perform publicly and display publicly, and to permit others to do so. //======================================================================================== - #ifndef TASKS_TASKS_HPP_ #define TASKS_TASKS_HPP_ @@ -25,6 +24,7 @@ #include #include +#include "utils/error_checking.hpp" #include #include @@ -68,10 +68,14 @@ class TaskID { TaskID operator|(const TaskID &other) const { // calling this operator means you're building a TaskID to hold a dependency TaskID result; - if (task != nullptr) result.dep.push_back(task); - result.dep.insert(result.dep.end(), dep.begin(), dep.end()); - if (other.task != nullptr) result.dep.push_back(other.task); - result.dep.insert(result.dep.end(), other.dep.begin(), other.dep.end()); + if (task != nullptr) + result.dep.push_back(task); + else + result.dep.insert(result.dep.end(), dep.begin(), dep.end()); + if (other.task != nullptr) + result.dep.push_back(other.task); + else + result.dep.insert(result.dep.end(), other.dep.begin(), other.dep.end()); return result; } @@ -234,12 +238,12 @@ class TaskList { // an MPI function after Finalize global_comm.emplace_back(new MPI_Comm, [&](MPI_Comm *d) { int finalized; - MPI_Finalized(&finalized); - if (!finalized) MPI_Comm_free(d); + PARTHENON_MPI_CHECK(MPI_Finalized(&finalized)); + if (!finalized) PARTHENON_MPI_CHECK(MPI_Comm_free(d)); }); // we need another communicator to support multiple in flight non-blocking // collectives where we can't guarantee calling order across ranks - MPI_Comm_dup(MPI_COMM_WORLD, global_comm.back().get()); + PARTHENON_MPI_CHECK(MPI_Comm_dup(MPI_COMM_WORLD, global_comm.back().get())); do_mpi = true; #endif // MPI_PARALLEL TaskID start; @@ -260,7 +264,8 @@ class TaskList { for (auto dep : mytask->GetDependencies()) { stat = std::max(stat, static_cast(dep->GetStatus())); } - MPI_Iallreduce(MPI_IN_PLACE, &stat, 1, MPI_INT, MPI_MAX, comm, &req); + PARTHENON_MPI_CHECK( + MPI_Iallreduce(MPI_IN_PLACE, &stat, 1, MPI_INT, MPI_MAX, comm, &req)); return TaskStatus::complete; }, exec_limits)); @@ -270,7 +275,7 @@ class TaskList { start, [&stat = *global_status.back(), &req = *global_request.back()]() { int check; - MPI_Test(&req, &check, MPI_STATUS_IGNORE); + PARTHENON_MPI_CHECK(MPI_Test(&req, &check, MPI_STATUS_IGNORE)); if (check) { return static_cast(stat); } @@ -391,6 +396,10 @@ class TaskRegion { } TaskListStatus Execute(ThreadPool &pool) { + // for now, require a pool with one thread + PARTHENON_REQUIRE_THROWS(pool.size() == 1, + "ThreadPool size != 1 is not currently supported.") + // first, if needed, finish building the graph if (!graph_built) BuildGraph(); diff --git a/src/tasks/thread_pool.hpp b/src/tasks/thread_pool.hpp index 187325f0df7c..2a9c4bd73fde 100644 --- a/src/tasks/thread_pool.hpp +++ b/src/tasks/thread_pool.hpp @@ -126,6 +126,8 @@ class ThreadPool { return result; } + int size() const { return nthreads; } + private: const int nthreads; std::vector threads; From b400c11b675432701ecc833a25741ec4554fc794 Mon Sep 17 00:00:00 2001 From: jdolence Date: Tue, 9 Jan 2024 11:49:26 -0700 Subject: [PATCH 24/31] style --- src/tasks/tasks.hpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/tasks/tasks.hpp b/src/tasks/tasks.hpp index 6aa8bedf2b4a..8628a9bdb2bc 100644 --- a/src/tasks/tasks.hpp +++ b/src/tasks/tasks.hpp @@ -24,11 +24,11 @@ #include #include -#include "utils/error_checking.hpp" #include #include #include "thread_pool.hpp" +#include "utils/error_checking.hpp" namespace parthenon { From 9957538d27c62a56261ac94587839f05a027f7cf Mon Sep 17 00:00:00 2001 From: jdolence Date: Tue, 9 Jan 2024 11:52:51 -0700 Subject: [PATCH 25/31] add a comment about the dependent variable in Task --- src/tasks/tasks.hpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/tasks/tasks.hpp b/src/tasks/tasks.hpp index 8628a9bdb2bc..ed5fa5bd9441 100644 --- a/src/tasks/tasks.hpp +++ b/src/tasks/tasks.hpp @@ -153,6 +153,8 @@ class Task { private: std::function f; + // store a list of tasks that might be available to + // run for each possible status this task returns std::array, 3> dependent; std::unordered_set dependencies; std::pair exec_limits; From 48426769dfe18536f570a61ff9a6e5afe8047ef3 Mon Sep 17 00:00:00 2001 From: jdolence Date: Fri, 12 Jan 2024 09:31:12 -0700 Subject: [PATCH 26/31] add locks to sparse pack caching --- src/interface/sparse_pack_base.cpp | 11 +++++++---- src/interface/sparse_pack_base.hpp | 8 ++++++++ .../test_meshblock_data_iterator.cpp | 17 ++++++++--------- 3 files changed, 23 insertions(+), 13 deletions(-) diff --git a/src/interface/sparse_pack_base.cpp b/src/interface/sparse_pack_base.cpp index e8250c4b58bc..6c474c3823b2 100644 --- a/src/interface/sparse_pack_base.cpp +++ b/src/interface/sparse_pack_base.cpp @@ -287,8 +287,9 @@ template SparsePackBase SparsePackBase::Build>(MeshData *, template SparsePackBase &SparsePackCache::Get(T *pmd, const PackDescriptor &desc, const std::vector &include_block) { - if (pack_map.count(desc.identifier) > 0) { - auto &cache_tuple = pack_map[desc.identifier]; + auto cache_tuple_itr = GetFromMap(desc.identifier); + if (cache_tuple_itr != pack_map.end()) { + auto &cache_tuple = cache_tuple_itr->second; auto &pack = std::get<0>(cache_tuple); auto alloc_status_in = SparsePackBase::GetAllocStatus(pmd, desc, include_block); auto &alloc_status = std::get<1>(cache_tuple); @@ -320,8 +321,10 @@ SparsePackCache::Get>(MeshBlockData *, const PackDescr template SparsePackBase &SparsePackCache::BuildAndAdd(T *pmd, const PackDescriptor &desc, const std::vector &include_block) { - if (pack_map.count(desc.identifier) > 0) pack_map.erase(desc.identifier); - pack_map[desc.identifier] = {SparsePackBase::Build(pmd, desc, include_block), + auto pack = SparsePackBase::Build(pmd, desc, include_block); + std::lock_guard lock(mutex); + pack_map.erase(desc.identifier); + pack_map[desc.identifier] = {std::move(pack), SparsePackBase::GetAllocStatus(pmd, desc, include_block), include_block}; return std::get<0>(pack_map[desc.identifier]); diff --git a/src/interface/sparse_pack_base.hpp b/src/interface/sparse_pack_base.hpp index 26a5f3fd7b61..76673a6c2440 100644 --- a/src/interface/sparse_pack_base.hpp +++ b/src/interface/sparse_pack_base.hpp @@ -18,6 +18,7 @@ #include #include #include +#include #include #include #include @@ -121,6 +122,13 @@ class SparsePackCache { SparsePackBase::include_t>> pack_map; + private: + auto GetFromMap(const std::string &key) { + std::lock_guard lock(mutex); + return pack_map.find(key); + } + std::mutex mutex; + friend class SparsePackBase; }; diff --git a/tst/performance/test_meshblock_data_iterator.cpp b/tst/performance/test_meshblock_data_iterator.cpp index 923899d26e26..f1004ff95888 100644 --- a/tst/performance/test_meshblock_data_iterator.cpp +++ b/tst/performance/test_meshblock_data_iterator.cpp @@ -76,7 +76,8 @@ void performance_test_wrapper(const std::string &test_name, InitFunc init_func, // we need to connect the MeshBlockData to a dummy mesh block, otherwise variables // won't be allocated -static MeshBlockData createTestContainer(std::shared_ptr &dummy_mb) { +static void createTestContainer(std::shared_ptr dummy_mb, + MeshBlockData &mbd) { // Make a container for testing performance std::vector scalar_shape{N, N, N}; std::vector vector_shape{N, N, N, 3}; @@ -93,10 +94,7 @@ static MeshBlockData createTestContainer(std::shared_ptr &dummy pkg->AddField("v4", m_in_vec); pkg->AddField("v5", m_in); - MeshBlockData mbd; mbd.Initialize(pkg, dummy_mb); - - return mbd; } // std::function createLambdaRaw(ParArrayND &raw_array) { @@ -125,7 +123,6 @@ std::function createLambdaContainer(MeshBlockData &container) { v(l, k, j, i) = static_cast((l + 1) * (k + 1) * (j + 1) * (i + 1)); }); } - return container; }; } @@ -142,7 +139,6 @@ std::function createLambdaContainerVar(MeshBlockData &container, data(l, k, j, i) = static_cast((l + 1) * (k + 1) * (j + 1) * (i + 1)); }); } - return container; }; } @@ -196,7 +192,8 @@ TEST_CASE("Catch2 Container Iterator Performance", SECTION("Iterate Variables") { GIVEN("A container.") { auto dummy_mb = std::make_shared(16, 3); - MeshBlockData container = createTestContainer(dummy_mb); + MeshBlockData container; + createTestContainer(dummy_mb, container); auto init_container = createLambdaContainer(container); // Make a function for initializing the container variables @@ -215,7 +212,8 @@ TEST_CASE("Catch2 Container Iterator Performance", } // GIVEN GIVEN("A container Var.") { auto dummy_mb = std::make_shared(16, 3); - MeshBlockData container = createTestContainer(dummy_mb); + MeshBlockData container; + createTestContainer(dummy_mb, container); std::vector names({"v0", "v1", "v2", "v3", "v4", "v5"}); auto init_container = createLambdaContainerVar(container, names); @@ -238,7 +236,8 @@ TEST_CASE("Catch2 Container Iterator Performance", SECTION("View of Views") { GIVEN("A container.") { auto dummy_mb = std::make_shared(16, 3); - MeshBlockData container = createTestContainer(dummy_mb); + MeshBlockData container; + createTestContainer(dummy_mb, container); WHEN("The view of views does not have any names.") { parthenon::VariablePack var_view = container.PackVariables({Metadata::Independent}); From 7ce9ed5d458afeea3197343ca838b55f1fb28b01 Mon Sep 17 00:00:00 2001 From: jdolence Date: Fri, 12 Jan 2024 10:22:07 -0700 Subject: [PATCH 27/31] move thread pool to utils --- src/tasks/tasks.hpp | 2 +- src/{tasks => utils}/thread_pool.hpp | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) rename src/{tasks => utils}/thread_pool.hpp (98%) diff --git a/src/tasks/tasks.hpp b/src/tasks/tasks.hpp index ed5fa5bd9441..7dd3e2d8fc54 100644 --- a/src/tasks/tasks.hpp +++ b/src/tasks/tasks.hpp @@ -27,8 +27,8 @@ #include #include -#include "thread_pool.hpp" #include "utils/error_checking.hpp" +#include "utils/thread_pool.hpp" namespace parthenon { diff --git a/src/tasks/thread_pool.hpp b/src/utils/thread_pool.hpp similarity index 98% rename from src/tasks/thread_pool.hpp rename to src/utils/thread_pool.hpp index 2a9c4bd73fde..77a7ba66cac5 100644 --- a/src/tasks/thread_pool.hpp +++ b/src/utils/thread_pool.hpp @@ -11,8 +11,8 @@ // the public, perform publicly and display publicly, and to permit others to do so. //======================================================================================== -#ifndef TASKS_THREAD_POOL_HPP_ -#define TASKS_THREAD_POOL_HPP_ +#ifndef UTILS_THREAD_POOL_HPP_ +#define UTILS_THREAD_POOL_HPP_ #include #include From 97874e0174bd87e8c31d4566769f1649c7be007b Mon Sep 17 00:00:00 2001 From: jdolence Date: Fri, 12 Jan 2024 11:11:16 -0700 Subject: [PATCH 28/31] add thread pool to driver/mesh --- src/CMakeLists.txt | 2 +- src/driver/driver.hpp | 13 +++++++++---- src/mesh/mesh.hpp | 5 +++++ 3 files changed, 15 insertions(+), 5 deletions(-) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index d60a37d56355..427230fa57d0 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -214,7 +214,6 @@ add_library(parthenon solvers/solver_utils.hpp tasks/tasks.hpp - tasks/thread_pool.hpp time_integration/butcher_integrator.cpp time_integration/low_storage_integrator.cpp @@ -251,6 +250,7 @@ add_library(parthenon utils/sort.hpp utils/string_utils.cpp utils/string_utils.hpp + utils/thread_pool.hpp utils/unique_id.cpp utils/unique_id.hpp utils/utils.hpp diff --git a/src/driver/driver.hpp b/src/driver/driver.hpp index 929ea19c3f2b..3934e92f77d7 100644 --- a/src/driver/driver.hpp +++ b/src/driver/driver.hpp @@ -38,7 +38,11 @@ enum class DriverStatus { complete, timeout, failed }; class Driver { public: Driver(ParameterInput *pin, ApplicationInput *app_in, Mesh *pm) - : pinput(pin), app_input(app_in), pmesh(pm), mbcnt_prev(), time_LBandAMR() {} + : pinput(pin), app_input(app_in), pmesh(pm), mbcnt_prev(), time_LBandAMR() { + const int nthreads = pin->GetOrAddInteger("parthenon/driver", "nthreads", 1); + pool = std::make_shared(nthreads); + pm->SetThreadPool(pool); + } virtual DriverStatus Execute() = 0; void InitializeOutputs() { pouts = std::make_unique(pmesh, pinput); } @@ -46,14 +50,15 @@ class Driver { ApplicationInput *app_input; Mesh *pmesh; std::unique_ptr pouts; + std::shared_ptr pool; static double elapsed_main() { return timer_main.seconds(); } static double elapsed_cycle() { return timer_cycle.seconds(); } static double elapsed_LBandAMR() { return timer_LBandAMR.seconds(); } protected: static Kokkos::Timer timer_cycle, timer_main, timer_LBandAMR; - double time_LBandAMR; std::uint64_t mbcnt_prev; + double time_LBandAMR; virtual void PreExecute(); virtual void PostExecute(DriverStatus status); @@ -105,7 +110,7 @@ TaskListStatus ConstructAndExecuteBlockTasks(T *driver, Args... args) { for (auto &pmb : driver->pmesh->block_list) { tr[i++] = driver->MakeTaskList(pmb.get(), std::forward(args)...); } - TaskListStatus status = tc.Execute(); + TaskListStatus status = tc.Execute(*driver->pool); return status; } @@ -113,7 +118,7 @@ template TaskListStatus ConstructAndExecuteTaskLists(T *driver, Args... args) { TaskCollection tc = driver->MakeTaskCollection(driver->pmesh->block_list, std::forward(args)...); - TaskListStatus status = tc.Execute(); + TaskListStatus status = tc.Execute(*driver->pool); return status; } diff --git a/src/mesh/mesh.hpp b/src/mesh/mesh.hpp index bf21853d3a82..0346d894b5d2 100644 --- a/src/mesh/mesh.hpp +++ b/src/mesh/mesh.hpp @@ -54,6 +54,7 @@ #include "utils/hash.hpp" #include "utils/object_pool.hpp" #include "utils/partition_stl_containers.hpp" +#include "utils/thread_pool.hpp" namespace parthenon { @@ -233,6 +234,8 @@ class Mesh { return resolved_packages->GetVariableNames(std::forward(args)...); } + void SetThreadPool(std::shared_ptr p) { pool = p; } + private: // data int root_level, max_level, current_level; @@ -274,6 +277,8 @@ class Mesh { int gmg_min_logical_level_ = 0; + std::shared_ptr pool; + #ifdef MPI_PARALLEL // Global map of MPI comms for separate variables std::unordered_map mpi_comm_map_; From e9630b70bc1c49bc65daa7fbf685ce761d2000fe Mon Sep 17 00:00:00 2001 From: jdolence Date: Fri, 22 Mar 2024 08:46:29 -0600 Subject: [PATCH 29/31] random intermediate commit --- CMakeLists.txt | 3 + example/advection/CMakeLists.txt | 2 + example/advection/advection_package.cpp | 78 ++++++++++++---------- src/CMakeLists.txt | 3 + src/bvals/comms/boundary_communication.cpp | 13 ++++ src/interface/sparse_pack_base.cpp | 2 +- src/interface/update.cpp | 32 ++++++--- src/interface/update.hpp | 24 ++++--- src/interface/variable_pack.hpp | 2 + src/reconstruct/dc_inline.hpp | 15 ++--- src/tasks/tasks.hpp | 4 +- src/utils/thread_pool.hpp | 24 ++++--- 12 files changed, 127 insertions(+), 75 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 0f6b0695c5e9..116f47086e3b 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -202,6 +202,9 @@ endif() set(CMAKE_CXX_STANDARD_REQUIRED ON) set(CMAKE_CXX_STANDARD 17) +add_compile_options(-fsanitize=thread) +add_link_options(-fsanitize=thread) + option(PARTHENON_IMPORT_KOKKOS "If ON, attempt to link to an external Kokkos library. If OFF, build Kokkos from source and package with Parthenon" OFF) if (NOT TARGET Kokkos::kokkos) if (PARTHENON_IMPORT_KOKKOS) diff --git a/example/advection/CMakeLists.txt b/example/advection/CMakeLists.txt index 97541db4006c..a2174386a418 100644 --- a/example/advection/CMakeLists.txt +++ b/example/advection/CMakeLists.txt @@ -22,6 +22,8 @@ if( "advection-example" IN_LIST DRIVER_LIST OR NOT PARTHENON_DISABLE_EXAMPLES) main.cpp parthenon_app_inputs.cpp ) + add_compile_options(-fsanitize=thread) + add_link_options(-fsanitize=thread) target_link_libraries(advection-example PRIVATE Parthenon::parthenon) lint_target(advection-example) endif() diff --git a/example/advection/advection_package.cpp b/example/advection/advection_package.cpp index d04e2179f1d7..24ceb6e7e509 100644 --- a/example/advection/advection_package.cpp +++ b/example/advection/advection_package.cpp @@ -239,8 +239,8 @@ AmrTag CheckRefinement(MeshBlockData *rc) { for (int var = 1; var < num_vars; ++var) { vars.push_back("advected_" + std::to_string(var)); } - // type is parthenon::VariablePack> - auto v = rc->PackVariables(vars); + auto desc = parthenon::MakePackDescriptor(pmb->resolved_packages.get(), vars); + auto v = desc.GetPack(rc); IndexRange ib = pmb->cellbounds.GetBoundsI(IndexDomain::entire); IndexRange jb = pmb->cellbounds.GetBoundsJ(IndexDomain::entire); @@ -248,13 +248,13 @@ AmrTag CheckRefinement(MeshBlockData *rc) { typename Kokkos::MinMax::value_type minmax; pmb->par_reduce( - PARTHENON_AUTO_LABEL, 0, v.GetDim(4) - 1, kb.s, kb.e, jb.s, jb.e, ib.s, ib.e, + PARTHENON_AUTO_LABEL, 0, v.GetMaxNumberOfVars() - 1, kb.s, kb.e, jb.s, jb.e, ib.s, ib.e, KOKKOS_LAMBDA(const int n, const int k, const int j, const int i, typename Kokkos::MinMax::value_type &lminmax) { lminmax.min_val = - (v(n, k, j, i) < lminmax.min_val ? v(n, k, j, i) : lminmax.min_val); + (v(0, n, k, j, i) < lminmax.min_val ? v(0, n, k, j, i) : lminmax.min_val); lminmax.max_val = - (v(n, k, j, i) > lminmax.max_val ? v(n, k, j, i) : lminmax.max_val); + (v(0, n, k, j, i) > lminmax.max_val ? v(0, n, k, j, i) : lminmax.max_val); }, Kokkos::MinMax(minmax)); @@ -279,16 +279,17 @@ void PreFill(MeshBlockData *rc) { // packing in principle unnecessary/convoluted here and just done for demonstration std::vector vars({"advected", "one_minus_advected"}); - PackIndexMap imap; - const auto &v = rc->PackVariables(vars, imap); + auto desc = parthenon::MakePackDescriptor(pmb->resolved_packages.get(), vars); + auto v = desc.GetPack(rc); + auto imap = desc.GetMap(); - const int in = imap.get("advected").first; - const int out = imap.get("one_minus_advected").first; + const int in = imap["advected"]; + const int out = imap["one_minus_advected"]; const auto num_vars = rc->Get("advected").data.GetDim(4); pmb->par_for( PARTHENON_AUTO_LABEL, 0, num_vars - 1, kb.s, kb.e, jb.s, jb.e, ib.s, ib.e, KOKKOS_LAMBDA(const int n, const int k, const int j, const int i) { - v(out + n, k, j, i) = 1.0 - v(in + n, k, j, i); + v(0, out + n, k, j, i) = 1.0 - v(0, in + n, k, j, i); }); } } @@ -303,16 +304,18 @@ void SquareIt(MeshBlockData *rc) { // packing in principle unnecessary/convoluted here and just done for demonstration std::vector vars({"one_minus_advected", "one_minus_advected_sq"}); - PackIndexMap imap; - const auto &v = rc->PackVariables(vars, imap); + auto desc = parthenon::MakePackDescriptor(pmb->resolved_packages.get(), vars); + auto v = desc.GetPack(rc); + auto imap = desc.GetMap(); - const int in = imap.get("one_minus_advected").first; - const int out = imap.get("one_minus_advected_sq").first; + + const int in = imap["one_minus_advected"]; + const int out = imap["one_minus_advected_sq"]; const auto num_vars = rc->Get("advected").data.GetDim(4); pmb->par_for( PARTHENON_AUTO_LABEL, 0, num_vars - 1, kb.s, kb.e, jb.s, jb.e, ib.s, ib.e, KOKKOS_LAMBDA(const int n, const int k, const int j, const int i) { - v(out + n, k, j, i) = v(in + n, k, j, i) * v(in + n, k, j, i); + v(0, out + n, k, j, i) = v(0, in + n, k, j, i) * v(0, in + n, k, j, i); }); // The following block/logic is also just added for regression testing. @@ -351,16 +354,17 @@ void PostFill(MeshBlockData *rc) { pmb->AllocSparseID("one_minus_sqrt_one_minus_advected_sq", 37); // packing in principle unnecessary/convoluted here and just done for demonstration - std::vector vars( - {"one_minus_advected_sq", "one_minus_sqrt_one_minus_advected_sq"}); - PackIndexMap imap; - const auto &v = rc->PackVariables(vars, imap); + std::vector> vars( + {{"one_minus_advected_sq", false}, {"one_minus_sqrt_one_minus_advected_sq*", true}}); + auto desc = parthenon::MakePackDescriptor(pmb->resolved_packages.get(), vars); + auto v = desc.GetPack(rc); + auto imap = desc.GetMap(); - const int in = imap.get("one_minus_advected_sq").first; + const int in = imap["one_minus_advected_sq"]; // we can get sparse fields either by specifying base name and sparse id, or the full // name - const int out12 = imap.get("one_minus_sqrt_one_minus_advected_sq", 12).first; - const int out37 = imap.get("one_minus_sqrt_one_minus_advected_sq_37").first; + const int out12 = imap["one_minus_sqrt_one_minus_advected_sq_12"]; + const int out37 = imap["one_minus_sqrt_one_minus_advected_sq_37"]; const auto num_vars = rc->Get("advected").data.GetDim(4); pmb->par_for( PARTHENON_AUTO_LABEL, 0, num_vars - 1, kb.s, kb.e, jb.s, jb.e, ib.s, ib.e, @@ -386,7 +390,8 @@ Real AdvectionHst(MeshData *md) { // Packing variable over MeshBlock as the function is called for MeshData, i.e., a // collection of blocks - const auto &advected_pack = md->PackVariables(std::vector{"advected"}); + auto desc = parthenon::MakePackDescriptor(pmb->resolved_packages.get(), std::vector{"advected"}); + auto advected_pack = desc.GetPack(md); Real result = 0.0; T reducer(result); @@ -397,10 +402,10 @@ Real AdvectionHst(MeshData *md) { const bool volume_weighting = std::is_same>::value; pmb->par_reduce( - PARTHENON_AUTO_LABEL, 0, advected_pack.GetDim(5) - 1, kb.s, kb.e, jb.s, jb.e, ib.s, + PARTHENON_AUTO_LABEL, 0, advected_pack.GetNBlocks() - 1, kb.s, kb.e, jb.s, jb.e, ib.s, ib.e, KOKKOS_LAMBDA(const int b, const int k, const int j, const int i, Real &lresult) { - const auto &coords = advected_pack.GetCoords(b); + const auto &coords = advected_pack.GetCoordinates(b); // `join` is a function of the Kokkos::ReducerConecpt that allows to use the same // call for different reductions const Real vol = volume_weighting ? coords.CellVolume(k, j, i) : 1.0; @@ -461,18 +466,23 @@ TaskStatus CalculateFluxes(std::shared_ptr> &rc) { const auto &vy = pkg->Param("vy"); const auto &vz = pkg->Param("vz"); - PackIndexMap index_map; - auto v = rc->PackVariablesAndFluxes(std::vector{Metadata::WithFluxes}, - index_map); - - // For non constant velocity, we need the index of the velocity vector as it's part of - // the variable pack. - const auto idx_v = index_map["v"].first; - const auto v_const = idx_v < 0; // using "at own perill" magic number + auto desc = parthenon::MakePackDescriptor(pmb->resolved_packages.get(), + std::vector{Metadata::WithFluxes}, + std::set{parthenon::PDOpt::WithFluxes}); + auto v = desc.GetPack(rc.get()); + //auto imap = desc.GetMap(); + int idx_v; + bool v_const; + //if (imap.count("v") > 0) { + //idx_v = imap["v"]; + // v_const = false; + //} else { + v_const = true; + //} const int scratch_level = 1; // 0 is actual scratch (tiny); 1 is HBM const int nx1 = pmb->cellbounds.ncellsi(IndexDomain::entire); - const int nvar = v.GetDim(4); + const int nvar = v.GetMaxNumberOfVars(); size_t scratch_size_in_bytes = parthenon::ScratchPad2D::shmem_size(nvar, nx1); // get x-fluxes pmb->par_for_outer( diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 427230fa57d0..bbe6cdb5141a 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -310,6 +310,9 @@ if (Kokkos_ENABLE_CUDA AND NOT CMAKE_CXX_COMPILER_ID STREQUAL "Clang") target_compile_options(parthenon PUBLIC --expt-relaxed-constexpr) endif() +add_compile_options(-fsanitize=thread) +add_link_options(-fsanitize=thread) + target_link_libraries(parthenon PUBLIC Kokkos::kokkos Threads::Threads) if (PARTHENON_ENABLE_ASCENT) diff --git a/src/bvals/comms/boundary_communication.cpp b/src/bvals/comms/boundary_communication.cpp index cc093a34f080..f4c4f9da5fe8 100644 --- a/src/bvals/comms/boundary_communication.cpp +++ b/src/bvals/comms/boundary_communication.cpp @@ -44,10 +44,14 @@ namespace parthenon { using namespace loops; using namespace loops::shorthands; +static std::mutex mutex; + template TaskStatus SendBoundBufs(std::shared_ptr> &md) { PARTHENON_INSTRUMENT + std::lock_guard lock(mutex); + Mesh *pmesh = md->GetMeshPointer(); auto &cache = md->GetBvarsCache().GetSubCache(bound_type, true); @@ -165,6 +169,9 @@ SendBoundBufs(std::shared_ptr> template TaskStatus StartReceiveBoundBufs(std::shared_ptr> &md) { PARTHENON_INSTRUMENT + + std::lock_guard lock(mutex); + Mesh *pmesh = md->GetMeshPointer(); auto &cache = md->GetBvarsCache().GetSubCache(bound_type, false); if (cache.buf_vec.size() == 0) @@ -192,6 +199,8 @@ template TaskStatus ReceiveBoundBufs(std::shared_ptr> &md) { PARTHENON_INSTRUMENT + std::lock_guard lock(mutex); + Mesh *pmesh = md->GetMeshPointer(); auto &cache = md->GetBvarsCache().GetSubCache(bound_type, false); if (cache.buf_vec.size() == 0) @@ -240,6 +249,8 @@ template TaskStatus SetBounds(std::shared_ptr> &md) { PARTHENON_INSTRUMENT + std::lock_guard lock(mutex); + Mesh *pmesh = md->GetMeshPointer(); auto &cache = md->GetBvarsCache().GetSubCache(bound_type, false); @@ -333,6 +344,8 @@ template TaskStatus ProlongateBounds(std::shared_ptr> &md) { PARTHENON_INSTRUMENT + std::lock_guard lock(mutex); + Mesh *pmesh = md->GetMeshPointer(); auto &cache = md->GetBvarsCache().GetSubCache(bound_type, false); diff --git a/src/interface/sparse_pack_base.cpp b/src/interface/sparse_pack_base.cpp index 6c474c3823b2..c57e777fd86b 100644 --- a/src/interface/sparse_pack_base.cpp +++ b/src/interface/sparse_pack_base.cpp @@ -288,7 +288,7 @@ template SparsePackBase &SparsePackCache::Get(T *pmd, const PackDescriptor &desc, const std::vector &include_block) { auto cache_tuple_itr = GetFromMap(desc.identifier); - if (cache_tuple_itr != pack_map.end()) { + if (false && cache_tuple_itr != pack_map.end()) { auto &cache_tuple = cache_tuple_itr->second; auto &pack = std::get<0>(cache_tuple); auto alloc_status_in = SparsePackBase::GetAllocStatus(pmd, desc, include_block); diff --git a/src/interface/update.cpp b/src/interface/update.cpp index 0e0d1195943e..69ab99d0990d 100644 --- a/src/interface/update.cpp +++ b/src/interface/update.cpp @@ -62,23 +62,37 @@ template <> TaskStatus FluxDivergence(MeshData *in_obj, MeshData *dudt_obj) { const IndexDomain interior = IndexDomain::interior; + auto pm = in_obj->GetMeshPointer(); std::vector flags({Metadata::WithFluxes, Metadata::Cell}); - const auto &vin = in_obj->PackVariablesAndFluxes(flags); - auto dudt = dudt_obj->PackVariables(flags); + auto desc = MakePackDescriptor(pm->resolved_packages.get(), + flags, std::set{PDOpt::WithFluxes}); + auto v = desc.GetPack(in_obj); + auto dudt = desc.GetPack(dudt_obj); + const IndexRange ib = in_obj->GetBoundsI(interior); const IndexRange jb = in_obj->GetBoundsJ(interior); const IndexRange kb = in_obj->GetBoundsK(interior); - const int ndim = vin.GetNdim(); + const int nblocks = v.GetNBlocks(); + const int nvars = v.GetMaxNumberOfVars(); + + const int ndim = pm->ndim; parthenon::par_for( - DEFAULT_LOOP_PATTERN, PARTHENON_AUTO_LABEL, DevExecSpace(), 0, vin.GetDim(5) - 1, 0, - vin.GetDim(4) - 1, kb.s, kb.e, jb.s, jb.e, ib.s, ib.e, + DEFAULT_LOOP_PATTERN, PARTHENON_AUTO_LABEL, DevExecSpace(), 0, nblocks - 1, 0, + nvars - 1, kb.s, kb.e, jb.s, jb.e, ib.s, ib.e, KOKKOS_LAMBDA(const int m, const int l, const int k, const int j, const int i) { - if (dudt.IsAllocated(m, l) && vin.IsAllocated(m, l)) { - const auto &coords = vin.GetCoords(m); - const auto &v = vin(m); - dudt(m, l, k, j, i) = FluxDivHelper(l, k, j, i, ndim, coords, v); + const auto &coords = v.GetCoordinates(m); + Real du = (coords.FaceArea(k, j, i + 1) * v.flux(m, X1DIR, l, k, j, i + 1) - + coords.FaceArea(k, j, i) * v.flux(m, X1DIR, l, k, j, i)); + if (ndim >= 2) { + du += (coords.FaceArea(k, j + 1, i) * v.flux(m, X2DIR, l, k, j + 1, i) - + coords.FaceArea(k, j, i) * v.flux(m, X2DIR, l, k, j, i)); + } + if (ndim == 3) { + du += (coords.FaceArea(k + 1, j, i) * v.flux(m, X3DIR, l, k + 1, j, i) - + coords.FaceArea(k, j, i) * v.flux(m, X3DIR, l, k, j, i)); } + dudt(m, l, k, j, i) = -du / coords.CellVolume(k, j, i); }); return TaskStatus::complete; } diff --git a/src/interface/update.hpp b/src/interface/update.hpp index 21f035caefa3..79f660ee6ff1 100644 --- a/src/interface/update.hpp +++ b/src/interface/update.hpp @@ -71,19 +71,21 @@ template TaskStatus WeightedSumData(const F &flags, T *in1, T *in2, const Real w1, const Real w2, T *out) { PARTHENON_INSTRUMENT - const auto &x = in1->PackVariables(flags); - const auto &y = in2->PackVariables(flags); - const auto &z = out->PackVariables(flags); + auto pm = in1->GetMeshPointer(); + auto desc = MakePackDescriptor(pm->resolved_packages.get(), flags); + auto x = desc.GetPack(in1); + auto y = desc.GetPack(in2); + auto z = desc.GetPack(out); + auto num_var = x.GetMaxNumberOfVars(); + auto num_blocks = x.GetNBlocks(); + auto ib = in1->GetBoundsI(IndexDomain::entire); + auto jb = in1->GetBoundsJ(IndexDomain::entire); + auto kb = in1->GetBoundsK(IndexDomain::entire); parthenon::par_for( - DEFAULT_LOOP_PATTERN, PARTHENON_AUTO_LABEL, DevExecSpace(), 0, x.GetDim(5) - 1, 0, - x.GetDim(4) - 1, 0, x.GetDim(3) - 1, 0, x.GetDim(2) - 1, 0, x.GetDim(1) - 1, + DEFAULT_LOOP_PATTERN, PARTHENON_AUTO_LABEL, DevExecSpace(), 0, num_blocks - 1, 0, + num_var - 1, kb.s, kb.e, jb.s, jb.e, ib.s, ib.e, KOKKOS_LAMBDA(const int b, const int l, const int k, const int j, const int i) { - // TOOD(someone) This is potentially dangerous and/or not intended behavior - // as we still may want to update (or populate) z if any of those vars are - // not allocated yet. - if (x.IsAllocated(b, l) && y.IsAllocated(b, l) && z.IsAllocated(b, l)) { - z(b, l, k, j, i) = w1 * x(b, l, k, j, i) + w2 * y(b, l, k, j, i); - } + z(b, l, k, j, i) = w1 * x(b, l, k, j, i) + w2 * y(b, l, k, j, i); }); return TaskStatus::complete; } diff --git a/src/interface/variable_pack.hpp b/src/interface/variable_pack.hpp index 924d15166aaf..9fd87cfbb566 100644 --- a/src/interface/variable_pack.hpp +++ b/src/interface/variable_pack.hpp @@ -716,6 +716,7 @@ template VariableFluxPack MakeFluxPack(const VarListWithKeys &var_list, const VarListWithKeys &flux_var_list, PackIndexMap *pvmap) { + printf("Making a FluxPack\n"); const auto &vars = var_list.vars(); // for convenience const auto &flux_vars = flux_var_list.vars(); // for convenience @@ -777,6 +778,7 @@ VariableFluxPack MakeFluxPack(const VarListWithKeys &var_list, template VariablePack MakePack(const VarListWithKeys &var_list, bool coarse, PackIndexMap *pvmap) { + printf("Making a Pack\n"); const auto &vars = var_list.vars(); // for convenience if (vars.empty()) { diff --git a/src/reconstruct/dc_inline.hpp b/src/reconstruct/dc_inline.hpp index bd8401f8ab54..c0dd180da997 100644 --- a/src/reconstruct/dc_inline.hpp +++ b/src/reconstruct/dc_inline.hpp @@ -33,13 +33,12 @@ template KOKKOS_FORCEINLINE_FUNCTION void DonorCellX1(parthenon::team_mbr_t const &member, const int k, const int j, const int il, const int iu, const T &q, ScratchPad2D &ql, ScratchPad2D &qr) { - const int nu = q.GetDim(4) - 1; + const int nu = q.GetMaxNumberOfVars() - 1; // compute L/R states for each variable for (int n = 0; n <= nu; ++n) { - if (!q.IsAllocated(n)) continue; parthenon::par_for_inner( - member, il, iu, [&](const int i) { ql(n, i + 1) = qr(n, i) = q(n, k, j, i); }); + member, il, iu, [&](const int i) { ql(n, i + 1) = qr(n, i) = q(0, n, k, j, i); }); } } @@ -50,12 +49,11 @@ template KOKKOS_FORCEINLINE_FUNCTION void DonorCellX2(parthenon::team_mbr_t const &member, const int k, const int j, const int il, const int iu, const T &q, ScratchPad2D &ql, ScratchPad2D &qr) { - const int nu = q.GetDim(4) - 1; + const int nu = q.GetMaxNumberOfVars() - 1; // compute L/R states for each variable for (int n = 0; n <= nu; ++n) { - if (!q.IsAllocated(n)) continue; parthenon::par_for_inner(member, il, iu, - [&](const int i) { ql(n, i) = qr(n, i) = q(n, k, j, i); }); + [&](const int i) { ql(n, i) = qr(n, i) = q(0, n, k, j, i); }); } } @@ -66,12 +64,11 @@ template KOKKOS_FORCEINLINE_FUNCTION void DonorCellX3(parthenon::team_mbr_t const &member, const int k, const int j, const int il, const int iu, const T &q, ScratchPad2D &ql, ScratchPad2D &qr) { - const int nu = q.GetDim(4) - 1; + const int nu = q.GetMaxNumberOfVars() - 1; // compute L/R states for each variable for (int n = 0; n <= nu; ++n) { - if (!q.IsAllocated(n)) continue; parthenon::par_for_inner(member, il, iu, - [&](const int i) { ql(n, i) = qr(n, i) = q(n, k, j, i); }); + [&](const int i) { ql(n, i) = qr(n, i) = q(0, n, k, j, i); }); } } } // namespace parthenon diff --git a/src/tasks/tasks.hpp b/src/tasks/tasks.hpp index 7dd3e2d8fc54..768d45cf4beb 100644 --- a/src/tasks/tasks.hpp +++ b/src/tasks/tasks.hpp @@ -399,8 +399,8 @@ class TaskRegion { TaskListStatus Execute(ThreadPool &pool) { // for now, require a pool with one thread - PARTHENON_REQUIRE_THROWS(pool.size() == 1, - "ThreadPool size != 1 is not currently supported.") + //PARTHENON_REQUIRE_THROWS(pool.size() == 1, + //"ThreadPool size != 1 is not currently supported.") // first, if needed, finish building the graph if (!graph_built) BuildGraph(); diff --git a/src/utils/thread_pool.hpp b/src/utils/thread_pool.hpp index 77a7ba66cac5..6191edbcd22e 100644 --- a/src/utils/thread_pool.hpp +++ b/src/utils/thread_pool.hpp @@ -39,19 +39,16 @@ class ThreadQueue { std::unique_lock lock(mutex); if (queue.empty()) { nwaiting++; - if (nwaiting == nworkers && started) { + if (waiting && nwaiting == nworkers) { complete = true; complete_cv.notify_all(); } cv.wait(lock, [this]() { return exit || !queue.empty(); }); nwaiting--; + if (exit) return true; } - started = true; - if (exit && complete && queue.empty()) return true; - if (!queue.empty()) { - q = queue.front(); - queue.pop(); - } + q = queue.front(); + queue.pop(); return false; } void signal_kill() { @@ -69,12 +66,19 @@ class ThreadQueue { } void wait_for_complete() { std::unique_lock lock(mutex); - if (complete) { + waiting = true; + if (queue.empty() && nwaiting == nworkers) { complete = false; + waiting = false; return; } complete_cv.wait(lock, [this]() { return complete; }); complete = false; + waiting = false; + } + size_t size() { + std::lock_guard lock(mutex); + return queue.size(); } private: @@ -86,7 +90,7 @@ class ThreadQueue { std::condition_variable complete_cv; bool complete = false; bool exit = false; - bool started = false; + bool waiting = false; }; class ThreadPool { @@ -128,6 +132,8 @@ class ThreadPool { int size() const { return nthreads; } + size_t num_queued() { return queue.size(); } + private: const int nthreads; std::vector threads; From e51f4db28b3653adc06048842367b1c3cda7bdbb Mon Sep 17 00:00:00 2001 From: jdolence Date: Fri, 5 Apr 2024 09:01:06 -0600 Subject: [PATCH 30/31] seems to be thread safe -- advection example works --- CMakeLists.txt | 4 +- example/advection/CMakeLists.txt | 4 +- example/advection/advection_package.cpp | 51 ++++---- src/CMakeLists.txt | 4 +- src/bvals/comms/boundary_communication.cpp | 28 ++++- src/bvals/comms/flux_correction.cpp | 21 ++++ src/driver/driver.hpp | 1 + src/tasks/tasks.hpp | 2 +- src/tasks/thread_pool.hpp | 139 --------------------- 9 files changed, 81 insertions(+), 173 deletions(-) delete mode 100644 src/tasks/thread_pool.hpp diff --git a/CMakeLists.txt b/CMakeLists.txt index 697e340f6016..73f4550f8d3b 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -202,8 +202,8 @@ endif() set(CMAKE_CXX_STANDARD_REQUIRED ON) set(CMAKE_CXX_STANDARD 17) -add_compile_options(-fsanitize=thread) -add_link_options(-fsanitize=thread) +#add_compile_options(-fsanitize=thread) +#add_link_options(-fsanitize=thread) option(PARTHENON_IMPORT_KOKKOS "If ON, attempt to link to an external Kokkos library. If OFF, build Kokkos from source and package with Parthenon" OFF) if (NOT TARGET Kokkos::kokkos) diff --git a/example/advection/CMakeLists.txt b/example/advection/CMakeLists.txt index a2174386a418..347f5a1b81f5 100644 --- a/example/advection/CMakeLists.txt +++ b/example/advection/CMakeLists.txt @@ -22,8 +22,8 @@ if( "advection-example" IN_LIST DRIVER_LIST OR NOT PARTHENON_DISABLE_EXAMPLES) main.cpp parthenon_app_inputs.cpp ) - add_compile_options(-fsanitize=thread) - add_link_options(-fsanitize=thread) + #add_compile_options(-fsanitize=thread) + #add_link_options(-fsanitize=thread) target_link_libraries(advection-example PRIVATE Parthenon::parthenon) lint_target(advection-example) endif() diff --git a/example/advection/advection_package.cpp b/example/advection/advection_package.cpp index 19795971a366..b4455f3e87c1 100644 --- a/example/advection/advection_package.cpp +++ b/example/advection/advection_package.cpp @@ -251,8 +251,9 @@ AmrTag CheckRefinement(MeshBlockData *rc) { IndexRange kb = pmb->cellbounds.GetBoundsK(IndexDomain::entire); typename Kokkos::MinMax::value_type minmax; - pmb->par_reduce( - PARTHENON_AUTO_LABEL, 0, v.GetMaxNumberOfVars() - 1, kb.s, kb.e, jb.s, jb.e, ib.s, ib.e, + parthenon::par_reduce( + parthenon::loop_pattern_mdrange_tag, PARTHENON_AUTO_LABEL, DevExecSpace(), + 0, v.GetMaxNumberOfVars() - 1, kb.s, kb.e, jb.s, jb.e, ib.s, ib.e, KOKKOS_LAMBDA(const int n, const int k, const int j, const int i, typename Kokkos::MinMax::value_type &lminmax) { lminmax.min_val = @@ -290,8 +291,9 @@ void PreFill(MeshBlockData *rc) { const int in = imap["advected"]; const int out = imap["one_minus_advected"]; const auto num_vars = rc->Get("advected").data.GetDim(4); - pmb->par_for( - PARTHENON_AUTO_LABEL, 0, num_vars - 1, kb.s, kb.e, jb.s, jb.e, ib.s, ib.e, + parthenon::par_for( + DEFAULT_LOOP_PATTERN, PARTHENON_AUTO_LABEL, DevExecSpace(), + 0, num_vars - 1, kb.s, kb.e, jb.s, jb.e, ib.s, ib.e, KOKKOS_LAMBDA(const int n, const int k, const int j, const int i) { v(0, out + n, k, j, i) = 1.0 - v(0, in + n, k, j, i); }); @@ -316,8 +318,9 @@ void SquareIt(MeshBlockData *rc) { const int in = imap["one_minus_advected"]; const int out = imap["one_minus_advected_sq"]; const auto num_vars = rc->Get("advected").data.GetDim(4); - pmb->par_for( - PARTHENON_AUTO_LABEL, 0, num_vars - 1, kb.s, kb.e, jb.s, jb.e, ib.s, ib.e, + parthenon::par_for( + DEFAULT_LOOP_PATTERN, PARTHENON_AUTO_LABEL, DevExecSpace(), + 0, num_vars - 1, kb.s, kb.e, jb.s, jb.e, ib.s, ib.e, KOKKOS_LAMBDA(const int n, const int k, const int j, const int i) { v(0, out + n, k, j, i) = v(0, in + n, k, j, i) * v(0, in + n, k, j, i); }); @@ -333,8 +336,9 @@ void SquareIt(MeshBlockData *rc) { const auto &profile = pkg->Param("profile"); if (profile == "smooth_gaussian") { const auto &advected = rc->Get("advected").data; - pmb->par_for( - PARTHENON_AUTO_LABEL, 0, num_vars - 1, kb.s, kb.e, jb.s, jb.e, ib.s, ib.e, + parthenon::par_for( + DEFAULT_LOOP_PATTERN, PARTHENON_AUTO_LABEL, DevExecSpace(), + 0, num_vars - 1, kb.s, kb.e, jb.s, jb.e, ib.s, ib.e, KOKKOS_LAMBDA(const int n, const int k, const int j, const int i) { PARTHENON_REQUIRE(advected(n, k, j, i) != 0.0, "Advected not properly initialized."); @@ -370,8 +374,9 @@ void PostFill(MeshBlockData *rc) { const int out12 = imap["one_minus_sqrt_one_minus_advected_sq_12"]; const int out37 = imap["one_minus_sqrt_one_minus_advected_sq_37"]; const auto num_vars = rc->Get("advected").data.GetDim(4); - pmb->par_for( - PARTHENON_AUTO_LABEL, 0, num_vars - 1, kb.s, kb.e, jb.s, jb.e, ib.s, ib.e, + parthenon::par_for( + DEFAULT_LOOP_PATTERN, PARTHENON_AUTO_LABEL, DevExecSpace(), + 0, num_vars - 1, kb.s, kb.e, jb.s, jb.e, ib.s, ib.e, KOKKOS_LAMBDA(const int n, const int k, const int j, const int i) { v(out12 + n, k, j, i) = 1.0 - sqrt(v(in + n, k, j, i)); v(out37 + n, k, j, i) = 1.0 - v(out12 + n, k, j, i); @@ -405,9 +410,9 @@ Real AdvectionHst(MeshData *md) { // weighting needs to be applied in the reduction region. const bool volume_weighting = std::is_same>::value; - pmb->par_reduce( - PARTHENON_AUTO_LABEL, 0, advected_pack.GetNBlocks() - 1, kb.s, kb.e, jb.s, jb.e, ib.s, - ib.e, + parthenon::par_reduce( + parthenon::loop_pattern_mdrange_tag, PARTHENON_AUTO_LABEL, DevExecSpace(), + 0, advected_pack.GetNBlocks() - 1, kb.s, kb.e, jb.s, jb.e, ib.s, ib.e, KOKKOS_LAMBDA(const int b, const int k, const int j, const int i, Real &lresult) { const auto &coords = advected_pack.GetCoordinates(b); // `join` is a function of the Kokkos::ReducerConecpt that allows to use the same @@ -437,8 +442,9 @@ Real EstimateTimestepBlock(MeshBlockData *rc) { // this is obviously overkill for this constant velocity problem Real min_dt; - pmb->par_reduce( - PARTHENON_AUTO_LABEL, kb.s, kb.e, jb.s, jb.e, ib.s, ib.e, + parthenon::par_reduce( + parthenon::loop_pattern_mdrange_tag, PARTHENON_AUTO_LABEL, DevExecSpace(), + kb.s, kb.e, jb.s, jb.e, ib.s, ib.e, KOKKOS_LAMBDA(const int k, const int j, const int i, Real &lmin_dt) { if (vx != 0.0) lmin_dt = std::min(lmin_dt, coords.Dxc(k, j, i) / std::abs(vx)); @@ -489,8 +495,9 @@ TaskStatus CalculateFluxes(std::shared_ptr> &rc) { const int nvar = v.GetMaxNumberOfVars(); size_t scratch_size_in_bytes = parthenon::ScratchPad2D::shmem_size(nvar, nx1); // get x-fluxes - pmb->par_for_outer( - PARTHENON_AUTO_LABEL, 2 * scratch_size_in_bytes, scratch_level, kb.s, kb.e, jb.s, + parthenon::par_for_outer( + DEFAULT_OUTER_LOOP_PATTERN, PARTHENON_AUTO_LABEL, DevExecSpace(), + 2 * scratch_size_in_bytes, scratch_level, kb.s, kb.e, jb.s, jb.e, KOKKOS_LAMBDA(parthenon::team_mbr_t member, const int k, const int j) { parthenon::ScratchPad2D ql(member.team_scratch(scratch_level), nvar, nx1); parthenon::ScratchPad2D qr(member.team_scratch(scratch_level), nvar, nx1); @@ -522,8 +529,9 @@ TaskStatus CalculateFluxes(std::shared_ptr> &rc) { // get y-fluxes if (pmb->pmy_mesh->ndim >= 2) { - pmb->par_for_outer( - PARTHENON_AUTO_LABEL, 3 * scratch_size_in_bytes, scratch_level, kb.s, kb.e, jb.s, + parthenon::par_for_outer( + DEFAULT_OUTER_LOOP_PATTERN, PARTHENON_AUTO_LABEL, DevExecSpace(), + 3 * scratch_size_in_bytes, scratch_level, kb.s, kb.e, jb.s, jb.e + 1, KOKKOS_LAMBDA(parthenon::team_mbr_t member, const int k, const int j) { // the overall algorithm/use of scratch pad here is clear inefficient and kept // just for demonstrating purposes. The key point is that we cannot reuse @@ -565,8 +573,9 @@ TaskStatus CalculateFluxes(std::shared_ptr> &rc) { // get z-fluxes if (pmb->pmy_mesh->ndim == 3) { - pmb->par_for_outer( - PARTHENON_AUTO_LABEL, 3 * scratch_size_in_bytes, scratch_level, kb.s, kb.e + 1, + parthenon::par_for_outer( + DEFAULT_OUTER_LOOP_PATTERN, PARTHENON_AUTO_LABEL, DevExecSpace(), + 3 * scratch_size_in_bytes, scratch_level, kb.s, kb.e + 1, jb.s, jb.e, KOKKOS_LAMBDA(parthenon::team_mbr_t member, const int k, const int j) { // the overall algorithm/use of scratch pad here is clear inefficient and kept diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index b0b24960b0fa..66d53482f094 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -321,8 +321,8 @@ if (Kokkos_ENABLE_CUDA AND NOT CMAKE_CXX_COMPILER_ID STREQUAL "Clang") target_compile_options(parthenon PUBLIC --expt-relaxed-constexpr) endif() -add_compile_options(-fsanitize=thread) -add_link_options(-fsanitize=thread) +#add_compile_options(-fsanitize=thread) +#add_link_options(-fsanitize=thread) target_link_libraries(parthenon PUBLIC Kokkos::kokkos Threads::Threads) diff --git a/src/bvals/comms/boundary_communication.cpp b/src/bvals/comms/boundary_communication.cpp index f4c4f9da5fe8..4e28d2e0a493 100644 --- a/src/bvals/comms/boundary_communication.cpp +++ b/src/bvals/comms/boundary_communication.cpp @@ -44,13 +44,14 @@ namespace parthenon { using namespace loops; using namespace loops::shorthands; -static std::mutex mutex; +static std::array mutex; template TaskStatus SendBoundBufs(std::shared_ptr> &md) { PARTHENON_INSTRUMENT - std::lock_guard lock(mutex); + int mutex_id = 2 * static_cast(bound_type) + 1; + mutex[mutex_id].lock(); Mesh *pmesh = md->GetMeshPointer(); auto &cache = md->GetBvarsCache().GetSubCache(bound_type, true); @@ -63,9 +64,11 @@ TaskStatus SendBoundBufs(std::shared_ptr> &md) { CheckSendBufferCacheForRebuild(md); if (nbound == 0) { + mutex[mutex_id].unlock(); return TaskStatus::complete; } if (other_communication_unfinished) { + mutex[mutex_id].unlock(); return TaskStatus::incomplete; } @@ -81,6 +84,9 @@ TaskStatus SendBoundBufs(std::shared_ptr> &md) { ProResInfo::GetSend); } } + + mutex[mutex_id].unlock(); + // Restrict auto pmb = md->GetBlockData(0)->GetBlockPointer(); StateDescriptor *resolved_packages = pmb->resolved_packages.get(); @@ -170,13 +176,15 @@ template TaskStatus StartReceiveBoundBufs(std::shared_ptr> &md) { PARTHENON_INSTRUMENT - std::lock_guard lock(mutex); + int mutex_id = 2 * static_cast(bound_type); + mutex[mutex_id].lock(); Mesh *pmesh = md->GetMeshPointer(); auto &cache = md->GetBvarsCache().GetSubCache(bound_type, false); if (cache.buf_vec.size() == 0) InitializeBufferCache(md, &(pmesh->boundary_comm_map), &cache, ReceiveKey, false); + mutex[mutex_id].unlock(); std::for_each(std::begin(cache.buf_vec), std::end(cache.buf_vec), [](auto pbuf) { pbuf->TryStartReceive(); }); @@ -199,13 +207,15 @@ template TaskStatus ReceiveBoundBufs(std::shared_ptr> &md) { PARTHENON_INSTRUMENT - std::lock_guard lock(mutex); + int mutex_id = 2 * static_cast(bound_type); + mutex[mutex_id].lock(); Mesh *pmesh = md->GetMeshPointer(); auto &cache = md->GetBvarsCache().GetSubCache(bound_type, false); if (cache.buf_vec.size() == 0) InitializeBufferCache(md, &(pmesh->boundary_comm_map), &cache, ReceiveKey, false); + mutex[mutex_id].unlock(); bool all_received = true; std::for_each( @@ -249,7 +259,8 @@ template TaskStatus SetBounds(std::shared_ptr> &md) { PARTHENON_INSTRUMENT - std::lock_guard lock(mutex); + int mutex_id = 2 * static_cast(bound_type); + mutex[mutex_id].lock(); Mesh *pmesh = md->GetMeshPointer(); auto &cache = md->GetBvarsCache().GetSubCache(bound_type, false); @@ -268,6 +279,9 @@ TaskStatus SetBounds(std::shared_ptr> &md) { ProResInfo::GetSet); } } + + mutex[mutex_id].unlock(); + // const Real threshold = Globals::sparse_config.allocation_threshold; auto &bnd_info = cache.bnd_info; Kokkos::parallel_for( @@ -344,7 +358,8 @@ template TaskStatus ProlongateBounds(std::shared_ptr> &md) { PARTHENON_INSTRUMENT - std::lock_guard lock(mutex); + int mutex_id = 2 * static_cast(bound_type); + mutex[mutex_id].lock(); Mesh *pmesh = md->GetMeshPointer(); auto &cache = md->GetBvarsCache().GetSubCache(bound_type, false); @@ -363,6 +378,7 @@ TaskStatus ProlongateBounds(std::shared_ptr> &md) { ProResInfo::GetSet); } } + mutex[mutex_id].unlock(); if (nbound > 0 && pmesh->multilevel) { auto pmb = md->GetBlockData(0)->GetBlockPointer(); diff --git a/src/bvals/comms/flux_correction.cpp b/src/bvals/comms/flux_correction.cpp index ff76bcba0014..e36b8dbf9267 100644 --- a/src/bvals/comms/flux_correction.cpp +++ b/src/bvals/comms/flux_correction.cpp @@ -38,9 +38,14 @@ namespace parthenon { using namespace impl; +static std::array mutex; + TaskStatus LoadAndSendFluxCorrections(std::shared_ptr> &md) { PARTHENON_INSTRUMENT + int mutex_id = 2 * static_cast(BoundaryType::flxcor_send) + 1; + mutex[mutex_id].lock(); + Mesh *pmesh = md->GetMeshPointer(); auto &cache = md->GetBvarsCache().GetSubCache(BoundaryType::flxcor_send, true); const int ndim = pmesh->ndim; @@ -53,16 +58,19 @@ TaskStatus LoadAndSendFluxCorrections(std::shared_ptr> &md) { CheckSendBufferCacheForRebuild(md); if (nbound == 0) { + mutex[mutex_id].unlock(); return TaskStatus::complete; } if (other_communication_unfinished) { + mutex[mutex_id].unlock(); return TaskStatus::incomplete; } if (rebuild) RebuildBufferCache( md, nbound, BndInfo::GetSendCCFluxCor, ProResInfo::GetSend); + mutex[mutex_id].unlock(); auto &bnd_info = cache.bnd_info; PARTHENON_REQUIRE(bnd_info.size() == nbound, "Need same size for boundary info"); @@ -119,11 +127,16 @@ TaskStatus LoadAndSendFluxCorrections(std::shared_ptr> &md) { TaskStatus StartReceiveFluxCorrections(std::shared_ptr> &md) { PARTHENON_INSTRUMENT + + int mutex_id = 2 * static_cast(BoundaryType::flxcor_recv); + mutex[mutex_id].lock(); + Mesh *pmesh = md->GetMeshPointer(); auto &cache = md->GetBvarsCache().GetSubCache(BoundaryType::flxcor_recv, false); if (cache.buf_vec.size() == 0) InitializeBufferCache( md, &(pmesh->boundary_comm_flxcor_map), &cache, ReceiveKey, false); + mutex[mutex_id].unlock(); std::for_each(std::begin(cache.buf_vec), std::end(cache.buf_vec), [](auto pbuf) { pbuf->TryStartReceive(); }); @@ -134,11 +147,15 @@ TaskStatus StartReceiveFluxCorrections(std::shared_ptr> &md) { TaskStatus ReceiveFluxCorrections(std::shared_ptr> &md) { PARTHENON_INSTRUMENT + int mutex_id = 2 * static_cast(BoundaryType::flxcor_recv); + mutex[mutex_id].lock(); + Mesh *pmesh = md->GetMeshPointer(); auto &cache = md->GetBvarsCache().GetSubCache(BoundaryType::flxcor_recv, false); if (cache.buf_vec.size() == 0) InitializeBufferCache( md, &(pmesh->boundary_comm_flxcor_map), &cache, ReceiveKey, false); + mutex[mutex_id].unlock(); bool all_received = true; std::for_each( @@ -152,6 +169,9 @@ TaskStatus ReceiveFluxCorrections(std::shared_ptr> &md) { TaskStatus SetFluxCorrections(std::shared_ptr> &md) { PARTHENON_INSTRUMENT + int mutex_id = 2 * static_cast(BoundaryType::flxcor_recv); + mutex[mutex_id].lock(); + Mesh *pmesh = md->GetMeshPointer(); auto &cache = md->GetBvarsCache().GetSubCache(BoundaryType::flxcor_recv, false); @@ -160,6 +180,7 @@ TaskStatus SetFluxCorrections(std::shared_ptr> &md) { if (rebuild) RebuildBufferCache( md, nbound, BndInfo::GetSetCCFluxCor, ProResInfo::GetSend); + mutex[mutex_id].unlock(); auto &bnd_info = cache.bnd_info; Kokkos::parallel_for( diff --git a/src/driver/driver.hpp b/src/driver/driver.hpp index 3934e92f77d7..b4d331283198 100644 --- a/src/driver/driver.hpp +++ b/src/driver/driver.hpp @@ -40,6 +40,7 @@ class Driver { Driver(ParameterInput *pin, ApplicationInput *app_in, Mesh *pm) : pinput(pin), app_input(app_in), pmesh(pm), mbcnt_prev(), time_LBandAMR() { const int nthreads = pin->GetOrAddInteger("parthenon/driver", "nthreads", 1); + printf("Initializing with %d threads...\n", nthreads); pool = std::make_shared(nthreads); pm->SetThreadPool(pool); } diff --git a/src/tasks/tasks.hpp b/src/tasks/tasks.hpp index b8f03a99ddf3..43ce64563058 100644 --- a/src/tasks/tasks.hpp +++ b/src/tasks/tasks.hpp @@ -27,8 +27,8 @@ #include #include -#include "thread_pool.hpp" #include "utils/error_checking.hpp" +#include "utils/thread_pool.hpp" namespace parthenon { diff --git a/src/tasks/thread_pool.hpp b/src/tasks/thread_pool.hpp deleted file mode 100644 index b8f526750230..000000000000 --- a/src/tasks/thread_pool.hpp +++ /dev/null @@ -1,139 +0,0 @@ -//======================================================================================== -// (C) (or copyright) 2023. Triad National Security, LLC. All rights reserved. -// -// This program was produced under U.S. Government contract 89233218CNA000001 for Los -// Alamos National Laboratory (LANL), which is operated by Triad National Security, LLC -// for the U.S. Department of Energy/National Nuclear Security Administration. All rights -// in the program are reserved by Triad National Security, LLC, and the U.S. Department -// of Energy/National Nuclear Security Administration. The Government is granted for -// itself and others acting on its behalf a nonexclusive, paid-up, irrevocable worldwide -// license in this material to reproduce, prepare derivative works, distribute copies to -// the public, perform publicly and display publicly, and to permit others to do so. -//======================================================================================== - -#ifndef TASKS_THREAD_POOL_HPP_ -#define TASKS_THREAD_POOL_HPP_ - -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace parthenon { - -template -class ThreadQueue { - public: - explicit ThreadQueue(const int num_workers) : nworkers(num_workers), nwaiting(0) {} - void push(T q) { - std::lock_guard lock(mutex); - queue.push(q); - cv.notify_one(); - } - bool pop(T &q) { - std::unique_lock lock(mutex); - if (queue.empty()) { - nwaiting++; - if (waiting && nwaiting == nworkers) { - complete = true; - complete_cv.notify_all(); - } - cv.wait(lock, [this]() { return exit || !queue.empty(); }); - nwaiting--; - if (exit) return true; - } - q = queue.front(); - queue.pop(); - return false; - } - void signal_kill() { - std::lock_guard lock(mutex); - std::queue().swap(queue); - complete = true; - exit = true; - cv.notify_all(); - } - void signal_exit_when_finished() { - std::lock_guard lock(mutex); - exit = true; - complete = true; - cv.notify_all(); - } - void wait_for_complete() { - std::unique_lock lock(mutex); - waiting = true; - if (queue.empty() && nwaiting == nworkers) { - complete = false; - waiting = false; - return; - } - complete_cv.wait(lock, [this]() { return complete; }); - complete = false; - waiting = false; - } - - private: - const int nworkers; - int nwaiting; - std::queue queue; - std::mutex mutex; - std::condition_variable cv; - std::condition_variable complete_cv; - bool complete = false; - bool exit = false; - bool waiting = false; -}; - -class ThreadPool { - public: - explicit ThreadPool(const int numthreads = std::thread::hardware_concurrency()) - : nthreads(numthreads), queue(nthreads) { - for (int i = 0; i < nthreads; i++) { - auto worker = [&]() { - while (true) { - std::function f; - auto stop = queue.pop(f); - if (stop) break; - if (f) f(); - } - }; - threads.emplace_back(worker); - } - } - ~ThreadPool() { - queue.signal_exit_when_finished(); - for (auto &t : threads) { - t.join(); - } - } - - void wait() { queue.wait_for_complete(); } - - void kill() { queue.signal_kill(); } - - template - std::future::type> enqueue(F &&f, Args &&...args) { - using return_t = typename std::result_of::type; - auto task = std::make_shared>( - [=, func = std::forward(f)] { return func(std::forward(args)...); }); - std::future result = task->get_future(); - queue.push([task]() { (*task)(); }); - return result; - } - - int size() const { return nthreads; } - - private: - const int nthreads; - std::vector threads; - ThreadQueue> queue; -}; - -} // namespace parthenon - -#endif // TASKS_THREAD_POOL_HPP_ From be1f029ec5b8c3b1ac5b8471207651b32f39c9af Mon Sep 17 00:00:00 2001 From: jdolence Date: Tue, 23 Apr 2024 14:19:01 -0600 Subject: [PATCH 31/31] crazy state --- CMakeLists.txt | 1 + example/advection/CMakeLists.txt | 1 + example/advection/advection_driver.cpp | 14 +- example/advection/advection_package.cpp | 7 +- external/Kokkos | 2 +- .../parthenon_tools/movie2d.py | 3 +- src/CMakeLists.txt | 1 + src/basic_types.hpp | 2 + src/bvals/comms/bnd_info.cpp | 17 +- src/bvals/comms/bnd_info.hpp | 3 + src/bvals/comms/boundary_communication.cpp | 38 +- src/bvals/comms/bvals_utils.hpp | 6 +- src/bvals/comms/flux_correction.cpp | 33 +- src/driver/driver.hpp | 5 +- src/kokkos_abstraction.hpp | 82 ++- src/mesh/amr_loadbalance.cpp | 80 ++- src/mesh/mesh.cpp | 7 +- src/mesh/mesh.hpp | 3 +- src/mesh/meshblock.hpp | 2 +- src/parthenon_manager.hpp | 3 +- src/tasks/tasks.hpp | 3 + src/utils/communication_buffer.hpp | 39 +- src/utils/sort.hpp | 4 +- src/utils/thread_pool.hpp | 532 ++++++++++++++++-- 24 files changed, 737 insertions(+), 151 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index 73f4550f8d3b..28c59c84fa8c 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -204,6 +204,7 @@ set(CMAKE_CXX_STANDARD 17) #add_compile_options(-fsanitize=thread) #add_link_options(-fsanitize=thread) +add_link_options(-Wl,-no_pie -L/opt/homebrew/lib -lprofiler -ltcmalloc) option(PARTHENON_IMPORT_KOKKOS "If ON, attempt to link to an external Kokkos library. If OFF, build Kokkos from source and package with Parthenon" OFF) if (NOT TARGET Kokkos::kokkos) diff --git a/example/advection/CMakeLists.txt b/example/advection/CMakeLists.txt index 347f5a1b81f5..d59e0b04414f 100644 --- a/example/advection/CMakeLists.txt +++ b/example/advection/CMakeLists.txt @@ -24,6 +24,7 @@ if( "advection-example" IN_LIST DRIVER_LIST OR NOT PARTHENON_DISABLE_EXAMPLES) ) #add_compile_options(-fsanitize=thread) #add_link_options(-fsanitize=thread) + add_link_options(-Wl,-no_pie -L/opt/homebrew/lib -lprofiler -ltcmalloc) target_link_libraries(advection-example PRIVATE Parthenon::parthenon) lint_target(advection-example) endif() diff --git a/example/advection/advection_driver.cpp b/example/advection/advection_driver.cpp index c9e2d5a2b435..d64bde97b8ae 100644 --- a/example/advection/advection_driver.cpp +++ b/example/advection/advection_driver.cpp @@ -86,8 +86,8 @@ TaskCollection AdvectionDriver::MakeTaskCollection(BlockList_t &blocks, const in const auto any = parthenon::BoundaryType::any; - tl.AddTask(none, parthenon::StartReceiveBoundBufs, mc1); - tl.AddTask(none, parthenon::StartReceiveFluxCorrections, mc0); + //tl.AddTask(none, parthenon::StartReceiveBoundBufs, mc1); + //tl.AddTask(none, parthenon::StartReceiveFluxCorrections, mc0); } // Number of task lists that can be executed independently and thus *may* @@ -124,13 +124,13 @@ TaskCollection AdvectionDriver::MakeTaskCollection(BlockList_t &blocks, const in auto &mc1 = pmesh->mesh_data.GetOrAdd(stage_name[stage], i); auto &mdudt = pmesh->mesh_data.GetOrAdd("dUdt", i); - auto send_flx = tl.AddTask(none, parthenon::LoadAndSendFluxCorrections, mc0); - auto recv_flx = tl.AddTask(none, parthenon::ReceiveFluxCorrections, mc0); - auto set_flx = tl.AddTask(recv_flx, parthenon::SetFluxCorrections, mc0); + //auto send_flx = tl.AddTask(none, parthenon::LoadAndSendFluxCorrections, mc0); + //auto recv_flx = tl.AddTask(none, parthenon::ReceiveFluxCorrections, mc0); + //auto set_flx = tl.AddTask(recv_flx, parthenon::SetFluxCorrections, mc0); // compute the divergence of fluxes of conserved variables auto flux_div = - tl.AddTask(set_flx, FluxDivergence>, mc0.get(), mdudt.get()); + tl.AddTask(none, FluxDivergence>, mc0.get(), mdudt.get()); auto avg_data = tl.AddTask(flux_div, AverageIndependentData>, mc0.get(), mbase.get(), beta); @@ -139,7 +139,7 @@ TaskCollection AdvectionDriver::MakeTaskCollection(BlockList_t &blocks, const in mdudt.get(), beta * dt, mc1.get()); // do boundary exchange - parthenon::AddBoundaryExchangeTasks(update, tl, mc1, pmesh->multilevel); + //parthenon::AddBoundaryExchangeTasks(update, tl, mc1, pmesh->multilevel); } TaskRegion &async_region2 = tc.AddRegion(num_task_lists_executed_independently); diff --git a/example/advection/advection_package.cpp b/example/advection/advection_package.cpp index b4455f3e87c1..ddce9effcee5 100644 --- a/example/advection/advection_package.cpp +++ b/example/advection/advection_package.cpp @@ -28,6 +28,7 @@ #include "kokkos_abstraction.hpp" #include "reconstruct/dc_inline.hpp" #include "utils/error_checking.hpp" +#include "utils/thread_pool.hpp" using namespace parthenon::package::prelude; @@ -75,6 +76,7 @@ std::shared_ptr Initialize(ParameterInput *pin) { auto fill_derived = pin->GetOrAddBoolean("Advection", "fill_derived", true); pkg->AddParam<>("fill_derived", fill_derived); + printf("fill_derived = %d\n", fill_derived); // For wavevector along coordinate axes, set desired values of ang_2/ang_3. // For example, for 1D problem use ang_2 = ang_3 = 0.0 @@ -314,7 +316,6 @@ void SquareIt(MeshBlockData *rc) { auto v = desc.GetPack(rc); auto imap = desc.GetMap(); - const int in = imap["one_minus_advected"]; const int out = imap["one_minus_advected_sq"]; const auto num_vars = rc->Get("advected").data.GetDim(4); @@ -495,6 +496,8 @@ TaskStatus CalculateFluxes(std::shared_ptr> &rc) { const int nvar = v.GetMaxNumberOfVars(); size_t scratch_size_in_bytes = parthenon::ScratchPad2D::shmem_size(nvar, nx1); // get x-fluxes + //std::cout << "hello from thread " << std::this_thread::get_id() << std::endl; + //for (int cnt=0; cnt<300; cnt++) { parthenon::par_for_outer( DEFAULT_OUTER_LOOP_PATTERN, PARTHENON_AUTO_LABEL, DevExecSpace(), 2 * scratch_size_in_bytes, scratch_level, kb.s, kb.e, jb.s, @@ -615,6 +618,8 @@ TaskStatus CalculateFluxes(std::shared_ptr> &rc) { } }); } + //} + //std::cout << "done on thread " << std::this_thread::get_id() << std::endl; return TaskStatus::complete; } diff --git a/external/Kokkos b/external/Kokkos index 62d2b6c879b7..486cc745cb9a 160000 --- a/external/Kokkos +++ b/external/Kokkos @@ -1 +1 @@ -Subproject commit 62d2b6c879b74b6ae7bd06eb3e5e80139c4708e6 +Subproject commit 486cc745cb9a287f3915061455105a3ee588c616 diff --git a/scripts/python/packages/parthenon_tools/parthenon_tools/movie2d.py b/scripts/python/packages/parthenon_tools/parthenon_tools/movie2d.py index 363f4d00eb66..25bb914efa01 100644 --- a/scripts/python/packages/parthenon_tools/parthenon_tools/movie2d.py +++ b/scripts/python/packages/parthenon_tools/parthenon_tools/movie2d.py @@ -247,7 +247,8 @@ def plot_dump( for i in range(n_blocks): # Plot the actual data, should work if parthenon/output*/ghost_zones = true/false # but obviously no ghost data will be shown if ghost_zones = false - p.pcolormesh(xf[i, :], yf[i, :], q[i, :, :], vmin=qmin, vmax=qmax) + # print(xf[i,:].shape, yf[i,:].shape, q[i,:,:].shape) + p.pcolormesh(xf[i, :], yf[i, :], np.squeeze(q[i, :, :]), vmin=qmin, vmax=qmax) # Print the block gid in the center of the block if len(block_ids) > 0: diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 66d53482f094..b7f06f704d9d 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -323,6 +323,7 @@ endif() #add_compile_options(-fsanitize=thread) #add_link_options(-fsanitize=thread) +add_link_options(-Wl,-no_pie -lprofiler -ltcmalloc) target_link_libraries(parthenon PUBLIC Kokkos::kokkos Threads::Threads) diff --git a/src/basic_types.hpp b/src/basic_types.hpp index 886788727d48..6d10f4dfa5b0 100644 --- a/src/basic_types.hpp +++ b/src/basic_types.hpp @@ -25,6 +25,8 @@ namespace parthenon { +inline thread_local Kokkos::Serial t_exec_space; + // primitive type alias that allows code to run with either floats or doubles #if SINGLE_PRECISION_ENABLED using Real = float; diff --git a/src/bvals/comms/bnd_info.cpp b/src/bvals/comms/bnd_info.cpp index b4a4072758bd..fcf47dd78150 100644 --- a/src/bvals/comms/bnd_info.cpp +++ b/src/bvals/comms/bnd_info.cpp @@ -220,6 +220,11 @@ int GetBufferSize(MeshBlock *pmb, const NeighborBlock &nb, v->GetDim(4) * topo_comp; } +void BndInfo::LockedBufferCopy(CommBuffer::owner_t> *input_buf) { + //std::lock_guard lock(mutex); + buf = input_buf->buffer(); +} + BndInfo BndInfo::GetSendBndInfo(MeshBlock *pmb, const NeighborBlock &nb, std::shared_ptr> v, CommBuffer::owner_t> *buf) { @@ -229,7 +234,8 @@ BndInfo BndInfo::GetSendBndInfo(MeshBlock *pmb, const NeighborBlock &nb, out.alloc_status = v->GetAllocationStatus(); if (!out.allocated) return out; - out.buf = buf->buffer(); + out.LockedBufferCopy(buf); + //out.buf = buf->buffer(); int Nv = v->GetDim(4); int Nu = v->GetDim(5); @@ -257,7 +263,8 @@ BndInfo BndInfo::GetSetBndInfo(MeshBlock *pmb, const NeighborBlock &nb, std::shared_ptr> v, CommBuffer::owner_t> *buf) { BndInfo out; - out.buf = buf->buffer(); + //out.buf = buf->buffer(); + out.LockedBufferCopy(buf); auto buf_state = buf->GetState(); if (buf_state == BufferState::received) { out.buf_allocated = true; @@ -454,7 +461,8 @@ BndInfo BndInfo::GetSendCCFluxCor(MeshBlock *pmb, const NeighborBlock &nb, // Not going to actually do anything with this buffer return out; } - out.buf = buf->buffer(); + //out.buf = buf->buffer(); + out.LockedBufferCopy(buf); IndexRange ib = pmb->cellbounds.GetBoundsI(IndexDomain::interior); IndexRange jb = pmb->cellbounds.GetBoundsJ(IndexDomain::interior); @@ -514,7 +522,8 @@ BndInfo BndInfo::GetSetCCFluxCor(MeshBlock *pmb, const NeighborBlock &nb, } out.allocated = true; out.alloc_status = v->GetAllocationStatus(); - out.buf = buf->buffer(); + //out.buf = buf->buffer(); + out.LockedBufferCopy(buf); IndexRange ib = pmb->cellbounds.GetBoundsI(IndexDomain::interior); IndexRange jb = pmb->cellbounds.GetBoundsJ(IndexDomain::interior); diff --git a/src/bvals/comms/bnd_info.hpp b/src/bvals/comms/bnd_info.hpp index 4860bdc7245f..bbc74ed67f71 100644 --- a/src/bvals/comms/bnd_info.hpp +++ b/src/bvals/comms/bnd_info.hpp @@ -56,6 +56,9 @@ struct BndInfo { BndInfo() = default; BndInfo(const BndInfo &) = default; + inline static std::mutex mutex; + void LockedBufferCopy(CommBuffer::owner_t> *buf); + // These are are used to generate the BndInfo struct for various // kinds of boundary types and operations. static BndInfo GetSendBndInfo(MeshBlock *pmb, const NeighborBlock &nb, diff --git a/src/bvals/comms/boundary_communication.cpp b/src/bvals/comms/boundary_communication.cpp index 4e28d2e0a493..4a84f432a16f 100644 --- a/src/bvals/comms/boundary_communication.cpp +++ b/src/bvals/comms/boundary_communication.cpp @@ -45,13 +45,15 @@ using namespace loops; using namespace loops::shorthands; static std::array mutex; +//static std::mutex mutex; template TaskStatus SendBoundBufs(std::shared_ptr> &md) { PARTHENON_INSTRUMENT int mutex_id = 2 * static_cast(bound_type) + 1; - mutex[mutex_id].lock(); + //mutex[mutex_id].lock(); + //mutex.lock(); Mesh *pmesh = md->GetMeshPointer(); auto &cache = md->GetBvarsCache().GetSubCache(bound_type, true); @@ -64,11 +66,13 @@ TaskStatus SendBoundBufs(std::shared_ptr> &md) { CheckSendBufferCacheForRebuild(md); if (nbound == 0) { - mutex[mutex_id].unlock(); + //mutex[mutex_id].unlock(); + //mutex.unlock(); return TaskStatus::complete; } if (other_communication_unfinished) { - mutex[mutex_id].unlock(); + //mutex[mutex_id].unlock(); + //mutex.unlock(); return TaskStatus::incomplete; } @@ -85,7 +89,8 @@ TaskStatus SendBoundBufs(std::shared_ptr> &md) { } } - mutex[mutex_id].unlock(); + //mutex[mutex_id].unlock(); + //mutex.unlock(); // Restrict auto pmb = md->GetBlockData(0)->GetBlockPointer(); @@ -159,6 +164,7 @@ TaskStatus SendBoundBufs(std::shared_ptr> &md) { else buf.SendNull(); } + //mutex.unlock(); return TaskStatus::complete; } @@ -177,14 +183,16 @@ TaskStatus StartReceiveBoundBufs(std::shared_ptr> &md) { PARTHENON_INSTRUMENT int mutex_id = 2 * static_cast(bound_type); - mutex[mutex_id].lock(); + //mutex[mutex_id].lock(); + //mutex.lock(); Mesh *pmesh = md->GetMeshPointer(); auto &cache = md->GetBvarsCache().GetSubCache(bound_type, false); if (cache.buf_vec.size() == 0) InitializeBufferCache(md, &(pmesh->boundary_comm_map), &cache, ReceiveKey, false); - mutex[mutex_id].unlock(); + //mutex[mutex_id].unlock(); + //mutex.unlock(); std::for_each(std::begin(cache.buf_vec), std::end(cache.buf_vec), [](auto pbuf) { pbuf->TryStartReceive(); }); @@ -208,14 +216,16 @@ TaskStatus ReceiveBoundBufs(std::shared_ptr> &md) { PARTHENON_INSTRUMENT int mutex_id = 2 * static_cast(bound_type); - mutex[mutex_id].lock(); + //mutex[mutex_id].lock(); + //mutex.lock(); Mesh *pmesh = md->GetMeshPointer(); auto &cache = md->GetBvarsCache().GetSubCache(bound_type, false); if (cache.buf_vec.size() == 0) InitializeBufferCache(md, &(pmesh->boundary_comm_map), &cache, ReceiveKey, false); - mutex[mutex_id].unlock(); + //mutex[mutex_id].unlock(); + //mutex.unlock(); bool all_received = true; std::for_each( @@ -260,7 +270,8 @@ TaskStatus SetBounds(std::shared_ptr> &md) { PARTHENON_INSTRUMENT int mutex_id = 2 * static_cast(bound_type); - mutex[mutex_id].lock(); + //mutex[mutex_id].lock(); + //mutex.lock(); Mesh *pmesh = md->GetMeshPointer(); auto &cache = md->GetBvarsCache().GetSubCache(bound_type, false); @@ -280,7 +291,8 @@ TaskStatus SetBounds(std::shared_ptr> &md) { } } - mutex[mutex_id].unlock(); + //mutex[mutex_id].unlock(); + //mutex.unlock(); // const Real threshold = Globals::sparse_config.allocation_threshold; auto &bnd_info = cache.bnd_info; @@ -359,7 +371,8 @@ TaskStatus ProlongateBounds(std::shared_ptr> &md) { PARTHENON_INSTRUMENT int mutex_id = 2 * static_cast(bound_type); - mutex[mutex_id].lock(); + //mutex[mutex_id].lock(); + //mutex.lock(); Mesh *pmesh = md->GetMeshPointer(); auto &cache = md->GetBvarsCache().GetSubCache(bound_type, false); @@ -378,7 +391,8 @@ TaskStatus ProlongateBounds(std::shared_ptr> &md) { ProResInfo::GetSet); } } - mutex[mutex_id].unlock(); + //mutex[mutex_id].unlock(); + //mutex.unlock(); if (nbound > 0 && pmesh->multilevel) { auto pmb = md->GetBlockData(0)->GetBlockPointer(); diff --git a/src/bvals/comms/bvals_utils.hpp b/src/bvals/comms/bvals_utils.hpp index bdb5e95dc4c2..a9a9b3392d15 100644 --- a/src/bvals/comms/bvals_utils.hpp +++ b/src/bvals/comms/bvals_utils.hpp @@ -95,9 +95,9 @@ void InitializeBufferCache(std::shared_ptr> &md, COMM_MAP *comm_m // Or, what the hell, you could put them in random order if you want, which // frighteningly seems to run faster in some cases - std::random_device rd; - std::mt19937 g(rd()); - std::shuffle(key_order.begin(), key_order.end(), g); + //std::random_device rd; + //std::mt19937 g(rd()); + //std::shuffle(key_order.begin(), key_order.end(), g); int buff_idx = 0; pcache->buf_vec.clear(); diff --git a/src/bvals/comms/flux_correction.cpp b/src/bvals/comms/flux_correction.cpp index e36b8dbf9267..9240be85e5ee 100644 --- a/src/bvals/comms/flux_correction.cpp +++ b/src/bvals/comms/flux_correction.cpp @@ -39,12 +39,14 @@ namespace parthenon { using namespace impl; static std::array mutex; +//static std::mutex mutex; TaskStatus LoadAndSendFluxCorrections(std::shared_ptr> &md) { PARTHENON_INSTRUMENT int mutex_id = 2 * static_cast(BoundaryType::flxcor_send) + 1; - mutex[mutex_id].lock(); + //mutex[mutex_id].lock(); + //mutex.lock(); Mesh *pmesh = md->GetMeshPointer(); auto &cache = md->GetBvarsCache().GetSubCache(BoundaryType::flxcor_send, true); @@ -58,19 +60,22 @@ TaskStatus LoadAndSendFluxCorrections(std::shared_ptr> &md) { CheckSendBufferCacheForRebuild(md); if (nbound == 0) { - mutex[mutex_id].unlock(); + //mutex[mutex_id].unlock(); + //mutex.unlock(); return TaskStatus::complete; } if (other_communication_unfinished) { - mutex[mutex_id].unlock(); + //mutex[mutex_id].unlock(); + //mutex.unlock(); return TaskStatus::incomplete; } if (rebuild) RebuildBufferCache( md, nbound, BndInfo::GetSendCCFluxCor, ProResInfo::GetSend); - mutex[mutex_id].unlock(); + //mutex[mutex_id].unlock(); + //mutex.unlock(); auto &bnd_info = cache.bnd_info; PARTHENON_REQUIRE(bnd_info.size() == nbound, "Need same size for boundary info"); @@ -122,6 +127,7 @@ TaskStatus LoadAndSendFluxCorrections(std::shared_ptr> &md) { // Calling Send will send null if the underlying buffer is unallocated for (auto &buf : cache.buf_vec) buf->Send(); + //mutex.unlock(); return TaskStatus::complete; } @@ -129,14 +135,16 @@ TaskStatus StartReceiveFluxCorrections(std::shared_ptr> &md) { PARTHENON_INSTRUMENT int mutex_id = 2 * static_cast(BoundaryType::flxcor_recv); - mutex[mutex_id].lock(); + //mutex[mutex_id].lock(); + //mutex.lock(); Mesh *pmesh = md->GetMeshPointer(); auto &cache = md->GetBvarsCache().GetSubCache(BoundaryType::flxcor_recv, false); if (cache.buf_vec.size() == 0) InitializeBufferCache( md, &(pmesh->boundary_comm_flxcor_map), &cache, ReceiveKey, false); - mutex[mutex_id].unlock(); + //mutex[mutex_id].unlock(); + //mutex.unlock(); std::for_each(std::begin(cache.buf_vec), std::end(cache.buf_vec), [](auto pbuf) { pbuf->TryStartReceive(); }); @@ -148,14 +156,16 @@ TaskStatus ReceiveFluxCorrections(std::shared_ptr> &md) { PARTHENON_INSTRUMENT int mutex_id = 2 * static_cast(BoundaryType::flxcor_recv); - mutex[mutex_id].lock(); + //mutex[mutex_id].lock(); + //mutex.lock(); Mesh *pmesh = md->GetMeshPointer(); auto &cache = md->GetBvarsCache().GetSubCache(BoundaryType::flxcor_recv, false); if (cache.buf_vec.size() == 0) InitializeBufferCache( md, &(pmesh->boundary_comm_flxcor_map), &cache, ReceiveKey, false); - mutex[mutex_id].unlock(); + //mutex[mutex_id].unlock(); + //mutex.unlock(); bool all_received = true; std::for_each( @@ -170,7 +180,8 @@ TaskStatus SetFluxCorrections(std::shared_ptr> &md) { PARTHENON_INSTRUMENT int mutex_id = 2 * static_cast(BoundaryType::flxcor_recv); - mutex[mutex_id].lock(); + //mutex[mutex_id].lock(); + //mutex.lock(); Mesh *pmesh = md->GetMeshPointer(); auto &cache = md->GetBvarsCache().GetSubCache(BoundaryType::flxcor_recv, false); @@ -180,7 +191,8 @@ TaskStatus SetFluxCorrections(std::shared_ptr> &md) { if (rebuild) RebuildBufferCache( md, nbound, BndInfo::GetSetCCFluxCor, ProResInfo::GetSend); - mutex[mutex_id].unlock(); + //mutex[mutex_id].unlock(); + //mutex.unlock(); auto &bnd_info = cache.bnd_info; Kokkos::parallel_for( @@ -203,6 +215,7 @@ TaskStatus SetFluxCorrections(std::shared_ptr> &md) { std::for_each(std::begin(cache.buf_vec), std::end(cache.buf_vec), [](auto pbuf) { pbuf->Stale(); }); + //mutex.unlock(); return TaskStatus::complete; } diff --git a/src/driver/driver.hpp b/src/driver/driver.hpp index b4d331283198..2fd27f5d21fd 100644 --- a/src/driver/driver.hpp +++ b/src/driver/driver.hpp @@ -39,10 +39,7 @@ class Driver { public: Driver(ParameterInput *pin, ApplicationInput *app_in, Mesh *pm) : pinput(pin), app_input(app_in), pmesh(pm), mbcnt_prev(), time_LBandAMR() { - const int nthreads = pin->GetOrAddInteger("parthenon/driver", "nthreads", 1); - printf("Initializing with %d threads...\n", nthreads); - pool = std::make_shared(nthreads); - pm->SetThreadPool(pool); + pool = pm->pool; } virtual DriverStatus Execute() = 0; void InitializeOutputs() { pouts = std::make_unique(pmesh, pinput); } diff --git a/src/kokkos_abstraction.hpp b/src/kokkos_abstraction.hpp index c2cc09ce879f..6a2a5e40c033 100644 --- a/src/kokkos_abstraction.hpp +++ b/src/kokkos_abstraction.hpp @@ -43,9 +43,12 @@ using DevExecSpace = Kokkos::Cuda; #else using DevMemSpace = Kokkos::DefaultExecutionSpace::memory_space; using HostMemSpace = Kokkos::HostSpace; -using DevExecSpace = Kokkos::DefaultExecutionSpace; +using DevExecSpace_t = Kokkos::DefaultExecutionSpace; +inline Kokkos::Serial DevExecSpace() { + return t_exec_space; +} #endif -using ScratchMemSpace = DevExecSpace::scratch_memory_space; +using ScratchMemSpace = DevExecSpace_t::scratch_memory_space; using HostExecSpace = Kokkos::DefaultHostExecutionSpace; using LayoutWrapper = Kokkos::LayoutRight; @@ -214,10 +217,21 @@ inline void kokkos_dispatch(ParallelScanDispatch, Args &&...args) { } // namespace dispatch_impl +class ExecSpace { + public: + explicit ExecSpace(const int npartitions) { + std::vector weights(npartitions, 1); + partitions = Kokkos::Experimental::partition_space(DevExecSpace_t(), weights); + } + Kokkos::DefaultExecutionSpace& operator[](const int i) { return partitions[i]; } + private: + std::vector partitions; +}; + // 1D loop using RangePolicy loops template inline typename std::enable_if::type -par_dispatch(LoopPatternFlatRange, const std::string &name, DevExecSpace exec_space, +par_dispatch(LoopPatternFlatRange, const std::string &name, DevExecSpace_t exec_space, const int &il, const int &iu, const Function &function, Args &&...args) { Tag tag; kokkos_dispatch(tag, name, @@ -230,7 +244,7 @@ par_dispatch(LoopPatternFlatRange, const std::string &name, DevExecSpace exec_sp // 2D loop using MDRange loops template inline typename std::enable_if::type -par_dispatch(LoopPatternMDRange, const std::string &name, DevExecSpace exec_space, +par_dispatch(LoopPatternMDRange, const std::string &name, DevExecSpace_t exec_space, const int jl, const int ju, const int il, const int iu, const Function &function, Args &&...args) { Tag tag; @@ -245,7 +259,7 @@ par_dispatch(LoopPatternMDRange, const std::string &name, DevExecSpace exec_spac // 3D loop using Kokkos 1D Range template inline typename std::enable_if::type -par_dispatch(LoopPatternFlatRange, const std::string &name, DevExecSpace exec_space, +par_dispatch(LoopPatternFlatRange, const std::string &name, DevExecSpace_t exec_space, const int kl, const int ku, const int jl, const int ju, const int il, const int iu, const Function &function, Args &&...args) { Tag tag; @@ -271,7 +285,7 @@ par_dispatch(LoopPatternFlatRange, const std::string &name, DevExecSpace exec_sp // 3D loop using MDRange loops template inline typename std::enable_if::type -par_dispatch(LoopPatternMDRange, const std::string &name, DevExecSpace exec_space, +par_dispatch(LoopPatternMDRange, const std::string &name, DevExecSpace_t exec_space, const int &kl, const int &ku, const int &jl, const int &ju, const int &il, const int &iu, const Function &function, Args &&...args) { Tag tag; @@ -287,7 +301,7 @@ par_dispatch(LoopPatternMDRange, const std::string &name, DevExecSpace exec_spac // 3D loop using TeamPolicy with single inner TeamThreadRange template inline void par_dispatch(LoopPatternTPTTR, const std::string &name, - DevExecSpace exec_space, const int &kl, const int &ku, + DevExecSpace_t exec_space, const int &kl, const int &ku, const int &jl, const int &ju, const int &il, const int &iu, const Function &function) { const int Nk = ku - kl + 1; @@ -306,7 +320,7 @@ inline void par_dispatch(LoopPatternTPTTR, const std::string &name, // 3D loop using TeamPolicy with single inner ThreadVectorRange template inline void par_dispatch(LoopPatternTPTVR, const std::string &name, - DevExecSpace exec_space, const int &kl, const int &ku, + DevExecSpace_t exec_space, const int &kl, const int &ku, const int &jl, const int &ju, const int &il, const int &iu, const Function &function) { // TODO(pgrete) if exec space is Cuda,throw error @@ -326,7 +340,7 @@ inline void par_dispatch(LoopPatternTPTVR, const std::string &name, // 3D loop using TeamPolicy with nested TeamThreadRange and ThreadVectorRange template inline void par_dispatch(LoopPatternTPTTRTVR, const std::string &name, - DevExecSpace exec_space, const int &kl, const int &ku, + DevExecSpace_t exec_space, const int &kl, const int &ku, const int &jl, const int &ju, const int &il, const int &iu, const Function &function) { const int Nk = ku - kl + 1; @@ -345,7 +359,7 @@ inline void par_dispatch(LoopPatternTPTTRTVR, const std::string &name, // 3D loop using SIMD FOR loops template inline void par_dispatch(LoopPatternSimdFor, const std::string &name, - DevExecSpace exec_space, const int &kl, const int &ku, + DevExecSpace_t exec_space, const int &kl, const int &ku, const int &jl, const int &ju, const int &il, const int &iu, const Function &function) { PARTHENON_INSTRUMENT_REGION(name) @@ -359,7 +373,7 @@ inline void par_dispatch(LoopPatternSimdFor, const std::string &name, // 4D loop using Kokkos 1D Range template inline typename std::enable_if::type -par_dispatch(LoopPatternFlatRange, const std::string &name, DevExecSpace exec_space, +par_dispatch(LoopPatternFlatRange, const std::string &name, DevExecSpace_t exec_space, const int nl, const int nu, const int kl, const int ku, const int jl, const int ju, const int il, const int iu, const Function &function, Args &&...args) { @@ -390,7 +404,7 @@ par_dispatch(LoopPatternFlatRange, const std::string &name, DevExecSpace exec_sp // 4D loop using MDRange loops template inline typename std::enable_if::type -par_dispatch(LoopPatternMDRange, const std::string &name, DevExecSpace exec_space, +par_dispatch(LoopPatternMDRange, const std::string &name, DevExecSpace_t exec_space, const int nl, const int nu, const int kl, const int ku, const int jl, const int ju, const int il, const int iu, const Function &function, Args &&...args) { @@ -407,7 +421,7 @@ par_dispatch(LoopPatternMDRange, const std::string &name, DevExecSpace exec_spac // 4D loop using TeamPolicy loop with inner TeamThreadRange template inline void par_dispatch(LoopPatternTPTTR, const std::string &name, - DevExecSpace exec_space, const int nl, const int nu, + DevExecSpace_t exec_space, const int nl, const int nu, const int kl, const int ku, const int jl, const int ju, const int il, const int iu, const Function &function) { const int Nn = nu - nl + 1; @@ -431,7 +445,7 @@ inline void par_dispatch(LoopPatternTPTTR, const std::string &name, // 4D loop using TeamPolicy loop with inner ThreadVectorRange template inline void par_dispatch(LoopPatternTPTVR, const std::string &name, - DevExecSpace exec_space, const int nl, const int nu, + DevExecSpace_t exec_space, const int nl, const int nu, const int kl, const int ku, const int jl, const int ju, const int il, const int iu, const Function &function) { // TODO(pgrete) if exec space is Cuda,throw error @@ -456,7 +470,7 @@ inline void par_dispatch(LoopPatternTPTVR, const std::string &name, // 4D loop using TeamPolicy with nested TeamThreadRange and ThreadVectorRange template inline void par_dispatch(LoopPatternTPTTRTVR, const std::string &name, - DevExecSpace exec_space, const int nl, const int nu, + DevExecSpace_t exec_space, const int nl, const int nu, const int kl, const int ku, const int jl, const int ju, const int il, const int iu, const Function &function) { const int Nn = nu - nl + 1; @@ -478,7 +492,7 @@ inline void par_dispatch(LoopPatternTPTTRTVR, const std::string &name, // 4D loop using SIMD FOR loops template inline void par_dispatch(LoopPatternSimdFor, const std::string &name, - DevExecSpace exec_space, const int nl, const int nu, + DevExecSpace_t exec_space, const int nl, const int nu, const int kl, const int ku, const int jl, const int ju, const int il, const int iu, const Function &function) { PARTHENON_INSTRUMENT_REGION(name) @@ -493,7 +507,7 @@ inline void par_dispatch(LoopPatternSimdFor, const std::string &name, // 5D loop using MDRange loops template inline typename std::enable_if::type -par_dispatch(LoopPatternMDRange, const std::string &name, DevExecSpace exec_space, +par_dispatch(LoopPatternMDRange, const std::string &name, DevExecSpace_t exec_space, const int ml, const int mu, const int nl, const int nu, const int kl, const int ku, const int jl, const int ju, const int il, const int iu, const Function &function, Args &&...args) { @@ -511,7 +525,7 @@ par_dispatch(LoopPatternMDRange, const std::string &name, DevExecSpace exec_spac // 5D loop using Kokkos 1D Range template inline void par_dispatch(LoopPatternFlatRange, const std::string &name, - DevExecSpace exec_space, const int bl, const int bu, + DevExecSpace_t exec_space, const int bl, const int bu, const int nl, const int nu, const int kl, const int ku, const int jl, const int ju, const int il, const int iu, const Function &function) { @@ -544,7 +558,7 @@ inline void par_dispatch(LoopPatternFlatRange, const std::string &name, // 5D loop using SIMD FOR loops template inline void par_dispatch(LoopPatternSimdFor, const std::string &name, - DevExecSpace exec_space, const int bl, const int bu, + DevExecSpace_t exec_space, const int bl, const int bu, const int nl, const int nu, const int kl, const int ku, const int jl, const int ju, const int il, const int iu, const Function &function) { @@ -561,7 +575,7 @@ inline void par_dispatch(LoopPatternSimdFor, const std::string &name, // 6D loop using MDRange loops template inline typename std::enable_if::type -par_dispatch(LoopPatternMDRange, const std::string &name, DevExecSpace exec_space, +par_dispatch(LoopPatternMDRange, const std::string &name, DevExecSpace_t exec_space, const int ll, const int lu, const int ml, const int mu, const int nl, const int nu, const int kl, const int ku, const int jl, const int ju, const int il, const int iu, const Function &function, Args &&...args) { @@ -579,7 +593,7 @@ par_dispatch(LoopPatternMDRange, const std::string &name, DevExecSpace exec_spac // 6D loop using Kokkos 1D Range template inline void par_dispatch(LoopPatternFlatRange, const std::string &name, - DevExecSpace exec_space, const int ll, const int lu, + DevExecSpace_t exec_space, const int ll, const int lu, const int ml, const int mu, const int nl, const int nu, const int kl, const int ku, const int jl, const int ju, const int il, const int iu, const Function &function) { @@ -616,7 +630,7 @@ inline void par_dispatch(LoopPatternFlatRange, const std::string &name, // 6D loop using SIMD FOR loops template inline void par_dispatch(LoopPatternSimdFor, const std::string &name, - DevExecSpace exec_space, const int ll, const int lu, + DevExecSpace_t exec_space, const int ll, const int lu, const int ml, const int mu, const int nl, const int nu, const int kl, const int ku, const int jl, const int ju, const int il, const int iu, const Function &function) { @@ -649,7 +663,7 @@ inline void par_scan(Args &&...args) { // 1D outer parallel loop using Kokkos Teams template inline void par_for_outer(OuterLoopPatternTeams, const std::string &name, - DevExecSpace exec_space, size_t scratch_size_in_bytes, + DevExecSpace_t exec_space, size_t scratch_size_in_bytes, const int scratch_level, const int kl, const int ku, const Function &function) { const int Nk = ku + 1 - kl; @@ -668,7 +682,7 @@ inline void par_for_outer(OuterLoopPatternTeams, const std::string &name, // 2D outer parallel loop using Kokkos Teams template inline void par_for_outer(OuterLoopPatternTeams, const std::string &name, - DevExecSpace exec_space, size_t scratch_size_in_bytes, + DevExecSpace_t exec_space, size_t scratch_size_in_bytes, const int scratch_level, const int kl, const int ku, const int jl, const int ju, const Function &function) { const int Nk = ku + 1 - kl; @@ -690,7 +704,7 @@ inline void par_for_outer(OuterLoopPatternTeams, const std::string &name, // 3D outer parallel loop using Kokkos Teams template inline void par_for_outer(OuterLoopPatternTeams, const std::string &name, - DevExecSpace exec_space, size_t scratch_size_in_bytes, + DevExecSpace_t exec_space, size_t scratch_size_in_bytes, const int scratch_level, const int nl, const int nu, const int kl, const int ku, const int jl, const int ju, const Function &function) { @@ -715,6 +729,20 @@ inline void par_for_outer(OuterLoopPatternTeams, const std::string &name, }); } +template +inline void par_for_outer_stupid(OuterLoopPatternTeams, const std::string &name, + DevExecSpace_t exec_space, const int nl, const int nu, + const int kl, const int ku, const int jl, const int ju, + const Function &function) { + for (int n = nl; n <= nu; n++) { + for (int k = kl; k <= ku; k++) { + for (int j = jl; j <= ju; j++) { + function(n, k, j); + } + } + } +} + // Inner parallel loop using TeamThreadRange template KOKKOS_FORCEINLINE_FUNCTION void @@ -944,7 +972,7 @@ struct DeviceDeleter { } }; -template +template std::unique_ptr> DeviceAllocate() { static_assert(std::is_trivially_destructible::value, "DeviceAllocate only supports trivially destructible classes!"); @@ -957,7 +985,7 @@ std::unique_ptr> DeviceAllocate() { return up; } -template +template std::unique_ptr> DeviceCopy(const T &host_object) { static_assert(std::is_trivially_destructible::value, "DeviceCopy only supports trivially destructible classes!"); diff --git a/src/mesh/amr_loadbalance.cpp b/src/mesh/amr_loadbalance.cpp index 6359dbee3e89..92b07f283765 100644 --- a/src/mesh/amr_loadbalance.cpp +++ b/src/mesh/amr_loadbalance.cpp @@ -42,6 +42,7 @@ #include "parthenon_arrays.hpp" #include "utils/buffer_utils.hpp" #include "utils/error_checking.hpp" +#include "utils/thread_pool.hpp" namespace parthenon { @@ -650,6 +651,12 @@ bool Mesh::GatherCostListAndCheckBalance() { return true; } +void thread_safe_insert(std::unordered_set &myset, LogicalLocation &myval) { + static std::mutex mutex; + std::lock_guard lock(mutex); + myset.insert(myval); +} + //---------------------------------------------------------------------------------------- // \!fn void Mesh::RedistributeAndRefineMeshBlocks(ParameterInput *pin, int ntot) // \brief redistribute MeshBlocks according to the new load balance @@ -682,8 +689,10 @@ void Mesh::RedistributeAndRefineMeshBlocks(ParameterInput *pin, ApplicationInput PARTHENON_INSTRUMENT newloc = forest.GetMeshBlockListAndResolveGids(); nbtotal = newloc.size(); - for (int ib = 0; ib < nbtotal; ++ib) - newtoold[ib] = forest.GetOldGid(newloc[ib]); + + ThreadPoolLoopBlocking(*pool, 0, nbtotal-1, [&](const int i) { + newtoold[i] = forest.GetOldGid(newloc[i]); + }); // create a list mapping the previous gid to the current one oldtonew[0] = 0; @@ -701,17 +710,19 @@ void Mesh::RedistributeAndRefineMeshBlocks(ParameterInput *pin, ApplicationInput for (; mb_idx < nbtold; mb_idx++) oldtonew[mb_idx] = ntot - 1; - current_level = 0; - for (int n = 0; n < ntot; n++) { + //current_level = 0; + //for (int n = 0; n < ntot; n++) { + current_level = ThreadPoolReduce(*pool, 0, ntot-1, [&](const int n) { // "on" = "old n" = "old gid" = "old global MeshBlock ID" int on = newtoold[n]; - if (newloc[n].level() > current_level) // set the current max level - current_level = newloc[n].level(); + //if (newloc[n].level() > current_level) // set the current max level + // current_level = newloc[n].level(); if (newloc[n].level() >= loclist[on].level()) { // same or refined newcost[n] = costlist[on]; // Keep a list of all blocks refined for below if (newloc[n].level() > loclist[on].level()) { - newly_refined.insert(newloc[n]); + thread_safe_insert(newly_refined, newloc[n]); + //newly_refined.insert(newloc[n]); } } else { double acost = 0.0; @@ -719,7 +730,8 @@ void Mesh::RedistributeAndRefineMeshBlocks(ParameterInput *pin, ApplicationInput acost += costlist[on + l]; newcost[n] = acost / nleaf; } - } + return newloc[n].level(); + }, [](int a, int b) { return std::max(a,b); }, int(0)); } // Construct new list region // Calculate new load balance @@ -729,30 +741,39 @@ void Mesh::RedistributeAndRefineMeshBlocks(ParameterInput *pin, ApplicationInput int nbe = nbs + nblist[Globals::my_rank] - 1; // Restrict fine to coarse buffers - ProResCache_t restriction_cache; - int nrestrict = 0; - for (int on = onbs; on <= onbe; on++) { - int nn = oldtonew[on]; - auto pmb = FindMeshBlock(on); - if (newloc[nn].level() < loclist[on].level()) nrestrict += pmb->vars_cc_.size(); - } - restriction_cache.Initialize(nrestrict, resolved_packages.get()); - int irestrict = 0; - for (int on = onbs; on <= onbe; on++) { - int nn = oldtonew[on]; - if (newloc[nn].level() < loclist[on].level()) { + auto restrict_fine_to_coarse_buffers = [&](const int t_onbs, const int t_onbe) { + ProResCache_t restriction_cache; + int nrestrict = 0; + for (int on = t_onbs; on <= t_onbe; on++) { + int nn = oldtonew[on]; auto pmb = FindMeshBlock(on); - for (auto &var : pmb->vars_cc_) { - restriction_cache.RegisterRegionHost( - irestrict++, ProResInfo::GetInteriorRestrict(pmb.get(), NeighborBlock(), var), - var.get(), resolved_packages.get()); + if (newloc[nn].level() < loclist[on].level()) nrestrict += pmb->vars_cc_.size(); + } + + restriction_cache.Initialize(nrestrict, resolved_packages.get()); + int irestrict = 0; + for (int on = t_onbs; on <= t_onbe; on++) { + int nn = oldtonew[on]; + if (newloc[nn].level() < loclist[on].level()) { + auto pmb = FindMeshBlock(on); + for (auto &var : pmb->vars_cc_) { + restriction_cache.RegisterRegionHost( + irestrict++, ProResInfo::GetInteriorRestrict(pmb.get(), NeighborBlock(), var), + var.get(), resolved_packages.get()); + } } } + restriction_cache.CopyToDevice(); + refinement::Restrict(resolved_packages.get(), restriction_cache, + block_list[0]->cellbounds, block_list[0]->c_cellbounds); + }; + std::vector onstart, onstop; + ThreadLoopBounds(*pool, onbs, onbe, onstart, onstop); + for (int it = 0; it < pool->size(); it++) { + pool->enqueue(restrict_fine_to_coarse_buffers, onstart[it], onstop[it]); } - restriction_cache.CopyToDevice(); - refinement::Restrict(resolved_packages.get(), restriction_cache, - block_list[0]->cellbounds, block_list[0]->c_cellbounds); - + //pool->run(); + pool->wait(); Kokkos::fence(); #ifdef MPI_PARALLEL @@ -795,6 +816,7 @@ void Mesh::RedistributeAndRefineMeshBlocks(ParameterInput *pin, ApplicationInput RegionSize block_size = GetBlockSize(); for (int n = nbs; n <= nbe; n++) { + //ThreadPoolLoopBlocking(*pool, nbs, nbe, [&, this](const int n) { int on = newtoold[n]; if ((ranklist[on] == Globals::my_rank) && (loclist[on].level() == newloc[n].level())) { @@ -816,7 +838,7 @@ void Mesh::RedistributeAndRefineMeshBlocks(ParameterInput *pin, ApplicationInput MeshBlock::Make(n, n - nbs, newloc[n], block_size, block_bcs, this, pin, app_in, packages, resolved_packages, gflag); } - } + }//); } // AMR Construct new MeshBlockList region // Replace the MeshBlock list diff --git a/src/mesh/mesh.cpp b/src/mesh/mesh.cpp index d8630d98dcb2..c7a77b17dae5 100644 --- a/src/mesh/mesh.cpp +++ b/src/mesh/mesh.cpp @@ -43,6 +43,7 @@ #include "globals.hpp" #include "interface/state_descriptor.hpp" #include "interface/update.hpp" +#include "kokkos_abstraction.hpp" #include "mesh/mesh.hpp" #include "mesh/mesh_refinement.hpp" #include "mesh/meshblock.hpp" @@ -103,7 +104,8 @@ Mesh::Mesh(ParameterInput *pin, ApplicationInput *app_in, Packages_t &packages, // private members: num_mesh_threads_(pin->GetOrAddInteger("parthenon/mesh", "num_threads", 1)), use_uniform_meshgen_fn_{true, true, true, true}, lb_flag_(true), lb_automatic_(), - lb_manual_(), MeshBndryFnctn{nullptr, nullptr, nullptr, nullptr, nullptr, nullptr} { + lb_manual_(), MeshBndryFnctn{nullptr, nullptr, nullptr, nullptr, nullptr, nullptr}, + pool(std::make_shared(pin->GetOrAddInteger("parthenon/execution", "nthreads", 1))) { std::stringstream msg; RegionSize block_size; BoundaryFlag block_bcs[6]; @@ -479,7 +481,8 @@ Mesh::Mesh(ParameterInput *pin, ApplicationInput *app_in, RestartReader &rr, // private members: num_mesh_threads_(pin->GetOrAddInteger("parthenon/mesh", "num_threads", 1)), use_uniform_meshgen_fn_{true, true, true, true}, lb_flag_(true), lb_automatic_(), - lb_manual_(), MeshBndryFnctn{nullptr, nullptr, nullptr, nullptr, nullptr, nullptr} { + lb_manual_(), MeshBndryFnctn{nullptr, nullptr, nullptr, nullptr, nullptr, nullptr}, + pool(std::make_shared(pin->GetOrAddInteger("parthenon/execution", "nthreads", 1))) { std::stringstream msg; RegionSize block_size; BoundaryFlag block_bcs[6]; diff --git a/src/mesh/mesh.hpp b/src/mesh/mesh.hpp index 051b1f18f500..2a5cf3832bec 100644 --- a/src/mesh/mesh.hpp +++ b/src/mesh/mesh.hpp @@ -260,6 +260,7 @@ class Mesh { return resolved_packages->GetVariableNames(std::forward(args)...); } + std::shared_ptr pool; void SetThreadPool(std::shared_ptr p) { pool = p; } private: @@ -303,8 +304,6 @@ class Mesh { int gmg_min_logical_level_ = 0; - std::shared_ptr pool; - #ifdef MPI_PARALLEL // Global map of MPI comms for separate variables std::unordered_map mpi_comm_map_; diff --git a/src/mesh/meshblock.hpp b/src/mesh/meshblock.hpp index 54e11bcff119..cef22be3f9d6 100644 --- a/src/mesh/meshblock.hpp +++ b/src/mesh/meshblock.hpp @@ -84,7 +84,7 @@ class MeshBlock : public std::enable_shared_from_this { int igflag, double icost = 1.0); // Kokkos execution space for this MeshBlock - DevExecSpace exec_space; + DevExecSpace_t exec_space; // data Mesh *pmy_mesh = nullptr; // ptr to Mesh containing this MeshBlock diff --git a/src/parthenon_manager.hpp b/src/parthenon_manager.hpp index 4dc31f169b1f..516b054fc568 100644 --- a/src/parthenon_manager.hpp +++ b/src/parthenon_manager.hpp @@ -24,6 +24,7 @@ #include "driver/driver.hpp" #include "interface/state_descriptor.hpp" #include "interface/swarm.hpp" +#include "kokkos_abstraction.hpp" #include "mesh/domain.hpp" #include "mesh/mesh.hpp" #include "outputs/restart.hpp" @@ -36,7 +37,7 @@ enum class ParthenonStatus { ok, complete, error }; class ParthenonManager { public: - ParthenonManager() { app_input.reset(new ApplicationInput()); } + ParthenonManager() { t_exec_space = DevExecSpace_t(); app_input.reset(new ApplicationInput()); } ParthenonStatus ParthenonInitEnv(int argc, char *argv[]); void ParthenonInitPackagesAndMesh(); ParthenonStatus ParthenonFinalize(); diff --git a/src/tasks/tasks.hpp b/src/tasks/tasks.hpp index 43ce64563058..d2f626582f92 100644 --- a/src/tasks/tasks.hpp +++ b/src/tasks/tasks.hpp @@ -420,6 +420,9 @@ class TaskRegion { pool.enqueue([t, &ProcessTask]() { return ProcessTask(t); }); } + // and run it + //pool.run(); + // then wait until everything is done pool.wait(); diff --git a/src/utils/communication_buffer.hpp b/src/utils/communication_buffer.hpp index edfc04f11c77..eee7ab563fe3 100644 --- a/src/utils/communication_buffer.hpp +++ b/src/utils/communication_buffer.hpp @@ -21,6 +21,7 @@ #include #include #include +#include #include #include #include @@ -37,6 +38,32 @@ namespace parthenon { // received: X enum class BufferState { stale, sending, sending_null, received, received_null }; +class BufferStateThreadSafe { + public: + BufferStateThreadSafe() = default; + BufferStateThreadSafe(BufferState bs) : state_(bs) {} + BufferStateThreadSafe(const BufferStateThreadSafe &bs) = delete; + BufferStateThreadSafe &operator =(BufferState bs) { + mutex.lock(); + state_ = bs; + mutex.unlock(); + return *this; + } + bool operator==(BufferState bs) { + return (Get() == bs); + } + BufferState Get() { + mutex.lock_shared(); + auto val = state_; + mutex.unlock_shared(); + return val; + } + + private: + BufferState state_; + std::shared_mutex mutex; +}; + enum class BuffCommType { sender, receiver, both, sparse_receiver }; template @@ -46,7 +73,7 @@ class CommBuffer { template friend class CommBuffer; - std::shared_ptr state_; + std::shared_ptr state_; std::shared_ptr comm_type_; std::shared_ptr started_irecv_; std::shared_ptr nrecv_tries_; @@ -65,6 +92,7 @@ class CommBuffer { std::function get_resource_; T buf_; + inline static std::mutex mutex; public: CommBuffer() @@ -81,6 +109,9 @@ class CommBuffer { ~CommBuffer(); + //CommBuffer(const CommBuffer &in) = delete; + //CommBuffer &operator=(const CommBuffer &in) = delete; + template CommBuffer(const CommBuffer &in); @@ -95,19 +126,21 @@ class CommBuffer { void Allocate() { if (!active_) { + std::lock_guard lock(mutex); buf_ = get_resource_(); active_ = true; } } void Free() { + std::lock_guard lock(mutex); buf_ = T(); active_ = false; } bool IsActive() const { return active_; } - BufferState GetState() { return *state_; } + BufferState GetState() { return state_->Get(); } void Send() noexcept; void SendNull() noexcept; @@ -132,7 +165,7 @@ class CommBuffer { template CommBuffer::CommBuffer(int tag, int send_rank, int recv_rank, mpi_comm_t comm, std::function get_resource, bool do_sparse_allocation) - : state_(std::make_shared(BufferState::stale)), + : state_(std::make_shared(BufferState::stale)), comm_type_(std::make_shared(BuffCommType::both)), started_irecv_(std::make_shared(false)), nrecv_tries_(std::make_shared(0)), diff --git a/src/utils/sort.hpp b/src/utils/sort.hpp index 97e9c77a88e4..c92e80865621 100644 --- a/src/utils/sort.hpp +++ b/src/utils/sort.hpp @@ -75,7 +75,7 @@ void sort(ParArray1D data, KeyComparator comparator, size_t min_idx, thrust::sort(first_d, last_d, comparator); #endif #else - if (std::is_same::value) { + if (std::is_same::value) { std::sort(data.data() + min_idx, data.data() + max_idx + 1, comparator); } else { PARTHENON_FAIL("sort is not supported outside of CPU or NVIDIA GPU. If you need sort " @@ -103,7 +103,7 @@ void sort(ParArray1D data, size_t min_idx, size_t max_idx) { thrust::sort(first_d, last_d); #endif #else - if (std::is_same::value) { + if (std::is_same::value) { std::sort(data.data() + min_idx, data.data() + max_idx + 1); } else { PARTHENON_FAIL("sort is not supported outside of CPU or NVIDIA GPU. If you need sort " diff --git a/src/utils/thread_pool.hpp b/src/utils/thread_pool.hpp index 6191edbcd22e..6bfeddf6e611 100644 --- a/src/utils/thread_pool.hpp +++ b/src/utils/thread_pool.hpp @@ -6,7 +6,7 @@ // for the U.S. Department of Energy/National Nuclear Security Administration. All rights // in the program are reserved by Triad National Security, LLC, and the U.S. Department // of Energy/National Nuclear Security Administration. The Government is granted for -// itself and others acting on its behalf a nonexclusive, paid-up, irrevocable worldwide +// itself and others acting on its behalf a nonexclusive, paid-up, irrevocable worldmy_tide // license in this material to reproduce, prepare derivative works, distribute copies to // the public, perform publicly and display publicly, and to permit others to do so. //======================================================================================== @@ -14,6 +14,22 @@ #ifndef UTILS_THREAD_POOL_HPP_ #define UTILS_THREAD_POOL_HPP_ + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include "basic_types.hpp" +#include "kokkos_abstraction.hpp" + +/* #include #include #include @@ -23,69 +39,420 @@ #include #include #include +*/ namespace parthenon { +inline thread_local int my_tid; + +/* +template +class ConcurrentQueue { + public: + explicit ConcurrentQueue(const int nthreads) + : built(false), main_waiting(false), exit(false), nqueues(nthreads+1), nsleep(0), + mutex(nqueues), queue(nqueues) {} + + void push(T q) { + { + std::lock_guard lock(mutex[my_tid]); + queue[my_tid].push(q); + } + nqueued++; + if (nsleep.load() > 0) { + std::lock_guard lock(complete_mutex); + sleep_cv.notify_all(); + } + } + + void push(int i, T q) { + { + std::lock_guard lock(mutex[i]); + queue[i].push(q); + } + nqueued++; + if (nsleep.load() > 0) { + std::lock_guard lock(complete_mutex); + sleep_cv.notify_all(); + } + } + + void push_lazy(T q) { + assert(my_tid == 0); + queue[0].push(q); + nqueued++; + } + void push_lazy(int i, T q) { + queue[i].push(q); + nqueued++; + } + + void go() { + sleep_cv.notify_all(); + } + + void pop(T &q) { + if (mutex[my_tid].try_lock()) { + if (!queue[my_tid].empty()) { + q = queue[my_tid].front(); + queue[my_tid].pop(); + mutex[my_tid].unlock(); + nqueued--; + return; + } + mutex[my_tid].unlock(); + } + for (int i = 0; i < nqueues; i++) { + if (mutex[i].try_lock()) { + if (!queue[i].empty()) { + q = queue[i].front(); + queue[i].pop(); + mutex[i].unlock(); + nqueued--; + return; + } + mutex[i].unlock(); + } + } + return; + } + + bool thread_sleep() { + nsleep++; + std::unique_lock lock(complete_mutex); + if (nsleep == nqueues - 1 && main_waiting && nqueued.load() == 0) complete_cv.notify_all(); + sleep_cv.wait(lock, [this]() { return nqueued.load() > 0 || exit; }); + nsleep--; + return exit && nqueued.load() == 0; + } + + bool signal_if_complete() { + if (nqueued.load() > 0) return false; + bool should_exit = thread_sleep(); + return should_exit; + } + + void signal_exit_when_finished() { + std::unique_lock lock(complete_mutex); + exit = true; + sleep_cv.notify_all(); + } + + void wait_for_complete() { + std::unique_lock lock(complete_mutex); + main_waiting = true; + complete_cv.wait(lock, [this]() { return nqueued == 0; }); + } + + private: + bool built, main_waiting, exit; + const int nqueues; + std::atomic_int nsleep; + std::vector mutex; + std::vector> queue; + std::map worker_id; + std::mutex built_mutex; + std::condition_variable built_cv; + std::atomic_int nqueued; + std::shared_mutex complete_mutex; + std::condition_variable_any complete_cv; + std::condition_variable_any sleep_cv; +}; + + +class ThreadPool { + public: + explicit ThreadPool(const int numthreads = std::thread::hardware_concurrency()) + : nthreads(numthreads), main_id(std::this_thread::get_id()), thread_idx(0), queue(nthreads), exec_space(nthreads+1) { + + auto worker = [&](const int i) { + my_tid = i; + t_exec_space = exec_space[i]; + while (true) { + auto stop = queue.signal_if_complete(); + if(stop) break; + std::function f; + queue.pop(f); + if (f) f(); + } + }; + my_tid = 0; + t_exec_space = exec_space[0]; + threads.reserve(nthreads); + for (int i = 0; i < nthreads; i++) { + threads.emplace_back(worker, i+1); + } + //queue.BuildMap(threads); + } + ~ThreadPool() { + queue.signal_exit_when_finished(); + for (auto &t : threads) { + t.join(); + } + } + void wait() { + queue.wait_for_complete(); + } + template + std::future::type> enqueue(F &&f, Args... args) { + using return_t = typename std::invoke_result::type; + auto task = std::make_shared>( + [=, func = std::forward(f)] { return func(args...); }); + std::future result = task->get_future(); + if (std::this_thread::get_id() == main_id) { + queue.push(thread_idx + 1, [task]() { (*task)(); }); + thread_idx = (thread_idx + 1) % nthreads; + } else { + queue.push([task]() { (*task)(); }); + } + return result; + } + template + std::future::type> enqueue(const int tid, F &&f, Args... args) { + using return_t = typename std::invoke_result::type; + auto task = std::make_shared>( + [=, func = std::forward(f)] { return func(args...); }); + std::future result = task->get_future(); + queue.push(tid, [task]() { (*task)(); }); + return result; + } + template + std::future::type> enqueue_lazy(F &&f, Args... args) { + using return_t = typename std::invoke_result::type; + auto task = std::make_shared>( + [=, func = std::forward(f)] { return func(args...); }); + std::future result = task->get_future(); + if (std::this_thread::get_id() == main_id) { + queue.push_lazy(thread_idx + 1, [task]() { (*task)(); }); + thread_idx = (thread_idx + 1) % nthreads; + } else { + printf("THIS IS WRONG!\n"); + } + return result; + } + template + std::future::type> enqueue_lazy(const int tid, F &&f, Args... args) { + using return_t = typename std::invoke_result::type; + auto task = std::make_shared>( + [=, func = std::forward(f)] { return func(args...); }); + std::future result = task->get_future(); + queue.push_lazy(tid, [task]() { (*task)(); }); + return result; + } + + int size() const { return nthreads; } + void run() { queue.go(); } + private: + const int nthreads; + const std::thread::id main_id; + int thread_idx; + std::vector threads; + ConcurrentQueue> queue; + ExecSpace exec_space; +}; + +template +void ThreadPoolLoop(ThreadPool &pool, const int is, const int ie, F &&f) { + const int nthreads = pool.size(); + const double niter = (ie - is + 1) / (1.0*nthreads) + 1.e-10; + auto looper = [func = std::forward(f)](int t_is, int t_ie) { + for (int i = t_is; i <= t_ie; i++) func(i); + }; + for (int it = 0; it < nthreads; it++) { + int start = is + static_cast(it*niter); + int end = is + static_cast((it+1)*niter) - 1; + pool.enqueue_lazy(it+1, looper, start, end); + } + pool.run(); +} +template +void ThreadPoolLoopBlocking(ThreadPool &pool, Args &&... args) { + ThreadPoolLoop(pool, std::forward(args)...); + pool.wait(); +} + +template +reduce_t ThreadPoolReduce(ThreadPool &pool, const int is, const int ie, F &&f, Reducer &&r, reduce_t init_val) { + const int nthreads = pool.size(); + std::vector> futures(nthreads); + const double niter = (ie - is + 1) / (1.0*nthreads) + 1.e-10; + auto looper = [=, func = std::forward(f), red = std::forward(r)](int t_is, int t_ie) { + reduce_t rval = 0; + for (int i = t_is; i <= t_ie; i++) { + reduce_t val = func(i); + rval = red(rval, val); + } + return rval; + }; + for (int it = 0; it < nthreads; it++) { + int start = is + static_cast(it*niter); + int end = is + static_cast((it+1)*niter) - 1; + futures[it] = pool.enqueue_lazy(it + 1, looper, start, end); + } + pool.run(); + pool.wait(); + reduce_t reduced_val = init_val; + for (int it = 0; it < nthreads; it++) { + if (!futures[it].valid()) { + printf("Got an invalid future somehow!\n"); + } + reduced_val = r(reduced_val, futures[it].get()); + } + return reduced_val; +} + +inline +void ThreadLoopBounds(ThreadPool &pool, const int is, const int ie, std::vector &start, std::vector &stop) { + const int nthreads = pool.size(); + start.resize(nthreads); + stop.resize(nthreads); + const double niter = (ie - is + 1) / (1.0 * nthreads) + 1.e-10; + for (int it = 0; it < nthreads; it++) { + start[it] = is + static_cast(it*niter); + stop[it] = is + static_cast((it+1)*niter) - 1; + } +} +*/ + + + + + template class ThreadQueue { public: - explicit ThreadQueue(const int num_workers) : nworkers(num_workers), nwaiting(0) {} + explicit ThreadQueue(const int num_workers) + : nworkers(num_workers), nwaiting(0), nqueued(0), nqueued0(0), queue(nworkers+1), + mutex(nworkers+1), cv_go(nworkers+1) {} void push(T q) { - std::lock_guard lock(mutex); - queue.push(q); + //std::lock_guard lock(mutex[my_tid]); + mutex[my_tid].lock(); + queue[my_tid].push(q); + mutex[my_tid].unlock(); + nqueued++; + std::lock_guard lock(mutex[0]); cv.notify_one(); + //if (my_tid == 0) { + //nqueued0++; + //cv.notify_one(); + //} + } + //void push(int i, T q) { + //std::lock_guard lock(mutex[i]); + //queue[i].push(q); + //nqueued++; + //cv_go[i].notify_one(); + //} + void pop_tag(int tag, T &q) { + std::lock_guard lock(mutex[tag]); + if (queue[tag].empty()) return; + q = queue[tag].front(); + queue[tag].pop(); + nqueued--; + return; } bool pop(T &q) { - std::unique_lock lock(mutex); - if (queue.empty()) { - nwaiting++; - if (waiting && nwaiting == nworkers) { - complete = true; - complete_cv.notify_all(); + pop_tag(0, q); + if (q) return false; + pop_tag(my_tid, q); + if (q) return false; + + for (int i = 0; i <= nworkers; i++) { + if (mutex[i].try_lock()) { + if (!queue[i].empty()) { + q = queue[i].front(); + queue[i].pop(); + mutex[i].unlock(); + nqueued--; + return false; + } + mutex[i].unlock(); } - cv.wait(lock, [this]() { return exit || !queue.empty(); }); - nwaiting--; - if (exit) return true; } - q = queue.front(); - queue.pop(); - return false; + + //printf("thread %d going into waiting...\n", my_tid); + + std::unique_lock lock(mutex[0]); + nwaiting++; + if (waiting && nwaiting.load() == nworkers && nqueued.load() == 0) { + complete = true; + complete_cv.notify_all(); + } + cv.wait(lock, [this]() { return exit || nqueued.load() > 0; }); + nwaiting--; + return exit; } - void signal_kill() { - std::lock_guard lock(mutex); - std::queue().swap(queue); - complete = true; - exit = true; - cv.notify_all(); +/* + //std::unique_lock lock(mutex[my_tid]); + if (nqueued0.load() > 0) { + std::lock_guard lock(mutex[0]); + if (!queue[0].empty()) { + q = queue[0].front(); + queue[0].pop(); + nqueued--; + nqueued0--; + return false; + } + } + mutex[my_tid].lock(); + if (queue[my_tid].empty()) { + //mutex[my_tid].unlock(); + //std::unique_lock lock(mutex[0]); + //if (queue[0].empty()) { + nwaiting++; + if (waiting && nwaiting.load() == nworkers) { + complete = true; + complete_cv.notify_all(); + } + cv_go[my_tid].wait(lock, [this]() { return exit || !queue[my_tid].empty(); }); + nwaiting--; + if (exit) return true; + } + q = queue[my_tid].front(); + queue[my_tid].pop(); + mutex[my_tid].unlock(); + nqueued--; + return false; } + */ + //void signal_kill() { + //std::lock_guard lock(mutex[0]); + //std::queue().swap(queue); + //complete = true; + //exit = true; + //cv.notify_all(); + //} void signal_exit_when_finished() { - std::lock_guard lock(mutex); + std::lock_guard lock(mutex[0]); exit = true; complete = true; cv.notify_all(); } void wait_for_complete() { - std::unique_lock lock(mutex); + std::unique_lock lock(mutex[0]); waiting = true; - if (queue.empty() && nwaiting == nworkers) { + if (nqueued.load() == 0 && nwaiting.load() == nworkers) { complete = false; waiting = false; return; } complete_cv.wait(lock, [this]() { return complete; }); + //printf("got complete!\n"); complete = false; waiting = false; } - size_t size() { - std::lock_guard lock(mutex); - return queue.size(); - } + //size_t size() { + //std::lock_guard lock(mutex[0]); + //return queue.size(); + //} private: const int nworkers; - int nwaiting; - std::queue queue; - std::mutex mutex; + std::atomic_int nwaiting; + std::atomic_int nqueued, nqueued0; + std::vector> queue; + std::vector mutex; + std::vector cv_go; std::condition_variable cv; std::condition_variable complete_cv; bool complete = false; @@ -96,9 +463,13 @@ class ThreadQueue { class ThreadPool { public: explicit ThreadPool(const int numthreads = std::thread::hardware_concurrency()) - : nthreads(numthreads), queue(nthreads) { + : nthreads(numthreads), queue(nthreads), exec_space(nthreads+1) { + my_tid = 0; + t_exec_space = exec_space[0]; for (int i = 0; i < nthreads; i++) { - auto worker = [&]() { + auto worker = [&](const int i) { + my_tid = i; + t_exec_space = exec_space[i]; while (true) { std::function f; auto stop = queue.pop(f); @@ -106,7 +477,7 @@ class ThreadPool { if (f) f(); } }; - threads.emplace_back(worker); + threads.emplace_back(worker, i+1); } } ~ThreadPool() { @@ -116,15 +487,16 @@ class ThreadPool { } } - void wait() { queue.wait_for_complete(); } + void wait() { //queue.notify_all(); + queue.wait_for_complete(); } - void kill() { queue.signal_kill(); } + //void kill() { queue.signal_kill(); } template - std::future::type> enqueue(F &&f, Args &&...args) { - using return_t = typename std::result_of::type; + std::future::type> enqueue(F &&f, Args... args) { + using return_t = typename std::invoke_result::type; auto task = std::make_shared>( - [=, func = std::forward(f)] { return func(std::forward(args)...); }); + [=, func = std::forward(f)] { return func(args...); }); std::future result = task->get_future(); queue.push([task]() { (*task)(); }); return result; @@ -132,14 +504,92 @@ class ThreadPool { int size() const { return nthreads; } - size_t num_queued() { return queue.size(); } + //size_t num_queued() { return queue.size(); } private: const int nthreads; std::vector threads; ThreadQueue> queue; + ExecSpace exec_space; }; +template +void ThreadPoolLoop(ThreadPool &pool, const int is, const int ie, F &&f) { + const int nthreads = pool.size(); + const double niter = (ie - is + 1) / (1.0*nthreads) + 1.e-10; + auto looper = [func = std::forward(f)](int t_is, int t_ie) { + for (int i = t_is; i <= t_ie; i++) func(i); + }; + for (int it = 0; it < nthreads; it++) { + int start = is + static_cast(it*niter); + int end = is + static_cast((it+1)*niter) - 1; + pool.enqueue(looper, start, end); + } +} +template +void ThreadPoolLoopBlocking(ThreadPool &pool, Args &&... args) { + ThreadPoolLoop(pool, std::forward(args)...); + pool.wait(); +} + +template +reduce_t ThreadPoolReduce(ThreadPool &pool, const int is, const int ie, F &&f, Reducer &&r, reduce_t init_val) { + const int nthreads = pool.size(); + std::vector> futures(nthreads); + const double niter = (ie - is + 1) / (1.0*nthreads) + 1.e-10; + auto looper = [=, func = std::forward(f), red = std::forward(r)](int t_is, int t_ie) { + reduce_t rval = 0; + for (int i = t_is; i <= t_ie; i++) { + reduce_t val = func(i); + rval = red(rval, val); + } + return rval; + }; + for (int it = 0; it < nthreads; it++) { + int start = is + static_cast(it*niter); + int end = is + static_cast((it+1)*niter) - 1; + futures[it] = pool.enqueue(looper, start, end); + } + pool.wait(); + reduce_t reduced_val = init_val; + for (int it = 0; it < nthreads; it++) { + reduced_val = r(reduced_val, futures[it].get()); + } + return reduced_val; +} + +template +reduce_t ThreadPoolReduce2(ThreadPool &pool, const int is, const int ie, F &&f, Reducer &&r, reduce_t init_val) { + std::vector> futures(ie+1); + auto flb = [=, func = std::forward(f), red = std::forward(r)](int t_i) { + reduce_t rval = init_val; + return red(rval, func(t_i)); + }; + for (int it = is; it <= ie; it++) { + futures[it] = pool.enqueue(flb, it); + } + pool.wait(); + reduce_t reduced_val = init_val; + for (int it = is; it <= ie; it++) { + reduced_val = r(reduced_val, futures[it].get()); + } + return reduced_val; +} + +inline +void ThreadLoopBounds(ThreadPool &pool, const int is, const int ie, std::vector &start, std::vector &stop) { + const int nthreads = pool.size(); + start.resize(nthreads); + stop.resize(nthreads); + const double niter = (ie - is + 1) / (1.0 * nthreads) + 1.e-10; + for (int it = 0; it < nthreads; it++) { + start[it] = is + static_cast(it*niter); + stop[it] = is + static_cast((it+1)*niter) - 1; + } +} + + + } // namespace parthenon #endif // TASKS_THREAD_POOL_HPP_