Source file src/runtime/mgcmark_greenteagc.go

     1  // Copyright 2025 The Go Authors. All rights reserved.
     2  // Use of this source code is governed by a BSD-style
     3  // license that can be found in the LICENSE file.
     4  
     5  // Green Tea mark algorithm
     6  //
     7  // The core idea behind Green Tea is simple: achieve better locality during
     8  // mark/scan by delaying scanning so that we can accumulate objects to scan
     9  // within the same span, then scan the objects that have accumulated on the
    10  // span all together.
    11  //
    12  // By batching objects this way, we increase the chance that adjacent objects
    13  // will be accessed, amortize the cost of accessing object metadata, and create
    14  // better opportunities for prefetching. We can take this even further and
    15  // optimize the scan loop by size class (not yet completed) all the way to the
    16  // point of applying SIMD techniques to really tear through the heap.
    17  //
    18  // Naturally, this depends on being able to create opportunties to batch objects
    19  // together. The basic idea here is to have two sets of mark bits. One set is the
    20  // regular set of mark bits ("marks"), while the other essentially says that the
    21  // objects have been scanned already ("scans"). When we see a pointer for the first
    22  // time we set its mark and enqueue its span. We track these spans in work queues
    23  // with a FIFO policy, unlike workbufs which have a LIFO policy. Empirically, a
    24  // FIFO policy appears to work best for accumulating objects to scan on a span.
    25  // Later, when we dequeue the span, we find both the union and intersection of the
    26  // mark and scan bitsets. The union is then written back into the scan bits, while
    27  // the intersection is used to decide which objects need scanning, such that the GC
    28  // is still precise.
    29  //
    30  // Below is the bulk of the implementation, focusing on the worst case
    31  // for locality, small objects. Specifically, those that are smaller than
    32  // a few cache lines in size and whose metadata is stored the same way (at the
    33  // end of the span).
    34  
    35  //go:build goexperiment.greenteagc
    36  
    37  package runtime
    38  
    39  import (
    40  	"internal/cpu"
    41  	"internal/goarch"
    42  	"internal/runtime/atomic"
    43  	"internal/runtime/gc"
    44  	"internal/runtime/sys"
    45  	"unsafe"
    46  )
    47  
    48  const doubleCheckGreenTea = false
    49  
    50  // spanInlineMarkBits are mark bits that are inlined into the span
    51  // itself. gcUsesSpanInlineMarkBits may be used to check if objects
    52  // of a particular size use inline mark bits.
    53  //
    54  // Inline mark bits are a little bit more than just mark bits. They
    55  // consist of two parts: scans and marks. Marks are like pre-mark
    56  // bits. They're set once a pointer to an object is discovered for
    57  // the first time. The marks allow us to scan many objects in bulk
    58  // if we queue the whole span for scanning. Before we scan such objects
    59  // in bulk, we copy the marks to the scans, computing a diff along the
    60  // way. The resulting bitmap tells us which objects we should scan.
    61  //
    62  // The inlineMarkBits also hold state sufficient for scanning any
    63  // object in the span, as well as state for acquiring ownership of
    64  // the span for queuing. This avoids the need to look at the mspan when
    65  // scanning.
    66  type spanInlineMarkBits struct {
    67  	scans [63]uint8         // scanned bits.
    68  	owned spanScanOwnership // see the comment on spanScanOwnership.
    69  	marks [63]uint8         // mark bits.
    70  	class spanClass
    71  }
    72  
    73  // spanScanOwnership indicates whether some thread has acquired
    74  // the span for scanning, and whether there has been one or more
    75  // attempts to acquire the span. The latter information helps to
    76  // fast-track span scans that only apply to a single mark, skipping
    77  // the relatively costly merge-and-diff process for scans and marks
    78  // by allowing one to just set the mark directly.
    79  type spanScanOwnership uint8
    80  
    81  const (
    82  	spanScanUnowned  spanScanOwnership = 0         // Indicates the span is not acquired for scanning.
    83  	spanScanOneMark                    = 1 << iota // Indicates that only one mark bit is set relative to the scan bits.
    84  	spanScanManyMark                               // Indicates one or more scan bits may be set relative to the mark bits.
    85  	// "ManyMark" need not be exactly the value it has. In practice we just
    86  	// want to distinguish "none" from "one" from "many," so a comparison is
    87  	// sufficient (as opposed to a bit test) to check between these cases.
    88  )
    89  
    90  // load atomically loads from a pointer to a spanScanOwnership.
    91  func (o *spanScanOwnership) load() spanScanOwnership {
    92  	return spanScanOwnership(atomic.Load8((*uint8)(unsafe.Pointer(o))))
    93  }
    94  
    95  func (o *spanScanOwnership) or(v spanScanOwnership) spanScanOwnership {
    96  	// N.B. We round down the address and use Or32 because Or8 doesn't
    97  	// return a result, and it's strictly necessary for this protocol.
    98  	//
    99  	// Making Or8 return a result, while making the code look nicer, would
   100  	// not be strictly better on any supported platform, as an Or8 that
   101  	// returns a result is not a common instruction. On many platforms it
   102  	// would be implemented exactly as it is here, and since Or8 is
   103  	// exclusively used in the runtime and a hot function, we want to keep
   104  	// using its no-result version elsewhere for performance.
   105  	o32 := (*uint32)(unsafe.Pointer(uintptr(unsafe.Pointer(o)) &^ 0b11))
   106  	off := (uintptr(unsafe.Pointer(o)) & 0b11) * 8
   107  	if goarch.BigEndian {
   108  		off = 32 - off - 8
   109  	}
   110  	return spanScanOwnership(atomic.Or32(o32, uint32(v)<<off) >> off)
   111  }
   112  
   113  func (imb *spanInlineMarkBits) init(class spanClass, needzero bool) {
   114  	if imb == nil {
   115  		// This nil check and throw is almost pointless. Normally we would
   116  		// expect imb to never be nil. However, this is called on potentially
   117  		// freshly-allocated virtual memory. As of 2025, the compiler-inserted
   118  		// nil check is not a branch but a memory read that we expect to fault
   119  		// if the pointer really is nil.
   120  		//
   121  		// However, this causes a read of the page, and operating systems may
   122  		// take it as a hint to back the accessed memory with a read-only zero
   123  		// page. However, we immediately write to this memory, which can then
   124  		// force operating systems to have to update the page table and flush
   125  		// the TLB, causing a lot of churn for programs that are short-lived
   126  		// and monotonically grow in size.
   127  		//
   128  		// This nil check is thus an explicit branch instead of what the compiler
   129  		// would insert circa 2025, which is a memory read instruction.
   130  		//
   131  		// See go.dev/issue/74375 for details.
   132  		throw("runtime: span inline mark bits nil?")
   133  	}
   134  	if needzero {
   135  		// Use memclrNoHeapPointers to avoid having the compiler make a worse
   136  		// decision. We know that imb is both aligned and a nice power-of-two
   137  		// size that works well for wider SIMD instructions. The compiler likely
   138  		// has no idea that imb is aligned to 128 bytes.
   139  		memclrNoHeapPointers(unsafe.Pointer(imb), unsafe.Sizeof(spanInlineMarkBits{}))
   140  	}
   141  	imb.class = class
   142  }
   143  
   144  // tryAcquire attempts to acquire the span for scanning. On success, the caller
   145  // must queue the span for scanning or scan the span immediately.
   146  func (imb *spanInlineMarkBits) tryAcquire() bool {
   147  	switch imb.owned.load() {
   148  	case spanScanUnowned:
   149  		// Try to mark the span as having only one object marked.
   150  		if imb.owned.or(spanScanOneMark) == spanScanUnowned {
   151  			return true
   152  		}
   153  		// If we didn't see an old value of spanScanUnowned, then we must
   154  		// have raced with someone else and seen spanScanOneMark or greater.
   155  		// Fall through and try to set spanScanManyMark.
   156  		fallthrough
   157  	case spanScanOneMark:
   158  		// We may be the first to set *any* bit on owned. In such a case,
   159  		// we still need to make sure the span is queued.
   160  		return imb.owned.or(spanScanManyMark) == spanScanUnowned
   161  	}
   162  	return false
   163  }
   164  
   165  // release releases the span for scanning, allowing another thread to queue the span.
   166  //
   167  // Returns an upper bound on the number of mark bits set since the span was queued. The
   168  // upper bound is described as "one" (spanScanOneMark) or "many" (spanScanManyMark, with or
   169  // without spanScanOneMark). If the return value indicates only one mark bit was set, the
   170  // caller can be certain that it was the same mark bit that caused the span to get queued.
   171  // Take note of the fact that this is *only* an upper-bound. In particular, it may still
   172  // turn out that only one mark bit was set, even if the return value indicates "many".
   173  func (imb *spanInlineMarkBits) release() spanScanOwnership {
   174  	return spanScanOwnership(atomic.Xchg8((*uint8)(unsafe.Pointer(&imb.owned)), uint8(spanScanUnowned)))
   175  }
   176  
   177  // spanInlineMarkBitsFromBase returns the spanInlineMarkBits for a span whose start address is base.
   178  //
   179  // The span must be gcUsesSpanInlineMarkBits(span.elemsize).
   180  func spanInlineMarkBitsFromBase(base uintptr) *spanInlineMarkBits {
   181  	return (*spanInlineMarkBits)(unsafe.Pointer(base + gc.PageSize - unsafe.Sizeof(spanInlineMarkBits{})))
   182  }
   183  
   184  // initInlineMarkBits initializes the inlineMarkBits stored at the end of the span.
   185  func (s *mspan) initInlineMarkBits() {
   186  	if doubleCheckGreenTea && !gcUsesSpanInlineMarkBits(s.elemsize) {
   187  		throw("expected span with inline mark bits")
   188  	}
   189  	// Zeroing is only necessary if this span wasn't just freshly allocated from the OS.
   190  	s.inlineMarkBits().init(s.spanclass, s.needzero != 0)
   191  }
   192  
   193  // moveInlineMarks merges the span's inline mark bits into dst and clears them.
   194  //
   195  // gcUsesSpanInlineMarkBits(s.elemsize) must be true.
   196  func (s *mspan) moveInlineMarks(dst *gcBits) {
   197  	if doubleCheckGreenTea && !gcUsesSpanInlineMarkBits(s.elemsize) {
   198  		throw("expected span with inline mark bits")
   199  	}
   200  	bytes := divRoundUp(uintptr(s.nelems), 8)
   201  	imb := s.inlineMarkBits()
   202  	imbMarks := (*gc.ObjMask)(unsafe.Pointer(&imb.marks))
   203  	for i := uintptr(0); i < bytes; i += goarch.PtrSize {
   204  		marks := bswapIfBigEndian(imbMarks[i/goarch.PtrSize])
   205  		if i/goarch.PtrSize == uintptr(len(imb.marks)+1)/goarch.PtrSize-1 {
   206  			marks &^= 0xff << ((goarch.PtrSize - 1) * 8) // mask out class
   207  		}
   208  		*(*uintptr)(unsafe.Pointer(dst.bytep(i))) |= bswapIfBigEndian(marks)
   209  	}
   210  	if doubleCheckGreenTea && !s.spanclass.noscan() && imb.marks != imb.scans {
   211  		throw("marks don't match scans for span with pointer")
   212  	}
   213  
   214  	// Reset the inline mark bits.
   215  	imb.init(s.spanclass, true /* We know these bits are always dirty now. */)
   216  }
   217  
   218  // inlineMarkBits returns the inline mark bits for the span.
   219  //
   220  // gcUsesSpanInlineMarkBits(s.elemsize) must be true.
   221  func (s *mspan) inlineMarkBits() *spanInlineMarkBits {
   222  	if doubleCheckGreenTea && !gcUsesSpanInlineMarkBits(s.elemsize) {
   223  		throw("expected span with inline mark bits")
   224  	}
   225  	return spanInlineMarkBitsFromBase(s.base())
   226  }
   227  
   228  func (s *mspan) markBitsForIndex(objIndex uintptr) (bits markBits) {
   229  	if gcUsesSpanInlineMarkBits(s.elemsize) {
   230  		bits.bytep = &s.inlineMarkBits().marks[objIndex/8]
   231  	} else {
   232  		bits.bytep = s.gcmarkBits.bytep(objIndex / 8)
   233  	}
   234  	bits.mask = uint8(1) << (objIndex % 8)
   235  	bits.index = objIndex
   236  	return
   237  }
   238  
   239  func (s *mspan) markBitsForBase() markBits {
   240  	if gcUsesSpanInlineMarkBits(s.elemsize) {
   241  		return markBits{&s.inlineMarkBits().marks[0], uint8(1), 0}
   242  	}
   243  	return markBits{&s.gcmarkBits.x, uint8(1), 0}
   244  }
   245  
   246  // scannedBitsForIndex returns a markBits representing the scanned bit
   247  // for objIndex in the inline mark bits.
   248  func (s *mspan) scannedBitsForIndex(objIndex uintptr) markBits {
   249  	return markBits{&s.inlineMarkBits().scans[objIndex/8], uint8(1) << (objIndex % 8), objIndex}
   250  }
   251  
   252  // gcUsesSpanInlineMarkBits returns true if a span holding objects of a certain size
   253  // has inline mark bits. size must be the span's elemsize.
   254  //
   255  // nosplit because this is called from gcmarknewobject, which is nosplit.
   256  //
   257  //go:nosplit
   258  func gcUsesSpanInlineMarkBits(size uintptr) bool {
   259  	return heapBitsInSpan(size) && size >= 16
   260  }
   261  
   262  // tryQueueOnSpan tries to queue p on the span it points to, if it
   263  // points to a small object span (gcUsesSpanQueue size).
   264  func tryDeferToSpanScan(p uintptr, gcw *gcWork) bool {
   265  	if useCheckmark {
   266  		return false
   267  	}
   268  
   269  	// Quickly to see if this is a span that has inline mark bits.
   270  	ha := heapArenaOf(p)
   271  	if ha == nil {
   272  		return false
   273  	}
   274  	pageIdx := ((p / pageSize) / 8) % uintptr(len(ha.pageInUse))
   275  	pageMask := byte(1 << ((p / pageSize) % 8))
   276  	if ha.pageUseSpanInlineMarkBits[pageIdx]&pageMask == 0 {
   277  		return false
   278  	}
   279  
   280  	// Find the object's index from the span class info stored in the inline mark bits.
   281  	base := alignDown(p, gc.PageSize)
   282  	q := spanInlineMarkBitsFromBase(base)
   283  	objIndex := uint16((uint64(p-base) * uint64(gc.SizeClassToDivMagic[q.class.sizeclass()])) >> 32)
   284  
   285  	// Set mark bit.
   286  	idx, mask := objIndex/8, uint8(1)<<(objIndex%8)
   287  	if atomic.Load8(&q.marks[idx])&mask != 0 {
   288  		return true
   289  	}
   290  	atomic.Or8(&q.marks[idx], mask)
   291  
   292  	// Fast-track noscan objects.
   293  	if q.class.noscan() {
   294  		gcw.bytesMarked += uint64(gc.SizeClassToSize[q.class.sizeclass()])
   295  		return true
   296  	}
   297  
   298  	// Queue up the pointer (as a representative for its span).
   299  	if q.tryAcquire() {
   300  		if gcw.spanq.put(makeObjPtr(base, objIndex)) {
   301  			if gcphase == _GCmark {
   302  				gcw.mayNeedWorker = true
   303  			}
   304  			gcw.flushedWork = true
   305  		}
   306  	}
   307  	return true
   308  }
   309  
   310  // tryGetSpan attempts to get an entire span to scan.
   311  func (w *gcWork) tryGetSpan(slow bool) objptr {
   312  	if s := w.spanq.get(); s != 0 {
   313  		return s
   314  	}
   315  
   316  	if slow {
   317  		// Check the global span queue.
   318  		if s := work.spanq.get(w); s != 0 {
   319  			return s
   320  		}
   321  
   322  		// Attempt to steal spans to scan from other Ps.
   323  		return spanQueueSteal(w)
   324  	}
   325  	return 0
   326  }
   327  
   328  // spanQueue is a concurrent safe queue of mspans. Each mspan is represented
   329  // as an objptr whose spanBase is the base address of the span.
   330  type spanQueue struct {
   331  	avail atomic.Bool      // optimization to check emptiness w/o the lock
   332  	_     cpu.CacheLinePad // prevents false-sharing between lock and avail
   333  	lock  mutex
   334  	q     mSpanQueue
   335  }
   336  
   337  func (q *spanQueue) empty() bool {
   338  	return !q.avail.Load()
   339  }
   340  
   341  func (q *spanQueue) size() int {
   342  	return q.q.n
   343  }
   344  
   345  // putBatch adds a whole batch of spans to the queue.
   346  func (q *spanQueue) putBatch(batch []objptr) {
   347  	var list mSpanQueue
   348  	for _, p := range batch {
   349  		s := spanOfUnchecked(p.spanBase())
   350  		s.scanIdx = p.objIndex()
   351  		list.push(s)
   352  	}
   353  
   354  	lock(&q.lock)
   355  	if q.q.n == 0 {
   356  		q.avail.Store(true)
   357  	}
   358  	q.q.takeAll(&list)
   359  	unlock(&q.lock)
   360  }
   361  
   362  // get tries to take a span off the queue.
   363  //
   364  // Returns a non-zero objptr on success. Also, moves additional
   365  // spans to gcw's local span queue.
   366  func (q *spanQueue) get(gcw *gcWork) objptr {
   367  	if q.empty() {
   368  		return 0
   369  	}
   370  	lock(&q.lock)
   371  	if q.q.n == 0 {
   372  		unlock(&q.lock)
   373  		return 0
   374  	}
   375  	n := q.q.n/int(gomaxprocs) + 1
   376  	if n > q.q.n {
   377  		n = q.q.n
   378  	}
   379  	if max := len(gcw.spanq.ring) / 2; n > max {
   380  		n = max
   381  	}
   382  	newQ := q.q.popN(n)
   383  	if q.q.n == 0 {
   384  		q.avail.Store(false)
   385  	}
   386  	unlock(&q.lock)
   387  
   388  	s := newQ.pop()
   389  	for newQ.n > 0 {
   390  		s := newQ.pop()
   391  		gcw.spanq.put(makeObjPtr(s.base(), s.scanIdx))
   392  	}
   393  	return makeObjPtr(s.base(), s.scanIdx)
   394  }
   395  
   396  // localSpanQueue is a P-local ring buffer of objptrs that represent spans.
   397  // Accessed without a lock.
   398  //
   399  // Multi-consumer, single-producer. The only producer is the P that owns this
   400  // queue, but any other P may consume from it.
   401  //
   402  // This is based on the scheduler runqueues. If making changes there, consider
   403  // also making them here.
   404  type localSpanQueue struct {
   405  	head atomic.Uint32
   406  	tail atomic.Uint32
   407  	ring [256]objptr
   408  }
   409  
   410  // put adds s to the queue. Returns true if put flushed to the global queue
   411  // because it was full.
   412  func (q *localSpanQueue) put(s objptr) (flushed bool) {
   413  	for {
   414  		h := q.head.Load() // synchronize with consumers
   415  		t := q.tail.Load()
   416  		if t-h < uint32(len(q.ring)) {
   417  			q.ring[t%uint32(len(q.ring))] = s
   418  			q.tail.Store(t + 1) // Makes the item avail for consumption.
   419  			return false
   420  		}
   421  		if q.putSlow(s, h, t) {
   422  			return true
   423  		}
   424  		// The queue is not full, now the put above must succeed.
   425  	}
   426  }
   427  
   428  // putSlow is a helper for put to move spans to the global queue.
   429  // Returns true on success, false on failure (nothing moved).
   430  func (q *localSpanQueue) putSlow(s objptr, h, t uint32) bool {
   431  	var batch [len(q.ring)/2 + 1]objptr
   432  
   433  	// First, grab a batch from local queue.
   434  	n := t - h
   435  	n = n / 2
   436  	if n != uint32(len(q.ring)/2) {
   437  		throw("localSpanQueue.putSlow: queue is not full")
   438  	}
   439  	for i := uint32(0); i < n; i++ {
   440  		batch[i] = q.ring[(h+i)%uint32(len(q.ring))]
   441  	}
   442  	if !q.head.CompareAndSwap(h, h+n) { // Commits consume.
   443  		return false
   444  	}
   445  	batch[n] = s
   446  
   447  	work.spanq.putBatch(batch[:])
   448  	return true
   449  }
   450  
   451  // get attempts to take a span off the queue. Might fail if the
   452  // queue is empty. May be called by multiple threads, but callers
   453  // are better off using stealFrom to amortize the cost of stealing.
   454  // This method is intended for use by the owner of this queue.
   455  func (q *localSpanQueue) get() objptr {
   456  	for {
   457  		h := q.head.Load()
   458  		t := q.tail.Load()
   459  		if t == h {
   460  			return 0
   461  		}
   462  		s := q.ring[h%uint32(len(q.ring))]
   463  		if q.head.CompareAndSwap(h, h+1) {
   464  			return s
   465  		}
   466  	}
   467  }
   468  
   469  func (q *localSpanQueue) empty() bool {
   470  	h := q.head.Load()
   471  	t := q.tail.Load()
   472  	return t == h
   473  }
   474  
   475  // stealFrom takes spans from q2 and puts them into q1. One span is removed
   476  // from the stolen spans and returned on success. Failure to steal returns a
   477  // zero objptr.
   478  func (q1 *localSpanQueue) stealFrom(q2 *localSpanQueue) objptr {
   479  	writeHead := q1.tail.Load()
   480  
   481  	var n uint32
   482  	for {
   483  		h := q2.head.Load() // load-acquire, synchronize with other consumers
   484  		t := q2.tail.Load() // load-acquire, synchronize with the producer
   485  		n = t - h
   486  		n = n - n/2
   487  		if n == 0 {
   488  			return 0
   489  		}
   490  		if n > uint32(len(q2.ring)/2) { // read inconsistent h and t
   491  			continue
   492  		}
   493  		for i := uint32(0); i < n; i++ {
   494  			c := q2.ring[(h+i)%uint32(len(q2.ring))]
   495  			q1.ring[(writeHead+i)%uint32(len(q1.ring))] = c
   496  		}
   497  		if q2.head.CompareAndSwap(h, h+n) {
   498  			break
   499  		}
   500  	}
   501  	n--
   502  	c := q1.ring[(writeHead+n)%uint32(len(q1.ring))]
   503  	if n == 0 {
   504  		return c
   505  	}
   506  	h := q1.head.Load()
   507  	if writeHead-h+n >= uint32(len(q1.ring)) {
   508  		throw("localSpanQueue.stealFrom: queue overflow")
   509  	}
   510  	q1.tail.Store(writeHead + n)
   511  	return c
   512  }
   513  
   514  // drain moves all spans in the queue to the global queue.
   515  //
   516  // Returns true if anything was moved.
   517  func (q *localSpanQueue) drain() bool {
   518  	var batch [len(q.ring)]objptr
   519  
   520  	var n uint32
   521  	for {
   522  		var h uint32
   523  		for {
   524  			h = q.head.Load()
   525  			t := q.tail.Load()
   526  			n = t - h
   527  			if n == 0 {
   528  				return false
   529  			}
   530  			if n <= uint32(len(q.ring)) {
   531  				break
   532  			}
   533  			// Read inconsistent h and t.
   534  		}
   535  		for i := uint32(0); i < n; i++ {
   536  			batch[i] = q.ring[(h+i)%uint32(len(q.ring))]
   537  		}
   538  		if q.head.CompareAndSwap(h, h+n) { // Commits consume.
   539  			break
   540  		}
   541  	}
   542  	if !q.empty() {
   543  		throw("drained local span queue, but not empty")
   544  	}
   545  
   546  	work.spanq.putBatch(batch[:n])
   547  	return true
   548  }
   549  
   550  // spanQueueSteal attempts to steal a span from another P's local queue.
   551  //
   552  // Returns a non-zero objptr on success.
   553  func spanQueueSteal(gcw *gcWork) objptr {
   554  	pp := getg().m.p.ptr()
   555  
   556  	for enum := stealOrder.start(cheaprand()); !enum.done(); enum.next() {
   557  		p2 := allp[enum.position()]
   558  		if pp == p2 {
   559  			continue
   560  		}
   561  		if s := gcw.spanq.stealFrom(&p2.gcw.spanq); s != 0 {
   562  			return s
   563  		}
   564  	}
   565  	return 0
   566  }
   567  
   568  // objptr consists of a span base and the index of the object in the span.
   569  type objptr uintptr
   570  
   571  // makeObjPtr creates an objptr from a span base address and an object index.
   572  func makeObjPtr(spanBase uintptr, objIndex uint16) objptr {
   573  	if doubleCheckGreenTea && spanBase&((1<<gc.PageShift)-1) != 0 {
   574  		throw("created objptr with address that is incorrectly aligned")
   575  	}
   576  	return objptr(spanBase | uintptr(objIndex))
   577  }
   578  
   579  func (p objptr) spanBase() uintptr {
   580  	return uintptr(p) &^ ((1 << gc.PageShift) - 1)
   581  }
   582  
   583  func (p objptr) objIndex() uint16 {
   584  	return uint16(p) & ((1 << gc.PageShift) - 1)
   585  }
   586  
   587  // scanSpan scans objects indicated marks&^scans and then scans those objects,
   588  // queuing the resulting pointers into gcw.
   589  func scanSpan(p objptr, gcw *gcWork) {
   590  	spanBase := p.spanBase()
   591  	imb := spanInlineMarkBitsFromBase(spanBase)
   592  	spanclass := imb.class
   593  	if spanclass.noscan() {
   594  		throw("noscan object in scanSpan")
   595  	}
   596  	elemsize := uintptr(gc.SizeClassToSize[spanclass.sizeclass()])
   597  
   598  	// Release span.
   599  	if imb.release() == spanScanOneMark {
   600  		// Nobody else set any mark bits on this span while it was acquired.
   601  		// That means p is the sole object we need to handle. Fast-track it.
   602  		objIndex := p.objIndex()
   603  		bytep := &imb.scans[objIndex/8]
   604  		mask := uint8(1) << (objIndex % 8)
   605  		if atomic.Load8(bytep)&mask != 0 {
   606  			return
   607  		}
   608  		atomic.Or8(bytep, mask)
   609  		gcw.bytesMarked += uint64(elemsize)
   610  		if debug.gctrace > 1 {
   611  			gcw.stats[spanclass.sizeclass()].spansSparseScanned++
   612  			gcw.stats[spanclass.sizeclass()].spanObjsSparseScanned++
   613  		}
   614  		b := spanBase + uintptr(objIndex)*elemsize
   615  		scanObjectSmall(spanBase, b, elemsize, gcw)
   616  		return
   617  	}
   618  
   619  	// Compute nelems.
   620  	divMagic := uint64(gc.SizeClassToDivMagic[spanclass.sizeclass()])
   621  	usableSpanSize := uint64(gc.PageSize - unsafe.Sizeof(spanInlineMarkBits{}))
   622  	if !spanclass.noscan() {
   623  		usableSpanSize -= gc.PageSize / goarch.PtrSize / 8
   624  	}
   625  	nelems := uint16((usableSpanSize * divMagic) >> 32)
   626  
   627  	// Grey objects and return if there's nothing else to do.
   628  	var toScan gc.ObjMask
   629  	objsMarked := spanSetScans(spanBase, nelems, imb, &toScan)
   630  	if objsMarked == 0 {
   631  		return
   632  	}
   633  	gcw.bytesMarked += uint64(objsMarked) * uint64(elemsize)
   634  	if debug.gctrace > 1 {
   635  		gcw.stats[spanclass.sizeclass()].spansDenseScanned++
   636  		gcw.stats[spanclass.sizeclass()].spanObjsDenseScanned += uint64(objsMarked)
   637  	}
   638  	scanObjectsSmall(spanBase, elemsize, nelems, gcw, &toScan)
   639  }
   640  
   641  // spanSetScans sets any unset mark bits that have their mark bits set in the inline mark bits.
   642  //
   643  // toScan is populated with bits indicating whether a particular mark bit was set.
   644  //
   645  // Returns the number of objects marked, which could be zero.
   646  func spanSetScans(spanBase uintptr, nelems uint16, imb *spanInlineMarkBits, toScan *gc.ObjMask) int {
   647  	arena, pageIdx, pageMask := pageIndexOf(spanBase)
   648  	if arena.pageMarks[pageIdx]&pageMask == 0 {
   649  		atomic.Or8(&arena.pageMarks[pageIdx], pageMask)
   650  	}
   651  
   652  	bytes := divRoundUp(uintptr(nelems), 8)
   653  	objsMarked := 0
   654  
   655  	// Careful: these two structures alias since ObjMask is much bigger
   656  	// than marks or scans. We do these unsafe shenanigans so that we can
   657  	// access the marks and scans by uintptrs rather than by byte.
   658  	imbMarks := (*gc.ObjMask)(unsafe.Pointer(&imb.marks))
   659  	imbScans := (*gc.ObjMask)(unsafe.Pointer(&imb.scans))
   660  
   661  	// Iterate over one uintptr-sized chunks at a time, computing both
   662  	// the union and intersection of marks and scans. Store the union
   663  	// into scans, and the intersection into toScan.
   664  	for i := uintptr(0); i < bytes; i += goarch.PtrSize {
   665  		scans := atomic.Loaduintptr(&imbScans[i/goarch.PtrSize])
   666  		marks := imbMarks[i/goarch.PtrSize]
   667  		scans = bswapIfBigEndian(scans)
   668  		marks = bswapIfBigEndian(marks)
   669  		if i/goarch.PtrSize == uintptr(len(imb.marks)+1)/goarch.PtrSize-1 {
   670  			scans &^= 0xff << ((goarch.PtrSize - 1) * 8) // mask out owned
   671  			marks &^= 0xff << ((goarch.PtrSize - 1) * 8) // mask out class
   672  		}
   673  		toGrey := marks &^ scans
   674  		toScan[i/goarch.PtrSize] = toGrey
   675  
   676  		// If there's anything left to grey, do it.
   677  		if toGrey != 0 {
   678  			toGrey = bswapIfBigEndian(toGrey)
   679  			if goarch.PtrSize == 4 {
   680  				atomic.Or32((*uint32)(unsafe.Pointer(&imbScans[i/goarch.PtrSize])), uint32(toGrey))
   681  			} else {
   682  				atomic.Or64((*uint64)(unsafe.Pointer(&imbScans[i/goarch.PtrSize])), uint64(toGrey))
   683  			}
   684  		}
   685  		objsMarked += sys.OnesCount64(uint64(toGrey))
   686  	}
   687  	return objsMarked
   688  }
   689  
   690  func scanObjectSmall(spanBase, b, objSize uintptr, gcw *gcWork) {
   691  	ptrBits := heapBitsSmallForAddrInline(spanBase, b, objSize)
   692  	gcw.heapScanWork += int64(sys.Len64(uint64(ptrBits)) * goarch.PtrSize)
   693  	nptrs := 0
   694  	n := sys.OnesCount64(uint64(ptrBits))
   695  	for range n {
   696  		k := sys.TrailingZeros64(uint64(ptrBits))
   697  		ptrBits &^= 1 << k
   698  		addr := b + uintptr(k)*goarch.PtrSize
   699  
   700  		// Prefetch addr since we're about to use it. This point for prefetching
   701  		// was chosen empirically.
   702  		sys.Prefetch(addr)
   703  
   704  		// N.B. ptrBuf is always large enough to hold pointers for an entire 1-page span.
   705  		gcw.ptrBuf[nptrs] = addr
   706  		nptrs++
   707  	}
   708  
   709  	// Process all the pointers we just got.
   710  	for _, p := range gcw.ptrBuf[:nptrs] {
   711  		p = *(*uintptr)(unsafe.Pointer(p))
   712  		if p == 0 {
   713  			continue
   714  		}
   715  		if !tryDeferToSpanScan(p, gcw) {
   716  			if obj, span, objIndex := findObject(p, 0, 0); obj != 0 {
   717  				greyobject(obj, 0, 0, span, gcw, objIndex)
   718  			}
   719  		}
   720  	}
   721  }
   722  
   723  func scanObjectsSmall(base, objSize uintptr, elems uint16, gcw *gcWork, scans *gc.ObjMask) {
   724  	nptrs := 0
   725  	for i, bits := range scans {
   726  		if i*(goarch.PtrSize*8) > int(elems) {
   727  			break
   728  		}
   729  		n := sys.OnesCount64(uint64(bits))
   730  		for range n {
   731  			j := sys.TrailingZeros64(uint64(bits))
   732  			bits &^= 1 << j
   733  
   734  			b := base + uintptr(i*(goarch.PtrSize*8)+j)*objSize
   735  			ptrBits := heapBitsSmallForAddrInline(base, b, objSize)
   736  			gcw.heapScanWork += int64(sys.Len64(uint64(ptrBits)) * goarch.PtrSize)
   737  
   738  			n := sys.OnesCount64(uint64(ptrBits))
   739  			for range n {
   740  				k := sys.TrailingZeros64(uint64(ptrBits))
   741  				ptrBits &^= 1 << k
   742  				addr := b + uintptr(k)*goarch.PtrSize
   743  
   744  				// Prefetch addr since we're about to use it. This point for prefetching
   745  				// was chosen empirically.
   746  				sys.Prefetch(addr)
   747  
   748  				// N.B. ptrBuf is always large enough to hold pointers for an entire 1-page span.
   749  				gcw.ptrBuf[nptrs] = addr
   750  				nptrs++
   751  			}
   752  		}
   753  	}
   754  
   755  	// Process all the pointers we just got.
   756  	for _, p := range gcw.ptrBuf[:nptrs] {
   757  		p = *(*uintptr)(unsafe.Pointer(p))
   758  		if p == 0 {
   759  			continue
   760  		}
   761  		if !tryDeferToSpanScan(p, gcw) {
   762  			if obj, span, objIndex := findObject(p, 0, 0); obj != 0 {
   763  				greyobject(obj, 0, 0, span, gcw, objIndex)
   764  			}
   765  		}
   766  	}
   767  }
   768  
   769  func heapBitsSmallForAddrInline(spanBase, addr, elemsize uintptr) uintptr {
   770  	hbitsBase, _ := spanHeapBitsRange(spanBase, gc.PageSize, elemsize)
   771  	hbits := (*byte)(unsafe.Pointer(hbitsBase))
   772  
   773  	// These objects are always small enough that their bitmaps
   774  	// fit in a single word, so just load the word or two we need.
   775  	//
   776  	// Mirrors mspan.writeHeapBitsSmall.
   777  	//
   778  	// We should be using heapBits(), but unfortunately it introduces
   779  	// both bounds checks panics and throw which causes us to exceed
   780  	// the nosplit limit in quite a few cases.
   781  	i := (addr - spanBase) / goarch.PtrSize / ptrBits
   782  	j := (addr - spanBase) / goarch.PtrSize % ptrBits
   783  	bits := elemsize / goarch.PtrSize
   784  	word0 := (*uintptr)(unsafe.Pointer(addb(hbits, goarch.PtrSize*(i+0))))
   785  	word1 := (*uintptr)(unsafe.Pointer(addb(hbits, goarch.PtrSize*(i+1))))
   786  
   787  	var read uintptr
   788  	if j+bits > ptrBits {
   789  		// Two reads.
   790  		bits0 := ptrBits - j
   791  		bits1 := bits - bits0
   792  		read = *word0 >> j
   793  		read |= (*word1 & ((1 << bits1) - 1)) << bits0
   794  	} else {
   795  		// One read.
   796  		read = (*word0 >> j) & ((1 << bits) - 1)
   797  	}
   798  	return read
   799  }
   800  
   801  type sizeClassScanStats struct {
   802  	spansDenseScanned     uint64
   803  	spanObjsDenseScanned  uint64
   804  	spansSparseScanned    uint64
   805  	spanObjsSparseScanned uint64
   806  	sparseObjsScanned     uint64
   807  }
   808  
   809  func dumpScanStats() {
   810  	var (
   811  		spansDenseScanned     uint64
   812  		spanObjsDenseScanned  uint64
   813  		spansSparseScanned    uint64
   814  		spanObjsSparseScanned uint64
   815  		sparseObjsScanned     uint64
   816  	)
   817  	for _, stats := range memstats.lastScanStats {
   818  		spansDenseScanned += stats.spansDenseScanned
   819  		spanObjsDenseScanned += stats.spanObjsDenseScanned
   820  		spansSparseScanned += stats.spansSparseScanned
   821  		spanObjsSparseScanned += stats.spanObjsSparseScanned
   822  		sparseObjsScanned += stats.sparseObjsScanned
   823  	}
   824  	totalObjs := sparseObjsScanned + spanObjsSparseScanned + spanObjsDenseScanned
   825  	totalSpans := spansSparseScanned + spansDenseScanned
   826  	print("scan: total ", sparseObjsScanned, "+", spanObjsSparseScanned, "+", spanObjsDenseScanned, "=", totalObjs, " objs")
   827  	print(", ", spansSparseScanned, "+", spansDenseScanned, "=", totalSpans, " spans\n")
   828  	for i, stats := range memstats.lastScanStats {
   829  		if stats == (sizeClassScanStats{}) {
   830  			continue
   831  		}
   832  		totalObjs := stats.sparseObjsScanned + stats.spanObjsSparseScanned + stats.spanObjsDenseScanned
   833  		totalSpans := stats.spansSparseScanned + stats.spansDenseScanned
   834  		if i == 0 {
   835  			print("scan: class L ")
   836  		} else {
   837  			print("scan: class ", gc.SizeClassToSize[i], "B ")
   838  		}
   839  		print(stats.sparseObjsScanned, "+", stats.spanObjsSparseScanned, "+", stats.spanObjsDenseScanned, "=", totalObjs, " objs")
   840  		print(", ", stats.spansSparseScanned, "+", stats.spansDenseScanned, "=", totalSpans, " spans\n")
   841  	}
   842  }
   843  
   844  func (w *gcWork) flushScanStats(dst *[gc.NumSizeClasses]sizeClassScanStats) {
   845  	for i := range w.stats {
   846  		dst[i].spansDenseScanned += w.stats[i].spansDenseScanned
   847  		dst[i].spanObjsDenseScanned += w.stats[i].spanObjsDenseScanned
   848  		dst[i].spansSparseScanned += w.stats[i].spansSparseScanned
   849  		dst[i].spanObjsSparseScanned += w.stats[i].spanObjsSparseScanned
   850  		dst[i].sparseObjsScanned += w.stats[i].sparseObjsScanned
   851  	}
   852  	clear(w.stats[:])
   853  }
   854  

View as plain text