Source File
proc.go
Belonging Package
runtime
// Copyright 2014 The Go Authors. All rights reserved.// Use of this source code is governed by a BSD-style// license that can be found in the LICENSE file.package runtimeimport ()// set using cmd/go/internal/modload.ModInfoProgvar modinfo string// Goroutine scheduler// The scheduler's job is to distribute ready-to-run goroutines over worker threads.//// The main concepts are:// G - goroutine.// M - worker thread, or machine.// P - processor, a resource that is required to execute Go code.// M must have an associated P to execute Go code, however it can be// blocked or in a syscall w/o an associated P.//// Design doc at https://golang.org/s/go11sched.// Worker thread parking/unparking.// We need to balance between keeping enough running worker threads to utilize// available hardware parallelism and parking excessive running worker threads// to conserve CPU resources and power. This is not simple for two reasons:// (1) scheduler state is intentionally distributed (in particular, per-P work// queues), so it is not possible to compute global predicates on fast paths;// (2) for optimal thread management we would need to know the future (don't park// a worker thread when a new goroutine will be readied in near future).//// Three rejected approaches that would work badly:// 1. Centralize all scheduler state (would inhibit scalability).// 2. Direct goroutine handoff. That is, when we ready a new goroutine and there// is a spare P, unpark a thread and handoff it the thread and the goroutine.// This would lead to thread state thrashing, as the thread that readied the// goroutine can be out of work the very next moment, we will need to park it.// Also, it would destroy locality of computation as we want to preserve// dependent goroutines on the same thread; and introduce additional latency.// 3. Unpark an additional thread whenever we ready a goroutine and there is an// idle P, but don't do handoff. This would lead to excessive thread parking/// unparking as the additional threads will instantly park without discovering// any work to do.//// The current approach://// This approach applies to three primary sources of potential work: readying a// goroutine, new/modified-earlier timers, and idle-priority GC. See below for// additional details.//// We unpark an additional thread when we submit work if (this is wakep()):// 1. There is an idle P, and// 2. There are no "spinning" worker threads.//// A worker thread is considered spinning if it is out of local work and did// not find work in the global run queue or netpoller; the spinning state is// denoted in m.spinning and in sched.nmspinning. Threads unparked this way are// also considered spinning; we don't do goroutine handoff so such threads are// out of work initially. Spinning threads spin on looking for work in per-P// run queues and timer heaps or from the GC before parking. If a spinning// thread finds work it takes itself out of the spinning state and proceeds to// execution. If it does not find work it takes itself out of the spinning// state and then parks.//// If there is at least one spinning thread (sched.nmspinning>1), we don't// unpark new threads when submitting work. To compensate for that, if the last// spinning thread finds work and stops spinning, it must unpark a new spinning// thread. This approach smooths out unjustified spikes of thread unparking,// but at the same time guarantees eventual maximal CPU parallelism// utilization.//// The main implementation complication is that we need to be very careful// during spinning->non-spinning thread transition. This transition can race// with submission of new work, and either one part or another needs to unpark// another worker thread. If they both fail to do that, we can end up with// semi-persistent CPU underutilization.//// The general pattern for submission is:// 1. Submit work to the local or global run queue, timer heap, or GC state.// 2. #StoreLoad-style memory barrier.// 3. Check sched.nmspinning.//// The general pattern for spinning->non-spinning transition is:// 1. Decrement nmspinning.// 2. #StoreLoad-style memory barrier.// 3. Check all per-P work queues and GC for new work.//// Note that all this complexity does not apply to global run queue as we are// not sloppy about thread unparking when submitting to global queue. Also see// comments for nmspinning manipulation.//// How these different sources of work behave varies, though it doesn't affect// the synchronization approach:// * Ready goroutine: this is an obvious source of work; the goroutine is// immediately ready and must run on some thread eventually.// * New/modified-earlier timer: The current timer implementation (see time.go)// uses netpoll in a thread with no work available to wait for the soonest// timer. If there is no thread waiting, we want a new spinning thread to go// wait.// * Idle-priority GC: The GC wakes a stopped idle thread to contribute to// background GC work (note: currently disabled per golang.org/issue/19112).// Also see golang.org/issue/44313, as this should be extended to all GC// workers.var (m0 mg0 gmcache0 *mcacheraceprocctx0 uintptrraceFiniLock mutex)// This slice records the initializing tasks that need to be// done to start up the runtime. It is built by the linker.var runtime_inittasks []*initTask// main_init_done is a signal used by cgocallbackg that initialization// has been completed. It is made before _cgo_notify_runtime_init_done,// so all cgo calls can rely on it existing. When main_init is complete,// it is closed, meaning cgocallbackg can reliably receive from it.var main_init_done chan bool//go:linkname main_main main.mainfunc main_main()// mainStarted indicates that the main M has started.var mainStarted bool// runtimeInitTime is the nanotime() at which the runtime started.var runtimeInitTime int64// Value to use for signal mask for newly created M's.var initSigmask sigset// The main goroutine.func main() {:= getg().m// Racectx of m0->g0 is used only as the parent of the main goroutine.// It must not be used for anything else..g0.racectx = 0// Max stack size is 1 GB on 64-bit, 250 MB on 32-bit.// Using decimal instead of binary GB and MB because// they look nicer in the stack overflow failure message.if goarch.PtrSize == 8 {maxstacksize = 1000000000} else {maxstacksize = 250000000}// An upper limit for max stack size. Used to avoid random crashes// after calling SetMaxStack and trying to allocate a stack that is too big,// since stackalloc works with 32-bit sizes.maxstackceiling = 2 * maxstacksize// Allow newproc to start new Ms.mainStarted = trueif haveSysmon {systemstack(func() {newm(sysmon, nil, -1)})}// Lock the main goroutine onto this, the main OS thread,// during initialization. Most programs won't care, but a few// do require certain calls to be made by the main thread.// Those can arrange for main.main to run in the main thread// by calling runtime.LockOSThread during initialization// to preserve the lock.lockOSThread()if != &m0 {throw("runtime.main not on m0")}// Record when the world started.// Must be before doInit for tracing init.runtimeInitTime = nanotime()if runtimeInitTime == 0 {throw("nanotime returning zero")}if debug.inittrace != 0 {inittrace.id = getg().goidinittrace.active = true}doInit(runtime_inittasks) // Must be before defer.// Defer unlock so that runtime.Goexit during init does the unlock too.:= truedefer func() {if {unlockOSThread()}}()gcenable()defaultGOMAXPROCSUpdateEnable() // don't STW before runtime initialized.main_init_done = make(chan bool)if iscgo {if _cgo_pthread_key_created == nil {throw("_cgo_pthread_key_created missing")}if _cgo_thread_start == nil {throw("_cgo_thread_start missing")}if GOOS != "windows" {if _cgo_setenv == nil {throw("_cgo_setenv missing")}if _cgo_unsetenv == nil {throw("_cgo_unsetenv missing")}}if _cgo_notify_runtime_init_done == nil {throw("_cgo_notify_runtime_init_done missing")}// Set the x_crosscall2_ptr C function pointer variable point to crosscall2.if set_crosscall2 == nil {throw("set_crosscall2 missing")}set_crosscall2()// Start the template thread in case we enter Go from// a C-created thread and need to create a new thread.startTemplateThread()cgocall(_cgo_notify_runtime_init_done, nil)}// Run the initializing tasks. Depending on build mode this// list can arrive a few different ways, but it will always// contain the init tasks computed by the linker for all the// packages in the program (excluding those added at runtime// by package plugin). Run through the modules in dependency// order (the order they are initialized by the dynamic// loader, i.e. they are added to the moduledata linked list).for := &firstmoduledata; != nil; = .next {doInit(.inittasks)}// Disable init tracing after main init done to avoid overhead// of collecting statistics in malloc and newprocinittrace.active = falseclose(main_init_done)= falseunlockOSThread()if isarchive || islibrary {// A program compiled with -buildmode=c-archive or c-shared// has a main, but it is not executed.if GOARCH == "wasm" {// On Wasm, pause makes it return to the host.// Unlike cgo callbacks where Ms are created on demand,// on Wasm we have only one M. So we keep this M (and this// G) for callbacks.// Using the caller's SP unwinds this frame and backs to// goexit. The -16 is: 8 for goexit's (fake) return PC,// and pause's epilogue pops 8.pause(sys.GetCallerSP() - 16) // should not returnpanic("unreachable")}return}:= main_main // make an indirect call, as the linker doesn't know the address of the main package when laying down the runtime():= falseif raceenabled {runExitHooks(0) // run hooks now, since racefini does not return= trueracefini()}// Check for C memory leaks if using ASAN and we've made cgo calls,// or if we are running as a library in a C program.// We always make one cgo call, above, to notify_runtime_init_done,// so we ignore that one.// No point in leak checking if no cgo calls, since leak checking// just looks for objects allocated using malloc and friends.// Just checking iscgo doesn't help because asan implies iscgo.if asanenabled && (isarchive || islibrary || NumCgoCall() > 1) {runExitHooks(0) // lsandoleakcheck may not return= truelsandoleakcheck()}// Make racy client program work: if panicking on// another goroutine at the same time as main returns,// let the other goroutine finish printing the panic trace.// Once it does, it will exit. See issues 3934 and 20018.if runningPanicDefers.Load() != 0 {// Running deferred functions should not take long.for := 0; < 1000; ++ {if runningPanicDefers.Load() == 0 {break}Gosched()}}if panicking.Load() != 0 {gopark(nil, nil, waitReasonPanicWait, traceBlockForever, 1)}if ! {runExitHooks(0)}exit(0)for {var *int32* = 0}}// os_beforeExit is called from os.Exit(0).////go:linkname os_beforeExit os.runtime_beforeExitfunc os_beforeExit( int) {runExitHooks()if == 0 && raceenabled {racefini()}// See comment in main, above.if == 0 && asanenabled && (isarchive || islibrary || NumCgoCall() > 1) {lsandoleakcheck()}}func init() {exithook.Gosched = Goschedexithook.Goid = func() uint64 { return getg().goid }exithook.Throw = throw}func runExitHooks( int) {exithook.Run()}// start forcegc helper goroutinefunc init() {go forcegchelper()}func forcegchelper() {forcegc.g = getg()lockInit(&forcegc.lock, lockRankForcegc)for {lock(&forcegc.lock)if forcegc.idle.Load() {throw("forcegc: phase error")}forcegc.idle.Store(true)goparkunlock(&forcegc.lock, waitReasonForceGCIdle, traceBlockSystemGoroutine, 1)// this goroutine is explicitly resumed by sysmonif debug.gctrace > 0 {println("GC forced")}// Time-triggered, fully concurrent.gcStart(gcTrigger{kind: gcTriggerTime, now: nanotime()})}}// Gosched yields the processor, allowing other goroutines to run. It does not// suspend the current goroutine, so execution resumes automatically.////go:nosplitfunc () {checkTimeouts()mcall(gosched_m)}// goschedguarded yields the processor like gosched, but also checks// for forbidden states and opts out of the yield in those cases.////go:nosplitfunc goschedguarded() {mcall(goschedguarded_m)}// goschedIfBusy yields the processor like gosched, but only does so if// there are no idle Ps or if we're on the only P and there's nothing in// the run queue. In both cases, there is freely available idle time.////go:nosplitfunc goschedIfBusy() {:= getg()// Call gosched if gp.preempt is set; we may be in a tight loop that// doesn't otherwise yield.if !.preempt && sched.npidle.Load() > 0 {return}mcall(gosched_m)}// Puts the current goroutine into a waiting state and calls unlockf on the// system stack.//// If unlockf returns false, the goroutine is resumed.//// unlockf must not access this G's stack, as it may be moved between// the call to gopark and the call to unlockf.//// Note that because unlockf is called after putting the G into a waiting// state, the G may have already been readied by the time unlockf is called// unless there is external synchronization preventing the G from being// readied. If unlockf returns false, it must guarantee that the G cannot be// externally readied.//// Reason explains why the goroutine has been parked. It is displayed in stack// traces and heap dumps. Reasons should be unique and descriptive. Do not// re-use reasons, add new ones.//// gopark should be an internal detail,// but widely used packages access it using linkname.// Notable members of the hall of shame include:// - gvisor.dev/gvisor// - github.com/sagernet/gvisor//// Do not remove or change the type signature.// See go.dev/issue/67401.////go:linkname goparkfunc gopark( func(*g, unsafe.Pointer) bool, unsafe.Pointer, waitReason, traceBlockReason, int) {if != waitReasonSleep {checkTimeouts() // timeouts may expire while two goroutines keep the scheduler busy}:= acquirem():= .curg:= readgstatus()if != _Grunning && != _Gscanrunning {throw("gopark: bad g status")}.waitlock =.waitunlockf =.waitreason =.waitTraceBlockReason =.waitTraceSkip =releasem()// can't do anything that might move the G between Ms here.mcall(park_m)}// Puts the current goroutine into a waiting state and unlocks the lock.// The goroutine can be made runnable again by calling goready(gp).func goparkunlock( *mutex, waitReason, traceBlockReason, int) {gopark(parkunlock_c, unsafe.Pointer(), , , )}// goready should be an internal detail,// but widely used packages access it using linkname.// Notable members of the hall of shame include:// - gvisor.dev/gvisor// - github.com/sagernet/gvisor//// Do not remove or change the type signature.// See go.dev/issue/67401.////go:linkname goreadyfunc goready( *g, int) {systemstack(func() {ready(, , true)})}//go:nosplitfunc acquireSudog() *sudog {// Delicate dance: the semaphore implementation calls// acquireSudog, acquireSudog calls new(sudog),// new calls malloc, malloc can call the garbage collector,// and the garbage collector calls the semaphore implementation// in stopTheWorld.// Break the cycle by doing acquirem/releasem around new(sudog).// The acquirem/releasem increments m.locks during new(sudog),// which keeps the garbage collector from being invoked.:= acquirem():= .p.ptr()if len(.sudogcache) == 0 {lock(&sched.sudoglock)// First, try to grab a batch from central cache.for len(.sudogcache) < cap(.sudogcache)/2 && sched.sudogcache != nil {:= sched.sudogcachesched.sudogcache = .next.next = nil.sudogcache = append(.sudogcache, )}unlock(&sched.sudoglock)// If the central cache is empty, allocate a new one.if len(.sudogcache) == 0 {.sudogcache = append(.sudogcache, new(sudog))}}:= len(.sudogcache):= .sudogcache[-1].sudogcache[-1] = nil.sudogcache = .sudogcache[:-1]if .elem != nil {throw("acquireSudog: found s.elem != nil in cache")}releasem()return}//go:nosplitfunc releaseSudog( *sudog) {if .elem != nil {throw("runtime: sudog with non-nil elem")}if .isSelect {throw("runtime: sudog with non-false isSelect")}if .next != nil {throw("runtime: sudog with non-nil next")}if .prev != nil {throw("runtime: sudog with non-nil prev")}if .waitlink != nil {throw("runtime: sudog with non-nil waitlink")}if .c != nil {throw("runtime: sudog with non-nil c")}:= getg()if .param != nil {throw("runtime: releaseSudog with non-nil gp.param")}:= acquirem() // avoid rescheduling to another P:= .p.ptr()if len(.sudogcache) == cap(.sudogcache) {// Transfer half of local cache to the central cache.var , *sudogfor len(.sudogcache) > cap(.sudogcache)/2 {:= len(.sudogcache):= .sudogcache[-1].sudogcache[-1] = nil.sudogcache = .sudogcache[:-1]if == nil {=} else {.next =}=}lock(&sched.sudoglock).next = sched.sudogcachesched.sudogcache =unlock(&sched.sudoglock)}.sudogcache = append(.sudogcache, )releasem()}// called from assembly.func badmcall( func(*g)) {throw("runtime: mcall called on m->g0 stack")}func badmcall2( func(*g)) {throw("runtime: mcall function returned")}func badreflectcall() {panic(plainError("arg size to reflect.call more than 1GB"))}//go:nosplit//go:nowritebarrierrecfunc badmorestackg0() {if !crashStackImplemented {writeErrStr("fatal: morestack on g0\n")return}:= getg()switchToCrashStack(func() {print("runtime: morestack on g0, stack [", hex(.stack.lo), " ", hex(.stack.hi), "], sp=", hex(.sched.sp), ", called from\n").m.traceback = 2 // include pc and sp in stack tracetraceback1(.sched.pc, .sched.sp, .sched.lr, , 0)print("\n")throw("morestack on g0")})}//go:nosplit//go:nowritebarrierrecfunc badmorestackgsignal() {writeErrStr("fatal: morestack on gsignal\n")}//go:nosplitfunc badctxt() {throw("ctxt != 0")}// gcrash is a fake g that can be used when crashing due to bad// stack conditions.var gcrash gvar crashingG atomic.Pointer[g]// Switch to crashstack and call fn, with special handling of// concurrent and recursive cases.//// Nosplit as it is called in a bad stack condition (we know// morestack would fail).////go:nosplit//go:nowritebarrierrecfunc switchToCrashStack( func()) {:= getg()if crashingG.CompareAndSwapNoWB(nil, ) {switchToCrashStack0() // should never returnabort()}if crashingG.Load() == {// recursive crashing. too bad.writeErrStr("fatal: recursive switchToCrashStack\n")abort()}// Another g is crashing. Give it some time, hopefully it will finish traceback.usleep_no_g(100)writeErrStr("fatal: concurrent switchToCrashStack\n")abort()}// Disable crash stack on Windows for now. Apparently, throwing an exception// on a non-system-allocated crash stack causes EXCEPTION_STACK_OVERFLOW and// hangs the process (see issue 63938).const crashStackImplemented = GOOS != "windows"//go:noescapefunc switchToCrashStack0( func()) // in assemblyfunc lockedOSThread() bool {:= getg()return .lockedm != 0 && .m.lockedg != 0}var (// allgs contains all Gs ever created (including dead Gs), and thus// never shrinks.//// Access via the slice is protected by allglock or stop-the-world.// Readers that cannot take the lock may (carefully!) use the atomic// variables below.allglock mutexallgs []*g// allglen and allgptr are atomic variables that contain len(allgs) and// &allgs[0] respectively. Proper ordering depends on totally-ordered// loads and stores. Writes are protected by allglock.//// allgptr is updated before allglen. Readers should read allglen// before allgptr to ensure that allglen is always <= len(allgptr). New// Gs appended during the race can be missed. For a consistent view of// all Gs, allglock must be held.//// allgptr copies should always be stored as a concrete type or// unsafe.Pointer, not uintptr, to ensure that GC can still reach it// even if it points to a stale array.allglen uintptrallgptr **g)func allgadd( *g) {if readgstatus() == _Gidle {throw("allgadd: bad status Gidle")}lock(&allglock)allgs = append(allgs, )if &allgs[0] != allgptr {atomicstorep(unsafe.Pointer(&allgptr), unsafe.Pointer(&allgs[0]))}atomic.Storeuintptr(&allglen, uintptr(len(allgs)))unlock(&allglock)}// allGsSnapshot returns a snapshot of the slice of all Gs.//// The world must be stopped or allglock must be held.func allGsSnapshot() []*g {assertWorldStoppedOrLockHeld(&allglock)// Because the world is stopped or allglock is held, allgadd// cannot happen concurrently with this. allgs grows// monotonically and existing entries never change, so we can// simply return a copy of the slice header. For added safety,// we trim everything past len because that can still change.return allgs[:len(allgs):len(allgs)]}// atomicAllG returns &allgs[0] and len(allgs) for use with atomicAllGIndex.func atomicAllG() (**g, uintptr) {:= atomic.Loaduintptr(&allglen):= (**g)(atomic.Loadp(unsafe.Pointer(&allgptr)))return ,}// atomicAllGIndex returns ptr[i] with the allgptr returned from atomicAllG.func atomicAllGIndex( **g, uintptr) *g {return *(**g)(add(unsafe.Pointer(), *goarch.PtrSize))}// forEachG calls fn on every G from allgs.//// forEachG takes a lock to exclude concurrent addition of new Gs.func forEachG( func( *g)) {lock(&allglock)for , := range allgs {()}unlock(&allglock)}// forEachGRace calls fn on every G from allgs.//// forEachGRace avoids locking, but does not exclude addition of new Gs during// execution, which may be missed.func forEachGRace( func( *g)) {, := atomicAllG()for := uintptr(0); < ; ++ {:= atomicAllGIndex(, )()}return}const (// Number of goroutine ids to grab from sched.goidgen to local per-P cache at once.// 16 seems to provide enough amortization, but other than that it's mostly arbitrary number._GoidCacheBatch = 16)// cpuinit sets up CPU feature flags and calls internal/cpu.Initialize. env should be the complete// value of the GODEBUG environment variable.func cpuinit( string) {switch GOOS {case "aix", "darwin", "ios", "dragonfly", "freebsd", "netbsd", "openbsd", "illumos", "solaris", "linux":cpu.DebugOptions = true}cpu.Initialize()// Support cpu feature variables are used in code generated by the compiler// to guard execution of instructions that can not be assumed to be always supported.switch GOARCH {case "386", "amd64":x86HasPOPCNT = cpu.X86.HasPOPCNTx86HasSSE41 = cpu.X86.HasSSE41x86HasFMA = cpu.X86.HasFMAcase "arm":armHasVFPv4 = cpu.ARM.HasVFPv4case "arm64":arm64HasATOMICS = cpu.ARM64.HasATOMICScase "loong64":loong64HasLAMCAS = cpu.Loong64.HasLAMCASloong64HasLAM_BH = cpu.Loong64.HasLAM_BHloong64HasLSX = cpu.Loong64.HasLSXcase "riscv64":riscv64HasZbb = cpu.RISCV64.HasZbb}}// getGodebugEarly extracts the environment variable GODEBUG from the environment on// Unix-like operating systems and returns it. This function exists to extract GODEBUG// early before much of the runtime is initialized.func getGodebugEarly() string {const = "GODEBUG="var stringswitch GOOS {case "aix", "darwin", "ios", "dragonfly", "freebsd", "netbsd", "openbsd", "illumos", "solaris", "linux":// Similar to goenv_unix but extracts the environment value for// GODEBUG directly.// TODO(moehrmann): remove when general goenvs() can be called before cpuinit():= int32(0)for argv_index(argv, argc+1+) != nil {++}for := int32(0); < ; ++ {:= argv_index(argv, argc+1+):= unsafe.String(, findnull())if stringslite.HasPrefix(, ) {= gostring()[len():]break}}}return}// The bootstrap sequence is://// call osinit// call schedinit// make & queue new G// call runtime·mstart//// The new G calls runtime·main.func schedinit() {lockInit(&sched.lock, lockRankSched)lockInit(&sched.sysmonlock, lockRankSysmon)lockInit(&sched.deferlock, lockRankDefer)lockInit(&sched.sudoglock, lockRankSudog)lockInit(&deadlock, lockRankDeadlock)lockInit(&paniclk, lockRankPanic)lockInit(&allglock, lockRankAllg)lockInit(&allpLock, lockRankAllp)lockInit(&reflectOffs.lock, lockRankReflectOffs)lockInit(&finlock, lockRankFin)lockInit(&cpuprof.lock, lockRankCpuprof)lockInit(&computeMaxProcsLock, lockRankComputeMaxProcs)allocmLock.init(lockRankAllocmR, lockRankAllocmRInternal, lockRankAllocmW)execLock.init(lockRankExecR, lockRankExecRInternal, lockRankExecW)traceLockInit()// Enforce that this lock is always a leaf lock.// All of this lock's critical sections should be// extremely short.lockInit(&memstats.heapStats.noPLock, lockRankLeafRank)lockVerifyMSize()// raceinit must be the first call to race detector.// In particular, it must be done before mallocinit below calls racemapshadow.:= getg()if raceenabled {.racectx, raceprocctx0 = raceinit()}sched.maxmcount = 10000crashFD.Store(^uintptr(0))// The world starts stopped.worldStopped()ticks.init() // run as early as possiblemoduledataverify()stackinit()mallocinit():= getGodebugEarly()cpuinit() // must run before alginitrandinit() // must run before alginit, mcommoninitalginit() // maps, hash, rand must not be used before this callmcommoninit(.m, -1)modulesinit() // provides activeModulestypelinksinit() // uses maps, activeModulesitabsinit() // uses activeModulesstkobjinit() // must run before GC startssigsave(&.m.sigmask)initSigmask = .m.sigmaskgoargs()goenvs()secure()checkfds()parsedebugvars()gcinit()// Allocate stack space that can be used when crashing due to bad stack// conditions, e.g. morestack on g0.gcrash.stack = stackalloc(16384)gcrash.stackguard0 = gcrash.stack.lo + 1000gcrash.stackguard1 = gcrash.stack.lo + 1000// if disableMemoryProfiling is set, update MemProfileRate to 0 to turn off memprofile.// Note: parsedebugvars may update MemProfileRate, but when disableMemoryProfiling is// set to true by the linker, it means that nothing is consuming the profile, it is// safe to set MemProfileRate to 0.if disableMemoryProfiling {MemProfileRate = 0}// mcommoninit runs before parsedebugvars, so init profstacks again.mProfStackInit(.m)defaultGOMAXPROCSInit()lock(&sched.lock)sched.lastpoll.Store(nanotime())var int32if , := strconv.Atoi32(gogetenv("GOMAXPROCS")); && > 0 {=sched.customGOMAXPROCS = true} else {// Use numCPUStartup for initial GOMAXPROCS for two reasons://// 1. We just computed it in osinit, recomputing is (minorly) wasteful.//// 2. More importantly, if debug.containermaxprocs == 0 &&// debug.updatemaxprocs == 0, we want to guarantee that// runtime.GOMAXPROCS(0) always equals runtime.NumCPU (which is// just numCPUStartup).= defaultGOMAXPROCS(numCPUStartup)}if procresize() != nil {throw("unknown runnable goroutine during bootstrap")}unlock(&sched.lock)// World is effectively started now, as P's can run.worldStarted()if buildVersion == "" {// Condition should never trigger. This code just serves// to ensure runtime·buildVersion is kept in the resulting binary.buildVersion = "unknown"}if len(modinfo) == 1 {// Condition should never trigger. This code just serves// to ensure runtime·modinfo is kept in the resulting binary.modinfo = ""}}func dumpgstatus( *g) {:= getg()print("runtime: gp: gp=", , ", goid=", .goid, ", gp->atomicstatus=", readgstatus(), "\n")print("runtime: getg: g=", , ", goid=", .goid, ", g->atomicstatus=", readgstatus(), "\n")}// sched.lock must be held.func checkmcount() {assertLockHeld(&sched.lock)// Exclude extra M's, which are used for cgocallback from threads// created in C.//// The purpose of the SetMaxThreads limit is to avoid accidental fork// bomb from something like millions of goroutines blocking on system// calls, causing the runtime to create millions of threads. By// definition, this isn't a problem for threads created in C, so we// exclude them from the limit. See https://go.dev/issue/60004.:= mcount() - int32(extraMInUse.Load()) - int32(extraMLength.Load())if > sched.maxmcount {print("runtime: program exceeds ", sched.maxmcount, "-thread limit\n")throw("thread exhaustion")}}// mReserveID returns the next ID to use for a new m. This new m is immediately// considered 'running' by checkdead.//// sched.lock must be held.func mReserveID() int64 {assertLockHeld(&sched.lock)if sched.mnext+1 < sched.mnext {throw("runtime: thread ID overflow")}:= sched.mnextsched.mnext++checkmcount()return}// Pre-allocated ID may be passed as 'id', or omitted by passing -1.func mcommoninit( *m, int64) {:= getg()// g0 stack won't make sense for user (and is not necessary unwindable).if != .m.g0 {callers(1, .createstack[:])}lock(&sched.lock)if >= 0 {.id =} else {.id = mReserveID()}mrandinit()mpreinit()if .gsignal != nil {.gsignal.stackguard1 = .gsignal.stack.lo + stackGuard}// Add to allm so garbage collector doesn't free g->m// when it is just in a register or thread-local storage..alllink = allm// NumCgoCall() and others iterate over allm w/o schedlock,// so we need to publish it safely.atomicstorep(unsafe.Pointer(&allm), unsafe.Pointer())unlock(&sched.lock)// Allocate memory to hold a cgo traceback if the cgo call crashes.if iscgo || GOOS == "solaris" || GOOS == "illumos" || GOOS == "windows" {.cgoCallers = new(cgoCallers)}mProfStackInit()}// mProfStackInit is used to eagerly initialize stack trace buffers for// profiling. Lazy allocation would have to deal with reentrancy issues in// malloc and runtime locks for mLockProfile.// TODO(mknyszek): Implement lazy allocation if this becomes a problem.func mProfStackInit( *m) {if debug.profstackdepth == 0 {// debug.profstack is set to 0 by the user, or we're being called from// schedinit before parsedebugvars.return}.profStack = makeProfStackFP().mLockProfile.stack = makeProfStackFP()}// makeProfStackFP creates a buffer large enough to hold a maximum-sized stack// trace as well as any additional frames needed for frame pointer unwinding// with delayed inline expansion.func makeProfStackFP() []uintptr {// The "1" term is to account for the first stack entry being// taken up by a "skip" sentinel value for profilers which// defer inline frame expansion until the profile is reported.// The "maxSkip" term is for frame pointer unwinding, where we// want to end up with debug.profstackdebth frames but will discard// some "physical" frames to account for skipping.return make([]uintptr, 1+maxSkip+debug.profstackdepth)}// makeProfStack returns a buffer large enough to hold a maximum-sized stack// trace.func makeProfStack() []uintptr { return make([]uintptr, debug.profstackdepth) }//go:linkname pprof_makeProfStackfunc pprof_makeProfStack() []uintptr { return makeProfStack() }func ( *m) () {.spinning = truesched.nmspinning.Add(1)sched.needspinning.Store(0)}// Take a snapshot of allp, for use after dropping the P.//// Must be called with a P, but the returned slice may be used after dropping// the P. The M holds a reference on the snapshot to keep the backing array// alive.////go:yeswritebarrierrecfunc ( *m) () []*p {.allpSnapshot = allpreturn .allpSnapshot}// Clear the saved allp snapshot. Should be called as soon as the snapshot is// no longer required.//// Must be called after reacquiring a P, as it requires a write barrier.////go:yeswritebarrierrecfunc ( *m) () {.allpSnapshot = nil}func ( *m) () bool {return .ncgo > 0 || .isextra}const (// osHasLowResTimer indicates that the platform's internal timer system has a low resolution,// typically on the order of 1 ms or more.osHasLowResTimer = GOOS == "windows" || GOOS == "openbsd" || GOOS == "netbsd"// osHasLowResClockInt is osHasLowResClock but in integer form, so it can be used to create// constants conditionally.osHasLowResClockInt = goos.IsWindows// osHasLowResClock indicates that timestamps produced by nanotime on the platform have a// low resolution, typically on the order of 1 ms or more.osHasLowResClock = osHasLowResClockInt > 0)// Mark gp ready to run.func ready( *g, int, bool) {:= readgstatus()// Mark runnable.:= acquirem() // disable preemption because it can be holding p in a local varif &^_Gscan != _Gwaiting {dumpgstatus()throw("bad g->status in ready")}// status is Gwaiting or Gscanwaiting, make Grunnable and put on runq:= traceAcquire()casgstatus(, _Gwaiting, _Grunnable)if .ok() {.GoUnpark(, )traceRelease()}runqput(.p.ptr(), , )wakep()releasem()}// freezeStopWait is a large value that freezetheworld sets// sched.stopwait to in order to request that all Gs permanently stop.const freezeStopWait = 0x7fffffff// freezing is set to non-zero if the runtime is trying to freeze the// world.var freezing atomic.Bool// Similar to stopTheWorld but best-effort and can be called several times.// There is no reverse operation, used during crashing.// This function must not lock any mutexes.func freezetheworld() {freezing.Store(true)if debug.dontfreezetheworld > 0 {// Don't prempt Ps to stop goroutines. That will perturb// scheduler state, making debugging more difficult. Instead,// allow goroutines to continue execution.//// fatalpanic will tracebackothers to trace all goroutines. It// is unsafe to trace a running goroutine, so tracebackothers// will skip running goroutines. That is OK and expected, we// expect users of dontfreezetheworld to use core files anyway.//// However, allowing the scheduler to continue running free// introduces a race: a goroutine may be stopped when// tracebackothers checks its status, and then start running// later when we are in the middle of traceback, potentially// causing a crash.//// To mitigate this, when an M naturally enters the scheduler,// schedule checks if freezing is set and if so stops// execution. This guarantees that while Gs can transition from// running to stopped, they can never transition from stopped// to running.//// The sleep here allows racing Ms that missed freezing and are// about to run a G to complete the transition to running// before we start traceback.usleep(1000)return}// stopwait and preemption requests can be lost// due to races with concurrently executing threads,// so try several timesfor := 0; < 5; ++ {// this should tell the scheduler to not start any new goroutinessched.stopwait = freezeStopWaitsched.gcwaiting.Store(true)// this should stop running goroutinesif !preemptall() {break // no running goroutines}usleep(1000)}// to be sureusleep(1000)preemptall()usleep(1000)}// All reads and writes of g's status go through readgstatus, casgstatus// castogscanstatus, casfrom_Gscanstatus.////go:nosplitfunc readgstatus( *g) uint32 {return .atomicstatus.Load()}// The Gscanstatuses are acting like locks and this releases them.// If it proves to be a performance hit we should be able to make these// simple atomic stores but for now we are going to throw if// we see an inconsistent state.func casfrom_Gscanstatus( *g, , uint32) {:= false// Check that transition is valid.switch {default:print("runtime: casfrom_Gscanstatus bad oldval gp=", , ", oldval=", hex(), ", newval=", hex(), "\n")dumpgstatus()throw("casfrom_Gscanstatus:top gp->status is not in scan state")case _Gscanrunnable,_Gscanwaiting,_Gscanrunning,_Gscansyscall,_Gscanpreempted:if == &^_Gscan {= .atomicstatus.CompareAndSwap(, )}}if ! {print("runtime: casfrom_Gscanstatus failed gp=", , ", oldval=", hex(), ", newval=", hex(), "\n")dumpgstatus()throw("casfrom_Gscanstatus: gp->status is not in scan state")}releaseLockRankAndM(lockRankGscan)}// This will return false if the gp is not in the expected status and the cas fails.// This acts like a lock acquire while the casfromgstatus acts like a lock release.func castogscanstatus( *g, , uint32) bool {switch {case _Grunnable,_Grunning,_Gwaiting,_Gsyscall:if == |_Gscan {:= .atomicstatus.CompareAndSwap(, )if {acquireLockRankAndM(lockRankGscan)}return}}print("runtime: castogscanstatus oldval=", hex(), " newval=", hex(), "\n")throw("castogscanstatus")panic("not reached")}// casgstatusAlwaysTrack is a debug flag that causes casgstatus to always track// various latencies on every transition instead of sampling them.var casgstatusAlwaysTrack = false// If asked to move to or from a Gscanstatus this will throw. Use the castogscanstatus// and casfrom_Gscanstatus instead.// casgstatus will loop if the g->atomicstatus is in a Gscan status until the routine that// put it in the Gscan state is finished.////go:nosplitfunc casgstatus( *g, , uint32) {if (&_Gscan != 0) || (&_Gscan != 0) || == {systemstack(func() {// Call on the systemstack to prevent print and throw from counting// against the nosplit stack reservation.print("runtime: casgstatus: oldval=", hex(), " newval=", hex(), "\n")throw("casgstatus: bad incoming values")})}lockWithRankMayAcquire(nil, lockRankGscan)// See https://golang.org/cl/21503 for justification of the yield delay.const = 5 * 1000var int64// loop if gp->atomicstatus is in a scan state giving// GC time to finish and change the state to oldval.for := 0; !.atomicstatus.CompareAndSwap(, ); ++ {if == _Gwaiting && .atomicstatus.Load() == _Grunnable {systemstack(func() {// Call on the systemstack to prevent throw from counting// against the nosplit stack reservation.throw("casgstatus: waiting for Gwaiting but is Grunnable")})}if == 0 {= nanotime() +}if nanotime() < {for := 0; < 10 && .atomicstatus.Load() != ; ++ {procyield(1)}} else {osyield()= nanotime() + /2}}if .bubble != nil {systemstack(func() {.bubble.changegstatus(, , )})}if == _Grunning {// Track every gTrackingPeriod time a goroutine transitions out of running.if casgstatusAlwaysTrack || .trackingSeq%gTrackingPeriod == 0 {.tracking = true}.trackingSeq++}if !.tracking {return}// Handle various kinds of tracking.//// Currently:// - Time spent in runnable.// - Time spent blocked on a sync.Mutex or sync.RWMutex.switch {case _Grunnable:// We transitioned out of runnable, so measure how much// time we spent in this state and add it to// runnableTime.:= nanotime().runnableTime += - .trackingStamp.trackingStamp = 0case _Gwaiting:if !.waitreason.isMutexWait() {// Not blocking on a lock.break}// Blocking on a lock, measure it. Note that because we're// sampling, we have to multiply by our sampling period to get// a more representative estimate of the absolute value.// gTrackingPeriod also represents an accurate sampling period// because we can only enter this state from _Grunning.:= nanotime()sched.totalMutexWaitTime.Add(( - .trackingStamp) * gTrackingPeriod).trackingStamp = 0}switch {case _Gwaiting:if !.waitreason.isMutexWait() {// Not blocking on a lock.break}// Blocking on a lock. Write down the timestamp.:= nanotime().trackingStamp =case _Grunnable:// We just transitioned into runnable, so record what// time that happened.:= nanotime().trackingStamp =case _Grunning:// We're transitioning into running, so turn off// tracking and record how much time we spent in// runnable..tracking = falsesched.timeToRun.record(.runnableTime).runnableTime = 0}}// casGToWaiting transitions gp from old to _Gwaiting, and sets the wait reason.//// Use this over casgstatus when possible to ensure that a waitreason is set.func casGToWaiting( *g, uint32, waitReason) {// Set the wait reason before calling casgstatus, because casgstatus will use it..waitreason =casgstatus(, , _Gwaiting)}// casGToWaitingForSuspendG transitions gp from old to _Gwaiting, and sets the wait reason.// The wait reason must be a valid isWaitingForSuspendG wait reason.//// Use this over casgstatus when possible to ensure that a waitreason is set.func casGToWaitingForSuspendG( *g, uint32, waitReason) {if !.isWaitingForSuspendG() {throw("casGToWaitingForSuspendG with non-isWaitingForSuspendG wait reason")}casGToWaiting(, , )}// casGToPreemptScan transitions gp from _Grunning to _Gscan|_Gpreempted.//// TODO(austin): This is the only status operation that both changes// the status and locks the _Gscan bit. Rethink this.func casGToPreemptScan( *g, , uint32) {if != _Grunning || != _Gscan|_Gpreempted {throw("bad g transition")}acquireLockRankAndM(lockRankGscan)for !.atomicstatus.CompareAndSwap(_Grunning, _Gscan|_Gpreempted) {}// We never notify gp.bubble that the goroutine state has moved// from _Grunning to _Gpreempted. We call bubble.changegstatus// after status changes happen, but doing so here would violate the// ordering between the gscan and synctest locks. The bubble doesn't// distinguish between _Grunning and _Gpreempted anyway, so not// notifying it is fine.}// casGFromPreempted attempts to transition gp from _Gpreempted to// _Gwaiting. If successful, the caller is responsible for// re-scheduling gp.func casGFromPreempted( *g, , uint32) bool {if != _Gpreempted || != _Gwaiting {throw("bad g transition")}.waitreason = waitReasonPreemptedif !.atomicstatus.CompareAndSwap(_Gpreempted, _Gwaiting) {return false}if := .bubble; != nil {.changegstatus(, _Gpreempted, _Gwaiting)}return true}// stwReason is an enumeration of reasons the world is stopping.type stwReason uint8// Reasons to stop-the-world.//// Avoid reusing reasons and add new ones instead.const (stwUnknown stwReason = iota // "unknown"stwGCMarkTerm // "GC mark termination"stwGCSweepTerm // "GC sweep termination"stwWriteHeapDump // "write heap dump"stwGoroutineProfile // "goroutine profile"stwGoroutineProfileCleanup // "goroutine profile cleanup"stwAllGoroutinesStack // "all goroutines stack trace"stwReadMemStats // "read mem stats"stwAllThreadsSyscall // "AllThreadsSyscall"stwGOMAXPROCS // "GOMAXPROCS"stwStartTrace // "start trace"stwStopTrace // "stop trace"stwForTestCountPagesInUse // "CountPagesInUse (test)"stwForTestReadMetricsSlow // "ReadMetricsSlow (test)"stwForTestReadMemStatsSlow // "ReadMemStatsSlow (test)"stwForTestPageCachePagesLeaked // "PageCachePagesLeaked (test)"stwForTestResetDebugLog // "ResetDebugLog (test)")func ( stwReason) () string {return stwReasonStrings[]}func ( stwReason) () bool {return == stwGCMarkTerm || == stwGCSweepTerm}// If you add to this list, also add it to src/internal/trace/parser.go.// If you change the values of any of the stw* constants, bump the trace// version number and make a copy of this.var stwReasonStrings = [...]string{stwUnknown: "unknown",stwGCMarkTerm: "GC mark termination",stwGCSweepTerm: "GC sweep termination",stwWriteHeapDump: "write heap dump",stwGoroutineProfile: "goroutine profile",stwGoroutineProfileCleanup: "goroutine profile cleanup",stwAllGoroutinesStack: "all goroutines stack trace",stwReadMemStats: "read mem stats",stwAllThreadsSyscall: "AllThreadsSyscall",stwGOMAXPROCS: "GOMAXPROCS",stwStartTrace: "start trace",stwStopTrace: "stop trace",stwForTestCountPagesInUse: "CountPagesInUse (test)",stwForTestReadMetricsSlow: "ReadMetricsSlow (test)",stwForTestReadMemStatsSlow: "ReadMemStatsSlow (test)",stwForTestPageCachePagesLeaked: "PageCachePagesLeaked (test)",stwForTestResetDebugLog: "ResetDebugLog (test)",}// worldStop provides context from the stop-the-world required by the// start-the-world.type worldStop struct {reason stwReasonstartedStopping int64finishedStopping int64stoppingCPUTime int64}// Temporary variable for stopTheWorld, when it can't write to the stack.//// Protected by worldsema.var stopTheWorldContext worldStop// stopTheWorld stops all P's from executing goroutines, interrupting// all goroutines at GC safe points and records reason as the reason// for the stop. On return, only the current goroutine's P is running.// stopTheWorld must not be called from a system stack and the caller// must not hold worldsema. The caller must call startTheWorld when// other P's should resume execution.//// stopTheWorld is safe for multiple goroutines to call at the// same time. Each will execute its own stop, and the stops will// be serialized.//// This is also used by routines that do stack dumps. If the system is// in panic or being exited, this may not reliably stop all// goroutines.//// Returns the STW context. When starting the world, this context must be// passed to startTheWorld.func stopTheWorld( stwReason) worldStop {semacquire(&worldsema):= getg().m.preemptoff = .String()systemstack(func() {stopTheWorldContext = stopTheWorldWithSema() // avoid write to stack})return stopTheWorldContext}// startTheWorld undoes the effects of stopTheWorld.//// w must be the worldStop returned by stopTheWorld.func startTheWorld( worldStop) {systemstack(func() { startTheWorldWithSema(0, ) })// worldsema must be held over startTheWorldWithSema to ensure// gomaxprocs cannot change while worldsema is held.//// Release worldsema with direct handoff to the next waiter, but// acquirem so that semrelease1 doesn't try to yield our time.//// Otherwise if e.g. ReadMemStats is being called in a loop,// it might stomp on other attempts to stop the world, such as// for starting or ending GC. The operation this blocks is// so heavy-weight that we should just try to be as fair as// possible here.//// We don't want to just allow us to get preempted between now// and releasing the semaphore because then we keep everyone// (including, for example, GCs) waiting longer.:= acquirem().preemptoff = ""semrelease1(&worldsema, true, 0)releasem()}// stopTheWorldGC has the same effect as stopTheWorld, but blocks// until the GC is not running. It also blocks a GC from starting// until startTheWorldGC is called.func stopTheWorldGC( stwReason) worldStop {semacquire(&gcsema)return stopTheWorld()}// startTheWorldGC undoes the effects of stopTheWorldGC.//// w must be the worldStop returned by stopTheWorld.func startTheWorldGC( worldStop) {startTheWorld()semrelease(&gcsema)}// Holding worldsema grants an M the right to try to stop the world.var worldsema uint32 = 1// Holding gcsema grants the M the right to block a GC, and blocks// until the current GC is done. In particular, it prevents gomaxprocs// from changing concurrently.//// TODO(mknyszek): Once gomaxprocs and the execution tracer can handle// being changed/enabled during a GC, remove this.var gcsema uint32 = 1// stopTheWorldWithSema is the core implementation of stopTheWorld.// The caller is responsible for acquiring worldsema and disabling// preemption first and then should stopTheWorldWithSema on the system// stack://// semacquire(&worldsema, 0)// m.preemptoff = "reason"// var stw worldStop// systemstack(func() {// stw = stopTheWorldWithSema(reason)// })//// When finished, the caller must either call startTheWorld or undo// these three operations separately://// m.preemptoff = ""// systemstack(func() {// now = startTheWorldWithSema(stw)// })// semrelease(&worldsema)//// It is allowed to acquire worldsema once and then execute multiple// startTheWorldWithSema/stopTheWorldWithSema pairs.// Other P's are able to execute between successive calls to// startTheWorldWithSema and stopTheWorldWithSema.// Holding worldsema causes any other goroutines invoking// stopTheWorld to block.//// Returns the STW context. When starting the world, this context must be// passed to startTheWorldWithSema.////go:systemstackfunc stopTheWorldWithSema( stwReason) worldStop {// Mark the goroutine which called stopTheWorld preemptible so its// stack may be scanned by the GC or observed by the execution tracer.//// This lets a mark worker scan us or the execution tracer take our// stack while we try to stop the world since otherwise we could get// in a mutual preemption deadlock.//// We must not modify anything on the G stack because a stack shrink// may occur, now that we switched to _Gwaiting, specifically if we're// doing this during the mark phase (mark termination excepted, since// we know that stack scanning is done by that point). A stack shrink// is otherwise OK though because in order to return from this function// (and to leave the system stack) we must have preempted all// goroutines, including any attempting to scan our stack, in which// case, any stack shrinking will have already completed by the time we// exit.//// N.B. The execution tracer is not aware of this status transition and// andles it specially based on the wait reason.casGToWaitingForSuspendG(getg().m.curg, _Grunning, waitReasonStoppingTheWorld):= traceAcquire()if .ok() {.STWStart()traceRelease()}:= getg()// If we hold a lock, then we won't be able to stop another M// that is blocked trying to acquire the lock.if .m.locks > 0 {throw("stopTheWorld: holding locks")}lock(&sched.lock):= nanotime() // exclude time waiting for sched.lock from start and total time metrics.sched.stopwait = gomaxprocssched.gcwaiting.Store(true)preemptall()// stop current P.m.p.ptr().status = _Pgcstop // Pgcstop is only diagnostic..m.p.ptr().gcStopTime =sched.stopwait--// try to retake all P's in Psyscall status= traceAcquire()for , := range allp {:= .statusif == _Psyscall && atomic.Cas(&.status, , _Pgcstop) {if .ok() {.ProcSteal(, false)}.syscalltick++.gcStopTime = nanotime()sched.stopwait--}}if .ok() {traceRelease()}// stop idle P's:= nanotime()for {, := pidleget()if == nil {break}.status = _Pgcstop.gcStopTime = nanotime()sched.stopwait--}:= sched.stopwait > 0unlock(&sched.lock)// wait for remaining P's to stop voluntarilyif {for {// wait for 100us, then try to re-preempt in case of any racesif notetsleep(&sched.stopnote, 100*1000) {noteclear(&sched.stopnote)break}preemptall()}}:= nanotime():= -if .isGC() {sched.stwStoppingTimeGC.record()} else {sched.stwStoppingTimeOther.record()}// Double-check we actually stopped everything, and all the invariants hold.// Also accumulate all the time spent by each P in _Pgcstop up to the point// where everything was stopped. This will be accumulated into the total pause// CPU time by the caller.:= int64(0):= ""if sched.stopwait != 0 {= "stopTheWorld: not stopped (stopwait != 0)"} else {for , := range allp {if .status != _Pgcstop {= "stopTheWorld: not stopped (status != _Pgcstop)"}if .gcStopTime == 0 && == "" {= "stopTheWorld: broken CPU time accounting"}+= - .gcStopTime.gcStopTime = 0}}if freezing.Load() {// Some other thread is panicking. This can cause the// sanity checks above to fail if the panic happens in// the signal handler on a stopped thread. Either way,// we should halt this thread.lock(&deadlock)lock(&deadlock)}if != "" {throw()}worldStopped()// Switch back to _Grunning, now that the world is stopped.casgstatus(getg().m.curg, _Gwaiting, _Grunning)return worldStop{reason: ,startedStopping: ,finishedStopping: ,stoppingCPUTime: ,}}// reason is the same STW reason passed to stopTheWorld. start is the start// time returned by stopTheWorld.//// now is the current time; prefer to pass 0 to capture a fresh timestamp.//// stattTheWorldWithSema returns now.func startTheWorldWithSema( int64, worldStop) int64 {assertWorldStopped():= acquirem() // disable preemption because it can be holding p in a local varif netpollinited() {, := netpoll(0) // non-blockinginjectglist(&)netpollAdjustWaiters()}lock(&sched.lock):= gomaxprocsif newprocs != 0 {= newprocsnewprocs = 0}:= procresize()sched.gcwaiting.Store(false)if sched.sysmonwait.Load() {sched.sysmonwait.Store(false)notewakeup(&sched.sysmonnote)}unlock(&sched.lock)worldStarted()for != nil {:== .link.ptr()if .m != 0 {:= .m.ptr().m = 0if .nextp != 0 {throw("startTheWorld: inconsistent mp->nextp")}.nextp.set()notewakeup(&.park)} else {// Start M to run P. Do not start another M below.newm(nil, , -1)}}// Capture start-the-world time before doing clean-up tasks.if == 0 {= nanotime()}:= - .startedStoppingif .reason.isGC() {sched.stwTotalTimeGC.record()} else {sched.stwTotalTimeOther.record()}:= traceAcquire()if .ok() {.STWDone()traceRelease()}// Wakeup an additional proc in case we have excessive runnable goroutines// in local queues or in the global queue. If we don't, the proc will park itself.// If we have lots of excessive work, resetspinning will unpark additional procs as necessary.wakep()releasem()return}// usesLibcall indicates whether this runtime performs system calls// via libcall.func usesLibcall() bool {switch GOOS {case "aix", "darwin", "illumos", "ios", "solaris", "windows":return truecase "openbsd":return GOARCH != "mips64"}return false}// mStackIsSystemAllocated indicates whether this runtime starts on a// system-allocated stack.func mStackIsSystemAllocated() bool {switch GOOS {case "aix", "darwin", "plan9", "illumos", "ios", "solaris", "windows":return truecase "openbsd":return GOARCH != "mips64"}return false}// mstart is the entry-point for new Ms.// It is written in assembly, uses ABI0, is marked TOPFRAME, and calls mstart0.func mstart()// mstart0 is the Go entry-point for new Ms.// This must not split the stack because we may not even have stack// bounds set up yet.//// May run during STW (because it doesn't have a P yet), so write// barriers are not allowed.////go:nosplit//go:nowritebarrierrecfunc mstart0() {:= getg():= .stack.lo == 0if {// Initialize stack bounds from system stack.// Cgo may have left stack size in stack.hi.// minit may update the stack bounds.//// Note: these bounds may not be very accurate.// We set hi to &size, but there are things above// it. The 1024 is supposed to compensate this,// but is somewhat arbitrary.:= .stack.hiif == 0 {= 16384 * sys.StackGuardMultiplier}.stack.hi = uintptr(noescape(unsafe.Pointer(&))).stack.lo = .stack.hi - + 1024}// Initialize stack guard so that we can start calling regular// Go code..stackguard0 = .stack.lo + stackGuard// This is the g0, so we can also call go:systemstack// functions, which check stackguard1..stackguard1 = .stackguard0mstart1()// Exit this thread.if mStackIsSystemAllocated() {// Windows, Solaris, illumos, Darwin, AIX and Plan 9 always system-allocate// the stack, but put it in gp.stack before mstart,// so the logic above hasn't set osStack yet.= true}mexit()}// The go:noinline is to guarantee the sys.GetCallerPC/sys.GetCallerSP below are safe,// so that we can set up g0.sched to return to the call of mstart1 above.////go:noinlinefunc mstart1() {:= getg()if != .m.g0 {throw("bad runtime·mstart")}// Set up m.g0.sched as a label returning to just// after the mstart1 call in mstart0 above, for use by goexit0 and mcall.// We're never coming back to mstart1 after we call schedule,// so other calls can reuse the current frame.// And goexit0 does a gogo that needs to return from mstart1// and let mstart0 exit the thread..sched.g = guintptr(unsafe.Pointer()).sched.pc = sys.GetCallerPC().sched.sp = sys.GetCallerSP()asminit()minit()// Install signal handlers; after minit so that minit can// prepare the thread to be able to handle the signals.if .m == &m0 {mstartm0()}if debug.dataindependenttiming == 1 {sys.EnableDIT()}if := .m.mstartfn; != nil {()}if .m != &m0 {acquirep(.m.nextp.ptr()).m.nextp = 0}schedule()}// mstartm0 implements part of mstart1 that only runs on the m0.//// Write barriers are allowed here because we know the GC can't be// running yet, so they'll be no-ops.////go:yeswritebarrierrecfunc mstartm0() {// Create an extra M for callbacks on threads not created by Go.// An extra M is also needed on Windows for callbacks created by// syscall.NewCallback. See issue #6751 for details.if (iscgo || GOOS == "windows") && !cgoHasExtraM {cgoHasExtraM = truenewextram()}initsig(false)}// mPark causes a thread to park itself, returning once woken.////go:nosplitfunc mPark() {:= getg()notesleep(&.m.park)noteclear(&.m.park)}// mexit tears down and exits the current thread.//// Don't call this directly to exit the thread, since it must run at// the top of the thread stack. Instead, use gogo(&gp.m.g0.sched) to// unwind the stack to the point that exits the thread.//// It is entered with m.p != nil, so write barriers are allowed. It// will release the P before exiting.////go:yeswritebarrierrecfunc mexit( bool) {:= getg().mif == &m0 {// This is the main thread. Just wedge it.//// On Linux, exiting the main thread puts the process// into a non-waitable zombie state. On Plan 9,// exiting the main thread unblocks wait even though// other threads are still running. On Solaris we can// neither exitThread nor return from mstart. Other// bad things probably happen on other platforms.//// We could try to clean up this M more before wedging// it, but that complicates signal handling.handoffp(releasep())lock(&sched.lock)sched.nmfreed++checkdead()unlock(&sched.lock)mPark()throw("locked m0 woke up")}sigblock(true)unminit()// Free the gsignal stack.if .gsignal != nil {stackfree(.gsignal.stack)if valgrindenabled {valgrindDeregisterStack(.gsignal.valgrindStackID).gsignal.valgrindStackID = 0}// On some platforms, when calling into VDSO (e.g. nanotime)// we store our g on the gsignal stack, if there is one.// Now the stack is freed, unlink it from the m, so we// won't write to it when calling VDSO code..gsignal = nil}// Free vgetrandom state.vgetrandomDestroy()// Remove m from allm.lock(&sched.lock)for := &allm; * != nil; = &(*).alllink {if * == {* = .alllinkgoto}}throw("m not found in allm"):// Events must not be traced after this point.// Delay reaping m until it's done with the stack.//// Put mp on the free list, though it will not be reaped while freeWait// is freeMWait. mp is no longer reachable via allm, so even if it is// on an OS stack, we must keep a reference to mp alive so that the GC// doesn't free mp while we are still using it.//// Note that the free list must not be linked through alllink because// some functions walk allm without locking, so may be using alllink.//// N.B. It's important that the M appears on the free list simultaneously// with it being removed so that the tracer can find it..freeWait.Store(freeMWait).freelink = sched.freemsched.freem =unlock(&sched.lock)atomic.Xadd64(&ncgocall, int64(.ncgocall))sched.totalRuntimeLockWaitTime.Add(.mLockProfile.waitTime.Load())// Release the P.handoffp(releasep())// After this point we must not have write barriers.// Invoke the deadlock detector. This must happen after// handoffp because it may have started a new M to take our// P's work.lock(&sched.lock)sched.nmfreed++checkdead()unlock(&sched.lock)if GOOS == "darwin" || GOOS == "ios" {// Make sure pendingPreemptSignals is correct when an M exits.// For #41702.if .signalPending.Load() != 0 {pendingPreemptSignals.Add(-1)}}// Destroy all allocated resources. After this is called, we may no// longer take any locks.mdestroy()if {// No more uses of mp, so it is safe to drop the reference..freeWait.Store(freeMRef)// Return from mstart and let the system thread// library free the g0 stack and terminate the thread.return}// mstart is the thread's entry point, so there's nothing to// return to. Exit the thread directly. exitThread will clear// m.freeWait when it's done with the stack and the m can be// reaped.exitThread(&.freeWait)}// forEachP calls fn(p) for every P p when p reaches a GC safe point.// If a P is currently executing code, this will bring the P to a GC// safe point and execute fn on that P. If the P is not executing code// (it is idle or in a syscall), this will call fn(p) directly while// preventing the P from exiting its state. This does not ensure that// fn will run on every CPU executing Go code, but it acts as a global// memory barrier. GC uses this as a "ragged barrier."//// The caller must hold worldsema. fn must not refer to any// part of the current goroutine's stack, since the GC may move it.func forEachP( waitReason, func(*p)) {systemstack(func() {:= getg().m.curg// Mark the user stack as preemptible so that it may be scanned// by the GC or observed by the execution tracer. Otherwise, our// attempt to force all P's to a safepoint could result in a// deadlock as we attempt to preempt a goroutine that's trying// to preempt us (e.g. for a stack scan).//// We must not modify anything on the G stack because a stack shrink// may occur. A stack shrink is otherwise OK though because in order// to return from this function (and to leave the system stack) we// must have preempted all goroutines, including any attempting// to scan our stack, in which case, any stack shrinking will// have already completed by the time we exit.//// N.B. The execution tracer is not aware of this status// transition and handles it specially based on the// wait reason.casGToWaitingForSuspendG(, _Grunning, )forEachPInternal()casgstatus(, _Gwaiting, _Grunning)})}// forEachPInternal calls fn(p) for every P p when p reaches a GC safe point.// It is the internal implementation of forEachP.//// The caller must hold worldsema and either must ensure that a GC is not// running (otherwise this may deadlock with the GC trying to preempt this P)// or it must leave its goroutine in a preemptible state before it switches// to the systemstack. Due to these restrictions, prefer forEachP when possible.////go:systemstackfunc forEachPInternal( func(*p)) {:= acquirem():= getg().m.p.ptr()lock(&sched.lock)if sched.safePointWait != 0 {throw("forEachP: sched.safePointWait != 0")}sched.safePointWait = gomaxprocs - 1sched.safePointFn =// Ask all Ps to run the safe point function.for , := range allp {if != {atomic.Store(&.runSafePointFn, 1)}}preemptall()// Any P entering _Pidle or _Psyscall from now on will observe// p.runSafePointFn == 1 and will call runSafePointFn when// changing its status to _Pidle/_Psyscall.// Run safe point function for all idle Ps. sched.pidle will// not change because we hold sched.lock.for := sched.pidle.ptr(); != nil; = .link.ptr() {if atomic.Cas(&.runSafePointFn, 1, 0) {()sched.safePointWait--}}:= sched.safePointWait > 0unlock(&sched.lock)// Run fn for the current P.()// Force Ps currently in _Psyscall into _Pidle and hand them// off to induce safe point function execution.for , := range allp {:= .status// We need to be fine-grained about tracing here, since handoffp// might call into the tracer, and the tracer is non-reentrant.:= traceAcquire()if == _Psyscall && .runSafePointFn == 1 && atomic.Cas(&.status, , _Pidle) {if .ok() {// It's important that we traceRelease before we call handoffp, which may also traceAcquire..ProcSteal(, false)traceRelease()}.syscalltick++handoffp()} else if .ok() {traceRelease()}}// Wait for remaining Ps to run fn.if {for {// Wait for 100us, then try to re-preempt in// case of any races.//// Requires system stack.if notetsleep(&sched.safePointNote, 100*1000) {noteclear(&sched.safePointNote)break}preemptall()}}if sched.safePointWait != 0 {throw("forEachP: not done")}for , := range allp {if .runSafePointFn != 0 {throw("forEachP: P did not run fn")}}lock(&sched.lock)sched.safePointFn = nilunlock(&sched.lock)releasem()}// runSafePointFn runs the safe point function, if any, for this P.// This should be called like//// if getg().m.p.runSafePointFn != 0 {// runSafePointFn()// }//// runSafePointFn must be checked on any transition in to _Pidle or// _Psyscall to avoid a race where forEachP sees that the P is running// just before the P goes into _Pidle/_Psyscall and neither forEachP// nor the P run the safe-point function.func runSafePointFn() {:= getg().m.p.ptr()// Resolve the race between forEachP running the safe-point// function on this P's behalf and this P running the// safe-point function directly.if !atomic.Cas(&.runSafePointFn, 1, 0) {return}sched.safePointFn()lock(&sched.lock)sched.safePointWait--if sched.safePointWait == 0 {notewakeup(&sched.safePointNote)}unlock(&sched.lock)}// When running with cgo, we call _cgo_thread_start// to start threads for us so that we can play nicely with// foreign code.var cgoThreadStart unsafe.Pointertype cgothreadstart struct {g guintptrtls *uint64fn unsafe.Pointer}// Allocate a new m unassociated with any thread.// Can use p for allocation context if needed.// fn is recorded as the new m's m.mstartfn.// id is optional pre-allocated m ID. Omit by passing -1.//// This function is allowed to have write barriers even if the caller// isn't because it borrows pp.////go:yeswritebarrierrecfunc allocm( *p, func(), int64) *m {allocmLock.rlock()// The caller owns pp, but we may borrow (i.e., acquirep) it. We must// disable preemption to ensure it is not stolen, which would make the// caller lose ownership.acquirem():= getg()if .m.p == 0 {acquirep() // temporarily borrow p for mallocs in this function}// Release the free M list. We need to do this somewhere and// this may free up a stack we can use.if sched.freem != nil {lock(&sched.lock)var *mfor := sched.freem; != nil; {// Wait for freeWait to indicate that freem's stack is unused.:= .freeWait.Load()if == freeMWait {:= .freelink.freelink ===continue}// Drop any remaining trace resources.// Ms can continue to emit events all the way until wait != freeMWait,// so it's only safe to call traceThreadDestroy at this point.if traceEnabled() || traceShuttingDown() {traceThreadDestroy()}// Free the stack if needed. For freeMRef, there is// nothing to do except drop freem from the sched.freem// list.if == freeMStack {// stackfree must be on the system stack, but allocm is// reachable off the system stack transitively from// startm.systemstack(func() {stackfree(.g0.stack)if valgrindenabled {valgrindDeregisterStack(.g0.valgrindStackID).g0.valgrindStackID = 0}})}= .freelink}sched.freem =unlock(&sched.lock)}:= &new(mPadded).m.mstartfn =mcommoninit(, )// In case of cgo or Solaris or illumos or Darwin, pthread_create will make us a stack.// Windows and Plan 9 will layout sched stack on OS stack.if iscgo || mStackIsSystemAllocated() {.g0 = malg(-1)} else {.g0 = malg(16384 * sys.StackGuardMultiplier)}.g0.m =if == .m.p.ptr() {releasep()}releasem(.m)allocmLock.runlock()return}// needm is called when a cgo callback happens on a// thread without an m (a thread not created by Go).// In this case, needm is expected to find an m to use// and return with m, g initialized correctly.// Since m and g are not set now (likely nil, but see below)// needm is limited in what routines it can call. In particular// it can only call nosplit functions (textflag 7) and cannot// do any scheduling that requires an m.//// In order to avoid needing heavy lifting here, we adopt// the following strategy: there is a stack of available m's// that can be stolen. Using compare-and-swap// to pop from the stack has ABA races, so we simulate// a lock by doing an exchange (via Casuintptr) to steal the stack// head and replace the top pointer with MLOCKED (1).// This serves as a simple spin lock that we can use even// without an m. The thread that locks the stack in this way// unlocks the stack by storing a valid stack head pointer.//// In order to make sure that there is always an m structure// available to be stolen, we maintain the invariant that there// is always one more than needed. At the beginning of the// program (if cgo is in use) the list is seeded with a single m.// If needm finds that it has taken the last m off the list, its job// is - once it has installed its own m so that it can do things like// allocate memory - to create a spare m and put it on the list.//// Each of these extra m's also has a g0 and a curg that are// pressed into service as the scheduling stack and current// goroutine for the duration of the cgo callback.//// It calls dropm to put the m back on the list,// 1. when the callback is done with the m in non-pthread platforms,// 2. or when the C thread exiting on pthread platforms.//// The signal argument indicates whether we're called from a signal// handler.////go:nosplitfunc needm( bool) {if (iscgo || GOOS == "windows") && !cgoHasExtraM {// Can happen if C/C++ code calls Go from a global ctor.// Can also happen on Windows if a global ctor uses a// callback created by syscall.NewCallback. See issue #6751// for details.//// Can not throw, because scheduler is not initialized yet.writeErrStr("fatal error: cgo callback before cgo call\n")exit(1)}// Save and block signals before getting an M.// The signal handler may call needm itself,// and we must avoid a deadlock. Also, once g is installed,// any incoming signals will try to execute,// but we won't have the sigaltstack settings and other data// set up appropriately until the end of minit, which will// unblock the signals. This is the same dance as when// starting a new m to run Go code via newosproc.var sigsetsigsave(&)sigblock(false)// getExtraM is safe here because of the invariant above,// that the extra list always contains or will soon contain// at least one m., := getExtraM()// Set needextram when we've just emptied the list,// so that the eventual call into cgocallbackg will// allocate a new m for the extra list. We delay the// allocation until then so that it can be done// after exitsyscall makes sure it is okay to be// running at all (that is, there's no garbage collection// running right now)..needextram =// Store the original signal mask for use by minit..sigmask =// Install TLS on some platforms (previously setg// would do this if necessary).osSetupTLS()// Install g (= m->g0) and set the stack bounds// to match the current stack.setg(.g0):= sys.GetCallerSP()callbackUpdateSystemStack(, , )// Should mark we are already in Go now.// Otherwise, we may call needm again when we get a signal, before cgocallbackg1,// which means the extram list may be empty, that will cause a deadlock..isExtraInC = false// Initialize this thread to use the m.asminit()minit()// Emit a trace event for this dead -> syscall transition,// but only if we're not in a signal handler.//// N.B. the tracer can run on a bare M just fine, we just have// to make sure to do this before setg(nil) and unminit.var traceLockerif ! {= traceAcquire()}// mp.curg is now a real goroutine.casgstatus(.curg, _Gdead, _Gsyscall)sched.ngsys.Add(-1)if ! {if .ok() {.GoCreateSyscall(.curg)traceRelease()}}.isExtraInSig =}// Acquire an extra m and bind it to the C thread when a pthread key has been created.////go:nosplitfunc needAndBindM() {needm(false)if _cgo_pthread_key_created != nil && *(*uintptr)(_cgo_pthread_key_created) != 0 {cgoBindM()}}// newextram allocates m's and puts them on the extra list.// It is called with a working local m, so that it can do things// like call schedlock and allocate.func newextram() {:= extraMWaiters.Swap(0)if > 0 {for := uint32(0); < ; ++ {oneNewExtraM()}} else if extraMLength.Load() == 0 {// Make sure there is at least one extra M.oneNewExtraM()}}// oneNewExtraM allocates an m and puts it on the extra list.func oneNewExtraM() {// Create extra goroutine locked to extra m.// The goroutine is the context in which the cgo callback will run.// The sched.pc will never be returned to, but setting it to// goexit makes clear to the traceback routines where// the goroutine stack ends.:= allocm(nil, nil, -1):= malg(4096).sched.pc = abi.FuncPCABI0(goexit) + sys.PCQuantum.sched.sp = .stack.hi.sched.sp -= 4 * goarch.PtrSize // extra space in case of reads slightly beyond frame.sched.lr = 0.sched.g = guintptr(unsafe.Pointer()).syscallpc = .sched.pc.syscallsp = .sched.sp.stktopsp = .sched.sp// malg returns status as _Gidle. Change to _Gdead before// adding to allg where GC can see it. We use _Gdead to hide// this from tracebacks and stack scans since it isn't a// "real" goroutine until needm grabs it.casgstatus(, _Gidle, _Gdead).m =.curg =.isextra = true// mark we are in C by default..isExtraInC = true.lockedInt++.lockedg.set().lockedm.set().goid = sched.goidgen.Add(1)if raceenabled {.racectx = racegostart(abi.FuncPCABIInternal(newextram) + sys.PCQuantum)}// put on allg for garbage collectorallgadd()// gp is now on the allg list, but we don't want it to be// counted by gcount. It would be more "proper" to increment// sched.ngfree, but that requires locking. Incrementing ngsys// has the same effect.sched.ngsys.Add(1)// Add m to the extra list.addExtraM()}// dropm puts the current m back onto the extra list.//// 1. On systems without pthreads, like Windows// dropm is called when a cgo callback has called needm but is now// done with the callback and returning back into the non-Go thread.//// The main expense here is the call to signalstack to release the// m's signal stack, and then the call to needm on the next callback// from this thread. It is tempting to try to save the m for next time,// which would eliminate both these costs, but there might not be// a next time: the current thread (which Go does not control) might exit.// If we saved the m for that thread, there would be an m leak each time// such a thread exited. Instead, we acquire and release an m on each// call. These should typically not be scheduling operations, just a few// atomics, so the cost should be small.//// 2. On systems with pthreads// dropm is called while a non-Go thread is exiting.// We allocate a pthread per-thread variable using pthread_key_create,// to register a thread-exit-time destructor.// And store the g into a thread-specific value associated with the pthread key,// when first return back to C.// So that the destructor would invoke dropm while the non-Go thread is exiting.// This is much faster since it avoids expensive signal-related syscalls.//// This always runs without a P, so //go:nowritebarrierrec is required.//// This may run with a different stack than was recorded in g0 (there is no// call to callbackUpdateSystemStack prior to dropm), so this must be// //go:nosplit to avoid the stack bounds check.////go:nowritebarrierrec//go:nosplitfunc dropm() {// Clear m and g, and return m to the extra list.// After the call to setg we can only call nosplit functions// with no pointer manipulation.:= getg().m// Emit a trace event for this syscall -> dead transition.//// N.B. the tracer can run on a bare M just fine, we just have// to make sure to do this before setg(nil) and unminit.var traceLockerif !.isExtraInSig {= traceAcquire()}// Return mp.curg to dead state.casgstatus(.curg, _Gsyscall, _Gdead).curg.preemptStop = falsesched.ngsys.Add(1)if !.isExtraInSig {if .ok() {.GoDestroySyscall()traceRelease()}}// Trash syscalltick so that it doesn't line up with mp.old.syscalltick anymore.//// In the new tracer, we model needm and dropm and a goroutine being created and// destroyed respectively. The m then might get reused with a different procid but// still with a reference to oldp, and still with the same syscalltick. The next// time a G is "created" in needm, it'll return and quietly reacquire its P from a// different m with a different procid, which will confuse the trace parser. By// trashing syscalltick, we ensure that it'll appear as if we lost the P to the// tracer parser and that we just reacquired it.//// Trash the value by decrementing because that gets us as far away from the value// the syscall exit code expects as possible. Setting to zero is risky because// syscalltick could already be zero (and in fact, is initialized to zero)..syscalltick--// Reset trace state unconditionally. This goroutine is being 'destroyed'// from the perspective of the tracer..curg.trace.reset()// Flush all the M's buffers. This is necessary because the M might// be used on a different thread with a different procid, so we have// to make sure we don't write into the same buffer.if traceEnabled() || traceShuttingDown() {// Acquire sched.lock across thread destruction. One of the invariants of the tracer// is that a thread cannot disappear from the tracer's view (allm or freem) without// it noticing, so it requires that sched.lock be held over traceThreadDestroy.//// This isn't strictly necessary in this case, because this thread never leaves allm,// but the critical section is short and dropm is rare on pthread platforms, so just// take the lock and play it safe. traceThreadDestroy also asserts that the lock is held.lock(&sched.lock)traceThreadDestroy()unlock(&sched.lock)}.isExtraInSig = false// Block signals before unminit.// Unminit unregisters the signal handling stack (but needs g on some systems).// Setg(nil) clears g, which is the signal handler's cue not to run Go handlers.// It's important not to try to handle a signal between those two steps.:= .sigmasksigblock(false)unminit()setg(nil)// Clear g0 stack bounds to ensure that needm always refreshes the// bounds when reusing this M.:= .g0.stack.hi = 0.stack.lo = 0.stackguard0 = 0.stackguard1 = 0.g0StackAccurate = falseputExtraM()msigrestore()}// bindm store the g0 of the current m into a thread-specific value.//// We allocate a pthread per-thread variable using pthread_key_create,// to register a thread-exit-time destructor.// We are here setting the thread-specific value of the pthread key, to enable the destructor.// So that the pthread_key_destructor would dropm while the C thread is exiting.//// And the saved g will be used in pthread_key_destructor,// since the g stored in the TLS by Go might be cleared in some platforms,// before the destructor invoked, so, we restore g by the stored g, before dropm.//// We store g0 instead of m, to make the assembly code simpler,// since we need to restore g0 in runtime.cgocallback.//// On systems without pthreads, like Windows, bindm shouldn't be used.//// NOTE: this always runs without a P, so, nowritebarrierrec required.////go:nosplit//go:nowritebarrierrecfunc cgoBindM() {if GOOS == "windows" || GOOS == "plan9" {fatal("bindm in unexpected GOOS")}:= getg()if .m.g0 != {fatal("the current g is not g0")}if _cgo_bindm != nil {asmcgocall(_cgo_bindm, unsafe.Pointer())}}// A helper function for EnsureDropM.//// getm should be an internal detail,// but widely used packages access it using linkname.// Notable members of the hall of shame include:// - fortio.org/log//// Do not remove or change the type signature.// See go.dev/issue/67401.////go:linkname getmfunc getm() uintptr {return uintptr(unsafe.Pointer(getg().m))}var (// Locking linked list of extra M's, via mp.schedlink. Must be accessed// only via lockextra/unlockextra.//// Can't be atomic.Pointer[m] because we use an invalid pointer as a// "locked" sentinel value. M's on this list remain visible to the GC// because their mp.curg is on allgs.extraM atomic.Uintptr// Number of M's in the extraM list.extraMLength atomic.Uint32// Number of waiters in lockextra.extraMWaiters atomic.Uint32// Number of extra M's in use by threads.extraMInUse atomic.Uint32)// lockextra locks the extra list and returns the list head.// The caller must unlock the list by storing a new list head// to extram. If nilokay is true, then lockextra will// return a nil list head if that's what it finds. If nilokay is false,// lockextra will keep waiting until the list head is no longer nil.////go:nosplitfunc lockextra( bool) *m {const = 1:= falsefor {:= extraM.Load()if == {osyield_no_g()continue}if == 0 && ! {if ! {// Add 1 to the number of threads// waiting for an M.// This is cleared by newextram.extraMWaiters.Add(1)= true}usleep_no_g(1)continue}if extraM.CompareAndSwap(, ) {return (*m)(unsafe.Pointer())}osyield_no_g()continue}}//go:nosplitfunc unlockextra( *m, int32) {extraMLength.Add()extraM.Store(uintptr(unsafe.Pointer()))}// Return an M from the extra M list. Returns last == true if the list becomes// empty because of this call.//// Spins waiting for an extra M, so caller must ensure that the list always// contains or will soon contain at least one M.////go:nosplitfunc getExtraM() ( *m, bool) {= lockextra(false)extraMInUse.Add(1)unlockextra(.schedlink.ptr(), -1)return , .schedlink.ptr() == nil}// Returns an extra M back to the list. mp must be from getExtraM. Newly// allocated M's should use addExtraM.////go:nosplitfunc putExtraM( *m) {extraMInUse.Add(-1)addExtraM()}// Adds a newly allocated M to the extra M list.////go:nosplitfunc addExtraM( *m) {:= lockextra(true).schedlink.set()unlockextra(, 1)}var (// allocmLock is locked for read when creating new Ms in allocm and their// addition to allm. Thus acquiring this lock for write blocks the// creation of new Ms.allocmLock rwmutex// execLock serializes exec and clone to avoid bugs or unspecified// behaviour around exec'ing while creating/destroying threads. See// issue #19546.execLock rwmutex)// These errors are reported (via writeErrStr) by some OS-specific// versions of newosproc and newosproc0.const (failthreadcreate = "runtime: failed to create new OS thread\n"failallocatestack = "runtime: failed to allocate stack for the new OS thread\n")// newmHandoff contains a list of m structures that need new OS threads.// This is used by newm in situations where newm itself can't safely// start an OS thread.var newmHandoff struct {lock mutex// newm points to a list of M structures that need new OS// threads. The list is linked through m.schedlink.newm muintptr// waiting indicates that wake needs to be notified when an m// is put on the list.waiting boolwake note// haveTemplateThread indicates that the templateThread has// been started. This is not protected by lock. Use cas to set// to 1.haveTemplateThread uint32}// Create a new m. It will start off with a call to fn, or else the scheduler.// fn needs to be static and not a heap allocated closure.// May run with m.p==nil, so write barriers are not allowed.//// id is optional pre-allocated m ID. Omit by passing -1.////go:nowritebarrierrecfunc newm( func(), *p, int64) {// allocm adds a new M to allm, but they do not start until created by// the OS in newm1 or the template thread.//// doAllThreadsSyscall requires that every M in allm will eventually// start and be signal-able, even with a STW.//// Disable preemption here until we start the thread to ensure that// newm is not preempted between allocm and starting the new thread,// ensuring that anything added to allm is guaranteed to eventually// start.acquirem():= allocm(, , ).nextp.set().sigmask = initSigmaskif := getg(); != nil && .m != nil && (.m.lockedExt != 0 || .m.incgo) && GOOS != "plan9" {// We're on a locked M or a thread that may have been// started by C. The kernel state of this thread may// be strange (the user may have locked it for that// purpose). We don't want to clone that into another// thread. Instead, ask a known-good thread to create// the thread for us.//// This is disabled on Plan 9. See golang.org/issue/22227.//// TODO: This may be unnecessary on Windows, which// doesn't model thread creation off fork.lock(&newmHandoff.lock)if newmHandoff.haveTemplateThread == 0 {throw("on a locked thread with no template thread")}.schedlink = newmHandoff.newmnewmHandoff.newm.set()if newmHandoff.waiting {newmHandoff.waiting = falsenotewakeup(&newmHandoff.wake)}unlock(&newmHandoff.lock)// The M has not started yet, but the template thread does not// participate in STW, so it will always process queued Ms and// it is safe to releasem.releasem(getg().m)return}newm1()releasem(getg().m)}func newm1( *m) {if iscgo {var cgothreadstartif _cgo_thread_start == nil {throw("_cgo_thread_start missing")}.g.set(.g0).tls = (*uint64)(unsafe.Pointer(&.tls[0])).fn = unsafe.Pointer(abi.FuncPCABI0(mstart))if msanenabled {msanwrite(unsafe.Pointer(&), unsafe.Sizeof())}if asanenabled {asanwrite(unsafe.Pointer(&), unsafe.Sizeof())}execLock.rlock() // Prevent process clone.asmcgocall(_cgo_thread_start, unsafe.Pointer(&))execLock.runlock()return}execLock.rlock() // Prevent process clone.newosproc()execLock.runlock()}// startTemplateThread starts the template thread if it is not already// running.//// The calling thread must itself be in a known-good state.func startTemplateThread() {if GOARCH == "wasm" { // no threads on wasm yetreturn}// Disable preemption to guarantee that the template thread will be// created before a park once haveTemplateThread is set.:= acquirem()if !atomic.Cas(&newmHandoff.haveTemplateThread, 0, 1) {releasem()return}newm(templateThread, nil, -1)releasem()}// templateThread is a thread in a known-good state that exists solely// to start new threads in known-good states when the calling thread// may not be in a good state.//// Many programs never need this, so templateThread is started lazily// when we first enter a state that might lead to running on a thread// in an unknown state.//// templateThread runs on an M without a P, so it must not have write// barriers.////go:nowritebarrierrecfunc templateThread() {lock(&sched.lock)sched.nmsys++checkdead()unlock(&sched.lock)for {lock(&newmHandoff.lock)for newmHandoff.newm != 0 {:= newmHandoff.newm.ptr()newmHandoff.newm = 0unlock(&newmHandoff.lock)for != nil {:= .schedlink.ptr().schedlink = 0newm1()=}lock(&newmHandoff.lock)}newmHandoff.waiting = truenoteclear(&newmHandoff.wake)unlock(&newmHandoff.lock)notesleep(&newmHandoff.wake)}}// Stops execution of the current m until new work is available.// Returns with acquired P.func stopm() {:= getg()if .m.locks != 0 {throw("stopm holding locks")}if .m.p != 0 {throw("stopm holding p")}if .m.spinning {throw("stopm spinning")}lock(&sched.lock)mput(.m)unlock(&sched.lock)mPark()acquirep(.m.nextp.ptr()).m.nextp = 0}func mspinning() {// startm's caller incremented nmspinning. Set the new M's spinning.getg().m.spinning = true}// Schedules some M to run the p (creates an M if necessary).// If p==nil, tries to get an idle P, if no idle P's does nothing.// May run with m.p==nil, so write barriers are not allowed.// If spinning is set, the caller has incremented nmspinning and must provide a// P. startm will set m.spinning in the newly started M.//// Callers passing a non-nil P must call from a non-preemptible context. See// comment on acquirem below.//// Argument lockheld indicates whether the caller already acquired the// scheduler lock. Callers holding the lock when making the call must pass// true. The lock might be temporarily dropped, but will be reacquired before// returning.//// Must not have write barriers because this may be called without a P.////go:nowritebarrierrecfunc startm( *p, , bool) {// Disable preemption.//// Every owned P must have an owner that will eventually stop it in the// event of a GC stop request. startm takes transient ownership of a P// (either from argument or pidleget below) and transfers ownership to// a started M, which will be responsible for performing the stop.//// Preemption must be disabled during this transient ownership,// otherwise the P this is running on may enter GC stop while still// holding the transient P, leaving that P in limbo and deadlocking the// STW.//// Callers passing a non-nil P must already be in non-preemptible// context, otherwise such preemption could occur on function entry to// startm. Callers passing a nil P may be preemptible, so we must// disable preemption before acquiring a P from pidleget below.:= acquirem()if ! {lock(&sched.lock)}if == nil {if {// TODO(prattmic): All remaining calls to this function// with _p_ == nil could be cleaned up to find a P// before calling startm.throw("startm: P required for spinning=true")}, _ = pidleget(0)if == nil {if ! {unlock(&sched.lock)}releasem()return}}:= mget()if == nil {// No M is available, we must drop sched.lock and call newm.// However, we already own a P to assign to the M.//// Once sched.lock is released, another G (e.g., in a syscall),// could find no idle P while checkdead finds a runnable G but// no running M's because this new M hasn't started yet, thus// throwing in an apparent deadlock.// This apparent deadlock is possible when startm is called// from sysmon, which doesn't count as a running M.//// Avoid this situation by pre-allocating the ID for the new M,// thus marking it as 'running' before we drop sched.lock. This// new M will eventually run the scheduler to execute any// queued G's.:= mReserveID()unlock(&sched.lock)var func()if {// The caller incremented nmspinning, so set m.spinning in the new M.= mspinning}newm(, , )if {lock(&sched.lock)}// Ownership transfer of pp committed by start in newm.// Preemption is now safe.releasem()return}if ! {unlock(&sched.lock)}if .spinning {throw("startm: m is spinning")}if .nextp != 0 {throw("startm: m has p")}if && !runqempty() {throw("startm: p has runnable gs")}// The caller incremented nmspinning, so set m.spinning in the new M..spinning =.nextp.set()notewakeup(&.park)// Ownership transfer of pp committed by wakeup. Preemption is now// safe.releasem()}// Hands off P from syscall or locked M.// Always runs without a P, so write barriers are not allowed.////go:nowritebarrierrecfunc handoffp( *p) {// handoffp must start an M in any situation where// findrunnable would return a G to run on pp.// if it has local work, start it straight awayif !runqempty() || !sched.runq.empty() {startm(, false, false)return}// if there's trace work to do, start it straight awayif (traceEnabled() || traceShuttingDown()) && traceReaderAvailable() != nil {startm(, false, false)return}// if it has GC work, start it straight awayif gcBlackenEnabled != 0 && gcMarkWorkAvailable() {startm(, false, false)return}// no local work, check that there are no spinning/idle M's,// otherwise our help is not requiredif sched.nmspinning.Load()+sched.npidle.Load() == 0 && sched.nmspinning.CompareAndSwap(0, 1) { // TODO: fast atomicsched.needspinning.Store(0)startm(, true, false)return}lock(&sched.lock)if sched.gcwaiting.Load() {.status = _Pgcstop.gcStopTime = nanotime()sched.stopwait--if sched.stopwait == 0 {notewakeup(&sched.stopnote)}unlock(&sched.lock)return}if .runSafePointFn != 0 && atomic.Cas(&.runSafePointFn, 1, 0) {sched.safePointFn()sched.safePointWait--if sched.safePointWait == 0 {notewakeup(&sched.safePointNote)}}if !sched.runq.empty() {unlock(&sched.lock)startm(, false, false)return}// If this is the last running P and nobody is polling network,// need to wakeup another M to poll network.if sched.npidle.Load() == gomaxprocs-1 && sched.lastpoll.Load() != 0 {unlock(&sched.lock)startm(, false, false)return}// The scheduler lock cannot be held when calling wakeNetPoller below// because wakeNetPoller may call wakep which may call startm.:= .timers.wakeTime()pidleput(, 0)unlock(&sched.lock)if != 0 {wakeNetPoller()}}// Tries to add one more P to execute G's.// Called when a G is made runnable (newproc, ready).// Must be called with a P.//// wakep should be an internal detail,// but widely used packages access it using linkname.// Notable members of the hall of shame include:// - gvisor.dev/gvisor//// Do not remove or change the type signature.// See go.dev/issue/67401.////go:linkname wakepfunc wakep() {// Be conservative about spinning threads, only start one if none exist// already.if sched.nmspinning.Load() != 0 || !sched.nmspinning.CompareAndSwap(0, 1) {return}// Disable preemption until ownership of pp transfers to the next M in// startm. Otherwise preemption here would leave pp stuck waiting to// enter _Pgcstop.//// See preemption comment on acquirem in startm for more details.:= acquirem()var *plock(&sched.lock), _ = pidlegetSpinning(0)if == nil {if sched.nmspinning.Add(-1) < 0 {throw("wakep: negative nmspinning")}unlock(&sched.lock)releasem()return}// Since we always have a P, the race in the "No M is available"// comment in startm doesn't apply during the small window between the// unlock here and lock in startm. A checkdead in between will always// see at least one running M (ours).unlock(&sched.lock)startm(, true, false)releasem()}// Stops execution of the current m that is locked to a g until the g is runnable again.// Returns with acquired P.func stoplockedm() {:= getg()if .m.lockedg == 0 || .m.lockedg.ptr().lockedm.ptr() != .m {throw("stoplockedm: inconsistent locking")}if .m.p != 0 {// Schedule another M to run this p.:= releasep()handoffp()}incidlelocked(1)// Wait until another thread schedules lockedg again.mPark():= readgstatus(.m.lockedg.ptr())if &^_Gscan != _Grunnable {print("runtime:stoplockedm: lockedg (atomicstatus=", , ") is not Grunnable or Gscanrunnable\n")dumpgstatus(.m.lockedg.ptr())throw("stoplockedm: not runnable")}acquirep(.m.nextp.ptr()).m.nextp = 0}// Schedules the locked m to run the locked gp.// May run during STW, so write barriers are not allowed.////go:nowritebarrierrecfunc startlockedm( *g) {:= .lockedm.ptr()if == getg().m {throw("startlockedm: locked to me")}if .nextp != 0 {throw("startlockedm: m has p")}// directly handoff current P to the locked mincidlelocked(-1):= releasep().nextp.set()notewakeup(&.park)stopm()}// Stops the current m for stopTheWorld.// Returns when the world is restarted.func gcstopm() {:= getg()if !sched.gcwaiting.Load() {throw("gcstopm: not waiting for gc")}if .m.spinning {.m.spinning = false// OK to just drop nmspinning here,// startTheWorld will unpark threads as necessary.if sched.nmspinning.Add(-1) < 0 {throw("gcstopm: negative nmspinning")}}:= releasep()lock(&sched.lock).status = _Pgcstop.gcStopTime = nanotime()sched.stopwait--if sched.stopwait == 0 {notewakeup(&sched.stopnote)}unlock(&sched.lock)stopm()}// Schedules gp to run on the current M.// If inheritTime is true, gp inherits the remaining time in the// current time slice. Otherwise, it starts a new time slice.// Never returns.//// Write barriers are allowed because this is called immediately after// acquiring a P in several places.////go:yeswritebarrierrecfunc execute( *g, bool) {:= getg().mif goroutineProfile.active {// Make sure that gp has had its stack written out to the goroutine// profile, exactly as it was when the goroutine profiler first stopped// the world.tryRecordGoroutineProfile(, nil, osyield)}// Assign gp.m before entering _Grunning so running Gs have an M..curg =.m =.syncSafePoint = false // Clear the flag, which may have been set by morestack.casgstatus(, _Grunnable, _Grunning).waitsince = 0.preempt = false.stackguard0 = .stack.lo + stackGuardif ! {.p.ptr().schedtick++}// Check whether the profiler needs to be turned on or off.:= sched.profilehzif .profilehz != {setThreadCPUProfiler()}:= traceAcquire()if .ok() {.GoStart()traceRelease()}gogo(&.sched)}// Finds a runnable goroutine to execute.// Tries to steal from other P's, get g from local or global queue, poll network.// tryWakeP indicates that the returned goroutine is not normal (GC worker, trace// reader) so the caller should try to wake a P.func findRunnable() ( *g, , bool) {:= getg().m// The conditions here and in handoffp must agree: if// findrunnable would return a G to run, handoffp must start// an M.:// We may have collected an allp snapshot below. The snapshot is only// required in each loop iteration. Clear it to all GC to collect the// slice..clearAllpSnapshot():= .p.ptr()if sched.gcwaiting.Load() {gcstopm()goto}if .runSafePointFn != 0 {runSafePointFn()}// now and pollUntil are saved for work stealing later,// which may steal timers. It's important that between now// and then, nothing blocks, so these numbers remain mostly// relevant., , := .timers.check(0, nil)// Try to schedule the trace reader.if traceEnabled() || traceShuttingDown() {:= traceReader()if != nil {:= traceAcquire()casgstatus(, _Gwaiting, _Grunnable)if .ok() {.GoUnpark(, 0)traceRelease()}return , false, true}}// Try to schedule a GC worker.if gcBlackenEnabled != 0 {, := gcController.findRunnableGCWorker(, )if != nil {return , false, true}=}// Check the global runnable queue once in a while to ensure fairness.// Otherwise two goroutines can completely occupy the local runqueue// by constantly respawning each other.if .schedtick%61 == 0 && !sched.runq.empty() {lock(&sched.lock):= globrunqget()unlock(&sched.lock)if != nil {return , false, false}}// Wake up the finalizer G.if fingStatus.Load()&(fingWait|fingWake) == fingWait|fingWake {if := wakefing(); != nil {ready(, 0, true)}}// Wake up one or more cleanup Gs.if gcCleanups.needsWake() {gcCleanups.wake()}if *cgo_yield != nil {asmcgocall(*cgo_yield, nil)}// local runqif , := runqget(); != nil {return , , false}// global runqif !sched.runq.empty() {lock(&sched.lock), := globrunqgetbatch(int32(len(.runq)) / 2)unlock(&sched.lock)if != nil {if runqputbatch(, &); !.empty() {throw("Couldn't put Gs into empty local runq")}return , false, false}}// Poll network.// This netpoll is only an optimization before we resort to stealing.// We can safely skip it if there are no waiters or a thread is blocked// in netpoll already. If there is any kind of logical race with that// blocked thread (e.g. it has already returned from netpoll, but does// not set lastpoll yet), this thread will do blocking netpoll below// anyway.// We only poll from one thread at a time to avoid kernel contention// on machines with many cores.if netpollinited() && netpollAnyWaiters() && sched.lastpoll.Load() != 0 && sched.pollingNet.Swap(1) == 0 {, := netpoll(0)sched.pollingNet.Store(0)if !.empty() { // non-blocking:= .pop()injectglist(&)netpollAdjustWaiters():= traceAcquire()casgstatus(, _Gwaiting, _Grunnable)if .ok() {.GoUnpark(, 0)traceRelease()}return , false, false}}// Spinning Ms: steal work from other Ps.//// Limit the number of spinning Ms to half the number of busy Ps.// This is necessary to prevent excessive CPU consumption when// GOMAXPROCS>>1 but the program parallelism is low.if .spinning || 2*sched.nmspinning.Load() < gomaxprocs-sched.npidle.Load() {if !.spinning {.becomeSpinning()}, , , , := stealWork()if != nil {// Successfully stole.return , , false}if {// There may be new timer or GC work; restart to// discover.goto}=if != 0 && ( == 0 || < ) {// Earlier timer to wait for.=}}// We have nothing to do.//// If we're in the GC mark phase, can safely scan and blacken objects,// and have work to do, run idle-time marking rather than give up the P.if gcBlackenEnabled != 0 && gcMarkWorkAvailable() && gcController.addIdleMarkWorker() {:= (*gcBgMarkWorkerNode)(gcBgMarkWorkerPool.pop())if != nil {.gcMarkWorkerMode = gcMarkWorkerIdleMode:= .gp.ptr():= traceAcquire()casgstatus(, _Gwaiting, _Grunnable)if .ok() {.GoUnpark(, 0)traceRelease()}return , false, false}gcController.removeIdleMarkWorker()}// wasm only:// If a callback returned and no other goroutine is awake,// then wake event handler goroutine which pauses execution// until a callback was triggered., := beforeIdle(, )if != nil {:= traceAcquire()casgstatus(, _Gwaiting, _Grunnable)if .ok() {.GoUnpark(, 0)traceRelease()}return , false, false}if {goto}// Before we drop our P, make a snapshot of the allp slice,// which can change underfoot once we no longer block// safe-points. We don't need to snapshot the contents because// everything up to cap(allp) is immutable.//// We clear the snapshot from the M after return via// mp.clearAllpSnapshop (in schedule) and on each iteration of the top// loop.:= .snapshotAllp()// Also snapshot masks. Value changes are OK, but we can't allow// len to change out from under us.:= idlepMask:= timerpMask// return P and blocklock(&sched.lock)if sched.gcwaiting.Load() || .runSafePointFn != 0 {unlock(&sched.lock)goto}if !sched.runq.empty() {, := globrunqgetbatch(int32(len(.runq)) / 2)unlock(&sched.lock)if == nil {throw("global runq empty with non-zero runqsize")}if runqputbatch(, &); !.empty() {throw("Couldn't put Gs into empty local runq")}return , false, false}if !.spinning && sched.needspinning.Load() == 1 {// See "Delicate dance" comment below..becomeSpinning()unlock(&sched.lock)goto}if releasep() != {throw("findrunnable: wrong p")}= pidleput(, )unlock(&sched.lock)// Delicate dance: thread transitions from spinning to non-spinning// state, potentially concurrently with submission of new work. We must// drop nmspinning first and then check all sources again (with// #StoreLoad memory barrier in between). If we do it the other way// around, another thread can submit work after we've checked all// sources but before we drop nmspinning; as a result nobody will// unpark a thread to run the work.//// This applies to the following sources of work://// * Goroutines added to the global or a per-P run queue.// * New/modified-earlier timers on a per-P timer heap.// * Idle-priority GC work (barring golang.org/issue/19112).//// If we discover new work below, we need to restore m.spinning as a// signal for resetspinning to unpark a new worker thread (because// there can be more than one starving goroutine).//// However, if after discovering new work we also observe no idle Ps// (either here or in resetspinning), we have a problem. We may be// racing with a non-spinning M in the block above, having found no// work and preparing to release its P and park. Allowing that P to go// idle will result in loss of work conservation (idle P while there is// runnable work). This could result in complete deadlock in the// unlikely event that we discover new work (from netpoll) right as we// are racing with _all_ other Ps going idle.//// We use sched.needspinning to synchronize with non-spinning Ms going// idle. If needspinning is set when they are about to drop their P,// they abort the drop and instead become a new spinning M on our// behalf. If we are not racing and the system is truly fully loaded// then no spinning threads are required, and the next thread to// naturally become spinning will clear the flag.//// Also see "Worker thread parking/unparking" comment at the top of the// file.:= .spinningif .spinning {.spinning = falseif sched.nmspinning.Add(-1) < 0 {throw("findrunnable: negative nmspinning")}// Note the for correctness, only the last M transitioning from// spinning to non-spinning must perform these rechecks to// ensure no missed work. However, the runtime has some cases// of transient increments of nmspinning that are decremented// without going through this path, so we must be conservative// and perform the check on all spinning Ms.//// See https://go.dev/issue/43997.// Check global and P runqueues again.lock(&sched.lock)if !sched.runq.empty() {, := pidlegetSpinning(0)if != nil {, := globrunqgetbatch(int32(len(.runq)) / 2)unlock(&sched.lock)if == nil {throw("global runq empty with non-zero runqsize")}if runqputbatch(, &); !.empty() {throw("Couldn't put Gs into empty local runq")}acquirep().becomeSpinning()return , false, false}}unlock(&sched.lock):= checkRunqsNoP(, )if != nil {acquirep().becomeSpinning()goto}// Check for idle-priority GC work again., := checkIdleGCNoP()if != nil {acquirep().becomeSpinning()// Run the idle worker..gcMarkWorkerMode = gcMarkWorkerIdleMode:= traceAcquire()casgstatus(, _Gwaiting, _Grunnable)if .ok() {.GoUnpark(, 0)traceRelease()}return , false, false}// Finally, check for timer creation or expiry concurrently with// transitioning from spinning to non-spinning.//// Note that we cannot use checkTimers here because it calls// adjusttimers which may need to allocate memory, and that isn't// allowed when we don't have an active P.= checkTimersNoP(, , )}// We don't need allp anymore at this pointer, but can't clear the// snapshot without a P for the write barrier..// Poll network until next timer.if netpollinited() && (netpollAnyWaiters() || != 0) && sched.lastpoll.Swap(0) != 0 {sched.pollUntil.Store()if .p != 0 {throw("findrunnable: netpoll with p")}if .spinning {throw("findrunnable: netpoll with spinning")}:= int64(-1)if != 0 {if == 0 {= nanotime()}= -if < 0 {= 0}}if faketime != 0 {// When using fake time, just poll.= 0}, := netpoll() // block until new work is available// Refresh now again, after potentially blocking.= nanotime()sched.pollUntil.Store(0)sched.lastpoll.Store()if faketime != 0 && .empty() {// Using fake time and nothing is ready; stop M.// When all M's stop, checkdead will call timejump.stopm()goto}lock(&sched.lock), := pidleget()unlock(&sched.lock)if == nil {injectglist(&)netpollAdjustWaiters()} else {acquirep()if !.empty() {:= .pop()injectglist(&)netpollAdjustWaiters():= traceAcquire()casgstatus(, _Gwaiting, _Grunnable)if .ok() {.GoUnpark(, 0)traceRelease()}return , false, false}if {.becomeSpinning()}goto}} else if != 0 && netpollinited() {:= sched.pollUntil.Load()if == 0 || > {netpollBreak()}}stopm()goto}// pollWork reports whether there is non-background work this P could// be doing. This is a fairly lightweight check to be used for// background work loops, like idle GC. It checks a subset of the// conditions checked by the actual scheduler.func pollWork() bool {if !sched.runq.empty() {return true}:= getg().m.p.ptr()if !runqempty() {return true}if netpollinited() && netpollAnyWaiters() && sched.lastpoll.Load() != 0 {if , := netpoll(0); !.empty() {injectglist(&)netpollAdjustWaiters()return true}}return false}// stealWork attempts to steal a runnable goroutine or timer from any P.//// If newWork is true, new work may have been readied.//// If now is not 0 it is the current time. stealWork returns the passed time or// the current time if now was passed as 0.func stealWork( int64) ( *g, bool, , int64, bool) {:= getg().m.p.ptr():= falseconst = 4for := 0; < ; ++ {:= == -1for := stealOrder.start(cheaprand()); !.done(); .next() {if sched.gcwaiting.Load() {// GC work may be available.return nil, false, , , true}:= allp[.position()]if == {continue}// Steal timers from p2. This call to checkTimers is the only place// where we might hold a lock on a different P's timers. We do this// once on the last pass before checking runnext because stealing// from the other P's runnext should be the last resort, so if there// are timers to steal do that first.//// We only check timers on one of the stealing iterations because// the time stored in now doesn't change in this loop and checking// the timers for each P more than once with the same value of now// is probably a waste of time.//// timerpMask tells us whether the P may have timers at all. If it// can't, no need to check at all.if && timerpMask.read(.position()) {, , := .timers.check(, nil)=if != 0 && ( == 0 || < ) {=}if {// Running the timers may have// made an arbitrary number of G's// ready and added them to this P's// local run queue. That invalidates// the assumption of runqsteal// that it always has room to add// stolen G's. So check now if there// is a local G to run.if , := runqget(); != nil {return , , , ,}= true}}// Don't bother to attempt to steal if p2 is idle.if !idlepMask.read(.position()) {if := runqsteal(, , ); != nil {return , false, , ,}}}}// No goroutines found to steal. Regardless, running a timer may have// made some goroutine ready that we missed. Indicate the next timer to// wait for.return nil, false, , ,}// Check all Ps for a runnable G to steal.//// On entry we have no P. If a G is available to steal and a P is available,// the P is returned which the caller should acquire and attempt to steal the// work to.func checkRunqsNoP( []*p, pMask) *p {for , := range {if !.read(uint32()) && !runqempty() {lock(&sched.lock), := pidlegetSpinning(0)if == nil {// Can't get a P, don't bother checking remaining Ps.unlock(&sched.lock)return nil}unlock(&sched.lock)return}}// No work available.return nil}// Check all Ps for a timer expiring sooner than pollUntil.//// Returns updated pollUntil value.func checkTimersNoP( []*p, pMask, int64) int64 {for , := range {if .read(uint32()) {:= .timers.wakeTime()if != 0 && ( == 0 || < ) {=}}}return}// Check for idle-priority GC, without a P on entry.//// If some GC work, a P, and a worker G are all available, the P and G will be// returned. The returned P has not been wired yet.func checkIdleGCNoP() (*p, *g) {// N.B. Since we have no P, gcBlackenEnabled may change at any time; we// must check again after acquiring a P. As an optimization, we also check// if an idle mark worker is needed at all. This is OK here, because if we// observe that one isn't needed, at least one is currently running. Even if// it stops running, its own journey into the scheduler should schedule it// again, if need be (at which point, this check will pass, if relevant).if atomic.Load(&gcBlackenEnabled) == 0 || !gcController.needIdleMarkWorker() {return nil, nil}if !gcMarkWorkAvailable(nil) {return nil, nil}// Work is available; we can start an idle GC worker only if there is// an available P and available worker G.//// We can attempt to acquire these in either order, though both have// synchronization concerns (see below). Workers are almost always// available (see comment in findRunnableGCWorker for the one case// there may be none). Since we're slightly less likely to find a P,// check for that first.//// Synchronization: note that we must hold sched.lock until we are// committed to keeping it. Otherwise we cannot put the unnecessary P// back in sched.pidle without performing the full set of idle// transition checks.//// If we were to check gcBgMarkWorkerPool first, we must somehow handle// the assumption in gcControllerState.findRunnableGCWorker that an// empty gcBgMarkWorkerPool is only possible if gcMarkDone is running.lock(&sched.lock), := pidlegetSpinning(0)if == nil {unlock(&sched.lock)return nil, nil}// Now that we own a P, gcBlackenEnabled can't change (as it requires STW).if gcBlackenEnabled == 0 || !gcController.addIdleMarkWorker() {pidleput(, )unlock(&sched.lock)return nil, nil}:= (*gcBgMarkWorkerNode)(gcBgMarkWorkerPool.pop())if == nil {pidleput(, )unlock(&sched.lock)gcController.removeIdleMarkWorker()return nil, nil}unlock(&sched.lock)return , .gp.ptr()}// wakeNetPoller wakes up the thread sleeping in the network poller if it isn't// going to wake up before the when argument; or it wakes an idle P to service// timers and the network poller if there isn't one already.func wakeNetPoller( int64) {if sched.lastpoll.Load() == 0 {// In findrunnable we ensure that when polling the pollUntil// field is either zero or the time to which the current// poll is expected to run. This can have a spurious wakeup// but should never miss a wakeup.:= sched.pollUntil.Load()if == 0 || > {netpollBreak()}} else {// There are no threads in the network poller, try to get// one there so it can handle new timers.if GOOS != "plan9" { // Temporary workaround - see issue #42303.wakep()}}}func resetspinning() {:= getg()if !.m.spinning {throw("resetspinning: not a spinning m")}.m.spinning = false:= sched.nmspinning.Add(-1)if < 0 {throw("findrunnable: negative nmspinning")}// M wakeup policy is deliberately somewhat conservative, so check if we// need to wakeup another P here. See "Worker thread parking/unparking"// comment at the top of the file for details.wakep()}// injectglist adds each runnable G on the list to some run queue,// and clears glist. If there is no current P, they are added to the// global queue, and up to npidle M's are started to run them.// Otherwise, for each idle P, this adds a G to the global queue// and starts an M. Any remaining G's are added to the current P's// local run queue.// This may temporarily acquire sched.lock.// Can run concurrently with GC.func injectglist( *gList) {if .empty() {return}// Mark all the goroutines as runnable before we put them// on the run queues.var *g:= traceAcquire()for := .head.ptr(); != nil; = .schedlink.ptr() {=casgstatus(, _Gwaiting, _Grunnable)if .ok() {.GoUnpark(, 0)}}if .ok() {traceRelease()}// Turn the gList into a gQueue.:= gQueue{.head, .guintptr(), .size}* = gList{}:= func( int32) {for ; > 0; -- {:= acquirem() // See comment in startm.lock(&sched.lock), := pidlegetSpinning(0)if == nil {unlock(&sched.lock)releasem()break}startm(, false, true)unlock(&sched.lock)releasem()}}:= getg().m.p.ptr()if == nil {:= .sizelock(&sched.lock)globrunqputbatch(&)unlock(&sched.lock)()return}var gQueue:= sched.npidle.Load()for ; > 0 && !.empty(); -- {:= .pop().pushBack()}if !.empty() {:= .sizelock(&sched.lock)globrunqputbatch(&)unlock(&sched.lock)()}if runqputbatch(, &); !.empty() {lock(&sched.lock)globrunqputbatch(&)unlock(&sched.lock)}// Some P's might have become idle after we loaded `sched.npidle`// but before any goroutines were added to the queue, which could// lead to idle P's when there is work available in the global queue.// That could potentially last until other goroutines become ready// to run. That said, we need to find a way to hedge//// Calling wakep() here is the best bet, it will do nothing in the// common case (no racing on `sched.npidle`), while it could wake one// more P to execute G's, which might end up with >1 P's: the first one// wakes another P and so forth until there is no more work, but this// ought to be an extremely rare case.//// Also see "Worker thread parking/unparking" comment at the top of the file for details.wakep()}// One round of scheduler: find a runnable goroutine and execute it.// Never returns.func schedule() {:= getg().mif .locks != 0 {throw("schedule: holding locks")}if .lockedg != 0 {stoplockedm()execute(.lockedg.ptr(), false) // Never returns.}// We should not schedule away from a g that is executing a cgo call,// since the cgo call is using the m's g0 stack.if .incgo {throw("schedule: in cgo")}::= .p.ptr().preempt = false// Safety check: if we are spinning, the run queue should be empty.// Check this before calling checkTimers, as that might call// goready to put a ready goroutine on the local run queue.if .spinning && (.runnext != 0 || .runqhead != .runqtail) {throw("schedule: spinning with local work")}, , := findRunnable() // blocks until work is available// findRunnable may have collected an allp snapshot. The snapshot is// only required within findRunnable. Clear it to all GC to collect the// slice..clearAllpSnapshot()if debug.dontfreezetheworld > 0 && freezing.Load() {// See comment in freezetheworld. We don't want to perturb// scheduler state, so we didn't gcstopm in findRunnable, but// also don't want to allow new goroutines to run.//// Deadlock here rather than in the findRunnable loop so if// findRunnable is stuck in a loop we don't perturb that// either.lock(&deadlock)lock(&deadlock)}// This thread is going to run a goroutine and is not spinning anymore,// so if it was marked as spinning we need to reset it now and potentially// start a new spinning M.if .spinning {resetspinning()}if sched.disable.user && !schedEnabled() {// Scheduling of this goroutine is disabled. Put it on// the list of pending runnable goroutines for when we// re-enable user scheduling and look again.lock(&sched.lock)if schedEnabled() {// Something re-enabled scheduling while we// were acquiring the lock.unlock(&sched.lock)} else {sched.disable.runnable.pushBack()unlock(&sched.lock)goto}}// If about to schedule a not-normal goroutine (a GCworker or tracereader),// wake a P if there is one.if {wakep()}if .lockedm != 0 {// Hands off own p to the locked m,// then blocks waiting for a new p.startlockedm()goto}execute(, )}// dropg removes the association between m and the current goroutine m->curg (gp for short).// Typically a caller sets gp's status away from Grunning and then// immediately calls dropg to finish the job. The caller is also responsible// for arranging that gp will be restarted using ready at an// appropriate time. After calling dropg and arranging for gp to be// readied later, the caller can do other work but eventually should// call schedule to restart the scheduling of goroutines on this m.func dropg() {:= getg()setMNoWB(&.m.curg.m, nil)setGNoWB(&.m.curg, nil)}func parkunlock_c( *g, unsafe.Pointer) bool {unlock((*mutex)())return true}// park continuation on g0.func park_m( *g) {:= getg().m:= traceAcquire()// If g is in a synctest group, we don't want to let the group// become idle until after the waitunlockf (if any) has confirmed// that the park is happening.// We need to record gp.bubble here, since waitunlockf can change it.:= .bubbleif != nil {.incActive()}if .ok() {// Trace the event before the transition. It may take a// stack trace, but we won't own the stack after the// transition anymore..GoPark(.waitTraceBlockReason, .waitTraceSkip)}// N.B. Not using casGToWaiting here because the waitreason is// set by park_m's caller.casgstatus(, _Grunning, _Gwaiting)if .ok() {traceRelease()}dropg()if := .waitunlockf; != nil {:= (, .waitlock).waitunlockf = nil.waitlock = nilif ! {:= traceAcquire()casgstatus(, _Gwaiting, _Grunnable)if != nil {.decActive()}if .ok() {.GoUnpark(, 2)traceRelease()}execute(, true) // Schedule it back, never returns.}}if != nil {.decActive()}schedule()}func goschedImpl( *g, bool) {:= traceAcquire():= readgstatus()if &^_Gscan != _Grunning {dumpgstatus()throw("bad g status")}if .ok() {// Trace the event before the transition. It may take a// stack trace, but we won't own the stack after the// transition anymore.if {.GoPreempt()} else {.GoSched()}}casgstatus(, _Grunning, _Grunnable)if .ok() {traceRelease()}dropg()lock(&sched.lock)globrunqput()unlock(&sched.lock)if mainStarted {wakep()}schedule()}// Gosched continuation on g0.func gosched_m( *g) {goschedImpl(, false)}// goschedguarded is a forbidden-states-avoided version of gosched_m.func goschedguarded_m( *g) {if !canPreemptM(.m) {gogo(&.sched) // never return}goschedImpl(, false)}func gopreempt_m( *g) {goschedImpl(, true)}// preemptPark parks gp and puts it in _Gpreempted.////go:systemstackfunc preemptPark( *g) {:= readgstatus()if &^_Gscan != _Grunning {dumpgstatus()throw("bad g status")}if .asyncSafePoint {// Double-check that async preemption does not// happen in SPWRITE assembly functions.// isAsyncSafePoint must exclude this case.:= findfunc(.sched.pc)if !.valid() {throw("preempt at unknown pc")}if .flag&abi.FuncFlagSPWrite != 0 {println("runtime: unexpected SPWRITE function", funcname(), "in async preempt")throw("preempt SPWRITE")}}// Transition from _Grunning to _Gscan|_Gpreempted. We can't// be in _Grunning when we dropg because then we'd be running// without an M, but the moment we're in _Gpreempted,// something could claim this G before we've fully cleaned it// up. Hence, we set the scan bit to lock down further// transitions until we can dropg.casGToPreemptScan(, _Grunning, _Gscan|_Gpreempted)dropg()// Be careful about how we trace this next event. The ordering// is subtle.//// The moment we CAS into _Gpreempted, suspendG could CAS to// _Gwaiting, do its work, and ready the goroutine. All of// this could happen before we even get the chance to emit// an event. The end result is that the events could appear// out of order, and the tracer generally assumes the scheduler// takes care of the ordering between GoPark and GoUnpark.//// The answer here is simple: emit the event while we still hold// the _Gscan bit on the goroutine. We still need to traceAcquire// and traceRelease across the CAS because the tracer could be// what's calling suspendG in the first place, and we want the// CAS and event emission to appear atomic to the tracer.:= traceAcquire()if .ok() {.GoPark(traceBlockPreempted, 0)}casfrom_Gscanstatus(, _Gscan|_Gpreempted, _Gpreempted)if .ok() {traceRelease()}schedule()}// goyield is like Gosched, but it:// - emits a GoPreempt trace event instead of a GoSched trace event// - puts the current G on the runq of the current P instead of the globrunq//// goyield should be an internal detail,// but widely used packages access it using linkname.// Notable members of the hall of shame include:// - gvisor.dev/gvisor// - github.com/sagernet/gvisor//// Do not remove or change the type signature.// See go.dev/issue/67401.////go:linkname goyieldfunc goyield() {checkTimeouts()mcall(goyield_m)}func goyield_m( *g) {:= traceAcquire():= .m.p.ptr()if .ok() {// Trace the event before the transition. It may take a// stack trace, but we won't own the stack after the// transition anymore..GoPreempt()}casgstatus(, _Grunning, _Grunnable)if .ok() {traceRelease()}dropg()runqput(, , false)schedule()}// Finishes execution of the current goroutine.func goexit1() {if raceenabled {if := getg(); .bubble != nil {racereleasemergeg(, .bubble.raceaddr())}racegoend()}:= traceAcquire()if .ok() {.GoEnd()traceRelease()}mcall(goexit0)}// goexit continuation on g0.func goexit0( *g) {gdestroy()schedule()}func gdestroy( *g) {:= getg().m:= .p.ptr()casgstatus(, _Grunning, _Gdead)gcController.addScannableStack(, -int64(.stack.hi-.stack.lo))if isSystemGoroutine(, false) {sched.ngsys.Add(-1)}.m = nil:= .lockedm != 0.lockedm = 0.lockedg = 0.preemptStop = false.paniconfault = false._defer = nil // should be true already but just in case.._panic = nil // non-nil for Goexit during panic. points at stack-allocated data..writebuf = nil.waitreason = waitReasonZero.param = nil.labels = nil.timer = nil.bubble = nilif gcBlackenEnabled != 0 && .gcAssistBytes > 0 {// Flush assist credit to the global pool. This gives// better information to pacing if the application is// rapidly creating an exiting goroutines.:= gcController.assistWorkPerByte.Load():= int64( * float64(.gcAssistBytes))gcController.bgScanCredit.Add().gcAssistBytes = 0}dropg()if GOARCH == "wasm" { // no threads yet on wasmgfput(, )return}if && .lockedInt != 0 {print("runtime: mp.lockedInt = ", .lockedInt, "\n")if .isextra {throw("runtime.Goexit called in a thread that was not created by the Go runtime")}throw("exited a goroutine internally locked to the OS thread")}gfput(, )if {// The goroutine may have locked this thread because// it put it in an unusual kernel state. Kill it// rather than returning it to the thread pool.// Return to mstart, which will release the P and exit// the thread.if GOOS != "plan9" { // See golang.org/issue/22227.gogo(&.g0.sched)} else {// Clear lockedExt on plan9 since we may end up re-using// this thread..lockedExt = 0}}}// save updates getg().sched to refer to pc and sp so that a following// gogo will restore pc and sp.//// save must not have write barriers because invoking a write barrier// can clobber getg().sched.////go:nosplit//go:nowritebarrierrecfunc save(, , uintptr) {:= getg()if == .m.g0 || == .m.gsignal {// m.g0.sched is special and must describe the context// for exiting the thread. mstart1 writes to it directly.// m.gsignal.sched should not be used at all.// This check makes sure save calls do not accidentally// run in contexts where they'd write to system g's.throw("save on system g not allowed")}.sched.pc =.sched.sp =.sched.lr = 0.sched.bp =// We need to ensure ctxt is zero, but can't have a write// barrier here. However, it should always already be zero.// Assert that.if .sched.ctxt != nil {badctxt()}}// The goroutine g is about to enter a system call.// Record that it's not using the cpu anymore.// This is called only from the go syscall library and cgocall,// not from the low-level system calls used by the runtime.//// Entersyscall cannot split the stack: the save must// make g->sched refer to the caller's stack segment, because// entersyscall is going to return immediately after.//// Nothing entersyscall calls can split the stack either.// We cannot safely move the stack during an active call to syscall,// because we do not know which of the uintptr arguments are// really pointers (back into the stack).// In practice, this means that we make the fast path run through// entersyscall doing no-split things, and the slow path has to use systemstack// to run bigger things on the system stack.//// reentersyscall is the entry point used by cgo callbacks, where explicitly// saved SP and PC are restored. This is needed when exitsyscall will be called// from a function further up in the call stack than the parent, as g->syscallsp// must always point to a valid stack frame. entersyscall below is the normal// entry point for syscalls, which obtains the SP and PC from the caller.////go:nosplitfunc reentersyscall(, , uintptr) {:= traceAcquire():= getg()// Disable preemption because during this function g is in Gsyscall status,// but can have inconsistent g->sched, do not let GC observe it..m.locks++// Entersyscall must not call any function that might split/grow the stack.// (See details in comment above.)// Catch calls that might, by replacing the stack guard with something that// will trip any stack check and leaving a flag to tell newstack to die..stackguard0 = stackPreempt.throwsplit = true// Leave SP around for GC and traceback.save(, , ).syscallsp =.syscallpc =.syscallbp =casgstatus(, _Grunning, _Gsyscall)if staticLockRanking {// When doing static lock ranking casgstatus can call// systemstack which clobbers g.sched.save(, , )}if .syscallsp < .stack.lo || .stack.hi < .syscallsp {systemstack(func() {print("entersyscall inconsistent sp ", hex(.syscallsp), " [", hex(.stack.lo), ",", hex(.stack.hi), "]\n")throw("entersyscall")})}if .syscallbp != 0 && .syscallbp < .stack.lo || .stack.hi < .syscallbp {systemstack(func() {print("entersyscall inconsistent bp ", hex(.syscallbp), " [", hex(.stack.lo), ",", hex(.stack.hi), "]\n")throw("entersyscall")})}if .ok() {systemstack(func() {.GoSysCall()traceRelease()})// systemstack itself clobbers g.sched.{pc,sp} and we might// need them later when the G is genuinely blocked in a// syscallsave(, , )}if sched.sysmonwait.Load() {systemstack(entersyscall_sysmon)save(, , )}if .m.p.ptr().runSafePointFn != 0 {// runSafePointFn may stack split if run on this stacksystemstack(runSafePointFn)save(, , )}.m.syscalltick = .m.p.ptr().syscalltick:= .m.p.ptr().m = 0.m.oldp.set().m.p = 0atomic.Store(&.status, _Psyscall)if sched.gcwaiting.Load() {systemstack(entersyscall_gcwait)save(, , )}.m.locks--}// Standard syscall entry used by the go syscall library and normal cgo calls.//// This is exported via linkname to assembly in the syscall package and x/sys.//// Other packages should not be accessing entersyscall directly,// but widely used packages access it using linkname.// Notable members of the hall of shame include:// - gvisor.dev/gvisor//// Do not remove or change the type signature.// See go.dev/issue/67401.////go:nosplit//go:linkname entersyscallfunc entersyscall() {// N.B. getcallerfp cannot be written directly as argument in the call// to reentersyscall because it forces spilling the other arguments to// the stack. This results in exceeding the nosplit stack requirements// on some platforms.:= getcallerfp()reentersyscall(sys.GetCallerPC(), sys.GetCallerSP(), )}func entersyscall_sysmon() {lock(&sched.lock)if sched.sysmonwait.Load() {sched.sysmonwait.Store(false)notewakeup(&sched.sysmonnote)}unlock(&sched.lock)}func entersyscall_gcwait() {:= getg():= .m.oldp.ptr()lock(&sched.lock):= traceAcquire()if sched.stopwait > 0 && atomic.Cas(&.status, _Psyscall, _Pgcstop) {if .ok() {// This is a steal in the new tracer. While it's very likely// that we were the ones to put this P into _Psyscall, between// then and now it's totally possible it had been stolen and// then put back into _Psyscall for us to acquire here. In such// case ProcStop would be incorrect.//// TODO(mknyszek): Consider emitting a ProcStop instead when// gp.m.syscalltick == pp.syscalltick, since then we know we never// lost the P..ProcSteal(, true)traceRelease()}.gcStopTime = nanotime().syscalltick++if sched.stopwait--; sched.stopwait == 0 {notewakeup(&sched.stopnote)}} else if .ok() {traceRelease()}unlock(&sched.lock)}// The same as entersyscall(), but with a hint that the syscall is blocking.// entersyscallblock should be an internal detail,// but widely used packages access it using linkname.// Notable members of the hall of shame include:// - gvisor.dev/gvisor//// Do not remove or change the type signature.// See go.dev/issue/67401.////go:linkname entersyscallblock//go:nosplitfunc entersyscallblock() {:= getg().m.locks++ // see comment in entersyscall.throwsplit = true.stackguard0 = stackPreempt // see comment in entersyscall.m.syscalltick = .m.p.ptr().syscalltick.m.p.ptr().syscalltick++// Leave SP around for GC and traceback.:= sys.GetCallerPC():= sys.GetCallerSP():= getcallerfp()save(, , ).syscallsp = .sched.sp.syscallpc = .sched.pc.syscallbp = .sched.bpif .syscallsp < .stack.lo || .stack.hi < .syscallsp {:=:= .sched.sp:= .syscallspsystemstack(func() {print("entersyscallblock inconsistent sp ", hex(), " ", hex(), " ", hex(), " [", hex(.stack.lo), ",", hex(.stack.hi), "]\n")throw("entersyscallblock")})}casgstatus(, _Grunning, _Gsyscall)if .syscallsp < .stack.lo || .stack.hi < .syscallsp {systemstack(func() {print("entersyscallblock inconsistent sp ", hex(), " ", hex(.sched.sp), " ", hex(.syscallsp), " [", hex(.stack.lo), ",", hex(.stack.hi), "]\n")throw("entersyscallblock")})}if .syscallbp != 0 && .syscallbp < .stack.lo || .stack.hi < .syscallbp {systemstack(func() {print("entersyscallblock inconsistent bp ", hex(), " ", hex(.sched.bp), " ", hex(.syscallbp), " [", hex(.stack.lo), ",", hex(.stack.hi), "]\n")throw("entersyscallblock")})}systemstack(entersyscallblock_handoff)// Resave for traceback during blocked call.save(sys.GetCallerPC(), sys.GetCallerSP(), getcallerfp()).m.locks--}func entersyscallblock_handoff() {:= traceAcquire()if .ok() {.GoSysCall()traceRelease()}handoffp(releasep())}// The goroutine g exited its system call.// Arrange for it to run on a cpu again.// This is called only from the go syscall library, not// from the low-level system calls used by the runtime.//// Write barriers are not allowed because our P may have been stolen.//// This is exported via linkname to assembly in the syscall package.//// exitsyscall should be an internal detail,// but widely used packages access it using linkname.// Notable members of the hall of shame include:// - gvisor.dev/gvisor//// Do not remove or change the type signature.// See go.dev/issue/67401.////go:nosplit//go:nowritebarrierrec//go:linkname exitsyscallfunc exitsyscall() {:= getg().m.locks++ // see comment in entersyscallif sys.GetCallerSP() > .syscallsp {throw("exitsyscall: syscall frame is no longer valid")}.waitsince = 0:= .m.oldp.ptr().m.oldp = 0if exitsyscallfast() {// When exitsyscallfast returns success, we have a P so can now use// write barriersif goroutineProfile.active {// Make sure that gp has had its stack written out to the goroutine// profile, exactly as it was when the goroutine profiler first// stopped the world.systemstack(func() {tryRecordGoroutineProfileWB()})}:= traceAcquire()if .ok() {:= != .m.p.ptr() || .m.syscalltick != .m.p.ptr().syscallticksystemstack(func() {// Write out syscall exit eagerly.//// It's important that we write this *after* we know whether we// lost our P or not (determined by exitsyscallfast)..GoSysExit()if {// We lost the P at some point, even though we got it back here.// Trace that we're starting again, because there was a tracev2.GoSysBlock// call somewhere in exitsyscallfast (indicating that this goroutine// had blocked) and we're about to start running again..GoStart()}})}// There's a cpu for us, so we can run..m.p.ptr().syscalltick++// We need to cas the status and scan before resuming...casgstatus(, _Gsyscall, _Grunning)if .ok() {traceRelease()}// Garbage collector isn't running (since we are),// so okay to clear syscallsp..syscallsp = 0.m.locks--if .preempt {// restore the preemption request in case we've cleared it in newstack.stackguard0 = stackPreempt} else {// otherwise restore the real stackGuard, we've spoiled it in entersyscall/entersyscallblock.stackguard0 = .stack.lo + stackGuard}.throwsplit = falseif sched.disable.user && !schedEnabled() {// Scheduling of this goroutine is disabled.Gosched()}return}.m.locks--// Call the scheduler.mcall(exitsyscall0)// Scheduler returned, so we're allowed to run now.// Delete the syscallsp information that we left for// the garbage collector during the system call.// Must wait until now because until gosched returns// we don't know for sure that the garbage collector// is not running..syscallsp = 0.m.p.ptr().syscalltick++.throwsplit = false}//go:nosplitfunc exitsyscallfast( *p) bool {// Freezetheworld sets stopwait but does not retake P's.if sched.stopwait == freezeStopWait {return false}// Try to re-acquire the last P.:= traceAcquire()if != nil && .status == _Psyscall && atomic.Cas(&.status, _Psyscall, _Pidle) {// There's a cpu for us, so we can run.wirep()exitsyscallfast_reacquired()if .ok() {traceRelease()}return true}if .ok() {traceRelease()}// Try to get any other idle P.if sched.pidle != 0 {var boolsystemstack(func() {= exitsyscallfast_pidle()})if {return true}}return false}// exitsyscallfast_reacquired is the exitsyscall path on which this G// has successfully reacquired the P it was running on before the// syscall.////go:nosplitfunc exitsyscallfast_reacquired( traceLocker) {:= getg()if .m.syscalltick != .m.p.ptr().syscalltick {if .ok() {// The p was retaken and then enter into syscall again (since gp.m.syscalltick has changed).// tracev2.GoSysBlock for this syscall was already emitted,// but here we effectively retake the p from the new syscall running on the same p.systemstack(func() {// We're stealing the P. It's treated// as if it temporarily stopped running. Then, start running..ProcSteal(.m.p.ptr(), true).ProcStart()})}.m.p.ptr().syscalltick++}}func exitsyscallfast_pidle() bool {lock(&sched.lock), := pidleget(0)if != nil && sched.sysmonwait.Load() {sched.sysmonwait.Store(false)notewakeup(&sched.sysmonnote)}unlock(&sched.lock)if != nil {acquirep()return true}return false}// exitsyscall slow path on g0.// Failed to acquire P, enqueue gp as runnable.//// Called via mcall, so gp is the calling g from this M.////go:nowritebarrierrecfunc exitsyscall0( *g) {var traceLockertraceExitingSyscall()= traceAcquire()casgstatus(, _Gsyscall, _Grunnable)traceExitedSyscall()if .ok() {// Write out syscall exit eagerly.//// It's important that we write this *after* we know whether we// lost our P or not (determined by exitsyscallfast)..GoSysExit(true)traceRelease()}dropg()lock(&sched.lock)var *pif schedEnabled() {, _ = pidleget(0)}var boolif == nil {globrunqput()// Below, we stoplockedm if gp is locked. globrunqput releases// ownership of gp, so we must check if gp is locked prior to// committing the release by unlocking sched.lock, otherwise we// could race with another M transitioning gp from unlocked to// locked.= .lockedm != 0} else if sched.sysmonwait.Load() {sched.sysmonwait.Store(false)notewakeup(&sched.sysmonnote)}unlock(&sched.lock)if != nil {acquirep()execute(, false) // Never returns.}if {// Wait until another thread schedules gp and so m again.//// N.B. lockedm must be this M, as this g was running on this M// before entersyscall.stoplockedm()execute(, false) // Never returns.}stopm()schedule() // Never returns.}// Called from syscall package before fork.//// syscall_runtime_BeforeFork is for package syscall,// but widely used packages access it using linkname.// Notable members of the hall of shame include:// - gvisor.dev/gvisor//// Do not remove or change the type signature.// See go.dev/issue/67401.////go:linkname syscall_runtime_BeforeFork syscall.runtime_BeforeFork//go:nosplitfunc syscall_runtime_BeforeFork() {:= getg().m.curg// Block signals during a fork, so that the child does not run// a signal handler before exec if a signal is sent to the process// group. See issue #18600..m.locks++sigsave(&.m.sigmask)sigblock(false)// This function is called before fork in syscall package.// Code between fork and exec must not allocate memory nor even try to grow stack.// Here we spoil g.stackguard0 to reliably detect any attempts to grow stack.// runtime_AfterFork will undo this in parent process, but not in child..stackguard0 = stackFork}// Called from syscall package after fork in parent.//// syscall_runtime_AfterFork is for package syscall,// but widely used packages access it using linkname.// Notable members of the hall of shame include:// - gvisor.dev/gvisor//// Do not remove or change the type signature.// See go.dev/issue/67401.////go:linkname syscall_runtime_AfterFork syscall.runtime_AfterFork//go:nosplitfunc syscall_runtime_AfterFork() {:= getg().m.curg// See the comments in beforefork..stackguard0 = .stack.lo + stackGuardmsigrestore(.m.sigmask).m.locks--}// inForkedChild is true while manipulating signals in the child process.// This is used to avoid calling libc functions in case we are using vfork.var inForkedChild bool// Called from syscall package after fork in child.// It resets non-sigignored signals to the default handler, and// restores the signal mask in preparation for the exec.//// Because this might be called during a vfork, and therefore may be// temporarily sharing address space with the parent process, this must// not change any global variables or calling into C code that may do so.//// syscall_runtime_AfterForkInChild is for package syscall,// but widely used packages access it using linkname.// Notable members of the hall of shame include:// - gvisor.dev/gvisor//// Do not remove or change the type signature.// See go.dev/issue/67401.////go:linkname syscall_runtime_AfterForkInChild syscall.runtime_AfterForkInChild//go:nosplit//go:nowritebarrierrecfunc syscall_runtime_AfterForkInChild() {// It's OK to change the global variable inForkedChild here// because we are going to change it back. There is no race here,// because if we are sharing address space with the parent process,// then the parent process can not be running concurrently.inForkedChild = trueclearSignalHandlers()// When we are the child we are the only thread running,// so we know that nothing else has changed gp.m.sigmask.msigrestore(getg().m.sigmask)inForkedChild = false}// pendingPreemptSignals is the number of preemption signals// that have been sent but not received. This is only used on Darwin.// For #41702.var pendingPreemptSignals atomic.Int32// Called from syscall package before Exec.////go:linkname syscall_runtime_BeforeExec syscall.runtime_BeforeExecfunc syscall_runtime_BeforeExec() {// Prevent thread creation during exec.execLock.lock()// On Darwin, wait for all pending preemption signals to// be received. See issue #41702.if GOOS == "darwin" || GOOS == "ios" {for pendingPreemptSignals.Load() > 0 {osyield()}}}// Called from syscall package after Exec.////go:linkname syscall_runtime_AfterExec syscall.runtime_AfterExecfunc syscall_runtime_AfterExec() {execLock.unlock()}// Allocate a new g, with a stack big enough for stacksize bytes.func malg( int32) *g {:= new(g)if >= 0 {= round2(stackSystem + )systemstack(func() {.stack = stackalloc(uint32())if valgrindenabled {.valgrindStackID = valgrindRegisterStack(unsafe.Pointer(.stack.lo), unsafe.Pointer(.stack.hi))}}).stackguard0 = .stack.lo + stackGuard.stackguard1 = ^uintptr(0)// Clear the bottom word of the stack. We record g// there on gsignal stack during VDSO on ARM and ARM64.*(*uintptr)(unsafe.Pointer(.stack.lo)) = 0}return}// Create a new g running fn.// Put it on the queue of g's waiting to run.// The compiler turns a go statement into a call to this.func newproc( *funcval) {:= getg():= sys.GetCallerPC()systemstack(func() {:= newproc1(, , , false, waitReasonZero):= getg().m.p.ptr()runqput(, , true)if mainStarted {wakep()}})}// Create a new g in state _Grunnable (or _Gwaiting if parked is true), starting at fn.// callerpc is the address of the go statement that created this. The caller is responsible// for adding the new g to the scheduler. If parked is true, waitreason must be non-zero.func newproc1( *funcval, *g, uintptr, bool, waitReason) *g {if == nil {fatal("go of nil func value")}:= acquirem() // disable preemption because we hold M and P in local vars.:= .p.ptr():= gfget()if == nil {= malg(stackMin)casgstatus(, _Gidle, _Gdead)allgadd() // publishes with a g->status of Gdead so GC scanner doesn't look at uninitialized stack.}if .stack.hi == 0 {throw("newproc1: newg missing stack")}if readgstatus() != _Gdead {throw("newproc1: new g is not Gdead")}:= uintptr(4*goarch.PtrSize + sys.MinFrameSize) // extra space in case of reads slightly beyond frame= alignUp(, sys.StackAlign):= .stack.hi -if usesLR {// caller's LR*(*uintptr)(unsafe.Pointer()) = 0prepGoExitFrame()}if GOARCH == "arm64" {// caller's FP*(*uintptr)(unsafe.Pointer( - goarch.PtrSize)) = 0}memclrNoHeapPointers(unsafe.Pointer(&.sched), unsafe.Sizeof(.sched)).sched.sp =.stktopsp =.sched.pc = abi.FuncPCABI0(goexit) + sys.PCQuantum // +PCQuantum so that previous instruction is in same function.sched.g = guintptr(unsafe.Pointer())gostartcallfn(&.sched, ).parentGoid = .goid.gopc =.ancestors = saveAncestors().startpc = .fn.runningCleanups.Store(false)if isSystemGoroutine(, false) {sched.ngsys.Add(1)} else {// Only user goroutines inherit synctest groups and pprof labels..bubble = .bubbleif .curg != nil {.labels = .curg.labels}if goroutineProfile.active {// A concurrent goroutine profile is running. It should include// exactly the set of goroutines that were alive when the goroutine// profiler first stopped the world. That does not include newg, so// mark it as not needing a profile before transitioning it from// _Gdead..goroutineProfiled.Store(goroutineProfileSatisfied)}}// Track initial transition?.trackingSeq = uint8(cheaprand())if .trackingSeq%gTrackingPeriod == 0 {.tracking = true}gcController.addScannableStack(, int64(.stack.hi-.stack.lo))// Get a goid and switch to runnable. Make all this atomic to the tracer.:= traceAcquire()var uint32 = _Grunnableif {= _Gwaiting.waitreason =}if .goidcache == .goidcacheend {// Sched.goidgen is the last allocated id,// this batch must be [sched.goidgen+1, sched.goidgen+GoidCacheBatch].// At startup sched.goidgen=0, so main goroutine receives goid=1..goidcache = sched.goidgen.Add(_GoidCacheBatch).goidcache -= _GoidCacheBatch - 1.goidcacheend = .goidcache + _GoidCacheBatch}.goid = .goidcachecasgstatus(, _Gdead, ).goidcache++.trace.reset()if .ok() {.GoCreate(, .startpc, )traceRelease()}// Set up race context.if raceenabled {.racectx = racegostart().raceignore = 0if .labels != nil {// See note in proflabel.go on labelSync's role in synchronizing// with the reads in the signal handler.racereleasemergeg(, unsafe.Pointer(&labelSync))}}releasem()return}// saveAncestors copies previous ancestors of the given caller g and// includes info for the current caller into a new set of tracebacks for// a g being created.func saveAncestors( *g) *[]ancestorInfo {// Copy all prior info, except for the root goroutine (goid 0).if debug.tracebackancestors <= 0 || .goid == 0 {return nil}var []ancestorInfoif .ancestors != nil {= *.ancestors}:= int32(len()) + 1if > debug.tracebackancestors {= debug.tracebackancestors}:= make([]ancestorInfo, )copy([1:], )var [tracebackInnerFrames]uintptr:= gcallers(, 0, [:]):= make([]uintptr, )copy(, [:])[0] = ancestorInfo{pcs: ,goid: .goid,gopc: .gopc,}:= new([]ancestorInfo)* =return}// Put on gfree list.// If local list is too long, transfer a batch to the global list.func gfput( *p, *g) {if readgstatus() != _Gdead {throw("gfput: bad status (not Gdead)")}:= .stack.hi - .stack.loif != uintptr(startingStackSize) {// non-standard stack size - free it.stackfree(.stack).stack.lo = 0.stack.hi = 0.stackguard0 = 0if valgrindenabled {valgrindDeregisterStack(.valgrindStackID).valgrindStackID = 0}}.gFree.push()if .gFree.size >= 64 {var (gQueuegQueue)for .gFree.size >= 32 {:= .gFree.pop()if .stack.lo == 0 {.push()} else {.push()}}lock(&sched.gFree.lock)sched.gFree.noStack.pushAll()sched.gFree.stack.pushAll()unlock(&sched.gFree.lock)}}// Get from gfree list.// If local list is empty, grab a batch from global list.func gfget( *p) *g {:if .gFree.empty() && (!sched.gFree.stack.empty() || !sched.gFree.noStack.empty()) {lock(&sched.gFree.lock)// Move a batch of free Gs to the P.for .gFree.size < 32 {// Prefer Gs with stacks.:= sched.gFree.stack.pop()if == nil {= sched.gFree.noStack.pop()if == nil {break}}.gFree.push()}unlock(&sched.gFree.lock)goto}:= .gFree.pop()if == nil {return nil}if .stack.lo != 0 && .stack.hi-.stack.lo != uintptr(startingStackSize) {// Deallocate old stack. We kept it in gfput because it was the// right size when the goroutine was put on the free list, but// the right size has changed since then.systemstack(func() {stackfree(.stack).stack.lo = 0.stack.hi = 0.stackguard0 = 0if valgrindenabled {valgrindDeregisterStack(.valgrindStackID).valgrindStackID = 0}})}if .stack.lo == 0 {// Stack was deallocated in gfput or just above. Allocate a new one.systemstack(func() {.stack = stackalloc(startingStackSize)if valgrindenabled {.valgrindStackID = valgrindRegisterStack(unsafe.Pointer(.stack.lo), unsafe.Pointer(.stack.hi))}}).stackguard0 = .stack.lo + stackGuard} else {if raceenabled {racemalloc(unsafe.Pointer(.stack.lo), .stack.hi-.stack.lo)}if msanenabled {msanmalloc(unsafe.Pointer(.stack.lo), .stack.hi-.stack.lo)}if asanenabled {asanunpoison(unsafe.Pointer(.stack.lo), .stack.hi-.stack.lo)}}return}// Purge all cached G's from gfree list to the global list.func gfpurge( *p) {var (gQueuegQueue)for !.gFree.empty() {:= .gFree.pop()if .stack.lo == 0 {.push()} else {.push()}}lock(&sched.gFree.lock)sched.gFree.noStack.pushAll()sched.gFree.stack.pushAll()unlock(&sched.gFree.lock)}// Breakpoint executes a breakpoint trap.func () {breakpoint()}// dolockOSThread is called by LockOSThread and lockOSThread below// after they modify m.locked. Do not allow preemption during this call,// or else the m might be different in this function than in the caller.////go:nosplitfunc dolockOSThread() {if GOARCH == "wasm" {return // no threads on wasm yet}:= getg().m.lockedg.set().lockedm.set(.m)}// LockOSThread wires the calling goroutine to its current operating system thread.// The calling goroutine will always execute in that thread,// and no other goroutine will execute in it,// until the calling goroutine has made as many calls to// [UnlockOSThread] as to LockOSThread.// If the calling goroutine exits without unlocking the thread,// the thread will be terminated.//// All init functions are run on the startup thread. Calling LockOSThread// from an init function will cause the main function to be invoked on// that thread.//// A goroutine should call LockOSThread before calling OS services or// non-Go library functions that depend on per-thread state.////go:nosplitfunc () {if atomic.Load(&newmHandoff.haveTemplateThread) == 0 && GOOS != "plan9" {// If we need to start a new thread from the locked// thread, we need the template thread. Start it now// while we're in a known-good state.startTemplateThread()}:= getg().m.lockedExt++if .m.lockedExt == 0 {.m.lockedExt--panic("LockOSThread nesting overflow")}dolockOSThread()}//go:nosplitfunc lockOSThread() {getg().m.lockedInt++dolockOSThread()}// dounlockOSThread is called by UnlockOSThread and unlockOSThread below// after they update m->locked. Do not allow preemption during this call,// or else the m might be in different in this function than in the caller.////go:nosplitfunc dounlockOSThread() {if GOARCH == "wasm" {return // no threads on wasm yet}:= getg()if .m.lockedInt != 0 || .m.lockedExt != 0 {return}.m.lockedg = 0.lockedm = 0}// UnlockOSThread undoes an earlier call to LockOSThread.// If this drops the number of active LockOSThread calls on the// calling goroutine to zero, it unwires the calling goroutine from// its fixed operating system thread.// If there are no active LockOSThread calls, this is a no-op.//// Before calling UnlockOSThread, the caller must ensure that the OS// thread is suitable for running other goroutines. If the caller made// any permanent changes to the state of the thread that would affect// other goroutines, it should not call this function and thus leave// the goroutine locked to the OS thread until the goroutine (and// hence the thread) exits.////go:nosplitfunc () {:= getg()if .m.lockedExt == 0 {return}.m.lockedExt--dounlockOSThread()}//go:nosplitfunc unlockOSThread() {:= getg()if .m.lockedInt == 0 {systemstack(badunlockosthread)}.m.lockedInt--dounlockOSThread()}func badunlockosthread() {throw("runtime: internal error: misuse of lockOSThread/unlockOSThread")}func gcount() int32 {:= int32(atomic.Loaduintptr(&allglen)) - sched.gFree.stack.size - sched.gFree.noStack.size - sched.ngsys.Load()for , := range allp {-= .gFree.size}// All these variables can be changed concurrently, so the result can be inconsistent.// But at least the current goroutine is running.if < 1 {= 1}return}func mcount() int32 {return int32(sched.mnext - sched.nmfreed)}var prof struct {signalLock atomic.Uint32// Must hold signalLock to write. Reads may be lock-free, but// signalLock should be taken to synchronize with changes.hz atomic.Int32}func _System() { () }func _ExternalCode() { () }func _LostExternalCode() { () }func _GC() { () }func _LostSIGPROFDuringAtomic64() { () }func _LostContendedRuntimeLock() { () }func _VDSO() { () }// Called if we receive a SIGPROF signal.// Called by the signal handler, may run during STW.////go:nowritebarrierrecfunc sigprof(, , uintptr, *g, *m) {if prof.hz.Load() == 0 {return}// If mp.profilehz is 0, then profiling is not enabled for this thread.// We must check this to avoid a deadlock between setcpuprofilerate// and the call to cpuprof.add, below.if != nil && .profilehz == 0 {return}// On mips{,le}/arm, 64bit atomics are emulated with spinlocks, in// internal/runtime/atomic. If SIGPROF arrives while the program is inside// the critical section, it creates a deadlock (when writing the sample).// As a workaround, create a counter of SIGPROFs while in critical section// to store the count, and pass it to sigprof.add() later when SIGPROF is// received from somewhere else (with _LostSIGPROFDuringAtomic64 as pc).if GOARCH == "mips" || GOARCH == "mipsle" || GOARCH == "arm" {if := findfunc(); .valid() {if stringslite.HasPrefix(funcname(), "internal/runtime/atomic") {cpuprof.lostAtomic++return}}if GOARCH == "arm" && goarm < 7 && GOOS == "linux" && &0xffff0000 == 0xffff0000 {// internal/runtime/atomic functions call into kernel// helpers on arm < 7. See// internal/runtime/atomic/sys_linux_arm.s.cpuprof.lostAtomic++return}}// Profiling runs concurrently with GC, so it must not allocate.// Set a trap in case the code does allocate.// Note that on windows, one thread takes profiles of all the// other threads, so mp is usually not getg().m.// In fact mp may not even be stopped.// See golang.org/issue/17165.getg().m.mallocing++var unwindervar [maxCPUProfStack]uintptr:= 0if .ncgo > 0 && .curg != nil && .curg.syscallpc != 0 && .curg.syscallsp != 0 {:= 0// Check cgoCallersUse to make sure that we are not// interrupting other code that is fiddling with// cgoCallers. We are running in a signal handler// with all signals blocked, so we don't have to worry// about any other code interrupting us.if .cgoCallersUse.Load() == 0 && .cgoCallers != nil && .cgoCallers[0] != 0 {for < len(.cgoCallers) && .cgoCallers[] != 0 {++}+= copy([:], .cgoCallers[:]).cgoCallers[0] = 0}// Collect Go stack that leads to the cgo call..initAt(.curg.syscallpc, .curg.syscallsp, 0, .curg, unwindSilentErrors)} else if usesLibcall() && .libcallg != 0 && .libcallpc != 0 && .libcallsp != 0 {// Libcall, i.e. runtime syscall on windows.// Collect Go stack that leads to the call..initAt(.libcallpc, .libcallsp, 0, .libcallg.ptr(), unwindSilentErrors)} else if != nil && .vdsoSP != 0 {// VDSO call, e.g. nanotime1 on Linux.// Collect Go stack that leads to the call..initAt(.vdsoPC, .vdsoSP, 0, , unwindSilentErrors|unwindJumpStack)} else {.initAt(, , , , unwindSilentErrors|unwindTrap|unwindJumpStack)}+= tracebackPCs(&, 0, [:])if <= 0 {// Normal traceback is impossible or has failed.// Account it against abstract "System" or "GC".= 2if inVDSOPage() {= abi.FuncPCABIInternal(_VDSO) + sys.PCQuantum} else if > firstmoduledata.etext {// "ExternalCode" is better than "etext".= abi.FuncPCABIInternal(_ExternalCode) + sys.PCQuantum}[0] =if .preemptoff != "" {[1] = abi.FuncPCABIInternal(_GC) + sys.PCQuantum} else {[1] = abi.FuncPCABIInternal(_System) + sys.PCQuantum}}if prof.hz.Load() != 0 {// Note: it can happen on Windows that we interrupted a system thread// with no g, so gp could nil. The other nil checks are done out of// caution, but not expected to be nil in practice.var *unsafe.Pointerif != nil && .m != nil && .m.curg != nil {= &.m.curg.labels}cpuprof.add(, [:]):=var *mvar *pif != nil && .m != nil {if .m.curg != nil {= .m.curg}= .m= .m.p.ptr()}traceCPUSample(, , , [:])}getg().m.mallocing--}// setcpuprofilerate sets the CPU profiling rate to hz times per second.// If hz <= 0, setcpuprofilerate turns off CPU profiling.func setcpuprofilerate( int32) {// Force sane arguments.if < 0 {= 0}// Disable preemption, otherwise we can be rescheduled to another thread// that has profiling enabled.:= getg().m.locks++// Stop profiler on this thread so that it is safe to lock prof.// if a profiling signal came in while we had prof locked,// it would deadlock.setThreadCPUProfiler(0)for !prof.signalLock.CompareAndSwap(0, 1) {osyield()}if prof.hz.Load() != {setProcessCPUProfiler()prof.hz.Store()}prof.signalLock.Store(0)lock(&sched.lock)sched.profilehz =unlock(&sched.lock)if != 0 {setThreadCPUProfiler()}.m.locks--}// init initializes pp, which may be a freshly allocated p or a// previously destroyed p, and transitions it to status _Pgcstop.func ( *p) ( int32) {.id =.status = _Pgcstop.sudogcache = .sudogbuf[:0].deferpool = .deferpoolbuf[:0].wbBuf.reset()if .mcache == nil {if == 0 {if mcache0 == nil {throw("missing mcache?")}// Use the bootstrap mcache0. Only one P will get// mcache0: the one with ID 0..mcache = mcache0} else {.mcache = allocmcache()}}if raceenabled && .raceprocctx == 0 {if == 0 {.raceprocctx = raceprocctx0raceprocctx0 = 0 // bootstrap} else {.raceprocctx = raceproccreate()}}lockInit(&.timers.mu, lockRankTimers)// This P may get timers when it starts running. Set the mask here// since the P may not go through pidleget (notably P 0 on startup).timerpMask.set()// Similarly, we may not go through pidleget before this P starts// running if it is P 0 on startup.idlepMask.clear()}// destroy releases all of the resources associated with pp and// transitions it to status _Pdead.//// sched.lock must be held and the world must be stopped.func ( *p) () {assertLockHeld(&sched.lock)assertWorldStopped()// Move all runnable goroutines to the global queuefor .runqhead != .runqtail {// Pop from tail of local queue.runqtail--:= .runq[.runqtail%uint32(len(.runq))].ptr()// Push onto head of global queueglobrunqputhead()}if .runnext != 0 {globrunqputhead(.runnext.ptr()).runnext = 0}// Move all timers to the local P.getg().m.p.ptr().timers.take(&.timers)// Flush p's write barrier buffer.if gcphase != _GCoff {wbBufFlush1().gcw.dispose()}clear(.sudogbuf[:]).sudogcache = .sudogbuf[:0].pinnerCache = nilclear(.deferpoolbuf[:]).deferpool = .deferpoolbuf[:0]systemstack(func() {for := 0; < .mspancache.len; ++ {// Safe to call since the world is stopped.mheap_.spanalloc.free(unsafe.Pointer(.mspancache.buf[]))}.mspancache.len = 0lock(&mheap_.lock).pcache.flush(&mheap_.pages)unlock(&mheap_.lock)})freemcache(.mcache).mcache = nilgfpurge()if raceenabled {if .timers.raceCtx != 0 {// The race detector code uses a callback to fetch// the proc context, so arrange for that callback// to see the right thing.// This hack only works because we are the only// thread running.:= getg().m:= .p.ptr().p.set()racectxend(.timers.raceCtx).timers.raceCtx = 0.p.set()}raceprocdestroy(.raceprocctx).raceprocctx = 0}.gcAssistTime = 0gcCleanups.queued += .cleanupsQueued.cleanupsQueued = 0.status = _Pdead}// Change number of processors.//// sched.lock must be held, and the world must be stopped.//// gcworkbufs must not be being modified by either the GC or the write barrier// code, so the GC must not be running if the number of Ps actually changes.//// Returns list of Ps with local work, they need to be scheduled by the caller.func procresize( int32) *p {assertLockHeld(&sched.lock)assertWorldStopped():= gomaxprocsif < 0 || <= 0 {throw("procresize: invalid arg")}:= traceAcquire()if .ok() {.Gomaxprocs()traceRelease()}// update statistics:= nanotime()if sched.procresizetime != 0 {sched.totaltime += int64() * ( - sched.procresizetime)}sched.procresizetime =:= ( + 31) / 32// Grow allp if necessary.if > int32(len(allp)) {// Synchronize with retake, which could be running// concurrently since it doesn't run on a P.lock(&allpLock)if <= int32(cap(allp)) {allp = allp[:]} else {:= make([]*p, )// Copy everything up to allp's cap so we// never lose old allocated Ps.copy(, allp[:cap(allp)])allp =}if <= int32(cap(idlepMask)) {idlepMask = idlepMask[:]timerpMask = timerpMask[:]} else {:= make([]uint32, )// No need to copy beyond len, old Ps are irrelevant.copy(, idlepMask)idlepMask =:= make([]uint32, )copy(, timerpMask)timerpMask =}unlock(&allpLock)}// initialize new P'sfor := ; < ; ++ {:= allp[]if == nil {= new(p)}.init()atomicstorep(unsafe.Pointer(&allp[]), unsafe.Pointer())}:= getg()if .m.p != 0 && .m.p.ptr().id < {// continue to use the current P.m.p.ptr().status = _Prunning.m.p.ptr().mcache.prepareForSweep()} else {// release the current P and acquire allp[0].//// We must do this before destroying our current P// because p.destroy itself has write barriers, so we// need to do that from a valid P.if .m.p != 0 {:= traceAcquire()if .ok() {// Pretend that we were descheduled// and then scheduled again to keep// the trace consistent..GoSched().ProcStop(.m.p.ptr())traceRelease()}.m.p.ptr().m = 0}.m.p = 0:= allp[0].m = 0.status = _Pidleacquirep():= traceAcquire()if .ok() {.GoStart()traceRelease()}}// g.m.p is now set, so we no longer need mcache0 for bootstrapping.mcache0 = nil// release resources from unused P'sfor := ; < ; ++ {:= allp[].destroy()// can't free P itself because it can be referenced by an M in syscall}// Trim allp.if int32(len(allp)) != {lock(&allpLock)allp = allp[:]idlepMask = idlepMask[:]timerpMask = timerpMask[:]unlock(&allpLock)}var *pfor := - 1; >= 0; -- {:= allp[]if .m.p.ptr() == {continue}.status = _Pidleif runqempty() {pidleput(, )} else {.m.set(mget()).link.set()=}}stealOrder.reset(uint32())var *int32 = &gomaxprocs // make compiler check that gomaxprocs is an int32atomic.Store((*uint32)(unsafe.Pointer()), uint32())if != {// Notify the limiter that the amount of procs has changed.gcCPULimiter.resetCapacity(, )}return}// Associate p and the current m.//// This function is allowed to have write barriers even if the caller// isn't because it immediately acquires pp.////go:yeswritebarrierrecfunc acquirep( *p) {// Do the part that isn't allowed to have write barriers.wirep()// Have p; write barriers now allowed.// Perform deferred mcache flush before this P can allocate// from a potentially stale mcache..mcache.prepareForSweep():= traceAcquire()if .ok() {.ProcStart()traceRelease()}}// wirep is the first step of acquirep, which actually associates the// current M to pp. This is broken out so we can disallow write// barriers for this part, since we don't yet have a P.////go:nowritebarrierrec//go:nosplitfunc wirep( *p) {:= getg()if .m.p != 0 {// Call on the systemstack to avoid a nosplit overflow build failure// on some platforms when built with -N -l. See #64113.systemstack(func() {throw("wirep: already in go")})}if .m != 0 || .status != _Pidle {// Call on the systemstack to avoid a nosplit overflow build failure// on some platforms when built with -N -l. See #64113.systemstack(func() {:= int64(0)if .m != 0 {= .m.ptr().id}print("wirep: p->m=", .m, "(", , ") p->status=", .status, "\n")throw("wirep: invalid p state")})}.m.p.set().m.set(.m).status = _Prunning}// Disassociate p and the current m.func releasep() *p {:= traceAcquire()if .ok() {.ProcStop(getg().m.p.ptr())traceRelease()}return releasepNoTrace()}// Disassociate p and the current m without tracing an event.func releasepNoTrace() *p {:= getg()if .m.p == 0 {throw("releasep: invalid arg")}:= .m.p.ptr()if .m.ptr() != .m || .status != _Prunning {print("releasep: m=", .m, " m->p=", .m.p.ptr(), " p->m=", hex(.m), " p->status=", .status, "\n")throw("releasep: invalid p state")}.m.p = 0.m = 0.status = _Pidlereturn}func incidlelocked( int32) {lock(&sched.lock)sched.nmidlelocked +=if > 0 {checkdead()}unlock(&sched.lock)}// Check for deadlock situation.// The check is based on number of running M's, if 0 -> deadlock.// sched.lock must be held.func checkdead() {assertLockHeld(&sched.lock)// For -buildmode=c-shared or -buildmode=c-archive it's OK if// there are no running goroutines. The calling program is// assumed to be running.// One exception is Wasm, which is single-threaded. If we are// in Go and all goroutines are blocked, it deadlocks.if (islibrary || isarchive) && GOARCH != "wasm" {return}// If we are dying because of a signal caught on an already idle thread,// freezetheworld will cause all running threads to block.// And runtime will essentially enter into deadlock state,// except that there is a thread that will call exit soon.if panicking.Load() > 0 {return}// If we are not running under cgo, but we have an extra M then account// for it. (It is possible to have an extra M on Windows without cgo to// accommodate callbacks created by syscall.NewCallback. See issue #6751// for details.)var int32if !iscgo && cgoHasExtraM && extraMLength.Load() > 0 {= 1}:= mcount() - sched.nmidle - sched.nmidlelocked - sched.nmsysif > {return}if < 0 {print("runtime: checkdead: nmidle=", sched.nmidle, " nmidlelocked=", sched.nmidlelocked, " mcount=", mcount(), " nmsys=", sched.nmsys, "\n")unlock(&sched.lock)throw("checkdead: inconsistent counts")}:= 0forEachG(func( *g) {if isSystemGoroutine(, false) {return}:= readgstatus()switch &^ _Gscan {case _Gwaiting,_Gpreempted:++case _Grunnable,_Grunning,_Gsyscall:print("runtime: checkdead: find g ", .goid, " in status ", , "\n")unlock(&sched.lock)throw("checkdead: runnable g")}})if == 0 { // possible if main goroutine calls runtime·Goexit()unlock(&sched.lock) // unlock so that GODEBUG=scheddetail=1 doesn't hangfatal("no goroutines (main called runtime.Goexit) - deadlock!")}// Maybe jump time forward for playground.if faketime != 0 {if := timeSleepUntil(); < maxWhen {faketime =// Start an M to steal the timer., := pidleget(faketime)if == nil {// There should always be a free P since// nothing is running.unlock(&sched.lock)throw("checkdead: no p for timer")}:= mget()if == nil {// There should always be a free M since// nothing is running.unlock(&sched.lock)throw("checkdead: no m for timer")}// M must be spinning to steal. We set this to be// explicit, but since this is the only M it would// become spinning on its own anyways.sched.nmspinning.Add(1).spinning = true.nextp.set()notewakeup(&.park)return}}// There are no goroutines running, so we can look at the P's.for , := range allp {if len(.timers.heap) > 0 {return}}unlock(&sched.lock) // unlock so that GODEBUG=scheddetail=1 doesn't hangfatal("all goroutines are asleep - deadlock!")}// forcegcperiod is the maximum time in nanoseconds between garbage// collections. If we go this long without a garbage collection, one// is forced to run.//// This is a variable for testing purposes. It normally doesn't change.var forcegcperiod int64 = 2 * 60 * 1e9// needSysmonWorkaround is true if the workaround for// golang.org/issue/42515 is needed on NetBSD.var needSysmonWorkaround bool = false// haveSysmon indicates whether there is sysmon thread support.//// No threads on wasm yet, so no sysmon.const haveSysmon = GOARCH != "wasm"// Always runs without a P, so write barriers are not allowed.////go:nowritebarrierrecfunc sysmon() {lock(&sched.lock)sched.nmsys++checkdead()unlock(&sched.lock):= int64(0):= int64(0):= 0 // how many cycles in succession we had not wokeup somebody:= uint32(0)for {if == 0 { // start with 20us sleep...= 20} else if > 50 { // start doubling the sleep after 1ms...*= 2}if > 10*1000 { // up to 10ms= 10 * 1000}usleep()// sysmon should not enter deep sleep if schedtrace is enabled so that// it can print that information at the right time.//// It should also not enter deep sleep if there are any active P's so// that it can retake P's from syscalls, preempt long running G's, and// poll the network if all P's are busy for long stretches.//// It should wakeup from deep sleep if any P's become active either due// to exiting a syscall or waking up due to a timer expiring so that it// can resume performing those duties. If it wakes from a syscall it// resets idle and delay as a bet that since it had retaken a P from a// syscall before, it may need to do it again shortly after the// application starts work again. It does not reset idle when waking// from a timer to avoid adding system load to applications that spend// most of their time sleeping.:= nanotime()if debug.schedtrace <= 0 && (sched.gcwaiting.Load() || sched.npidle.Load() == gomaxprocs) {lock(&sched.lock)if sched.gcwaiting.Load() || sched.npidle.Load() == gomaxprocs {:= false:= timeSleepUntil()if > {sched.sysmonwait.Store(true)unlock(&sched.lock)// Make wake-up period small enough// for the sampling to be correct.:= forcegcperiod / 2if - < {= -}:= >= osRelaxMinNSif {osRelax(true)}= notetsleep(&sched.sysmonnote, )if {osRelax(false)}lock(&sched.lock)sched.sysmonwait.Store(false)noteclear(&sched.sysmonnote)}if {= 0= 20}}unlock(&sched.lock)}lock(&sched.sysmonlock)// Update now in case we blocked on sysmonnote or spent a long time// blocked on schedlock or sysmonlock above.= nanotime()// trigger libc interceptors if neededif *cgo_yield != nil {asmcgocall(*cgo_yield, nil)}// poll network if not polled for more than 10ms:= sched.lastpoll.Load()if netpollinited() && != 0 && +10*1000*1000 < {sched.lastpoll.CompareAndSwap(, ), := netpoll(0) // non-blocking - returns list of goroutinesif !.empty() {// Need to decrement number of idle locked M's// (pretending that one more is running) before injectglist.// Otherwise it can lead to the following situation:// injectglist grabs all P's but before it starts M's to run the P's,// another M returns from syscall, finishes running its G,// observes that there is no work to do and no other running M's// and reports deadlock.incidlelocked(-1)injectglist(&)incidlelocked(1)netpollAdjustWaiters()}}if GOOS == "netbsd" && needSysmonWorkaround {// netpoll is responsible for waiting for timer// expiration, so we typically don't have to worry// about starting an M to service timers. (Note that// sleep for timeSleepUntil above simply ensures sysmon// starts running again when that timer expiration may// cause Go code to run again).//// However, netbsd has a kernel bug that sometimes// misses netpollBreak wake-ups, which can lead to// unbounded delays servicing timers. If we detect this// overrun, then startm to get something to handle the// timer.//// See issue 42515 and// https://gnats.netbsd.org/cgi-bin/query-pr-single.pl?number=50094.if := timeSleepUntil(); < {startm(nil, false, false)}}// Check if we need to update GOMAXPROCS at most once per second.if debug.updatemaxprocs != 0 && +1e9 <= {sysmonUpdateGOMAXPROCS()=}if scavenger.sysmonWake.Load() != 0 {// Kick the scavenger awake if someone requested it.scavenger.wake()}// retake P's blocked in syscalls// and preempt long running G'sif retake() != 0 {= 0} else {++}// check if we need to force a GCif := (gcTrigger{kind: gcTriggerTime, now: }); .test() && forcegc.idle.Load() {lock(&forcegc.lock)forcegc.idle.Store(false)var gList.push(forcegc.g)injectglist(&)unlock(&forcegc.lock)}if debug.schedtrace > 0 && +int64(debug.schedtrace)*1000000 <= {=schedtrace(debug.scheddetail > 0)}unlock(&sched.sysmonlock)}}type sysmontick struct {schedtick uint32syscalltick uint32schedwhen int64syscallwhen int64}// forcePreemptNS is the time slice given to a G before it is// preempted.const forcePreemptNS = 10 * 1000 * 1000 // 10msfunc retake( int64) uint32 {:= 0// Prevent allp slice changes. This lock will be completely// uncontended unless we're already stopping the world.lock(&allpLock)// We can't use a range loop over allp because we may// temporarily drop the allpLock. Hence, we need to re-fetch// allp each time around the loop.for := 0; < len(allp); ++ {:= allp[]if == nil {// This can happen if procresize has grown// allp but not yet created new Ps.continue}:= &.sysmontick:= .status:= falseif == _Prunning || == _Psyscall {// Preempt G if it's running on the same schedtick for// too long. This could be from a single long-running// goroutine or a sequence of goroutines run via// runnext, which share a single schedtick time slice.:= int64(.schedtick)if int64(.schedtick) != {.schedtick = uint32().schedwhen =} else if .schedwhen+forcePreemptNS <= {preemptone()// In case of syscall, preemptone() doesn't// work, because there is no M wired to P.= true}}if == _Psyscall {// Retake P from syscall if it's there for more than 1 sysmon tick (at least 20us).:= int64(.syscalltick)if ! && int64(.syscalltick) != {.syscalltick = uint32().syscallwhen =continue}// On the one hand we don't want to retake Ps if there is no other work to do,// but on the other hand we want to retake them eventually// because they can prevent the sysmon thread from deep sleep.if runqempty() && sched.nmspinning.Load()+sched.npidle.Load() > 0 && .syscallwhen+10*1000*1000 > {continue}// Drop allpLock so we can take sched.lock.unlock(&allpLock)// Need to decrement number of idle locked M's// (pretending that one more is running) before the CAS.// Otherwise the M from which we retake can exit the syscall,// increment nmidle and report deadlock.incidlelocked(-1):= traceAcquire()if atomic.Cas(&.status, , _Pidle) {if .ok() {.ProcSteal(, false)traceRelease()}++.syscalltick++handoffp()} else if .ok() {traceRelease()}incidlelocked(1)lock(&allpLock)}}unlock(&allpLock)return uint32()}// Tell all goroutines that they have been preempted and they should stop.// This function is purely best-effort. It can fail to inform a goroutine if a// processor just started running it.// No locks need to be held.// Returns true if preemption request was issued to at least one goroutine.func preemptall() bool {:= falsefor , := range allp {if .status != _Prunning {continue}if preemptone() {= true}}return}// Tell the goroutine running on processor P to stop.// This function is purely best-effort. It can incorrectly fail to inform the// goroutine. It can inform the wrong goroutine. Even if it informs the// correct goroutine, that goroutine might ignore the request if it is// simultaneously executing newstack.// No lock needs to be held.// Returns true if preemption request was issued.// The actual preemption will happen at some point in the future// and will be indicated by the gp->status no longer being// Grunningfunc preemptone( *p) bool {:= .m.ptr()if == nil || == getg().m {return false}:= .curgif == nil || == .g0 {return false}.preempt = true// Every call in a goroutine checks for stack overflow by// comparing the current stack pointer to gp->stackguard0.// Setting gp->stackguard0 to StackPreempt folds// preemption into the normal stack overflow check..stackguard0 = stackPreempt// Request an async preemption of this P.if preemptMSupported && debug.asyncpreemptoff == 0 {.preempt = truepreemptM()}return true}var starttime int64func schedtrace( bool) {:= nanotime()if starttime == 0 {starttime =}lock(&sched.lock)print("SCHED ", (-starttime)/1e6, "ms: gomaxprocs=", gomaxprocs, " idleprocs=", sched.npidle.Load(), " threads=", mcount(), " spinningthreads=", sched.nmspinning.Load(), " needspinning=", sched.needspinning.Load(), " idlethreads=", sched.nmidle, " runqueue=", sched.runq.size)if {print(" gcwaiting=", sched.gcwaiting.Load(), " nmidlelocked=", sched.nmidlelocked, " stopwait=", sched.stopwait, " sysmonwait=", sched.sysmonwait.Load(), "\n")}// We must be careful while reading data from P's, M's and G's.// Even if we hold schedlock, most data can be changed concurrently.// E.g. (p->m ? p->m->id : -1) can crash if p->m changes from non-nil to nil.for , := range allp {:= atomic.Load(&.runqhead):= atomic.Load(&.runqtail)if {print(" P", , ": status=", .status, " schedtick=", .schedtick, " syscalltick=", .syscalltick, " m="):= .m.ptr()if != nil {print(.id)} else {print("nil")}print(" runqsize=", -, " gfreecnt=", .gFree.size, " timerslen=", len(.timers.heap), "\n")} else {// In non-detailed mode format lengths of per-P run queues as:// [ len1 len2 len3 len4 ]print(" ")if == 0 {print("[ ")}print( - )if == len(allp)-1 {print(" ]")}}}if ! {// Format per-P schedticks as: schedticks=[ tick1 tick2 tick3 tick4 ].print(" schedticks=[ ")for , := range allp {print(.schedtick)print(" ")}print("]\n")}if ! {unlock(&sched.lock)return}for := allm; != nil; = .alllink {:= .p.ptr()print(" M", .id, ": p=")if != nil {print(.id)} else {print("nil")}print(" curg=")if .curg != nil {print(.curg.goid)} else {print("nil")}print(" mallocing=", .mallocing, " throwing=", .throwing, " preemptoff=", .preemptoff, " locks=", .locks, " dying=", .dying, " spinning=", .spinning, " blocked=", .blocked, " lockedg=")if := .lockedg.ptr(); != nil {print(.goid)} else {print("nil")}print("\n")}forEachG(func( *g) {print(" G", .goid, ": status=", readgstatus(), "(", .waitreason.String(), ") m=")if .m != nil {print(.m.id)} else {print("nil")}print(" lockedm=")if := .lockedm.ptr(); != nil {print(.id)} else {print("nil")}print("\n")})unlock(&sched.lock)}type updateMaxProcsGState struct {lock mutexg *gidle atomic.Bool// Readable when idle == false, writable when idle == true.procs int32 // new GOMAXPROCS value}var (// GOMAXPROCS update godebug metric. Incremented if automatic// GOMAXPROCS updates actually change the value of GOMAXPROCS.updatemaxprocs = &godebugInc{name: "updatemaxprocs"}// Synchronization and state between updateMaxProcsGoroutine and// sysmon.updateMaxProcsG updateMaxProcsGState// Synchronization between GOMAXPROCS and sysmon.//// Setting GOMAXPROCS via a call to GOMAXPROCS disables automatic// GOMAXPROCS updates.//// We want to make two guarantees to callers of GOMAXPROCS. After// GOMAXPROCS returns://// 1. The runtime will not make any automatic changes to GOMAXPROCS.//// 2. The runtime will not perform any of the system calls used to// determine the appropriate value of GOMAXPROCS (i.e., it won't// call defaultGOMAXPROCS).//// (1) is the baseline guarantee that everyone needs. The GOMAXPROCS// API isn't useful to anyone if automatic updates may occur after it// returns. This is easily achieved by double-checking the state under// STW before committing an automatic GOMAXPROCS update.//// (2) doesn't matter to most users, as it is isn't observable as long// as (1) holds. However, it can be important to users sandboxing Go.// They want disable these system calls and need some way to know when// they are guaranteed the calls will stop.//// This would be simple to achieve if we simply called// defaultGOMAXPROCS under STW in updateMaxProcsGoroutine below.// However, we would like to avoid scheduling this goroutine every// second when it will almost never do anything. Instead, sysmon calls// defaultGOMAXPROCS to decide whether to schedule// updateMaxProcsGoroutine. Thus we need to synchronize between sysmon// and GOMAXPROCS calls.//// GOMAXPROCS can't hold a runtime mutex across STW. It could hold a// semaphore, but sysmon cannot take semaphores. Instead, we have a// more complex scheme://// * sysmon holds computeMaxProcsLock while calling defaultGOMAXPROCS.// * sysmon skips the current update if sched.customGOMAXPROCS is// set.// * GOMAXPROCS sets sched.customGOMAXPROCS once it is committed to// changing GOMAXPROCS.// * GOMAXPROCS takes computeMaxProcsLock to wait for outstanding// defaultGOMAXPROCS calls to complete.//// N.B. computeMaxProcsLock could simply be sched.lock, but we want to// avoid holding that lock during the potentially slow// defaultGOMAXPROCS.computeMaxProcsLock mutex)// Start GOMAXPROCS update helper goroutine.//// This is based on forcegchelper.func defaultGOMAXPROCSUpdateEnable() {if debug.updatemaxprocs == 0 {// Unconditionally increment the metric when updates are disabled.//// It would be more descriptive if we did a dry run of the// complete update, determining the appropriate value of// GOMAXPROCS and the bailing out and just incrementing the// metric if a change would occur.//// Not only is that a lot of ongoing work for a disabled// feature, but some users need to be able to completely// disable the update system calls (such as sandboxes).// Currently, updatemaxprocs=0 serves that purpose.updatemaxprocs.IncNonDefault()return}go updateMaxProcsGoroutine()}func updateMaxProcsGoroutine() {updateMaxProcsG.g = getg()lockInit(&updateMaxProcsG.lock, lockRankUpdateMaxProcsG)for {lock(&updateMaxProcsG.lock)if updateMaxProcsG.idle.Load() {throw("updateMaxProcsGoroutine: phase error")}updateMaxProcsG.idle.Store(true)goparkunlock(&updateMaxProcsG.lock, waitReasonUpdateGOMAXPROCSIdle, traceBlockSystemGoroutine, 1)// This goroutine is explicitly resumed by sysmon.:= stopTheWorldGC(stwGOMAXPROCS)// Still OK to update?lock(&sched.lock):= sched.customGOMAXPROCSunlock(&sched.lock)if {startTheWorldGC()return}// newprocs will be processed by startTheWorld//// TODO(prattmic): this could use a nicer API. Perhaps add it to the// stw parameter?newprocs = updateMaxProcsG.procslock(&sched.lock)sched.customGOMAXPROCS = falseunlock(&sched.lock)startTheWorldGC()}}func sysmonUpdateGOMAXPROCS() {// Synchronize with GOMAXPROCS. See comment on computeMaxProcsLock.lock(&computeMaxProcsLock)// No update if GOMAXPROCS was set manually.lock(&sched.lock):= sched.customGOMAXPROCS:= gomaxprocsunlock(&sched.lock)if {unlock(&computeMaxProcsLock)return}// Don't hold sched.lock while we read the filesystem.:= defaultGOMAXPROCS(0)unlock(&computeMaxProcsLock)if == {// Nothing to do.return}// Sysmon can't directly stop the world. Run the helper to do so on our// behalf. If updateGOMAXPROCS.idle is false, then a previous update is// still pending.if updateMaxProcsG.idle.Load() {lock(&updateMaxProcsG.lock)updateMaxProcsG.procs =updateMaxProcsG.idle.Store(false)var gList.push(updateMaxProcsG.g)injectglist(&)unlock(&updateMaxProcsG.lock)}}// schedEnableUser enables or disables the scheduling of user// goroutines.//// This does not stop already running user goroutines, so the caller// should first stop the world when disabling user goroutines.func schedEnableUser( bool) {lock(&sched.lock)if sched.disable.user == ! {unlock(&sched.lock)return}sched.disable.user = !if {:= sched.disable.runnable.sizeglobrunqputbatch(&sched.disable.runnable)unlock(&sched.lock)for ; != 0 && sched.npidle.Load() != 0; -- {startm(nil, false, false)}} else {unlock(&sched.lock)}}// schedEnabled reports whether gp should be scheduled. It returns// false is scheduling of gp is disabled.//// sched.lock must be held.func schedEnabled( *g) bool {assertLockHeld(&sched.lock)if sched.disable.user {return isSystemGoroutine(, true)}return true}// Put mp on midle list.// sched.lock must be held.// May run during STW, so write barriers are not allowed.////go:nowritebarrierrecfunc mput( *m) {assertLockHeld(&sched.lock).schedlink = sched.midlesched.midle.set()sched.nmidle++checkdead()}// Try to get an m from midle list.// sched.lock must be held.// May run during STW, so write barriers are not allowed.////go:nowritebarrierrecfunc mget() *m {assertLockHeld(&sched.lock):= sched.midle.ptr()if != nil {sched.midle = .schedlinksched.nmidle--}return}// Put gp on the global runnable queue.// sched.lock must be held.// May run during STW, so write barriers are not allowed.////go:nowritebarrierrecfunc globrunqput( *g) {assertLockHeld(&sched.lock)sched.runq.pushBack()}// Put gp at the head of the global runnable queue.// sched.lock must be held.// May run during STW, so write barriers are not allowed.////go:nowritebarrierrecfunc globrunqputhead( *g) {assertLockHeld(&sched.lock)sched.runq.push()}// Put a batch of runnable goroutines on the global runnable queue.// This clears *batch.// sched.lock must be held.// May run during STW, so write barriers are not allowed.////go:nowritebarrierrecfunc globrunqputbatch( *gQueue) {assertLockHeld(&sched.lock)sched.runq.pushBackAll(*)* = gQueue{}}// Try get a single G from the global runnable queue.// sched.lock must be held.func globrunqget() *g {assertLockHeld(&sched.lock)if sched.runq.size == 0 {return nil}return sched.runq.pop()}// Try get a batch of G's from the global runnable queue.// sched.lock must be held.func globrunqgetbatch( int32) ( *g, gQueue) {assertLockHeld(&sched.lock)if sched.runq.size == 0 {return}= min(, sched.runq.size, sched.runq.size/gomaxprocs+1)= sched.runq.pop()--for ; > 0; -- {:= sched.runq.pop().pushBack()}return}// pMask is an atomic bitstring with one bit per P.type pMask []uint32// read returns true if P id's bit is set.func ( pMask) ( uint32) bool {:= / 32:= uint32(1) << ( % 32)return (atomic.Load(&[]) & ) != 0}// set sets P id's bit.func ( pMask) ( int32) {:= / 32:= uint32(1) << ( % 32)atomic.Or(&[], )}// clear clears P id's bit.func ( pMask) ( int32) {:= / 32:= uint32(1) << ( % 32)atomic.And(&[], ^)}// pidleput puts p on the _Pidle list. now must be a relatively recent call// to nanotime or zero. Returns now or the current time if now was zero.//// This releases ownership of p. Once sched.lock is released it is no longer// safe to use p.//// sched.lock must be held.//// May run during STW, so write barriers are not allowed.////go:nowritebarrierrecfunc pidleput( *p, int64) int64 {assertLockHeld(&sched.lock)if !runqempty() {throw("pidleput: P has non-empty run queue")}if == 0 {= nanotime()}if .timers.len.Load() == 0 {timerpMask.clear(.id)}idlepMask.set(.id).link = sched.pidlesched.pidle.set()sched.npidle.Add(1)if !.limiterEvent.start(limiterEventIdle, ) {throw("must be able to track idle limiter event")}return}// pidleget tries to get a p from the _Pidle list, acquiring ownership.//// sched.lock must be held.//// May run during STW, so write barriers are not allowed.////go:nowritebarrierrecfunc pidleget( int64) (*p, int64) {assertLockHeld(&sched.lock):= sched.pidle.ptr()if != nil {// Timer may get added at any time now.if == 0 {= nanotime()}timerpMask.set(.id)idlepMask.clear(.id)sched.pidle = .linksched.npidle.Add(-1).limiterEvent.stop(limiterEventIdle, )}return ,}// pidlegetSpinning tries to get a p from the _Pidle list, acquiring ownership.// This is called by spinning Ms (or callers than need a spinning M) that have// found work. If no P is available, this must synchronized with non-spinning// Ms that may be preparing to drop their P without discovering this work.//// sched.lock must be held.//// May run during STW, so write barriers are not allowed.////go:nowritebarrierrecfunc pidlegetSpinning( int64) (*p, int64) {assertLockHeld(&sched.lock), := pidleget()if == nil {// See "Delicate dance" comment in findrunnable. We found work// that we cannot take, we must synchronize with non-spinning// Ms that may be preparing to drop their P.sched.needspinning.Store(1)return nil,}return ,}// runqempty reports whether pp has no Gs on its local run queue.// It never returns true spuriously.func runqempty( *p) bool {// Defend against a race where 1) pp has G1 in runqnext but runqhead == runqtail,// 2) runqput on pp kicks G1 to the runq, 3) runqget on pp empties runqnext.// Simply observing that runqhead == runqtail and then observing that runqnext == nil// does not mean the queue is empty.for {:= atomic.Load(&.runqhead):= atomic.Load(&.runqtail):= atomic.Loaduintptr((*uintptr)(unsafe.Pointer(&.runnext)))if == atomic.Load(&.runqtail) {return == && == 0}}}// To shake out latent assumptions about scheduling order,// we introduce some randomness into scheduling decisions// when running with the race detector.// The need for this was made obvious by changing the// (deterministic) scheduling order in Go 1.5 and breaking// many poorly-written tests.// With the randomness here, as long as the tests pass// consistently with -race, they shouldn't have latent scheduling// assumptions.const randomizeScheduler = raceenabled// runqput tries to put g on the local runnable queue.// If next is false, runqput adds g to the tail of the runnable queue.// If next is true, runqput puts g in the pp.runnext slot.// If the run queue is full, runnext puts g on the global queue.// Executed only by the owner P.func runqput( *p, *g, bool) {if !haveSysmon && {// A runnext goroutine shares the same time slice as the// current goroutine (inheritTime from runqget). To prevent a// ping-pong pair of goroutines from starving all others, we// depend on sysmon to preempt "long-running goroutines". That// is, any set of goroutines sharing the same time slice.//// If there is no sysmon, we must avoid runnext entirely or// risk starvation.= false}if randomizeScheduler && && randn(2) == 0 {= false}if {::= .runnextif !.runnext.cas(, guintptr(unsafe.Pointer())) {goto}if == 0 {return}// Kick the old runnext out to the regular run queue.= .ptr()}::= atomic.LoadAcq(&.runqhead) // load-acquire, synchronize with consumers:= .runqtailif - < uint32(len(.runq)) {.runq[%uint32(len(.runq))].set()atomic.StoreRel(&.runqtail, +1) // store-release, makes the item available for consumptionreturn}if runqputslow(, , , ) {return}// the queue is not full, now the put above must succeedgoto}// Put g and a batch of work from local runnable queue on global queue.// Executed only by the owner P.func runqputslow( *p, *g, , uint32) bool {var [len(.runq)/2 + 1]*g// First, grab a batch from local queue.:= -= / 2if != uint32(len(.runq)/2) {throw("runqputslow: queue is not full")}for := uint32(0); < ; ++ {[] = .runq[(+)%uint32(len(.runq))].ptr()}if !atomic.CasRel(&.runqhead, , +) { // cas-release, commits consumereturn false}[] =if randomizeScheduler {for := uint32(1); <= ; ++ {:= cheaprandn( + 1)[], [] = [], []}}// Link the goroutines.for := uint32(0); < ; ++ {[].schedlink.set([+1])}:= gQueue{[0].guintptr(), [].guintptr(), int32( + 1)}// Now put the batch on global queue.lock(&sched.lock)globrunqputbatch(&)unlock(&sched.lock)return true}// runqputbatch tries to put all the G's on q on the local runnable queue.// If the local runq is full the input queue still contains unqueued Gs.// Executed only by the owner P.func runqputbatch( *p, *gQueue) {if .empty() {return}:= atomic.LoadAcq(&.runqhead):= .runqtail:= uint32(0)for !.empty() && - < uint32(len(.runq)) {:= .pop().runq[%uint32(len(.runq))].set()++++}if randomizeScheduler {:= func( uint32) uint32 {return (.runqtail + ) % uint32(len(.runq))}for := uint32(1); < ; ++ {:= cheaprandn( + 1).runq[()], .runq[()] = .runq[()], .runq[()]}}atomic.StoreRel(&.runqtail, )return}// Get g from local runnable queue.// If inheritTime is true, gp should inherit the remaining time in the// current time slice. Otherwise, it should start a new time slice.// Executed only by the owner P.func runqget( *p) ( *g, bool) {// If there's a runnext, it's the next G to run.:= .runnext// If the runnext is non-0 and the CAS fails, it could only have been stolen by another P,// because other Ps can race to set runnext to 0, but only the current P can set it to non-0.// Hence, there's no need to retry this CAS if it fails.if != 0 && .runnext.cas(, 0) {return .ptr(), true}for {:= atomic.LoadAcq(&.runqhead) // load-acquire, synchronize with other consumers:= .runqtailif == {return nil, false}:= .runq[%uint32(len(.runq))].ptr()if atomic.CasRel(&.runqhead, , +1) { // cas-release, commits consumereturn , false}}}// runqdrain drains the local runnable queue of pp and returns all goroutines in it.// Executed only by the owner P.func runqdrain( *p) ( gQueue) {:= .runnextif != 0 && .runnext.cas(, 0) {.pushBack(.ptr())}::= atomic.LoadAcq(&.runqhead) // load-acquire, synchronize with other consumers:= .runqtail:= -if == 0 {return}if > uint32(len(.runq)) { // read inconsistent h and tgoto}if !atomic.CasRel(&.runqhead, , +) { // cas-release, commits consumegoto}// We've inverted the order in which it gets G's from the local P's runnable queue// and then advances the head pointer because we don't want to mess up the statuses of G's// while runqdrain() and runqsteal() are running in parallel.// Thus we should advance the head pointer before draining the local P into a gQueue,// so that we can update any gp.schedlink only after we take the full ownership of G,// meanwhile, other P's can't access to all G's in local P's runnable queue and steal them.// See https://groups.google.com/g/golang-dev/c/0pTKxEKhHSc/m/6Q85QjdVBQAJ for more details.for := uint32(0); < ; ++ {:= .runq[(+)%uint32(len(.runq))].ptr().pushBack()}return}// Grabs a batch of goroutines from pp's runnable queue into batch.// Batch is a ring buffer starting at batchHead.// Returns number of grabbed goroutines.// Can be executed by any P.func runqgrab( *p, *[256]guintptr, uint32, bool) uint32 {for {:= atomic.LoadAcq(&.runqhead) // load-acquire, synchronize with other consumers:= atomic.LoadAcq(&.runqtail) // load-acquire, synchronize with the producer:= -= - /2if == 0 {if {// Try to steal from pp.runnext.if := .runnext; != 0 {if .status == _Prunning {// Sleep to ensure that pp isn't about to run the g// we are about to steal.// The important use case here is when the g running// on pp ready()s another g and then almost// immediately blocks. Instead of stealing runnext// in this window, back off to give pp a chance to// schedule runnext. This will avoid thrashing gs// between different Ps.// A sync chan send/recv takes ~50ns as of time of// writing, so 3us gives ~50x overshoot.if !osHasLowResTimer {usleep(3)} else {// On some platforms system timer granularity is// 1-15ms, which is way too much for this// optimization. So just yield.osyield()}}if !.runnext.cas(, 0) {continue}[%uint32(len())] =return 1}}return 0}if > uint32(len(.runq)/2) { // read inconsistent h and tcontinue}for := uint32(0); < ; ++ {:= .runq[(+)%uint32(len(.runq))][(+)%uint32(len())] =}if atomic.CasRel(&.runqhead, , +) { // cas-release, commits consumereturn}}}// Steal half of elements from local runnable queue of p2// and put onto local runnable queue of p.// Returns one of the stolen elements (or nil if failed).func runqsteal(, *p, bool) *g {:= .runqtail:= runqgrab(, &.runq, , )if == 0 {return nil}--:= .runq[(+)%uint32(len(.runq))].ptr()if == 0 {return}:= atomic.LoadAcq(&.runqhead) // load-acquire, synchronize with consumersif -+ >= uint32(len(.runq)) {throw("runqsteal: runq overflow")}atomic.StoreRel(&.runqtail, +) // store-release, makes the item available for consumptionreturn}// A gQueue is a dequeue of Gs linked through g.schedlink. A G can only// be on one gQueue or gList at a time.type gQueue struct {head guintptrtail guintptrsize int32}// empty reports whether q is empty.func ( *gQueue) () bool {return .head == 0}// push adds gp to the head of q.func ( *gQueue) ( *g) {.schedlink = .head.head.set()if .tail == 0 {.tail.set()}.size++}// pushBack adds gp to the tail of q.func ( *gQueue) ( *g) {.schedlink = 0if .tail != 0 {.tail.ptr().schedlink.set()} else {.head.set()}.tail.set().size++}// pushBackAll adds all Gs in q2 to the tail of q. After this q2 must// not be used.func ( *gQueue) ( gQueue) {if .tail == 0 {return}.tail.ptr().schedlink = 0if .tail != 0 {.tail.ptr().schedlink = .head} else {.head = .head}.tail = .tail.size += .size}// pop removes and returns the head of queue q. It returns nil if// q is empty.func ( *gQueue) () *g {:= .head.ptr()if != nil {.head = .schedlinkif .head == 0 {.tail = 0}.size--}return}// popList takes all Gs in q and returns them as a gList.func ( *gQueue) () gList {:= gList{.head, .size}* = gQueue{}return}// A gList is a list of Gs linked through g.schedlink. A G can only be// on one gQueue or gList at a time.type gList struct {head guintptrsize int32}// empty reports whether l is empty.func ( *gList) () bool {return .head == 0}// push adds gp to the head of l.func ( *gList) ( *g) {.schedlink = .head.head.set().size++}// pushAll prepends all Gs in q to l. After this q must not be used.func ( *gList) ( gQueue) {if !.empty() {.tail.ptr().schedlink = .head.head = .head.size += .size}}// pop removes and returns the head of l. If l is empty, it returns nil.func ( *gList) () *g {:= .head.ptr()if != nil {.head = .schedlink.size--}return}//go:linkname setMaxThreads runtime/debug.setMaxThreadsfunc setMaxThreads( int) ( int) {lock(&sched.lock)= int(sched.maxmcount)if > 0x7fffffff { // MaxInt32sched.maxmcount = 0x7fffffff} else {sched.maxmcount = int32()}checkmcount()unlock(&sched.lock)return}// procPin should be an internal detail,// but widely used packages access it using linkname.// Notable members of the hall of shame include:// - github.com/bytedance/gopkg// - github.com/choleraehyq/pid// - github.com/songzhibin97/gkit//// Do not remove or change the type signature.// See go.dev/issue/67401.////go:linkname procPin//go:nosplitfunc procPin() int {:= getg():= .m.locks++return int(.p.ptr().id)}// procUnpin should be an internal detail,// but widely used packages access it using linkname.// Notable members of the hall of shame include:// - github.com/bytedance/gopkg// - github.com/choleraehyq/pid// - github.com/songzhibin97/gkit//// Do not remove or change the type signature.// See go.dev/issue/67401.////go:linkname procUnpin//go:nosplitfunc procUnpin() {:= getg().m.locks--}//go:linkname sync_runtime_procPin sync.runtime_procPin//go:nosplitfunc sync_runtime_procPin() int {return procPin()}//go:linkname sync_runtime_procUnpin sync.runtime_procUnpin//go:nosplitfunc sync_runtime_procUnpin() {procUnpin()}//go:linkname sync_atomic_runtime_procPin sync/atomic.runtime_procPin//go:nosplitfunc sync_atomic_runtime_procPin() int {return procPin()}//go:linkname sync_atomic_runtime_procUnpin sync/atomic.runtime_procUnpin//go:nosplitfunc sync_atomic_runtime_procUnpin() {procUnpin()}// Active spinning for sync.Mutex.////go:linkname internal_sync_runtime_canSpin internal/sync.runtime_canSpin//go:nosplitfunc internal_sync_runtime_canSpin( int) bool {// sync.Mutex is cooperative, so we are conservative with spinning.// Spin only few times and only if running on a multicore machine and// GOMAXPROCS>1 and there is at least one other running P and local runq is empty.// As opposed to runtime mutex we don't do passive spinning here,// because there can be work on global runq or on other Ps.if >= active_spin || numCPUStartup <= 1 || gomaxprocs <= sched.npidle.Load()+sched.nmspinning.Load()+1 {return false}if := getg().m.p.ptr(); !runqempty() {return false}return true}//go:linkname internal_sync_runtime_doSpin internal/sync.runtime_doSpin//go:nosplitfunc internal_sync_runtime_doSpin() {procyield(active_spin_cnt)}// Active spinning for sync.Mutex.//// sync_runtime_canSpin should be an internal detail,// but widely used packages access it using linkname.// Notable members of the hall of shame include:// - github.com/livekit/protocol// - github.com/sagernet/gvisor// - gvisor.dev/gvisor//// Do not remove or change the type signature.// See go.dev/issue/67401.////go:linkname sync_runtime_canSpin sync.runtime_canSpin//go:nosplitfunc sync_runtime_canSpin( int) bool {return internal_sync_runtime_canSpin()}// sync_runtime_doSpin should be an internal detail,// but widely used packages access it using linkname.// Notable members of the hall of shame include:// - github.com/livekit/protocol// - github.com/sagernet/gvisor// - gvisor.dev/gvisor//// Do not remove or change the type signature.// See go.dev/issue/67401.////go:linkname sync_runtime_doSpin sync.runtime_doSpin//go:nosplitfunc sync_runtime_doSpin() {internal_sync_runtime_doSpin()}var stealOrder randomOrder// randomOrder/randomEnum are helper types for randomized work stealing.// They allow to enumerate all Ps in different pseudo-random orders without repetitions.// The algorithm is based on the fact that if we have X such that X and GOMAXPROCS// are coprime, then a sequences of (i + X) % GOMAXPROCS gives the required enumeration.type randomOrder struct {count uint32coprimes []uint32}type randomEnum struct {i uint32count uint32pos uint32inc uint32}func ( *randomOrder) ( uint32) {.count =.coprimes = .coprimes[:0]for := uint32(1); <= ; ++ {if gcd(, ) == 1 {.coprimes = append(.coprimes, )}}}func ( *randomOrder) ( uint32) randomEnum {return randomEnum{count: .count,pos: % .count,inc: .coprimes[/.count%uint32(len(.coprimes))],}}func ( *randomEnum) () bool {return .i == .count}func ( *randomEnum) () {.i++.pos = (.pos + .inc) % .count}func ( *randomEnum) () uint32 {return .pos}func gcd(, uint32) uint32 {for != 0 {, = , %}return}// An initTask represents the set of initializations that need to be done for a package.// Keep in sync with ../../test/noinit.go:initTasktype initTask struct {state uint32 // 0 = uninitialized, 1 = in progress, 2 = donenfns uint32// followed by nfns pcs, uintptr sized, one per init function to run}// inittrace stores statistics for init functions which are// updated by malloc and newproc when active is true.var inittrace tracestattype tracestat struct {active bool // init tracing activation statusid uint64 // init goroutine idallocs uint64 // heap allocationsbytes uint64 // heap allocated bytes}func doInit( []*initTask) {for , := range {doInit1()}}func doInit1( *initTask) {switch .state {case 2: // fully initializedreturncase 1: // initialization in progressthrow("recursive call during initialization - linker skew")default: // not initialized yet.state = 1 // initialization in progressvar (int64tracestat)if inittrace.active {= nanotime()// Load stats non-atomically since tracinit is updated only by this init goroutine.= inittrace}if .nfns == 0 {// We should have pruned all of these in the linker.throw("inittask with no functions")}:= add(unsafe.Pointer(), 8)for := uint32(0); < .nfns; ++ {:= add(, uintptr()*goarch.PtrSize):= *(*func())(unsafe.Pointer(&))()}if inittrace.active {:= nanotime()// Load stats non-atomically since tracinit is updated only by this init goroutine.:= inittrace:= *(*func())(unsafe.Pointer(&)):= funcpkgpath(findfunc(abi.FuncPCABIInternal()))var [24]byteprint("init ", , " @")print(string(fmtNSAsMS([:], uint64(-runtimeInitTime))), " ms, ")print(string(fmtNSAsMS([:], uint64(-))), " ms clock, ")print(string(itoa([:], .bytes-.bytes)), " bytes, ")print(string(itoa([:], .allocs-.allocs)), " allocs")print("\n")}.state = 2 // initialization done}}
![]() |
The pages are generated with Golds v0.7.9-preview. (GOOS=linux GOARCH=amd64) Golds is a Go 101 project developed by Tapir Liu. PR and bug reports are welcome and can be submitted to the issue list. Please follow @zigo_101 (reachable from the left QR code) to get the latest news of Golds. |