parent
7521c2c644
commit
fabded95df
|
@ -8,16 +8,17 @@ import collections
|
|||
import math
|
||||
import re
|
||||
import statistics
|
||||
from collections import OrderedDict
|
||||
# from pprint import pprint
|
||||
|
||||
Sections = []
|
||||
Sections = OrderedDict()
|
||||
LongestVcdStrValueLength = 0
|
||||
Threads = collections.defaultdict(lambda: []) # List of records per thread id
|
||||
Mtasks = collections.defaultdict(lambda: {'elapsed': 0, 'end': 0})
|
||||
Cpus = collections.defaultdict(lambda: {'mtask_time': 0})
|
||||
Global = {'args': {}, 'cpuinfo': collections.defaultdict(lambda: {}), 'stats': {}}
|
||||
ElapsedTime = None # total elapsed time
|
||||
ExecGraphTime = 0 # total elapsed time excuting an exec graph
|
||||
ExecGraphTime = 0 # total elapsed time executing an exec graph
|
||||
ExecGraphIntervals = [] # list of (start, end) pairs
|
||||
|
||||
######################################################################
|
||||
|
@ -37,11 +38,11 @@ def read_data(filename):
|
|||
re_proc_dat = re.compile(r'VLPROFPROC ([a-z_ ]+)\s*:\s*(.*)$')
|
||||
cpu = None
|
||||
thread = 0
|
||||
execGraphStart = None
|
||||
|
||||
global LongestVcdStrValueLength
|
||||
global ExecGraphTime
|
||||
|
||||
ExecGraphStack = []
|
||||
SectionStack = []
|
||||
mTaskThread = {}
|
||||
|
||||
|
@ -88,16 +89,17 @@ def read_data(filename):
|
|||
Mtasks[mtask]['predict_cost'] = predict_cost
|
||||
Mtasks[mtask]['end'] = max(Mtasks[mtask]['end'], tick)
|
||||
elif kind == "EXEC_GRAPH_BEGIN":
|
||||
execGraphStart = tick
|
||||
ExecGraphStack.append(tick)
|
||||
elif kind == "EXEC_GRAPH_END":
|
||||
assert ExecGraphStack, "EXEC_GRAPH_END without EXEC_GRAPH_BEGIN"
|
||||
execGraphStart = ExecGraphStack.pop()
|
||||
ExecGraphTime += tick - execGraphStart
|
||||
ExecGraphIntervals.append((execGraphStart, tick))
|
||||
execGraphStart = None
|
||||
elif Args.debug:
|
||||
print("-Unknown execution trace record: %s" % line)
|
||||
elif re_thread.match(line):
|
||||
thread = int(re_thread.match(line).group(1))
|
||||
Sections.append([])
|
||||
Sections[thread] = []
|
||||
elif re.match(r'^VLPROF(THREAD|VERSION)', line):
|
||||
pass
|
||||
elif re_arg1.match(line):
|
||||
|
@ -308,7 +310,7 @@ def report_cpus():
|
|||
|
||||
|
||||
def report_sections():
|
||||
for thread, section in enumerate(Sections):
|
||||
for thread, section in Sections.items():
|
||||
if section:
|
||||
print(f"\nSection profile for thread {thread}:")
|
||||
report_section(section)
|
||||
|
@ -432,7 +434,10 @@ def write_vcd(filename):
|
|||
# Find the earliest MTask start after the start point, and the
|
||||
# latest MTask end before the end point, so we can scale to the
|
||||
# same range
|
||||
start = tStart[bisect.bisect_left(tStart, start)]
|
||||
tStartIdx = bisect.bisect_left(tStart, start)
|
||||
if tStartIdx >= len(tStart):
|
||||
continue
|
||||
start = tStart[tStartIdx]
|
||||
end = tEnd[bisect.bisect_right(tEnd, end) - 1]
|
||||
# Compute scale so predicted graph is of same width as interval
|
||||
measured_scaling = (end - start) / Global['predict_last_end']
|
||||
|
@ -462,7 +467,7 @@ def write_vcd(filename):
|
|||
addValue(pcode, time, value)
|
||||
|
||||
# Section graph
|
||||
for thread, section in enumerate(Sections):
|
||||
for thread, section in Sections.items():
|
||||
if section:
|
||||
scode = getCode(LongestVcdStrValueLength * 8, "section", f"t{thread}_trace")
|
||||
dcode = getCode(32, "section", f"t{thread}_depth")
|
||||
|
|
|
@ -2124,6 +2124,20 @@ The grammar of configuration commands is as follows:
|
|||
This option should not be used directly.
|
||||
See :ref:`Hierarchical Verilation`.
|
||||
|
||||
.. option:: hier_workers -hier-dpi "<function_name>" -workers <worker_count>
|
||||
|
||||
Specifies how many threads need to be used for scheduling hierarchical DPI
|
||||
tasks. This data is inserted internally during :vlopt:`--hierarchical`,
|
||||
based on value specified in :option:`hier_workers -module`. This option
|
||||
should not be used directly. See :ref:`Hierarchical Verilation`.
|
||||
|
||||
.. option:: hier_workers -module "<module_name>" -workers <worker_count>
|
||||
|
||||
Specifies how many threads need to be used for scheduling given module with
|
||||
:option:`/*verilator&32;hier_block*/` metacomment. This number needs to be
|
||||
smaller than :vlopt:`--threads` to fit in a thread schedule.
|
||||
See :ref:`Hierarchical Verilation`.
|
||||
|
||||
.. option:: inline -module "<modulename>"
|
||||
|
||||
Specifies the module may be inlined into any modules that use this
|
||||
|
|
|
@ -100,7 +100,10 @@ void VlWorkerThread::startWorker(VlWorkerThread* workerp, VerilatedContext* cont
|
|||
// VlThreadPool
|
||||
|
||||
VlThreadPool::VlThreadPool(VerilatedContext* contextp, unsigned nThreads) {
|
||||
for (unsigned i = 0; i < nThreads; ++i) m_workers.push_back(new VlWorkerThread{contextp});
|
||||
for (unsigned i = 0; i < nThreads; ++i) {
|
||||
m_workers.push_back(new VlWorkerThread{contextp});
|
||||
m_unassignedWorkers.push(i);
|
||||
}
|
||||
}
|
||||
|
||||
VlThreadPool::~VlThreadPool() {
|
||||
|
|
|
@ -30,6 +30,7 @@
|
|||
#include <atomic>
|
||||
#include <condition_variable>
|
||||
#include <set>
|
||||
#include <stack>
|
||||
#include <thread>
|
||||
#include <vector>
|
||||
|
||||
|
@ -205,6 +206,13 @@ class VlThreadPool final : public VerilatedVirtualBase {
|
|||
// MEMBERS
|
||||
std::vector<VlWorkerThread*> m_workers; // our workers
|
||||
|
||||
// Guards indexes of unassigned workers
|
||||
mutable VerilatedMutex m_mutex;
|
||||
// Indexes of unassigned workers
|
||||
std::stack<size_t> m_unassignedWorkers VL_GUARDED_BY(m_mutex);
|
||||
// Used for sequentially generating task IDs to avoid shadowing
|
||||
std::atomic<unsigned> m_assignedTasks{0};
|
||||
|
||||
public:
|
||||
// CONSTRUCTORS
|
||||
// Construct a thread pool with 'nThreads' dedicated threads. The thread
|
||||
|
@ -214,6 +222,19 @@ public:
|
|||
~VlThreadPool() override;
|
||||
|
||||
// METHODS
|
||||
size_t assignWorkerIndex() {
|
||||
const VerilatedLockGuard lock{m_mutex};
|
||||
assert(!m_unassignedWorkers.empty());
|
||||
const size_t index = m_unassignedWorkers.top();
|
||||
m_unassignedWorkers.pop();
|
||||
return index;
|
||||
}
|
||||
void freeWorkerIndexes(std::vector<size_t>& indexes) {
|
||||
const VerilatedLockGuard lock{m_mutex};
|
||||
for (size_t index : indexes) m_unassignedWorkers.push(index);
|
||||
indexes.clear();
|
||||
}
|
||||
unsigned assignTaskIndex() { return m_assignedTasks++; }
|
||||
int numThreads() const { return static_cast<int>(m_workers.size()); }
|
||||
VlWorkerThread* workerp(int index) {
|
||||
assert(index >= 0);
|
||||
|
|
|
@ -540,6 +540,8 @@ class V3ConfigResolver final {
|
|||
std::unordered_map<string, std::unordered_map<string, uint64_t>>
|
||||
m_profileData; // Access to profile_data records
|
||||
uint8_t m_mode = NONE;
|
||||
std::unordered_map<string, int> m_hierWorkers;
|
||||
FileLine* m_hierWorkersFileLine = nullptr;
|
||||
FileLine* m_profileFileLine = nullptr;
|
||||
|
||||
V3ConfigResolver() = default;
|
||||
|
@ -570,6 +572,16 @@ public:
|
|||
// Empty key for hierarchical DPI wrapper costs.
|
||||
return getProfileData(hierDpi, "");
|
||||
}
|
||||
void addHierWorkers(FileLine* fl, const string& model, int workers) {
|
||||
if (!m_hierWorkersFileLine) m_hierWorkersFileLine = fl;
|
||||
m_hierWorkers[model] = workers;
|
||||
}
|
||||
int getHierWorkers(const string& model) const {
|
||||
const auto mit = m_hierWorkers.find(model);
|
||||
// Assign a single worker if no specified.
|
||||
return mit != m_hierWorkers.cend() ? mit->second : 0;
|
||||
}
|
||||
FileLine* getHierWorkersFileLine() const { return m_hierWorkersFileLine; }
|
||||
uint64_t getProfileData(const string& model, const string& key) const {
|
||||
const auto mit = m_profileData.find(model);
|
||||
if (mit == m_profileData.cend()) return 0;
|
||||
|
@ -602,6 +614,10 @@ void V3Config::addCoverageBlockOff(const string& module, const string& blockname
|
|||
V3ConfigResolver::s().modules().at(module).addCoverageBlockOff(blockname);
|
||||
}
|
||||
|
||||
void V3Config::addHierWorkers(FileLine* fl, const string& model, int workers) {
|
||||
V3ConfigResolver::s().addHierWorkers(fl, model, workers);
|
||||
}
|
||||
|
||||
void V3Config::addIgnore(V3ErrorCode code, bool on, const string& filename, int min, int max) {
|
||||
if (filename == "*") {
|
||||
FileLine::globalWarnOff(code, !on);
|
||||
|
@ -741,6 +757,12 @@ void V3Config::applyVarAttr(AstNodeModule* modulep, AstNodeFTask* ftaskp, AstVar
|
|||
if (vp) vp->apply(varp);
|
||||
}
|
||||
|
||||
int V3Config::getHierWorkers(const string& model) {
|
||||
return V3ConfigResolver::s().getHierWorkers(model);
|
||||
}
|
||||
FileLine* V3Config::getHierWorkersFileLine() {
|
||||
return V3ConfigResolver::s().getHierWorkersFileLine();
|
||||
}
|
||||
uint64_t V3Config::getProfileData(const string& hierDpi) {
|
||||
return V3ConfigResolver::s().getProfileData(hierDpi);
|
||||
}
|
||||
|
|
|
@ -33,6 +33,7 @@ public:
|
|||
static void addCaseParallel(const string& file, int lineno);
|
||||
static void addCoverageBlockOff(const string& file, int lineno);
|
||||
static void addCoverageBlockOff(const string& module, const string& blockname);
|
||||
static void addHierWorkers(FileLine* fl, const string& model, int workers);
|
||||
static void addIgnore(V3ErrorCode code, bool on, const string& filename, int min, int max);
|
||||
static void addIgnoreMatch(V3ErrorCode code, const string& filename, const string& contents,
|
||||
const string& match);
|
||||
|
@ -52,6 +53,8 @@ public:
|
|||
static void applyModule(AstNodeModule* modulep);
|
||||
static void applyVarAttr(AstNodeModule* modulep, AstNodeFTask* ftaskp, AstVar* varp);
|
||||
|
||||
static int getHierWorkers(const string& model);
|
||||
static FileLine* getHierWorkersFileLine();
|
||||
static uint64_t getProfileData(const string& hierDpi);
|
||||
static uint64_t getProfileData(const string& model, const string& key);
|
||||
static FileLine* getProfileDataFileLine();
|
||||
|
|
|
@ -63,6 +63,10 @@ namespace V3ExecGraph {
|
|||
class ThreadSchedule final {
|
||||
friend class PackThreads;
|
||||
|
||||
uint32_t m_id; // Unique ID of a schedule
|
||||
static uint32_t s_nextId; // Next ID number to use
|
||||
std::unordered_set<const ExecMTask*> mtasks; // Mtasks in this schedule
|
||||
|
||||
public:
|
||||
// CONSTANTS
|
||||
static constexpr uint32_t UNASSIGNED = 0xffffffff;
|
||||
|
@ -79,21 +83,18 @@ public:
|
|||
// the sequence of MTasks to be executed by that thread.
|
||||
std::vector<std::vector<const ExecMTask*>> threads;
|
||||
|
||||
// State for each mtask.
|
||||
std::unordered_map<const ExecMTask*, MTaskState> mtaskState;
|
||||
// Global state for each mtask.
|
||||
static std::unordered_map<const ExecMTask*, MTaskState> mtaskState;
|
||||
|
||||
uint32_t threadId(const ExecMTask* mtaskp) const {
|
||||
const auto& it = mtaskState.find(mtaskp);
|
||||
return it != mtaskState.end() ? it->second.threadId : UNASSIGNED;
|
||||
}
|
||||
|
||||
private:
|
||||
explicit ThreadSchedule(uint32_t nThreads)
|
||||
: threads{nThreads} {}
|
||||
VL_UNCOPYABLE(ThreadSchedule); // But movable
|
||||
: m_id(s_nextId++)
|
||||
, threads{nThreads} {}
|
||||
ThreadSchedule(ThreadSchedule&&) = default;
|
||||
ThreadSchedule& operator=(ThreadSchedule&&) = default;
|
||||
|
||||
private:
|
||||
VL_UNCOPYABLE(ThreadSchedule);
|
||||
|
||||
// Debugging
|
||||
void dumpDotFile(const V3Graph& graph, const string& filename) const {
|
||||
// This generates a file used by graphviz, https://www.graphviz.org
|
||||
|
@ -168,6 +169,17 @@ private:
|
|||
}
|
||||
|
||||
public:
|
||||
static uint32_t threadId(const ExecMTask* mtaskp) {
|
||||
const auto& it = mtaskState.find(mtaskp);
|
||||
return it != mtaskState.end() ? it->second.threadId : UNASSIGNED;
|
||||
}
|
||||
static uint32_t startTime(const ExecMTask* mtaskp) {
|
||||
return mtaskState.at(mtaskp).completionTime - mtaskp->cost();
|
||||
}
|
||||
static uint32_t endTime(const ExecMTask* mtaskp) {
|
||||
return mtaskState.at(mtaskp).completionTime;
|
||||
}
|
||||
|
||||
// Returns the number of cross-thread dependencies of the given MTask. If > 0, the MTask must
|
||||
// test whether its dependencies are ready before starting, and therefore may need to block.
|
||||
uint32_t crossThreadDependencies(const ExecMTask* mtaskp) const {
|
||||
|
@ -175,19 +187,32 @@ public:
|
|||
uint32_t result = 0;
|
||||
for (const V3GraphEdge& edge : mtaskp->inEdges()) {
|
||||
const ExecMTask* const prevp = edge.fromp()->as<ExecMTask>();
|
||||
if (threadId(prevp) != thisThreadId) ++result;
|
||||
if (threadId(prevp) != thisThreadId && contains(prevp)) ++result;
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
uint32_t startTime(const ExecMTask* mtaskp) const {
|
||||
return mtaskState.at(mtaskp).completionTime - mtaskp->cost();
|
||||
}
|
||||
uint32_t endTime(const ExecMTask* mtaskp) const {
|
||||
return mtaskState.at(mtaskp).completionTime;
|
||||
uint32_t id() const { return m_id; }
|
||||
uint32_t scheduleOn(const ExecMTask* mtaskp, uint32_t bestThreadId) {
|
||||
mtasks.emplace(mtaskp);
|
||||
const uint32_t bestEndTime = mtaskp->predictStart() + mtaskp->cost();
|
||||
mtaskState[mtaskp].completionTime = bestEndTime;
|
||||
mtaskState[mtaskp].threadId = bestThreadId;
|
||||
|
||||
// Reference to thread in schedule we are assigning this MTask to.
|
||||
std::vector<const ExecMTask*>& bestThread = threads[bestThreadId];
|
||||
if (!bestThread.empty()) mtaskState[bestThread.back()].nextp = mtaskp;
|
||||
|
||||
// Add the MTask to the schedule
|
||||
bestThread.push_back(mtaskp);
|
||||
return bestEndTime;
|
||||
}
|
||||
bool contains(const ExecMTask* mtaskp) const { return mtasks.count(mtaskp); }
|
||||
};
|
||||
|
||||
uint32_t ThreadSchedule::s_nextId = 0;
|
||||
std::unordered_map<const ExecMTask*, ThreadSchedule::MTaskState> ThreadSchedule::mtaskState{};
|
||||
|
||||
//######################################################################
|
||||
// PackThreads
|
||||
|
||||
|
@ -260,7 +285,7 @@ class PackThreads final {
|
|||
return sandbaggedEndTime;
|
||||
}
|
||||
|
||||
bool isReady(ThreadSchedule& schedule, const ExecMTask* mtaskp) {
|
||||
static bool isReady(ThreadSchedule& schedule, const ExecMTask* mtaskp) {
|
||||
for (const V3GraphEdge& edgeIn : mtaskp->inEdges()) {
|
||||
const ExecMTask* const prevp = edgeIn.fromp()->as<const ExecMTask>();
|
||||
if (schedule.threadId(prevp) == ThreadSchedule::UNASSIGNED) {
|
||||
|
@ -272,20 +297,39 @@ class PackThreads final {
|
|||
}
|
||||
|
||||
// Pack an MTasks from given graph into m_nThreads threads, return the schedule.
|
||||
ThreadSchedule pack(V3Graph& mtaskGraph) {
|
||||
// The result
|
||||
ThreadSchedule schedule{m_nThreads};
|
||||
std::vector<ThreadSchedule> pack(V3Graph& mtaskGraph) {
|
||||
std::vector<ThreadSchedule> result;
|
||||
result.emplace_back(ThreadSchedule{m_nThreads});
|
||||
|
||||
// To support scheduling tasks that utilize more than one thread, we introduce a wide
|
||||
// task (ExecMTask with threads() > 1). Those tasks are scheduled on a separate thread
|
||||
// schedule to ensure that indexes for simulation-time thread pool workers are not shadowed
|
||||
// by another tasks.
|
||||
// For retaining control over thread schedules, we distinguish SchedulingModes:
|
||||
enum class SchedulingMode {
|
||||
SCHEDULING // Schedule normal tasks
|
||||
,
|
||||
WIDE_TASK_DISCOVERED // We found a wide task, if this is the only one available,
|
||||
// switch to WIDE_TASK_SCHEDULING
|
||||
,
|
||||
WIDE_TASK_SCHEDULING // Schedule wide tasks
|
||||
};
|
||||
SchedulingMode mode = SchedulingMode::SCHEDULING;
|
||||
|
||||
// Time each thread is occupied until
|
||||
std::vector<uint32_t> busyUntil(m_nThreads, 0);
|
||||
|
||||
// MTasks ready to be assigned next. All their dependencies are already assigned.
|
||||
std::set<ExecMTask*, MTaskCmp> readyMTasks;
|
||||
int maxThreadWorkers = 1;
|
||||
|
||||
// Build initial ready list
|
||||
for (V3GraphVertex& vtx : mtaskGraph.vertices()) {
|
||||
ExecMTask* const mtaskp = vtx.as<ExecMTask>();
|
||||
if (isReady(schedule, mtaskp)) readyMTasks.insert(mtaskp);
|
||||
if (isReady(result.back(), mtaskp)) readyMTasks.insert(mtaskp);
|
||||
// TODO right now we schedule tasks assuming they take the same number of threads for
|
||||
// simplification.
|
||||
maxThreadWorkers = std::max(maxThreadWorkers, mtaskp->threads());
|
||||
}
|
||||
|
||||
while (!readyMTasks.empty()) {
|
||||
|
@ -294,8 +338,16 @@ class PackThreads final {
|
|||
uint32_t bestTime = 0xffffffff;
|
||||
uint32_t bestThreadId = 0;
|
||||
ExecMTask* bestMtaskp = nullptr; // Todo: const ExecMTask*
|
||||
for (uint32_t threadId = 0; threadId < m_nThreads; ++threadId) {
|
||||
ThreadSchedule& schedule = result.back();
|
||||
for (uint32_t threadId = 0; threadId < schedule.threads.size(); ++threadId) {
|
||||
for (ExecMTask* const mtaskp : readyMTasks) {
|
||||
if (mode != SchedulingMode::WIDE_TASK_SCHEDULING && mtaskp->threads() > 1) {
|
||||
mode = SchedulingMode::WIDE_TASK_DISCOVERED;
|
||||
continue;
|
||||
}
|
||||
if (mode == SchedulingMode::WIDE_TASK_SCHEDULING && mtaskp->threads() <= 1)
|
||||
continue;
|
||||
|
||||
uint32_t timeBegin = busyUntil[threadId];
|
||||
if (timeBegin > bestTime) {
|
||||
UINFO(6, "th " << threadId << " busy until " << timeBegin
|
||||
|
@ -321,23 +373,44 @@ class PackThreads final {
|
|||
}
|
||||
}
|
||||
|
||||
if (!bestMtaskp && mode == SchedulingMode::WIDE_TASK_DISCOVERED) {
|
||||
mode = SchedulingMode::WIDE_TASK_SCHEDULING;
|
||||
const uint32_t size = m_nThreads / maxThreadWorkers;
|
||||
UASSERT(size, "Thread pool size should be bigger than 0");
|
||||
// If no tasks were added to the normal thread schedule, remove it.
|
||||
if (schedule.mtaskState.empty()) result.erase(result.begin());
|
||||
result.emplace_back(ThreadSchedule{size});
|
||||
continue;
|
||||
}
|
||||
|
||||
if (!bestMtaskp && mode == SchedulingMode::WIDE_TASK_SCHEDULING) {
|
||||
mode = SchedulingMode::SCHEDULING;
|
||||
if (!schedule.mtaskState.empty()) result.emplace_back(ThreadSchedule{m_nThreads});
|
||||
continue;
|
||||
}
|
||||
|
||||
UASSERT(bestMtaskp, "Should have found some task");
|
||||
UINFO(6, "Will schedule " << bestMtaskp->name() << " onto thread " << bestThreadId
|
||||
<< endl);
|
||||
|
||||
// Reference to thread in schedule we are assigning this MTask to.
|
||||
std::vector<const ExecMTask*>& bestThread = schedule.threads[bestThreadId];
|
||||
bestMtaskp->predictStart(bestTime);
|
||||
const uint32_t bestEndTime = schedule.scheduleOn(bestMtaskp, bestThreadId);
|
||||
|
||||
// Update algorithm state
|
||||
bestMtaskp->predictStart(bestTime); // Only for gantt reporting
|
||||
const uint32_t bestEndTime = bestTime + bestMtaskp->cost();
|
||||
schedule.mtaskState[bestMtaskp].completionTime = bestEndTime;
|
||||
schedule.mtaskState[bestMtaskp].threadId = bestThreadId;
|
||||
if (!bestThread.empty()) schedule.mtaskState[bestThread.back()].nextp = bestMtaskp;
|
||||
busyUntil[bestThreadId] = bestEndTime;
|
||||
|
||||
// Add the MTask to the schedule
|
||||
bestThread.push_back(bestMtaskp);
|
||||
// Populate busyUntil timestamps. For multi-worker tasks, set timestamps for
|
||||
// offsetted threads.
|
||||
if (mode != SchedulingMode::WIDE_TASK_SCHEDULING) {
|
||||
busyUntil[bestThreadId] = bestEndTime;
|
||||
} else {
|
||||
for (int i = 0; i < maxThreadWorkers; ++i) {
|
||||
const size_t threadId = bestThreadId + (i * schedule.threads.size());
|
||||
UASSERT(threadId < busyUntil.size(),
|
||||
"Incorrect busyUntil offset: threadId=" + cvtToStr(threadId)
|
||||
+ " bestThreadId=" + cvtToStr(bestThreadId) + " i=" + cvtToStr(i)
|
||||
+ " schedule-size=" + cvtToStr(schedule.threads.size())
|
||||
+ " maxThreadWorkers=" + cvtToStr(maxThreadWorkers));
|
||||
busyUntil[threadId] = bestEndTime;
|
||||
UINFO(6, "Will schedule " << bestMtaskp->name() << " onto thread " << threadId
|
||||
<< endl);
|
||||
}
|
||||
}
|
||||
|
||||
// Update the ready list
|
||||
const size_t erased = readyMTasks.erase(bestMtaskp);
|
||||
|
@ -357,9 +430,10 @@ class PackThreads final {
|
|||
}
|
||||
}
|
||||
|
||||
if (dumpGraphLevel() >= 4) schedule.dumpDotFilePrefixedAlways(mtaskGraph, "schedule");
|
||||
// All schedules are combined on a single graph
|
||||
if (dumpGraphLevel() >= 4) result.back().dumpDotFilePrefixedAlways(mtaskGraph, "schedule");
|
||||
|
||||
return schedule;
|
||||
return result;
|
||||
}
|
||||
|
||||
public:
|
||||
|
@ -383,49 +457,78 @@ public:
|
|||
ExecMTask* const t2 = new ExecMTask{&graph, makeBody()};
|
||||
t2->cost(100);
|
||||
t2->priority(100);
|
||||
t2->threads(2);
|
||||
ExecMTask* const t3 = new ExecMTask{&graph, makeBody()};
|
||||
t3->cost(100);
|
||||
t3->priority(100);
|
||||
t3->threads(3);
|
||||
ExecMTask* const t4 = new ExecMTask{&graph, makeBody()};
|
||||
t4->cost(100);
|
||||
t4->priority(100);
|
||||
t4->threads(3);
|
||||
|
||||
/*
|
||||
0
|
||||
/ \
|
||||
1 2
|
||||
/ \
|
||||
3 4
|
||||
*/
|
||||
new V3GraphEdge{&graph, t0, t1, 1};
|
||||
new V3GraphEdge{&graph, t0, t2, 1};
|
||||
new V3GraphEdge{&graph, t2, t3, 1};
|
||||
new V3GraphEdge{&graph, t2, t4, 1};
|
||||
|
||||
PackThreads packer{2, // Threads
|
||||
constexpr uint32_t threads = 6;
|
||||
PackThreads packer{threads,
|
||||
3, // Sandbag numerator
|
||||
10}; // Sandbag denom
|
||||
const ThreadSchedule& schedule = packer.pack(graph);
|
||||
|
||||
UASSERT_SELFTEST(size_t, schedule.threads.size(), 2);
|
||||
const std::vector<ThreadSchedule> scheduled = packer.pack(graph);
|
||||
UASSERT_SELFTEST(size_t, scheduled[0].threads.size(), threads);
|
||||
UASSERT_SELFTEST(size_t, scheduled[0].threads[0].size(), 2);
|
||||
for (size_t i = 1; i < scheduled[0].threads.size(); ++i)
|
||||
UASSERT_SELFTEST(size_t, scheduled[0].threads[i].size(), 0);
|
||||
|
||||
UASSERT_SELFTEST(size_t, schedule.threads[0].size(), 2);
|
||||
UASSERT_SELFTEST(size_t, schedule.threads[1].size(), 1);
|
||||
UASSERT_SELFTEST(const ExecMTask*, scheduled[0].threads[0][0], t0);
|
||||
UASSERT_SELFTEST(const ExecMTask*, scheduled[0].threads[0][1], t1);
|
||||
|
||||
UASSERT_SELFTEST(const ExecMTask*, schedule.threads[0][0], t0);
|
||||
UASSERT_SELFTEST(const ExecMTask*, schedule.threads[0][1], t1);
|
||||
UASSERT_SELFTEST(const ExecMTask*, schedule.threads[1][0], t2);
|
||||
UASSERT_SELFTEST(size_t, scheduled[1].threads.size(), threads / 3);
|
||||
UASSERT_SELFTEST(const ExecMTask*, scheduled[1].threads[1][0], t2);
|
||||
UASSERT_SELFTEST(const ExecMTask*, scheduled[1].threads[1][1], t3);
|
||||
UASSERT_SELFTEST(const ExecMTask*, scheduled[1].threads[0][0], t4);
|
||||
|
||||
UASSERT_SELFTEST(size_t, schedule.mtaskState.size(), 3);
|
||||
UASSERT_SELFTEST(size_t, ThreadSchedule::mtaskState.size(), 5);
|
||||
|
||||
UASSERT_SELFTEST(uint32_t, schedule.threadId(t0), 0);
|
||||
UASSERT_SELFTEST(uint32_t, schedule.threadId(t1), 0);
|
||||
UASSERT_SELFTEST(uint32_t, schedule.threadId(t2), 1);
|
||||
UASSERT_SELFTEST(uint32_t, ThreadSchedule::threadId(t0), 0);
|
||||
UASSERT_SELFTEST(uint32_t, ThreadSchedule::threadId(t1), 0);
|
||||
UASSERT_SELFTEST(uint32_t, ThreadSchedule::threadId(t2), 1);
|
||||
UASSERT_SELFTEST(uint32_t, ThreadSchedule::threadId(t3), 1);
|
||||
UASSERT_SELFTEST(uint32_t, ThreadSchedule::threadId(t4), 0);
|
||||
|
||||
// On its native thread, we see the actual end time for t0:
|
||||
UASSERT_SELFTEST(uint32_t, packer.completionTime(schedule, t0, 0), 1000);
|
||||
UASSERT_SELFTEST(uint32_t, packer.completionTime(scheduled[0], t0, 0), 1000);
|
||||
// On the other thread, we see a sandbagged end time which does not
|
||||
// exceed the t1 end time:
|
||||
UASSERT_SELFTEST(uint32_t, packer.completionTime(schedule, t0, 1), 1099);
|
||||
UASSERT_SELFTEST(uint32_t, packer.completionTime(scheduled[0], t0, 1), 1099);
|
||||
|
||||
// Actual end time on native thread:
|
||||
UASSERT_SELFTEST(uint32_t, packer.completionTime(schedule, t1, 0), 1100);
|
||||
UASSERT_SELFTEST(uint32_t, packer.completionTime(scheduled[0], t1, 0), 1100);
|
||||
// Sandbagged end time seen on thread 1. Note it does not compound
|
||||
// with t0's sandbagged time; compounding caused trouble in
|
||||
// practice.
|
||||
UASSERT_SELFTEST(uint32_t, packer.completionTime(schedule, t1, 1), 1130);
|
||||
UASSERT_SELFTEST(uint32_t, packer.completionTime(schedule, t2, 0), 1229);
|
||||
UASSERT_SELFTEST(uint32_t, packer.completionTime(schedule, t2, 1), 1199);
|
||||
UASSERT_SELFTEST(uint32_t, packer.completionTime(scheduled[0], t1, 1), 1130);
|
||||
UASSERT_SELFTEST(uint32_t, packer.completionTime(scheduled[1], t2, 0), 1229);
|
||||
UASSERT_SELFTEST(uint32_t, packer.completionTime(scheduled[1], t2, 1), 1199);
|
||||
UASSERT_SELFTEST(uint32_t, packer.completionTime(scheduled[1], t3, 0), 1329);
|
||||
UASSERT_SELFTEST(uint32_t, packer.completionTime(scheduled[1], t3, 1), 1299);
|
||||
UASSERT_SELFTEST(uint32_t, packer.completionTime(scheduled[1], t4, 0), 1329);
|
||||
UASSERT_SELFTEST(uint32_t, packer.completionTime(scheduled[1], t4, 1), 1359);
|
||||
|
||||
for (AstNode* const nodep : mTaskBodyps) nodep->deleteTree();
|
||||
}
|
||||
|
||||
static const ThreadSchedule apply(V3Graph& mtaskGraph) {
|
||||
static std::vector<ThreadSchedule> apply(V3Graph& mtaskGraph) {
|
||||
return PackThreads{}.pack(mtaskGraph);
|
||||
}
|
||||
};
|
||||
|
@ -644,7 +747,7 @@ void addMTaskToFunction(const ThreadSchedule& schedule, const uint32_t threadId,
|
|||
// For any dependent mtask that's on another thread, signal one dependency completion.
|
||||
for (const V3GraphEdge& edge : mtaskp->outEdges()) {
|
||||
const ExecMTask* const nextp = edge.top()->as<ExecMTask>();
|
||||
if (schedule.threadId(nextp) != threadId) {
|
||||
if (schedule.threadId(nextp) != threadId && schedule.contains(nextp)) {
|
||||
addStrStmt("vlSelf->__Vm_mtaskstate_" + cvtToStr(nextp->id())
|
||||
+ ".signalUpstreamDone(even_cycle);\n");
|
||||
}
|
||||
|
@ -662,7 +765,8 @@ const std::vector<AstCFunc*> createThreadFunctions(const ThreadSchedule& schedul
|
|||
for (const std::vector<const ExecMTask*>& thread : schedule.threads) {
|
||||
if (thread.empty()) continue;
|
||||
const uint32_t threadId = schedule.threadId(thread.front());
|
||||
const string name{"__Vthread__" + tag + "__" + cvtToStr(threadId)};
|
||||
const string name{"__Vthread__" + tag + "__t" + cvtToStr(threadId) + "__s"
|
||||
+ cvtToStr(schedule.id())};
|
||||
AstCFunc* const funcp = new AstCFunc{fl, name, nullptr, "void"};
|
||||
modp->addStmtsp(funcp);
|
||||
funcps.push_back(funcp);
|
||||
|
@ -681,7 +785,8 @@ const std::vector<AstCFunc*> createThreadFunctions(const ThreadSchedule& schedul
|
|||
}
|
||||
|
||||
// Unblock the fake "final" mtask when this thread is finished
|
||||
funcp->addStmtsp(new AstCStmt{fl, "vlSelf->__Vm_mtaskstate_final__" + tag
|
||||
funcp->addStmtsp(new AstCStmt{fl, "vlSelf->__Vm_mtaskstate_final__"
|
||||
+ cvtToStr(schedule.id()) + tag
|
||||
+ ".signalUpstreamDone(even_cycle);\n"});
|
||||
}
|
||||
|
||||
|
@ -689,7 +794,8 @@ const std::vector<AstCFunc*> createThreadFunctions(const ThreadSchedule& schedul
|
|||
AstBasicDType* const mtaskStateDtypep
|
||||
= v3Global.rootp()->typeTablep()->findBasicDType(fl, VBasicDTypeKwd::MTASKSTATE);
|
||||
AstVar* const varp
|
||||
= new AstVar{fl, VVarType::MODULETEMP, "__Vm_mtaskstate_final__" + tag, mtaskStateDtypep};
|
||||
= new AstVar{fl, VVarType::MODULETEMP,
|
||||
"__Vm_mtaskstate_final__" + cvtToStr(schedule.id()) + tag, mtaskStateDtypep};
|
||||
varp->valuep(new AstConst(fl, funcps.size()));
|
||||
varp->protect(false); // Do not protect as we still have references in AstText
|
||||
modp->addStmtsp(varp);
|
||||
|
@ -697,8 +803,40 @@ const std::vector<AstCFunc*> createThreadFunctions(const ThreadSchedule& schedul
|
|||
return funcps;
|
||||
}
|
||||
|
||||
void addThreadStartWrapper(AstExecGraph* const execGraphp) {
|
||||
// FileLine used for constructing nodes below
|
||||
FileLine* const fl = v3Global.rootp()->fileline();
|
||||
const string& tag = execGraphp->name();
|
||||
|
||||
// Add thread function invocations to execGraph
|
||||
const auto addStrStmt = [=](const string& stmt) -> void { //
|
||||
execGraphp->addStmtsp(new AstCStmt{fl, stmt});
|
||||
};
|
||||
|
||||
if (v3Global.opt.profExec()) {
|
||||
addStrStmt("VL_EXEC_TRACE_ADD_RECORD(vlSymsp).execGraphBegin();\n");
|
||||
}
|
||||
|
||||
addStrStmt("vlSymsp->__Vm_even_cycle__" + tag + " = !vlSymsp->__Vm_even_cycle__" + tag
|
||||
+ ";\n");
|
||||
|
||||
if (!v3Global.opt.hierBlocks().empty()) addStrStmt("std::vector<size_t> indexes;\n");
|
||||
}
|
||||
|
||||
void addThreadEndWrapper(AstExecGraph* const execGraphp) {
|
||||
// Add thread function invocations to execGraph
|
||||
const auto addStrStmt = [=](const string& stmt) -> void { //
|
||||
FileLine* const flp = v3Global.rootp()->fileline();
|
||||
execGraphp->addStmtsp(new AstCStmt{flp, stmt});
|
||||
};
|
||||
|
||||
addStrStmt("Verilated::mtaskId(0);\n");
|
||||
if (v3Global.opt.profExec()) {
|
||||
addStrStmt("VL_EXEC_TRACE_ADD_RECORD(vlSymsp).execGraphEnd();\n");
|
||||
}
|
||||
}
|
||||
void addThreadStartToExecGraph(AstExecGraph* const execGraphp,
|
||||
const std::vector<AstCFunc*>& funcps) {
|
||||
const std::vector<AstCFunc*>& funcps, uint32_t scheduleId) {
|
||||
// FileLine used for constructing nodes below
|
||||
FileLine* const fl = v3Global.rootp()->fileline();
|
||||
const string& tag = execGraphp->name();
|
||||
|
@ -711,19 +849,22 @@ void addThreadStartToExecGraph(AstExecGraph* const execGraphp,
|
|||
execGraphp->addStmtsp(new AstText{fl, text, /* tracking: */ true});
|
||||
};
|
||||
|
||||
if (v3Global.opt.profExec()) {
|
||||
addStrStmt("VL_EXEC_TRACE_ADD_RECORD(vlSymsp).execGraphBegin();\n");
|
||||
}
|
||||
|
||||
addStrStmt("vlSymsp->__Vm_even_cycle__" + tag + " = !vlSymsp->__Vm_even_cycle__" + tag
|
||||
+ ";\n");
|
||||
|
||||
const uint32_t last = funcps.size() - 1;
|
||||
for (uint32_t i = 0; i <= last; ++i) {
|
||||
AstCFunc* const funcp = funcps.at(i);
|
||||
if (!v3Global.opt.hierBlocks().empty() && last > 0) {
|
||||
addStrStmt(
|
||||
"for (size_t i = 0; i < " + cvtToStr(last)
|
||||
+ "; ++i) indexes.push_back(vlSymsp->__Vm_threadPoolp->assignWorkerIndex());\n");
|
||||
}
|
||||
uint32_t i = 0;
|
||||
for (AstCFunc* const funcp : funcps) {
|
||||
if (i != last) {
|
||||
// The first N-1 will run on the thread pool.
|
||||
addTextStmt("vlSymsp->__Vm_threadPoolp->workerp(" + cvtToStr(i) + ")->addTask(");
|
||||
if (v3Global.opt.hierChild() || !v3Global.opt.hierBlocks().empty()) {
|
||||
addTextStmt("vlSymsp->__Vm_threadPoolp->workerp(indexes[" + cvtToStr(i)
|
||||
+ "])->addTask(");
|
||||
} else {
|
||||
addTextStmt("vlSymsp->__Vm_threadPoolp->workerp(" + cvtToStr(i) + ")->addTask(");
|
||||
}
|
||||
execGraphp->addStmtsp(new AstAddrOfCFunc{fl, funcp});
|
||||
addTextStmt(", vlSelf, vlSymsp->__Vm_even_cycle__" + tag + ");\n");
|
||||
} else {
|
||||
|
@ -732,15 +873,16 @@ void addThreadStartToExecGraph(AstExecGraph* const execGraphp,
|
|||
callp->dtypeSetVoid();
|
||||
callp->argTypes("vlSelf, vlSymsp->__Vm_even_cycle__" + tag);
|
||||
execGraphp->addStmtsp(callp->makeStmt());
|
||||
addStrStmt("Verilated::mtaskId(0);\n");
|
||||
}
|
||||
++i;
|
||||
}
|
||||
V3Stats::addStatSum("Optimizations, Thread schedule total tasks", i);
|
||||
|
||||
addStrStmt("vlSelf->__Vm_mtaskstate_final__" + tag
|
||||
addStrStmt("vlSelf->__Vm_mtaskstate_final__" + std::to_string(scheduleId) + tag
|
||||
+ ".waitUntilUpstreamDone(vlSymsp->__Vm_even_cycle__" + tag + ");\n");
|
||||
|
||||
if (v3Global.opt.profExec()) {
|
||||
addStrStmt("VL_EXEC_TRACE_ADD_RECORD(vlSymsp).execGraphEnd();\n");
|
||||
// Free all assigned worker indices in this section
|
||||
if (!v3Global.opt.hierBlocks().empty() && last > 0) {
|
||||
addStrStmt("vlSymsp->__Vm_threadPoolp->freeWorkerIndexes(indexes);\n");
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -762,15 +904,22 @@ void wrapMTaskBodies(AstExecGraph* const execGraphp) {
|
|||
funcp->addStmtsp(new AstCStmt{flp, stmt});
|
||||
};
|
||||
|
||||
if (v3Global.opt.profExec()) {
|
||||
if (v3Global.opt.hierChild() || !v3Global.opt.hierBlocks().empty()) {
|
||||
addStrStmt(
|
||||
"static const unsigned taskId = vlSymsp->__Vm_threadPoolp->assignTaskIndex();\n");
|
||||
} else {
|
||||
const string& id = std::to_string(mtaskp->id());
|
||||
addStrStmt("static constexpr unsigned taskId = " + id + ";\n");
|
||||
}
|
||||
|
||||
if (v3Global.opt.profExec() && mtaskp->threads() <= 1) {
|
||||
const string& predictStart = std::to_string(mtaskp->predictStart());
|
||||
addStrStmt("VL_EXEC_TRACE_ADD_RECORD(vlSymsp).mtaskBegin(" + id + ", " + predictStart
|
||||
addStrStmt("VL_EXEC_TRACE_ADD_RECORD(vlSymsp).mtaskBegin(taskId, " + predictStart
|
||||
+ ");\n");
|
||||
}
|
||||
|
||||
// Set mtask ID in the run-time system
|
||||
addStrStmt("Verilated::mtaskId(" + std::to_string(mtaskp->id()) + ");\n");
|
||||
addStrStmt("Verilated::mtaskId(taskId);\n");
|
||||
|
||||
// Run body
|
||||
funcp->addStmtsp(mtaskBodyp->stmtsp()->unlinkFrBackWithNext());
|
||||
|
@ -778,10 +927,9 @@ void wrapMTaskBodies(AstExecGraph* const execGraphp) {
|
|||
// Flush message queue
|
||||
addStrStmt("Verilated::endOfThreadMTask(vlSymsp->__Vm_evalMsgQp);\n");
|
||||
|
||||
if (v3Global.opt.profExec()) {
|
||||
const string& id = std::to_string(mtaskp->id());
|
||||
if (v3Global.opt.profExec() && mtaskp->threads() <= 1) {
|
||||
const string& predictConst = std::to_string(mtaskp->cost());
|
||||
addStrStmt("VL_EXEC_TRACE_ADD_RECORD(vlSymsp).mtaskEnd(" + id + ", " + predictConst
|
||||
addStrStmt("VL_EXEC_TRACE_ADD_RECORD(vlSymsp).mtaskEnd(taskId, " + predictConst
|
||||
+ ");\n");
|
||||
}
|
||||
|
||||
|
@ -803,7 +951,7 @@ void implementExecGraph(AstExecGraph* const execGraphp, const ThreadSchedule& sc
|
|||
UASSERT(!funcps.empty(), "Non-empty ExecGraph yields no threads?");
|
||||
|
||||
// Start the thread functions at the point this AstExecGraph is located in the tree.
|
||||
addThreadStartToExecGraph(execGraphp, funcps);
|
||||
addThreadStartToExecGraph(execGraphp, funcps, schedule.id());
|
||||
}
|
||||
|
||||
void implement(AstNetlist* netlistp) {
|
||||
|
@ -817,15 +965,25 @@ void implement(AstNetlist* netlistp) {
|
|||
fillinCosts(execGraphp->depGraphp());
|
||||
finalizeCosts(execGraphp->depGraphp());
|
||||
|
||||
if (dumpGraphLevel() >= 4) execGraphp->depGraphp()->dumpDotFilePrefixedAlways("pack");
|
||||
|
||||
addThreadStartWrapper(execGraphp);
|
||||
|
||||
// Schedule the mtasks: statically associate each mtask with a thread,
|
||||
// and determine the order in which each thread will run its mtasks.
|
||||
const ThreadSchedule& schedule = PackThreads::apply(*execGraphp->depGraphp());
|
||||
const std::vector<ThreadSchedule> packed = PackThreads::apply(*execGraphp->depGraphp());
|
||||
V3Stats::addStatSum("Optimizations, Thread schedule count",
|
||||
static_cast<double>(packed.size()));
|
||||
|
||||
// Wrap each MTask body into a CFunc for better profiling/debugging
|
||||
wrapMTaskBodies(execGraphp);
|
||||
|
||||
// Replace the graph body with its multi-threaded implementation.
|
||||
implementExecGraph(execGraphp, schedule);
|
||||
for (const ThreadSchedule& schedule : packed) {
|
||||
// Replace the graph body with its multi-threaded implementation.
|
||||
implementExecGraph(execGraphp, schedule);
|
||||
}
|
||||
|
||||
addThreadEndWrapper(execGraphp);
|
||||
});
|
||||
}
|
||||
|
||||
|
|
|
@ -43,6 +43,7 @@ private:
|
|||
// Predicted runtime of this mtask, in the same abstract time units as priority().
|
||||
uint32_t m_cost = 0;
|
||||
uint64_t m_predictStart = 0; // Predicted start time of task
|
||||
int m_threads = 1; // Threads used by this mtask
|
||||
VL_UNCOPYABLE(ExecMTask);
|
||||
|
||||
public:
|
||||
|
@ -57,6 +58,8 @@ public:
|
|||
void predictStart(uint64_t time) { m_predictStart = time; }
|
||||
string name() const override VL_MT_STABLE { return "mt"s + std::to_string(id()); }
|
||||
string hashName() const { return m_hashName; }
|
||||
void threads(int threads) { m_threads = threads; }
|
||||
int threads() const { return m_threads; }
|
||||
void dump(std::ostream& str) const;
|
||||
|
||||
static uint32_t numUsedIds() VL_MT_SAFE { return s_nextId; }
|
||||
|
|
|
@ -88,6 +88,7 @@
|
|||
|
||||
#include "V3HierBlock.h"
|
||||
|
||||
#include "V3Config.h"
|
||||
#include "V3EmitV.h"
|
||||
#include "V3File.h"
|
||||
#include "V3Os.h"
|
||||
|
@ -188,6 +189,23 @@ V3StringList V3HierBlock::commandArgs(bool forCMake) const {
|
|||
if (!params().gTypeParams().empty())
|
||||
opts.push_back(" --hierarchical-params-file " + typeParametersFilename());
|
||||
|
||||
const int blockThreads = V3Config::getHierWorkers(m_modp->origName());
|
||||
if (blockThreads > 1) {
|
||||
if (hasParent()) {
|
||||
V3Config::getHierWorkersFileLine()->v3warn(
|
||||
E_UNSUPPORTED, "Specifying workers for nested hierarchical blocks");
|
||||
} else {
|
||||
if (v3Global.opt.threads() < blockThreads) {
|
||||
m_modp->v3error("Hierarchical blocks cannot be scheduled on more threads than in "
|
||||
"thread pool, threads = "
|
||||
<< v3Global.opt.threads()
|
||||
<< " hierarchical block threads = " << blockThreads);
|
||||
}
|
||||
|
||||
opts.push_back(" --threads " + std::to_string(blockThreads));
|
||||
}
|
||||
}
|
||||
|
||||
return opts;
|
||||
}
|
||||
|
||||
|
|
|
@ -92,6 +92,7 @@ public:
|
|||
~V3HierBlock() VL_MT_DISABLED;
|
||||
|
||||
void addParent(V3HierBlock* parentp) { m_parents.insert(parentp); }
|
||||
bool hasParent() const { return !m_parents.empty(); }
|
||||
void addChild(V3HierBlock* childp) { m_children.insert(childp); }
|
||||
bool hasChild() const { return !m_children.empty(); }
|
||||
const HierBlockSet& parents() const { return m_parents; }
|
||||
|
|
|
@ -1737,6 +1737,34 @@ private:
|
|||
VL_UNCOPYABLE(DpiImportCallVisitor);
|
||||
};
|
||||
|
||||
//######################################################################
|
||||
// DpiThreadsVisitor
|
||||
|
||||
// Get number of threads occupied by this mtask
|
||||
class DpiThreadsVisitor final : public VNVisitorConst {
|
||||
int m_threads = 1; // Max number of threads used by this mtask
|
||||
|
||||
// METHODS
|
||||
void visit(AstCFunc* nodep) override {
|
||||
m_threads = std::max(m_threads, V3Config::getHierWorkers(nodep->cname()));
|
||||
iterateChildrenConst(nodep);
|
||||
}
|
||||
void visit(AstNodeCCall* nodep) override {
|
||||
iterateChildrenConst(nodep);
|
||||
iterateConst(nodep->funcp());
|
||||
}
|
||||
void visit(AstNode* nodep) override { iterateChildrenConst(nodep); }
|
||||
|
||||
public:
|
||||
// CONSTRUCTORS
|
||||
explicit DpiThreadsVisitor(AstMTaskBody* nodep) { iterateConst(nodep); }
|
||||
int threads() const { return m_threads; }
|
||||
~DpiThreadsVisitor() override = default;
|
||||
|
||||
private:
|
||||
VL_UNCOPYABLE(DpiThreadsVisitor);
|
||||
};
|
||||
|
||||
//######################################################################
|
||||
// FixDataHazards
|
||||
|
||||
|
@ -2451,6 +2479,8 @@ AstExecGraph* V3Order::createParallel(OrderGraph& orderGraph, const std::string&
|
|||
|
||||
// Create the ExecMTask
|
||||
ExecMTask* const execMTaskp = new ExecMTask{depGraphp, bodyp};
|
||||
if (!v3Global.opt.hierBlocks().empty())
|
||||
execMTaskp->threads(DpiThreadsVisitor{bodyp}.threads());
|
||||
const bool newEntry = logicMTaskToExecMTask.emplace(mTaskp, execMTaskp).second;
|
||||
UASSERT_OBJ(newEntry, mTaskp, "LogicMTasks should be processed in dependencyorder");
|
||||
UINFO(3, "Final '" << tag << "' LogicMTask " << mTaskp->id() << " maps to ExecMTask"
|
||||
|
|
|
@ -18,6 +18,7 @@
|
|||
|
||||
#include "V3ProtectLib.h"
|
||||
|
||||
#include "V3Config.h"
|
||||
#include "V3Hasher.h"
|
||||
#include "V3InstrCount.h"
|
||||
#include "V3String.h"
|
||||
|
@ -119,8 +120,17 @@ class ProtectVisitor final : public VNVisitor {
|
|||
|
||||
// Mark remaining NDA protectlib wrapper DPIs as non-hazardous by deliberately forwarding
|
||||
// them with non-zero cost.
|
||||
// Also, specify hierarchical workers for those tasks for scheduling.
|
||||
txtp->addText(fl, "profile_data -hier-dpi \"" + m_libName
|
||||
+ "_protectlib_combo_ignore\" -cost 64'd1\n");
|
||||
|
||||
txtp->addText(fl, "hier_workers -hier-dpi \"" + m_libName
|
||||
+ "_protectlib_combo_update\" -workers 16'd"
|
||||
+ std::to_string(V3Config::getHierWorkers(m_libName)) + "\n");
|
||||
txtp->addText(fl, "hier_workers -hier-dpi \"" + m_libName
|
||||
+ "_protectlib_seq_update\" -workers 16'd"
|
||||
+ std::to_string(V3Config::getHierWorkers(m_libName)) + "\n");
|
||||
// No workers for combo_ignore
|
||||
txtp->addText(fl, "`verilog\n");
|
||||
txtp->addText(fl, "`endif\n");
|
||||
}
|
||||
|
|
|
@ -115,6 +115,7 @@ vnum {vnum1}|{vnum2}|{vnum3}|{vnum4}|{vnum5}
|
|||
"full_case" { FL; return yVLT_FULL_CASE; }
|
||||
"hier_block" { FL; return yVLT_HIER_BLOCK; }
|
||||
"hier_params" { FL; return yVLT_HIER_PARAMS; }
|
||||
"hier_workers" { FL; return yVLT_HIER_WORKERS; }
|
||||
"inline" { FL; return yVLT_INLINE; }
|
||||
"isolate_assignments" { FL; return yVLT_ISOLATE_ASSIGNMENTS; }
|
||||
"lint_off" { FL; return yVLT_LINT_OFF; }
|
||||
|
@ -152,6 +153,7 @@ vnum {vnum1}|{vnum2}|{vnum3}|{vnum4}|{vnum5}
|
|||
-?"-scope" { FL; return yVLT_D_SCOPE; }
|
||||
-?"-task" { FL; return yVLT_D_TASK; }
|
||||
-?"-var" { FL; return yVLT_D_VAR; }
|
||||
-?"-workers" { FL; return yVLT_D_WORKERS; }
|
||||
|
||||
/* Reachable by attr_event_control */
|
||||
"edge" { FL; return yEDGE; }
|
||||
|
|
|
@ -466,6 +466,7 @@ BISONPRE_VERSION(3.7,%define api.header.include {"V3ParseBison.h"})
|
|||
%token<fl> yVLT_FULL_CASE "full_case"
|
||||
%token<fl> yVLT_HIER_BLOCK "hier_block"
|
||||
%token<fl> yVLT_HIER_PARAMS "hier_params"
|
||||
%token<fl> yVLT_HIER_WORKERS "hier_workers"
|
||||
%token<fl> yVLT_INLINE "inline"
|
||||
%token<fl> yVLT_ISOLATE_ASSIGNMENTS "isolate_assignments"
|
||||
%token<fl> yVLT_LINT_OFF "lint_off"
|
||||
|
@ -503,6 +504,7 @@ BISONPRE_VERSION(3.7,%define api.header.include {"V3ParseBison.h"})
|
|||
%token<fl> yVLT_D_SCOPE "--scope"
|
||||
%token<fl> yVLT_D_TASK "--task"
|
||||
%token<fl> yVLT_D_VAR "--var"
|
||||
%token<fl> yVLT_D_WORKERS "--workers"
|
||||
|
||||
%token<strp> yaD_PLI "${pli-system}"
|
||||
|
||||
|
@ -7660,6 +7662,10 @@ vltItem:
|
|||
{ V3Config::addModulePragma(*$2, VPragmaType::HIER_BLOCK); }
|
||||
| yVLT_HIER_PARAMS vltDModuleE
|
||||
{ V3Config::addModulePragma(*$2, VPragmaType::HIER_PARAMS); }
|
||||
| yVLT_HIER_WORKERS vltDModuleE vltDWorkers
|
||||
{ V3Config::addHierWorkers($<fl>1, *$2, $3->toSInt()); }
|
||||
| yVLT_HIER_WORKERS vltDHierDpi vltDWorkers
|
||||
{ V3Config::addHierWorkers($<fl>1, *$2, $3->toSInt()); }
|
||||
| yVLT_PARALLEL_CASE vltDFile
|
||||
{ V3Config::addCaseParallel(*$2, 0); }
|
||||
| yVLT_PARALLEL_CASE vltDFile yVLT_D_LINES yaINTNUM
|
||||
|
@ -7749,6 +7755,10 @@ vltDFTaskE<strp>:
|
|||
| yVLT_D_TASK str { $$ = $2; }
|
||||
;
|
||||
|
||||
vltDWorkers<nump>: // --workers <arg>
|
||||
yVLT_D_WORKERS yaINTNUM { $$ = $2; }
|
||||
;
|
||||
|
||||
vltInlineFront<cbool>:
|
||||
yVLT_INLINE { $$ = true; }
|
||||
| yVLT_NO_INLINE { $$ = false; }
|
||||
|
|
|
@ -19,7 +19,7 @@ test.compile(v_flags2=["--dumpi-graph 6"], threads=2)
|
|||
for dotname in [
|
||||
"linkcells", "task_call", "gate_graph", "gate_final", "acyc_simp", "orderg_pre",
|
||||
"orderg_acyc", "orderg_order", "orderg_domain", "ordermv_initial", "ordermv_hazards",
|
||||
"ordermv_contraction", "ordermv_transitive1", "orderg_done", "schedule"
|
||||
"ordermv_contraction", "ordermv_transitive1", "orderg_done", "pack", "schedule"
|
||||
]:
|
||||
# Some files with identical prefix are generated multiple times during
|
||||
# Verilation. Ensure that at least one of each dotname-prefixed file is generated.
|
||||
|
|
|
@ -13,17 +13,30 @@ test.scenarios('vlt_all')
|
|||
test.init_benchmarksim()
|
||||
test.cycles = (int(test.benchmark) if test.benchmark else 1000000)
|
||||
test.sim_time = test.cycles * 10 + 1000
|
||||
THREADS = int(os.environ["SIM_THREADS"]) if "SIM_THREADS" in os.environ else 2
|
||||
|
||||
test.compile(benchmarksim=1,
|
||||
v_flags2=[
|
||||
"+define+SIM_CYCLES=" + str(test.cycles), "--prof-exec", "--hierarchical",
|
||||
"--stats"
|
||||
],
|
||||
threads=(THREADS if test.vltmt else 1))
|
||||
THREADS = int(os.environ["THREADS"]) if "THREADS" in os.environ else 4
|
||||
HIER_BLOCK_THREADS = int(
|
||||
os.environ["HIER_BLOCK_THREADS"]) if "HIER_BLOCK_THREADS" in os.environ else 2
|
||||
|
||||
config_file = test.t_dir + "/" + test.name + ".vlt"
|
||||
|
||||
test.compile(
|
||||
benchmarksim=1,
|
||||
v_flags2=[
|
||||
config_file, "+define+SIM_CYCLES=" + str(test.cycles), "--prof-exec", "--hierarchical",
|
||||
"--stats", "-Wno-UNOPTFLAT",
|
||||
(f"-DWORKERS={HIER_BLOCK_THREADS}" if test.vltmt and HIER_BLOCK_THREADS > 1 else "")
|
||||
],
|
||||
threads=(THREADS if test.vltmt else 1))
|
||||
|
||||
test.file_grep(test.obj_dir + "/V" + test.name + "__hier.dir/V" + test.name + "__stats.txt",
|
||||
r'Optimizations, Hierarchical DPI wrappers with costs\s+(\d+)', 3)
|
||||
r'Optimizations, Hierarchical DPI wrappers with costs\s+(\d+)', 6)
|
||||
|
||||
if test.vltmt:
|
||||
test.file_grep(test.obj_dir + "/V" + test.name + "__hier.dir/V" + test.name + "__stats.txt",
|
||||
r'Optimizations, Thread schedule count\s+(\d+)', 4)
|
||||
test.file_grep(test.obj_dir + "/V" + test.name + "__hier.dir/V" + test.name + "__stats.txt",
|
||||
r'Optimizations, Thread schedule total tasks\s+(\d+)', 10)
|
||||
|
||||
test.execute(all_run_flags=[
|
||||
"+verilator+prof+exec+start+2",
|
||||
|
|
|
@ -6,12 +6,8 @@
|
|||
|
||||
// based on t_gate_ormux
|
||||
|
||||
`ifndef HIER_CORES
|
||||
`define HIER_CORES 3
|
||||
`endif
|
||||
|
||||
`ifndef MAIN_CORES
|
||||
`define MAIN_CORES 1
|
||||
`ifndef CORES
|
||||
`define CORES 4
|
||||
`endif
|
||||
|
||||
module t (/*AUTOARG*/
|
||||
|
@ -21,37 +17,11 @@ module t (/*AUTOARG*/
|
|||
input clk;
|
||||
|
||||
generate
|
||||
for (genvar i = 0; i < `MAIN_CORES; ++i) NonHierCore mainCore(clk);
|
||||
endgenerate
|
||||
|
||||
generate
|
||||
for (genvar i = 0; i < `HIER_CORES; ++i) Core hierCore(clk);
|
||||
for (genvar i = 0; i < `CORES; ++i) Core core(clk);
|
||||
endgenerate
|
||||
endmodule
|
||||
|
||||
module Core(input clk); /* verilator hier_block */
|
||||
reg [63:0] crc;
|
||||
logic [31:0] rdata;
|
||||
logic [31:0] rdata2;
|
||||
wire [31:0] wdata = crc[31:0];
|
||||
wire [15:0] sel = {11'h0, crc[36:32]};
|
||||
wire we = crc[48];
|
||||
|
||||
Test test (
|
||||
// Outputs
|
||||
.rdata (rdata[31:0]),
|
||||
.rdata2 (rdata2[31:0]),
|
||||
// Inputs
|
||||
.clk (clk),
|
||||
.we (we),
|
||||
.sel (sel[15:0]),
|
||||
.wdata (wdata[31:0]));
|
||||
wire [63:0] result = {rdata2, rdata};
|
||||
|
||||
Check check(.clk(clk), .crc(crc), .result(result), .rdata(rdata), .rdata2(rdata2));
|
||||
endmodule
|
||||
|
||||
module NonHierCore(input clk);
|
||||
module Core(input clk);
|
||||
reg [63:0] crc;
|
||||
logic [31:0] rdata;
|
||||
logic [31:0] rdata2;
|
||||
|
@ -79,7 +49,7 @@ module Check(
|
|||
input wire [63:0] result,
|
||||
input logic [31:0] rdata,
|
||||
input logic [31:0] rdata2
|
||||
);
|
||||
); /*verilator hier_block*/
|
||||
integer cyc = 0;
|
||||
reg [63:0] sum;
|
||||
|
||||
|
@ -118,7 +88,7 @@ module Test(/*AUTOARG*/
|
|||
rdata, rdata2,
|
||||
// Inputs
|
||||
clk, we, sel, wdata
|
||||
);
|
||||
); /*verilator hier_block*/
|
||||
input clk;
|
||||
input we;
|
||||
input [15:0] sel;
|
||||
|
|
|
@ -0,0 +1,11 @@
|
|||
// DESCRIPTION: Verilator: Verilog Test module
|
||||
//
|
||||
// This file ONLY is placed into the Public Domain, for any use,
|
||||
// without warranty, 2025 by Antmicro.
|
||||
// SPDX-License-Identifier: CC0-1.0
|
||||
|
||||
`verilator_config
|
||||
`ifdef WORKERS
|
||||
hier_workers -module "Test" -workers `WORKERS
|
||||
hier_workers -module "Check" -workers `WORKERS
|
||||
`endif
|
|
@ -0,0 +1,9 @@
|
|||
%Error: t/t_hier_block_threads_bad.v:23:8: Hierarchical blocks cannot be scheduled on more threads than in thread pool, threads = 4 hierarchical block threads = 8
|
||||
: ... note: In instance 't.genblk1[1].hierCore'
|
||||
23 | module Core(input clk); /*verilator hier_block*/
|
||||
| ^~~~
|
||||
%Error-UNSUPPORTED: t/t_hier_block_threads_bad.vlt:8:1: Specifying workers for nested hierarchical blocks
|
||||
8 | hier_workers -module "Core" -workers 8
|
||||
| ^~~~~~~~~~~~
|
||||
... For error description see https://verilator.org/warn/UNSUPPORTED?v=latest
|
||||
%Error: Exiting due to
|
|
@ -0,0 +1,19 @@
|
|||
#!/usr/bin/env python3
|
||||
# DESCRIPTION: Verilator: Verilog Test driver/expect definition
|
||||
#
|
||||
# Copyright 2025 by Wilson Snyder. This program is free software; you
|
||||
# can redistribute it and/or modify it under the terms of either the GNU
|
||||
# Lesser General Public License Version 3 or the Perl Artistic License
|
||||
# Version 2.0.
|
||||
# SPDX-License-Identifier: LGPL-3.0-only OR Artistic-2.0
|
||||
|
||||
import vltest_bootstrap
|
||||
|
||||
test.scenarios('vltmt')
|
||||
|
||||
test.lint(fails=True,
|
||||
verilator_flags2=['t/t_hier_block_threads_bad.vlt', '-DWORKERS=8', '--hierarchical'],
|
||||
expect_filename=test.golden_filename,
|
||||
threads=4)
|
||||
|
||||
test.passes()
|
|
@ -0,0 +1,32 @@
|
|||
// DESCRIPTION: Verilator: Verilog Test module
|
||||
//
|
||||
// This file ONLY is placed under the Creative Commons Public Domain, for
|
||||
// any use, without warranty, 2025 by Wilson Snyder.
|
||||
// SPDX-License-Identifier: CC0-1.0
|
||||
|
||||
module t (/*AUTOARG*/
|
||||
// Inputs
|
||||
clk
|
||||
);
|
||||
input clk;
|
||||
|
||||
generate
|
||||
for (genvar i = 0; i < 2; ++i) Core hierCore(clk);
|
||||
endgenerate
|
||||
|
||||
always @(negedge clk) begin
|
||||
$write("*-* All Finished *-*\n");
|
||||
$finish;
|
||||
end
|
||||
endmodule
|
||||
|
||||
module Core(input clk); /* verilator hier_block */
|
||||
generate
|
||||
for (genvar i = 0; i < 2; ++i) SubCore sub(clk);
|
||||
endgenerate
|
||||
always @(posedge clk) $display("%m");
|
||||
endmodule
|
||||
|
||||
module SubCore(input clk); /* verilator hier_block */
|
||||
always @(posedge clk) $display("%m");
|
||||
endmodule
|
|
@ -0,0 +1,9 @@
|
|||
// DESCRIPTION: Verilator: Verilog Test module
|
||||
//
|
||||
// This file ONLY is placed into the Public Domain, for any use,
|
||||
// without warranty, 2025 by Antmicro.
|
||||
// SPDX-License-Identifier: CC0-1.0
|
||||
|
||||
`verilator_config
|
||||
hier_workers -module "Core" -workers `WORKERS
|
||||
hier_workers -module "SubCore" -workers `WORKERS
|
Loading…
Reference in New Issue