mirror of
https://github.com/golang/go.git
synced 2025-12-08 06:10:04 +00:00
runtime: add new page allocator core
This change adds a new bitmap-based allocator to the runtime with tests. It does not yet integrate the page allocator into the runtime and thus this change is almost purely additive. Updates #35112. Change-Id: Ic3d024c28abee8be8797d3918116a80f901cc2bf Reviewed-on: https://go-review.googlesource.com/c/go/+/190622 Run-TryBot: Michael Knyszek <mknyszek@google.com> TryBot-Result: Gobot Gobot <gobot@golang.org> Reviewed-by: Austin Clements <austin@google.com>
This commit is contained in:
parent
05aa4a7b74
commit
39e8cb0faa
6 changed files with 1716 additions and 2 deletions
|
|
@ -48,6 +48,10 @@
|
|||
|
||||
package runtime
|
||||
|
||||
import (
|
||||
"unsafe"
|
||||
)
|
||||
|
||||
const (
|
||||
// The size of a bitmap chunk, i.e. the amount of bits (that is, pages) to consider
|
||||
// in the bitmap at once.
|
||||
|
|
@ -61,7 +65,7 @@ const (
|
|||
// The value of 3 is chosen such that the block of summaries we need to scan at
|
||||
// each level fits in 64 bytes (2^3 summaries * 8 bytes per summary), which is
|
||||
// close to the L1 cache line width on many systems. Also, a value of 3 fits 4 tree
|
||||
// levels perfectly into the 21-bit mallocBits summary field at the root level.
|
||||
// levels perfectly into the 21-bit pallocBits summary field at the root level.
|
||||
//
|
||||
// The following equation explains how each of the constants relate:
|
||||
// summaryL0Bits + (summaryLevels-1)*summaryLevelBits + logPallocChunkBytes = heapAddrBits
|
||||
|
|
@ -69,13 +73,727 @@ const (
|
|||
// summaryLevels is an architecture-dependent value defined in mpagealloc_*.go.
|
||||
summaryLevelBits = 3
|
||||
summaryL0Bits = heapAddrBits - logPallocChunkBytes - (summaryLevels-1)*summaryLevelBits
|
||||
|
||||
// Maximum searchAddr value, which indicates that the heap has no free space.
|
||||
//
|
||||
// We subtract arenaBaseOffset because we want this to represent the maximum
|
||||
// value in the shifted address space, but searchAddr is stored as a regular
|
||||
// memory address. See arenaBaseOffset for details.
|
||||
maxSearchAddr = ^uintptr(0) - arenaBaseOffset
|
||||
)
|
||||
|
||||
// Global chunk index.
|
||||
//
|
||||
// Represents an index into the leaf level of the radix tree.
|
||||
// Similar to arenaIndex, except instead of arenas, it divides the address
|
||||
// space into chunks.
|
||||
type chunkIdx uint
|
||||
|
||||
// chunkIndex returns the global index of the palloc chunk containing the
|
||||
// pointer p.
|
||||
func chunkIndex(p uintptr) chunkIdx {
|
||||
return chunkIdx((p + arenaBaseOffset) / pallocChunkBytes)
|
||||
}
|
||||
|
||||
// chunkIndex returns the base address of the palloc chunk at index ci.
|
||||
func chunkBase(ci chunkIdx) uintptr {
|
||||
return uintptr(ci)*pallocChunkBytes - arenaBaseOffset
|
||||
}
|
||||
|
||||
// chunkPageIndex computes the index of the page that contains p,
|
||||
// relative to the chunk which contains p.
|
||||
func chunkPageIndex(p uintptr) uint {
|
||||
return uint(p % pallocChunkBytes / pageSize)
|
||||
}
|
||||
|
||||
// addrsToSummaryRange converts base and limit pointers into a range
|
||||
// of entries for the given summary level.
|
||||
//
|
||||
// The returned range is inclusive on the lower bound and exclusive on
|
||||
// the upper bound.
|
||||
func addrsToSummaryRange(level int, base, limit uintptr) (lo int, hi int) {
|
||||
// This is slightly more nuanced than just a shift for the exclusive
|
||||
// upper-bound. Note that the exclusive upper bound may be within a
|
||||
// summary at this level, meaning if we just do the obvious computation
|
||||
// hi will end up being an inclusive upper bound. Unfortunately, just
|
||||
// adding 1 to that is too broad since we might be on the very edge of
|
||||
// of a summary's max page count boundary for this level
|
||||
// (1 << levelLogPages[level]). So, make limit an inclusive upper bound
|
||||
// then shift, then add 1, so we get an exclusive upper bound at the end.
|
||||
lo = int((base + arenaBaseOffset) >> levelShift[level])
|
||||
hi = int(((limit-1)+arenaBaseOffset)>>levelShift[level]) + 1
|
||||
return
|
||||
}
|
||||
|
||||
// blockAlignSummaryRange aligns indices into the given level to that
|
||||
// level's block width (1 << levelBits[level]). It assumes lo is inclusive
|
||||
// and hi is exclusive, and so aligns them down and up respectively.
|
||||
func blockAlignSummaryRange(level int, lo, hi int) (int, int) {
|
||||
e := uintptr(1) << levelBits[level]
|
||||
return int(alignDown(uintptr(lo), e)), int(alignUp(uintptr(hi), e))
|
||||
}
|
||||
|
||||
type pageAlloc struct {
|
||||
// Radix tree of summaries.
|
||||
//
|
||||
// Each slice's cap represents the whole memory reservation.
|
||||
// Each slice's len reflects the allocator's maximum known
|
||||
// mapped heap address for that level.
|
||||
//
|
||||
// The backing store of each summary level is reserved in init
|
||||
// and may or may not be committed in grow (small address spaces
|
||||
// may commit all the memory in init).
|
||||
//
|
||||
// The purpose of keeping len <= cap is to enforce bounds checks
|
||||
// on the top end of the slice so that instead of an unknown
|
||||
// runtime segmentation fault, we get a much friendlier out-of-bounds
|
||||
// error.
|
||||
//
|
||||
// We may still get segmentation faults < len since some of that
|
||||
// memory may not be committed yet.
|
||||
summary [summaryLevels][]pallocSum
|
||||
|
||||
// chunks is a slice of bitmap chunks.
|
||||
//
|
||||
// The backing store for chunks is reserved in init and committed
|
||||
// by grow.
|
||||
//
|
||||
// To find the chunk containing a memory address `a`, do:
|
||||
// chunks[chunkIndex(a)]
|
||||
//
|
||||
// summary[len(s.summary)-1][i] should always be checked, at least
|
||||
// for a zero max value, before accessing chunks[i]. It's possible the
|
||||
// bitmap at that index is mapped in and zeroed, indicating that it
|
||||
// contains free space, but in actuality it is unused since its
|
||||
// corresponding summary was never updated. Tests may ignore this
|
||||
// and assume the zero value (and that it is mapped).
|
||||
//
|
||||
// TODO(mknyszek): Consider changing the definition of the bitmap
|
||||
// such that 1 means free and 0 means in-use so that summaries and
|
||||
// the bitmaps align better on zero-values.
|
||||
chunks []pallocBits
|
||||
|
||||
// The address to start an allocation search with.
|
||||
//
|
||||
// When added with arenaBaseOffset, we guarantee that
|
||||
// all valid heap addresses (when also added with
|
||||
// arenaBaseOffset) below this value are allocated and
|
||||
// not worth searching.
|
||||
//
|
||||
// Note that adding in arenaBaseOffset transforms addresses
|
||||
// to a new address space with a linear view of the full address
|
||||
// space on architectures with segmented address spaces.
|
||||
searchAddr uintptr
|
||||
|
||||
// start and end represent the chunk indices
|
||||
// which pageAlloc knows about. It assumes
|
||||
// chunks in the range [start, end) are
|
||||
// currently ready to use.
|
||||
start, end chunkIdx
|
||||
|
||||
// mheap_.lock. This level of indirection makes it possible
|
||||
// to test pageAlloc indepedently of the runtime allocator.
|
||||
mheapLock *mutex
|
||||
|
||||
// sysStat is the runtime memstat to update when new system
|
||||
// memory is committed by the pageAlloc for allocation metadata.
|
||||
sysStat *uint64
|
||||
}
|
||||
|
||||
func (s *pageAlloc) init(mheapLock *mutex, sysStat *uint64) {
|
||||
if levelLogPages[0] > logMaxPackedValue {
|
||||
// We can't represent 1<<levelLogPages[0] pages, the maximum number
|
||||
// of pages we need to represent at the root level, in a summary, which
|
||||
// is a big problem. Throw.
|
||||
print("runtime: root level max pages = ", 1<<levelLogPages[0], "\n")
|
||||
print("runtime: summary max pages = ", maxPackedValue, "\n")
|
||||
throw("root level max pages doesn't fit in summary")
|
||||
}
|
||||
s.sysStat = sysStat
|
||||
|
||||
// System-dependent initialization.
|
||||
s.sysInit()
|
||||
|
||||
// Start with the searchAddr in a state indicating there's no free memory.
|
||||
s.searchAddr = maxSearchAddr
|
||||
|
||||
// Reserve space for the bitmap and put this reservation
|
||||
// into the chunks slice.
|
||||
const maxChunks = (1 << heapAddrBits) / pallocChunkBytes
|
||||
r := sysReserve(nil, maxChunks*unsafe.Sizeof(s.chunks[0]))
|
||||
if r == nil {
|
||||
throw("failed to reserve page bitmap memory")
|
||||
}
|
||||
sl := notInHeapSlice{(*notInHeap)(r), 0, maxChunks}
|
||||
s.chunks = *(*[]pallocBits)(unsafe.Pointer(&sl))
|
||||
|
||||
// Set the mheapLock.
|
||||
s.mheapLock = mheapLock
|
||||
}
|
||||
|
||||
// extendMappedRegion ensures that all the memory in the range
|
||||
// [base+nbase, base+nlimit) is in the Ready state.
|
||||
// base must refer to the beginning of a memory region in the
|
||||
// Reserved state. extendMappedRegion assumes that the region
|
||||
// [base+mbase, base+mlimit) is already mapped.
|
||||
//
|
||||
// Note that extendMappedRegion only supports extending
|
||||
// mappings in one direction. Therefore,
|
||||
// nbase < mbase && nlimit > mlimit is an invalid input
|
||||
// and this function will throw.
|
||||
func extendMappedRegion(base unsafe.Pointer, mbase, mlimit, nbase, nlimit uintptr, sysStat *uint64) {
|
||||
if uintptr(base)%physPageSize != 0 {
|
||||
print("runtime: base = ", base, "\n")
|
||||
throw("extendMappedRegion: base not page-aligned")
|
||||
}
|
||||
// Round the offsets to a physical page.
|
||||
mbase = alignDown(mbase, physPageSize)
|
||||
nbase = alignDown(nbase, physPageSize)
|
||||
mlimit = alignUp(mlimit, physPageSize)
|
||||
nlimit = alignUp(nlimit, physPageSize)
|
||||
|
||||
// If none of the region is mapped, don't bother
|
||||
// trying to figure out which parts are.
|
||||
if mlimit-mbase != 0 {
|
||||
// Determine which part of the region actually needs
|
||||
// mapping.
|
||||
if nbase < mbase && nlimit > mlimit {
|
||||
// TODO(mknyszek): Consider supporting this case. It can't
|
||||
// ever happen currently in the page allocator, but may be
|
||||
// useful in the future. Also, it would make this function's
|
||||
// purpose simpler to explain.
|
||||
throw("mapped region extended in two directions")
|
||||
} else if nbase < mbase && nlimit <= mlimit {
|
||||
nlimit = mbase
|
||||
} else if nbase >= mbase && nlimit > mlimit {
|
||||
nbase = mlimit
|
||||
} else {
|
||||
return
|
||||
}
|
||||
}
|
||||
|
||||
// Transition from Reserved to Ready.
|
||||
rbase := add(base, nbase)
|
||||
sysMap(rbase, nlimit-nbase, sysStat)
|
||||
sysUsed(rbase, nlimit-nbase)
|
||||
}
|
||||
|
||||
// compareSearchAddrTo compares an address against s.searchAddr in a linearized
|
||||
// view of the address space on systems with discontinuous process address spaces.
|
||||
// This linearized view is the same one generated by chunkIndex and arenaIndex,
|
||||
// done by adding arenaBaseOffset.
|
||||
//
|
||||
// On systems without a discontinuous address space, it's just a normal comparison.
|
||||
//
|
||||
// Returns < 0 if addr is less than s.searchAddr in the linearized address space.
|
||||
// Returns > 0 if addr is greater than s.searchAddr in the linearized address space.
|
||||
// Returns 0 if addr and s.searchAddr are equal.
|
||||
func (s *pageAlloc) compareSearchAddrTo(addr uintptr) int {
|
||||
// Compare with arenaBaseOffset added because it gives us a linear, contiguous view
|
||||
// of the heap on architectures with signed address spaces.
|
||||
lAddr := addr + arenaBaseOffset
|
||||
lSearchAddr := s.searchAddr + arenaBaseOffset
|
||||
if lAddr < lSearchAddr {
|
||||
return -1
|
||||
} else if lAddr > lSearchAddr {
|
||||
return 1
|
||||
}
|
||||
return 0
|
||||
}
|
||||
|
||||
// grow sets up the metadata for the address range [base, base+size).
|
||||
// It may allocate metadata, in which case *s.sysStat will be updated.
|
||||
//
|
||||
// s.mheapLock must be held.
|
||||
func (s *pageAlloc) grow(base, size uintptr) {
|
||||
// Round up to chunks, since we can't deal with increments smaller
|
||||
// than chunks. Also, sysGrow expects aligned values.
|
||||
limit := alignUp(base+size, pallocChunkBytes)
|
||||
base = alignDown(base, pallocChunkBytes)
|
||||
|
||||
// Grow the summary levels in a system-dependent manner.
|
||||
// We just update a bunch of additional metadata here.
|
||||
s.sysGrow(base, limit)
|
||||
|
||||
// Update s.start and s.end.
|
||||
// If no growth happened yet, start == 0. This is generally
|
||||
// safe since the zero page is unmapped.
|
||||
oldStart, oldEnd := s.start, s.end
|
||||
firstGrowth := s.start == 0
|
||||
start, end := chunkIndex(base), chunkIndex(limit)
|
||||
if firstGrowth || start < s.start {
|
||||
s.start = start
|
||||
}
|
||||
if end > s.end {
|
||||
s.end = end
|
||||
|
||||
// s.end corresponds directly to the length of s.chunks,
|
||||
// so just update it here.
|
||||
s.chunks = s.chunks[:end]
|
||||
}
|
||||
|
||||
// Extend the mapped part of the chunk reservation.
|
||||
elemSize := unsafe.Sizeof(s.chunks[0])
|
||||
extendMappedRegion(
|
||||
unsafe.Pointer(&s.chunks[0]),
|
||||
uintptr(oldStart)*elemSize,
|
||||
uintptr(oldEnd)*elemSize,
|
||||
uintptr(s.start)*elemSize,
|
||||
uintptr(s.end)*elemSize,
|
||||
s.sysStat,
|
||||
)
|
||||
|
||||
// A grow operation is a lot like a free operation, so if our
|
||||
// chunk ends up below the (linearized) s.searchAddr, update
|
||||
// s.searchAddr to the new address, just like in free.
|
||||
if s.compareSearchAddrTo(base) < 0 {
|
||||
s.searchAddr = base
|
||||
}
|
||||
|
||||
// Update summaries accordingly. The grow acts like a free, so
|
||||
// we need to ensure this newly-free memory is visible in the
|
||||
// summaries.
|
||||
s.update(base, size/pageSize, true, false)
|
||||
}
|
||||
|
||||
// update updates heap metadata. It must be called each time the bitmap
|
||||
// is updated.
|
||||
//
|
||||
// If contig is true, update does some optimizations assuming that there was
|
||||
// a contiguous allocation or free between addr and addr+npages. alloc indicates
|
||||
// whether the operation performed was an allocation or a free.
|
||||
//
|
||||
// s.mheapLock must be held.
|
||||
func (s *pageAlloc) update(base, npages uintptr, contig, alloc bool) {
|
||||
// base, limit, start, and end are inclusive.
|
||||
limit := base + npages*pageSize - 1
|
||||
sc, ec := chunkIndex(base), chunkIndex(limit)
|
||||
|
||||
// Handle updating the lowest level first.
|
||||
if sc == ec {
|
||||
// Fast path: the allocation doesn't span more than one chunk,
|
||||
// so update this one and if the summary didn't change, return.
|
||||
x := s.summary[len(s.summary)-1][sc]
|
||||
y := s.chunks[sc].summarize()
|
||||
if x == y {
|
||||
return
|
||||
}
|
||||
s.summary[len(s.summary)-1][sc] = y
|
||||
} else if contig {
|
||||
// Slow contiguous path: the allocation spans more than one chunk
|
||||
// and at least one summary is guaranteed to change.
|
||||
summary := s.summary[len(s.summary)-1]
|
||||
|
||||
// Update the summary for chunk sc.
|
||||
summary[sc] = s.chunks[sc].summarize()
|
||||
|
||||
// Update the summaries for chunks in between, which are
|
||||
// either totally allocated or freed.
|
||||
whole := s.summary[len(s.summary)-1][sc+1 : ec]
|
||||
if alloc {
|
||||
// Should optimize into a memclr.
|
||||
for i := range whole {
|
||||
whole[i] = 0
|
||||
}
|
||||
} else {
|
||||
for i := range whole {
|
||||
whole[i] = freeChunkSum
|
||||
}
|
||||
}
|
||||
|
||||
// Update the summary for chunk ec.
|
||||
summary[ec] = s.chunks[ec].summarize()
|
||||
} else {
|
||||
// Slow general path: the allocation spans more than one chunk
|
||||
// and at least one summary is guaranteed to change.
|
||||
//
|
||||
// We can't assume a contiguous allocation happened, so walk over
|
||||
// every chunk in the range and manually recompute the summary.
|
||||
summary := s.summary[len(s.summary)-1]
|
||||
for c := sc; c <= ec; c++ {
|
||||
summary[c] = s.chunks[c].summarize()
|
||||
}
|
||||
}
|
||||
|
||||
// Walk up the radix tree and update the summaries appropriately.
|
||||
changed := true
|
||||
for l := len(s.summary) - 2; l >= 0 && changed; l-- {
|
||||
// Update summaries at level l from summaries at level l+1.
|
||||
changed = false
|
||||
|
||||
// "Constants" for the previous level which we
|
||||
// need to compute the summary from that level.
|
||||
logEntriesPerBlock := levelBits[l+1]
|
||||
logMaxPages := levelLogPages[l+1]
|
||||
|
||||
// lo and hi describe all the parts of the level we need to look at.
|
||||
lo, hi := addrsToSummaryRange(l, base, limit+1)
|
||||
|
||||
// Iterate over each block, updating the corresponding summary in the less-granular level.
|
||||
for i := lo; i < hi; i++ {
|
||||
children := s.summary[l+1][i<<logEntriesPerBlock : (i+1)<<logEntriesPerBlock]
|
||||
sum := mergeSummaries(children, logMaxPages)
|
||||
old := s.summary[l][i]
|
||||
if old != sum {
|
||||
changed = true
|
||||
s.summary[l][i] = sum
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// allocRange marks the range of memory [base, base+npages*pageSize) as
|
||||
// allocated. It also updates the summaries to reflect the newly-updated
|
||||
// bitmap.
|
||||
//
|
||||
// s.mheapLock must be held.
|
||||
func (s *pageAlloc) allocRange(base, npages uintptr) {
|
||||
limit := base + npages*pageSize - 1
|
||||
sc, ec := chunkIndex(base), chunkIndex(limit)
|
||||
si, ei := chunkPageIndex(base), chunkPageIndex(limit)
|
||||
|
||||
if sc == ec {
|
||||
// The range doesn't cross any chunk boundaries.
|
||||
s.chunks[sc].allocRange(si, ei+1-si)
|
||||
} else {
|
||||
// The range crosses at least one chunk boundary.
|
||||
s.chunks[sc].allocRange(si, pallocChunkPages-si)
|
||||
for c := sc + 1; c < ec; c++ {
|
||||
s.chunks[c].allocAll()
|
||||
}
|
||||
s.chunks[ec].allocRange(0, ei+1)
|
||||
}
|
||||
s.update(base, npages, true, true)
|
||||
}
|
||||
|
||||
// find searches for the first (address-ordered) contiguous free region of
|
||||
// npages in size and returns a base address for that region.
|
||||
//
|
||||
// It uses s.searchAddr to prune its search and assumes that no palloc chunks
|
||||
// below chunkIndex(s.searchAddr) contain any free memory at all.
|
||||
//
|
||||
// find also computes and returns a candidate s.searchAddr, which may or
|
||||
// may not prune more of the address space than s.searchAddr already does.
|
||||
//
|
||||
// find represents the slow path and the full radix tree search.
|
||||
//
|
||||
// Returns a base address of 0 on failure, in which case the candidate
|
||||
// searchAddr returned is invalid and must be ignored.
|
||||
//
|
||||
// s.mheapLock must be held.
|
||||
func (s *pageAlloc) find(npages uintptr) (uintptr, uintptr) {
|
||||
// Search algorithm.
|
||||
//
|
||||
// This algorithm walks each level l of the radix tree from the root level
|
||||
// to the leaf level. It iterates over at most 1 << levelBits[l] of entries
|
||||
// in a given level in the radix tree, and uses the summary information to
|
||||
// find either:
|
||||
// 1) That a given subtree contains a large enough contiguous region, at
|
||||
// which point it continues iterating on the next level, or
|
||||
// 2) That there are enough contiguous boundary-crossing bits to satisfy
|
||||
// the allocation, at which point it knows exactly where to start
|
||||
// allocating from.
|
||||
//
|
||||
// i tracks the index into the current level l's structure for the
|
||||
// contiguous 1 << levelBits[l] entries we're actually interested in.
|
||||
//
|
||||
// NOTE: Technically this search could allocate a region which crosses
|
||||
// the arenaBaseOffset boundary, which when arenaBaseOffset != 0, is
|
||||
// a discontinuity. However, the only way this could happen is if the
|
||||
// page at the zero address is mapped, and this is impossible on
|
||||
// every system we support where arenaBaseOffset != 0. So, the
|
||||
// discontinuity is already encoded in the fact that the OS will never
|
||||
// map the zero page for us, and this function doesn't try to handle
|
||||
// this case in any way.
|
||||
|
||||
// i is the beginning of the block of entries we're searching at the
|
||||
// current level.
|
||||
i := 0
|
||||
|
||||
// firstFree is the region of address space that we are certain to
|
||||
// find the first free page in the heap. base and bound are the inclusive
|
||||
// bounds of this window, and both are addresses in the linearized, contiguous
|
||||
// view of the address space (with arenaBaseOffset pre-added). At each level,
|
||||
// this window is narrowed as we find the memory region containing the
|
||||
// first free page of memory. To begin with, the range reflects the
|
||||
// full process address space.
|
||||
//
|
||||
// firstFree is updated by calling foundFree each time free space in the
|
||||
// heap is discovered.
|
||||
//
|
||||
// At the end of the search, base-arenaBaseOffset is the best new
|
||||
// searchAddr we could deduce in this search.
|
||||
firstFree := struct {
|
||||
base, bound uintptr
|
||||
}{
|
||||
base: 0,
|
||||
bound: (1<<heapAddrBits - 1),
|
||||
}
|
||||
// foundFree takes the given address range [addr, addr+size) and
|
||||
// updates firstFree if it is a narrower range. The input range must
|
||||
// either be fully contained within firstFree or not overlap with it
|
||||
// at all.
|
||||
//
|
||||
// This way, we'll record the first summary we find with any free
|
||||
// pages on the root level and narrow that down if we descend into
|
||||
// that summary. But as soon as we need to iterate beyond that summary
|
||||
// in a level to find a large enough range, we'll stop narrowing.
|
||||
foundFree := func(addr, size uintptr) {
|
||||
if firstFree.base <= addr && addr+size-1 <= firstFree.bound {
|
||||
// This range fits within the current firstFree window, so narrow
|
||||
// down the firstFree window to the base and bound of this range.
|
||||
firstFree.base = addr
|
||||
firstFree.bound = addr + size - 1
|
||||
} else if !(addr+size-1 < firstFree.base || addr > firstFree.bound) {
|
||||
// This range only partially overlaps with the firstFree range,
|
||||
// so throw.
|
||||
print("runtime: addr = ", hex(addr), ", size = ", size, "\n")
|
||||
print("runtime: base = ", hex(firstFree.base), ", bound = ", hex(firstFree.bound), "\n")
|
||||
throw("range partially overlaps")
|
||||
}
|
||||
}
|
||||
|
||||
// lastSum is the summary which we saw on the previous level that made us
|
||||
// move on to the next level. Used to print additional information in the
|
||||
// case of a catastrophic failure.
|
||||
// lastSumIdx is that summary's index in the previous level.
|
||||
lastSum := packPallocSum(0, 0, 0)
|
||||
lastSumIdx := -1
|
||||
|
||||
nextLevel:
|
||||
for l := 0; l < len(s.summary); l++ {
|
||||
// For the root level, entriesPerBlock is the whole level.
|
||||
entriesPerBlock := 1 << levelBits[l]
|
||||
logMaxPages := levelLogPages[l]
|
||||
|
||||
// We've moved into a new level, so let's update i to our new
|
||||
// starting index. This is a no-op for level 0.
|
||||
i <<= levelBits[l]
|
||||
|
||||
// Slice out the block of entries we care about.
|
||||
entries := s.summary[l][i : i+entriesPerBlock]
|
||||
|
||||
// Determine j0, the first index we should start iterating from.
|
||||
// The searchAddr may help us eliminate iterations if we followed the
|
||||
// searchAddr on the previous level or we're on the root leve, in which
|
||||
// case the searchAddr should be the same as i after levelShift.
|
||||
j0 := 0
|
||||
if searchIdx := int((s.searchAddr + arenaBaseOffset) >> levelShift[l]); searchIdx&^(entriesPerBlock-1) == i {
|
||||
j0 = searchIdx & (entriesPerBlock - 1)
|
||||
}
|
||||
|
||||
// Run over the level entries looking for
|
||||
// a contiguous run of at least npages either
|
||||
// within an entry or across entries.
|
||||
//
|
||||
// base contains the page index (relative to
|
||||
// the first entry's first page) of the currently
|
||||
// considered run of consecutive pages.
|
||||
//
|
||||
// size contains the size of the currently considered
|
||||
// run of consecutive pages.
|
||||
var base, size uint
|
||||
for j := j0; j < len(entries); j++ {
|
||||
sum := entries[j]
|
||||
if sum == 0 {
|
||||
// A full entry means we broke any streak and
|
||||
// that we should skip it altogether.
|
||||
size = 0
|
||||
continue
|
||||
}
|
||||
|
||||
// We've encountered a non-zero summary which means
|
||||
// free memory, so update firstFree.
|
||||
foundFree(uintptr((i+j)<<levelShift[l]), (uintptr(1)<<logMaxPages)*pageSize)
|
||||
|
||||
s := sum.start()
|
||||
if size+s >= uint(npages) {
|
||||
// If size == 0 we don't have a run yet,
|
||||
// which means base isn't valid. So, set
|
||||
// base to the first page in this block.
|
||||
if size == 0 {
|
||||
base = uint(j) << logMaxPages
|
||||
}
|
||||
// We hit npages; we're done!
|
||||
size += s
|
||||
break
|
||||
}
|
||||
if sum.max() >= uint(npages) {
|
||||
// The entry itself contains npages contiguous
|
||||
// free pages, so continue on the next level
|
||||
// to find that run.
|
||||
i += j
|
||||
lastSumIdx = i
|
||||
lastSum = sum
|
||||
continue nextLevel
|
||||
}
|
||||
if size == 0 || s < 1<<logMaxPages {
|
||||
// We either don't have a current run started, or this entry
|
||||
// isn't totally free (meaning we can't continue the current
|
||||
// one), so try to begin a new run by setting size and base
|
||||
// based on sum.end.
|
||||
size = sum.end()
|
||||
base = uint(j+1)<<logMaxPages - size
|
||||
continue
|
||||
}
|
||||
// The entry is completely free, so continue the run.
|
||||
size += 1 << logMaxPages
|
||||
}
|
||||
if size >= uint(npages) {
|
||||
// We found a sufficiently large run of free pages straddling
|
||||
// some boundary, so compute the address and return it.
|
||||
addr := uintptr(i<<levelShift[l]) - arenaBaseOffset + uintptr(base)*pageSize
|
||||
return addr, firstFree.base - arenaBaseOffset
|
||||
}
|
||||
if l == 0 {
|
||||
// We're at level zero, so that means we've exhausted our search.
|
||||
return 0, maxSearchAddr
|
||||
}
|
||||
|
||||
// We're not at level zero, and we exhausted the level we were looking in.
|
||||
// This means that either our calculations were wrong or the level above
|
||||
// lied to us. In either case, dump some useful state and throw.
|
||||
print("runtime: summary[", l-1, "][", lastSumIdx, "] = ", lastSum.start(), ", ", lastSum.max(), ", ", lastSum.end(), "\n")
|
||||
print("runtime: level = ", l, ", npages = ", npages, ", j0 = ", j0, "\n")
|
||||
print("runtime: s.searchAddr = ", hex(s.searchAddr), ", i = ", i, "\n")
|
||||
print("runtime: levelShift[level] = ", levelShift[l], ", levelBits[level] = ", levelBits[l], "\n")
|
||||
for j := 0; j < len(entries); j++ {
|
||||
sum := entries[j]
|
||||
print("runtime: summary[", l, "][", i+j, "] = (", sum.start(), ", ", sum.max(), ", ", sum.end(), ")\n")
|
||||
}
|
||||
throw("bad summary data")
|
||||
}
|
||||
|
||||
// Since we've gotten to this point, that means we haven't found a
|
||||
// sufficiently-sized free region straddling some boundary (chunk or larger).
|
||||
// This means the last summary we inspected must have had a large enough "max"
|
||||
// value, so look inside the chunk to find a suitable run.
|
||||
//
|
||||
// After iterating over all levels, i must contain a chunk index which
|
||||
// is what the final level represents.
|
||||
ci := chunkIdx(i)
|
||||
j, searchIdx := s.chunks[ci].find(npages, 0)
|
||||
if j < 0 {
|
||||
// We couldn't find any space in this chunk despite the summaries telling
|
||||
// us it should be there. There's likely a bug, so dump some state and throw.
|
||||
sum := s.summary[len(s.summary)-1][i]
|
||||
print("runtime: summary[", len(s.summary)-1, "][", i, "] = (", sum.start(), ", ", sum.max(), ", ", sum.end(), ")\n")
|
||||
print("runtime: npages = ", npages, "\n")
|
||||
throw("bad summary data")
|
||||
}
|
||||
|
||||
// Compute the address at which the free space starts.
|
||||
addr := chunkBase(ci) + uintptr(j)*pageSize
|
||||
|
||||
// Since we actually searched the chunk, we may have
|
||||
// found an even narrower free window.
|
||||
searchAddr := chunkBase(ci) + uintptr(searchIdx)*pageSize
|
||||
foundFree(searchAddr+arenaBaseOffset, chunkBase(ci+1)-searchAddr)
|
||||
return addr, firstFree.base - arenaBaseOffset
|
||||
}
|
||||
|
||||
// alloc allocates npages worth of memory from the page heap, returning the base
|
||||
// address for the allocation.
|
||||
//
|
||||
// Returns 0 on failure.
|
||||
//
|
||||
// s.mheapLock must be held.
|
||||
func (s *pageAlloc) alloc(npages uintptr) uintptr {
|
||||
// If the searchAddr refers to a region which has a higher address than
|
||||
// any known chunk, then we know we're out of memory.
|
||||
if chunkIndex(s.searchAddr) >= s.end {
|
||||
return 0
|
||||
}
|
||||
|
||||
// If npages has a chance of fitting in the chunk where the searchAddr is,
|
||||
// search it directly.
|
||||
var addr, searchAddr uintptr
|
||||
if pallocChunkPages-chunkPageIndex(s.searchAddr) >= uint(npages) {
|
||||
// npages is guaranteed to be no greater than pallocChunkPages here.
|
||||
i := chunkIndex(s.searchAddr)
|
||||
if max := s.summary[len(s.summary)-1][i].max(); max >= uint(npages) {
|
||||
j, searchIdx := s.chunks[i].find(npages, chunkPageIndex(s.searchAddr))
|
||||
if j < 0 {
|
||||
print("runtime: max = ", max, ", npages = ", npages, "\n")
|
||||
print("runtime: searchIdx = ", chunkPageIndex(s.searchAddr), ", s.searchAddr = ", hex(s.searchAddr), "\n")
|
||||
throw("bad summary data")
|
||||
}
|
||||
addr = chunkBase(i) + uintptr(j)*pageSize
|
||||
searchAddr = chunkBase(i) + uintptr(searchIdx)*pageSize
|
||||
goto Found
|
||||
}
|
||||
}
|
||||
// We failed to use a searchAddr for one reason or another, so try
|
||||
// the slow path.
|
||||
addr, searchAddr = s.find(npages)
|
||||
if addr == 0 {
|
||||
if npages == 1 {
|
||||
// We failed to find a single free page, the smallest unit
|
||||
// of allocation. This means we know the heap is completely
|
||||
// exhausted. Otherwise, the heap still might have free
|
||||
// space in it, just not enough contiguous space to
|
||||
// accommodate npages.
|
||||
s.searchAddr = maxSearchAddr
|
||||
}
|
||||
return 0
|
||||
}
|
||||
Found:
|
||||
// Go ahead and actually mark the bits now that we have an address.
|
||||
s.allocRange(addr, npages)
|
||||
|
||||
// If we found a higher (linearized) searchAddr, we know that all the
|
||||
// heap memory before that searchAddr in a linear address space is
|
||||
// allocated, so bump s.searchAddr up to the new one.
|
||||
if s.compareSearchAddrTo(searchAddr) > 0 {
|
||||
s.searchAddr = searchAddr
|
||||
}
|
||||
return addr
|
||||
}
|
||||
|
||||
// free returns npages worth of memory starting at base back to the page heap.
|
||||
//
|
||||
// s.mheapLock must be held.
|
||||
func (s *pageAlloc) free(base, npages uintptr) {
|
||||
// If we're freeing pages below the (linearized) s.searchAddr, update searchAddr.
|
||||
if s.compareSearchAddrTo(base) < 0 {
|
||||
s.searchAddr = base
|
||||
}
|
||||
if npages == 1 {
|
||||
// Fast path: we're clearing a single bit, and we know exactly
|
||||
// where it is, so mark it directly.
|
||||
s.chunks[chunkIndex(base)].free1(chunkPageIndex(base))
|
||||
} else {
|
||||
// Slow path: we're clearing more bits so we may need to iterate.
|
||||
limit := base + npages*pageSize - 1
|
||||
sc, ec := chunkIndex(base), chunkIndex(limit)
|
||||
si, ei := chunkPageIndex(base), chunkPageIndex(limit)
|
||||
|
||||
if sc == ec {
|
||||
// The range doesn't cross any chunk boundaries.
|
||||
s.chunks[sc].free(si, ei+1-si)
|
||||
} else {
|
||||
// The range crosses at least one chunk boundary.
|
||||
s.chunks[sc].free(si, pallocChunkPages-si)
|
||||
for c := sc + 1; c < ec; c++ {
|
||||
s.chunks[c].freeAll()
|
||||
}
|
||||
s.chunks[ec].free(0, ei+1)
|
||||
}
|
||||
}
|
||||
s.update(base, npages, true, false)
|
||||
}
|
||||
|
||||
const (
|
||||
pallocSumBytes = unsafe.Sizeof(pallocSum(0))
|
||||
|
||||
// maxPackedValue is the maximum value that any of the three fields in
|
||||
// the pallocSum may take on.
|
||||
maxPackedValue = 1 << logMaxPackedValue
|
||||
logMaxPackedValue = logPallocChunkPages + (summaryLevels-1)*summaryLevelBits
|
||||
|
||||
freeChunkSum = pallocSum(uint64(pallocChunkPages) |
|
||||
uint64(pallocChunkPages<<logMaxPackedValue) |
|
||||
uint64(pallocChunkPages<<(2*logMaxPackedValue)))
|
||||
)
|
||||
|
||||
// pallocSum is a packed summary type which packs three numbers: start, max,
|
||||
|
|
@ -128,3 +846,47 @@ func (p pallocSum) unpack() (uint, uint, uint) {
|
|||
uint((uint64(p) >> logMaxPackedValue) & (maxPackedValue - 1)),
|
||||
uint((uint64(p) >> (2 * logMaxPackedValue)) & (maxPackedValue - 1))
|
||||
}
|
||||
|
||||
// mergeSummaries merges consecutive summaries which may each represent at
|
||||
// most 1 << logMaxPagesPerSum pages each together into one.
|
||||
func mergeSummaries(sums []pallocSum, logMaxPagesPerSum uint) pallocSum {
|
||||
// Merge the summaries in sums into one.
|
||||
//
|
||||
// We do this by keeping a running summary representing the merged
|
||||
// summaries of sums[:i] in start, max, and end.
|
||||
start, max, end := sums[0].unpack()
|
||||
for i := 1; i < len(sums); i++ {
|
||||
// Merge in sums[i].
|
||||
si, mi, ei := sums[i].unpack()
|
||||
|
||||
// Merge in sums[i].start only if the running summary is
|
||||
// completely free, otherwise this summary's start
|
||||
// plays no role in the combined sum.
|
||||
if start == uint(i)<<logMaxPagesPerSum {
|
||||
start += si
|
||||
}
|
||||
|
||||
// Recompute the max value of the running sum by looking
|
||||
// across the boundary between the running sum and sums[i]
|
||||
// and at the max sums[i], taking the greatest of those two
|
||||
// and the max of the running sum.
|
||||
if end+si > max {
|
||||
max = end + si
|
||||
}
|
||||
if mi > max {
|
||||
max = mi
|
||||
}
|
||||
|
||||
// Merge in end by checking if this new summary is totally
|
||||
// free. If it is, then we want to extend the running sum's
|
||||
// end by the new summary. If not, then we have some alloc'd
|
||||
// pages in there and we just want to take the end value in
|
||||
// sums[i].
|
||||
if ei == 1<<logMaxPagesPerSum {
|
||||
end += 1 << logMaxPagesPerSum
|
||||
} else {
|
||||
end = ei
|
||||
}
|
||||
}
|
||||
return packPallocSum(start, max, end)
|
||||
}
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue