// Copyright 2025 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.

// Green Tea mark algorithm
//
// The core idea behind Green Tea is simple: achieve better locality during
// mark/scan by delaying scanning so that we can accumulate objects to scan
// within the same span, then scan the objects that have accumulated on the
// span all together.
//
// By batching objects this way, we increase the chance that adjacent objects
// will be accessed, amortize the cost of accessing object metadata, and create
// better opportunities for prefetching. We can take this even further and
// optimize the scan loop by size class (not yet completed) all the way to the
// point of applying SIMD techniques to really tear through the heap.
//
// Naturally, this depends on being able to create opportunties to batch objects
// together. The basic idea here is to have two sets of mark bits. One set is the
// regular set of mark bits ("marks"), while the other essentially says that the
// objects have been scanned already ("scans"). When we see a pointer for the first
// time we set its mark and enqueue its span. We track these spans in work queues
// with a FIFO policy, unlike workbufs which have a LIFO policy. Empirically, a
// FIFO policy appears to work best for accumulating objects to scan on a span.
// Later, when we dequeue the span, we find both the union and intersection of the
// mark and scan bitsets. The union is then written back into the scan bits, while
// the intersection is used to decide which objects need scanning, such that the GC
// is still precise.
//
// Below is the bulk of the implementation, focusing on the worst case
// for locality, small objects. Specifically, those that are smaller than
// a few cache lines in size and whose metadata is stored the same way (at the
// end of the span).

//go:build goexperiment.greenteagc

package runtime

import (
	
	
	
	
	
	
)

const doubleCheckGreenTea = false

// spanInlineMarkBits are mark bits that are inlined into the span
// itself. gcUsesSpanInlineMarkBits may be used to check if objects
// of a particular size use inline mark bits.
//
// Inline mark bits are a little bit more than just mark bits. They
// consist of two parts: scans and marks. Marks are like pre-mark
// bits. They're set once a pointer to an object is discovered for
// the first time. The marks allow us to scan many objects in bulk
// if we queue the whole span for scanning. Before we scan such objects
// in bulk, we copy the marks to the scans, computing a diff along the
// way. The resulting bitmap tells us which objects we should scan.
//
// The inlineMarkBits also hold state sufficient for scanning any
// object in the span, as well as state for acquiring ownership of
// the span for queuing. This avoids the need to look at the mspan when
// scanning.
type spanInlineMarkBits struct {
	scans [63]uint8         // scanned bits.
	owned spanScanOwnership // see the comment on spanScanOwnership.
	marks [63]uint8         // mark bits.
	class spanClass
}

// spanScanOwnership indicates whether some thread has acquired
// the span for scanning, and whether there has been one or more
// attempts to acquire the span. The latter information helps to
// fast-track span scans that only apply to a single mark, skipping
// the relatively costly merge-and-diff process for scans and marks
// by allowing one to just set the mark directly.
type spanScanOwnership uint8

const (
	spanScanUnowned  spanScanOwnership = 0         // Indicates the span is not acquired for scanning.
	spanScanOneMark                    = 1 << iota // Indicates that only one mark bit is set relative to the scan bits.
	spanScanManyMark                               // Indicates one or more scan bits may be set relative to the mark bits.
	// "ManyMark" need not be exactly the value it has. In practice we just
	// want to distinguish "none" from "one" from "many," so a comparison is
	// sufficient (as opposed to a bit test) to check between these cases.
)

// load atomically loads from a pointer to a spanScanOwnership.
func ( *spanScanOwnership) () spanScanOwnership {
	return spanScanOwnership(atomic.Load8((*uint8)(unsafe.Pointer())))
}

func ( *spanScanOwnership) ( spanScanOwnership) spanScanOwnership {
	// N.B. We round down the address and use Or32 because Or8 doesn't
	// return a result, and it's strictly necessary for this protocol.
	//
	// Making Or8 return a result, while making the code look nicer, would
	// not be strictly better on any supported platform, as an Or8 that
	// returns a result is not a common instruction. On many platforms it
	// would be implemented exactly as it is here, and since Or8 is
	// exclusively used in the runtime and a hot function, we want to keep
	// using its no-result version elsewhere for performance.
	 := (*uint32)(unsafe.Pointer(uintptr(unsafe.Pointer()) &^ 0b11))
	 := (uintptr(unsafe.Pointer()) & 0b11) * 8
	if goarch.BigEndian {
		 = 32 -  - 8
	}
	return spanScanOwnership(atomic.Or32(, uint32()<<) >> )
}

func ( *spanInlineMarkBits) ( spanClass,  bool) {
	if  == nil {
		// This nil check and throw is almost pointless. Normally we would
		// expect imb to never be nil. However, this is called on potentially
		// freshly-allocated virtual memory. As of 2025, the compiler-inserted
		// nil check is not a branch but a memory read that we expect to fault
		// if the pointer really is nil.
		//
		// However, this causes a read of the page, and operating systems may
		// take it as a hint to back the accessed memory with a read-only zero
		// page. However, we immediately write to this memory, which can then
		// force operating systems to have to update the page table and flush
		// the TLB, causing a lot of churn for programs that are short-lived
		// and monotonically grow in size.
		//
		// This nil check is thus an explicit branch instead of what the compiler
		// would insert circa 2025, which is a memory read instruction.
		//
		// See go.dev/issue/74375 for details.
		throw("runtime: span inline mark bits nil?")
	}
	if  {
		// Use memclrNoHeapPointers to avoid having the compiler make a worse
		// decision. We know that imb is both aligned and a nice power-of-two
		// size that works well for wider SIMD instructions. The compiler likely
		// has no idea that imb is aligned to 128 bytes.
		memclrNoHeapPointers(unsafe.Pointer(), unsafe.Sizeof(spanInlineMarkBits{}))
	}
	.class = 
}

