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) {
   114  	*imb = spanInlineMarkBits{}
   115  	imb.class = class
   116  }
   117  
   118  // tryAcquire attempts to acquire the span for scanning. On success, the caller
   119  // must queue the span for scanning or scan the span immediately.
   120  func (imb *spanInlineMarkBits) tryAcquire() bool {
   121  	switch imb.owned.load() {
   122  	case spanScanUnowned:
   123  		// Try to mark the span as having only one object marked.
   124  		if imb.owned.or(spanScanOneMark) == spanScanUnowned {
   125  			return true
   126  		}
   127  		// If we didn't see an old value of spanScanUnowned, then we must
   128  		// have raced with someone else and seen spanScanOneMark or greater.
   129  		// Fall through and try to set spanScanManyMark.
   130  		fallthrough
   131  	case spanScanOneMark:
   132  		// We may be the first to set *any* bit on owned. In such a case,
   133  		// we still need to make sure the span is queued.
   134  		return imb.owned.or(spanScanManyMark) == spanScanUnowned
   135  	}
   136  	return false
   137  }
   138  
   139  // release releases the span for scanning, allowing another thread to queue the span.
   140  //
   141  // Returns an upper bound on the number of mark bits set since the span was queued. The
   142  // upper bound is described as "one" (spanScanOneMark) or "many" (spanScanManyMark, with or
   143  // without spanScanOneMark). If the return value indicates only one mark bit was set, the
   144  // caller can be certain that it was the same mark bit that caused the span to get queued.
   145  // Take note of the fact that this is *only* an upper-bound. In particular, it may still
   146  // turn out that only one mark bit was set, even if the return value indicates "many".
   147  func (imb *spanInlineMarkBits) release() spanScanOwnership {
   148  	return spanScanOwnership(atomic.Xchg8((*uint8)(unsafe.Pointer(&imb.owned)), uint8(spanScanUnowned)))
   149  }
   150  
   151  // spanInlineMarkBitsFromBase returns the spanInlineMarkBits for a span whose start address is base.
   152  //
   153  // The span must be gcUsesSpanInlineMarkBits(span.elemsize).
   154  func spanInlineMarkBitsFromBase(base uintptr) *spanInlineMarkBits {
   155  	return (*spanInlineMarkBits)(unsafe.Pointer(base + gc.PageSize - unsafe.Sizeof(spanInlineMarkBits{})))
   156  }
   157  
   158  // initInlineMarkBits initializes the inlineMarkBits stored at the end of the span.
   159  func (s *mspan) initInlineMarkBits() {
   160  	if doubleCheckGreenTea && !gcUsesSpanInlineMarkBits(s.elemsize) {
   161  		throw("expected span with inline mark bits")
   162  	}
   163  	s.inlineMarkBits().init(s.spanclass)
   164  }
   165  
   166  // mergeInlineMarks merges the span's inline mark bits into dst.
   167  //
   168  // gcUsesSpanInlineMarkBits(s.elemsize) must be true.
   169  func (s *mspan) mergeInlineMarks(dst *gcBits) {
   170  	if doubleCheckGreenTea && !gcUsesSpanInlineMarkBits(s.elemsize) {
   171  		throw("expected span with inline mark bits")
   172  	}
   173  	bytes := divRoundUp(uintptr(s.nelems), 8)
   174  	imb := s.inlineMarkBits()
   175  	_ = imb.marks[bytes-1]
   176  	for i := uintptr(0); i < bytes; i++ {
   177  		*dst.bytep(i) |= imb.marks[i]
   178  	}
   179  	if doubleCheckGreenTea && !s.spanclass.noscan() && imb.marks != imb.scans {
   180  		throw("marks don't match scans for span with pointer")
   181  	}
   182  }
   183  
   184  // inlineMarkBits returns the inline mark bits for the span.
   185  //
   186  // gcUsesSpanInlineMarkBits(s.elemsize) must be true.
   187  func (s *mspan) inlineMarkBits() *spanInlineMarkBits {
   188  	if doubleCheckGreenTea && !gcUsesSpanInlineMarkBits(s.elemsize) {
   189  		throw("expected span with inline mark bits")
   190  	}
   191  	return spanInlineMarkBitsFromBase(s.base())
   192  }
   193  
   194  func (s *mspan) markBitsForIndex(objIndex uintptr) (bits markBits) {
   195  	if gcUsesSpanInlineMarkBits(s.elemsize) {
   196  		bits.bytep = &s.inlineMarkBits().marks[objIndex/8]
   197  	} else {
   198  		bits.bytep = s.gcmarkBits.bytep(objIndex / 8)
   199  	}
   200  	bits.mask = uint8(1) << (objIndex % 8)
   201  	bits.index = objIndex
   202  	return
   203  }
   204  
   205  func (s *mspan) markBitsForBase() markBits {
   206  	if gcUsesSpanInlineMarkBits(s.elemsize) {
   207  		return markBits{&s.inlineMarkBits().marks[0], uint8(1), 0}
   208  	}
   209  	return markBits{&s.gcmarkBits.x, uint8(1), 0}
   210  }
   211  
   212  // scannedBitsForIndex returns a markBits representing the scanned bit
   213  // for objIndex in the inline mark bits.
   214  func (s *mspan) scannedBitsForIndex(objIndex uintptr) markBits {
   215  	return markBits{&s.inlineMarkBits().scans[objIndex/8], uint8(1) << (objIndex % 8), objIndex}
   216  }
   217  
   218  // gcUsesSpanInlineMarkBits returns true if a span holding objects of a certain size
   219  // has inline mark bits. size must be the span's elemsize.
   220  //
   221  // nosplit because this is called from gcmarknewobject, which is nosplit.
   222  //
   223  //go:nosplit
   224  func gcUsesSpanInlineMarkBits(size uintptr) bool {
   225  	return heapBitsInSpan(size) && size >= 16
   226  }
   227  
   228  // tryQueueOnSpan tries to queue p on the span it points to, if it
   229  // points to a small object span (gcUsesSpanQueue size).
   230  func tryDeferToSpanScan(p uintptr, gcw *gcWork) bool {
   231  	if useCheckmark {
   232  		return false
   233  	}
   234  
   235  	// Quickly to see if this is a span that has inline mark bits.
   236  	ha := heapArenaOf(p)
   237  	if ha == nil {
   238  		return false
   239  	}
   240  	pageIdx := ((p / pageSize) / 8) % uintptr(len(ha.pageInUse))
   241  	pageMask := byte(1 << ((p / pageSize) % 8))
   242  	if ha.pageUseSpanInlineMarkBits[pageIdx]&pageMask == 0 {
   243  		return false
   244  	}
   245  
   246  	// Find the object's index from the span class info stored in the inline mark bits.
   247  	base := alignDown(p, gc.PageSize)
   248  	q := spanInlineMarkBitsFromBase(base)
   249  	objIndex := uint16((uint64(p-base) * uint64(gc.SizeClassToDivMagic[q.class.sizeclass()])) >> 32)
   250  
   251  	// Set mark bit.
   252  	idx, mask := objIndex/8, uint8(1)<<(objIndex%8)
   253  	if atomic.Load8(&q.marks[idx])&mask != 0 {
   254  		return true
   255  	}
   256  	atomic.Or8(&q.marks[idx], mask)
   257  
   258  	// Fast-track noscan objects.
   259  	if q.class.noscan() {
   260  		gcw.bytesMarked += uint64(gc.SizeClassToSize[q.class.sizeclass()])
   261  		return true
   262  	}
   263  
   264  	// Queue up the pointer (as a representative for its span).
   265  	if q.tryAcquire() {
   266  		if gcw.spanq.put(makeObjPtr(base, objIndex)) {
   267  			if gcphase == _GCmark {
   268  				gcw.mayNeedWorker = true
   269  			}
   270  			gcw.flushedWork = true
   271  		}
   272  	}
   273  	return true
   274  }
   275  
   276  // tryGetSpan attempts to get an entire span to scan.
   277  func (w *gcWork) tryGetSpan(slow bool) objptr {
   278  	if s := w.spanq.get(); s != 0 {
   279  		return s
   280  	}
   281  
   282  	if slow {
   283  		// Check the global span queue.
   284  		if s := work.spanq.get(w); s != 0 {
   285  			return s
   286  		}
   287  
   288  		// Attempt to steal spans to scan from other Ps.
   289  		return spanQueueSteal(w)
   290  	}
   291  	return 0
   292  }
   293  
   294  // spanQueue is a concurrent safe queue of mspans. Each mspan is represented
   295  // as an objptr whose spanBase is the base address of the span.
   296  type spanQueue struct {
   297  	avail atomic.Bool      // optimization to check emptiness w/o the lock
   298  	_     cpu.CacheLinePad // prevents false-sharing between lock and avail
   299  	lock  mutex
   300  	q     mSpanQueue
   301  }
   302  
   303  func (q *spanQueue) empty() bool {
   304  	return !q.avail.Load()
   305  }
   306  
   307  func (q *spanQueue) size() int {
   308  	return q.q.n
   309  }
   310  
   311  // putBatch adds a whole batch of spans to the queue.
   312  func (q *spanQueue) putBatch(batch []objptr) {
   313  	var list mSpanQueue
   314  	for _, p := range batch {
   315  		s := spanOfUnchecked(p.spanBase())
   316  		s.scanIdx = p.objIndex()
   317  		list.push(s)
   318  	}
   319  
   320  	lock(&q.lock)
   321  	if q.q.n == 0 {
   322  		q.avail.Store(true)
   323  	}
   324  	q.q.takeAll(&list)
   325  	unlock(&q.lock)
   326  }
   327  
   328  // get tries to take a span off the queue.
   329  //
   330  // Returns a non-zero objptr on success. Also, moves additional
   331  // spans to gcw's local span queue.
   332  func (q *spanQueue) get(gcw *gcWork) objptr {
   333  	if q.empty() {
   334  		return 0
   335  	}
   336  	lock(&q.lock)
   337  	if q.q.n == 0 {
   338  		unlock(&q.lock)
   339  		return 0
   340  	}
   341  	n := q.q.n/int(gomaxprocs) + 1
   342  	if n > q.q.n {
   343  		n = q.q.n
   344  	}
   345  	if max := len(gcw.spanq.ring) / 2; n > max {
   346  		n = max
   347  	}
   348  	newQ := q.q.popN(n)
   349  	if q.q.n == 0 {
   350  		q.avail.Store(false)
   351  	}
   352  	unlock(&q.lock)
   353  
   354  	s := newQ.pop()
   355  	for newQ.n > 0 {
   356  		s := newQ.pop()
   357  		gcw.spanq.put(makeObjPtr(s.base(), s.scanIdx))
   358  	}
   359  	return makeObjPtr(s.base(), s.scanIdx)
   360  }
   361  
   362  // localSpanQueue is a P-local ring buffer of objptrs that represent spans.
   363  // Accessed without a lock.
   364  //
   365  // Multi-consumer, single-producer. The only producer is the P that owns this
   366  // queue, but any other P may consume from it.
   367  //
   368  // This is based on the scheduler runqueues. If making changes there, consider
   369  // also making them here.
   370  type localSpanQueue struct {
   371  	head atomic.Uint32
   372  	tail atomic.Uint32
   373  	ring [256]objptr
   374  }
   375  
   376  // put adds s to the queue. Returns true if put flushed to the global queue
   377  // because it was full.
   378  func (q *localSpanQueue) put(s objptr) (flushed bool) {
   379  	for {
   380  		h := q.head.Load() // synchronize with consumers
   381  		t := q.tail.Load()
   382  		if t-h < uint32(len(q.ring)) {
   383  			q.ring[t%uint32(len(q.ring))] = s
   384  			q.tail.Store(t + 1) // Makes the item avail for consumption.
   385  			return false
   386  		}
   387  		if q.putSlow(s, h, t) {
   388  			return true
   389  		}
   390  		// The queue is not full, now the put above must succeed.
   391  	}
   392  }
   393  
   394  // putSlow is a helper for put to move spans to the global queue.
   395  // Returns true on success, false on failure (nothing moved).
   396  func (q *localSpanQueue) putSlow(s objptr, h, t uint32) bool {
   397  	var batch [len(q.ring)/2 + 1]objptr
   398  
   399  	// First, grab a batch from local queue.
   400  	n := t - h
   401  	n = n / 2
   402  	if n != uint32(len(q.ring)/2) {
   403  		throw("localSpanQueue.putSlow: queue is not full")
   404  	}
   405  	for i := uint32(0); i < n; i++ {
   406  		batch[i] = q.ring[(h+i)%uint32(len(q.ring))]
   407  	}
   408  	if !q.head.CompareAndSwap(h, h+n) { // Commits consume.
   409  		return false
   410  	}
   411  	batch[n] = s
   412  
   413  	work.spanq.putBatch(batch[:])
   414  	return true
   415  }
   416  
   417  // get attempts to take a span off the queue. Might fail if the
   418  // queue is empty. May be called by multiple threads, but callers
   419  // are better off using stealFrom to amortize the cost of stealing.
   420  // This method is intended for use by the owner of this queue.
   421  func (q *localSpanQueue) get() objptr {
   422  	for {
   423  		h := q.head.Load()
   424  		t := q.tail.Load()
   425  		if t == h {
   426  			return 0
   427  		}
   428  		s := q.ring[h%uint32(len(q.ring))]
   429  		if q.head.CompareAndSwap(h, h+1) {
   430  			return s
   431  		}
   432  	}
   433  }
   434  
   435  func (q *localSpanQueue) empty() bool {
   436  	h := q.head.Load()
   437  	t := q.tail.Load()
   438  	return t == h
   439  }
   440  
   441  // stealFrom takes spans from q2 and puts them into q1. One span is removed
   442  // from the stolen spans and returned on success. Failure to steal returns a
   443  // zero objptr.
   444  func (q1 *localSpanQueue) stealFrom(q2 *localSpanQueue) objptr {
   445  	writeHead := q1.tail.Load()
   446  
   447  	var n uint32
   448  	for {
   449  		h := q2.head.Load() // load-acquire, synchronize with other consumers
   450  		t := q2.tail.Load() // load-acquire, synchronize with the producer
   451  		n = t - h
   452  		n = n - n/2
   453  		if n == 0 {
   454  			return 0
   455  		}
   456  		if n > uint32(len(q2.ring)/2) { // read inconsistent h and t
   457  			continue
   458  		}
   459  		for i := uint32(0); i < n; i++ {
   460  			c := q2.ring[(h+i)%uint32(len(q2.ring))]
   461  			q1.ring[(writeHead+i)%uint32(len(q1.ring))] = c
   462  		}
   463  		if q2.head.CompareAndSwap(h, h+n) {
   464  			break
   465  		}
   466  	}
   467  	n--
   468  	c := q1.ring[(writeHead+n)%uint32(len(q1.ring))]
   469  	if n == 0 {
   470  		return c
   471  	}
   472  	h := q1.head.Load()
   473  	if writeHead-h+n >= uint32(len(q1.ring)) {
   474  		throw("localSpanQueue.stealFrom: queue overflow")
   475  	}
   476  	q1.tail.Store(writeHead + n)
   477  	return c
   478  }
   479  
   480  // drain moves all spans in the queue to the global queue.
   481  //
   482  // Returns true if anything was moved.
   483  func (q *localSpanQueue) drain() bool {
   484  	var batch [len(q.ring)]objptr
   485  
   486  	var n uint32
   487  	for {
   488  		var h uint32
   489  		for {
   490  			h = q.head.Load()
   491  			t := q.tail.Load()
   492  			n = t - h
   493  			if n == 0 {
   494  				return false
   495  			}
   496  			if n <= uint32(len(q.ring)) {
   497  				break
   498  			}
   499  			// Read inconsistent h and t.
   500  		}
   501  		for i := uint32(0); i < n; i++ {
   502  			batch[i] = q.ring[(h+i)%uint32(len(q.ring))]
   503  		}
   504  		if q.head.CompareAndSwap(h, h+n) { // Commits consume.
   505  			break
   506  		}
   507  	}
   508  	if !q.empty() {
   509  		throw("drained local span queue, but not empty")
   510  	}
   511  
   512  	work.spanq.putBatch(batch[:n])
   513  	return true
   514  }
   515  
   516  // spanQueueSteal attempts to steal a span from another P's local queue.
   517  //
   518  // Returns a non-zero objptr on success.
   519  func spanQueueSteal(gcw *gcWork) objptr {
   520  	pp := getg().m.p.ptr()
   521  
   522  	for enum := stealOrder.start(cheaprand()); !enum.done(); enum.next() {
   523  		p2 := allp[enum.position()]
   524  		if pp == p2 {
   525  			continue
   526  		}
   527  		if s := gcw.spanq.stealFrom(&p2.gcw.spanq); s != 0 {
   528  			return s
   529  		}
   530  	}
   531  	return 0
   532  }
   533  
   534  // objptr consists of a span base and the index of the object in the span.
   535  type objptr uintptr
   536  
   537  // makeObjPtr creates an objptr from a span base address and an object index.
   538  func makeObjPtr(spanBase uintptr, objIndex uint16) objptr {
   539  	if doubleCheckGreenTea && spanBase&((1<<gc.PageShift)-1) != 0 {
   540  		throw("created objptr with address that is incorrectly aligned")
   541  	}
   542  	return objptr(spanBase | uintptr(objIndex))
   543  }
   544  
   545  func (p objptr) spanBase() uintptr {
   546  	return uintptr(p) &^ ((1 << gc.PageShift) - 1)
   547  }
   548  
   549  func (p objptr) objIndex() uint16 {
   550  	return uint16(p) & ((1 << gc.PageShift) - 1)
   551  }
   552  
   553  // scanSpan scans objects indicated marks&^scans and then scans those objects,
   554  // queuing the resulting pointers into gcw.
   555  func scanSpan(p objptr, gcw *gcWork) {
   556  	spanBase := p.spanBase()
   557  	imb := spanInlineMarkBitsFromBase(spanBase)
   558  	spanclass := imb.class
   559  	if spanclass.noscan() {
   560  		throw("noscan object in scanSpan")
   561  	}
   562  	elemsize := uintptr(gc.SizeClassToSize[spanclass.sizeclass()])
   563  
   564  	// Release span.
   565  	if imb.release() == spanScanOneMark {
   566  		// Nobody else set any mark bits on this span while it was acquired.
   567  		// That means p is the sole object we need to handle. Fast-track it.
   568  		objIndex := p.objIndex()
   569  		bytep := &imb.scans[objIndex/8]
   570  		mask := uint8(1) << (objIndex % 8)
   571  		if atomic.Load8(bytep)&mask != 0 {
   572  			return
   573  		}
   574  		atomic.Or8(bytep, mask)
   575  		gcw.bytesMarked += uint64(elemsize)
   576  		if debug.gctrace > 1 {
   577  			gcw.stats[spanclass.sizeclass()].spansSparseScanned++
   578  			gcw.stats[spanclass.sizeclass()].spanObjsSparseScanned++
   579  		}
   580  		b := spanBase + uintptr(objIndex)*elemsize
   581  		scanObjectSmall(spanBase, b, elemsize, gcw)
   582  		return
   583  	}
   584  
   585  	// Compute nelems.
   586  	divMagic := uint64(gc.SizeClassToDivMagic[spanclass.sizeclass()])
   587  	usableSpanSize := uint64(gc.PageSize - unsafe.Sizeof(spanInlineMarkBits{}))
   588  	if !spanclass.noscan() {
   589  		usableSpanSize -= gc.PageSize / goarch.PtrSize / 8
   590  	}
   591  	nelems := uint16((usableSpanSize * divMagic) >> 32)
   592  
   593  	// Grey objects and return if there's nothing else to do.
   594  	var toScan gc.ObjMask
   595  	objsMarked := spanSetScans(spanBase, nelems, imb, &toScan)
   596  	if objsMarked == 0 {
   597  		return
   598  	}
   599  	gcw.bytesMarked += uint64(objsMarked) * uint64(elemsize)
   600  	if debug.gctrace > 1 {
   601  		gcw.stats[spanclass.sizeclass()].spansDenseScanned++
   602  		gcw.stats[spanclass.sizeclass()].spanObjsDenseScanned += uint64(objsMarked)
   603  	}
   604  	scanObjectsSmall(spanBase, elemsize, nelems, gcw, &toScan)
   605  }
   606  
   607  // spanSetScans sets any unset mark bits that have their mark bits set in the inline mark bits.
   608  //
   609  // toScan is populated with bits indicating whether a particular mark bit was set.
   610  //
   611  // Returns the number of objects marked, which could be zero.
   612  func spanSetScans(spanBase uintptr, nelems uint16, imb *spanInlineMarkBits, toScan *gc.ObjMask) int {
   613  	arena, pageIdx, pageMask := pageIndexOf(spanBase)
   614  	if arena.pageMarks[pageIdx]&pageMask == 0 {
   615  		atomic.Or8(&arena.pageMarks[pageIdx], pageMask)
   616  	}
   617  
   618  	bytes := divRoundUp(uintptr(nelems), 8)
   619  	objsMarked := 0
   620  
   621  	// Careful: these two structures alias since ObjMask is much bigger
   622  	// than marks or scans. We do these unsafe shenanigans so that we can
   623  	// access the marks and scans by uintptrs rather than by byte.
   624  	imbMarks := (*gc.ObjMask)(unsafe.Pointer(&imb.marks))
   625  	imbScans := (*gc.ObjMask)(unsafe.Pointer(&imb.scans))
   626  
   627  	// Iterate over one uintptr-sized chunks at a time, computing both
   628  	// the union and intersection of marks and scans. Store the union
   629  	// into scans, and the intersection into toScan.
   630  	for i := uintptr(0); i < bytes; i += goarch.PtrSize {
   631  		scans := atomic.Loaduintptr(&imbScans[i/goarch.PtrSize])
   632  		marks := imbMarks[i/goarch.PtrSize]
   633  		scans = bswapIfBigEndian(scans)
   634  		marks = bswapIfBigEndian(marks)
   635  		if i/goarch.PtrSize == 64/goarch.PtrSize-1 {
   636  			scans &^= 0xff << ((goarch.PtrSize - 1) * 8) // mask out owned
   637  			marks &^= 0xff << ((goarch.PtrSize - 1) * 8) // mask out class
   638  		}
   639  		toGrey := marks &^ scans
   640  		toScan[i/goarch.PtrSize] = toGrey
   641  
   642  		// If there's anything left to grey, do it.
   643  		if toGrey != 0 {
   644  			toGrey = bswapIfBigEndian(toGrey)
   645  			if goarch.PtrSize == 4 {
   646  				atomic.Or32((*uint32)(unsafe.Pointer(&imbScans[i/goarch.PtrSize])), uint32(toGrey))
   647  			} else {
   648  				atomic.Or64((*uint64)(unsafe.Pointer(&imbScans[i/goarch.PtrSize])), uint64(toGrey))
   649  			}
   650  		}
   651  		objsMarked += sys.OnesCount64(uint64(toGrey))
   652  	}
   653  	return objsMarked
   654  }
   655  
   656  func scanObjectSmall(spanBase, b, objSize uintptr, gcw *gcWork) {
   657  	ptrBits := heapBitsSmallForAddrInline(spanBase, b, objSize)
   658  	gcw.heapScanWork += int64(sys.Len64(uint64(ptrBits)) * goarch.PtrSize)
   659  	nptrs := 0
   660  	n := sys.OnesCount64(uint64(ptrBits))
   661  	for range n {
   662  		k := sys.TrailingZeros64(uint64(ptrBits))
   663  		ptrBits &^= 1 << k
   664  		addr := b + uintptr(k)*goarch.PtrSize
   665  
   666  		// Prefetch addr since we're about to use it. This point for prefetching
   667  		// was chosen empirically.
   668  		sys.Prefetch(addr)
   669  
   670  		// N.B. ptrBuf is always large enough to hold pointers for an entire 1-page span.
   671  		gcw.ptrBuf[nptrs] = addr
   672  		nptrs++
   673  	}
   674  
   675  	// Process all the pointers we just got.
   676  	for _, p := range gcw.ptrBuf[:nptrs] {
   677  		p = *(*uintptr)(unsafe.Pointer(p))
   678  		if p == 0 {
   679  			continue
   680  		}
   681  		if !tryDeferToSpanScan(p, gcw) {
   682  			if obj, span, objIndex := findObject(p, 0, 0); obj != 0 {
   683  				greyobject(obj, 0, 0, span, gcw, objIndex)
   684  			}
   685  		}
   686  	}
   687  }
   688  
   689  func scanObjectsSmall(base, objSize uintptr, elems uint16, gcw *gcWork, scans *gc.ObjMask) {
   690  	nptrs := 0
   691  	for i, bits := range scans {
   692  		if i*(goarch.PtrSize*8) > int(elems) {
   693  			break
   694  		}
   695  		n := sys.OnesCount64(uint64(bits))
   696  		for range n {
   697  			j := sys.TrailingZeros64(uint64(bits))
   698  			bits &^= 1 << j
   699  
   700  			b := base + uintptr(i*(goarch.PtrSize*8)+j)*objSize
   701  			ptrBits := heapBitsSmallForAddrInline(base, b, objSize)
   702  			gcw.heapScanWork += int64(sys.Len64(uint64(ptrBits)) * goarch.PtrSize)
   703  
   704  			n := sys.OnesCount64(uint64(ptrBits))
   705  			for range n {
   706  				k := sys.TrailingZeros64(uint64(ptrBits))
   707  				ptrBits &^= 1 << k
   708  				addr := b + uintptr(k)*goarch.PtrSize
   709  
   710  				// Prefetch addr since we're about to use it. This point for prefetching
   711  				// was chosen empirically.
   712  				sys.Prefetch(addr)
   713  
   714  				// N.B. ptrBuf is always large enough to hold pointers for an entire 1-page span.
   715  				gcw.ptrBuf[nptrs] = addr
   716  				nptrs++
   717  			}
   718  		}
   719  	}
   720  
   721  	// Process all the pointers we just got.
   722  	for _, p := range gcw.ptrBuf[:nptrs] {
   723  		p = *(*uintptr)(unsafe.Pointer(p))
   724  		if p == 0 {
   725  			continue
   726  		}
   727  		if !tryDeferToSpanScan(p, gcw) {
   728  			if obj, span, objIndex := findObject(p, 0, 0); obj != 0 {
   729  				greyobject(obj, 0, 0, span, gcw, objIndex)
   730  			}
   731  		}
   732  	}
   733  }
   734  
   735  func heapBitsSmallForAddrInline(spanBase, addr, elemsize uintptr) uintptr {
   736  	hbitsBase, _ := spanHeapBitsRange(spanBase, gc.PageSize, elemsize)
   737  	hbits := (*byte)(unsafe.Pointer(hbitsBase))
   738  
   739  	// These objects are always small enough that their bitmaps
   740  	// fit in a single word, so just load the word or two we need.
   741  	//
   742  	// Mirrors mspan.writeHeapBitsSmall.
   743  	//
   744  	// We should be using heapBits(), but unfortunately it introduces
   745  	// both bounds checks panics and throw which causes us to exceed
   746  	// the nosplit limit in quite a few cases.
   747  	i := (addr - spanBase) / goarch.PtrSize / ptrBits
   748  	j := (addr - spanBase) / goarch.PtrSize % ptrBits
   749  	bits := elemsize / goarch.PtrSize
   750  	word0 := (*uintptr)(unsafe.Pointer(addb(hbits, goarch.PtrSize*(i+0))))
   751  	word1 := (*uintptr)(unsafe.Pointer(addb(hbits, goarch.PtrSize*(i+1))))
   752  
   753  	var read uintptr
   754  	if j+bits > ptrBits {
   755  		// Two reads.
   756  		bits0 := ptrBits - j
   757  		bits1 := bits - bits0
   758  		read = *word0 >> j
   759  		read |= (*word1 & ((1 << bits1) - 1)) << bits0
   760  	} else {
   761  		// One read.
   762  		read = (*word0 >> j) & ((1 << bits) - 1)
   763  	}
   764  	return read
   765  }
   766  

View as plain text