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 runtime

import (
	
	
	
	
	
)

var buildVersion = sys.TheVersion
set using cmd/go/internal/modload.ModInfoProg
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: We unpark an additional thread when we ready a goroutine if (1) there is an idle P and 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 global run queue/ 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 do some spinning looking for work in per-P run queues 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 readying goroutines. 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 a new goroutine, 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 goroutine readying is: submit a goroutine to local work queue, #StoreLoad-style memory barrier, check sched.nmspinning. The general pattern for spinning->non-spinning transition is: decrement nmspinning, #StoreLoad-style memory barrier, check all per-P work queues 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.
go:linkname runtime_inittask runtime..inittask
go:linkname main_inittask main..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.
go:linkname main_main main.main
func ()
mainStarted indicates that the main M has started.
runtimeInitTime is the nanotime() at which the runtime started.
Value to use for signal mask for newly created M's.
The main goroutine.
func () {
	 := getg()
Racectx of m0->g0 is used only as the parent of the main goroutine. It must not be used for anything else.
	.m.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 sys.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.
Allow newproc to start new Ms.
For runtime_syscall_doAllThreadsSyscall, we register sysmon is not ready for the world to be stopped.
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 .m != &m0 {
		throw("runtime.main not on m0")
	}
	m0.doesPark = true
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().goid
		inittrace.active = true
	}

	doInit(&runtime_inittask) // Must be before defer.
Defer unlock so that runtime.Goexit during init does the unlock too.
	 := true
	defer func() {
		if  {
			unlockOSThread()
		}
	}()

	gcenable()

	main_init_done = make(chan bool)
	if iscgo {
		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")
Start the template thread in case we enter Go from a C-created thread and need to create a new thread.
Disable init tracing after main init done to avoid overhead of collecting statistics in malloc and newproc
A program compiled with -buildmode=c-archive or c-shared has a main, but it is not executed.
		return
	}
	 := main_main // make an indirect call, as the linker doesn't know the address of the main package when laying down the runtime
	()
	if raceenabled {
		racefini()
	}
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.
Running deferred functions should not take long.
		for  := 0;  < 1000; ++ {
			if atomic.Load(&runningPanicDefers) == 0 {
				break
			}
			Gosched()
		}
	}
	if atomic.Load(&panicking) != 0 {
		gopark(nil, nil, waitReasonPanicWait, traceEvGoStop, 1)
	}

	exit(0)
	for {
		var  *int32
		* = 0
	}
}
os_beforeExit is called from os.Exit(0).go:linkname os_beforeExit os.runtime_beforeExit
func () {
	if raceenabled {
		racefini()
	}
}
start forcegc helper goroutine
func () {
	go forcegchelper()
}

func () {
	forcegc.g = getg()
	lockInit(&forcegc.lock, lockRankForcegc)
	for {
		lock(&forcegc.lock)
		if forcegc.idle != 0 {
			throw("forcegc: phase error")
		}
		atomic.Store(&forcegc.idle, 1)
this goroutine is explicitly resumed by sysmon
		if debug.gctrace > 0 {
			println("GC forced")
Time-triggered, fully concurrent.
go:nosplit
Gosched yields the processor, allowing other goroutines to run. It does not suspend the current goroutine, so execution resumes automatically.
goschedguarded yields the processor like gosched, but also checks for forbidden states and opts out of the yield in those cases.go:nosplit
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.
func ( func(*g, unsafe.Pointer) bool,  unsafe.Pointer,  waitReason,  byte,  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 = 
	.waittraceev = 
	.waittraceskip = 
can't do anything that might move the G between Ms here.
Puts the current goroutine into a waiting state and unlocks the lock. The goroutine can be made runnable again by calling goready(gp).
func ( *mutex,  waitReason,  byte,  int) {
	gopark(parkunlock_c, unsafe.Pointer(), , , )
}

func ( *g,  int) {
	systemstack(func() {
		ready(, , true)
	})
}
go:nosplit
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 {
First, try to grab a batch from central cache.
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:nosplit
func ( *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()
Transfer half of local cache to the central cache.
		var ,  *sudog
		for len(.sudogcache) > cap(.sudogcache)/2 {
			 := len(.sudogcache)
			 := .sudogcache[-1]
			.sudogcache[-1] = nil
			.sudogcache = .sudogcache[:-1]
			if  == nil {
				 = 
			} else {
				.next = 
			}
			 = 
		}
		lock(&sched.sudoglock)
		.next = sched.sudogcache
		sched.sudogcache = 
		unlock(&sched.sudoglock)
	}
	.sudogcache = append(.sudogcache, )
	releasem()
}
funcPC returns the entry PC of the function f. It assumes that f is a func value. Otherwise the behavior is undefined. CAREFUL: In programs with plugins, funcPC can return different values for the same function (because there are actually multiple copies of the same function in the address space). To be safe, don't use the results of this function in any == expression. It is only safe to use the result as an address at which to start executing code.go:nosplit
func ( interface{}) uintptr {
	return *(*uintptr)(efaceOf(&).data)
}
called from assembly
func ( func(*g)) {
	throw("runtime: mcall called on m->g0 stack")
}

func ( func(*g)) {
	throw("runtime: mcall function returned")
}

func () {
	panic(plainError("arg size to reflect.call more than 1GB"))
}

var badmorestackg0Msg = "fatal: morestack on g0\n"
go:nosplitgo:nowritebarrierrec
func () {
	 := stringStructOf(&badmorestackg0Msg)
	write(2, .str, int32(.len))
}

var badmorestackgsignalMsg = "fatal: morestack on gsignal\n"
go:nosplitgo:nowritebarrierrec
go:nosplit
func () {
	throw("ctxt != 0")
}

func () bool {
	 := getg()
	return .lockedm != 0 && .m.lockedg != 0
}

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.
allglen and allgptr are atomic variables that contain len(allg) and &allg[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 uintptr
	allgptr **g
)

func ( *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)
}
atomicAllG returns &allgs[0] and len(allgs) for use with atomicAllGIndex.
func () (**g, uintptr) {
	 := atomic.Loaduintptr(&allglen)
	 := (**g)(atomic.Loadp(unsafe.Pointer(&allgptr)))
	return , 
}
atomicAllGIndex returns ptr[i] with the allgptr returned from atomicAllG.
func ( **g,  uintptr) *g {
	return *(**g)(add(unsafe.Pointer(), *sys.PtrSize))
}

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.
cpuinit extracts the environment variable GODEBUG from the environment on Unix-like operating systems and calls internal/cpu.Initialize.
func () {
	const  = "GODEBUG="
	var  string

	switch GOOS {
	case "aix", "darwin", "ios", "dragonfly", "freebsd", "netbsd", "openbsd", "illumos", "solaris", "linux":
		cpu.DebugOptions = true
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+)
			 := *(*string)(unsafe.Pointer(&stringStruct{unsafe.Pointer(), findnull()}))

			if hasPrefix(, ) {
				 = gostring()[len():]
				break
			}
		}
	}

	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.
Enforce that this lock is always a leaf lock. All of this lock's critical sections should be extremely short.
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 = 10000
The world starts stopped.
	worldStopped()

	moduledataverify()
	stackinit()
	mallocinit()
	fastrandinit() // must run before mcommoninit
	mcommoninit(.m, -1)
	cpuinit()       // must run before alginit
	alginit()       // maps must not be used before this call
	modulesinit()   // provides activeModules
	typelinksinit() // uses maps, activeModules
	itabsinit()     // uses activeModules

	sigsave(&.m.sigmask)
	initSigmask = .m.sigmask

	goargs()
	goenvs()
	parsedebugvars()
	gcinit()

	lock(&sched.lock)
	sched.lastpoll = uint64(nanotime())
	 := ncpu
	if ,  := atoi32(gogetenv("GOMAXPROCS"));  &&  > 0 {
		 = 
	}
	if procresize() != nil {
		throw("unknown runnable goroutine during bootstrap")
	}
	unlock(&sched.lock)
World is effectively started now, as P's can run.
For cgocheck > 1, we turn on the write barrier at all times and check all pointer writes. We can't do this until after procresize because the write barrier needs a P.
	if debug.cgocheck > 1 {
		writeBarrier.cgo = true
		writeBarrier.enabled = true
		for ,  := range allp {
			.wbBuf.reset()
		}
	}

Condition should never trigger. This code just serves to ensure runtime·buildVersion is kept in the resulting binary.
		buildVersion = "unknown"
	}
Condition should never trigger. This code just serves to ensure runtime·modinfo is kept in the resulting binary.
		modinfo = ""
	}
}

func ( *g) {
	 := getg()
	print("runtime: gp: gp=", , ", goid=", .goid, ", gp->atomicstatus=", readgstatus(), "\n")
	print("runtime:  g:  g=", , ", goid=", .goid, ",  g->atomicstatus=", readgstatus(), "\n")
}
sched.lock must be held.
func () {
	assertLockHeld(&sched.lock)

	if mcount() > 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 () int64 {
	assertLockHeld(&sched.lock)

	if sched.mnext+1 < sched.mnext {
		throw("runtime: thread ID overflow")
	}
	 := sched.mnext
	sched.mnext++
	checkmcount()
	return 
}
Pre-allocated ID may be passed as 'id', or omitted by passing -1.
func ( *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()
	}

	.fastrand[0] = uint32(int64Hash(uint64(.id), fastrandseed))
	.fastrand[1] = uint32(int64Hash(uint64(cputicks()), ^fastrandseed))
	if .fastrand[0]|.fastrand[1] == 0 {
		.fastrand[1] = 1
	}

	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() iterates over allm w/o schedlock, so we need to publish it safely.
Allocate memory to hold a cgo traceback if the cgo call crashes.
	if iscgo || GOOS == "solaris" || GOOS == "illumos" || GOOS == "windows" {
		.cgoCallers = new(cgoCallers)
	}
}

var fastrandseed uintptr