// tryAcquire attempts to acquire the span for scanning. On success, the caller
// must queue the span for scanning or scan the span immediately.
func ( *spanInlineMarkBits) () bool {
	switch .owned.load() {
	case spanScanUnowned:
		// Try to mark the span as having only one object marked.
		if .owned.or(spanScanOneMark) == spanScanUnowned {
			return true
		}
		// If we didn't see an old value of spanScanUnowned, then we must
		// have raced with someone else and seen spanScanOneMark or greater.
		// Fall through and try to set spanScanManyMark.
		fallthrough
	case spanScanOneMark:
		// We may be the first to set *any* bit on owned. In such a case,
		// we still need to make sure the span is queued.
		return .owned.or(spanScanManyMark) == spanScanUnowned
	}
	return false
}

// release releases the span for scanning, allowing another thread to queue the span.
//
// Returns an upper bound on the number of mark bits set since the span was queued. The
// upper bound is described as "one" (spanScanOneMark) or "many" (spanScanManyMark, with or
// without spanScanOneMark). If the return value indicates only one mark bit was set, the
// caller can be certain that it was the same mark bit that caused the span to get queued.
// Take note of the fact that this is *only* an upper-bound. In particular, it may still
// turn out that only one mark bit was set, even if the return value indicates "many".
func ( *spanInlineMarkBits) () spanScanOwnership {
	return spanScanOwnership(atomic.Xchg8((*uint8)(unsafe.Pointer(&.owned)), uint8(spanScanUnowned)))
}

// spanInlineMarkBitsFromBase returns the spanInlineMarkBits for a span whose start address is base.
//
// The span must be gcUsesSpanInlineMarkBits(span.elemsize).
func spanInlineMarkBitsFromBase( uintptr) *spanInlineMarkBits {
	return (*spanInlineMarkBits)(unsafe.Pointer( + gc.PageSize - unsafe.Sizeof(spanInlineMarkBits{})))
}

// initInlineMarkBits initializes the inlineMarkBits stored at the end of the span.
func ( *mspan) () {
	if doubleCheckGreenTea && !gcUsesSpanInlineMarkBits(.elemsize) {
		throw("expected span with inline mark bits")
	}
	// Zeroing is only necessary if this span wasn't just freshly allocated from the OS.
	.inlineMarkBits().init(.spanclass, .needzero != 0)
}

// moveInlineMarks merges the span's inline mark bits into dst and clears them.
//
// gcUsesSpanInlineMarkBits(s.elemsize) must be true.
func ( *mspan) ( *gcBits) {
	if doubleCheckGreenTea && !gcUsesSpanInlineMarkBits(.elemsize) {
		throw("expected span with inline mark bits")
	}
	 := divRoundUp(uintptr(.nelems), 8)
	 := .inlineMarkBits()
	 := (*gc.ObjMask)(unsafe.Pointer(&.marks))
	for  := uintptr(0);  < ;  += goarch.PtrSize {
		 := bswapIfBigEndian([/goarch.PtrSize])
		if /goarch.PtrSize == uintptr(len(.marks)+1)/goarch.PtrSize-1 {
			 &^= 0xff << ((goarch.PtrSize - 1) * 8) // mask out class
		}
		*(*uintptr)(unsafe.Pointer(.bytep())) |= bswapIfBigEndian()
	}
	if doubleCheckGreenTea && !.spanclass.noscan() && .marks != .scans {
		throw("marks don't match scans for span with pointer")
	}

	// Reset the inline mark bits.
	.init(.spanclass, true /* We know these bits are always dirty now. */)
}

// inlineMarkBits returns the inline mark bits for the span.
//
// gcUsesSpanInlineMarkBits(s.elemsize) must be true.
func ( *mspan) () *spanInlineMarkBits {
	if doubleCheckGreenTea && !gcUsesSpanInlineMarkBits(.elemsize) {
		throw("expected span with inline mark bits")
	}
	return spanInlineMarkBitsFromBase(.base())
}

func ( *mspan) ( uintptr) ( markBits) {
	if gcUsesSpanInlineMarkBits(.elemsize) {
		.bytep = &.inlineMarkBits().marks[/8]
	} else {
		.bytep = .gcmarkBits.bytep( / 8)
	}
	.mask = uint8(1) << ( % 8)
	.index = 
	return
}

func ( *mspan) () markBits {
	if gcUsesSpanInlineMarkBits(.elemsize) {
		return markBits{&.inlineMarkBits().marks[0], uint8(1), 0}
	}
	return markBits{&.gcmarkBits.x, uint8(1), 0}
}

// scannedBitsForIndex returns a markBits representing the scanned bit
// for objIndex in the inline mark bits.
func ( *mspan) ( uintptr) markBits {
	return markBits{&.inlineMarkBits().scans[/8], uint8(1) << ( % 8), }
}

// gcUsesSpanInlineMarkBits returns true if a span holding objects of a certain size
// has inline mark bits. size must be the span's elemsize.
//
// nosplit because this is called from gcmarknewobject, which is nosplit.
//
//go:nosplit
func gcUsesSpanInlineMarkBits( uintptr) bool {
	return heapBitsInSpan() &&  >= 16
}

