From e0101d59a0944e7a87e55f157ae4fc0f7d666860 Mon Sep 17 00:00:00 2001 From: Christian von Elm Date: Thu, 7 Nov 2024 08:38:20 +0100 Subject: [PATCH] Simplify calling context managment, include timepoint tracking -> rename ref stuff All the stuff just being reference numbers in OTF2 underneath is such an unimportant implementation detail. rename it to "LocalCctx" "GlobalCctx". This way it becomes clear that we first generate local calling contexts in every sample::Writer and then map them to the global calling contexts later on in post processing. -> simplify calling context tracking most of it was needlessly complex, as we don't need to record context in most cases we simply have it. -> use Mmap2 Record format instead of mmap2 according to the perf_event_open man page, RecordMmap does not support adding the sample_id fields, such as the event timestamp to RecordMmap events, only to RecordMmap2 events, so change to the latest and greatest format. -> introduce exec and mmap event timestamp tracking Keep track of execs and when memory mappings happen by time. Currently, lo2s does process tracking solely by pid. This approach failes in the context of execs, where instruction pointer 0xc0ffee can refer to one thing before the exec and to something different altogether after the exec. We now can differentiate between different processes sharing the same pid over time. However, one important part is still missing, time tracking for instruction pointers. This is programmatically the hard part. But for now this at least makes the common case (fork() then immediate exec()) work better. -> simplify cctx tracking Track Thread CCTX (used for calling context enter/leave events) separately from Instruction Pointer CCTx (used for calling context sample events). Remove a layer of indirection. Track instruction pointers per process instead of per thread. (Threads can not have different memory images. --- include/lo2s/monitor/main_monitor.hpp | 8 +- include/lo2s/perf/calling_context_manager.hpp | 137 +++++++++++------- include/lo2s/perf/sample/reader.hpp | 1 + include/lo2s/perf/sample/writer.hpp | 10 +- include/lo2s/perf/types.hpp | 4 +- include/lo2s/trace/trace.hpp | 76 +++------- src/monitor/cpu_set_monitor.cpp | 3 +- src/monitor/main_monitor.cpp | 39 ++++- src/monitor/process_monitor.cpp | 2 +- src/perf/sample/writer.cpp | 49 ++++--- src/trace/trace.cpp | 136 ++++++++--------- 11 files changed, 248 insertions(+), 217 deletions(-) diff --git a/include/lo2s/monitor/main_monitor.hpp b/include/lo2s/monitor/main_monitor.hpp index 3e58f4c9..944bcd11 100644 --- a/include/lo2s/monitor/main_monitor.hpp +++ b/include/lo2s/monitor/main_monitor.hpp @@ -31,7 +31,6 @@ #ifdef HAVE_SENSORS #include #endif -#include #include #ifdef HAVE_VEOSINFO #include @@ -61,16 +60,17 @@ class MainMonitor return trace_; } - void insert_cached_mmap_events(const RawMemoryMapCache& cached_events); + void insert_cached_events(const RawMemoryMapCache& cached_events, + const RawCommCache& cached_comms); - std::map& get_process_infos() + ProcessMap& get_process_infos() { return process_infos_; } protected: trace::Trace trace_; - std::map process_infos_; + ProcessMap process_infos_; metric::plugin::Metrics metrics_; std::vector> tracepoint_monitors_; diff --git a/include/lo2s/perf/calling_context_manager.hpp b/include/lo2s/perf/calling_context_manager.hpp index 7dc32c49..268a0469 100644 --- a/include/lo2s/perf/calling_context_manager.hpp +++ b/include/lo2s/perf/calling_context_manager.hpp @@ -20,62 +20,45 @@ */ #pragma once -#include -#include +#include +#include #include -namespace lo2s +extern "C" { -namespace perf +#include +} + +namespace lo2s { -class CallingContextManager +struct LocalCctx { -public: - CallingContextManager(trace::Trace& trace) : local_cctx_refs_(trace.create_cctx_refs()) + LocalCctx(otf2::definition::calling_context::reference_type r) : ref(r) { } - void thread_enter(Process process, Thread thread) - { - auto ret = - local_cctx_refs_.map.emplace(std::piecewise_construct, std::forward_as_tuple(thread), - std::forward_as_tuple(process, next_cctx_ref_)); - if (ret.second) - { - next_cctx_ref_++; - } + otf2::definition::calling_context::reference_type ref; + std::map children; +}; - current_thread_cctx_refs_ = &(*ret.first); +class LocalCctxMap +{ +public: + LocalCctxMap() + { } void finalize(otf2::writer::local* otf2_writer) { - local_cctx_refs_.ref_count = next_cctx_ref_; + ref_count_ = next_cctx_ref_; // set writer last, because it is used as sentry to confirm that the cctx refs are properly // finalized. - local_cctx_refs_.writer = otf2_writer; - } - - bool thread_changed(Thread thread) - { - return !current_thread_cctx_refs_ || current_thread_cctx_refs_->first != thread; - } - - otf2::definition::calling_context::reference_type current() - { - if (current_thread_cctx_refs_) - { - return current_thread_cctx_refs_->second.entry.ref; - } - else - { - return otf2::definition::calling_context::reference_type::undefined(); - } + writer_ = otf2_writer; } - otf2::definition::calling_context::reference_type sample_ref(uint64_t num_ips, + otf2::definition::calling_context::reference_type sample_ref(Process p, uint64_t num_ips, const uint64_t ips[]) { // For unwind distance definiton, see: @@ -96,40 +79,80 @@ class CallingContextManager // information. // // Having these things in mind, look at this line and tell me, why it is still wrong: - auto children = ¤t_thread_cctx_refs_->second.entry.children; + auto children = &map[p]; + uint64_t ref = -1; for (uint64_t i = num_ips - 1;; i--) { + if (ips[i] == PERF_CONTEXT_KERNEL) + { + if (i <= 1) + { + return ref; + } + continue; + } + else if (ips[i] == PERF_CONTEXT_USER) + { + if (i <= 1) + { + return ref; + } + continue; + } auto it = find_ip_child(ips[i], *children); - // We intentionally discard the last sample as it is somewhere in the kernel - if (i == 1) + ref = it->second.ref; + if (i == 0) { - return it->second.ref; + return ref; } children = &it->second.children; } } - otf2::definition::calling_context::reference_type sample_ref(uint64_t ip) + otf2::definition::calling_context::reference_type sample_ref(Process p, uint64_t ip) { - auto it = find_ip_child(ip, current_thread_cctx_refs_->second.entry.children); + auto it = find_ip_child(ip, map[p]); return it->second.ref; } - void thread_leave(Thread thread) + otf2::definition::calling_context::reference_type thread(Process process, Thread thread) { - assert(current_thread_cctx_refs_); - if (current_thread_cctx_refs_->first != thread) + auto ret = + thread_cctxs_[process].emplace(std::piecewise_construct, std::forward_as_tuple(thread), + std::forward_as_tuple(next_cctx_ref_)); + if (ret.second) { - Log::debug() << "inconsistent leave thread"; // will probably set to trace sooner or - // later + next_cctx_ref_++; } - current_thread_cctx_refs_ = nullptr; + + return ret.first->second.ref; + } + + size_t num_cctx() const + { + return ref_count_; + } + + const std::map>& get_threads() const + { + return thread_cctxs_; + } + + const std::map>& get_functions() const + { + return map; + } + + otf2::writer::local* writer() + { + return writer_; } private: - trace::IpRefMap::iterator find_ip_child(Address addr, trace::IpRefMap& children) + std::map::iterator find_ip_child(Address addr, + std::map& children) { // -1 can't be inserted into the ip map, as it imples a 1-byte region from -1 to 0. if (addr == -1) @@ -147,9 +170,17 @@ class CallingContextManager } private: - trace::ThreadCctxRefMap& local_cctx_refs_; + std::map> map; + std::map> thread_cctxs_; + + /* + * Stores calling context information for each sample writer / monitoring thread. + * While the `Trace` always owns this data, the `sample::Writer` should have exclusive access to + * this data during its lifetime. Only afterwards, the `writer` and `refcount` are set by the + * `sample::Writer`. + */ + std::atomic writer_ = nullptr; + std::atomic ref_count_; size_t next_cctx_ref_ = 0; - trace::ThreadCctxRefMap::value_type* current_thread_cctx_refs_ = nullptr; }; -} // namespace perf } // namespace lo2s diff --git a/include/lo2s/perf/sample/reader.hpp b/include/lo2s/perf/sample/reader.hpp index ffec9d92..36ed89ed 100644 --- a/include/lo2s/perf/sample/reader.hpp +++ b/include/lo2s/perf/sample/reader.hpp @@ -109,6 +109,7 @@ class Reader : public EventReader perf_attr.config1 = sampling_event.config1; perf_attr.mmap = 1; + perf_attr.mmap2 = 1; } else { diff --git a/include/lo2s/perf/sample/writer.hpp b/include/lo2s/perf/sample/writer.hpp index 784c648a..b589dbac 100644 --- a/include/lo2s/perf/sample/writer.hpp +++ b/include/lo2s/perf/sample/writer.hpp @@ -22,7 +22,6 @@ #pragma once #include -#include #include #include @@ -64,7 +63,7 @@ class Writer : public Reader public: using Reader::handle; bool handle(const Reader::RecordSampleType* sample); - bool handle(const Reader::RecordMmapType* mmap_event); + bool handle(const Reader::RecordMmap2Type* mmap_event); bool handle(const Reader::RecordCommType* comm); bool handle(const Reader::RecordSwitchCpuWideType* context_switch); bool handle(const Reader::RecordSwitchType* context_switch); @@ -76,7 +75,7 @@ class Writer : public Reader void update_calling_context(Process process, Thread thread, otf2::chrono::time_point tp, bool switch_out); - void leave_current_thread(Thread thread, otf2::chrono::time_point tp); + void leave_current_thread(Process process, Thread thread, otf2::chrono::time_point tp); otf2::chrono::time_point adjust_timepoints(otf2::chrono::time_point tp); ExecutionScope scope_; @@ -89,8 +88,9 @@ class Writer : public Reader otf2::definition::metric_instance cpuid_metric_instance_; otf2::event::metric cpuid_metric_event_; - CallingContextManager cctx_manager_; + LocalCctxMap& local_cctx_map_; RawMemoryMapCache cached_mmap_events_; + RawCommCache cached_comm_events_; std::unordered_map comms_; const time::Converter time_converter_; @@ -98,6 +98,8 @@ class Writer : public Reader bool first_event_ = true; otf2::chrono::time_point first_time_point_; otf2::chrono::time_point last_time_point_; + Process cur_process_ = Process::invalid(); + Thread cur_thread_ = Thread::invalid(); }; } // namespace sample } // namespace perf diff --git a/include/lo2s/perf/types.hpp b/include/lo2s/perf/types.hpp index 6d560ee6..79188855 100644 --- a/include/lo2s/perf/types.hpp +++ b/include/lo2s/perf/types.hpp @@ -43,9 +43,9 @@ class PerfEventCache PerfEventCache(const PerfEventCache&) = delete; PerfEventCache& operator=(const PerfEventCache&) = delete; - PerfEventCache(const T* event, size_t size) : data_(std::make_unique(size)) + PerfEventCache(const T* event) : data_(std::make_unique(event->header.size)) { - memcpy(data_.get(), event, size); + memcpy(data_.get(), event, event->header.size); } T* get() diff --git a/include/lo2s/trace/trace.hpp b/include/lo2s/trace/trace.hpp index 9075292e..779b76b2 100644 --- a/include/lo2s/trace/trace.hpp +++ b/include/lo2s/trace/trace.hpp @@ -22,11 +22,10 @@ #include "otf2xx/definition/calling_context.hpp" #include #include -#include #include #include #include -#include +#include #include #include #include @@ -48,58 +47,16 @@ namespace trace { class MainMonitor; -template -using IpMap = std::map; - -struct IpRefEntry +struct GlobalCctx { - IpRefEntry(otf2::definition::calling_context::reference_type r) : ref(r) - { - } - - otf2::definition::calling_context::reference_type ref; - IpMap children; -}; - -struct ThreadCctxRefs -{ - ThreadCctxRefs(Process p, otf2::definition::calling_context::reference_type r) - : process(p), entry(r) - { - } - - Process process; - IpRefEntry entry; -}; - -struct IpCctxEntry -{ - IpCctxEntry(otf2::definition::calling_context& c) : cctx(c) + GlobalCctx(otf2::definition::calling_context& c) : cctx(c) { } otf2::definition::calling_context& cctx; - IpMap children; -}; - -/* - * Stores calling context information for each sample writer / monitoring thread. - * While the `Trace` always owns this data, the `sample::Writer` should have exclusive access to - * this data during its lifetime. Only afterwards, the `writer` and `refcount` are set by the - * `sample::Writer`. - */ -struct ThreadCctxRefMap -{ - std::map map; - std::atomic writer = nullptr; - std::atomic ref_count; - - using value_type = std::map::value_type; + std::map children; }; -using IpRefMap = IpMap; -using IpCctxMap = IpMap; - class Trace { public: @@ -123,11 +80,10 @@ class Trace void update_process_name(Process p, const std::string& name); void update_thread_name(Thread t, const std::string& name); - ThreadCctxRefMap& create_cctx_refs(); - otf2::definition::mapping_table - merge_calling_contexts(const std::map& new_ips, size_t num_ip_refs, - const std::map& infos); - void merge_calling_contexts(const std::map& process_infos); + LocalCctxMap& create_local_cctx_map(); + otf2::definition::mapping_table merge_calling_contexts(const LocalCctxMap& local_cctxs, + ProcessMap& infos); + void merge_calling_contexts(ProcessMap& process_infos); otf2::definition::mapping_table merge_syscall_contexts(const std::set& used_syscalls); @@ -342,9 +298,9 @@ class Trace void add_thread_exclusive(Thread thread, const std::string& name, const std::lock_guard&); - void merge_ips(const IpRefMap& new_children, IpCctxMap& children, - std::vector& mapping_table, otf2::definition::calling_context& parent, - const std::map& infos, Process p); + void merge_ips(const std::map& new_children, + std::map& children, std::vector& mapping_table, + otf2::definition::calling_context& parent, ProcessMap& infos, Process p); const otf2::definition::system_tree_node bio_parent_node(BlockDevice& device) { @@ -392,7 +348,9 @@ class Trace // TODO add location groups (processes), read path from /proc/self/exe symlink std::map thread_names_; - std::map calling_context_tree_; + + std::map global_thread_cctxs_; + std::map> calling_context_tree_; otf2::definition::comm_locations_group& comm_locations_group_; otf2::definition::comm_locations_group& hardware_comm_locations_group_; @@ -413,11 +371,11 @@ class Trace ExecutionScopeGroup& groups_; - std::deque cctx_refs_; + std::deque local_cctx_maps_; // Mutex is only used for accessing the cctx_refs_ - std::mutex cctx_refs_mutex_; + std::mutex local_cctx_maps_mutex_; // I wanted to use atomic_flag, but I need test and that's a C++20 exclusive. - std::atomic_bool cctx_refs_finalized_ = false; + std::atomic_bool local_cctx_maps_finalized_ = false; }; } // namespace trace } // namespace lo2s diff --git a/src/monitor/cpu_set_monitor.cpp b/src/monitor/cpu_set_monitor.cpp index 44fb8586..d269e21f 100644 --- a/src/monitor/cpu_set_monitor.cpp +++ b/src/monitor/cpu_set_monitor.cpp @@ -58,8 +58,7 @@ CpuSetMonitor::CpuSetMonitor() : MainMonitor() { pid = std::stol(pid_match[1]); - process_infos_.emplace(std::piecewise_construct, std::forward_as_tuple(pid), - std::forward_as_tuple(Process(pid), false)); + process_infos_.insert(Process(pid), 0, true); } } } diff --git a/src/monitor/main_monitor.cpp b/src/monitor/main_monitor.cpp index e2a168fe..af631709 100644 --- a/src/monitor/main_monitor.cpp +++ b/src/monitor/main_monitor.cpp @@ -128,20 +128,47 @@ MainMonitor::MainMonitor() : trace_(), metrics_(trace_) #endif } -void MainMonitor::insert_cached_mmap_events(const RawMemoryMapCache& cached_events) +static uint64_t mmap_get_time(const RecordMmap2Type* t) { - for (auto& event : cached_events) + struct sample_id* id = + (struct sample_id*)((char*)t + t->header.size - sizeof(struct sample_id)); + return id->time; +} + +static uint64_t comm_get_time(const RecordCommType* t) +{ + struct sample_id* id = + (struct sample_id*)((char*)t + t->header.size - sizeof(struct sample_id)); + return id->time; +} + +void MainMonitor::insert_cached_events(const RawMemoryMapCache& cached_mmaps, + const RawCommCache& cached_execs) +{ + for (auto& event : cached_execs) { - auto process_info = - process_infos_.emplace(std::piecewise_construct, std::forward_as_tuple(event.process), - std::forward_as_tuple(event.process, true)); - process_info.first->second.mmap(event); + process_infos_.insert(Process(event.get()->pid), comm_get_time(event.get()), false); + } + for (auto& event : cached_mmaps) + { + const uint64_t timestamp = mmap_get_time(event.get()); + Process p = Process(event.get()->pid); + + if (!process_infos_.has(p, timestamp)) + { + process_infos_.insert(p, timestamp, false); + } + + ProcessInfo& pinfo = process_infos_.get(p, timestamp); + + pinfo.mmap(*event.get()); } } MainMonitor::~MainMonitor() { // Note: call stop() in reverse order than start() in constructor + // #ifdef HAVE_SENSORS if (config().use_sensors) diff --git a/src/monitor/process_monitor.cpp b/src/monitor/process_monitor.cpp index f4d66047..a5f5bf9c 100644 --- a/src/monitor/process_monitor.cpp +++ b/src/monitor/process_monitor.cpp @@ -48,7 +48,7 @@ void ProcessMonitor::insert_thread(Process process, Thread thread, std::string n if (config().sampling) { - process_infos_.try_emplace(process, process, spawn); + process_infos_.insert(process, 0, !spawn); } if (config().sampling || diff --git a/src/perf/sample/writer.cpp b/src/perf/sample/writer.cpp index 64e56d7c..924384fe 100644 --- a/src/perf/sample/writer.cpp +++ b/src/perf/sample/writer.cpp @@ -58,7 +58,8 @@ Writer::Writer(ExecutionScope scope, monitor::MainMonitor& Monitor, trace::Trace otf2_writer_(trace.sample_writer(scope)), cpuid_metric_instance_(trace.metric_instance(trace.cpuid_metric_class(), otf2_writer_.location(), otf2_writer_.location())), - cpuid_metric_event_(otf2::chrono::genesis(), cpuid_metric_instance_), cctx_manager_(trace), + cpuid_metric_event_(otf2::chrono::genesis(), cpuid_metric_instance_), + local_cctx_map_(trace.create_local_cctx_map()), time_converter_(perf::time::Converter::instance()), first_time_point_(lo2s::time::now()), last_time_point_(first_time_point_) { @@ -66,13 +67,13 @@ Writer::Writer(ExecutionScope scope, monitor::MainMonitor& Monitor, trace::Trace Writer::~Writer() { - if (!cctx_manager_.current().is_undefined()) + if (cur_thread_ != Thread::invalid()) { otf2_writer_.write_calling_context_leave(adjust_timepoints(lo2s::time::now()), - cctx_manager_.current()); + local_cctx_map_.thread(cur_process_, cur_thread_)); } - cctx_manager_.finalize(&otf2_writer_); + local_cctx_map_.finalize(&otf2_writer_); } bool Writer::handle(const Reader::RecordSampleType* sample) @@ -84,19 +85,20 @@ bool Writer::handle(const Reader::RecordSampleType* sample) if (!has_cct_) { - otf2_writer_.write_calling_context_sample(tp, cctx_manager_.sample_ref(sample->ip), 2, - trace_.interrupt_generator().ref()); + otf2_writer_.write_calling_context_sample( + tp, local_cctx_map_.sample_ref(Process(sample->pid), sample->ip), 2, + trace_.interrupt_generator().ref()); } else { - otf2_writer_.write_calling_context_sample(tp, - cctx_manager_.sample_ref(sample->nr, sample->ips), - sample->nr, trace_.interrupt_generator().ref()); + otf2_writer_.write_calling_context_sample( + tp, local_cctx_map_.sample_ref(Process(sample->pid), sample->nr, sample->ips), + sample->nr, trace_.interrupt_generator().ref()); } return false; } -bool Writer::handle(const Reader::RecordMmapType* mmap_event) +bool Writer::handle(const Reader::RecordMmap2Type* mmap_event) { // Since this is an mmap record (as opposed to mmap2), it will only be generated for executable if (!scope_.is_cpu() && scope_ != ExecutionScope(Thread(mmap_event->tid))) @@ -119,23 +121,23 @@ void Writer::update_current_thread(Process process, Thread thread, otf2::chrono: otf2_writer_ << otf2::event::thread_begin(tp, trace_.process_comm(scope_.as_thread()), -1); first_event_ = false; } - if (!cctx_manager_.thread_changed(thread)) + if (thread == cur_thread_) { return; } - else if (!cctx_manager_.current().is_undefined()) + else if (cur_thread_ != Thread::invalid()) { - leave_current_thread(thread, tp); + leave_current_thread(process, thread, tp); } - cctx_manager_.thread_enter(process, thread); - otf2_writer_.write_calling_context_enter(tp, cctx_manager_.current(), 2); + otf2_writer_.write_calling_context_enter(tp, local_cctx_map_.thread(process, thread), 2); } -void Writer::leave_current_thread(Thread thread, otf2::chrono::time_point tp) +void Writer::leave_current_thread(Process process, Thread thread, otf2::chrono::time_point tp) { - otf2_writer_.write_calling_context_leave(tp, cctx_manager_.current()); - cctx_manager_.thread_leave(thread); + otf2_writer_.write_calling_context_leave(tp, local_cctx_map_.thread(process, thread)); + cur_process_ = Process::invalid(); + cur_thread_ = Thread::invalid(); } otf2::chrono::time_point Writer::adjust_timepoints(otf2::chrono::time_point tp) @@ -186,12 +188,12 @@ void Writer::update_calling_context(Process process, Thread thread, otf2::chrono { if (switch_out) { - if (cctx_manager_.current().is_undefined()) + if (cur_thread_ == Thread::invalid()) { Log::debug() << "Leave event but not in a thread!"; return; } - leave_current_thread(thread, tp); + leave_current_thread(process, thread, tp); } else { @@ -217,6 +219,11 @@ bool Writer::handle(const Reader::RecordCommType* comm) trace_.update_process_name(Process(comm->pid), new_command); } } + + if (comm->header.misc & PERF_RECORD_MISC_COMM_EXEC) + { + cached_comm_events_.emplace_back(comm); + } summary().register_process(Process(comm->pid)); comms_[Thread(comm->tid)] = comm->comm; @@ -252,7 +259,7 @@ void Writer::end() trace_.add_threads(comms_); - monitor_.insert_cached_mmap_events(cached_mmap_events_); + monitor_.insert_cached_events(cached_mmap_events_, cached_comm_events_); } } // namespace sample } // namespace perf diff --git a/src/trace/trace.cpp b/src/trace/trace.cpp index a3f7ecba..dffe96d3 100644 --- a/src/trace/trace.cpp +++ b/src/trace/trace.cpp @@ -22,10 +22,8 @@ #include #include -#include #include #include -#include #include #include #include @@ -230,7 +228,7 @@ otf2::chrono::time_point Trace::record_to() const Trace::~Trace() { - if (!cctx_refs_finalized_) + if (!local_cctx_maps_finalized_) { Log::error() << "cctx refs have not been finalized, please report this bug to the developers"; @@ -611,10 +609,9 @@ otf2::definition::metric_class& Trace::metric_class() otf2::common::recorder_kind::abstract); } -void Trace::merge_ips(const IpRefMap& new_children, IpCctxMap& children, - std::vector& mapping_table, - otf2::definition::calling_context& parent, - const std::map& infos, Process process) +void Trace::merge_ips(const std::map& new_children, + std::map& children, std::vector& mapping_table, + otf2::definition::calling_context& parent, ProcessMap& infos, Process process) { for (const auto& elem : new_children) { @@ -623,11 +620,14 @@ void Trace::merge_ips(const IpRefMap& new_children, IpCctxMap& children, auto& local_children = elem.second.children; LineInfo line_info = LineInfo::for_unknown_function(); - auto info_it = infos.find(process); - if (info_it != infos.end()) + if (infos.has(process, UINT64_MAX)) { - MemoryMap maps = info_it->second.maps(); - line_info = maps.lookup_line_info(ip); + line_info = infos.get(process, UINT64_MAX).lookup_line_info(ip); + } + else + { + infos.insert(process, 0, false); + line_info = infos.get(process, UINT64_MAX).lookup_line_info(ip); } Log::trace() << "resolved " << ip << ": " << line_info; @@ -639,11 +639,11 @@ void Trace::merge_ips(const IpRefMap& new_children, IpCctxMap& children, auto r = children.emplace(ip, new_cctx); cctx_it = r.first; - if (config().disassemble && infos.count(process) == 1) + if (config().disassemble && infos.has(process, UINT64_MAX) == 1) { try { - auto instruction = infos.at(process).maps().lookup_instruction(ip); + auto instruction = infos.get(process, UINT64_MAX).lookup_instruction(ip); Log::trace() << "mapped " << ip << " to " << instruction; registry_.create( @@ -663,64 +663,71 @@ void Trace::merge_ips(const IpRefMap& new_children, IpCctxMap& children, } } -otf2::definition::mapping_table -Trace::merge_calling_contexts(const std::map& new_ips, size_t num_ip_refs, - const std::map& infos) +otf2::definition::mapping_table Trace::merge_calling_contexts(const LocalCctxMap& local_cctxs, + ProcessMap& infos) { std::lock_guard guard(mutex_); #ifndef NDEBUG - std::vector mappings(num_ip_refs, -1u); + std::vector mappings(local_cctxs.num_cctx(), -1u); #else - std::vector mappings(num_ip_refs); + std::vector mappings(local_cctxs.num_cctx()); #endif // Merge local thread tree into global thread tree - for (auto& local_thread_cctx : new_ips) + for (auto& process_map : local_cctxs.get_threads()) { + Process process = process_map.first; - auto thread = local_thread_cctx.first; - auto process = local_thread_cctx.second.process; + for (auto& local_thread_cctx : process_map.second) + { + Thread thread = local_thread_cctx.first; + groups_.add_thread(thread, process); - groups_.add_thread(thread, process); - auto local_ref = local_thread_cctx.second.entry.ref; + auto local_ref = local_thread_cctx.second; - auto global_thread_cctx = calling_context_tree_.find(thread); + auto global_thread_cctx = global_thread_cctxs_.find(thread); - if (global_thread_cctx == calling_context_tree_.end()) - { - if (thread != Thread(0)) + if (global_thread_cctx == global_thread_cctxs_.end()) { - - if (auto thread_name = thread_names_.find(thread); - thread_name != thread_names_.end()) + if (thread != Thread(0)) { - add_thread(thread, thread_name->second); - } - else - { - if (auto process_name = thread_names_.find(process.as_thread()); - process_name != thread_names_.end()) + + if (auto thread_name = thread_names_.find(thread); + thread_name != thread_names_.end()) { - add_thread(thread, process_name->second); + add_thread(thread, thread_name->second); } else { - add_thread(thread, ""); + if (auto process_name = thread_names_.find(process.as_thread()); + process_name != thread_names_.end()) + { + add_thread(thread, process_name->second); + } + else + { + add_thread(thread, ""); + } } } + else + { + add_thread(thread, ""); + } } - else - { - add_thread(thread, ""); - } - global_thread_cctx = calling_context_tree_.find(thread); + const auto& foo = global_thread_cctxs_.at(thread).cctx.ref(); + mappings.at(local_ref.ref) = foo; } + } - assert(global_thread_cctx != calling_context_tree_.end()); - mappings.at(local_ref) = global_thread_cctx->second.cctx.ref(); - - merge_ips(local_thread_cctx.second.entry.children, global_thread_cctx->second.children, - mappings, global_thread_cctx->second.cctx, infos, process); + for (auto& local_process_map : local_cctxs.get_functions()) + { + auto& parent = registry_.get( + ByThread(local_process_map.first.as_thread())); + auto ret = + calling_context_tree_.emplace(local_process_map.first, std::map()); + merge_ips(local_process_map.second, ret.first->second, mappings, parent, infos, + local_process_map.first); } #ifndef NDEBUG @@ -785,9 +792,8 @@ void Trace::add_thread_exclusive(Thread thread, const std::string& name, // create calling context auto& thread_cctx = registry_.create( ByThread(thread), thread_region, otf2::definition::source_code_location()); - - calling_context_tree_.emplace(std::piecewise_construct, std::forward_as_tuple(thread), - std::forward_as_tuple(thread_cctx)); + global_thread_cctxs_.emplace(std::piecewise_construct, std::forward_as_tuple(thread), + std::forward_as_tuple(thread_cctx)); } void Trace::add_thread(Thread thread, const std::string& name) @@ -817,8 +823,8 @@ void Trace::add_monitoring_thread(Thread thread, const std::string& name, const auto& lo2s_cctx = registry_.create( ByThread(thread), ret, otf2::definition::source_code_location()); - calling_context_tree_.emplace(std::piecewise_construct, std::forward_as_tuple(thread), - std::forward_as_tuple(lo2s_cctx)); + global_thread_cctxs_.emplace(std::piecewise_construct, std::forward_as_tuple(thread), + std::forward_as_tuple(lo2s_cctx)); } } @@ -873,28 +879,28 @@ const otf2::definition::string& Trace::intern(const std::string& name) return registry_.emplace(ByString(name), name); } -ThreadCctxRefMap& Trace::create_cctx_refs() +LocalCctxMap& Trace::create_local_cctx_map() { - std::lock_guard guard(cctx_refs_mutex_); + std::lock_guard guard(local_cctx_maps_mutex_); - assert(!cctx_refs_finalized_); + assert(!local_cctx_maps_finalized_); - return cctx_refs_.emplace_back(); + return local_cctx_maps_.emplace_back(); } -void Trace::merge_calling_contexts(const std::map& process_infos) +void Trace::merge_calling_contexts(ProcessMap& process_infos) { - for (auto& cctx : cctx_refs_) + for (auto& local_cctx : local_cctx_maps_) { - assert(cctx.writer != nullptr); - if (cctx.ref_count > 0) + assert(local_cctx.writer != nullptr); + if (local_cctx.num_cctx() > 0) { - const auto& mapping = merge_calling_contexts(cctx.map, cctx.ref_count, process_infos); - (*cctx.writer) << mapping; + const auto& mapping = merge_calling_contexts(local_cctx, process_infos); + (*local_cctx.writer()) << mapping; } } - cctx_refs_.clear(); - auto finalized_twice = cctx_refs_finalized_.exchange(true); + local_cctx_maps_.clear(); + auto finalized_twice = local_cctx_maps_finalized_.exchange(true); if (finalized_twice) { Log::error() << "Trace::merge_calling_contexts was called twice."