diff --git a/clients/drcachesim/analyzer.cpp b/clients/drcachesim/analyzer.cpp index 110a847fc75..97159ed342a 100644 --- a/clients/drcachesim/analyzer.cpp +++ b/clients/drcachesim/analyzer.cpp @@ -308,11 +308,13 @@ analyzer_tmpl_t::init_scheduler_common( } } else if (parallel_) { sched_ops = sched_type_t::make_scheduler_parallel_options(verbosity_); + sched_ops.read_inputs_in_init = options.read_inputs_in_init; if (worker_count_ <= 0) worker_count_ = std::thread::hardware_concurrency(); output_count = worker_count_; } else { sched_ops = sched_type_t::make_scheduler_serial_options(verbosity_); + sched_ops.read_inputs_in_init = options.read_inputs_in_init; worker_count_ = 1; output_count = 1; } @@ -326,6 +328,14 @@ analyzer_tmpl_t::init_scheduler_common( for (int i = 0; i < worker_count_; ++i) { worker_data_.push_back(analyzer_worker_data_t(i, scheduler_.get_stream(i))); + if (options.read_inputs_in_init) { + // The docs say we can query the filetype up front. + uint64_t filetype = scheduler_.get_stream(i)->get_filetype(); + VPRINT(this, 2, "Worker %d filetype %" PRIx64 "\n", i, filetype); + if (TESTANY(OFFLINE_FILE_TYPE_CORE_SHARDED, filetype)) { + shard_type_ = SHARD_BY_CORE; + } + } } return true; @@ -628,9 +638,7 @@ analyzer_tmpl_t::process_tasks_internal( } return false; } - int shard_index = shard_type_ == SHARD_BY_CORE - ? worker->index - : worker->stream->get_input_stream_ordinal(); + int shard_index = worker->stream->get_shard_index(); if (worker->shard_data.find(shard_index) == worker->shard_data.end()) { VPRINT(this, 1, "Worker %d starting on trace shard %d stream is %p\n", worker->index, shard_index, worker->stream); diff --git a/clients/drcachesim/analyzer.h b/clients/drcachesim/analyzer.h index f0b54bc4c05..63a196bed43 100644 --- a/clients/drcachesim/analyzer.h +++ b/clients/drcachesim/analyzer.h @@ -223,6 +223,8 @@ template class analyzer_tmpl_t { // For core-sharded, worker_count_ must be set prior to calling this; for parallel // mode if it is not set it will be set to the underlying core count. + // For core-sharded, all of "options" is used; otherwise, only the + // read_inputs_in_init field is preserved. bool init_scheduler(std::unique_ptr reader, std::unique_ptr reader_end, int verbosity, diff --git a/clients/drcachesim/analyzer_multi.cpp b/clients/drcachesim/analyzer_multi.cpp index 88f65f364e1..492e87c499e 100644 --- a/clients/drcachesim/analyzer_multi.cpp +++ b/clients/drcachesim/analyzer_multi.cpp @@ -455,6 +455,8 @@ analyzer_multi_tmpl_t::analyzer_multi_tmpl_t() return; } auto end = create_ipc_reader_end(); + // We do not want the scheduler's init() to block. + sched_ops.read_inputs_in_init = false; if (!this->init_scheduler(std::move(reader), std::move(end), op_verbose.get_value(), std::move(sched_ops))) { this->success_ = false; diff --git a/clients/drcachesim/common/trace_entry.h b/clients/drcachesim/common/trace_entry.h index 454deddb2b2..342ebd8b252 100644 --- a/clients/drcachesim/common/trace_entry.h +++ b/clients/drcachesim/common/trace_entry.h @@ -1,5 +1,5 @@ /* ********************************************************** - * Copyright (c) 2015-2023 Google, Inc. All rights reserved. + * Copyright (c) 2015-2024 Google, Inc. All rights reserved. * **********************************************************/ /* @@ -963,6 +963,10 @@ typedef enum { * execution. */ OFFLINE_FILE_TYPE_KERNEL_SYSCALL_INSTR_ONLY = 0x8000, + /** + * Each trace shard represents one core and contains interleaved software threads. + */ + OFFLINE_FILE_TYPE_CORE_SHARDED = 0x10000, } offline_file_type_t; static inline const char * diff --git a/clients/drcachesim/reader/reader.cpp b/clients/drcachesim/reader/reader.cpp index 7bc31621fac..7c435cf05e4 100644 --- a/clients/drcachesim/reader/reader.cpp +++ b/clients/drcachesim/reader/reader.cpp @@ -1,5 +1,5 @@ /* ********************************************************** - * Copyright (c) 2016-2023 Google, Inc. All rights reserved. + * Copyright (c) 2016-2024 Google, Inc. All rights reserved. * **********************************************************/ /* @@ -135,7 +135,7 @@ reader_t::process_input_entry() case TRACE_TYPE_PREFETCH_WRITE_L3: case TRACE_TYPE_PREFETCH_WRITE_L3_NT: have_memref = true; - assert(cur_tid_ != 0 && cur_pid_ != 0); + assert((cur_tid_ != 0 && cur_pid_ != 0) || core_sharded_); cur_ref_.data.pid = cur_pid_; cur_ref_.data.tid = cur_tid_; cur_ref_.data.type = (trace_type_t)input_entry_->type; @@ -177,7 +177,7 @@ reader_t::process_input_entry() case TRACE_TYPE_INSTR_RETURN: case TRACE_TYPE_INSTR_SYSENTER: case TRACE_TYPE_INSTR_NO_FETCH: - assert(cur_tid_ != 0 && cur_pid_ != 0); + assert((cur_tid_ != 0 && cur_pid_ != 0) || core_sharded_); if (input_entry_->size == 0) { // Just an entry to tell us the PC of the subsequent memref, // used with -L0_filter where we don't reliably have icache @@ -250,7 +250,7 @@ reader_t::process_input_entry() break; case TRACE_TYPE_INSTR_FLUSH: case TRACE_TYPE_DATA_FLUSH: - assert(cur_tid_ != 0 && cur_pid_ != 0); + assert((cur_tid_ != 0 && cur_pid_ != 0) || core_sharded_); cur_ref_.flush.pid = cur_pid_; cur_ref_.flush.tid = cur_tid_; cur_ref_.flush.type = (trace_type_t)input_entry_->type; @@ -274,7 +274,7 @@ reader_t::process_input_entry() case TRACE_TYPE_THREAD_EXIT: cur_tid_ = (memref_tid_t)input_entry_->addr; cur_pid_ = tid2pid_[cur_tid_]; - assert(cur_tid_ != 0 && cur_pid_ != 0); + assert((cur_tid_ != 0 && cur_pid_ != 0) || core_sharded_); // We do pass this to the caller but only some fields are valid: cur_ref_.exit.pid = cur_pid_; cur_ref_.exit.tid = cur_tid_; @@ -288,7 +288,7 @@ reader_t::process_input_entry() break; case TRACE_TYPE_MARKER: cur_ref_.marker.type = (trace_type_t)input_entry_->type; - assert(cur_tid_ != 0 && cur_pid_ != 0); + assert((cur_tid_ != 0 && cur_pid_ != 0) || core_sharded_); cur_ref_.marker.pid = cur_pid_; cur_ref_.marker.tid = cur_tid_; cur_ref_.marker.marker_type = (trace_marker_type_t)input_entry_->size; @@ -330,6 +330,9 @@ reader_t::process_input_entry() if (TESTANY(OFFLINE_FILE_TYPE_ENCODINGS, filetype_)) { expect_no_encodings_ = false; } + if (TESTANY(OFFLINE_FILE_TYPE_CORE_SHARDED, filetype_)) { + core_sharded_ = true; + } } else if (cur_ref_.marker.marker_type == TRACE_MARKER_TYPE_CACHE_LINE_SIZE) cache_line_size_ = cur_ref_.marker.marker_value; else if (cur_ref_.marker.marker_type == TRACE_MARKER_TYPE_PAGE_SIZE) @@ -346,6 +349,14 @@ reader_t::process_input_entry() in_kernel_trace_ = false; } break; + case TRACE_TYPE_HEADER: + // We support complete traces being packaged in archives and then read + // sequentially, or core-sharded record_filter operation. + // We just keep going past the header. + VPRINT( + this, 2, + "Assuming header is part of concatenated or on-disk-core-sharded traces\n"); + break; default: ERRMSG("Unknown trace entry type %s (%d)\n", trace_type_names[input_entry_->type], input_entry_->type); diff --git a/clients/drcachesim/reader/reader.h b/clients/drcachesim/reader/reader.h index 7b888dce5df..b08ee418dc2 100644 --- a/clients/drcachesim/reader/reader.h +++ b/clients/drcachesim/reader/reader.h @@ -1,5 +1,5 @@ /* ********************************************************** - * Copyright (c) 2015-2023 Google, Inc. All rights reserved. + * Copyright (c) 2015-2024 Google, Inc. All rights reserved. * **********************************************************/ /* @@ -272,6 +272,7 @@ class reader_t : public std::iterator, }; std::unordered_map encodings_; + bool core_sharded_ = false; private: memref_t cur_ref_; diff --git a/clients/drcachesim/scheduler/scheduler.cpp b/clients/drcachesim/scheduler/scheduler.cpp index bd24a88c5b8..77767d300a2 100644 --- a/clients/drcachesim/scheduler/scheduler.cpp +++ b/clients/drcachesim/scheduler/scheduler.cpp @@ -692,6 +692,40 @@ typename scheduler_tmpl_t::scheduler_status_t scheduler_tmpl_t::set_initial_schedule( std::unordered_map> &workload2inputs) { + // Determine whether we need to read ahead in the inputs. There are cases where we + // do not want to do that as it would block forever if the inputs are not available + // (e.g., online analysis IPC readers); it also complicates ordinals so we avoid it + // if we can and enumerate all the cases that do need it. + bool gather_timestamps = false; + if (((options_.mapping == MAP_AS_PREVIOUSLY || + options_.mapping == MAP_TO_ANY_OUTPUT) && + options_.deps == DEPENDENCY_TIMESTAMPS) || + (options_.mapping == MAP_TO_RECORDED_OUTPUT && + options_.replay_as_traced_istream == nullptr && inputs_.size() > 1)) { + gather_timestamps = true; + if (!options_.read_inputs_in_init) { + error_string_ = "Timestamp dependencies require read_inputs_in_init"; + return STATUS_ERROR_INVALID_PARAMETER; + } + } + // The filetype, if present, is before the first timestamp. If we only need the + // filetype we avoid going as far as the timestamp. + bool gather_filetype = options_.read_inputs_in_init; + // Avoid reading ahead for replay as it makes the input ords not match in tests. + if (options_.mapping == MAP_TO_RECORDED_OUTPUT && + options_.replay_as_traced_istream != nullptr) + gather_filetype = false; + if (gather_filetype || gather_timestamps) { + sched_type_t::scheduler_status_t res = + get_initial_input_content(gather_timestamps); + if (res != STATUS_SUCCESS) { + error_string_ = "Failed to read initial input contents for filetype"; + if (gather_timestamps) + error_string_ += " and initial timestamps"; + return res; + } + } + if (options_.mapping == MAP_AS_PREVIOUSLY) { live_replay_output_count_.store(static_cast(outputs_.size()), std::memory_order_release); @@ -703,9 +737,7 @@ scheduler_tmpl_t::set_initial_schedule( return STATUS_ERROR_INVALID_PARAMETER; if (options_.deps == DEPENDENCY_TIMESTAMPS) { // Match the ordinals from the original run by pre-reading the timestamps. - sched_type_t::scheduler_status_t res = get_initial_timestamps(); - if (res != STATUS_SUCCESS) - return res; + assert(gather_timestamps); } } else if (options_.schedule_replay_istream != nullptr) { return STATUS_ERROR_INVALID_PARAMETER; @@ -740,9 +772,7 @@ scheduler_tmpl_t::set_initial_schedule( // thread first and then pick the oldest timestamp once it reached a // timestamp. We instead queue those headers so we can start directly with the // oldest timestamp's thread. - sched_type_t::scheduler_status_t res = get_initial_timestamps(); - if (res != STATUS_SUCCESS) - return res; + assert(gather_timestamps); uint64_t min_time = std::numeric_limits::max(); input_ordinal_t min_input = -1; for (int i = 0; i < static_cast(inputs_.size()); ++i) { @@ -758,11 +788,7 @@ scheduler_tmpl_t::set_initial_schedule( } else { // Assign initial inputs. if (options_.deps == DEPENDENCY_TIMESTAMPS) { - sched_type_t::scheduler_status_t res = get_initial_timestamps(); - if (res != STATUS_SUCCESS) { - error_string_ = "Failed to find initial timestamps"; - return res; - } + assert(gather_timestamps); // Compute the min timestamp (==base_timestamp) per workload and sort // all inputs by relative time from the base. for (int workload_idx = 0; @@ -1254,28 +1280,75 @@ scheduler_tmpl_t::read_switch_sequences() template typename scheduler_tmpl_t::scheduler_status_t -scheduler_tmpl_t::get_initial_timestamps() -{ +scheduler_tmpl_t::get_initial_input_content( + bool gather_timestamps) +{ + // For every mode, read ahead until we see a filetype record so the user can + // examine it prior to retrieving any records. + VPRINT(this, 1, "Reading headers from inputs to find filetypes%s\n", + gather_timestamps ? " and timestamps" : ""); + assert(options_.read_inputs_in_init); // Read ahead in each input until we find a timestamp record. // Queue up any skipped records to ensure we present them to the // output stream(s). for (size_t i = 0; i < inputs_.size(); ++i) { input_info_t &input = inputs_[i]; - if (input.next_timestamp <= 0) { + bool found_filetype = false; + bool found_timestamp = !gather_timestamps || input.next_timestamp > 0; + if (!found_filetype || !found_timestamp) { + // First, check any queued records in the input. + // XXX: Can we create a helper to iterate the queue and then the + // reader, and avoid the duplicated loops here? The challenge is + // the non-consuming queue loop vs the consuming and queue-pushback + // reader loop. for (const auto &record : input.queue) { + trace_marker_type_t marker_type; + uintptr_t marker_value; + if (record_type_is_marker(record, marker_type, marker_value) && + marker_type == TRACE_MARKER_TYPE_FILETYPE) { + found_filetype = true; + VPRINT(this, 2, "Input %zu filetype %zu\n", i, marker_value); + } if (record_type_is_timestamp(record, input.next_timestamp)) + found_timestamp = true; + if (found_filetype && found_timestamp) break; } } - if (input.next_timestamp <= 0) { + if (input.next_timestamp > 0) + found_timestamp = true; + if (!found_filetype || !found_timestamp) { + // If we didn't find our targets in the queue, request new records. if (input.needs_init) { input.reader->init(); input.needs_init = false; } while (*input.reader != *input.reader_end) { RecordType record = **input.reader; + trace_marker_type_t marker_type; + uintptr_t marker_value; + if (record_type_is_marker(record, marker_type, marker_value) && + marker_type == TRACE_MARKER_TYPE_FILETYPE) { + found_filetype = true; + VPRINT(this, 2, "Input %zu filetype %zu\n", i, marker_value); + } if (record_type_is_timestamp(record, input.next_timestamp)) + found_timestamp = true; + if (found_filetype && found_timestamp) break; + // Don't go too far if only looking for filetype, to avoid reaching + // the first instruction, which causes problems with ordinals when + // there is no filetype as happens in legacy traces (and unit tests). + // Just exit with a 0 filetype. + if (!found_filetype && + (record_type_is_timestamp(record, marker_value) || + (record_type_is_marker(record, marker_type, marker_value) && + marker_type == TRACE_MARKER_TYPE_PAGE_SIZE))) { + VPRINT(this, 2, "No filetype found: assuming unit test input.\n"); + found_filetype = true; + if (!gather_timestamps) + break; + } // If we see an instruction, there may be no timestamp (a malformed // synthetic trace in a test) or we may have to read thousands of records // to find it if it were somehow missing, which we do not want to do. We @@ -1287,7 +1360,7 @@ scheduler_tmpl_t::get_initial_timestamps() ++(*input.reader); } } - if (input.next_timestamp <= 0) + if (gather_timestamps && input.next_timestamp <= 0) return STATUS_ERROR_INVALID_PARAMETER; } return STATUS_SUCCESS; @@ -1456,6 +1529,10 @@ scheduler_tmpl_t::get_output_cpuid(output_ordinal_t outp { if (options_.replay_as_traced_istream != nullptr) return outputs_[output].as_traced_cpuid; + int index = outputs_[output].cur_input; + if (index >= 0 && + TESTANY(OFFLINE_FILE_TYPE_CORE_SHARDED, inputs_[index].reader->get_filetype())) + return outputs_[output].cur_input; return output; } @@ -1471,6 +1548,64 @@ scheduler_tmpl_t::get_input_stream(output_ordinal_t outp return inputs_[index].reader.get(); } +template +uint64_t +scheduler_tmpl_t::get_input_record_ordinal( + output_ordinal_t output) +{ + if (output < 0 || output >= static_cast(outputs_.size())) + return 0; + int index = outputs_[output].cur_input; + if (index < 0) + return 0; + uint64_t ord = inputs_[index].reader->get_record_ordinal(); + if (inputs_[index].reader->get_instruction_ordinal() == 0) { + // Account for get_initial_input_content() readahead for filetype/timestamp. + // If this gets any more complex, the scheduler stream should track its + // own counts for every input and just ignore the input stream's tracking. + ord -= inputs_[index].queue.size() + (inputs_[index].cur_from_queue ? 1 : 0); + } + return ord; +} + +template +uint64_t +scheduler_tmpl_t::get_input_first_timestamp( + output_ordinal_t output) +{ + if (output < 0 || output >= static_cast(outputs_.size())) + return 0; + int index = outputs_[output].cur_input; + if (index < 0) + return 0; + uint64_t res = inputs_[index].reader->get_first_timestamp(); + if (inputs_[index].reader->get_instruction_ordinal() == 0 && + (!inputs_[index].queue.empty() || inputs_[index].cur_from_queue)) { + // Account for get_initial_input_content() readahead for filetype/timestamp. + res = 0; + } + return res; +} + +template +uint64_t +scheduler_tmpl_t::get_input_last_timestamp( + output_ordinal_t output) +{ + if (output < 0 || output >= static_cast(outputs_.size())) + return 0; + int index = outputs_[output].cur_input; + if (index < 0) + return 0; + uint64_t res = inputs_[index].reader->get_last_timestamp(); + if (inputs_[index].reader->get_instruction_ordinal() == 0 && + (!inputs_[index].queue.empty() || inputs_[index].cur_from_queue)) { + // Account for get_initial_input_content() readahead for filetype/timestamp. + res = 0; + } + return res; +} + template typename scheduler_tmpl_t::stream_status_t scheduler_tmpl_t::advance_region_of_interest( @@ -1895,6 +2030,11 @@ scheduler_tmpl_t::set_cur_input(output_ordinal_t output, std::lock_guard lock(*inputs_[input].lock); + if (prev_input < 0 && outputs_[output].stream->filetype_ == 0) { + // Set the filetype up front, to let the user query at init time as documented. + outputs_[output].stream->filetype_ = inputs_[input].reader->get_filetype(); + } + if (!switch_sequence_.empty() && outputs_[output].stream->get_instruction_ordinal() > 0) { sched_type_t::switch_type_t switch_type = SWITCH_INVALID; @@ -2327,7 +2467,7 @@ scheduler_tmpl_t::next_record(output_ordinal_t output, return sched_type_t::STATUS_OK; } while (true) { - bool from_queue = false; + input->cur_from_queue = false; if (input->needs_init) { // We pay the cost of this conditional to support ipc_reader_t::init() which // blocks and must be called right before reading its first record. @@ -2341,7 +2481,7 @@ scheduler_tmpl_t::next_record(output_ordinal_t output, if (!input->queue.empty()) { record = input->queue.front(); input->queue.pop_front(); - from_queue = true; + input->cur_from_queue = true; } else { // We again have a flag check because reader_t::init() does an initial ++ // and so we want to skip that on the first record but perform a ++ prior @@ -2409,7 +2549,7 @@ scheduler_tmpl_t::next_record(output_ordinal_t output, // to get into the trace reading loop and then do something like a skip // from the start rather than adding logic into the setup code). if (input->reader->get_instruction_ordinal() >= stop && - (!from_queue || (start == 0 && stop == 0))) { + (!input->cur_from_queue || (start == 0 && stop == 0))) { VPRINT(this, 5, "next_record[%d]: need new input: at end of segment in=%d " "stop=%" PRId64 "\n", diff --git a/clients/drcachesim/scheduler/scheduler.h b/clients/drcachesim/scheduler/scheduler.h index 78bc71fbf64..4e6ea8271c2 100644 --- a/clients/drcachesim/scheduler/scheduler.h +++ b/clients/drcachesim/scheduler/scheduler.h @@ -394,6 +394,8 @@ template class scheduler_tmpl_t { * dependencies: thus, timestamp ordering will be followed at context switch * points for picking the next input, but timestamps will not preempt an input. * To precisely follow the recorded timestamps, use #MAP_TO_RECORDED_OUTPUT. + * If this flag is on, #dynamorio::drmemtrace::scheduler_tmpl_t:: + * scheduler_options_t.read_inputs_in_init must be set to true. */ DEPENDENCY_TIMESTAMPS_BITFIELD = 0x01, /** @@ -623,6 +625,17 @@ template class scheduler_tmpl_t { * sensitivity studies. */ bool randomize_next_input = false; + /** + * If true, the scheduler will read from each input to determine its filetype + * during initialization. If false, the filetype will not be available prior + * to explicit record retrieval by the user, but this may be required for + * inputs whose sources are not yet set up at scheduler init time (e.g., + * inputs over blocking pipes with data only becoming available after + * initializing the scheduler, as happens with online trace analyzers). + * This must be true for #DEPENDENCY_TIMESTAMPS as it also requires reading + * ahead. + */ + bool read_inputs_in_init = true; }; /** @@ -774,7 +787,7 @@ template class scheduler_tmpl_t { { if (TESTANY(sched_type_t::SCHEDULER_USE_INPUT_ORDINALS, scheduler_->options_.flags)) - return scheduler_->get_input_stream(ordinal_)->get_record_ordinal(); + return scheduler_->get_input_record_ordinal(ordinal_); return cur_ref_count_; } /** @@ -837,7 +850,7 @@ template class scheduler_tmpl_t { { if (TESTANY(sched_type_t::SCHEDULER_USE_INPUT_ORDINALS, scheduler_->options_.flags)) - return scheduler_->get_input_stream(ordinal_)->get_last_timestamp(); + return scheduler_->get_input_last_timestamp(ordinal_); return last_timestamp_; } /** @@ -848,7 +861,7 @@ template class scheduler_tmpl_t { { if (TESTANY(sched_type_t::SCHEDULER_USE_INPUT_ORDINALS, scheduler_->options_.flags)) - return scheduler_->get_input_stream(ordinal_)->get_first_timestamp(); + return scheduler_->get_input_first_timestamp(ordinal_); return first_timestamp_; } /** @@ -865,6 +878,10 @@ template class scheduler_tmpl_t { * #offline_file_type_t identifying the architecture and * other key high-level attributes of the trace from the * #TRACE_MARKER_TYPE_FILETYPE record in the trace header. + * This can be queried prior to explicitly retrieving any records from + * output streams, unless #dynamorio::drmemtrace::scheduler_tmpl_t:: + * scheduler_options_t.read_inputs_in_init is false. + */ uint64_t get_filetype() const override @@ -916,7 +933,9 @@ template class scheduler_tmpl_t { /** * Returns a unique identifier for the current output stream. For * #MAP_TO_RECORDED_OUTPUT, the identifier is the as-traced cpuid mapped to this - * output. For dynamic schedules, the identifier is the output stream ordinal. + * output. For dynamic schedules, the identifier is the output stream ordinal, + * except for #OFFLINE_FILE_TYPE_CORE_SHARDED inputs where the identifier + * is the input stream ordinal. */ int64_t get_output_cpuid() const override @@ -1138,6 +1157,7 @@ template class scheduler_tmpl_t { // This is used for read-ahead and inserting synthetic records. // We use a deque so we can iterate over it. std::deque queue; + bool cur_from_queue; std::set binding; int priority = 0; std::vector regions_of_interest; @@ -1306,10 +1326,11 @@ template class scheduler_tmpl_t { set_initial_schedule(std::unordered_map> &workload2inputs); // Assumed to only be called at initialization time. - // Reads ahead in each input to find its first timestamp (queuing the records - // read to feed to the user's first requests). + // Reads ahead in each input to find its filetype, and if "gather_timestamps" + // is set, to find its first timestamp, queuing all records + // read to feed to the user's first requests. scheduler_status_t - get_initial_timestamps(); + get_initial_input_content(bool gather_timestamps); // Opens up all the readers for each file in 'path' which may be a directory. // Returns a map of the thread id of each file to its index in inputs_. @@ -1485,6 +1506,21 @@ template class scheduler_tmpl_t { memtrace_stream_t * get_input_stream(output_ordinal_t output); + // Returns the record ordinal for the current input stream interface for the + // 'output_ordinal'-th output stream. + uint64_t + get_input_record_ordinal(output_ordinal_t output); + + // Returns the first timestamp for the current input stream interface for the + // 'output_ordinal'-th output stream. + uint64_t + get_input_first_timestamp(output_ordinal_t output); + + // Returns the last timestamp for the current input stream interface for the + // 'output_ordinal'-th output stream. + uint64_t + get_input_last_timestamp(output_ordinal_t output); + stream_status_t start_speculation(output_ordinal_t output, addr_t start_address, bool queue_current_record); diff --git a/clients/drcachesim/tests/core_on_disk.templatex b/clients/drcachesim/tests/core_on_disk.templatex new file mode 100644 index 00000000000..3b9203f203d --- /dev/null +++ b/clients/drcachesim/tests/core_on_disk.templatex @@ -0,0 +1,29 @@ +Basic counts tool results: +Total counts: +.* + 8 total threads +.* +Core [0-5] counts: +.* + 2 threads +.* +Core [0-5] counts: +.* + 2 threads +.* +Core [0-5] counts: +.* + 2 threads +.* +Core [0-5] counts: +.* + 1 threads +.* +Core [0-5] counts: +.* + 1 threads +.* +Core [0-5] counts: +.* + 1 threads +.* diff --git a/clients/drcachesim/tests/drmemtrace.threadsig-core-sharded.x64.tracedir/drmemtrace.threadsig.1257596.5136.trace.zip b/clients/drcachesim/tests/drmemtrace.threadsig-core-sharded.x64.tracedir/drmemtrace.threadsig.1257596.5136.trace.zip new file mode 100644 index 00000000000..1a2de7d9aa3 Binary files /dev/null and b/clients/drcachesim/tests/drmemtrace.threadsig-core-sharded.x64.tracedir/drmemtrace.threadsig.1257596.5136.trace.zip differ diff --git a/clients/drcachesim/tests/drmemtrace.threadsig-core-sharded.x64.tracedir/drmemtrace.threadsig.1257598.2655.trace.zip b/clients/drcachesim/tests/drmemtrace.threadsig-core-sharded.x64.tracedir/drmemtrace.threadsig.1257598.2655.trace.zip new file mode 100644 index 00000000000..e2c996f3493 Binary files /dev/null and b/clients/drcachesim/tests/drmemtrace.threadsig-core-sharded.x64.tracedir/drmemtrace.threadsig.1257598.2655.trace.zip differ diff --git a/clients/drcachesim/tests/drmemtrace.threadsig-core-sharded.x64.tracedir/drmemtrace.threadsig.1257599.2824.trace.zip b/clients/drcachesim/tests/drmemtrace.threadsig-core-sharded.x64.tracedir/drmemtrace.threadsig.1257599.2824.trace.zip new file mode 100644 index 00000000000..3fda1bf3c86 Binary files /dev/null and b/clients/drcachesim/tests/drmemtrace.threadsig-core-sharded.x64.tracedir/drmemtrace.threadsig.1257599.2824.trace.zip differ diff --git a/clients/drcachesim/tests/drmemtrace.threadsig-core-sharded.x64.tracedir/drmemtrace.threadsig.1257600.9375.trace.zip b/clients/drcachesim/tests/drmemtrace.threadsig-core-sharded.x64.tracedir/drmemtrace.threadsig.1257600.9375.trace.zip new file mode 100644 index 00000000000..9c36953f97b Binary files /dev/null and b/clients/drcachesim/tests/drmemtrace.threadsig-core-sharded.x64.tracedir/drmemtrace.threadsig.1257600.9375.trace.zip differ diff --git a/clients/drcachesim/tests/drmemtrace.threadsig-core-sharded.x64.tracedir/drmemtrace.threadsig.1257601.8161.trace.zip b/clients/drcachesim/tests/drmemtrace.threadsig-core-sharded.x64.tracedir/drmemtrace.threadsig.1257601.8161.trace.zip new file mode 100644 index 00000000000..61e682e43fa Binary files /dev/null and b/clients/drcachesim/tests/drmemtrace.threadsig-core-sharded.x64.tracedir/drmemtrace.threadsig.1257601.8161.trace.zip differ diff --git a/clients/drcachesim/tests/drmemtrace.threadsig-core-sharded.x64.tracedir/drmemtrace.threadsig.1257604.1983.trace.zip b/clients/drcachesim/tests/drmemtrace.threadsig-core-sharded.x64.tracedir/drmemtrace.threadsig.1257604.1983.trace.zip new file mode 100644 index 00000000000..ca76bc2b329 Binary files /dev/null and b/clients/drcachesim/tests/drmemtrace.threadsig-core-sharded.x64.tracedir/drmemtrace.threadsig.1257604.1983.trace.zip differ diff --git a/clients/drcachesim/tests/scheduler_unit_tests.cpp b/clients/drcachesim/tests/scheduler_unit_tests.cpp index 56757dc8874..d18bcd0253a 100644 --- a/clients/drcachesim/tests/scheduler_unit_tests.cpp +++ b/clients/drcachesim/tests/scheduler_unit_tests.cpp @@ -244,10 +244,12 @@ test_parallel() else assert(tid2stream[memref.instr.tid] == i); // Ensure the ordinals do not accumulate across inputs. - assert( - stream->get_record_ordinal() == - scheduler.get_input_stream_interface(stream->get_input_stream_ordinal()) - ->get_record_ordinal()); + assert(stream->get_record_ordinal() == + scheduler + .get_input_stream_interface(stream->get_input_stream_ordinal()) + ->get_record_ordinal() || + // Relax for early on where the scheduler has read ahead. + stream->get_last_timestamp() == 0); assert( stream->get_instruction_ordinal() == scheduler.get_input_stream_interface(stream->get_input_stream_ordinal()) diff --git a/clients/drcachesim/tests/trace_interval_analysis_unit_tests.cpp b/clients/drcachesim/tests/trace_interval_analysis_unit_tests.cpp index ac684d21594..5f991cb729a 100644 --- a/clients/drcachesim/tests/trace_interval_analysis_unit_tests.cpp +++ b/clients/drcachesim/tests/trace_interval_analysis_unit_tests.cpp @@ -145,6 +145,11 @@ class test_stream_t : public scheduler_t::stream_t { } return 0; } + int + get_shard_index() const override + { + return get_input_stream_ordinal(); + } private: std::unordered_map tid2ordinal_; diff --git a/suite/tests/CMakeLists.txt b/suite/tests/CMakeLists.txt index 8df22fa5c1b..362997e98cb 100644 --- a/suite/tests/CMakeLists.txt +++ b/suite/tests/CMakeLists.txt @@ -3872,6 +3872,13 @@ if (BUILD_CLIENTS) "-indir ${thread_trace_dir} -simulator_type reuse_time:reuse_distance:histogram:opcode_mix:syscall_mix -core_sharded" "") set(tool.core_sharded_rawtemp ON) # no preprocessor + + # Test analysis of core-sharded-on-disk traces. + set(core_sharded_dir + "${PROJECT_SOURCE_DIR}/clients/drcachesim/tests/drmemtrace.threadsig-core-sharded.x64.tracedir") + torunonly_simtool(core_on_disk ${ci_shared_app} + "-indir ${core_sharded_dir} -simulator_type basic_counts" "") + set(tool.core_on_disk_rawtemp ON) # no preprocessor endif () endif ()