// tryDeferToSpanScan tries to queue p on the span it points to, if it
// points to a small object span (gcUsesSpanQueue size).
func tryDeferToSpanScan( uintptr,  *gcWork) bool {
	if useCheckmark {
		return false
	}

	// Quickly to see if this is a span that has inline mark bits.
	 := heapArenaOf()
	if  == nil {
		return false
	}
	 := (( / pageSize) / 8) % uintptr(len(.pageInUse))
	 := byte(1 << (( / pageSize) % 8))
	if .pageUseSpanInlineMarkBits[]& == 0 {
		return false
	}

	// Find the object's index from the span class info stored in the inline mark bits.
	 := alignDown(, gc.PageSize)
	 := spanInlineMarkBitsFromBase()
	 := uint16((uint64(-) * uint64(gc.SizeClassToDivMagic[.class.sizeclass()])) >> 32)

	// Set mark bit.
	,  := /8, uint8(1)<<(%8)
	if atomic.Load8(&.marks[])& != 0 {
		return true
	}
	atomic.Or8(&.marks[], )

	// Fast-track noscan objects.
	if .class.noscan() {
		.bytesMarked += uint64(gc.SizeClassToSize[.class.sizeclass()])
		return true
	}

	// Queue up the pointer (as a representative for its span).
	if .tryAcquire() {
		if .spanq.put(makeObjPtr(, )) {
			if gcphase == _GCmark {
				// This is intentionally racy; the bit set here might get
				// stomped on by a stealing P. See the comment in tryStealSpan
				// for an explanation as to why this is OK.
				if !work.spanqMask.read(uint32(.id)) {
					work.spanqMask.set(.id)
				}
				.mayNeedWorker = true
			}
			.flushedWork = true
		}
	}
	return true
}

// tryGetSpanFast attempts to get an entire span to scan.
func ( *gcWork) () objptr {
	return .spanq.tryGetFast()
}

// tryGetSpan attempts to get an entire span to scan.
func ( *gcWork) () objptr {
	if  := .spanq.tryGetFast();  != 0 {
		return 
	}
	// "Steal" from ourselves.
	if  := .spanq.steal(&.spanq);  != 0 {
		return 
	}
	// We failed to get any local work, so we're fresh out.
	// Nobody else is going to add work for us. Clear our bit.
	if work.spanqMask.read(uint32(.id)) {
		work.spanqMask.clear(.id)
	}
	return 0
}

// spanQueue is a P-local stealable span queue.
type spanQueue struct {
	// head, tail, and ring represent a local non-thread-safe ring buffer.
	head, tail uint32
	ring       [256]objptr

	// putsSinceDrain counts the number of put calls since the last drain.
	putsSinceDrain int

	// chain contains state visible to other Ps.
	//
	// In particular, that means a linked chain of single-producer multi-consumer
	// ring buffers where the single producer is this P only.
	//
	// This linked chain structure is based off the sync.Pool dequeue.
	chain struct {
		// head is the spanSPMC to put to. This is only accessed
		// by the producer, so doesn't need to be synchronized.
		head *spanSPMC

		// tail is the spanSPMC to steal from. This is accessed
		// by consumers, so reads and writes must be atomic.
		tail atomic.UnsafePointer // *spanSPMC
	}
}

// putFast tries to put s onto the queue, but may fail if it's full.
func ( *spanQueue) ( objptr) ( bool) {
	if .tail-.head == uint32(len(.ring)) {
		return false
	}
	.ring[.tail%uint32(len(.ring))] = 
	.tail++
	return true
}

// put puts s onto the queue.
//
// Returns whether the caller should spin up a new worker.
func ( *spanQueue) ( objptr) bool {
	// The constants below define the period of and volume of
	// spans we spill to the spmc chain when the local queue is
	// not full.
	//
	// spillPeriod must be > spillMax, otherwise that sets the
	// effective maximum size of our local span queue. Even if
	// we have a span ring of size N, but we flush K spans every
	// K puts, then K becomes our effective maximum length. When
	// spillPeriod > spillMax, then we're always spilling spans
	// at a slower rate than we're accumulating them.
	const (
		// spillPeriod defines how often to check if we should
		// spill some spans, counted in the number of calls to put.
		 = 64

		// spillMax defines, at most, how many spans to drain with
		// each spill.
		 = 16
	)

	if .putFast() {
		// Occasionally try to spill some work to generate parallelism.
		.putsSinceDrain++
		if .putsSinceDrain >=  {
			// Reset even if we don't drain, so we don't check every time.
			.putsSinceDrain = 0

			// Try to drain some spans. Don't bother if there's very
			// few of them or there's already spans in the spmc chain.
			 := min((.tail-.head)/2, )
			if  > 4 && .chainEmpty() {
				.drain()
				return true
			}
		}
		return false
	}

	// We're out of space. Drain out our local spans.
	.drain(uint32(len(.ring)) / 2)
	if !.putFast() {
		throw("failed putFast after drain")
	}
	return true
}

// flush publishes all spans in the local queue to the spmc chain.
func ( *spanQueue) () {
	 := .tail - .head
	if  == 0 {
		return
	}
	.drain()
}

// empty returns true if there's no more work on the queue.
//
// Not thread-safe. Must only be called by the owner of q.
func ( *spanQueue) () bool {
	// Check the local queue for work.
	if .tail-.head > 0 {
		return false
	}
	return .chainEmpty()
}

// chainEmpty returns true if the spmc chain is empty.
//
// Thread-safe.
func ( *spanQueue) () bool {
	// Check the rest of the rings for work.
	 := (*spanSPMC)(.chain.tail.Load())
	for  != nil {
		if !.empty() {
			return false
		}
		 = (*spanSPMC)(.prev.Load())
	}
	return true
}

