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

View as plain text