// 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.greenteagcpackage runtimeimport ()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 uint8const ( 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 {returnspanScanOwnership(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) * 8ifgoarch.BigEndian { = 32 - - 8 }returnspanScanOwnership(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() {casespanScanUnowned:// Try to mark the span as having only one object marked.if .owned.or(spanScanOneMark) == spanScanUnowned {returntrue }// 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.fallthroughcasespanScanOneMark:// 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 }returnfalse}// 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 {returnspanScanOwnership(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) () {ifdoubleCheckGreenTea && !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) {ifdoubleCheckGreenTea && !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() }ifdoubleCheckGreenTea && !.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 {ifdoubleCheckGreenTea && !gcUsesSpanInlineMarkBits(.elemsize) {throw("expected span with inline mark bits") }returnspanInlineMarkBitsFromBase(.base())}func ( *mspan) ( uintptr) ( markBits) {ifgcUsesSpanInlineMarkBits(.elemsize) { .bytep = &.inlineMarkBits().marks[/8] } else { .bytep = .gcmarkBits.bytep( / 8) } .mask = uint8(1) << ( % 8) .index = return}func ( *mspan) () markBits {ifgcUsesSpanInlineMarkBits(.elemsize) {returnmarkBits{&.inlineMarkBits().marks[0], uint8(1), 0} }returnmarkBits{&.gcmarkBits.x, uint8(1), 0}}// scannedBitsForIndex returns a markBits representing the scanned bit// for objIndex in the inline mark bits.func ( *mspan) ( uintptr) markBits {returnmarkBits{&.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:nosplitfunc gcUsesSpanInlineMarkBits( uintptr) bool {returnheapBitsInSpan() && >= 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 {ifuseCheckmark {returnfalse }// Quickly to see if this is a span that has inline mark bits. := heapArenaOf()if == nil {returnfalse } := (( / pageSize) / 8) % uintptr(len(.pageInUse)) := byte(1 << (( / pageSize) % 8))if .pageUseSpanInlineMarkBits[]& == 0 {returnfalse }// 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)ifatomic.Load8(&.marks[])& != 0 {returntrue }atomic.Or8(&.marks[], )// Fast-track noscan objects.if .class.noscan() { .bytesMarked += uint64(gc.SizeClassToSize[.class.sizeclass()])returntrue }// Queue up the pointer (as a representative for its span).if .tryAcquire() {if .spanq.put(makeObjPtr(, )) {ifgcphase == _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 } }returntrue}// 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.ifwork.spanqMask.read(uint32(.id)) {work.spanqMask.clear(.id) }return0}// 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)) {returnfalse } .ring[.tail%uint32(len(.ring))] = .tail++returntrue}// 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()returntrue } }returnfalse }// We're out of space. Drain out our local spans. .drain(uint32(len(.ring)) / 2)if !.putFast() {throw("failed putFast after drain") }returntrue}// flush publishes all spans in the local queue to the spmc chain.func ( *spanQueue) () { := .tail - .headif == 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 {returnfalse }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() {returnfalse } = (*spanSPMC)(.prev.Load()) }returntrue}// drain publishes n spans from the local queue to the spmc chain.func ( *spanQueue) ( uint32) { .putsSinceDrain = 0if .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 * 2if > { = } := 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 += returntrue }returnfalse}// 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 {return0 } := .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 {return0 }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 ringsif := .refill(); != 0 {return }if == nil {// This is the only ring. It's empty right // now, but could be pushed to in the future.return0 }// 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.varuint32for { := .head.Load() // load-acquire, synchronize with other consumers := .tail.Load() // load-acquire, synchronize with the producer = - = - /2if == 0 {return0 }if > .cap { // read inconsistent h and tcontinue } = 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()ifgcphase != _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 = 0unlock(&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 pageif < { = }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)ifgcphase != _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())) }return0}// 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 {ifdoubleCheckGreenTea && &((1<<gc.PageShift)-1) != 0 {throw("created objptr with address that is incorrectly aligned") }returnobjptr( | uintptr())}func ( objptr) () uintptr {returnuintptr() &^ ((1 << gc.PageShift) - 1)}func ( objptr) () uint16 {returnuint16() & ((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() := .classif .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)ifatomic.Load8()& != 0 {return }atomic.Or8(, ) .bytesMarked += uint64()ifdebug.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.vargc.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) {ifdebug.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()ifdebug.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()ifgoarch.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())forrange { := 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) { := 0for , := range {if *(goarch.PtrSize*8) > int() {break } := sys.OnesCount64(uint64()) , := spanHeapBitsRange(, gc.PageSize, ) := (*byte)(unsafe.Pointer())forrange { := sys.TrailingZeros64(uint64()) &^= 1 << := + uintptr(*(goarch.PtrSize*8)+)* := extractHeapBitsSmall(, , , ) .heapScanWork += int64(sys.Len64(uint64()) * goarch.PtrSize) := sys.OnesCount64(uint64())forrange { := 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))))varuintptrif + > 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 (uint64uint64uint64uint64uint64 )for , := rangememstats.lastScanStats { += .spansDenseScanned += .spanObjsDenseScanned += .spansSparseScanned += .spanObjsSparseScanned += .sparseObjsScanned } := + + := + print("scan: total ", , "+", , "+", , "=", , " objs")print(", ", , "+", , "=", , " spans\n")for , := rangememstats.lastScanStats {if == (sizeClassScanStats{}) {continue } := .sparseObjsScanned + .spanObjsSparseScanned + .spanObjsDenseScanned := .spansSparseScanned + .spansDenseScannedif == 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() {returntrue// global work available }ifwork.markrootNext.Load() < work.markrootJobs.Load() {returntrue// root scan work available }ifwork.spanqMask.any() {returntrue// stealable local work available }returnfalse}// 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:nowritebarrierfunc 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() := .elemsizeif == 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") }vartypePointersif > 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() }varuintptrfor {varuintptrif , = .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()ifdebug.gctrace > 1 { .stats[.spanclass.sizeclass()].sparseObjsScanned++ }}
The pages are generated with Goldsv0.8.3-preview. (GOOS=linux GOARCH=amd64)
Golds is a Go 101 project developed by Tapir Liu.
PR and bug reports are welcome and can be submitted to the issue list.
Please follow @zigo_101 (reachable from the left QR code) to get the latest news of Golds.