// drain publishes n spans from the local queue to the spmc chain.
func ( *spanQueue) ( uint32) {
	.putsSinceDrain = 0

	if .chain.head == nil {
		// N.B. We target 1024, but this may be bigger if the physical
		// page size is bigger, or if we can fit more uintptrs into a
		// physical page. See newSpanSPMC docs.
		 := newSpanSPMC(1024)
		.chain.head = 
		.chain.tail.StoreNoWB(unsafe.Pointer())
	}

	// Try to drain some of the queue to the head spmc.
	if .tryDrain(.chain.head, ) {
		return
	}
	// No space. Create a bigger spmc and add it to the chain.

	// Double the size of the next one, up to a maximum.
	//
	// We double each time so we can avoid taking this slow path
	// in the future, which involves a global lock. Ideally we want
	// to hit a steady-state where the deepest any queue goes during
	// a mark phase can fit in the ring.
	//
	// However, we still set a maximum on this. We set the maximum
	// to something large to amortize the cost of lock acquisition, but
	// still at a reasonable size for big heaps and/or a lot of Ps (which
	// tend to be correlated).
	//
	// It's not too bad to burn relatively large-but-fixed amounts of per-P
	// memory if we need to deal with really, really deep queues, since the
	// constants of proportionality are small. Simultaneously, we want to
	// avoid a situation where a single worker ends up queuing O(heap)
	// work and then forever retains a queue of that size.
	const  = 1 << 20 / goarch.PtrSize
	 := .chain.head.cap * 2
	if  >  {
		 = 
	}
	 := newSpanSPMC()
	if !.tryDrain(, ) {
		throw("failed to put span on newly-allocated spanSPMC")
	}
	.chain.head.prev.StoreNoWB(unsafe.Pointer())
	.chain.head = 
}

// tryDrain attempts to drain n spans from q's local queue to the chain.
//
// Returns whether it succeeded.
func ( *spanQueue) ( *spanSPMC,  uint32) bool {
	if .head+ > .tail {
		throw("attempt to drain too many elements")
	}
	 := .head.Load() // synchronize with consumers
	 := .tail.Load()
	 :=  - 
	if + <= .cap {
		for  := uint32(0);  < ; ++ {
			*.slot( + ) = .ring[(.head+)%uint32(len(.ring))]
		}
		.tail.Store( + ) // Makes the items avail for consumption.
		.head += 
		return true
	}
	return false
}

// tryGetFast attempts to get a span from the local queue, but may fail if it's empty,
// returning false.
func ( *spanQueue) () objptr {
	if .tail-.head == 0 {
		return 0
	}
	 := .ring[.head%uint32(len(.ring))]
	.head++
	return 
}

// steal takes some spans from the ring chain of another span queue.
//
// q == q2 is OK.
func ( *spanQueue) ( *spanQueue) objptr {
	 := (*spanSPMC)(.chain.tail.Load())
	if  == nil {
		return 0
	}
	for {
		// It's important that we load the next pointer
		// *before* popping the tail. In general, r may be
		// transiently empty, but if next is non-nil before
		// the pop and the pop fails, then r is permanently
		// empty, which is the only condition under which it's
		// safe to drop r from the chain.
		 := (*spanSPMC)(.prev.Load())

		// Try to refill from one of the rings
		if  := .refill();  != 0 {
			return 
		}

		if  == nil {
			// This is the only ring. It's empty right
			// now, but could be pushed to in the future.
			return 0
		}

		// The tail of the chain has been drained, so move on
		// to the next ring. Try to drop it from the chain
		// so the next consumer doesn't have to look at the empty
		// ring again.
		if .chain.tail.CompareAndSwapNoWB(unsafe.Pointer(), unsafe.Pointer()) {
			.dead.Store(true)
		}

		 = 
	}
}

// refill takes some spans from r and puts them into q's local queue.
//
// One span is removed from the stolen spans and returned on success.
// Failure to steal returns a zero objptr.
//
// steal is thread-safe with respect to r.
func ( *spanQueue) ( *spanSPMC) objptr {
	if .tail-.head != 0 {
		throw("steal with local work available")
	}

	// Steal some spans.
	var  uint32
	for {
		 := .head.Load() // load-acquire, synchronize with other consumers
		 := .tail.Load() // load-acquire, synchronize with the producer
		 =  - 
		 =  - /2
		if  == 0 {
			return 0
		}
		if  > .cap { // read inconsistent h and t
			continue
		}
		 = min(, uint32(len(.ring)/2))
		for  := uint32(0);  < ; ++ {
			.ring[] = *.slot( + )
		}
		if .head.CompareAndSwap(, +) {
			break
		}
	}

	// Update local queue head and tail to reflect new buffered values.
	.head = 0
	.tail = 

	// Pop off the head of the queue and return it.
	return .tryGetFast()
}

// destroy frees all chains in an empty spanQueue.
//
// Preconditions:
// - World is stopped.
// - GC is outside of the mark phase.
// - (Therefore) the queue is empty.
func ( *spanQueue) () {
	assertWorldStopped()
	if gcphase != _GCoff {
		throw("spanQueue.destroy during the mark phase")
	}
	if !.empty() {
		throw("spanQueue.destroy on non-empty queue")
	}

	lock(&work.spanSPMCs.lock)

	// Remove, deinitialize, and free each ring.
	for  := (*spanSPMC)(.chain.tail.Load());  != nil;  = (*spanSPMC)(.prev.Load()) {
		work.spanSPMCs.list.remove(unsafe.Pointer())
		.deinit()
		mheap_.spanSPMCAlloc.free(unsafe.Pointer())
	}

	.chain.head = nil
	.chain.tail.Store(nil)
	.putsSinceDrain = 0

	unlock(&work.spanSPMCs.lock)
}