func () {
	 := (*[unsafe.Sizeof(fastrandseed)]byte)(unsafe.Pointer(&fastrandseed))[:]
	getRandomData()
}
Mark gp ready to run.
func ( *g,  int,  bool) {
	if trace.enabled {
		traceGoUnpark(, )
	}

	 := readgstatus()
Mark runnable.
	 := getg()
	 := acquirem() // disable preemption because it can be holding p in a local var
	if &^_Gscan != _Gwaiting {
		dumpgstatus()
		throw("bad g->status in ready")
	}
status is Gwaiting or Gscanwaiting, make Grunnable and put on runq
	casgstatus(, _Gwaiting, _Grunnable)
	runqput(.m.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.
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 () {
stopwait and preemption requests can be lost due to races with concurrently executing threads, so try several times
this should tell the scheduler to not start any new goroutines
this should stop running goroutines
		if !preemptall() {
			break // no running goroutines
		}
		usleep(1000)
to be sure
	usleep(1000)
	preemptall()
	usleep(1000)
}
All reads and writes of g's status go through readgstatus, casgstatus castogscanstatus, casfrom_Gscanstatus.go:nosplit
func ( *g) uint32 {
	return atomic.Load(&.atomicstatus)
}
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 ( *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 {
			 = atomic.Cas(&.atomicstatus, , )
		}
	}
	if ! {
		print("runtime: casfrom_Gscanstatus failed gp=", , ", oldval=", hex(), ", newval=", hex(), "\n")
		dumpgstatus()
		throw("casfrom_Gscanstatus: gp->status is not in scan state")
	}
	releaseLockRank(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 ( *g, ,  uint32) bool {
	switch  {
	case _Grunnable,
		_Grunning,
		_Gwaiting,
		_Gsyscall:
		if  == |_Gscan {
			 := atomic.Cas(&.atomicstatus, , )
			if  {
				acquireLockRank(lockRankGscan)
			}
			return 

		}
	}
	print("runtime: castogscanstatus oldval=", hex(), " newval=", hex(), "\n")
	throw("castogscanstatus")
	panic("not reached")
}
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:nosplit
func ( *g, ,  uint32) {
	if (&_Gscan != 0) || (&_Gscan != 0) ||  ==  {
		systemstack(func() {
			print("runtime: casgstatus: oldval=", hex(), " newval=", hex(), "\n")
			throw("casgstatus: bad incoming values")
		})
	}

	acquireLockRank(lockRankGscan)
	releaseLockRank(lockRankGscan)
See https://golang.org/cl/21503 for justification of the yield delay.
	const  = 5 * 1000
	var  int64
loop if gp->atomicstatus is in a scan state giving GC time to finish and change the state to oldval.
	for  := 0; !atomic.Cas(&.atomicstatus, , ); ++ {
		if  == _Gwaiting && .atomicstatus == _Grunnable {
			throw("casgstatus: waiting for Gwaiting but is Grunnable")
		}
		if  == 0 {
			 = nanotime() + 
		}
		if nanotime() <  {
			for  := 0;  < 10 && .atomicstatus != ; ++ {
				procyield(1)
			}
		} else {
			osyield()
			 = nanotime() + /2
		}
	}
}
casgstatus(gp, oldstatus, Gcopystack), assuming oldstatus is Gwaiting or Grunnable. Returns old status. Cannot call casgstatus directly, because we are racing with an async wakeup that might come in from netpoll. If we see Gwaiting from the readgstatus, it might have become Grunnable by the time we get to the cas. If we called casgstatus, it would loop waiting for the status to go back to Gwaiting, which it never will.go:nosplit
func ( *g) uint32 {
	for {
		 := readgstatus() &^ _Gscan
		if  != _Gwaiting &&  != _Grunnable {
			throw("copystack: bad status, not Gwaiting or Grunnable")
		}
		if atomic.Cas(&.atomicstatus, , _Gcopystack) {
			return 
		}
	}
}
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 ( *g, ,  uint32) {
	if  != _Grunning ||  != _Gscan|_Gpreempted {
		throw("bad g transition")
	}
	acquireLockRank(lockRankGscan)
	for !atomic.Cas(&.atomicstatus, _Grunning, _Gscan|_Gpreempted) {
	}
}
casGFromPreempted attempts to transition gp from _Gpreempted to _Gwaiting. If successful, the caller is responsible for re-scheduling gp.
func ( *g, ,  uint32) bool {
	if  != _Gpreempted ||  != _Gwaiting {
		throw("bad g transition")
	}
	return atomic.Cas(&.atomicstatus, _Gpreempted, _Gwaiting)
}
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.
func ( string) {
	semacquire(&worldsema)
	 := getg()
	.m.preemptoff = 
Mark the goroutine which called stopTheWorld preemptible so its stack may be scanned. This lets a mark worker scan us 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. 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.
startTheWorld undoes the effects of stopTheWorld.
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 ( string) {
	semacquire(&gcsema)
	stopTheWorld()
}
startTheWorldGC undoes the effects of stopTheWorldGC.
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" systemstack(stopTheWorldWithSema) When finished, the caller must either call startTheWorld or undo these three operations separately: m.preemptoff = "" systemstack(startTheWorldWithSema) 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.
func () {
	 := 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)
	sched.stopwait = gomaxprocs
	atomic.Store(&sched.gcwaiting, 1)
stop current P
	.m.p.ptr().status = _Pgcstop // Pgcstop is only diagnostic.
try to retake all P's in Psyscall status
	for ,  := range allp {
		 := .status
		if  == _Psyscall && atomic.Cas(&.status, , _Pgcstop) {
			if trace.enabled {
				traceGoSysBlock()
				traceProcStop()
			}
			.syscalltick++
			sched.stopwait--
		}
stop idle P's
	for {
		 := pidleget()
		if  == nil {
			break
		}
		.status = _Pgcstop
		sched.stopwait--
	}
	 := sched.stopwait > 0
	unlock(&sched.lock)
wait for remaining P's to stop voluntarily
	if  {
wait for 100us, then try to re-preempt in case of any races
			if notetsleep(&sched.stopnote, 100*1000) {
				noteclear(&sched.stopnote)
				break
			}
			preemptall()
		}
	}
sanity checks
	 := ""
	if sched.stopwait != 0 {
		 = "stopTheWorld: not stopped (stopwait != 0)"
	} else {
		for ,  := range allp {
			if .status != _Pgcstop {
				 = "stopTheWorld: not stopped (status != _Pgcstop)"
			}
		}
	}
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()
}

func ( bool) int64 {
	assertWorldStopped()

	 := acquirem() // disable preemption because it can be holding p in a local var
	if netpollinited() {
		 := netpoll(0) // non-blocking
		injectglist(&)
	}
	lock(&sched.lock)

	 := gomaxprocs
	if newprocs != 0 {
		 = newprocs
		newprocs = 0
	}
	 := procresize()
	sched.gcwaiting = 0
	if sched.sysmonwait != 0 {
		sched.sysmonwait = 0
		notewakeup(&sched.sysmonnote)
	}
	unlock(&sched.lock)

	worldStarted()

	for  != nil {
		 := 
		 = .link.ptr()
		if .m != 0 {
			 := .m.ptr()
			.m = 0
			if .nextp != 0 {
				throw("startTheWorld: inconsistent mp->nextp")
			}
			.nextp.set()
			notewakeup(&.park)
Start M to run P. Do not start another M below.
			newm(nil, , -1)
		}
	}
Capture start-the-world time before doing clean-up tasks.
	 := nanotime()
	if  {
		traceGCSTWDone()
	}
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 () bool {
	switch GOOS {
	case "aix", "darwin", "illumos", "ios", "solaris", "windows":
		return true
	case "openbsd":
		return GOARCH == "amd64" || GOARCH == "arm64"
	}
	return false
}
mStackIsSystemAllocated indicates whether this runtime starts on a system-allocated stack.
func () bool {
	switch GOOS {
	case "aix", "darwin", "plan9", "illumos", "ios", "solaris", "windows":
		return true
	case "openbsd":
		switch GOARCH {
		case "amd64", "arm64":
			return true
		}
	}
	return false
}
mstart is the 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:nosplitgo:nowritebarrierrec
func () {
	 := getg()

	 := .stack.lo == 0
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.hi
		if  == 0 {
			 = 8192 * 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.
This is the g0, so we can also call go:systemstack functions, which check stackguard1.
Exit this thread.
Windows, Solaris, illumos, Darwin, AIX and Plan 9 always system-allocate the stack, but put it in _g_.stack before mstart, so the logic above hasn't set osStack yet.
		 = true
	}
	mexit()
}

func () {
	 := getg()

	if  != .m.g0 {
		throw("bad runtime·mstart")
	}
Record the caller for use as the top of stack in mcall and for terminating the thread. We're never coming back to mstart1 after we call schedule, so other calls can reuse the current frame.
Install signal handlers; after minit so that minit can prepare the thread to be able to handle the signals.
	if .m == &m0 {
		mstartm0()
	}

	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:yeswritebarrierrec
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 = true
		newextram()
	}
	initsig(false)
}
mPark causes a thread to park itself - temporarily waking for fixups but otherwise waiting to be fully woken. This is the only way that m's should park themselves.go:nosplit
func () {
	 := getg()
	for {
		notesleep(&.m.park)
		noteclear(&.m.park)
		if !mDoFixup() {
			return
		}
	}
}
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(&_g_.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:yeswritebarrierrec
func ( bool) {
	 := getg()
	 := .m

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.
Free the gsignal stack.
	if .gsignal != nil {
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
	}
Remove m from allm.
	lock(&sched.lock)
	for  := &allm; * != nil;  = &(*).alllink {
		if * ==  {
			* = .alllink
			goto 
		}
	}
	throw("m not found in allm")
:
Delay reaping m until it's done with the stack. If this is using an OS stack, the OS will free it so there's no need for reaping.
Put m on the free list, though it will not be reaped until freeWait is 0. Note that the free list must not be linked through alllink because some functions walk allm without locking, so may be using alllink.
Release the P.
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.
Make sure pendingPreemptSignals is correct when an M exits. For #41702.
Destroy all allocated resources. After this is called, we may no longer take any locks.
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.
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.go:systemstack
func ( func(*p)) {
	 := acquirem()
	 := getg().m.p.ptr()

	lock(&sched.lock)
	if sched.safePointWait != 0 {
		throw("forEachP: sched.safePointWait != 0")
	}
	sched.safePointWait = gomaxprocs - 1
	sched.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 > 0
	unlock(&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
		if  == _Psyscall && .runSafePointFn == 1 && atomic.Cas(&.status, , _Pidle) {
			if trace.enabled {
				traceGoSysBlock()
				traceProcStop()
			}
			.syscalltick++
			handoffp()
		}
	}
Wait for remaining Ps to run fn.
	if  {
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 = nil
	unlock(&sched.lock)
	releasem()
}
syscall_runtime_doAllThreadsSyscall serializes Go execution and executes a specified fn() call on all m's. The boolean argument to fn() indicates whether the function's return value will be consulted or not. That is, fn(true) should return true if fn() succeeds, and fn(true) should return false if it failed. When fn(false) is called, its return status will be ignored. syscall_runtime_doAllThreadsSyscall first invokes fn(true) on a single, coordinating, m, and only if it returns true does it go on to invoke fn(false) on all of the other m's known to the process.go:linkname syscall_runtime_doAllThreadsSyscall syscall.runtime_doAllThreadsSyscall
func ( func(bool) bool) {
	if iscgo {
		panic("doAllThreadsSyscall not supported with cgo enabled")
	}
	if  == nil {
		return
	}
	for atomic.Load(&sched.sysmonStarting) != 0 {
		osyield()
	}
	stopTheWorldGC("doAllThreadsSyscall")
Ensure that there are no in-flight thread creations: don't want to race with allm.
For m's running without racectx, we loan out the racectx of this call.
		lock(&mFixupRace.lock)
		mFixupRace.ctx = .racectx
		unlock(&mFixupRace.lock)
	}
	if  := (true);  {
		 := .m.procid
		for  := allm;  != nil;  = .alllink {
This m has already completed fn() call.
				continue
Be wary of mp's without procid values if they are known not to park. If they are marked as parking with a zero procid, then they will be racing with this code to be allocated a procid and we will annotate them with the need to execute the fn when they acquire a procid to run it.
Reaching here, we are either running Windows, or cgo linked code. Neither of which are currently supported by this API.
				throw("unsupported runtime environment")
stopTheWorldGC() doesn't guarantee stopping all the threads, so we lock here to avoid the possibility of racing with mp.
			lock(&.mFixup.lock)
			.mFixup.fn = 
For non-service threads this will cause the wakeup to be short lived (once the mutex is unlocked). The next real wakeup will occur after startTheWorldGC() is called.
				notewakeup(&.park)
			}
			unlock(&.mFixup.lock)
		}
		for {
			 := true
			for  := allm;  &&  != nil;  = .alllink {
				if .procid ==  {
					continue
				}
				lock(&.mFixup.lock)
				 =  && (.mFixup.fn == nil)
				unlock(&.mFixup.lock)
			}
			if  {
				break
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 () {
Resolve the race between forEachP running the safe-point function on this P's behalf and this P running the safe-point function directly.
When running with cgo, we call _cgo_thread_start to start threads for us so that we can play nicely with foreign code.
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 _p_.go:yeswritebarrierrec
func ( *p,  func(),  int64) *m {
	 := getg()
	acquirem() // disable GC because it can be called from sysmon
	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  *m
		for  := sched.freem;  != nil; {
			if .freeWait != 0 {
				 := .freelink
				.freelink = 
				 = 
				 = 
				continue
stackfree must be on the system stack, but allocm is reachable off the system stack transitively from startm.
			systemstack(func() {
				stackfree(.g0.stack)
			})
			 = .freelink
		}
		sched.freem = 
		unlock(&sched.lock)
	}

	 := new(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(8192 * sys.StackGuardMultiplier)
	}
	.g0.m = 

	if  == .m.p.ptr() {
		releasep()
	}
	releasem(.m)

	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. When the callback is done with the m, it calls dropm to put the m back on the list.go:nosplit
func () {
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.
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  sigset
	sigsave(&)
	sigblock(false)
Lock extra list, take head, unlock popped list. nilokay=false is safe here because of the invariant above, that the extra list always contains or will soon contain at least one m.
	 := lockextra(false)
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).
Store the original signal mask for use by minit.
	.sigmask = 
Install g (= m->g0) and set the stack bounds to match the current stack. We don't actually know how big the stack is, like we don't know how big any scheduling stack is, but we assume there's at least 32 kB, which is more than enough for us.
	setg(.g0)
	 := getg()
	.stack.hi = getcallersp() + 1024
	.stack.lo = getcallersp() - 32*1024
	.stackguard0 = .stack.lo + _StackGuard
Initialize this thread to use the m.
mp.curg is now a real goroutine.
	casgstatus(.curg, _Gdead, _Gsyscall)
	atomic.Xadd(&sched.ngsys, -1)
}

var earlycgocallback = []byte("fatal error: cgo callback before cgo call\n")
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 () {
	 := atomic.Xchg(&extraMWaiters, 0)
	if  > 0 {
		for  := uint32(0);  < ; ++ {
			oneNewExtraM()
		}
Make sure there is at least one extra M.
		 := lockextra(true)
		unlockextra()
		if  == nil {
			oneNewExtraM()
		}
	}
}
oneNewExtraM allocates an m and puts it on the extra list.
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 = funcPC(goexit) + sys.PCQuantum
	.sched.sp = .stack.hi
	.sched.sp -= 4 * sys.RegSize // extra space in case of reads slightly beyond frame
	.sched.lr = 0
	.sched.g = guintptr(unsafe.Pointer())
	.syscallpc = .sched.pc
	.syscallsp = .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.
put on allg for garbage collector
	allgadd()
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.
Add m to the extra list.
	 := lockextra(true)
	.schedlink.set()
	extraMCount++
	unlockextra()
}
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. It puts the current m back onto the extra list. 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. TODO(rsc): An alternative would be to allocate a dummy pthread per-thread variable using pthread_key_create. Unlike the pthread keys we already use on OS X, this dummy key would never be read by Go code. It would exist only so that we could register at thread-exit-time destructor. That destructor would put the m back onto the extra list. This is purely a performance optimization. The current version, in which dropm happens on each cgo call, is still correct too. We may have to keep the current version on systems with cgo but without pthreads, like Windows.
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
Return mp.curg to dead state.
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.
	 := .sigmask
	sigblock(false)
	unminit()

	 := lockextra(true)
	extraMCount++
	.schedlink.set()

	setg(nil)
Commit the release of mp.
	unlockextra()

	msigrestore()
}
A helper function for EnsureDropM.
func () uintptr {
	return uintptr(unsafe.Pointer(getg().m))
}

var extram uintptr
var extraMCount uint32 // Protected by lockextra
var extraMWaiters 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:nosplit
func ( bool) *m {
	const  = 1

	 := false
	for {
		 := atomic.Loaduintptr(&extram)
		if  ==  {
			osyield()
			continue
		}
		if  == 0 && ! {
Add 1 to the number of threads waiting for an M. This is cleared by newextram.
				atomic.Xadd(&extraMWaiters, 1)
				 = true
			}
			usleep(1)
			continue
		}
		if atomic.Casuintptr(&extram, , ) {
			return (*m)(unsafe.Pointer())
		}
		osyield()
		continue
	}
}
go:nosplit
execLock serializes exec and clone to avoid bugs or unspecified behaviour around exec'ing while creating/destroying threads. See issue #19546.
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 bool
	wake    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:nowritebarrierrec
func ( func(),  *p,  int64) {
	 := allocm(, , )
	.doesPark = ( != nil)
	.nextp.set()
	.sigmask = initSigmask
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.newm
		newmHandoff.newm.set()
		if newmHandoff.waiting {
			newmHandoff.waiting = false
			notewakeup(&newmHandoff.wake)
		}
		unlock(&newmHandoff.lock)
		return
	}
	newm1()
}

func ( *m) {
	if iscgo {
		var  cgothreadstart
		if _cgo_thread_start == nil {
			throw("_cgo_thread_start missing")
		}
		.g.set(.g0)
		.tls = (*uint64)(unsafe.Pointer(&.tls[0]))
		.fn = unsafe.Pointer(funcPC(mstart))
		if msanenabled {
			msanwrite(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 () {
	if GOARCH == "wasm" { // no threads on wasm yet
		return
	}
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()
}
mFixupRace is used to temporarily borrow the race context from the coordinating m during a syscall_runtime_doAllThreadsSyscall and loan it out to each of the m's of the runtime so they can execute a mFixup.fn in that context.
var mFixupRace struct {
	lock mutex
	ctx  uintptr
}
mDoFixup runs any outstanding fixup function for the running m. Returns true if a fixup was outstanding and actually executed.go:nosplit
func () bool {
	 := getg()
	lock(&.m.mFixup.lock)
	 := .m.mFixup.fn
	if  != nil {
We can't have a write barrier in this context since we may not have a P, but we clear fn to signal that we've executed the fixup. As long as fn is kept alive elsewhere, technically we should have no issues with the GC, but fn is likely generated in a different package altogether that may change independently. Just assert the GC is off so this lack of write barrier is more obviously safe.
			throw("GC must be disabled to protect validity of fn value")
		}
		*(*uintptr)(unsafe.Pointer(&.m.mFixup.fn)) = 0
		if .racectx != 0 || !raceenabled {
			(false)
temporarily acquire the context of the originator of the syscall_runtime_doAllThreadsSyscall and block others from using it for the duration of the fixup call.
			lock(&mFixupRace.lock)
			.racectx = mFixupRace.ctx
			(false)
			.racectx = 0
			unlock(&mFixupRace.lock)
		}
	}
	unlock(&.m.mFixup.lock)
	return  != nil
}
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:nowritebarrierrec
Stops execution of the current m until new work is available. Returns with acquired P.
func () {
	 := 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
}

startm's caller incremented nmspinning. Set the new M's spinning.
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 startm will either decrement nmspinning or 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. Must not have write barriers because this may be called without a P.go:nowritebarrierrec
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()
	lock(&sched.lock)
	if  == nil {
		 = pidleget()
		if  == nil {
			unlock(&sched.lock)
The caller incremented nmspinning, but there are no idle Ps, so it's okay to just undo the increment and give up.
				if int32(atomic.Xadd(&sched.nmspinning, -1)) < 0 {
					throw("startm: negative nmspinning")
				}
			}
			releasem()
			return
		}
	}
	 := mget()
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. 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()
The caller incremented nmspinning, so set m.spinning in the new M.
			 = mspinning
		}
Ownership transfer of _p_ committed by start in newm. Preemption is now safe.
		releasem()
		return
	}
	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()
Ownership transfer of _p_ 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:nowritebarrierrec
handoffp must start an M in any situation where findrunnable would return a G to run on _p_.
if it has local work, start it straight away
	if !runqempty() || sched.runqsize != 0 {
		startm(, false)
		return
if it has GC work, start it straight away
	if gcBlackenEnabled != 0 && gcMarkWorkAvailable() {
		startm(, false)
		return
no local work, check that there are no spinning/idle M's, otherwise our help is not required
	if atomic.Load(&sched.nmspinning)+atomic.Load(&sched.npidle) == 0 && atomic.Cas(&sched.nmspinning, 0, 1) { // TODO: fast atomic
		startm(, true)
		return
	}
	lock(&sched.lock)
	if sched.gcwaiting != 0 {
		.status = _Pgcstop
		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.runqsize != 0 {
		unlock(&sched.lock)
		startm(, 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 == uint32(gomaxprocs-1) && atomic.Load64(&sched.lastpoll) != 0 {
		unlock(&sched.lock)
		startm(, false)
		return
	}
The scheduler lock cannot be held when calling wakeNetPoller below because wakeNetPoller may call wakep which may call startm.
	 := nobarrierWakeTime()
	pidleput()
	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).
func () {
	if atomic.Load(&sched.npidle) == 0 {
		return
be conservative about spinning threads
	if atomic.Load(&sched.nmspinning) != 0 || !atomic.Cas(&sched.nmspinning, 0, 1) {
		return
	}
	startm(nil, true)
}
Stops execution of the current m that is locked to a g until the g is runnable again. Returns with acquired P.
func () {
	 := getg()

	if .m.lockedg == 0 || .m.lockedg.ptr().lockedm.ptr() != .m {
		throw("stoplockedm: inconsistent locking")
	}
Schedule another M to run this p.
		 := releasep()
		handoffp()
	}
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:nowritebarrierrec
func ( *g) {
	 := getg()

	 := .lockedm.ptr()
	if  == .m {
		throw("startlockedm: locked to me")
	}
	if .nextp != 0 {
		throw("startlockedm: m has p")
directly handoff current P to the locked m
	incidlelocked(-1)
	 := releasep()
	.nextp.set()
	notewakeup(&.park)
	stopm()
}
Stops the current m for stopTheWorld. Returns when the world is restarted.
func () {
	 := getg()

	if sched.gcwaiting == 0 {
		throw("gcstopm: not waiting for gc")
	}
	if .m.spinning {
OK to just drop nmspinning here, startTheWorld will unpark threads as necessary.
		if int32(atomic.Xadd(&sched.nmspinning, -1)) < 0 {
			throw("gcstopm: negative nmspinning")
		}
	}
	 := releasep()
	lock(&sched.lock)
	.status = _Pgcstop
	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:yeswritebarrierrec
func ( *g,  bool) {
	 := getg()
Assign gp.m before entering _Grunning so running Gs have an M.
	.m.curg = 
	.m = .m
	casgstatus(, _Grunnable, _Grunning)
	.waitsince = 0
	.preempt = false
	.stackguard0 = .stack.lo + _StackGuard
	if ! {
		.m.p.ptr().schedtick++
	}
Check whether the profiler needs to be turned on or off.
	 := sched.profilehz
	if .m.profilehz !=  {
		setThreadCPUProfiler()
	}

GoSysExit has to happen when we have a P, but before GoStart. So we emit it here.
Finds a runnable goroutine to execute. Tries to steal from other P's, get g from local or global queue, poll network.
func () ( *g,  bool) {
	 := getg()
The conditions here and in handoffp must agree: if findrunnable would return a G to run, handoffp must start an M.

:
	 := .m.p.ptr()
	if sched.gcwaiting != 0 {
		gcstopm()
		goto 
	}
	if .runSafePointFn != 0 {
		runSafePointFn()
	}

	, ,  := checkTimers(, 0)

	if fingwait && fingwake {
		if  := wakefing();  != nil {
			ready(, 0, true)
		}
	}
	if *cgo_yield != nil {
		asmcgocall(*cgo_yield, nil)
	}
local runq
	if ,  := runqget();  != nil {
		return , 
	}
global runq
	if sched.runqsize != 0 {
		lock(&sched.lock)
		 := globrunqget(, 0)
		unlock(&sched.lock)
		if  != nil {
			return , 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.
	if netpollinited() && atomic.Load(&netpollWaiters) > 0 && atomic.Load64(&sched.lastpoll) != 0 {
		if  := netpoll(0); !.empty() { // non-blocking
			 := .pop()
			injectglist(&)
			casgstatus(, _Gwaiting, _Grunnable)
			if trace.enabled {
				traceGoUnpark(, 0)
			}
			return , false
		}
	}
Steal work from other P's.
	 := uint32(gomaxprocs)
If number of spinning M's >= number of busy P's, block. This is necessary to prevent excessive CPU consumption when GOMAXPROCS>>1 but the program parallelism is low.
	if !.m.spinning && 2*atomic.Load(&sched.nmspinning) >= -atomic.Load(&sched.npidle) {
		goto 
	}
	if !.m.spinning {
		.m.spinning = true
		atomic.Xadd(&sched.nmspinning, 1)
	}
	const  = 4
	for  := 0;  < ; ++ {
		 :=  == -1

		for  := stealOrder.start(fastrand()); !.done(); .next() {
			if sched.gcwaiting != 0 {
				goto 
			}
			 := 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()) {
				, ,  := checkTimers(, )
				 = 
				if  != 0 && ( == 0 ||  < ) {
					 = 
				}
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 is 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
				}
			}
		}
	}
Running a timer may have made some goroutine ready.
		goto 
	}

:
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() {
		 := (*gcBgMarkWorkerNode)(gcBgMarkWorkerPool.pop())
		if  != nil {
			.gcMarkWorkerMode = gcMarkWorkerIdleMode
			 := .gp.ptr()
			casgstatus(, _Gwaiting, _Grunnable)
			if trace.enabled {
				traceGoUnpark(, 0)
			}
			return , false
		}
	}

	 := int64(-1)
checkTimers ensures that polluntil > now.
		 =  - 
	}
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 {
		casgstatus(, _Gwaiting, _Grunnable)
		if trace.enabled {
			traceGoUnpark(, 0)
		}
		return , 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.
Also snapshot masks. Value changes are OK, but we can't allow len to change out from under us.
	 := idlepMask
	 := timerpMask
return P and block
	lock(&sched.lock)
	if sched.gcwaiting != 0 || .runSafePointFn != 0 {
		unlock(&sched.lock)
		goto 
	}
	if sched.runqsize != 0 {
		 := globrunqget(, 0)
		unlock(&sched.lock)
		return , false
	}
	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 goroutines. We must drop nmspinning first and then check all per-P queues again (with #StoreLoad memory barrier in between). If we do it the other way around, another thread can submit a goroutine after we've checked all run queues but before we drop nmspinning; as a result nobody will unpark a thread to run the goroutine. 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, it is OK to just park the current thread: the system is fully loaded so no spinning threads are required. Also see "Worker thread parking/unparking" comment at the top of the file.
	 := .m.spinning
	if .m.spinning {
		.m.spinning = false
		if int32(atomic.Xadd(&sched.nmspinning, -1)) < 0 {
			throw("findrunnable: negative nmspinning")
		}
	}
check all runqueues once again
	for ,  := range  {
		if !.read(uint32()) && !runqempty() {
			lock(&sched.lock)
			 = pidleget()
			unlock(&sched.lock)
			if  != nil {
				acquirep()
				if  {
					.m.spinning = true
					atomic.Xadd(&sched.nmspinning, 1)
				}
				goto 
			}
			break
		}
	}
Similar to above, 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.
	for ,  := range  {
		if .read(uint32()) {
			 := nobarrierWakeTime()
			if  != 0 && ( == 0 ||  < ) {
				 = 
			}
		}
	}
	if  != 0 {
		if  == 0 {
			 = nanotime()
		}
		 =  - 
		if  < 0 {
			 = 0
		}
	}
Check for idle-priority GC work again. N.B. Since we have no P, gcBlackenEnabled may change at any time; we must check again after acquiring a P.
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. 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.
		lock(&sched.lock)
		var  *gcBgMarkWorkerNode
		 = pidleget()
Now that we own a P, gcBlackenEnabled can't change (as it requires STW).
			if gcBlackenEnabled != 0 {
				 = (*gcBgMarkWorkerNode)(gcBgMarkWorkerPool.pop())
				if  == nil {
					pidleput()
					 = nil
				}
			} else {
				pidleput()
				 = nil
			}
		}
		unlock(&sched.lock)
		if  != nil {
			acquirep()
			if  {
				.m.spinning = true
				atomic.Xadd(&sched.nmspinning, 1)
			}
Run the idle worker.
poll network
	if netpollinited() && (atomic.Load(&netpollWaiters) > 0 ||  != 0) && atomic.Xchg64(&sched.lastpoll, 0) != 0 {
		atomic.Store64(&sched.pollUntil, uint64())
		if .m.p != 0 {
			throw("findrunnable: netpoll with p")
		}
		if .m.spinning {
			throw("findrunnable: netpoll with spinning")
		}
When using fake time, just poll.
			 = 0
		}
		 := netpoll() // block until new work is available
		atomic.Store64(&sched.pollUntil, 0)
		atomic.Store64(&sched.lastpoll, uint64(nanotime()))
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(&)
		} else {
			acquirep()
			if !.empty() {
				 := .pop()
				injectglist(&)
				casgstatus(, _Gwaiting, _Grunnable)
				if trace.enabled {
					traceGoUnpark(, 0)
				}
				return , false
			}
			if  {
				.m.spinning = true
				atomic.Xadd(&sched.nmspinning, 1)
			}
			goto 
		}
	} else if  != 0 && netpollinited() {
		 := int64(atomic.Load64(&sched.pollUntil))
		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 () bool {
	if sched.runqsize != 0 {
		return true
	}
	 := getg().m.p.ptr()
	if !runqempty() {
		return true
	}
	if netpollinited() && atomic.Load(&netpollWaiters) > 0 && sched.lastpoll != 0 {
		if  := netpoll(0); !.empty() {
			injectglist(&)
			return true
		}
	}
	return false
}
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 ( int64) {
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.
		 := int64(atomic.Load64(&sched.pollUntil))
		if  == 0 ||  >  {
			netpollBreak()
		}
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 () {
	 := getg()
	if !.m.spinning {
		throw("resetspinning: not a spinning m")
	}
	.m.spinning = false
	 := atomic.Xadd(&sched.nmspinning, -1)
	if int32() < 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 ( *gList) {
	if .empty() {
		return
	}
	if trace.enabled {
		for  := .head.ptr();  != nil;  = .schedlink.ptr() {
			traceGoUnpark(, 0)
		}
	}
Mark all the goroutines as runnable before we put them on the run queues.
	 := .head.ptr()
	var  *g
	 := 0
	for  := ;  != nil;  = .schedlink.ptr() {
		 = 
		++
		casgstatus(, _Gwaiting, _Grunnable)
	}
Turn the gList into a gQueue.
	var  gQueue
	.head.set()
	.tail.set()
	* = gList{}

	 := func( int) {
		for ;  != 0 && sched.npidle != 0; -- {
			startm(nil, false)
		}
	}

	 := getg().m.p.ptr()
	if  == nil {
		lock(&sched.lock)
		globrunqputbatch(&, int32())
		unlock(&sched.lock)
		()
		return
	}

	 := int(atomic.Load(&sched.npidle))
	var  gQueue
	var  int
	for  = 0;  <  && !.empty(); ++ {
		 := .pop()
		.pushBack()
	}
	if  > 0 {
		lock(&sched.lock)
		globrunqputbatch(&, int32())
		unlock(&sched.lock)
		()
		 -= 
	}

	if !.empty() {
		runqputbatch(, &, )
	}
}
One round of scheduler: find a runnable goroutine and execute it. Never returns.
func () {
	 := getg()

	if .m.locks != 0 {
		throw("schedule: holding locks")
	}

	if .m.lockedg != 0 {
		stoplockedm()
		execute(.m.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 .m.incgo {
		throw("schedule: in cgo")
	}

:
	 := .m.p.ptr()
	.preempt = false

	if sched.gcwaiting != 0 {
		gcstopm()
		goto 
	}
	if .runSafePointFn != 0 {
		runSafePointFn()
	}
Sanity 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 .m.spinning && (.runnext != 0 || .runqhead != .runqtail) {
		throw("schedule: spinning with local work")
	}

	checkTimers(, 0)

	var  *g
	var  bool
Normal goroutines will check for need to wakeP in ready, but GCworkers and tracereaders will not, so the check must be done here instead.
	 := false
	if trace.enabled || trace.shutdown {
		 = traceReader()
		if  != nil {
			casgstatus(, _Gwaiting, _Grunnable)
			traceGoUnpark(, 0)
			 = true
		}
	}
	if  == nil && gcBlackenEnabled != 0 {
		 = gcController.findRunnableGCWorker(.m.p.ptr())
		 =  ||  != nil
	}
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 .m.p.ptr().schedtick%61 == 0 && sched.runqsize > 0 {
			lock(&sched.lock)
			 = globrunqget(.m.p.ptr(), 1)
			unlock(&sched.lock)
		}
	}
	if  == nil {
We can see gp != nil here even if the M is spinning, if checkTimers added a local goroutine via goready.
	}
	if  == nil {
		,  = findrunnable() // blocks until work is available
	}
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 .m.spinning {
		resetspinning()
	}

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.
Something re-enabled scheduling while we were acquiring the lock.
If about to schedule a not-normal goroutine (a GCworker or tracereader), wake a P if there is one.
	if  {
		wakep()
	}
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 () {
	 := getg()

	setMNoWB(&.m.curg.m, nil)
	setGNoWB(&.m.curg, nil)
}
checkTimers runs any timers for the P that are ready. If now is not 0 it is the current time. It returns the current time or 0 if it is not known, and the time when the next timer should run or 0 if there is no next timer, and reports whether it ran any timers. If the time when the next timer should run is not 0, it is always larger than the returned time. We pass now in and out to avoid extra calls of nanotime.go:yeswritebarrierrec
If it's not yet time for the first timer, or the first adjusted timer, then there is nothing to do.
	 := int64(atomic.Load64(&.timer0When))
	 := int64(atomic.Load64(&.timerModifiedEarliest))
	if  == 0 || ( != 0 &&  < ) {
		 = 
	}

No timers to run or adjust.
		return , 0, false
	}

	if  == 0 {
		 = nanotime()
	}
Next timer is not ready to run, but keep going if we would clear deleted timers. This corresponds to the condition below where we decide whether to call clearDeletedTimers.
		if  != getg().m.p.ptr() || int(atomic.Load(&.deletedTimers)) <= int(atomic.Load(&.numTimers)/4) {
			return , , false
		}
	}

	lock(&.timersLock)

	if len(.timers) > 0 {
		adjusttimers(, )
Note that runtimer may temporarily unlock pp.timersLock.
			if  := runtimer(, );  != 0 {
				if  > 0 {
					 = 
				}
				break
			}
			 = true
		}
	}
If this is the local P, and there are a lot of deleted timers, clear them out. We only do this for the local P to reduce lock contention on timersLock.
	if  == getg().m.p.ptr() && int(atomic.Load(&.deletedTimers)) > len(.timers)/4 {
		clearDeletedTimers()
	}

	unlock(&.timersLock)

	return , , 
}

func ( *g,  unsafe.Pointer) bool {
	unlock((*mutex)())
	return true
}
park continuation on g0.
func ( *g) {
	 := getg()

	if trace.enabled {
		traceGoPark(.m.waittraceev, .m.waittraceskip)
	}

	casgstatus(, _Grunning, _Gwaiting)
	dropg()

	if  := .m.waitunlockf;  != nil {
		 := (, .m.waitlock)
		.m.waitunlockf = nil
		.m.waitlock = nil
		if ! {
			if trace.enabled {
				traceGoUnpark(, 2)
			}
			casgstatus(, _Gwaiting, _Grunnable)
			execute(, true) // Schedule it back, never returns.
		}
	}
	schedule()
}

func ( *g) {
	 := readgstatus()
	if &^_Gscan != _Grunning {
		dumpgstatus()
		throw("bad g status")
	}
	casgstatus(, _Grunning, _Grunnable)
	dropg()
	lock(&sched.lock)
	globrunqput()
	unlock(&sched.lock)

	schedule()
}
Gosched continuation on g0.
func ( *g) {
	if trace.enabled {
		traceGoSched()
	}
	goschedImpl()
}
goschedguarded is a forbidden-states-avoided version of gosched_m
func ( *g) {

	if !canPreemptM(.m) {
		gogo(&.sched) // never return
	}

	if trace.enabled {
		traceGoSched()
	}
	goschedImpl()
}

func ( *g) {
	if trace.enabled {
		traceGoPreempt()
	}
	goschedImpl()
}
preemptPark parks gp and puts it in _Gpreempted.go:systemstack
func ( *g) {
	if trace.enabled {
		traceGoPark(traceEvGoBlock, 0)
	}
	 := readgstatus()
	if &^_Gscan != _Grunning {
		dumpgstatus()
		throw("bad g status")
	}
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.
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
func () {
	checkTimeouts()
	mcall(goyield_m)
}

func ( *g) {
	if trace.enabled {
		traceGoPreempt()
	}
	 := .m.p.ptr()
	casgstatus(, _Grunning, _Grunnable)
	dropg()
	runqput(, , false)
	schedule()
}
Finishes execution of the current goroutine.
goexit continuation on g0.
func ( *g) {
	 := getg()

	casgstatus(, _Grunning, _Gdead)
	if isSystemGoroutine(, false) {
		atomic.Xadd(&sched.ngsys, -1)
	}
	.m = nil
	 := .lockedm != 0
	.lockedm = 0
	.m.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 = 0
	.param = nil
	.labels = nil
	.timer = nil

Flush assist credit to the global pool. This gives better information to pacing if the application is rapidly creating an exiting goroutines.
		 := float64frombits(atomic.Load64(&gcController.assistWorkPerByte))
		 := int64( * float64(.gcAssistBytes))
		atomic.Xaddint64(&gcController.bgScanCredit, )
		.gcAssistBytes = 0
	}

	dropg()

	if GOARCH == "wasm" { // no threads yet on wasm
		gfput(.m.p.ptr(), )
		schedule() // never returns
	}

	if .m.lockedInt != 0 {
		print("invalid m->lockedInt = ", .m.lockedInt, "\n")
		throw("internal lockOSThread error")
	}
	gfput(.m.p.ptr(), )
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(&.m.g0.sched)
Clear lockedExt on plan9 since we may end up re-using this thread.
			.m.lockedExt = 0
		}
	}
	schedule()
}
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:nosplitgo:nowritebarrierrec
func (,  uintptr) {
	 := getg()

	.sched.pc = 
	.sched.sp = 
	.sched.lr = 0
	.sched.ret = 0
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 gosave 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. Syscall tracing: At the start of a syscall we emit traceGoSysCall to capture the stack trace. If the syscall does not block, that is it, we do not emit any other events. If the syscall blocks (that is, P is retaken), retaker emits traceGoSysBlock; when syscall returns we emit traceGoSysExit and when the goroutine starts running (potentially instantly, if exitsyscallfast returns true) we emit traceGoStart. To ensure that traceGoSysExit is emitted strictly after traceGoSysBlock, we remember current value of syscalltick in m (_g_.m.syscalltick = _g_.m.p.ptr().syscalltick), whoever emits traceGoSysBlock increments p.syscalltick afterwards; and we wait for the increment before emitting traceGoSysExit. Note that the increment is done even if tracing is not enabled, because tracing can be enabled in the middle of syscall. We don't want the wait to hang.go:nosplit
func (,  uintptr) {
	 := 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.
Leave SP around for GC and traceback.
	save(, )
	.syscallsp = 
	.syscallpc = 
	casgstatus(, _Grunning, _Gsyscall)
	if .syscallsp < .stack.lo || .stack.hi < .syscallsp {
		systemstack(func() {
			print("entersyscall inconsistent ", hex(.syscallsp), " [", hex(.stack.lo), ",", hex(.stack.hi), "]\n")
			throw("entersyscall")
		})
	}

	if trace.enabled {
systemstack itself clobbers g.sched.{pc,sp} and we might need them later when the G is genuinely blocked in a syscall
		save(, )
	}

	if atomic.Load(&sched.sysmonwait) != 0 {
		systemstack(entersyscall_sysmon)
		save(, )
	}

runSafePointFn may stack split if run on this stack
		systemstack(runSafePointFn)
		save(, )
	}

	.m.syscalltick = .m.p.ptr().syscalltick
	.sysblocktraced = true
	 := .m.p.ptr()
	.m = 0
	.m.oldp.set()
	.m.p = 0
	atomic.Store(&.status, _Psyscall)
	if sched.gcwaiting != 0 {
		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.go:nosplitgo:linkname entersyscall
The same as entersyscall(), but with a hint that the syscall is blocking.go:nosplit
func () {
	 := getg()

	.m.locks++ // see comment in entersyscall
	.throwsplit = true
	.stackguard0 = stackPreempt // see comment in entersyscall
	.m.syscalltick = .m.p.ptr().syscalltick
	.sysblocktraced = true
	.m.p.ptr().syscalltick++
Leave SP around for GC and traceback.
	 := getcallerpc()
	 := getcallersp()
	save(, )
	.syscallsp = .sched.sp
	.syscallpc = .sched.pc
	if .syscallsp < .stack.lo || .stack.hi < .syscallsp {
		 := 
		 := .sched.sp
		 := .syscallsp
		systemstack(func() {
			print("entersyscallblock inconsistent ", 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 ", hex(), " ", hex(.sched.sp), " ", hex(.syscallsp), " [", hex(.stack.lo), ",", hex(.stack.hi), "]\n")
			throw("entersyscallblock")
		})
	}

	systemstack(entersyscallblock_handoff)
Resave for traceback during blocked call.
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.go:nosplitgo:nowritebarrierrecgo:linkname exitsyscall
func () {
	 := getg()

	.m.locks++ // see comment in entersyscall
	if getcallersp() > .syscallsp {
		throw("exitsyscall: syscall frame is no longer valid")
	}

	.waitsince = 0
	 := .m.oldp.ptr()
	.m.oldp = 0
	if exitsyscallfast() {
		if trace.enabled {
			if  != .m.p.ptr() || .m.syscalltick != .m.p.ptr().syscalltick {
				systemstack(traceGoStart)
			}
There's a cpu for us, so we can run.
We need to cas the status and scan before resuming...
Garbage collector isn't running (since we are), so okay to clear syscallsp.
		.syscallsp = 0
		.m.locks--
restore the preemption request in case we've cleared it in newstack
otherwise restore the real _StackGuard, we've spoiled it in entersyscall/entersyscallblock
Scheduling of this goroutine is disabled.
			Gosched()
		}

		return
	}

	.sysexitticks = 0
Wait till traceGoSysBlock event is emitted. This ensures consistency of the trace (the goroutine is started after it is blocked).
		for  != nil && .syscalltick == .m.syscalltick {
			osyield()
We can't trace syscall exit right now because we don't have a P. Tracing code can invoke write barriers that cannot run without a P. So instead we remember the syscall exit time and emit the event in execute when we have a P.
		.sysexitticks = cputicks()
	}

	.m.locks--
Call the scheduler.
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:nosplit
func ( *p) bool {
	 := getg()
Freezetheworld sets stopwait but does not retake P's.
	if sched.stopwait == freezeStopWait {
		return false
	}
Try to re-acquire the last P.
There's a cpu for us, so we can run.
Try to get any other idle P.
	if sched.pidle != 0 {
		var  bool
		systemstack(func() {
			 = exitsyscallfast_pidle()
			if  && trace.enabled {
Wait till traceGoSysBlock event is emitted. This ensures consistency of the trace (the goroutine is started after it is blocked).
					for .syscalltick == .m.syscalltick {
						osyield()
					}
				}
				traceGoSysExit(0)
			}
		})
		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:nosplit
func () {
	 := getg()
	if .m.syscalltick != .m.p.ptr().syscalltick {
The p was retaken and then enter into syscall again (since _g_.m.syscalltick has changed). traceGoSysBlock for this syscall was already emitted, but here we effectively retake the p from the new syscall running on the same p.
Denote blocking of the new syscall.
Denote completion of the current syscall.
				traceGoSysExit(0)
			})
		}
		.m.p.ptr().syscalltick++
	}
}

func () bool {
	lock(&sched.lock)
	 := pidleget()
	if  != nil && atomic.Load(&sched.sysmonwait) != 0 {
		atomic.Store(&sched.sysmonwait, 0)
		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.go:nowritebarrierrec
func ( *g) {
	 := getg()

	casgstatus(, _Gsyscall, _Grunnable)
	dropg()
	lock(&sched.lock)
	var  *p
	if schedEnabled() {
		 = pidleget()
	}
	if  == nil {
		globrunqput()
	} else if atomic.Load(&sched.sysmonwait) != 0 {
		atomic.Store(&sched.sysmonwait, 0)
		notewakeup(&sched.sysmonnote)
	}
	unlock(&sched.lock)
	if  != nil {
		acquirep()
		execute(, false) // Never returns.
	}
Wait until another thread schedules gp and so m again.
		stoplockedm()
		execute(, false) // Never returns.
	}
	stopm()
	schedule() // Never returns.
}

func () {
	 := 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.
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->_StackGuard to reliably detect any attempts to grow stack. runtime_AfterFork will undo this in parent process, but not in child.
Called from syscall package before fork.go:linkname syscall_runtime_BeforeFork syscall.runtime_BeforeForkgo:nosplit
See the comments in beforefork.
Called from syscall package after fork in parent.go:linkname syscall_runtime_AfterFork syscall.runtime_AfterForkgo:nosplit
inForkedChild is true while manipulating signals in the child process. This is used to avoid calling libc functions in case we are using vfork.
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.go:linkname syscall_runtime_AfterForkInChild syscall.runtime_AfterForkInChildgo:nosplitgo:nowritebarrierrec
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.
When we are the child we are the only thread running, so we know that nothing else has changed gp.m.sigmask.
pendingPreemptSignals is the number of preemption signals that have been sent but not received. This is only used on Darwin. For #41702.
Called from syscall package before Exec.go:linkname syscall_runtime_BeforeExec syscall.runtime_BeforeExec
Prevent thread creation during exec.
On Darwin, wait for all pending preemption signals to be received. See issue #41702.
	if GOOS == "darwin" || GOOS == "ios" {
		for int32(atomic.Load(&pendingPreemptSignals)) > 0 {
			osyield()
		}
	}
}
Called from syscall package after Exec.go:linkname syscall_runtime_AfterExec syscall.runtime_AfterExec
Allocate a new g, with a stack big enough for stacksize bytes.
func ( int32) *g {
	 := new(g)
	if  >= 0 {
		 = round2(_StackSystem + )
		systemstack(func() {
			.stack = stackalloc(uint32())
		})
		.stackguard0 = .stack.lo + _StackGuard
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 with siz bytes of arguments. Put it on the queue of g's waiting to run. The compiler turns a go statement into a call to this. The stack layout of this call is unusual: it assumes that the arguments to pass to fn are on the stack sequentially immediately after &fn. Hence, they are logically part of newproc's argument frame, even though they don't appear in its signature (and can't because their types differ between call sites). This must be nosplit because this stack layout means there are untyped arguments in newproc's argument frame. Stack copies won't be able to adjust them and stack splits won't be able to copy them.go:nosplit
func ( int32,  *funcval) {
	 := add(unsafe.Pointer(&), sys.PtrSize)
	 := getg()
	 := getcallerpc()
	systemstack(func() {
		 := newproc1(, , , , )

		 := getg().m.p.ptr()
		runqput(, , true)

		if mainStarted {
			wakep()
		}
	})
}
Create a new g in state _Grunnable, starting at fn, with narg bytes of arguments starting at argp. callerpc is the address of the go statement that created this. The caller is responsible for adding the new g to the scheduler. This must run on the system stack because it's the continuation of newproc, which cannot split the stack.go:systemstack
func ( *funcval,  unsafe.Pointer,  int32,  *g,  uintptr) *g {
	 := getg()

	if  == nil {
		.m.throwing = -1 // do not dump full stacks
		throw("go of nil func value")
	}
	acquirem() // disable preemption because it can be holding p in a local var
	 := 
	 = ( + 7) &^ 7
We could allocate a larger initial stack if necessary. Not worth it: this is almost always an error. 4*sizeof(uintreg): extra space added below sizeof(uintreg): caller's LR (arm) or return address (x86, in gostartcall).
	if  >= _StackMin-4*sys.RegSize-sys.RegSize {
		throw("newproc: function arguments too large for new goroutine")
	}

	 := .m.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")
	}

	 := 4*sys.RegSize + uintptr() + sys.MinFrameSize // extra space in case of reads slightly beyond frame
	 += - & (sys.SpAlign - 1)                  // align to spAlign
	 := .stack.hi - 
	 := 
caller's LR
		*(*uintptr)(unsafe.Pointer()) = 0
		prepGoExitFrame()
		 += sys.MinFrameSize
	}
	if  > 0 {
This is a stack-to-stack copy. If write barriers are enabled and the source stack is grey (the destination is always black), then perform a barrier copy. We do this *after* the memmove because the destination stack may have garbage on it.
We're in the prologue, so it's always stack map index 0.
				 := stackmapdata(, 0)
				bulkBarrierBitmap(, , uintptr(.n)*sys.PtrSize, 0, .bytedata)
			}
		}
	}

	memclrNoHeapPointers(unsafe.Pointer(&.sched), unsafe.Sizeof(.sched))
	.sched.sp = 
	.stktopsp = 
	.sched.pc = funcPC(goexit) + sys.PCQuantum // +PCQuantum so that previous instruction is in same function
	.sched.g = guintptr(unsafe.Pointer())
	gostartcallfn(&.sched, )
	.gopc = 
	.ancestors = saveAncestors()
	.startpc = .fn
	if .m.curg != nil {
		.labels = .m.curg.labels
	}
	if isSystemGoroutine(, false) {
		atomic.Xadd(&sched.ngsys, +1)
	}
	casgstatus(, _Gdead, _Grunnable)

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 = atomic.Xadd64(&sched.goidgen, _GoidCacheBatch)
		.goidcache -= _GoidCacheBatch - 1
		.goidcacheend = .goidcache + _GoidCacheBatch
	}
	.goid = int64(.goidcache)
	.goidcache++
	if raceenabled {
		.racectx = racegostart()
	}
	if trace.enabled {
		traceGoCreate(, .startpc)
	}
	releasem(.m)

	return 
}
saveAncestors copies previous ancestors of the given caller g and includes infor for the current caller into a new set of tracebacks for a g being created.
Copy all prior info, except for the root goroutine (goid 0).
	if debug.tracebackancestors <= 0 || .goid == 0 {
		return nil
	}
	var  []ancestorInfo
	if .ancestors != nil {
		 = *.ancestors
	}
	 := int32(len()) + 1
	if  > debug.tracebackancestors {
		 = debug.tracebackancestors
	}
	 := make([]ancestorInfo, )
	copy([1:], )

	var  [_TracebackMaxFrames]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 ( *p,  *g) {
	if readgstatus() != _Gdead {
		throw("gfput: bad status (not Gdead)")
	}

	 := .stack.hi - .stack.lo

non-standard stack size - free it.
		stackfree(.stack)
		.stack.lo = 0
		.stack.hi = 0
		.stackguard0 = 0
	}

	.gFree.push()
	.gFree.n++
	if .gFree.n >= 64 {
		lock(&sched.gFree.lock)
		for .gFree.n >= 32 {
			.gFree.n--
			 = .gFree.pop()
			if .stack.lo == 0 {
				sched.gFree.noStack.push()
			} else {
				sched.gFree.stack.push()
			}
			sched.gFree.n++
		}
		unlock(&sched.gFree.lock)
	}
}
Get from gfree list. If local list is empty, grab a batch from global list.
func ( *p) *g {
:
	if .gFree.empty() && (!sched.gFree.stack.empty() || !sched.gFree.noStack.empty()) {
Move a batch of free Gs to the P.
Prefer Gs with stacks.
			 := sched.gFree.stack.pop()
			if  == nil {
				 = sched.gFree.noStack.pop()
				if  == nil {
					break
				}
			}
			sched.gFree.n--
			.gFree.push()
			.gFree.n++
		}
		unlock(&sched.gFree.lock)
		goto 
	}
	 := .gFree.pop()
	if  == nil {
		return nil
	}
	.gFree.n--
Stack was deallocated in gfput. Allocate a new one.
		systemstack(func() {
			.stack = stackalloc(_FixedStack)
		})
		.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)
		}
	}
	return 
}
Purge all cached G's from gfree list to the global list.
func ( *p) {
	lock(&sched.gFree.lock)
	for !.gFree.empty() {
		 := .gFree.pop()
		.gFree.n--
		if .stack.lo == 0 {
			sched.gFree.noStack.push()
		} else {
			sched.gFree.stack.push()
		}
		sched.gFree.n++
	}
	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:nosplit
func () {
	if GOARCH == "wasm" {
		return // no threads on wasm yet
	}
	 := getg()
	.m.lockedg.set()
	.lockedm.set(.m)
}
go:nosplit
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.
func () {
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:nosplit
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:nosplit
func () {
	if GOARCH == "wasm" {
		return // no threads on wasm yet
	}
	 := getg()
	if .m.lockedInt != 0 || .m.lockedExt != 0 {
		return
	}
	.m.lockedg = 0
	.lockedm = 0
}
go:nosplit
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.
func () {
	 := getg()
	if .m.lockedExt == 0 {
		return
	}
	.m.lockedExt--
	dounlockOSThread()
}
go:nosplit
func () {
	 := getg()
	if .m.lockedInt == 0 {
		systemstack(badunlockosthread)
	}
	.m.lockedInt--
	dounlockOSThread()
}

func () {
	throw("runtime: internal error: misuse of lockOSThread/unlockOSThread")
}

func () int32 {
	 := int32(atomic.Loaduintptr(&allglen)) - sched.gFree.n - int32(atomic.Load(&sched.ngsys))
	for ,  := range allp {
		 -= .gFree.n
	}
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 () int32 {
	return int32(sched.mnext - sched.nmfreed)
}

var prof struct {
	signalLock uint32
	hz         int32
}

func ()                    { () }
func ()              { () }
func ()          { () }
func ()                        { () }
func () { () }
func ()                      { () }
Called if we receive a SIGPROF signal. Called by the signal handler, may run during STW.go:nowritebarrierrec
func (, ,  uintptr,  *g,  *m) {
	if prof.hz == 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}, 64bit atomics are emulated with spinlocks, in runtime/internal/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 hasPrefix(funcname(), "runtime/internal/atomic") {
				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.
Define that a "user g" is a user-created goroutine, and a "system g" is one that is m->g0 or m->gsignal. We might be interrupted for profiling halfway through a goroutine switch. The switch involves updating three (or four) values: g, PC, SP, and (on arm) LR. The PC must be the last to be updated, because once it gets updated the new g is running. When switching from a user g to a system g, LR is not considered live, so the update only affects g, SP, and PC. Since PC must be last, there the possible partial transitions in ordinary execution are (1) g alone is updated, (2) both g and SP are updated, and (3) SP alone is updated. If SP or g alone is updated, we can detect the partial transition by checking whether the SP is within g's stack bounds. (We could also require that SP be changed only after g, but the stack bounds check is needed by other cases, so there is no need to impose an additional requirement.) There is one exceptional transition to a system g, not in ordinary execution. When a signal arrives, the operating system starts the signal handler running with an updated PC and SP. The g is updated last, at the beginning of the handler. There are two reasons this is okay. First, until g is updated the g and SP do not match, so the stack bounds check detects the partial transition. Second, signal handlers currently run with signals disabled, so a profiling signal cannot arrive during the handler. When switching from a system g to a user g, there are three possibilities. First, it may be that the g switch has no PC update, because the SP either corresponds to a user g throughout (as in asmcgocall) or because it has been arranged to look like a user g frame (as in cgocallback). In this case, since the entire transition is a g+SP update, a partial transition updating just one of those will be detected by the stack bounds check. Second, when returning from a signal handler, the PC and SP updates are performed by the operating system in an atomic update, so the g update must be done before them. The stack bounds check detects the partial transition here, and (again) signal handlers run with signals disabled, so a profiling signal cannot arrive then anyway. Third, the common case: it may be that the switch updates g, SP, and PC separately. If the PC is within any of the functions that does this, we don't ask for a traceback. C.F. the function setsSP for more about this. There is another apparently viable approach, recorded here in case the "PC within setsSP function" check turns out not to be usable. It would be possible to delay the update of either g or SP until immediately before the PC update instruction. Then, because of the stack bounds check, the only problematic interrupt point is just before that PC update instruction, and the sigprof handler can detect that instruction and simulate stepping past it in order to reach a consistent state. On ARM, the update of g must be made in two places (in R10 and also in a TLS slot), so the delayed update would need to be the SP update. The sigprof handler must read the instruction at the current PC and if it was the known instruction (for example, JMP BX or MOV R2, PC), use that other register in place of the PC value. The biggest drawback to this solution is that it requires that we can tell whether it's safe to read from the memory pointed at by PC. In a correct program, we can test PC == nil and otherwise read, but if a profiling signal happens at the instant that a program executes a bad jump (before the program manages to handle the resulting fault) the profiling handler could fault trying to read nonexistent memory. To recap, there are no constraints on the assembly being used for the transition. We simply require that g and SP match and that the PC is not in gogo.
	 := true
	if  == nil ||  < .stack.lo || .stack.hi <  || setsSP() || ( != nil && .vdsoSP != 0) {
		 = false
	}
	var  [maxCPUProfStack]uintptr
	 := 0
	if .ncgo > 0 && .curg != nil && .curg.syscallpc != 0 && .curg.syscallsp != 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 atomic.Load(&.cgoCallersUse) == 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.
		 = gentraceback(.curg.syscallpc, .curg.syscallsp, 0, .curg, 0, &[], len()-, nil, nil, 0)
		if  > 0 {
			 += 
		}
	} else if  {
		 = gentraceback(, , , , 0, &[0], len(), nil, nil, _TraceTrap|_TraceJumpStack)
	}

Normal traceback is impossible or has failed. See if it falls into several common cases.
		 = 0
Libcall, i.e. runtime syscall on windows. Collect Go stack that leads to the call.
			 = gentraceback(.libcallpc, .libcallsp, 0, .libcallg.ptr(), 0, &[0], len(), nil, nil, 0)
		}
		if  == 0 &&  != nil && .vdsoSP != 0 {
			 = gentraceback(.vdsoPC, .vdsoSP, 0, , 0, &[0], len(), nil, nil, _TraceTrap|_TraceJumpStack)
		}
If all of the above has failed, account it against abstract "System" or "GC".
			 = 2
			if inVDSOPage() {
				 = funcPC(_VDSO) + sys.PCQuantum
"ExternalCode" is better than "etext".
				 = funcPC(_ExternalCode) + sys.PCQuantum
			}
			[0] = 
			if .preemptoff != "" {
				[1] = funcPC(_GC) + sys.PCQuantum
			} else {
				[1] = funcPC(_System) + sys.PCQuantum
			}
		}
	}

	if prof.hz != 0 {
		cpuprof.add(, [:])
	}
	getg().m.mallocing--
}
If the signal handler receives a SIGPROF signal on a non-Go thread, it tries to collect a traceback into sigprofCallers. sigprofCallersUse is set to non-zero while sigprofCallers holds a traceback.
sigprofNonGo is called if we receive a SIGPROF signal on a non-Go thread, and the signal handler collected a stack trace in sigprofCallers. When this is called, sigprofCallersUse will be non-zero. g is nil, and what we can do is very limited.go:nosplitgo:nowritebarrierrec
func () {
	if prof.hz != 0 {
		 := 0
		for  < len(sigprofCallers) && sigprofCallers[] != 0 {
			++
		}
		cpuprof.addNonGo(sigprofCallers[:])
	}

	atomic.Store(&sigprofCallersUse, 0)
}
sigprofNonGoPC is called when a profiling signal arrived on a non-Go thread and we have a single PC value, not a stack trace. g is nil, and what we can do is very limited.go:nosplitgo:nowritebarrierrec
func ( uintptr) {
	if prof.hz != 0 {
		 := []uintptr{
			,
			funcPC(_ExternalCode) + sys.PCQuantum,
		}
		cpuprof.addNonGo()
	}
}
Reports whether a function will set the SP to an absolute value. Important that we don't traceback when these are at the bottom of the stack since we can't be sure that we will find the caller. If the function is not on the bottom of the stack we assume that it will have set it up so that traceback will be consistent, either by being a traceback terminating function or putting one on the stack at the right offset.
func ( uintptr) bool {
	 := findfunc()
couldn't find the function for this PC, so assume the worst and stop traceback
		return true
	}
	switch .funcID {
	case funcID_gogo, funcID_systemstack, funcID_mcall, funcID_morestack:
		return true
	}
	return false
}
setcpuprofilerate sets the CPU profiling rate to hz times per second. If hz <= 0, setcpuprofilerate turns off CPU profiling.
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.
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]
	for  := range .deferpool {
		.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 = raceprocctx0
			raceprocctx0 = 0 // bootstrap
		} else {
			.raceprocctx = raceproccreate()
		}
	}
	lockInit(&.timersLock, 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).
Similarly, we may not go through pidleget before this P starts running if it is P 0 on startup.
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.
Move all runnable goroutines to the global queue
Pop from tail of local queue
		.runqtail--
Push onto head of global queue
		globrunqputhead()
	}
	if .runnext != 0 {
		globrunqputhead(.runnext.ptr())
		.runnext = 0
	}
	if len(.timers) > 0 {
The world is stopped, but we acquire timersLock to protect against sysmon calling timeSleepUntil. This is the only case where we hold the timersLock of more than one P, so there are no deadlock concerns.
Flush p's write barrier buffer.
	if gcphase != _GCoff {
		wbBufFlush1()
		.gcw.dispose()
	}
	for  := range .sudogbuf {
		.sudogbuf[] = nil
	}
	.sudogcache = .sudogbuf[:0]
	for  := range .deferpool {
		for  := range .deferpoolbuf[] {
			.deferpoolbuf[][] = nil
		}
		.deferpool[] = .deferpoolbuf[][:0]
	}
	systemstack(func() {
Safe to call since the world is stopped.
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(.timerRaceCtx)
			.timerRaceCtx = 0

			.p.set()
		}
		raceprocdestroy(.raceprocctx)
		.raceprocctx = 0
	}
	.gcAssistTime = 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 ( int32) *p {
	assertLockHeld(&sched.lock)
	assertWorldStopped()

	 := gomaxprocs
	if  < 0 ||  <= 0 {
		throw("procresize: invalid arg")
	}
	if trace.enabled {
		traceGomaxprocs()
	}
update statistics
	 := nanotime()
	if sched.procresizetime != 0 {
		sched.totaltime += int64() * ( - sched.procresizetime)
	}
	sched.procresizetime = 

	 := ( + 31) / 32
Grow allp if necessary.
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 {
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 {
No need to copy beyond len, old Ps are irrelevant.
			copy(, idlepMask)
			idlepMask = 

			 := make([]uint32, )
			copy(, timerpMask)
			timerpMask = 
		}
		unlock(&allpLock)
	}
initialize new P's
	for  := ;  < ; ++ {
		 := allp[]
		if  == nil {
			 = new(p)
		}
		.init()
		atomicstorep(unsafe.Pointer(&allp[]), unsafe.Pointer())
	}

	 := getg()
continue to use the current P
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 {
Pretend that we were descheduled and then scheduled again to keep the trace sane.
				traceGoSched()
				traceProcStop(.m.p.ptr())
			}
			.m.p.ptr().m = 0
		}
		.m.p = 0
		 := allp[0]
		.m = 0
		.status = _Pidle
		acquirep()
		if trace.enabled {
			traceGoStart()
		}
	}
g.m.p is now set, so we no longer need mcache0 for bootstrapping.
release resources from unused P's
	for  := ;  < ; ++ {
		 := allp[]
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  *p
	for  :=  - 1;  >= 0; -- {
		 := allp[]
		if .m.p.ptr() ==  {
			continue
		}
		.status = _Pidle
		if runqempty() {
			pidleput()
		} else {
			.m.set(mget())
			.link.set()
			 = 
		}
	}
	stealOrder.reset(uint32())
	var  *int32 = &gomaxprocs // make compiler check that gomaxprocs is an int32
	atomic.Store((*uint32)(unsafe.Pointer()), uint32())
	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 _p_.go:yeswritebarrierrec
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.
wirep is the first step of acquirep, which actually associates the current M to _p_. This is broken out so we can disallow write barriers for this part, since we don't yet have a P.go:nowritebarrierrecgo:nosplit
func ( *p) {
	 := getg()

	if .m.p != 0 {
		throw("wirep: already in go")
	}
	if .m != 0 || .status != _Pidle {
		 := 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 () *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")
	}
	if trace.enabled {
		traceProcStop(.m.p.ptr())
	}
	.m.p = 0
	.m = 0
	.status = _Pidle
	return 
}

func ( 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.
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.
	if islibrary || isarchive {
		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 > 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  int32
	if !iscgo && cgoHasExtraM {
		 := lockextra(true)
		 := extraMCount > 0
		unlockextra()
		if  {
			 = 1
		}
	}

	 := mcount() - sched.nmidle - sched.nmidlelocked - sched.nmsys
	if  >  {
		return
	}
	if  < 0 {
		print("runtime: checkdead: nmidle=", sched.nmidle, " nmidlelocked=", sched.nmidlelocked, " mcount=", mcount(), " nmsys=", sched.nmsys, "\n")
		throw("checkdead: inconsistent counts")
	}

	 := 0
	lock(&allglock)
	for  := 0;  < len(allgs); ++ {
		 := allgs[]
		if isSystemGoroutine(, false) {
			continue
		}
		 := readgstatus()
		switch  &^ _Gscan {
		case _Gwaiting,
			_Gpreempted:
			++
		case _Grunnable,
			_Grunning,
			_Gsyscall:
			print("runtime: checkdead: find g ", .goid, " in status ", , "\n")
			throw("checkdead: runnable g")
		}
	}
	unlock(&allglock)
	if  == 0 { // possible if main goroutine calls runtime·Goexit()
		unlock(&sched.lock) // unlock so that GODEBUG=scheddetail=1 doesn't hang
		throw("no goroutines (main called runtime.Goexit) - deadlock!")
	}
Maybe jump time forward for playground.
	if faketime != 0 {
		,  := timeSleepUntil()
		if  != nil {
			faketime = 
			for  := &sched.pidle; * != 0;  = &(*).ptr().link {
				if (*).ptr() ==  {
					* = .link
					break
				}
			}
			 := mget()
There should always be a free M since nothing is running.
				throw("checkdead: no m for timer")
			}
			.nextp.set()
			notewakeup(&.park)
			return
		}
	}
There are no goroutines running, so we can look at the P's.
	for ,  := range allp {
		if len(.timers) > 0 {
			return
		}
	}

	getg().m.throwing = -1 // do not dump full stacks
	unlock(&sched.lock)    // unlock so that GODEBUG=scheddetail=1 doesn't hang
	throw("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
Always runs without a P, so write barriers are not allowed.go:nowritebarrierrec
For syscall_runtime_doAllThreadsSyscall, sysmon is sufficiently up to participate in fixups.
	atomic.Store(&sched.sysmonStarting, 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()
		mDoFixup()
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 != 0 || atomic.Load(&sched.npidle) == uint32(gomaxprocs)) {
			lock(&sched.lock)
			if atomic.Load(&sched.gcwaiting) != 0 || atomic.Load(&sched.npidle) == uint32(gomaxprocs) {
				 := false
				,  := timeSleepUntil()
				if  >  {
					atomic.Store(&sched.sysmonwait, 1)
Make wake-up period small enough for the sampling to be correct.
					 := forcegcperiod / 2
					if - <  {
						 =  - 
					}
					 :=  >= osRelaxMinNS
					if  {
						osRelax(true)
					}
					 = notetsleep(&sched.sysmonnote, )
					mDoFixup()
					if  {
						osRelax(false)
					}
					lock(&sched.lock)
					atomic.Store(&sched.sysmonwait, 0)
					noteclear(&sched.sysmonnote)
				}
				if  {
					 = 0
					 = 20
				}
			}
			unlock(&sched.lock)
		}

Update now in case we blocked on sysmonnote or spent a long time blocked on schedlock or sysmonlock above.
		 = nanotime()
trigger libc interceptors if needed
poll network if not polled for more than 10ms
		 := int64(atomic.Load64(&sched.lastpoll))
		if netpollinited() &&  != 0 && +10*1000*1000 <  {
			atomic.Cas64(&sched.lastpoll, uint64(), uint64())
			 := netpoll(0) // non-blocking - returns list of goroutines
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.
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)
			}
		}
Kick the scavenger awake if someone requested it.
retake P's blocked in syscalls and preempt long running G's
		if retake() != 0 {
			 = 0
		} else {
			++
check if we need to force a GC
		if  := (gcTrigger{kind: gcTriggerTime, now: }); .test() && atomic.Load(&forcegc.idle) != 0 {
			lock(&forcegc.lock)
			forcegc.idle = 0
			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   uint32
	schedwhen   int64
	syscalltick uint32
	syscallwhen int64
}
forcePreemptNS is the time slice given to a G before it is preempted.
const forcePreemptNS = 10 * 1000 * 1000 // 10ms

func ( int64) uint32 {
Prevent allp slice changes. This lock will be completely uncontended unless we're already stopping the world.
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[]
This can happen if procresize has grown allp but not yet created new Ps.
			continue
		}
		 := &.sysmontick
		 := .status
		 := false
Preempt G if it's running for too long.
			 := int64(.schedtick)
			if int64(.schedtick) !=  {
				.schedtick = uint32()
				.schedwhen = 
			} else if .schedwhen+forcePreemptNS <=  {
In case of syscall, preemptone() doesn't work, because there is no M wired to P.
				 = true
			}
		}
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() && atomic.Load(&sched.nmspinning)+atomic.Load(&sched.npidle) > 0 && .syscallwhen+10*1000*1000 >  {
				continue
Drop allpLock so we can take sched.lock.
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)
			if atomic.Cas(&.status, , _Pidle) {
				if trace.enabled {
					traceGoSysBlock()
					traceProcStop()
				}
				++
				.syscalltick++
				handoffp()
			}
			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 () bool {
	 := false
	for ,  := 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 send 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 Grunning
func ( *p) bool {
	 := .m.ptr()
	if  == nil ||  == getg().m {
		return false
	}
	 := .curg
	if  == nil ||  == .g0 {
		return false
	}

	.preempt = true
Every call in a go routine 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.
Request an async preemption of this P.
	if preemptMSupported && debug.asyncpreemptoff == 0 {
		.preempt = true
		preemptM()
	}

	return true
}

var starttime int64

func ( bool) {
	 := nanotime()
	if starttime == 0 {
		starttime = 
	}

	lock(&sched.lock)
	print("SCHED ", (-starttime)/1e6, "ms: gomaxprocs=", gomaxprocs, " idleprocs=", sched.npidle, " threads=", mcount(), " spinningthreads=", sched.nmspinning, " idlethreads=", sched.nmidle, " runqueue=", sched.runqsize)
	if  {
		print(" gcwaiting=", sched.gcwaiting, " nmidlelocked=", sched.nmidlelocked, " stopwait=", sched.stopwait, " sysmonwait=", sched.sysmonwait, "\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 {
		 := .m.ptr()
		 := atomic.Load(&.runqhead)
		 := atomic.Load(&.runqtail)
		if  {
			 := int64(-1)
			if  != nil {
				 = .id
			}
			print("  P", , ": status=", .status, " schedtick=", .schedtick, " syscalltick=", .syscalltick, " m=", , " runqsize=", -, " gfreecnt=", .gFree.n, " timerslen=", len(.timers), "\n")
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("]\n")
			}
		}
	}

	if ! {
		unlock(&sched.lock)
		return
	}

	for  := allm;  != nil;  = .alllink {
		 := .p.ptr()
		 := .curg
		 := .lockedg.ptr()
		 := int32(-1)
		if  != nil {
			 = .id
		}
		 := int64(-1)
		if  != nil {
			 = .goid
		}
		 := int64(-1)
		if  != nil {
			 = .goid
		}
		print("  M", .id, ": p=", , " curg=", , " mallocing=", .mallocing, " throwing=", .throwing, " preemptoff=", .preemptoff, ""+" locks=", .locks, " dying=", .dying, " spinning=", .spinning, " blocked=", .blocked, " lockedg=", , "\n")
	}

	lock(&allglock)
	for  := 0;  < len(allgs); ++ {
		 := allgs[]
		 := .m
		 := .lockedm.ptr()
		 := int64(-1)
		if  != nil {
			 = .id
		}
		 := int64(-1)
		if  != nil {
			 = .id
		}
		print("  G", .goid, ": status=", readgstatus(), "(", .waitreason.String(), ") m=", , " lockedm=", , "\n")
	}
	unlock(&allglock)
	unlock(&sched.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 ( bool) {
	lock(&sched.lock)
	if sched.disable.user == ! {
		unlock(&sched.lock)
		return
	}
	sched.disable.user = !
	if  {
		 := sched.disable.n
		sched.disable.n = 0
		globrunqputbatch(&sched.disable.runnable, )
		unlock(&sched.lock)
		for ;  != 0 && sched.npidle != 0; -- {
			startm(nil, 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 ( *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:nowritebarrierrec
Try to get an m from midle list. sched.lock must be held. May run during STW, so write barriers are not allowed.go:nowritebarrierrec
func () *m {
	assertLockHeld(&sched.lock)

	 := sched.midle.ptr()
	if  != nil {
		sched.midle = .schedlink
		sched.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:nowritebarrierrec
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:nowritebarrierrec
Put a batch of runnable goroutines on the global runnable queue. This clears *batch. sched.lock must be held.
Try get a batch of G's from the global runnable queue. sched.lock must be held.
func ( *p,  int32) *g {
	assertLockHeld(&sched.lock)

	if sched.runqsize == 0 {
		return nil
	}

	 := sched.runqsize/gomaxprocs + 1
	if  > sched.runqsize {
		 = sched.runqsize
	}
	if  > 0 &&  >  {
		 = 
	}
	if  > int32(len(.runq))/2 {
		 = int32(len(.runq)) / 2
	}

	sched.runqsize -= 

	 := sched.runq.pop()
	--
	for ;  > 0; -- {
		 := sched.runq.pop()
		runqput(, , false)
	}
	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(&[], ^)
}
updateTimerPMask clears pp's timer mask if it has no timers on its heap. Ideally, the timer mask would be kept immediately consistent on any timer operations. Unfortunately, updating a shared global data structure in the timer hot path adds too much overhead in applications frequently switching between no timers and some timers. As a compromise, the timer mask is updated only on pidleget / pidleput. A running P (returned by pidleget) may add a timer at any time, so its mask must be set. An idle P (passed to pidleput) cannot add new timers while idle, so if it has no timers at that time, its mask may be cleared. Thus, we get the following effects on timer-stealing in findrunnable: * Idle Ps with no timers when they go idle are never checked in findrunnable (for work- or timer-stealing; this is the ideal case). * Running Ps must always be checked. * Idle Ps whose timers are stolen must continue to be checked until they run again, even after timer expiration. When the P starts running again, the mask should be set, as a timer may be added at any time. TODO(prattmic): Additional targeted updates may improve the above cases. e.g., updating the mask when stealing a timer.
func ( *p) {
	if atomic.Load(&.numTimers) > 0 {
		return
	}
Looks like there are no timers, however another P may transiently decrement numTimers when handling a timerModified timer in checkTimers. We must take timersLock to serialize with these changes.
	lock(&.timersLock)
	if atomic.Load(&.numTimers) == 0 {
		timerpMask.clear(.id)
	}
	unlock(&.timersLock)
}
pidleput puts p to on the _Pidle list. 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:nowritebarrierrec
func ( *p) {
	assertLockHeld(&sched.lock)

	if !runqempty() {
		throw("pidleput: P has non-empty run queue")
	}
	updateTimerPMask() // clear if there are no timers.
	idlepMask.set(.id)
	.link = sched.pidle
	sched.pidle.set()
	atomic.Xadd(&sched.npidle, 1) // TODO: fast atomic
}
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:nowritebarrierrec
Timer may get added at any time now.
		timerpMask.set(.id)
		idlepMask.clear(.id)
		sched.pidle = .link
		atomic.Xadd(&sched.npidle, -1) // TODO: fast atomic
	}
	return 
}
runqempty reports whether _p_ has no Gs on its local run queue. It never returns true spuriously.
Defend against a race where 1) _p_ has G1 in runqnext but runqhead == runqtail, 2) runqput on _p_ kicks G1 to the runq, 3) runqget on _p_ 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.
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 _p_.runnext slot. If the run queue is full, runnext puts g on the global queue. Executed only by the owner P.
func ( *p,  *g,  bool) {
	if randomizeScheduler &&  && fastrand()%2 == 0 {
		 = false
	}

	if  {
	:
		 := .runnext
		if !.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
	 := .runqtail
	if - < uint32(len(.runq)) {
		.runq[%uint32(len(.runq))].set()
		atomic.StoreRel(&.runqtail, +1) // store-release, makes the item available for consumption
		return
	}
	if runqputslow(, , , ) {
		return
the queue is not full, now the put above must succeed
	goto 
}
Put g and a batch of work from local runnable queue on global queue. Executed only by the owner P.
func ( *p,  *g, ,  uint32) bool {
	var  [len(.runq)/2 + 1]*g
First, grab a batch from local queue.
	 :=  - 
	 =  / 2
	if  != 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 consume
		return false
	}
	[] = 

	if randomizeScheduler {
		for  := uint32(1);  <= ; ++ {
			 := fastrandn( + 1)
			[], [] = [], []
		}
	}
Link the goroutines.
	for  := uint32(0);  < ; ++ {
		[].schedlink.set([+1])
	}
	var  gQueue
	.head.set([0])
	.tail.set([])
Now put the batch on global queue.
runqputbatch tries to put all the G's on q on the local runnable queue. If the queue is full, they are put on the global queue; in that case this will temporarily acquire the scheduler lock. Executed only by the owner P.
func ( *p,  *gQueue,  int) {
	 := atomic.LoadAcq(&.runqhead)
	 := .runqtail
	 := uint32(0)
	for !.empty() && - < uint32(len(.runq)) {
		 := .pop()
		.runq[%uint32(len(.runq))].set()
		++
		++
	}
	 -= int()

	if randomizeScheduler {
		 := func( uint32) uint32 {
			return (.runqtail + ) % uint32(len(.runq))
		}
		for  := uint32(1);  < ; ++ {
			 := fastrandn( + 1)
			.runq[()], .runq[()] = .runq[()], .runq[()]
		}
	}

	atomic.StoreRel(&.runqtail, )
	if !.empty() {
		lock(&sched.lock)
		globrunqputbatch(, int32())
		unlock(&sched.lock)
	}
}
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.
If there's a runnext, it's the next G to run.
	for {
		 := .runnext
		if  == 0 {
			break
		}
		if .runnext.cas(, 0) {
			return .ptr(), true
		}
	}

	for {
		 := atomic.LoadAcq(&.runqhead) // load-acquire, synchronize with other consumers
		 := .runqtail
		if  ==  {
			return nil, false
		}
		 := .runq[%uint32(len(.runq))].ptr()
		if atomic.CasRel(&.runqhead, , +1) { // cas-release, commits consume
			return , false
		}
	}
}
Grabs a batch of goroutines from _p_'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 ( *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
		 :=  - 
		 =  - /2
		if  == 0 {
Try to steal from _p_.runnext.
				if  := .runnext;  != 0 {
Sleep to ensure that _p_ isn't about to run the g we are about to steal. The important use case here is when the g running on _p_ ready()s another g and then almost immediately blocks. Instead of stealing runnext in this window, back off to give _p_ 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 GOOS != "windows" {
							usleep(3)
On windows 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 t
			continue
		}
		for  := uint32(0);  < ; ++ {
			 := .runq[(+)%uint32(len(.runq))]
			[(+)%uint32(len())] = 
		}
		if atomic.CasRel(&.runqhead, , +) { // cas-release, commits consume
			return 
		}
	}
}
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 (,  *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 consumers
	if -+ >= uint32(len(.runq)) {
		throw("runqsteal: runq overflow")
	}
	atomic.StoreRel(&.runqtail, +) // store-release, makes the item available for consumption
	return 
}
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 guintptr
	tail guintptr
}
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()
	}
}
pushBack adds gp to the tail of q.
func ( *gQueue) ( *g) {
	.schedlink = 0
	if .tail != 0 {
		.tail.ptr().schedlink.set()
	} else {
		.head.set()
	}
	.tail.set()
}
pushBackAll adds all Gs in l2 to the tail of q. After this q2 must not be used.
func ( *gQueue) ( gQueue) {
	if .tail == 0 {
		return
	}
	.tail.ptr().schedlink = 0
	if .tail != 0 {
		.tail.ptr().schedlink = .head
	} else {
		.head = .head
	}
	.tail = .tail
}
pop removes and returns the head of queue q. It returns nil if q is empty.
func ( *gQueue) () *g {
	 := .head.ptr()
	if  != nil {
		.head = .schedlink
		if .head == 0 {
			.tail = 0
		}
	}
	return 
}
popList takes all Gs in q and returns them as a gList.
func ( *gQueue) () gList {
	 := gList{.head}
	* = 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 guintptr
}
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()
}
pushAll prepends all Gs in q to l.
func ( *gList) ( gQueue) {
	if !.empty() {
		.tail.ptr().schedlink = .head
		.head = .head
	}
}
pop removes and returns the head of l. If l is empty, it returns nil.
func ( *gList) () *g {
	 := .head.ptr()
	if  != nil {
		.head = .schedlink
	}
	return 
}
go:linkname setMaxThreads runtime/debug.setMaxThreads
func ( int) ( int) {
	lock(&sched.lock)
	 = int(sched.maxmcount)
	if  > 0x7fffffff { // MaxInt32
		sched.maxmcount = 0x7fffffff
	} else {
		sched.maxmcount = int32()
	}
	checkmcount()
	unlock(&sched.lock)
	return
}

func ( string) bool {
	 := sys.Goexperiment
	for  != "" {
		 := ""
		 := bytealg.IndexByteString(, ',')
		if  < 0 {
			,  = , ""
		} else {
			,  = [:], [+1:]
		}
		if  ==  {
			return true
		}
		if len() > 2 && [:2] == "no" && [2:] ==  {
			return false
		}
	}
	return false
}
go:nosplit
func () int {
	 := getg()
	 := .m

	.locks++
	return int(.p.ptr().id)
}
go:nosplit
func () {
	 := getg()
	.m.locks--
}
go:linkname sync_runtime_procPin sync.runtime_procPingo:nosplit
func () int {
	return procPin()
}
go:linkname sync_runtime_procUnpin sync.runtime_procUnpingo:nosplit
go:linkname sync_atomic_runtime_procPin sync/atomic.runtime_procPingo:nosplit
go:linkname sync_atomic_runtime_procUnpin sync/atomic.runtime_procUnpingo:nosplit
Active spinning for sync.Mutex.go:linkname sync_runtime_canSpin sync.runtime_canSpingo:nosplit
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 || ncpu <= 1 || gomaxprocs <= int32(sched.npidle+sched.nmspinning)+1 {
		return false
	}
	if  := getg().m.p.ptr(); !runqempty() {
		return false
	}
	return true
}
go:linkname sync_runtime_doSpin sync.runtime_doSpingo:nosplit
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    uint32
	coprimes []uint32
}

type randomEnum struct {
	i     uint32
	count uint32
	pos   uint32
	inc   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[%uint32(len(.coprimes))],
	}
}

