Move thread pool and execution profiler into the context. (#3477)

Fixes #3454
This commit is contained in:
Geza Lore 2022-07-12 11:41:15 +01:00 committed by GitHub
parent 8377514127
commit f4038e3674
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
23 changed files with 470 additions and 184 deletions

View File

@ -66,6 +66,10 @@
#if defined(_WIN32) || defined(__MINGW32__)
# include <direct.h> // mkdir
#endif
#ifdef VL_THREADED
# include "verilated_threads.h"
#endif
// clang-format on
// Max characters in static char string for VL_VALUE_STRING
@ -2428,6 +2432,33 @@ const char* VerilatedContext::timeprecisionString() const VL_MT_SAFE {
return vl_time_str(timeprecision());
}
void VerilatedContext::threads(unsigned n) {
if (n == 0) VL_FATAL_MT(__FILE__, __LINE__, "", "%Error: Simulation threads must be >= 1");
if (m_threadPool) {
VL_FATAL_MT(
__FILE__, __LINE__, "",
"%Error: Cannot set simulation threads after the thread pool has been created.");
}
#if VL_THREADED
if (m_threads == n) return; // To avoid unnecessary warnings
m_threads = n;
const unsigned hardwareThreadsAvailable = std::thread::hardware_concurrency();
if (m_threads > hardwareThreadsAvailable) {
VL_PRINTF_MT("%%Warning: System has %u hardware threads but simulation thread count set "
"to %u. This will likely cause significant slowdown.\n",
hardwareThreadsAvailable, m_threads);
}
#else
if (n > 1) {
VL_PRINTF_MT("%%Warning: Verilator run-time library built without VL_THREADS. Ignoring "
"call to 'VerilatedContext::threads' with argument %u.\n",
n);
}
#endif
}
void VerilatedContext::commandArgs(int argc, const char** argv) VL_MT_SAFE_EXCLUDES(m_argMutex) {
const VerilatedLockGuard lock{m_argMutex};
m_args.m_argVec.clear(); // Empty first, then add
@ -2458,6 +2489,33 @@ void VerilatedContext::internalsDump() const VL_MT_SAFE {
VerilatedImp::userDump();
}
void VerilatedContext::addModel(VerilatedModel* modelp) {
threadPoolp(); // Ensure thread pool is created, so m_threads cannot change any more
if (modelp->threads() > m_threads) {
std::ostringstream msg;
msg << "VerilatedContext has " << m_threads << " threads but model '"
<< modelp->modelName() << "' (instantiated as '" << modelp->hierName()
<< "') was Verilated with --threads " << modelp->threads() << ".\n";
const std::string str = msg.str();
VL_FATAL_MT(__FILE__, __LINE__, modelp->hierName(), str.c_str());
}
}
VerilatedVirtualBase* VerilatedContext::threadPoolp() {
if (m_threads == 1) return nullptr;
#if VL_THREADED
if (!m_threadPool) m_threadPool.reset(new VlThreadPool{this, m_threads - 1});
#endif
return m_threadPool.get();
}
VerilatedVirtualBase*
VerilatedContext::enableExecutionProfiler(VerilatedVirtualBase* (*construct)(VerilatedContext&)) {
if (!m_executionProfiler) m_executionProfiler.reset(construct(*this));
return m_executionProfiler.get();
}
//======================================================================
// VerilatedContextImp:: Methods - command line

View File

@ -252,6 +252,28 @@ public:
#endif
};
//=========================================================================
/// Base class of a Verilator generated (Verilated) model.
///
/// VerilatedModel is a base class of the user facing primary class generated
/// by Verilator.
class VerilatedModel VL_NOT_FINAL {
VL_UNCOPYABLE(VerilatedModel);
protected:
explicit VerilatedModel() = default;
virtual ~VerilatedModel() = default;
public:
/// Returns the hierarchical name of this module instance.
virtual const char* hierName() = 0;
/// Returns the name of this model (the name of the generated model class).
virtual const char* modelName() = 0;
/// Returns the thread level parallelism, this model was Verilated with. Always 1 or higher.
virtual unsigned threads() = 0;
};
//=========================================================================
/// Base class for all Verilated module classes.
@ -266,10 +288,6 @@ public:
const char* name() const { return m_namep; } ///< Return name of module
};
/// Declare a module, ala SC_MODULE
#define VL_MODULE(modname) class modname VL_NOT_FINAL : public VerilatedModule
// Not class final in VL_MODULE, as users might be abstracting our models (--hierarchical)
//=========================================================================
// Functions overridable by user defines
// (Internals however must use VL_PRINTF_MT, which calls these.)
@ -362,6 +380,16 @@ protected:
// Implementation details
const std::unique_ptr<VerilatedContextImpData> m_impdatap;
// Number of threads to use for simulation (size of m_threadPool + 1 for main thread)
#ifdef VL_THREADED
unsigned m_threads = std::thread::hardware_concurrency();
#else
const unsigned m_threads = 1;
#endif
// The thread pool shared by all models added to this context
std::unique_ptr<VerilatedVirtualBase> m_threadPool;
// The execution profiler shared by all models added to this context
std::unique_ptr<VerilatedVirtualBase> m_executionProfiler;
// Coverage access
std::unique_ptr<VerilatedVirtualBase> m_coveragep; // Pointer for coveragep()
@ -495,6 +523,12 @@ public:
/// Get time precision as IEEE-standard text
const char* timeprecisionString() const VL_MT_SAFE;
/// Get number of threads used for simulation (including the main thread)
unsigned threads() const { return m_threads; }
/// Set number of threads used for simulation (including the main thread)
/// Can only be called before the thread pool is created (before first model is added).
void threads(unsigned n);
/// Allow traces to at some point be enabled (disables some optimizations)
void traceEverOn(bool flag) VL_MT_SAFE {
if (flag) calcUnusedSigs(true);
@ -517,6 +551,12 @@ public: // But for internal use only
return reinterpret_cast<const VerilatedContextImp*>(this);
}
void addModel(VerilatedModel*);
VerilatedVirtualBase* threadPoolp();
VerilatedVirtualBase*
enableExecutionProfiler(VerilatedVirtualBase* (*construct)(VerilatedContext&));
// Internal: $dumpfile
void dumpfile(const std::string& flag) VL_MT_SAFE_EXCLUDES(m_timeDumpMutex);
std::string dumpfile() const VL_MT_SAFE_EXCLUDES(m_timeDumpMutex);