// spanSPMC is a ring buffer of objptrs that represent spans.
// Accessed without a lock.
//
// Single-producer, multi-consumer. The only producer is the P that owns this
// queue, but any other P may consume from it.
//
// ## Invariants for memory management
//
// 1. All spanSPMCs are allocated from mheap_.spanSPMCAlloc.
// 2. All allocated spanSPMCs must be on the work.spanSPMCs list.
// 3. spanSPMCs may only be allocated if gcphase != _GCoff.
// 4. spanSPMCs may only be deallocated if gcphase == _GCoff.
//
// Invariants (3) and (4) ensure that we do not need to concern ourselves with
// tricky reuse issues that stem from not knowing when a thread is truly done
// with a spanSPMC. For example, two threads could load the same spanSPMC from
// the tail of the chain. One thread is then paused while the other steals the
// last few elements off of it. It's not safe to free at that point since the
// other thread will still inspect that spanSPMC, and we have no way of knowing
// without more complex and/or heavyweight synchronization.
//
// Instead, we rely on the global synchronization inherent to GC phases, and
// the fact that spanSPMCs are only ever used during the mark phase, to ensure
// memory safety. This means we temporarily waste some memory, but it's only
// until the end of the mark phase.
type spanSPMC struct {
	_ sys.NotInHeap

	// allnode is the linked list node for work.spanSPMCs list. This is
	// used to find and free dead spanSPMCs. Protected by
	// work.spanSPMCs.lock.
	allnode listNodeManual

	// dead indicates whether the spanSPMC is no longer in use.
	// Protected by the CAS to the prev field of the spanSPMC pointing
	// to this spanSPMC. That is, whoever wins that CAS takes ownership
	// of marking this spanSPMC as dead. See spanQueue.steal for details.
	dead atomic.Bool

	// prev is the next link up a spanQueue's SPMC chain, from tail to head,
	// hence the name "prev." Set by a spanQueue's producer, cleared by a
	// CAS in spanQueue.steal.
	prev atomic.UnsafePointer // *spanSPMC

	// head, tail, cap, and ring together represent a fixed-size SPMC lock-free
	// ring buffer of size cap. The ring buffer contains objptr values.
	head atomic.Uint32
	tail atomic.Uint32
	cap  uint32 // cap(ring))
	ring *objptr
}

// newSpanSPMC allocates and initializes a new spmc with the provided capacity.
//
// newSpanSPMC may override the capacity with a larger one if the provided one would
// waste memory.
func newSpanSPMC( uint32) *spanSPMC {
	lock(&work.spanSPMCs.lock)
	 := (*spanSPMC)(mheap_.spanSPMCAlloc.alloc())
	work.spanSPMCs.list.push(unsafe.Pointer())
	unlock(&work.spanSPMCs.lock)

	// If cap < the capacity of a single physical page, round up.
	 := uint32(physPageSize / goarch.PtrSize) // capacity of a single page
	if  <  {
		 = 
	}
	if &(-1) != 0 {
		throw("spmc capacity must be a power of 2")
	}

	.cap = 
	 := sysAlloc(uintptr()*unsafe.Sizeof(objptr(0)), &memstats.gcMiscSys, "GC span queue")
	atomic.StorepNoWB(unsafe.Pointer(&.ring), )
	return 
}

// empty returns true if the spmc is empty.
//
// empty is thread-safe.
func ( *spanSPMC) () bool {
	 := .head.Load()
	 := .tail.Load()
	return  == 
}

// deinit frees any resources the spanSPMC is holding onto and zeroes it.
func ( *spanSPMC) () {
	sysFree(unsafe.Pointer(.ring), uintptr(.cap)*unsafe.Sizeof(objptr(0)), &memstats.gcMiscSys)
	.ring = nil
	.dead.Store(false)
	.prev.StoreNoWB(nil)
	.head.Store(0)
	.tail.Store(0)
	.cap = 0
	.allnode = listNodeManual{}
}

// slot returns a pointer to slot i%r.cap.
func ( *spanSPMC) ( uint32) *objptr {
	 := uintptr( & (.cap - 1))
	return (*objptr)(unsafe.Add(unsafe.Pointer(.ring), *unsafe.Sizeof(objptr(0))))
}

// freeDeadSpanSPMCs frees dead spanSPMCs back to the OS.
func freeDeadSpanSPMCs() {
	// According to the SPMC memory management invariants, we can only free
	// spanSPMCs outside of the mark phase. We ensure we do this in two ways.
	//
	// 1. We take the work.spanSPMCs lock, which we need anyway. This ensures
	//    that we are non-preemptible. If this path becomes lock-free, we will
	//    need to become non-preemptible in some other way.
	// 2. Once we are non-preemptible, we check the gcphase, and back out if
	//    it's not safe.
	//
	// This way, we ensure that we don't start freeing if we're in the wrong
	// phase, and the phase can't change on us while we're freeing.
	//
	// TODO(go.dev/issue/75771): Due to the grow semantics in
	// spanQueue.drain, we expect a steady-state of around one spanSPMC per
	// P, with some spikes higher when Ps have more than one. For high
	// GOMAXPROCS, or if this list otherwise gets long, it would be nice to
	// have a way to batch work that allows preemption during processing.
	lock(&work.spanSPMCs.lock)
	if gcphase != _GCoff || work.spanSPMCs.list.empty() {
		unlock(&work.spanSPMCs.lock)
		return
	}
	 := (*spanSPMC)(work.spanSPMCs.list.head())
	for  != nil {
		 := (*spanSPMC)(unsafe.Pointer(.allnode.next))
		if .dead.Load() {
			// It's dead. Remove, deinitialize and free it.
			work.spanSPMCs.list.remove(unsafe.Pointer())
			.deinit()
			mheap_.spanSPMCAlloc.free(unsafe.Pointer())
		}
		 = 
	}
	unlock(&work.spanSPMCs.lock)
}

