]> Cypherpunks.ru repositories - gostls13.git/blobdiff - src/runtime/mgc.go
runtime: refactor runtime->tracer API to appear more like a lock
[gostls13.git] / src / runtime / mgc.go
index 93d090f6edc3f5c370321a0557bb1123db5b3b7a..30d2f1d3852b4430c6ac2039e5a0f3d1dcf43ba3 100644 (file)
@@ -135,9 +135,12 @@ import (
 )
 
 const (
-       _DebugGC         = 0
-       _ConcurrentSweep = true
-       _FinBlockSize    = 4 * 1024
+       _DebugGC      = 0
+       _FinBlockSize = 4 * 1024
+
+       // concurrentSweep is a debug flag. Disabling this flag
+       // ensures all spans are swept while the world is stopped.
+       concurrentSweep = true
 
        // debugScanConservative enables debug logging for stack
        // frames that are scanned conservatively.
@@ -149,6 +152,28 @@ const (
        sweepMinHeapDistance = 1024 * 1024
 )
 
+// heapObjectsCanMove always returns false in the current garbage collector.
+// It exists for go4.org/unsafe/assume-no-moving-gc, which is an
+// unfortunate idea that had an even more unfortunate implementation.
+// Every time a new Go release happened, the package stopped building,
+// and the authors had to add a new file with a new //go:build line, and
+// then the entire ecosystem of packages with that as a dependency had to
+// explicitly update to the new version. Many packages depend on
+// assume-no-moving-gc transitively, through paths like
+// inet.af/netaddr -> go4.org/intern -> assume-no-moving-gc.
+// This was causing a significant amount of friction around each new
+// release, so we added this bool for the package to //go:linkname
+// instead. The bool is still unfortunate, but it's not as bad as
+// breaking the ecosystem on every new release.
+//
+// If the Go garbage collector ever does move heap objects, we can set
+// this to true to break all the programs using assume-no-moving-gc.
+//
+//go:linkname heapObjectsCanMove
+func heapObjectsCanMove() bool {
+       return false
+}
+
 func gcinit() {
        if unsafe.Sizeof(workbuf{}) != _WorkbufSize {
                throw("size of Workbuf is suboptimal")
@@ -159,7 +184,7 @@ func gcinit() {
        // Initialize GC pacer state.
        // Use the environment variable GOGC for the initial gcPercent value.
        // Use the environment variable GOMEMLIMIT for the initial memoryLimit value.
-       gcController.init(readGOGC(), maxInt64)
+       gcController.init(readGOGC(), readGOMEMLIMIT())
 
        work.startSema = 1
        work.markDoneSema = 1
@@ -193,8 +218,6 @@ var gcphase uint32
 var writeBarrier struct {
        enabled bool    // compiler emits a check of this before calling write barrier
        pad     [3]byte // compiler uses 32-bit load for "enabled" field
-       needed  bool    // whether we need a write barrier for current GC phase
-       cgo     bool    // whether we need a write barrier for a cgo check
        alignme uint64  // guarantee alignment so that compiler can use a 32 or 64-bit load
 }
 
@@ -212,8 +235,7 @@ const (
 //go:nosplit
 func setGCPhase(x uint32) {
        atomic.Store(&gcphase, x)
-       writeBarrier.needed = gcphase == _GCmark || gcphase == _GCmarktermination
-       writeBarrier.enabled = writeBarrier.needed || writeBarrier.cgo
+       writeBarrier.enabled = gcphase == _GCmark || gcphase == _GCmarktermination
 }
 
 // gcMarkWorkerMode represents the mode that a concurrent mark worker
@@ -279,10 +301,13 @@ func pollFractionalWorkerExit() bool {
        return float64(selfTime)/float64(delta) > 1.2*gcController.fractionalUtilizationGoal
 }
 
-var work struct {
+var work workType
+
+type workType struct {
        full  lfstack          // lock-free list of full blocks workbuf
+       _     cpu.CacheLinePad // prevents false-sharing between full and empty
        empty lfstack          // lock-free list of empty blocks workbuf
-       pad0  cpu.CacheLinePad // prevents false-sharing between full/empty and nproc/nwait
+       _     cpu.CacheLinePad // prevents false-sharing between empty and nproc/nwait
 
        wbufSpans struct {
                lock mutex
@@ -364,10 +389,6 @@ var work struct {
        // explicit user call.
        userForced bool
 
-       // totaltime is the CPU nanoseconds spent in GC since the
-       // program started if debug.gctrace > 0.
-       totaltime int64
-
        // initialHeapLive is the value of gcController.heapLive at the
        // beginning of this GC cycle.
        initialHeapLive uint64
@@ -391,7 +412,7 @@ var work struct {
        // cycle is sweep termination, mark, mark termination, and
        // sweep. This differs from memstats.numgc, which is
        // incremented at mark termination.
-       cycles uint32
+       cycles atomic.Uint32
 
        // Timing/utilization stats for this cycle.
        stwprocs, maxprocs                 int32
@@ -402,6 +423,9 @@ var work struct {
 
        // debug.gctrace heap sizes for this cycle.
        heap0, heap1, heap2 uint64
+
+       // Cumulative estimated CPU usage.
+       cpuStats
 }
 
 // GC runs a garbage collection and blocks the caller until the
@@ -434,7 +458,7 @@ func GC() {
 
        // Wait until the current sweep termination, mark, and mark
        // termination complete.
-       n := atomic.Load(&work.cycles)
+       n := work.cycles.Load()
        gcWaitOnMark(n)
 
        // We're now in sweep N or later. Trigger GC cycle N+1, which
@@ -449,8 +473,7 @@ func GC() {
        // complete the cycle and because runtime.GC() is often used
        // as part of tests and benchmarks to get the system into a
        // relatively stable and isolated state.
-       for atomic.Load(&work.cycles) == n+1 && sweepone() != ^uintptr(0) {
-               sweep.nbgsweep++
+       for work.cycles.Load() == n+1 && sweepone() != ^uintptr(0) {
                Gosched()
        }
 
@@ -465,7 +488,7 @@ func GC() {
        // First, wait for sweeping to finish. (We know there are no
        // more spans on the sweep queue, but we may be concurrently
        // sweeping spans, so we have to wait.)
-       for atomic.Load(&work.cycles) == n+1 && !isSweepDone() {
+       for work.cycles.Load() == n+1 && !isSweepDone() {
                Gosched()
        }
 
@@ -473,7 +496,7 @@ func GC() {
        // stable heap profile. Only do this if we haven't already hit
        // another mark termination.
        mp := acquirem()
-       cycle := atomic.Load(&work.cycles)
+       cycle := work.cycles.Load()
        if cycle == n+1 || (gcphase == _GCmark && cycle == n+2) {
                mProf_PostSweep()
        }
@@ -486,7 +509,7 @@ func gcWaitOnMark(n uint32) {
        for {
                // Disable phase transitions.
                lock(&work.sweepWaiters.lock)
-               nMarks := atomic.Load(&work.cycles)
+               nMarks := work.cycles.Load()
                if gcphase != _GCmark {
                        // We've already completed this cycle's mark.
                        nMarks++
@@ -500,7 +523,7 @@ func gcWaitOnMark(n uint32) {
                // Wait until sweep termination, mark, and mark
                // termination of cycle N complete.
                work.sweepWaiters.list.push(getg())
-               goparkunlock(&work.sweepWaiters.lock, waitReasonWaitForGCCycle, traceEvGoBlock, 1)
+               goparkunlock(&work.sweepWaiters.lock, waitReasonWaitForGCCycle, traceBlockUntilGCEnds, 1)
        }
 }
 
@@ -544,17 +567,13 @@ const (
 // that the exit condition for the _GCoff phase has been met. The exit
 // condition should be tested when allocating.
 func (t gcTrigger) test() bool {
-       if !memstats.enablegc || panicking != 0 || gcphase != _GCoff {
+       if !memstats.enablegc || panicking.Load() != 0 || gcphase != _GCoff {
                return false
        }
        switch t.kind {
        case gcTriggerHeap:
-               // Non-atomic access to gcController.heapLive for performance. If
-               // we are going to trigger on this, this thread just
-               // atomically wrote gcController.heapLive anyway and we'll see our
-               // own write.
                trigger, _ := gcController.trigger()
-               return atomic.Load64(&gcController.heapLive) >= trigger
+               return gcController.heapLive.Load() >= trigger
        case gcTriggerTime:
                if gcController.gcPercent.Load() < 0 {
                        return false
@@ -563,7 +582,7 @@ func (t gcTrigger) test() bool {
                return lastgc != 0 && t.now-lastgc > forcegcperiod
        case gcTriggerCycle:
                // t.n > work.cycles, but accounting for wraparound.
-               return int32(t.n-work.cycles) > 0
+               return int32(t.n-work.cycles.Load()) > 0
        }
        return true
 }
@@ -598,7 +617,6 @@ func gcStart(trigger gcTrigger) {
        // We check the transition condition continuously here in case
        // this G gets delayed in to the next GC cycle.
        for trigger.test() && sweepone() != ^uintptr(0) {
-               sweep.nbgsweep++
        }
 
        // Perform GC initialization and the sweep termination
@@ -610,9 +628,6 @@ func gcStart(trigger gcTrigger) {
                return
        }
 
-       // For stats, check if this GC was forced by the user.
-       work.userForced = trigger.kind == gcTriggerCycle
-
        // In gcstoptheworld debug mode, upgrade the mode accordingly.
        // We do this after re-checking the transition condition so
        // that multiple goroutines that detect the heap trigger don't
@@ -628,13 +643,19 @@ func gcStart(trigger gcTrigger) {
        semacquire(&gcsema)
        semacquire(&worldsema)
 
-       if trace.enabled {
-               traceGCStart()
+       // For stats, check if this GC was forced by the user.
+       // Update it under gcsema to avoid gctrace getting wrong values.
+       work.userForced = trigger.kind == gcTriggerCycle
+
+       trace := traceAcquire()
+       if trace.ok() {
+               trace.GCStart()
+               traceRelease(trace)
        }
 
        // Check that all Ps have finished deferred mcache flushes.
        for _, p := range allp {
-               if fg := atomic.Load(&p.mcache.flushGen); fg != mheap_.sweepgen {
+               if fg := p.mcache.flushGen.Load(); fg != mheap_.sweepgen {
                        println("runtime: p", p.id, "flushGen", fg, "!= sweepgen", mheap_.sweepgen)
                        throw("p mcache not flushed")
                }
@@ -650,34 +671,31 @@ func gcStart(trigger gcTrigger) {
                // so it can't be more than ncpu, even if GOMAXPROCS is.
                work.stwprocs = ncpu
        }
-       work.heap0 = atomic.Load64(&gcController.heapLive)
+       work.heap0 = gcController.heapLive.Load()
        work.pauseNS = 0
        work.mode = mode
 
        now := nanotime()
        work.tSweepTerm = now
        work.pauseStart = now
-       if trace.enabled {
-               traceGCSTWStart(1)
-       }
-       systemstack(stopTheWorldWithSema)
+       systemstack(func() { stopTheWorldWithSema(stwGCSweepTerm) })
        // Finish sweep before we start concurrent scan.
        systemstack(func() {
                finishsweep_m()
        })
 
-       // clearpools before we start the GC. If we wait they memory will not be
+       // clearpools before we start the GC. If we wait the memory will not be
        // reclaimed until the next GC cycle.
        clearpools()
 
-       work.cycles++
+       work.cycles.Add(1)
 
        // Assists and workers can start the moment we start
        // the world.
        gcController.startCycle(now, int(gomaxprocs), trigger)
 
        // Notify the CPU limiter that assists may begin.
-       gcCPULimiter.startGCTransition(true, 0, now)
+       gcCPULimiter.startGCTransition(true, now)
 
        // In STW mode, disable scheduling of user Gs. This may also
        // disable scheduling of this goroutine, so it may block as
@@ -698,11 +716,11 @@ func gcStart(trigger gcTrigger) {
        // enabled because they must be enabled before
        // any non-leaf heap objects are marked. Since
        // allocations are blocked until assists can
-       // happen, we want enable assists as early as
+       // happen, we want to enable assists as early as
        // possible.
        setGCPhase(_GCmark)
 
-       gcBgMarkPrepare() // Must happen before assist enable.
+       gcBgMarkPrepare() // Must happen before assists are enabled.
        gcMarkRootPrepare()
 
        // Mark all active tinyalloc blocks. Since we're
@@ -725,11 +743,15 @@ func gcStart(trigger gcTrigger) {
 
        // Concurrent mark.
        systemstack(func() {
-               now = startTheWorldWithSema(trace.enabled)
+               now = startTheWorldWithSema()
                work.pauseNS += now - work.pauseStart
                work.tMark = now
                memstats.gcPauseDist.record(now - work.pauseStart)
 
+               sweepTermCpu := int64(work.stwprocs) * (work.tMark - work.tSweepTerm)
+               work.cpuStats.gcPauseTime += sweepTermCpu
+               work.cpuStats.gcTotalTime += sweepTermCpu
+
                // Release the CPU limiter.
                gcCPULimiter.finishGCTransition(now)
        })
@@ -808,22 +830,22 @@ top:
                // Otherwise, our attempt to force all P's to a safepoint could
                // result in a deadlock as we attempt to preempt a worker that's
                // trying to preempt us (e.g. for a stack scan).
-               casgstatus(gp, _Grunning, _Gwaiting)
-               forEachP(func(_p_ *p) {
+               casGToWaiting(gp, _Grunning, waitReasonGCMarkTermination)
+               forEachP(func(pp *p) {
                        // Flush the write barrier buffer, since this may add
                        // work to the gcWork.
-                       wbBufFlush1(_p_)
+                       wbBufFlush1(pp)
 
                        // Flush the gcWork, since this may create global work
                        // and set the flushedWork flag.
                        //
                        // TODO(austin): Break up these workbufs to
                        // better distribute work.
-                       _p_.gcw.dispose()
+                       pp.gcw.dispose()
                        // Collect the flushedWork flag.
-                       if _p_.gcw.flushedWork {
+                       if pp.gcw.flushedWork {
                                atomic.Xadd(&gcMarkDoneFlushed, 1)
-                               _p_.gcw.flushedWork = false
+                               pp.gcw.flushedWork = false
                        }
                })
                casgstatus(gp, _Gwaiting, _Grunning)
@@ -847,10 +869,7 @@ top:
        work.tMarkTerm = now
        work.pauseStart = now
        getg().m.preemptoff = "gcing"
-       if trace.enabled {
-               traceGCSTWStart(0)
-       }
-       systemstack(stopTheWorldWithSema)
+       systemstack(func() { stopTheWorldWithSema(stwGCMarkTerm) })
        // The gcphase is _GCmark, it will transition to _GCmarktermination
        // below. The important thing is that the wb remains active until
        // all marking is complete. This includes writes made by the GC.
@@ -877,7 +896,7 @@ top:
        if restart {
                getg().m.preemptoff = ""
                systemstack(func() {
-                       now := startTheWorldWithSema(true)
+                       now := startTheWorldWithSema()
                        work.pauseNS += now - work.pauseStart
                        memstats.gcPauseDist.record(now - work.pauseStart)
                })
@@ -885,12 +904,14 @@ top:
                goto top
        }
 
+       gcComputeStartingStackSize()
+
        // Disable assists and background workers. We must do
        // this before waking blocked assists.
        atomic.Store(&gcBlackenEnabled, 0)
 
-       // Notify the CPU limiter that assists will now cease.
-       gcCPULimiter.startGCTransition(false, gcController.assistTime.Load(), now)
+       // Notify the CPU limiter that GC assists will now cease.
+       gcCPULimiter.startGCTransition(false, now)
 
        // Wake all blocked assists. These will run when we
        // start the world again.
@@ -920,16 +941,14 @@ func gcMarkTermination() {
        // Start marktermination (write barrier remains enabled for now).
        setGCPhase(_GCmarktermination)
 
-       work.heap1 = gcController.heapLive
+       work.heap1 = gcController.heapLive.Load()
        startTime := nanotime()
 
        mp := acquirem()
        mp.preemptoff = "gcing"
-       _g_ := getg()
-       _g_.m.traceback = 2
-       gp := _g_.m.curg
-       casgstatus(gp, _Grunning, _Gwaiting)
-       gp.waitreason = waitReasonGarbageCollection
+       mp.traceback = 2
+       curgp := mp.curg
+       casGToWaiting(curgp, _Grunning, waitReasonGarbageCollection)
 
        // Run gc on the g0 stack. We do this so that the g stack
        // we're currently running on will no longer change. Cuts
@@ -947,6 +966,7 @@ func gcMarkTermination() {
                // before continuing.
        })
 
+       var stwSwept bool
        systemstack(func() {
                work.heap2 = work.bytesMarked
                if debug.gccheckmark > 0 {
@@ -965,14 +985,16 @@ func gcMarkTermination() {
 
                // marking is complete so we can turn the write barrier off
                setGCPhase(_GCoff)
-               gcSweep(work.mode)
+               stwSwept = gcSweep(work.mode)
        })
 
-       _g_.m.traceback = 0
-       casgstatus(gp, _Gwaiting, _Grunning)
+       mp.traceback = 0
+       casgstatus(curgp, _Gwaiting, _Grunning)
 
-       if trace.enabled {
-               traceGCDone()
+       trace := traceAcquire()
+       if trace.ok() {
+               trace.GCDone()
+               traceRelease(trace)
        }
 
        // all done
@@ -1002,22 +1024,28 @@ func gcMarkTermination() {
        memstats.pause_end[memstats.numgc%uint32(len(memstats.pause_end))] = uint64(unixNow)
        memstats.pause_total_ns += uint64(work.pauseNS)
 
-       // Update work.totaltime.
-       sweepTermCpu := int64(work.stwprocs) * (work.tMark - work.tSweepTerm)
-       // We report idle marking time below, but omit it from the
-       // overall utilization here since it's "free".
-       markCpu := gcController.assistTime.Load() + gcController.dedicatedMarkTime + gcController.fractionalMarkTime
        markTermCpu := int64(work.stwprocs) * (work.tEnd - work.tMarkTerm)
-       cycleCpu := sweepTermCpu + markCpu + markTermCpu
-       work.totaltime += cycleCpu
+       work.cpuStats.gcPauseTime += markTermCpu
+       work.cpuStats.gcTotalTime += markTermCpu
+
+       // Accumulate CPU stats.
+       //
+       // Pass gcMarkPhase=true so we can get all the latest GC CPU stats in there too.
+       work.cpuStats.accumulate(now, true)
 
        // Compute overall GC CPU utilization.
-       totalCpu := sched.totaltime + (now-sched.procresizetime)*int64(gomaxprocs)
-       memstats.gc_cpu_fraction = float64(work.totaltime) / float64(totalCpu)
+       // Omit idle marking time from the overall utilization here since it's "free".
+       memstats.gc_cpu_fraction = float64(work.cpuStats.gcTotalTime-work.cpuStats.gcIdleTime) / float64(work.cpuStats.totalTime)
 
-       // Reset sweep state.
-       sweep.nbgsweep = 0
-       sweep.npausesweep = 0
+       // Reset assist time and background time stats.
+       //
+       // Do this now, instead of at the start of the next GC cycle, because
+       // these two may keep accumulating even if the GC is not active.
+       scavenge.assistTime.Store(0)
+       scavenge.backgroundTime.Store(0)
+
+       // Reset idle time stat.
+       sched.idleTime.Store(0)
 
        if work.userForced {
                memstats.numforcedgc++
@@ -1029,6 +1057,12 @@ func gcMarkTermination() {
        injectglist(&work.sweepWaiters.list)
        unlock(&work.sweepWaiters.lock)
 
+       // Increment the scavenge generation now.
+       //
+       // This moment represents peak heap in use because we're
+       // about to start sweeping.
+       mheap_.pages.scav.index.nextGen()
+
        // Release the CPU limiter.
        gcCPULimiter.finishGCTransition(now)
 
@@ -1041,12 +1075,22 @@ func gcMarkTermination() {
        // Those aren't tracked in any sweep lists, so we need to
        // count them against sweep completion until we ensure all
        // those spans have been forced out.
+       //
+       // If gcSweep fully swept the heap (for example if the sweep
+       // is not concurrent due to a GODEBUG setting), then we expect
+       // the sweepLocker to be invalid, since sweeping is done.
+       //
+       // N.B. Below we might duplicate some work from gcSweep; this is
+       // fine as all that work is idempotent within a GC cycle, and
+       // we're still holding worldsema so a new cycle can't start.
        sl := sweep.active.begin()
-       if !sl.valid {
+       if !stwSwept && !sl.valid {
                throw("failed to set sweep barrier")
+       } else if stwSwept && sl.valid {
+               throw("non-concurrent sweep failed to drain all sweep queues")
        }
 
-       systemstack(func() { startTheWorldWithSema(true) })
+       systemstack(func() { startTheWorldWithSema() })
 
        // Flush the heap profile so we can start a new cycle next GC.
        // This is relatively expensive, so we don't do it with the
@@ -1064,14 +1108,36 @@ func gcMarkTermination() {
        // mcache before allocating, but idle Ps may not. Since this
        // is necessary to sweep all spans, we need to ensure all
        // mcaches are flushed before we start the next GC cycle.
+       //
+       // While we're here, flush the page cache for idle Ps to avoid
+       // having pages get stuck on them. These pages are hidden from
+       // the scavenger, so in small idle heaps a significant amount
+       // of additional memory might be held onto.
+       //
+       // Also, flush the pinner cache, to avoid leaking that memory
+       // indefinitely.
        systemstack(func() {
-               forEachP(func(_p_ *p) {
-                       _p_.mcache.prepareForSweep()
+               forEachP(func(pp *p) {
+                       pp.mcache.prepareForSweep()
+                       if pp.status == _Pidle {
+                               systemstack(func() {
+                                       lock(&mheap_.lock)
+                                       pp.pcache.flush(&mheap_.pages)
+                                       unlock(&mheap_.lock)
+                               })
+                       }
+                       pp.pinnerCache = nil
                })
        })
-       // Now that we've swept stale spans in mcaches, they don't
-       // count against unswept spans.
-       sweep.active.end(sl)
+       if sl.valid {
+               // Now that we've swept stale spans in mcaches, they don't
+               // count against unswept spans.
+               //
+               // Note: this sweepLocker may not be valid if sweeping had
+               // already completed during the STW. See the corresponding
+               // begin() call that produced sl.
+               sweep.active.end(sl)
+       }
 
        // Print gctrace before dropping worldsema. As soon as we drop
        // worldsema another cycle could start and smash the stats
@@ -1094,10 +1160,10 @@ func gcMarkTermination() {
                }
                print(" ms clock, ")
                for i, ns := range []int64{
-                       sweepTermCpu,
+                       int64(work.stwprocs) * (work.tMark - work.tSweepTerm),
                        gcController.assistTime.Load(),
-                       gcController.dedicatedMarkTime + gcController.fractionalMarkTime,
-                       gcController.idleMarkTime,
+                       gcController.dedicatedMarkTime.Load() + gcController.fractionalMarkTime.Load(),
+                       gcController.idleMarkTime.Load(),
                        markTermCpu,
                } {
                        if i == 2 || i == 3 {
@@ -1110,9 +1176,9 @@ func gcMarkTermination() {
                }
                print(" ms cpu, ",
                        work.heap0>>20, "->", work.heap1>>20, "->", work.heap2>>20, " MB, ",
-                       gcController.heapGoal()>>20, " MB goal, ",
-                       gcController.stackScan>>20, " MB stacks, ",
-                       gcController.globalsScan>>20, " MB globals, ",
+                       gcController.lastHeapGoal>>20, " MB goal, ",
+                       gcController.lastStackScan.Load()>>20, " MB stacks, ",
+                       gcController.globalsScan.Load()>>20, " MB globals, ",
                        work.maxprocs, " P")
                if work.userForced {
                        print(" (forced)")
@@ -1121,6 +1187,20 @@ func gcMarkTermination() {
                printunlock()
        }
 
+       // Set any arena chunks that were deferred to fault.
+       lock(&userArenaState.lock)
+       faultList := userArenaState.fault
+       userArenaState.fault = nil
+       unlock(&userArenaState.lock)
+       for _, lc := range faultList {
+               lc.mspan.setUserArenaChunkToFault()
+       }
+
+       // Enable huge pages on some metadata if we cross a heap threshold.
+       if gcController.heapGoal() > minHeapForMetadataHugePages {
+               mheap_.enableMetadataHugePages()
+       }
+
        semrelease(&worldsema)
        semrelease(&gcsema)
        // Careful: another GC cycle may start now.
@@ -1173,7 +1253,7 @@ func gcBgMarkPrepare() {
        work.nwait = ^uint32(0)
 }
 
-// gcBgMarkWorker is an entry in the gcBgMarkWorkerPool. It points to a single
+// gcBgMarkWorkerNode is an entry in the gcBgMarkWorkerPool. It points to a single
 // gcBgMarkWorker goroutine.
 type gcBgMarkWorkerNode struct {
        // Unused workers are managed in a lock-free stack. This field must be first.
@@ -1249,7 +1329,7 @@ func gcBgMarkWorker() {
                        // Note that at this point, the G may immediately be
                        // rescheduled and may be running.
                        return true
-               }, unsafe.Pointer(node), waitReasonGCWorkerIdle, traceEvGoBlock, 0)
+               }, unsafe.Pointer(node), waitReasonGCWorkerIdle, traceBlockSystemGoroutine, 0)
 
                // Preemption must not occur here, or another G might see
                // p.gcMarkWorkerMode.
@@ -1271,6 +1351,10 @@ func gcBgMarkWorker() {
 
                startTime := nanotime()
                pp.gcMarkWorkerStartTime = startTime
+               var trackLimiterEvent bool
+               if pp.gcMarkWorkerMode == gcMarkWorkerIdleMode {
+                       trackLimiterEvent = pp.limiterEvent.start(limiterEventIdleMarkWork, startTime)
+               }
 
                decnwait := atomic.Xadd(&work.nwait, -1)
                if decnwait == work.nproc {
@@ -1286,12 +1370,12 @@ func gcBgMarkWorker() {
                        // the G stack. However, stack shrinking is
                        // disabled for mark workers, so it is safe to
                        // read from the G stack.
-                       casgstatus(gp, _Grunning, _Gwaiting)
+                       casGToWaiting(gp, _Grunning, waitReasonGCWorkerActive)
                        switch pp.gcMarkWorkerMode {
                        default:
                                throw("gcBgMarkWorker: unexpected gcMarkWorkerMode")
                        case gcMarkWorkerDedicatedMode:
-                               gcDrain(&pp.gcw, gcDrainUntilPreempt|gcDrainFlushBgCredit)
+                               gcDrainMarkWorkerDedicated(&pp.gcw, true)
                                if gp.preempt {
                                        // We were preempted. This is
                                        // a useful signal to kick
@@ -1306,18 +1390,22 @@ func gcBgMarkWorker() {
                                }
                                // Go back to draining, this time
                                // without preemption.
-                               gcDrain(&pp.gcw, gcDrainFlushBgCredit)
+                               gcDrainMarkWorkerDedicated(&pp.gcw, false)
                        case gcMarkWorkerFractionalMode:
-                               gcDrain(&pp.gcw, gcDrainFractional|gcDrainUntilPreempt|gcDrainFlushBgCredit)
+                               gcDrainMarkWorkerFractional(&pp.gcw)
                        case gcMarkWorkerIdleMode:
-                               gcDrain(&pp.gcw, gcDrainIdle|gcDrainUntilPreempt|gcDrainFlushBgCredit)
+                               gcDrainMarkWorkerIdle(&pp.gcw)
                        }
                        casgstatus(gp, _Gwaiting, _Grunning)
                })
 
                // Account for time and mark us as stopped.
-               duration := nanotime() - startTime
+               now := nanotime()
+               duration := now - startTime
                gcController.markWorkerStop(pp.gcMarkWorkerMode, duration)
+               if trackLimiterEvent {
+                       pp.limiterEvent.stop(limiterEventIdleMarkWork, now)
+               }
                if pp.gcMarkWorkerMode == gcMarkWorkerFractionalMode {
                        atomic.Xaddint64(&pp.gcFractionalMarkTime, duration)
                }
@@ -1391,9 +1479,6 @@ func gcMark(startTime int64) {
                // Gs, so only do it if checkmark is also enabled.
                gcMarkRootCheck()
        }
-       if work.full != 0 {
-               throw("work.full != 0")
-       }
 
        // Drop allg snapshot. allgs may have grown, in which case
        // this is the only reference to the old backing store and
@@ -1465,10 +1550,12 @@ func gcMark(startTime int64) {
 // gcSweep must be called on the system stack because it acquires the heap
 // lock. See mheap for details.
 //
+// Returns true if the heap was fully swept by this function.
+//
 // The world must be stopped.
 //
 //go:systemstack
-func gcSweep(mode gcMode) {
+func gcSweep(mode gcMode) bool {
        assertWorldStopped()
 
        if gcphase != _GCoff {
@@ -1486,15 +1573,18 @@ func gcSweep(mode gcMode) {
 
        sweep.centralIndex.clear()
 
-       if !_ConcurrentSweep || mode == gcForceBlockMode {
+       if !concurrentSweep || mode == gcForceBlockMode {
                // Special case synchronous sweep.
                // Record that no proportional sweeping has to happen.
                lock(&mheap_.lock)
                mheap_.sweepPagesPerByte = 0
                unlock(&mheap_.lock)
+               // Flush all mcaches.
+               for _, pp := range allp {
+                       pp.mcache.prepareForSweep()
+               }
                // Sweep all spans eagerly.
                for sweepone() != ^uintptr(0) {
-                       sweep.npausesweep++
                }
                // Free workbufs eagerly.
                prepareFreeWorkbufs()
@@ -1505,7 +1595,7 @@ func gcSweep(mode gcMode) {
                // available immediately.
                mProf_NextCycle()
                mProf_Flush()
-               return
+               return true
        }
 
        // Background sweep.
@@ -1515,6 +1605,7 @@ func gcSweep(mode gcMode) {
                ready(sweep.g, 0, true)
        }
        unlock(&sweep.lock)
+       return false
 }
 
 // gcResetMarkState resets global state prior to marking (concurrent
@@ -1548,7 +1639,7 @@ func gcResetMarkState() {
        }
 
        work.bytesMarked = 0
-       work.initialHeapLive = atomic.Load64(&gcController.heapLive)
+       work.initialHeapLive = gcController.heapLive.Load()
 }
 
 // Hooks for other packages
@@ -1561,7 +1652,7 @@ func sync_runtime_registerPoolCleanup(f func()) {
        poolcleanup = f
 }
 
-//go:linkname boring_registerCache crypto/internal/boring.registerCache
+//go:linkname boring_registerCache crypto/internal/boring/bcache.registerCache
 func boring_registerCache(p unsafe.Pointer) {
        boringCaches = append(boringCaches, p)
 }