View File

@ -66,41 +66,66 @@ template <size_t N> static size_t roundUptoMultipleOf(size_t value) {
return (value + mask) & ~mask;
}
VlExecutionProfiler::VlExecutionProfiler() {
VlExecutionProfiler::VlExecutionProfiler(VerilatedContext& context)
: m_context{context} {
// Setup profiling on main thread
setupThread(0);
}
void VlExecutionProfiler::configure(const VerilatedContext& context) {
void VlExecutionProfiler::configure() {
if (VL_UNLIKELY(m_enabled)) {
--m_windowCount;
if (VL_UNLIKELY(m_windowCount == context.profExecWindow())) {
if (VL_UNLIKELY(m_windowCount == m_context.profExecWindow())) {
VL_DEBUG_IF(VL_DBG_MSGF("+ profile start collection\n"););
clear(); // Clear the profile after the cache warm-up cycles.
m_tickBegin = VL_CPU_TICK();
} else if (VL_UNLIKELY(m_windowCount == 0)) {
const uint64_t tickEnd = VL_CPU_TICK();
VL_DEBUG_IF(VL_DBG_MSGF("+ profile end\n"););
const std::string& fileName = context.profExecFilename();
const std::string& fileName = m_context.profExecFilename();
dump(fileName.c_str(), tickEnd);
m_enabled = false;
}
return;
}
const uint64_t startReq = context.profExecStart() + 1; // + 1, so we can start at time 0
const uint64_t startReq = m_context.profExecStart() + 1; // + 1, so we can start at time 0
if (VL_UNLIKELY(m_lastStartReq < startReq && VL_TIME_Q() >= context.profExecStart())) {
if (VL_UNLIKELY(m_lastStartReq < startReq && VL_TIME_Q() >= m_context.profExecStart())) {
VL_DEBUG_IF(VL_DBG_MSGF("+ profile start warmup\n"););
VL_DEBUG_IF(assert(m_windowCount == 0););
m_enabled = true;
m_windowCount = context.profExecWindow() * 2;
m_windowCount = m_context.profExecWindow() * 2;
m_lastStartReq = startReq;
}
}
void VlExecutionProfiler::startWorkerSetup(VlExecutionProfiler* profilep, uint32_t threadId) {
profilep->setupThread(threadId);
VerilatedVirtualBase* VlExecutionProfiler::construct(VerilatedContext& context) {
VlExecutionProfiler* const selfp = new VlExecutionProfiler{context};
#if VL_THREADED
if (VlThreadPool* const threadPoolp = static_cast<VlThreadPool*>(context.threadPoolp())) {
for (int i = 0; i < threadPoolp->numThreads(); ++i) {
// Data to pass to worker thread initialization
struct Data {
VlExecutionProfiler* const selfp;
const uint32_t threadId;
} data{selfp, static_cast<uint32_t>(i + 1)};
// Initialize worker thread
threadPoolp->workerp(i)->addTask(
[](void* userp, bool) {
Data* const datap = static_cast<Data*>(userp);
datap->selfp->setupThread(datap->threadId);
},
&data);
// Wait until initializationis complete
threadPoolp->workerp(i)->wait();
}
}
#endif
return selfp;
}
void VlExecutionProfiler::setupThread(uint32_t threadId) {

View File

@ -33,13 +33,14 @@
#include <vector>
class VlExecutionProfiler;
class VlThreadPool;
//=============================================================================
// Macros to simplify generated code
#define VL_EXEC_TRACE_ADD_RECORD(vlSymsp) \
if (VL_UNLIKELY((vlSymsp)->__Vm_executionProfiler.enabled())) \
(vlSymsp)->__Vm_executionProfiler.addRecord()
if (VL_UNLIKELY((vlSymsp)->__Vm_executionProfilerp->enabled())) \
(vlSymsp)->__Vm_executionProfilerp->addRecord()
//=============================================================================
// Return high-precision counter for profiling, or 0x0 if not available
@ -131,7 +132,7 @@ static_assert(std::is_trivially_destructible<VlExecutionRecord>::value,
//=============================================================================
// VlExecutionProfiler is for collecting profiling data about model execution
class VlExecutionProfiler final {
class VlExecutionProfiler final : public VerilatedVirtualBase {
// CONSTANTS
// In order to try to avoid dynamic memory allocations during the actual profiling phase,
@ -149,6 +150,7 @@ class VlExecutionProfiler final {
using ExecutionTrace = std::vector<VlExecutionRecord>;
// STATE
VerilatedContext& m_context; // The context this profiler is under
static VL_THREAD_LOCAL ExecutionTrace t_trace; // thread-local trace buffers
mutable VerilatedMutex m_mutex;
// Map from thread id to &t_trace of given thread
@ -162,7 +164,8 @@ class VlExecutionProfiler final {
public:
// CONSTRUCTOR
VlExecutionProfiler();
explicit VlExecutionProfiler(VerilatedContext& context);
virtual ~VlExecutionProfiler() = default;
// METHODS
@ -174,7 +177,7 @@ public:
return t_trace.back();
}
// Configure profiler (called in beginning of 'eval')
void configure(const VerilatedContext&);
void configure();
// Setup profiling on a particular thread;
void setupThread(uint32_t threadId);
// Clear all profiling data
@ -182,8 +185,8 @@ public:
// Write profiling data into file
void dump(const char* filenamep, uint64_t tickEnd) VL_MT_SAFE_EXCLUDES(m_mutex);
// Called via VlStartWorkerCb in VlWorkerThread::startWorker
static void startWorkerSetup(VlExecutionProfiler* profilep, uint32_t threadId);
// Passed to VerilatedContext to create the VlExecutionProfiler profiler instance
static VerilatedVirtualBase* construct(VerilatedContext& context);
};
//=============================================================================

View File

@ -47,11 +47,9 @@ VlMTaskVertex::VlMTaskVertex(uint32_t upstreamDepCount)
//=============================================================================
// VlWorkerThread
VlWorkerThread::VlWorkerThread(uint32_t threadId, VerilatedContext* contextp,
VlExecutionProfiler* profilerp, VlStartWorkerCb startCb)
VlWorkerThread::VlWorkerThread(VerilatedContext* contextp)
: m_ready_size{0}
, m_cthread{startWorker, this, threadId, profilerp, startCb}
, m_contextp{contextp} {}
, m_cthread{startWorker, this, contextp} {}
VlWorkerThread::~VlWorkerThread() {
shutdown();
@ -59,47 +57,49 @@ VlWorkerThread::~VlWorkerThread() {
m_cthread.join();
}
void VlWorkerThread::shutdownTask(void*, bool) {
static void shutdownTask(void*, bool) {
// Deliberately empty, we use the address of this function as a magic number
}
void VlWorkerThread::shutdown() { addTask(shutdownTask, nullptr); }
void VlWorkerThread::wait() {
// Enqueue a task that sets this flag. Execution is in-order so this ensures completion.
std::atomic<bool> flag{false};
addTask([](void* flagp, bool) { static_cast<std::atomic<bool>*>(flagp)->store(true); }, &flag);
// Spin wait
for (unsigned i = 0; i < VL_LOCK_SPINS; ++i) {
if (flag.load()) return;
VL_CPU_RELAX();
}
// Yield wait
while (!flag.load()) std::this_thread::yield();
}
void VlWorkerThread::workerLoop() {
ExecRec work;
// Wait for the first task without spinning, in case the thread is never actually used.
dequeWork</* SpinWait: */ false>(&work);
while (true) {
dequeWork(&work);
if (VL_UNLIKELY(work.m_fnp == shutdownTask)) break;
work.m_fnp(work.m_selfp, work.m_evenCycle);
// Wait for next task with spinning.
dequeWork</* SpinWait: */ true>(&work);
}
}
void VlWorkerThread::startWorker(VlWorkerThread* workerp, uint32_t threadId,
VlExecutionProfiler* profilerp, VlStartWorkerCb startCb) {
Verilated::threadContextp(workerp->m_contextp);
if (VL_UNLIKELY(startCb)) startCb(profilerp, threadId);
void VlWorkerThread::startWorker(VlWorkerThread* workerp, VerilatedContext* contextp) {
Verilated::threadContextp(contextp);
workerp->workerLoop();
}
//=============================================================================
// VlThreadPool
VlThreadPool::VlThreadPool(VerilatedContext* contextp, int nThreads,
VlExecutionProfiler* profilerp, VlStartWorkerCb startCb) {
// --threads N passes nThreads=N-1, as the "main" threads counts as 1
++nThreads;
const unsigned cpus = std::thread::hardware_concurrency();
if (cpus < nThreads) {
static int warnedOnce = 0;
if (!warnedOnce++) {
VL_PRINTF_MT("%%Warning: System has %u CPUs but model Verilated with"
" --threads %d; may run slow.\n",
cpus, nThreads);
}
}
// Create worker threads
for (uint32_t threadId = 1; threadId < nThreads; ++threadId) {
m_workers.push_back(new VlWorkerThread{threadId, contextp, profilerp, startCb});
}
VlThreadPool::VlThreadPool(VerilatedContext* contextp, unsigned nThreads) {
for (unsigned i = 0; i < nThreads; ++i) m_workers.push_back(new VlWorkerThread{contextp});
}
VlThreadPool::~VlThreadPool() {

View File

@ -60,9 +60,6 @@ using VlSelfP = void*;
using VlExecFnp = void (*)(VlSelfP, bool);
// VlWorkerThread::startWorker callback, used to hook in VlExecutionProfiler
using VlStartWorkerCb = void (*)(VlExecutionProfiler*, uint32_t threadId);
// Track dependencies for a single MTask.
class VlMTaskVertex final {
// MEMBERS
@ -166,24 +163,23 @@ private:
std::atomic<size_t> m_ready_size;
std::thread m_cthread; // Underlying C++ thread record
VerilatedContext* const m_contextp; // Context for spawned thread
VL_UNCOPYABLE(VlWorkerThread);
public:
// CONSTRUCTORS
explicit VlWorkerThread(uint32_t threadId, VerilatedContext* contextp,
VlExecutionProfiler* profilerp, VlStartWorkerCb startCb);
explicit VlWorkerThread(VerilatedContext* contextp);
~VlWorkerThread();
// METHODS
template <bool SpinWait> //
inline void dequeWork(ExecRec* workp) VL_MT_SAFE_EXCLUDES(m_mutex) {
// Spin for a while, waiting for new data
for (int i = 0; i < VL_LOCK_SPINS; ++i) {
if (VL_LIKELY(m_ready_size.load(std::memory_order_relaxed))) { //
break;
if VL_CONSTEXPR_CXX17 (SpinWait) {
for (unsigned i = 0; i < VL_LOCK_SPINS; ++i) {
if (VL_LIKELY(m_ready_size.load(std::memory_order_relaxed))) break;
VL_CPU_RELAX();
}
VL_CPU_RELAX();
}
VerilatedLockGuard lock{m_mutex};
while (m_ready.empty()) {
@ -197,7 +193,7 @@ public:
m_ready.erase(m_ready.begin());
m_ready_size.fetch_sub(1, std::memory_order_relaxed);
}
inline void addTask(VlExecFnp fnp, VlSelfP selfp, bool evenCycle)
inline void addTask(VlExecFnp fnp, VlSelfP selfp, bool evenCycle = false)
VL_MT_SAFE_EXCLUDES(m_mutex) {
bool notify;
{
@ -209,15 +205,14 @@ public:
if (notify) m_cv.notify_one();
}
inline void shutdown() { addTask(shutdownTask, nullptr, false); }
static void shutdownTask(void*, bool);
void shutdown(); // Finish current tasks, then terminate thread
void wait(); // Blocks calling thread until all tasks complete in this thread
void workerLoop();
static void startWorker(VlWorkerThread* workerp, uint32_t threadId,
VlExecutionProfiler* profilerp, VlStartWorkerCb startCb);
static void startWorker(VlWorkerThread* workerp, VerilatedContext* contextp);
};
class VlThreadPool final {
class VlThreadPool final : public VerilatedVirtualBase {
// MEMBERS
std::vector<VlWorkerThread*> m_workers; // our workers
@ -226,9 +221,8 @@ public:
// Construct a thread pool with 'nThreads' dedicated threads. The thread
// pool will create these threads and make them available to execute tasks
// via this->workerp(index)->addTask(...)
VlThreadPool(VerilatedContext* contextp, int nThreads, VlExecutionProfiler* profilerp,
VlStartWorkerCb startCb);
~VlThreadPool();
VlThreadPool(VerilatedContext* contextp, unsigned nThreads);
virtual ~VlThreadPool();
// METHODS
inline int numThreads() const { return m_workers.size(); }

View File

@ -194,8 +194,6 @@ private:
static void parallelWorkerTask(void*, bool);
#endif
using ParallelCallbackMap = std::unordered_map<VlThreadPool*, std::vector<CallbackRecord>>;
protected:
uint32_t* m_sigs_oldvalp = nullptr; // Previous value store
EData* m_sigs_enabledp = nullptr; // Bit vector of enabled codes (nullptr = all on)
@ -203,10 +201,10 @@ private:
uint64_t m_timeLastDump = 0; // Last time we did a dump
std::vector<bool> m_sigs_enabledVec; // Staging for m_sigs_enabledp
std::vector<CallbackRecord> m_initCbs; // Routines to initialize tracing
ParallelCallbackMap m_fullCbs; // Routines to perform full dump
ParallelCallbackMap m_chgCbs; // Routines to perform incremental dump
std::vector<CallbackRecord> m_fullCbs; // Routines to perform full dump
std::vector<CallbackRecord> m_chgCbs; // Routines to perform incremental dump
std::vector<CallbackRecord> m_cleanupCbs; // Routines to call at the end of dump
std::vector<VlThreadPool*> m_threadPoolps; // All thread pools, in insertion order
VerilatedContext* m_contextp = nullptr; // The context used by the traced models
bool m_fullDump = true; // Whether a full dump is required on the next call to 'dump'
uint32_t m_nextCode = 0; // Next code number to assign
uint32_t m_numSignals = 0; // Number of distinct signals
@ -217,16 +215,16 @@ private:
double m_timeRes = 1e-9; // Time resolution (ns/ms etc)
double m_timeUnit = 1e-0; // Time units (ns/ms etc)
void addThreadPool(VlThreadPool* threadPoolp) VL_MT_SAFE_EXCLUDES(m_mutex);
void addContext(VerilatedContext*) VL_MT_SAFE_EXCLUDES(m_mutex);
void addCallbackRecord(std::vector<CallbackRecord>& cbVec, CallbackRecord& cbRec)
void addCallbackRecord(std::vector<CallbackRecord>& cbVec, CallbackRecord&& cbRec)
VL_MT_SAFE_EXCLUDES(m_mutex);
// Equivalent to 'this' but is of the sub-type 'T_Trace*'. Use 'self()->'
// to access duck-typed functions to avoid a virtual function call.
T_Trace* self() { return static_cast<T_Trace*>(this); }
void runParallelCallbacks(const ParallelCallbackMap& cbMap);
void runCallbacks(const std::vector<CallbackRecord>& cbVec);
// Flush any remaining data for this file
static void onFlush(void* selfp) VL_MT_UNSAFE_ONE;
@ -341,10 +339,10 @@ public:
//=========================================================================
// Non-hot path internal interface to Verilator generated code
void addInitCb(initCb_t cb, void* userp) VL_MT_SAFE;
void addFullCb(dumpCb_t cb, void* userp, VlThreadPool* = nullptr) VL_MT_SAFE;
void addChgCb(dumpCb_t cb, void* userp, VlThreadPool* = nullptr) VL_MT_SAFE;
void addCleanupCb(cleanupCb_t cb, void* userp) VL_MT_SAFE;
void addInitCb(initCb_t cb, void* userp, VerilatedContext*) VL_MT_SAFE;
void addFullCb(dumpCb_t cb, void* userp, VerilatedContext*) VL_MT_SAFE;
void addChgCb(dumpCb_t cb, void* userp, VerilatedContext*) VL_MT_SAFE;
void addCleanupCb(cleanupCb_t cb, void* userp, VerilatedContext*) VL_MT_SAFE;
void scopeEscape(char flag) { m_scopeEscape = flag; }

View File

@ -478,55 +478,52 @@ template <> VL_ATTR_NOINLINE void VerilatedTrace<VL_SUB_T, VL_BUF_T>::ParallelWo
#endif
template <>
void VerilatedTrace<VL_SUB_T, VL_BUF_T>::runParallelCallbacks(const ParallelCallbackMap& cbMap) {
for (VlThreadPool* threadPoolp : m_threadPoolps) {
void VerilatedTrace<VL_SUB_T, VL_BUF_T>::runCallbacks(const std::vector<CallbackRecord>& cbVec) {
#ifdef VL_TRACE_PARALLEL
// If tracing in parallel, dispatch to the thread pool (if exists)
if (threadPoolp && threadPoolp->numThreads()) {
// List of work items for thread (std::list, as ParallelWorkerData is not movable)
std::list<ParallelWorkerData> workerData;
// We use the whole pool + the main thread
const unsigned threads = threadPoolp->numThreads() + 1;
// Main thread executes all jobs with index % threads == 0
std::vector<ParallelWorkerData*> mainThreadWorkerData;
// The tracing callbacks to execute on this thread-pool
const auto& cbVec = cbMap.at(threadPoolp);
// Enuque all the jobs
for (unsigned i = 0; i < cbVec.size(); ++i) {
const CallbackRecord& cbr = cbVec[i];
// Always get the trace buffer on the main thread
Buffer* const bufp = getTraceBuffer();
// Create new work item
workerData.emplace_back(cbr.m_dumpCb, cbr.m_userp, bufp);
// Grab the new work item
ParallelWorkerData* const itemp = &workerData.back();
// Enqueue task to thread pool, or main thread
if (unsigned rem = i % threads) {
threadPoolp->workerp(rem - 1)->addTask(parallelWorkerTask, itemp, false);
} else {
mainThreadWorkerData.push_back(itemp);
}
}
// Execute main thead jobs
for (ParallelWorkerData* const itemp : mainThreadWorkerData) {
parallelWorkerTask(itemp, false);
}
// Commit all trace buffers in order
for (ParallelWorkerData& item : workerData) {
// Wait until ready
item.wait();
// Commit the buffer
commitTraceBuffer(item.m_bufp);
}
continue;
// If tracing in parallel, dispatch to the thread pool
VlThreadPool* threadPoolp = static_cast<VlThreadPool*>(m_contextp->threadPoolp());
// List of work items for thread (std::list, as ParallelWorkerData is not movable)
std::list<ParallelWorkerData> workerData;
// We use the whole pool + the main thread
const unsigned threads = threadPoolp->numThreads() + 1;
// Main thread executes all jobs with index % threads == 0
std::vector<ParallelWorkerData*> mainThreadWorkerData;
// Enuque all the jobs
for (unsigned i = 0; i < cbVec.size(); ++i) {
const CallbackRecord& cbr = cbVec[i];
// Always get the trace buffer on the main thread
Buffer* const bufp = getTraceBuffer();
// Create new work item
workerData.emplace_back(cbr.m_dumpCb, cbr.m_userp, bufp);
// Grab the new work item
ParallelWorkerData* const itemp = &workerData.back();
// Enqueue task to thread pool, or main thread
if (unsigned rem = i % threads) {
threadPoolp->workerp(rem - 1)->addTask(parallelWorkerTask, itemp);
} else {
mainThreadWorkerData.push_back(itemp);
}
}
// Execute main thead jobs
for (ParallelWorkerData* const itemp : mainThreadWorkerData) {
parallelWorkerTask(itemp, false);
}
// Commit all trace buffers in order
for (ParallelWorkerData& item : workerData) {
// Wait until ready
item.wait();
// Commit the buffer
commitTraceBuffer(item.m_bufp);
}
// Done
return;
#endif
// Fall back on sequential execution
for (const CallbackRecord& cbr : cbMap.at(threadPoolp)) {
Buffer* const traceBufferp = getTraceBuffer();
cbr.m_dumpCb(cbr.m_userp, traceBufferp);
commitTraceBuffer(traceBufferp);
}
// Fall back on sequential execution
for (const CallbackRecord& cbr : cbVec) {
Buffer* const traceBufferp = getTraceBuffer();
cbr.m_dumpCb(cbr.m_userp, traceBufferp);
commitTraceBuffer(traceBufferp);
}
}
@ -579,9 +576,9 @@ void VerilatedTrace<VL_SUB_T, VL_BUF_T>::dump(uint64_t timeui) VL_MT_SAFE_EXCLUD
// Run the callbacks
if (VL_UNLIKELY(m_fullDump)) {
m_fullDump = false; // No more need for next dump to be full
runParallelCallbacks(m_fullCbs);
runCallbacks(m_fullCbs);
} else {
runParallelCallbacks(m_chgCbs);
runCallbacks(m_chgCbs);
}
for (uint32_t i = 0; i < m_cleanupCbs.size(); ++i) {
@ -607,18 +604,20 @@ void VerilatedTrace<VL_SUB_T, VL_BUF_T>::dump(uint64_t timeui) VL_MT_SAFE_EXCLUD
// Non-hot path internal interface to Verilator generated code
template <>
void VerilatedTrace<VL_SUB_T, VL_BUF_T>::addThreadPool(VlThreadPool* threadPoolp)
void VerilatedTrace<VL_SUB_T, VL_BUF_T>::addContext(VerilatedContext* contextp)
VL_MT_SAFE_EXCLUDES(m_mutex) {
const VerilatedLockGuard lock{m_mutex};
for (VlThreadPool* const poolp : m_threadPoolps) {
if (poolp == threadPoolp) return;
if (m_contextp && contextp != m_contextp) {
VL_FATAL_MT(
__FILE__, __LINE__, "",
"A trace file instance can only handle models from the same simulation context");
}
m_threadPoolps.push_back(threadPoolp);
m_contextp = contextp;
}
template <>
void VerilatedTrace<VL_SUB_T, VL_BUF_T>::addCallbackRecord(std::vector<CallbackRecord>& cbVec,
CallbackRecord& cbRec)
CallbackRecord&& cbRec)
VL_MT_SAFE_EXCLUDES(m_mutex) {
const VerilatedLockGuard lock{m_mutex};
if (VL_UNCOVERABLE(timeLastDump() != 0)) { // LCOV_EXCL_START
@ -630,28 +629,28 @@ void VerilatedTrace<VL_SUB_T, VL_BUF_T>::addCallbackRecord(std::vector<CallbackR
}
template <>
void VerilatedTrace<VL_SUB_T, VL_BUF_T>::addInitCb(initCb_t cb, void* userp) VL_MT_SAFE {
CallbackRecord cbr{cb, userp};
addCallbackRecord(m_initCbs, cbr);
void VerilatedTrace<VL_SUB_T, VL_BUF_T>::addInitCb(initCb_t cb, void* userp,
VerilatedContext* contextp) VL_MT_SAFE {
addContext(contextp);
addCallbackRecord(m_initCbs, CallbackRecord{cb, userp});
}
template <>
void VerilatedTrace<VL_SUB_T, VL_BUF_T>::addFullCb(dumpCb_t cb, void* userp,
VlThreadPool* threadPoolp) VL_MT_SAFE {
CallbackRecord cbr{cb, userp};
addThreadPool(threadPoolp);
addCallbackRecord(m_fullCbs[threadPoolp], cbr);
VerilatedContext* contextp) VL_MT_SAFE {
addContext(contextp);
addCallbackRecord(m_fullCbs, CallbackRecord{cb, userp});
}
template <>
void VerilatedTrace<VL_SUB_T, VL_BUF_T>::addChgCb(dumpCb_t cb, void* userp,
VlThreadPool* threadPoolp) VL_MT_SAFE {
CallbackRecord cbr{cb, userp};
addThreadPool(threadPoolp);
addCallbackRecord(m_chgCbs[threadPoolp], cbr);
VerilatedContext* contextp) VL_MT_SAFE {
addContext(contextp);
addCallbackRecord(m_chgCbs, CallbackRecord{cb, userp});
}
template <>
void VerilatedTrace<VL_SUB_T, VL_BUF_T>::addCleanupCb(cleanupCb_t cb, void* userp) VL_MT_SAFE {
CallbackRecord cbr{cb, userp};
addCallbackRecord(m_cleanupCbs, cbr);
void VerilatedTrace<VL_SUB_T, VL_BUF_T>::addCleanupCb(cleanupCb_t cb, void* userp,
VerilatedContext* contextp) VL_MT_SAFE {
addContext(contextp);
addCallbackRecord(m_cleanupCbs, CallbackRecord{cb, userp});
}
template <> void VerilatedTrace<VL_SUB_T, VL_BUF_T>::pushNamePrefix(const std::string& prefix) {

View File

@ -251,15 +251,15 @@ class EmitCHeader final : public EmitCConstInit {
emitTextSection(modp, VNType::atScHdr);
// Open class body {{{
puts("\nclass ");
puts(prefixNameProtect(modp));
if (const AstClass* const classp = VN_CAST(modp, Class)) {
puts("class ");
puts(prefixNameProtect(modp));
if (classp->extendsp()) {
puts(" : public ");
puts(prefixNameProtect(classp->extendsp()->classp()));
}
} else {
puts("VL_MODULE(" + prefixNameProtect(modp) + ")");
puts(" final : public VerilatedModule");
}
puts(" {\n");
ofp()->resetPrivate();

View File

@ -173,7 +173,7 @@ class CMakeEmitter final {
+ ".cpp");
}
}
if (v3Global.opt.mtasks()) {
if (v3Global.opt.threads()) {
global.emplace_back("${VERILATOR_ROOT}/include/verilated_threads.cpp");
}
if (v3Global.opt.usesProfiler()) {

View File

@ -89,11 +89,12 @@ class EmitCModel final : public EmitCFunc {
puts("\n");
puts("// This class is the main interface to the Verilated model\n");
puts("class " + topClassName() + " VL_NOT_FINAL : ");
if (optSystemC()) {
puts("SC_MODULE(" + topClassName() + ") {\n");
} else {
puts("class " + topClassName() + " VL_NOT_FINAL {\n");
// SC_MODULE, but with multiple-inheritance of VerilatedModel
puts("public ::sc_core::sc_module, ");
}
puts("public VerilatedModel {\n");
ofp()->resetPrivate();
ofp()->putsPrivate(true); // private:
@ -221,6 +222,11 @@ class EmitCModel final : public EmitCFunc {
+ topClassName() + "& rhs);\n");
}
puts("\n// Abstract methods from VerilatedModel\n");
puts("const char* hierName() override;\n");
puts("const char* modelName() override;\n");
puts("unsigned threads() override;\n");
puts("} VL_ATTR_ALIGNED(VL_CACHE_LINE_BYTES);\n");
ofp()->putsEndGuard();
@ -235,7 +241,8 @@ class EmitCModel final : public EmitCFunc {
puts(topClassName() + "::" + topClassName());
if (optSystemC()) {
puts("(sc_module_name /* unused */)\n");
puts(" : vlSymsp{new " + symClassName() + "(nullptr, name(), this)}\n");
puts(" : vlSymsp{new " + symClassName()
+ "(Verilated::threadContextp(), name(), this)}\n");
} else {
puts(+"(VerilatedContext* _vcontextp__, const char* _vcname__)\n");
puts(" : vlSymsp{new " + symClassName() + "(_vcontextp__, _vcname__, this)}\n");
@ -263,6 +270,8 @@ class EmitCModel final : public EmitCFunc {
puts(" , rootp{&(vlSymsp->TOP)}\n");
puts("{\n");
puts("// Register model with the context\n");
puts("vlSymsp->_vm_contextp__->addModel(this);\n");
if (optSystemC()) {
// Create sensitivity list for when to evaluate the model.
@ -301,7 +310,7 @@ class EmitCModel final : public EmitCFunc {
if (!optSystemC()) {
puts("\n");
puts(topClassName() + "::" + topClassName() + "(const char* _vcname__)\n");
puts(" : " + topClassName() + "(nullptr, _vcname__)\n{\n}\n");
puts(" : " + topClassName() + "(Verilated::threadContextp(), _vcname__)\n{\n}\n");
}
}
@ -428,7 +437,7 @@ class EmitCModel final : public EmitCFunc {
}
if (v3Global.opt.profExec()) {
puts("vlSymsp->__Vm_executionProfiler.configure(*(vlSymsp->_vm_contextp__));\n");
puts("vlSymsp->__Vm_executionProfilerp->configure();\n");
puts("VL_EXEC_TRACE_ADD_RECORD(vlSymsp).evalBegin();\n");
}
@ -477,6 +486,13 @@ class EmitCModel final : public EmitCFunc {
puts("\nVL_ATTR_COLD void " + topClassName() + "::final() {\n");
puts(/**/ topModNameProtected + "__" + protect("_final") + "(&(vlSymsp->TOP));\n");
puts("}\n");
putSectionDelimiter("Implementations of abstract methods from VerilatedModel\n");
puts("const char* " + topClassName() + "::hierName() { return vlSymsp->name(); }\n");
puts("const char* " + topClassName() + "::modelName() { return \"" + topClassName()
+ "\"; }\n");
puts("unsigned " + topClassName() + "::threads() { return "
+ cvtToStr(std::max(1, v3Global.opt.threads())) + "; }\n");
}
void emitTraceMethods(AstNodeModule* modp) {
@ -529,7 +545,8 @@ class EmitCModel final : public EmitCFunc {
puts(/**/ "}");
}
puts(/**/ "if (false && levels && options) {} // Prevent unused\n");
puts(/**/ "tfp->spTrace()->addInitCb(&" + protect("trace_init") + ", &(vlSymsp->TOP));\n");
puts(/**/ "tfp->spTrace()->addInitCb(&" + protect("trace_init")
+ ", &(vlSymsp->TOP), contextp());\n");
puts(/**/ topModNameProtected + "__" + protect("trace_register")
+ "(&(vlSymsp->TOP), tfp->spTrace());\n");

View File

@ -445,17 +445,17 @@ void EmitCSyms::emitSymHdr() {
}
puts("bool __Vm_didInit = false;\n");
if (v3Global.opt.profExec()) {
puts("\n// EXECUTION PROFILING\n");
puts("VlExecutionProfiler __Vm_executionProfiler;\n");
}
if (v3Global.opt.mtasks()) {
puts("\n// MULTI-THREADING\n");
puts("VlThreadPool* const __Vm_threadPoolp;\n");
puts("bool __Vm_even_cycle = false;\n");
}
if (v3Global.opt.profExec()) {
puts("\n// EXECUTION PROFILING\n");
puts("VlExecutionProfiler* const __Vm_executionProfilerp;\n");
}
puts("\n// MODULE INSTANCE STATE\n");
for (const auto& i : m_scopes) {
const AstScope* const scopep = i.first;
@ -673,7 +673,6 @@ void EmitCSyms::emitSymImp() {
puts("_vm_pgoProfiler.write(\"" + topClassName()
+ "\", _vm_contextp__->profVltFilename());\n");
}
if (v3Global.opt.mtasks()) puts("delete __Vm_threadPoolp;\n");
puts("}\n\n");
// Constructor
@ -705,12 +704,13 @@ void EmitCSyms::emitSymImp() {
// Note we create N-1 threads in the thread pool. The thread
// that calls eval() becomes the final Nth thread for the
// duration of the eval call.
puts(" , __Vm_threadPoolp{new VlThreadPool{_vm_contextp__, "
+ cvtToStr(v3Global.opt.threads() - 1) + ", "
+ (v3Global.opt.profExec()
? "&__Vm_executionProfiler, &VlExecutionProfiler::startWorkerSetup"
: "nullptr, nullptr")
+ "}}\n");
puts(" , __Vm_threadPoolp{static_cast<VlThreadPool*>(contextp->threadPoolp())}\n");
}
if (v3Global.opt.profExec()) {
puts(" , "
"__Vm_executionProfilerp{static_cast<VlExecutionProfiler*>(contextp->"
"enableExecutionProfiler(&VlExecutionProfiler::construct))}\n");
}
puts(" // Setup module instances\n");

View File

@ -116,7 +116,7 @@ public:
putMakeClassEntry(of, v3Global.opt.traceSourceLang() + ".cpp");
}
}
if (v3Global.opt.mtasks()) putMakeClassEntry(of, "verilated_threads.cpp");
if (v3Global.opt.threads()) putMakeClassEntry(of, "verilated_threads.cpp");
if (v3Global.opt.usesProfiler()) {
putMakeClassEntry(of, "verilated_profiler.cpp");
}

View File

@ -512,8 +512,10 @@ private:
m_regFuncp->addStmtsp(new AstText(flp, "tracep->addChgCb(", true));
}
m_regFuncp->addStmtsp(new AstAddrOfCFunc(flp, funcp));
const string threadPool{m_parallelism > 1 ? "vlSymsp->__Vm_threadPoolp" : "nullptr"};
m_regFuncp->addStmtsp(new AstText(flp, ", vlSelf, " + threadPool + ");\n", true));
m_regFuncp->addStmtsp(new AstText(flp, ", vlSelf", true));
m_regFuncp->addStmtsp(
new AstText(flp, ", vlSelf->vlSymsp->__Vm_modelp->contextp()", true));
m_regFuncp->addStmtsp(new AstText(flp, ");\n", true));
} else {
// Sub functions
funcp->argTypes(v3Global.opt.traceClassBase() + "::Buffer* bufp");
@ -700,7 +702,8 @@ private:
// Register it
m_regFuncp->addStmtsp(new AstText(fl, "tracep->addCleanupCb(", true));
m_regFuncp->addStmtsp(new AstAddrOfCFunc(fl, cleanupFuncp));
m_regFuncp->addStmtsp(new AstText(fl, ", vlSelf);\n", true));
m_regFuncp->addStmtsp(
new AstText(fl, ", vlSelf, vlSelf->vlSymsp->__Vm_modelp->contextp());\n", true));
// Clear global activity flag
cleanupFuncp->addStmtsp(

View File

@ -579,6 +579,7 @@ sub new {
sc_time_resolution => "SC_PS", # Keep - PS is SystemC default
sim_time => 1100,
threads => -1, # --threads (negative means auto based on scenario)
context_threads => 0, # Number of threads to allocate in the context
benchmark => $opt_benchmark,
verbose => $opt_verbose,
run_env => '',
@ -974,7 +975,11 @@ sub compile {
$self->oprint("Compile\n") if $self->{verbose};
die "%Error: 'threads =>' argument must be <= 1 for vlt scenario" if $param{vlt} && $param{threads} > 1;
$param{threads} = ::calc_threads($Vltmt_threads) if ($param{threads} < 0 && $param{vltmt});
# Compute automatic parameter values
$param{threads} = ::calc_threads($Vltmt_threads) if $param{threads} < 0 && $param{vltmt};
$param{context_threads} = $param{threads} >= 1 ? $param{threads} : 1 if !$param{context_threads};
$self->{threads} = $param{threads};
$self->{context_threads} = $param{context_threads};
compile_vlt_cmd(%param);
@ -1795,6 +1800,7 @@ sub _make_main {
}
print $fh " const std::unique_ptr<VerilatedContext> contextp{new VerilatedContext};\n";
print $fh " contextp->threads($self->{context_threads});\n";
print $fh " contextp->commandArgs(argc, argv);\n";
print $fh " contextp->debug(" . ($self->{verilated_debug} ? 1 : 0) . ");\n";
print $fh " srand48(5);\n"; # Ensure determinism

View File

@ -22,7 +22,8 @@ mkdir $child_dir;
(VM_PREFIX => "$Self->{VM_PREFIX}_child",
top_filename => "$Self->{name}_child.v",
verilator_flags => ["-cc", "-Mdir", "${child_dir}", "--debug-check"],
threads => $Self->{vltmt} ? $Self->get_default_vltmt_threads() : 0
# Can't use multi threading (like hier blocks), but needs to be thread safe
threads => $Self->{vltmt} ? 1 : 0,
);
run(logfile => "${child_dir}/vlt_compile.log",

View File

@ -0,0 +1,43 @@
//
// DESCRIPTION: Verilator: Verilog Multiple Model Test Module
//
// This file ONLY is placed under the Creative Commons Public Domain, for
// any use, without warranty, 2022 by Geza Lore.
// SPDX-License-Identifier: CC0-1.0
//
#include <memory>
#include "verilated.h"
#include "Vt_gantt_two.h"
int main(int argc, char** argv, char** env) {
srand48(5);
const std::unique_ptr<VerilatedContext> contextp{new VerilatedContext};
#ifdef VL_THREADED
contextp->threads(2);
#endif
contextp->commandArgs(argc, argv);
contextp->debug(0);
std::unique_ptr<Vt_gantt_two> topap{new Vt_gantt_two{contextp.get(), "topa"}};
std::unique_ptr<Vt_gantt_two> topbp{new Vt_gantt_two{contextp.get(), "topb"}};
topap->clk = false;
topap->eval();
topbp->clk = false;
topbp->eval();
contextp->timeInc(10);
while ((contextp->time() < 1100) && !contextp->gotFinish()) {
topap->clk = !topap->clk;
topap->eval();
topbp->clk = !topbp->clk;
topbp->eval();
contextp->timeInc(5);
}
if (!contextp->gotFinish()) {
vl_fatal(__FILE__, __LINE__, "main", "%Error: Timeout; never got a $finish");
}
return 0;
}

61
test_regress/t/t_gantt_two.pl Executable file
View File

@ -0,0 +1,61 @@
#!/usr/bin/env perl
if (!$::Driver) { use FindBin; exec("$FindBin::Bin/bootstrap.pl", @ARGV, $0); die; }
# DESCRIPTION: Verilator: Verilog Test driver/expect definition
#
# Copyright 2003 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
# Test for bin/verilator_gantt,
scenarios(vlt_all => 1);
# It doesn't really matter what test
# we use, so long as it runs several cycles,
# enough for the profiling to happen:
top_filename("t/t_gen_alw.v");
compile(
make_top_shell => 0,
make_main => 0,
v_flags2 => ["--prof-exec --exe $Self->{t_dir}/$Self->{name}.cpp"],
# Checks below care about thread count, so use 2 (minimum reasonable)
threads => $Self->{vltmt} ? 2 : 0,
make_flags => 'CPPFLAGS_ADD=-DVL_NO_LEGACY',
);
execute(
all_run_flags => ["+verilator+prof+exec+start+4",
" +verilator+prof+exec+window+4",
" +verilator+prof+exec+file+$Self->{obj_dir}/profile_exec.dat",
" +verilator+prof+vlt+file+$Self->{obj_dir}/profile.vlt",
],
check_finished => 1,
);
# For now, verilator_gantt still reads from STDIN
# (probably it should take a file, gantt.dat like verilator_profcfunc)
# The profiling data still goes direct to the runtime's STDOUT
# (maybe that should go to a separate file - gantt.dat?)
run(cmd => ["$ENV{VERILATOR_ROOT}/bin/verilator_gantt",
"$Self->{obj_dir}/profile_exec.dat",
"--vcd $Self->{obj_dir}/profile_exec.vcd",
"| tee $Self->{obj_dir}/gantt.log"],
);
if ($Self->{vltmt}) {
file_grep("$Self->{obj_dir}/gantt.log", qr/Total threads += 2/i);
file_grep("$Self->{obj_dir}/gantt.log", qr/Total mtasks += 7/i);
} else {
file_grep("$Self->{obj_dir}/gantt.log", qr/Total threads += 1/i);
file_grep("$Self->{obj_dir}/gantt.log", qr/Total mtasks += 0/i);
}
file_grep("$Self->{obj_dir}/gantt.log", qr/Total evals += 4/i);
# Diff to itself, just to check parsing
vcd_identical("$Self->{obj_dir}/profile_exec.vcd", "$Self->{obj_dir}/profile_exec.vcd");
ok(1);
1;

View File

@ -14,8 +14,11 @@
int main(int argc, char *argv[]) {
const std::unique_ptr<VerilatedContext> contextp{new VerilatedContext};
std::unique_ptr<Vt_hier_block> top{new Vt_hier_block{contextp.get(), "top"}};
#if VL_THREADED
contextp->threads(6);
#endif
contextp->commandArgs(argc, argv);
std::unique_ptr<Vt_hier_block> top{new Vt_hier_block{contextp.get(), "top"}};
for (int i = 0; i < 100 && !contextp->gotFinish(); ++i) {
top->eval();
top->clk ^= 1;

View File

@ -59,7 +59,8 @@ while (1) {
"-LDFLAGS",
"'-Wl,-rpath,$abs_secret_dir -L$abs_secret_dir -l$secret_prefix'"],
xsim_flags2 => ["$secret_dir/secret.sv"],
threads => $Self->{vltmt} ? 1 : 0
threads => $Self->{vltmt} ? 1 : 0,
context_threads => $Self->{vltmt} ? 6 : 1
);
execute(

View File

@ -10,20 +10,16 @@ if (!$::Driver) { use FindBin; exec("$FindBin::Bin/bootstrap.pl", @ARGV, $0); di
scenarios(vltmt => 1);
if ($Self->cfg_with_m32) {
skip("Does not work with -m32 (resource unavailable)");
}
compile(
verilator_flags2 => ['--cc'],
threads => 1024
threads => 4,
context_threads => 2
);
execute(
check_finished => 1,
fails => 1
);
file_grep($Self->{run_log_filename}, qr/System has .* CPUs but.*--threads 1024/);
file_grep($Self->{run_log_filename}, qr/%Error: .*\/verilated\.cpp:\d+: VerilatedContext has 2 threads but model 'Vt_threads_crazy' \(instantiated as 'top'\) was Verilated with --threads 4\./);
ok(1);
1;

View File

@ -0,0 +1,36 @@
#!/usr/bin/env perl
if (!$::Driver) { use FindBin; exec("$FindBin::Bin/bootstrap.pl", @ARGV, $0); die; }
# DESCRIPTION: Verilator: Verilog Test driver/expect definition
#
# Copyright 2003-2009 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
scenarios(vlt_all => 1);
if ($Self->cfg_with_m32) {
skip("Does not work with -m32 (resource unavailable)");
}
top_filename("t/t_threads_crazy.v");
compile(
verilator_flags2 => ['--cc'],
threads => $Self->{vltmt} ? 2 : 0,
context_threads => 1024
);
execute(
check_finished => 1,
);
if ($Self->{vltmt}) {
file_grep($Self->{run_log_filename}, qr/System has \d+ hardware threads but simulation thread count set to 1024\. This will likely cause significant slowdown\./);
} else {
file_grep($Self->{run_log_filename}, qr/Verilator run-time library built without VL_THREADS\. Ignoring call to 'VerilatedContext::threads' with argument 1024\./);
}
ok(1);
1;

View File

@ -92,6 +92,8 @@ int main(int argc, char** argv, char** env) {
std::unique_ptr<VerilatedContext> context1p{new VerilatedContext};
// configuration
context0p->threads(1);
context1p->threads(1);
context0p->fatalOnError(false);
context1p->fatalOnError(false);
context0p->traceEverOn(true);