// tryStealSpan attempts to steal a span from another P's local queue.
//
// Returns a non-zero objptr on success.
func ( *gcWork) () objptr {
	 := getg().m.p.ptr()

	for  := stealOrder.start(cheaprand()); !.done(); .next() {
		if !work.spanqMask.read(.position()) {
			continue
		}
		 := allp[.position()]
		if  ==  {
			continue
		}
		if  := .spanq.steal(&.gcw.spanq);  != 0 {
			return 
		}
		// N.B. This is intentionally racy. We may stomp on a mask set by
		// a P that just put a bunch of work into its local queue.
		//
		// This is OK because the ragged barrier in gcMarkDone will set
		// the bit on each P if there's local work we missed. This race
		// should generally be rare, since the window between noticing
		// an empty local queue and this bit being set is quite small.
		work.spanqMask.clear(int32(.position()))
	}
	return 0
}

// objptr consists of a span base and the index of the object in the span.
type objptr uintptr

// makeObjPtr creates an objptr from a span base address and an object index.
func makeObjPtr( uintptr,  uint16) objptr {
	if doubleCheckGreenTea && &((1<<gc.PageShift)-1) != 0 {
		throw("created objptr with address that is incorrectly aligned")
	}
	return objptr( | uintptr())
}

func ( objptr) () uintptr {
	return uintptr() &^ ((1 << gc.PageShift) - 1)
}

func ( objptr) () uint16 {
	return uint16() & ((1 << gc.PageShift) - 1)
}

// scanSpan scans objects indicated marks&^scans and then scans those objects,
// queuing the resulting pointers into gcw.
func scanSpan( objptr,  *gcWork) {
	 := .spanBase()
	 := spanInlineMarkBitsFromBase()
	 := .class
	if .noscan() {
		throw("noscan object in scanSpan")
	}
	 := uintptr(gc.SizeClassToSize[.sizeclass()])

	// Release span.
	if .release() == spanScanOneMark {
		// Nobody else set any mark bits on this span while it was acquired.
		// That means p is the sole object we need to handle. Fast-track it.
		 := .objIndex()
		 := &.scans[/8]
		 := uint8(1) << ( % 8)
		if atomic.Load8()& != 0 {
			return
		}
		atomic.Or8(, )
		.bytesMarked += uint64()
		if debug.gctrace > 1 {
			.stats[.sizeclass()].sparseObjsScanned++
		}
		 :=  + uintptr()*
		scanObjectSmall(, , , )
		return
	}

	// Compute nelems.
	 := uint64(gc.SizeClassToDivMagic[.sizeclass()])
	 := uint64(gc.PageSize - unsafe.Sizeof(spanInlineMarkBits{}))
	if !.noscan() {
		 -= gc.PageSize / goarch.PtrSize / 8
	}
	 := uint16(( * ) >> 32)

	// Grey objects and return if there's nothing else to do.
	var  gc.ObjMask
	 := spanSetScans(, , , &)
	if  == 0 {
		return
	}
	.bytesMarked += uint64() * uint64()

	// Check if we have enough density to make a dartboard scan
	// worthwhile. If not, just do what scanobject does, but
	// localized to the span, using the dartboard.
	if !scan.HasFastScanSpanPacked() ||  < int(/8) {
		if debug.gctrace > 1 {
			.stats[.sizeclass()].spansSparseScanned++
			.stats[.sizeclass()].spanObjsSparseScanned += uint64()
		}
		scanObjectsSmall(, , , , &)
		return
	}

	// Scan the span.
	//
	// N.B. Use gcw.ptrBuf as the output buffer. This is a bit different
	// from scanObjectsSmall, which puts addresses to dereference. ScanSpanPacked
	// on the other hand, fills gcw.ptrBuf with already dereferenced pointers.
	 := scan.ScanSpanPacked(
		unsafe.Pointer(),
		&.ptrBuf[0],
		&,
		uintptr(.sizeclass()),
		spanPtrMaskUnsafe(),
	)
	.heapScanWork += int64() * int64()

	if debug.gctrace > 1 {
		// Write down some statistics.
		.stats[.sizeclass()].spansDenseScanned++
		.stats[.sizeclass()].spanObjsDenseScanned += uint64()
	}

	// Process all the pointers we just got.
	for ,  := range .ptrBuf[:] {
		if !tryDeferToSpanScan(, ) {
			if , ,  := findObject(, 0, 0);  != 0 {
				greyobject(, 0, 0, , , )
			}
		}
	}
}

// spanSetScans sets any unset mark bits that have their mark bits set in the inline mark bits.
//
// toScan is populated with bits indicating whether a particular mark bit was set.
//
// Returns the number of objects marked, which could be zero.
func spanSetScans( uintptr,  uint16,  *spanInlineMarkBits,  *gc.ObjMask) int {
	, ,  := pageIndexOf()
	if .pageMarks[]& == 0 {
		atomic.Or8(&.pageMarks[], )
	}

	 := divRoundUp(uintptr(), 8)
	 := 0

	// Careful: these two structures alias since ObjMask is much bigger
	// than marks or scans. We do these unsafe shenanigans so that we can
	// access the marks and scans by uintptrs rather than by byte.
	 := (*gc.ObjMask)(unsafe.Pointer(&.marks))
	 := (*gc.ObjMask)(unsafe.Pointer(&.scans))

	// Iterate over one uintptr-sized chunks at a time, computing both
	// the union and intersection of marks and scans. Store the union
	// into scans, and the intersection into toScan.
	for  := uintptr(0);  < ;  += goarch.PtrSize {
		 := atomic.Loaduintptr(&[/goarch.PtrSize])
		 := [/goarch.PtrSize]
		 = bswapIfBigEndian()
		 = bswapIfBigEndian()
		if /goarch.PtrSize == uintptr(len(.marks)+1)/goarch.PtrSize-1 {
			 &^= 0xff << ((goarch.PtrSize - 1) * 8) // mask out owned
			 &^= 0xff << ((goarch.PtrSize - 1) * 8) // mask out class
		}
		 :=  &^ 
		[/goarch.PtrSize] = 

		// If there's anything left to grey, do it.
		if  != 0 {
			 = bswapIfBigEndian()
			if goarch.PtrSize == 4 {
				atomic.Or32((*uint32)(unsafe.Pointer(&[/goarch.PtrSize])), uint32())
			} else {
				atomic.Or64((*uint64)(unsafe.Pointer(&[/goarch.PtrSize])), uint64())
			}
		}
		 += sys.OnesCount64(uint64())
	}
	return 
}