func ( *randomEnum) () bool {
	return .i == .count
}

func ( *randomEnum) () {
	.i++
	.pos = (.pos + .inc) % .count
}

func ( *randomEnum) () uint32 {
	return .pos
}

func (,  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/initempty.go:initTask
TODO: pack the first 3 fields more tightly?
	state uintptr // 0 = uninitialized, 1 = in progress, 2 = done
	ndeps uintptr
followed by ndeps instances of an *initTask, one per package depended on followed by nfns pcs, 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 tracestat

type tracestat struct {
	active bool   // init tracing activation status
	id     int64  // init go routine id
	allocs uint64 // heap allocations
	bytes  uint64 // heap allocated bytes
}

func ( *initTask) {
	switch .state {
	case 2: // fully initialized
		return
	case 1: // initialization in progress
		throw("recursive call during initialization - linker skew")
	default: // not initialized yet
		.state = 1 // initialization in progress

		for  := uintptr(0);  < .ndeps; ++ {
			 := add(unsafe.Pointer(), (3+)*sys.PtrSize)
			 := *(**initTask)()
			()
		}

		if .nfns == 0 {
			.state = 2 // initialization done
			return
		}

		var (
			  int64
			 tracestat
		)

		if inittrace.active {
Load stats non-atomically since tracinit is updated only by this init go routine.
			 = inittrace
		}

		 := add(unsafe.Pointer(), (3+.ndeps)*sys.PtrSize)
		for  := uintptr(0);  < .nfns; ++ {
			 := add(, *sys.PtrSize)
			 := *(*func())(unsafe.Pointer(&))
			()
		}

		if inittrace.active {
Load stats non-atomically since tracinit is updated only by this init go routine.
			 := inittrace

			 := funcpkgpath(findfunc(funcPC()))

			var  [24]byte
			print("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
	}