func scanObjectSmall(, ,  uintptr,  *gcWork) {
	,  := spanHeapBitsRange(, gc.PageSize, )
	 := (*byte)(unsafe.Pointer())
	 := extractHeapBitsSmall(, , , )
	.heapScanWork += int64(sys.Len64(uint64()) * goarch.PtrSize)
	 := 0
	 := sys.OnesCount64(uint64())
	for range  {
		 := sys.TrailingZeros64(uint64())
		 &^= 1 << 
		 :=  + uintptr()*goarch.PtrSize

		// Prefetch addr since we're about to use it. This point for prefetching
		// was chosen empirically.
		sys.Prefetch()

		// N.B. ptrBuf is always large enough to hold pointers for an entire 1-page span.
		.ptrBuf[] = 
		++
	}

	// Process all the pointers we just got.
	for ,  := range .ptrBuf[:] {
		 = *(*uintptr)(unsafe.Pointer())
		if  == 0 {
			continue
		}
		if !tryDeferToSpanScan(, ) {
			if , ,  := findObject(, 0, 0);  != 0 {
				greyobject(, 0, 0, , , )
			}
		}
	}
}

func scanObjectsSmall(,  uintptr,  uint16,  *gcWork,  *gc.ObjMask) {
	 := 0
	for ,  := range  {
		if *(goarch.PtrSize*8) > int() {
			break
		}
		 := sys.OnesCount64(uint64())
		,  := spanHeapBitsRange(, gc.PageSize, )
		 := (*byte)(unsafe.Pointer())
		for range  {
			 := sys.TrailingZeros64(uint64())
			 &^= 1 << 

			 :=  + uintptr(*(goarch.PtrSize*8)+)*
			 := extractHeapBitsSmall(, , , )
			.heapScanWork += int64(sys.Len64(uint64()) * goarch.PtrSize)

			 := sys.OnesCount64(uint64())
			for range  {
				 := sys.TrailingZeros64(uint64())
				 &^= 1 << 
				 :=  + uintptr()*goarch.PtrSize

				// Prefetch addr since we're about to use it. This point for prefetching
				// was chosen empirically.
				sys.Prefetch()

				// N.B. ptrBuf is always large enough to hold pointers for an entire 1-page span.
				.ptrBuf[] = 
				++
			}
		}
	}

	// Process all the pointers we just got.
	for ,  := range .ptrBuf[:] {
		 = *(*uintptr)(unsafe.Pointer())
		if  == 0 {
			continue
		}
		if !tryDeferToSpanScan(, ) {
			if , ,  := findObject(, 0, 0);  != 0 {
				greyobject(, 0, 0, , , )
			}
		}
	}
}

func extractHeapBitsSmall( *byte, , ,  uintptr) uintptr {
	// These objects are always small enough that their bitmaps
	// fit in a single word, so just load the word or two we need.
	//
	// Mirrors mspan.writeHeapBitsSmall.
	//
	// We should be using heapBits(), but unfortunately it introduces
	// both bounds checks panics and throw which causes us to exceed
	// the nosplit limit in quite a few cases.
	 := ( - ) / goarch.PtrSize / ptrBits
	 := ( - ) / goarch.PtrSize % ptrBits
	 :=  / goarch.PtrSize
	 := (*uintptr)(unsafe.Pointer(addb(, goarch.PtrSize*(+0))))
	 := (*uintptr)(unsafe.Pointer(addb(, goarch.PtrSize*(+1))))

	var  uintptr
	if + > ptrBits {
		// Two reads.
		 := ptrBits - 
		 :=  - 
		 = * >> 
		 |= (* & ((1 << ) - 1)) << 
	} else {
		// One read.
		 = (* >> ) & ((1 << ) - 1)
	}
	return 
}

// spanPtrMaskUnsafe returns the pointer mask for a span with inline mark bits.
//
// The caller must ensure spanBase is the base of a span that:
// - 1 page in size,
// - Uses inline mark bits,
// - Contains pointers.
func spanPtrMaskUnsafe( uintptr) *gc.PtrMask {
	 :=  + gc.PageSize - unsafe.Sizeof(gc.PtrMask{}) - unsafe.Sizeof(spanInlineMarkBits{})
	return (*gc.PtrMask)(unsafe.Pointer())
}

type sizeClassScanStats struct {
	spansDenseScanned     uint64 // Spans scanned with ScanSpanPacked.
	spanObjsDenseScanned  uint64 // Objects scanned with ScanSpanPacked.
	spansSparseScanned    uint64 // Spans scanned with scanObjectsSmall.
	spanObjsSparseScanned uint64 // Objects scanned with scanObjectsSmall.
	sparseObjsScanned     uint64 // Objects scanned with scanobject or scanObjectSmall.
	// Note: sparseObjsScanned is sufficient for both cases because
	// a particular size class either uses scanobject or scanObjectSmall,
	// not both. In the latter case, we also know that there was one
	// object scanned per span, so no need for a span counter.
}

func dumpScanStats() {
	var (
		     uint64
		  uint64
		    uint64
		 uint64
		     uint64
	)
	for ,  := range memstats.lastScanStats {
		 += .spansDenseScanned
		 += .spanObjsDenseScanned
		 += .spansSparseScanned
		 += .spanObjsSparseScanned
		 += .sparseObjsScanned
	}
	 :=  +  + 
	 :=  + 
	print("scan: total ", , "+", , "+", , "=", , " objs")
	print(", ", , "+", , "=", , " spans\n")
	for ,  := range memstats.lastScanStats {
		if  == (sizeClassScanStats{}) {
			continue
		}
		 := .sparseObjsScanned + .spanObjsSparseScanned + .spanObjsDenseScanned
		 := .spansSparseScanned + .spansDenseScanned
		if  == 0 {
			print("scan: class L ")
		} else {
			print("scan: class ", gc.SizeClassToSize[], "B ")
		}
		print(.sparseObjsScanned, "+", .spanObjsSparseScanned, "+", .spanObjsDenseScanned, "=", , " objs")
		print(", ", .spansSparseScanned, "+", .spansDenseScanned, "=", , " spans\n")
	}
}

func ( *gcWork) ( *[gc.NumSizeClasses]sizeClassScanStats) {
	for  := range .stats {
		[].spansDenseScanned += .stats[].spansDenseScanned
		[].spanObjsDenseScanned += .stats[].spanObjsDenseScanned
		[].spansSparseScanned += .stats[].spansSparseScanned
		[].spanObjsSparseScanned += .stats[].spanObjsSparseScanned
		[].sparseObjsScanned += .stats[].sparseObjsScanned
	}
	clear(.stats[:])
}

// gcMarkWorkAvailable reports whether there's any non-local work available to do.
//
// This is a heavyweight check and must only be used for correctness, not
// as a hint.
func gcMarkWorkAvailable() bool {
	if !work.full.empty() {
		return true // global work available
	}
	if work.markrootNext.Load() < work.markrootJobs.Load() {
		return true // root scan work available
	}
	if work.spanqMask.any() {
		return true // stealable local work available
	}
	return false
}

// scanObject scans the object starting at b, adding pointers to gcw.
// b must point to the beginning of a heap object or an oblet.
// scanObject consults the GC bitmap for the pointer mask and the
// spans for the size of the object.
//
// Used only for !gcUsesSpanInlineMarkBits spans, but supports all
// object sizes and is safe to be called on all heap objects.
//
//go:nowritebarrier
func scanObject( uintptr,  *gcWork) {
	// Prefetch object before we scan it.
	//
	// This will overlap fetching the beginning of the object with initial
	// setup before we start scanning the object.
	sys.Prefetch()

	// Find the bits for b and the size of the object at b.
	//
	// b is either the beginning of an object, in which case this
	// is the size of the object to scan, or it points to an
	// oblet, in which case we compute the size to scan below.
	 := spanOfUnchecked()
	 := .elemsize
	if  == 0 {
		throw("scanObject n == 0")
	}
	if .spanclass.noscan() {
		// Correctness-wise this is ok, but it's inefficient
		// if noscan objects reach here.
		throw("scanObject of a noscan object")
	}

	var  typePointers
	if  > maxObletBytes {
		// Large object. Break into oblets for better
		// parallelism and lower latency.
		if  == .base() {
			// Enqueue the other oblets to scan later.
			// Some oblets may be in b's scalar tail, but
			// these will be marked as "no more pointers",
			// so we'll drop out immediately when we go to
			// scan those.
			for  :=  + maxObletBytes;  < .base()+.elemsize;  += maxObletBytes {
				if !.putObjFast() {
					.putObj()
				}
			}
		}

		// Compute the size of the oblet. Since this object
		// must be a large object, s.base() is the beginning
		// of the object.
		 = .base() + .elemsize - 
		 = min(, maxObletBytes)
		 = .typePointersOfUnchecked(.base())
		 = .fastForward(-.addr, +)
	} else {
		 = .typePointersOfUnchecked()
	}

	var  uintptr
	for {
		var  uintptr
		if ,  = .nextFast();  == 0 {
			if ,  = .next( + );  == 0 {
				break
			}
		}

		// Keep track of farthest pointer we found, so we can
		// update heapScanWork. TODO: is there a better metric,
		// now that we can skip scalar portions pretty efficiently?
		 =  -  + goarch.PtrSize

		// Work here is duplicated in scanblock and above.
		// If you make changes here, make changes there too.
		 := *(*uintptr)(unsafe.Pointer())

		// At this point we have extracted the next potential pointer.
		// Quickly filter out nil and pointers back to the current object.
		if  != 0 && - >=  {
			// Test if obj points into the Go heap and, if so,
			// mark the object.
			//
			// Note that it's possible for findObject to
			// fail if obj points to a just-allocated heap
			// object because of a race with growing the
			// heap. In this case, we know the object was
			// just allocated and hence will be marked by
			// allocation itself.
			if !tryDeferToSpanScan(, ) {
				if , ,  := findObject(, , -);  != 0 {
					greyobject(, , -, , , )
				}
			}
		}
	}
	.bytesMarked += uint64()
	.heapScanWork += int64()
	if debug.gctrace > 1 {
		.stats[.spanclass.sizeclass()].sparseObjsScanned++
	}
}