diff options
author | Jamie Liu <jamieliu@google.com> | 2019-03-14 08:11:36 -0700 |
---|---|---|
committer | Shentubot <shentubot@google.com> | 2019-03-14 08:12:48 -0700 |
commit | 8f4634997bd97810a85a70b71f000378d9db2e55 (patch) | |
tree | 903096f91ee8f201fa622296e0f04cf7c7cd9013 /pkg/sentry/pgalloc | |
parent | fb9919881c7dc98eaf97cad2a70d187bd78f1566 (diff) |
Decouple filemem from platform and move it to pgalloc.MemoryFile.
This is in preparation for improved page cache reclaim, which requires
greater integration between the page cache and page allocator.
PiperOrigin-RevId: 238444706
Change-Id: Id24141b3678d96c7d7dc24baddd9be555bffafe4
Diffstat (limited to 'pkg/sentry/pgalloc')
-rw-r--r-- | pkg/sentry/pgalloc/BUILD | 57 | ||||
-rw-r--r-- | pkg/sentry/pgalloc/context.go | 48 | ||||
-rw-r--r-- | pkg/sentry/pgalloc/pgalloc.go | 922 | ||||
-rw-r--r-- | pkg/sentry/pgalloc/pgalloc_test.go | 168 | ||||
-rw-r--r-- | pkg/sentry/pgalloc/pgalloc_unsafe.go | 40 | ||||
-rw-r--r-- | pkg/sentry/pgalloc/save_restore.go | 205 |
6 files changed, 1440 insertions, 0 deletions
diff --git a/pkg/sentry/pgalloc/BUILD b/pkg/sentry/pgalloc/BUILD new file mode 100644 index 000000000..7efa55c20 --- /dev/null +++ b/pkg/sentry/pgalloc/BUILD @@ -0,0 +1,57 @@ +package(licenses = ["notice"]) + +load("//tools/go_generics:defs.bzl", "go_template_instance") +load("//tools/go_stateify:defs.bzl", "go_library", "go_test") + +go_template_instance( + name = "usage_set", + out = "usage_set.go", + consts = { + "minDegree": "10", + }, + imports = { + "platform": "gvisor.googlesource.com/gvisor/pkg/sentry/platform", + }, + package = "pgalloc", + prefix = "usage", + template = "//pkg/segment:generic_set", + types = { + "Key": "uint64", + "Range": "platform.FileRange", + "Value": "usageInfo", + "Functions": "usageSetFunctions", + }, +) + +go_library( + name = "pgalloc", + srcs = [ + "context.go", + "pgalloc.go", + "pgalloc_unsafe.go", + "save_restore.go", + "usage_set.go", + ], + importpath = "gvisor.googlesource.com/gvisor/pkg/sentry/pgalloc", + visibility = ["//pkg/sentry:internal"], + deps = [ + "//pkg/log", + "//pkg/sentry/arch", + "//pkg/sentry/context", + "//pkg/sentry/memutil", + "//pkg/sentry/platform", + "//pkg/sentry/safemem", + "//pkg/sentry/usage", + "//pkg/sentry/usermem", + "//pkg/state", + "//pkg/syserror", + ], +) + +go_test( + name = "pgalloc_test", + size = "small", + srcs = ["pgalloc_test.go"], + embed = [":pgalloc"], + deps = ["//pkg/sentry/usermem"], +) diff --git a/pkg/sentry/pgalloc/context.go b/pkg/sentry/pgalloc/context.go new file mode 100644 index 000000000..adc97e78f --- /dev/null +++ b/pkg/sentry/pgalloc/context.go @@ -0,0 +1,48 @@ +// Copyright 2019 Google Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package pgalloc + +import ( + "gvisor.googlesource.com/gvisor/pkg/sentry/context" +) + +// contextID is this package's type for context.Context.Value keys. +type contextID int + +const ( + // CtxMemoryFile is a Context.Value key for a MemoryFile. + CtxMemoryFile contextID = iota + + // CtxMemoryFileProvider is a Context.Value key for a MemoryFileProvider. + CtxMemoryFileProvider +) + +// MemoryFileFromContext returns the MemoryFile used by ctx, or nil if no such +// MemoryFile exists. +func MemoryFileFromContext(ctx context.Context) *MemoryFile { + if v := ctx.Value(CtxMemoryFile); v != nil { + return v.(*MemoryFile) + } + return nil +} + +// MemoryFileProviderFromContext returns the MemoryFileProvider used by ctx, or nil if no such +// MemoryFileProvider exists. +func MemoryFileProviderFromContext(ctx context.Context) MemoryFileProvider { + if v := ctx.Value(CtxMemoryFileProvider); v != nil { + return v.(MemoryFileProvider) + } + return nil +} diff --git a/pkg/sentry/pgalloc/pgalloc.go b/pkg/sentry/pgalloc/pgalloc.go new file mode 100644 index 000000000..0754e608f --- /dev/null +++ b/pkg/sentry/pgalloc/pgalloc.go @@ -0,0 +1,922 @@ +// Copyright 2018 Google LLC +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// Package pgalloc contains the page allocator subsystem, which manages memory +// that may be mapped into application address spaces. +// +// Lock order: +// +// pgalloc.MemoryFile.mu +// pgalloc.MemoryFile.mappingsMu +package pgalloc + +import ( + "fmt" + "math" + "os" + "sync" + "sync/atomic" + "syscall" + "time" + + "gvisor.googlesource.com/gvisor/pkg/log" + "gvisor.googlesource.com/gvisor/pkg/sentry/platform" + "gvisor.googlesource.com/gvisor/pkg/sentry/safemem" + "gvisor.googlesource.com/gvisor/pkg/sentry/usage" + "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" + "gvisor.googlesource.com/gvisor/pkg/syserror" +) + +// MemoryFile is a platform.File whose pages may be allocated to arbitrary +// users. +type MemoryFile struct { + // MemoryFile owns a single backing file, which is modeled as follows: + // + // Each page in the file can be committed or uncommitted. A page is + // committed if the host kernel is spending resources to store its contents + // and uncommitted otherwise. This definition includes pages that the host + // kernel has swapped; this is intentional, to ensure that accounting does + // not change even if host kernel swapping behavior changes, and that + // memory used by pseudo-swap mechanisms like zswap is still accounted. + // + // The initial contents of uncommitted pages are implicitly zero bytes. A + // read or write to the contents of an uncommitted page causes it to be + // committed. This is the only event that can cause a uncommitted page to + // be committed. + // + // fallocate(FALLOC_FL_PUNCH_HOLE) (MemoryFile.Decommit) causes committed + // pages to be uncommitted. This is the only event that can cause a + // committed page to be uncommitted. + // + // Memory accounting is based on identifying the set of committed pages. + // Since we do not have direct access to the MMU, tracking reads and writes + // to uncommitted pages to detect commitment would introduce additional + // page faults, which would be prohibitively expensive. Instead, we query + // the host kernel to determine which pages are committed. + + // file is the backing file. The file pointer is immutable. + file *os.File + + mu sync.Mutex + + // usage maps each page in the file to metadata for that page. Pages for + // which no segment exists in usage are both unallocated (not in use) and + // uncommitted. + // + // Since usage stores usageInfo objects by value, clients should usually + // use usageIterator.ValuePtr() instead of usageIterator.Value() to get a + // pointer to the usageInfo rather than a copy. + // + // usage must be kept maximally merged (that is, there should never be two + // adjacent segments with the same values). At least markReclaimed depends + // on this property. + // + // usage is protected by mu. + usage usageSet + + // The UpdateUsage function scans all segments with knownCommitted set + // to false, sees which pages are committed and creates corresponding + // segments with knownCommitted set to true. + // + // In order to avoid unnecessary scans, usageExpected tracks the total + // file blocks expected. This is used to elide the scan when this + // matches the underlying file blocks. + // + // To track swapped pages, usageSwapped tracks the discrepency between + // what is observed in core and what is reported by the file. When + // usageSwapped is non-zero, a sweep will be performed at least every + // second. The start of the last sweep is recorded in usageLast. + // + // All usage attributes are all protected by mu. + usageExpected uint64 + usageSwapped uint64 + usageLast time.Time + + // minUnallocatedPage is the minimum page that may be unallocated. + // i.e., there are no unallocated pages below minUnallocatedPage. + // + // minUnallocatedPage is protected by mu. + minUnallocatedPage uint64 + + // fileSize is the size of the backing memory file in bytes. fileSize is + // always a power-of-two multiple of chunkSize. + // + // fileSize is protected by mu. + fileSize int64 + + // destroyed is set by Destroy to instruct the reclaimer goroutine to + // release resources and exit. destroyed is protected by mu. + destroyed bool + + // reclaimable is true if usage may contain reclaimable pages. reclaimable + // is protected by mu. + reclaimable bool + + // minReclaimablePage is the minimum page that may be reclaimable. + // i.e., all reclaimable pages are >= minReclaimablePage. + // + // minReclaimablePage is protected by mu. + minReclaimablePage uint64 + + // reclaimCond is signaled (with mu locked) when reclaimable or destroyed + // transitions from false to true. + reclaimCond sync.Cond + + // Pages from the backing file are mapped into the local address space on + // the granularity of large pieces called chunks. mappings is a []uintptr + // that stores, for each chunk, the start address of a mapping of that + // chunk in the current process' address space, or 0 if no such mapping + // exists. Once a chunk is mapped, it is never remapped or unmapped until + // the MemoryFile is destroyed. + // + // Mutating the mappings slice or its contents requires both holding + // mappingsMu and using atomic memory operations. (The slice is mutated + // whenever the file is expanded. Per the above, the only permitted + // mutation of the slice's contents is the assignment of a mapping to a + // chunk that was previously unmapped.) Reading the slice or its contents + // only requires *either* holding mappingsMu or using atomic memory + // operations. This allows MemoryFile.MapInternal to avoid locking in the + // common case where chunk mappings already exist. + mappingsMu sync.Mutex + mappings atomic.Value +} + +// usage tracks usage information. +// +// +stateify savable +type usageInfo struct { + // kind is the usage kind. + kind usage.MemoryKind + + // knownCommitted is true if the tracked region is definitely committed. + // (If it is false, the tracked region may or may not be committed.) + knownCommitted bool + + refs uint64 +} + +const ( + chunkShift = 24 + chunkSize = 1 << chunkShift // 16 MB + chunkMask = chunkSize - 1 + + initialSize = chunkSize + + // maxPage is the highest 64-bit page. + maxPage = math.MaxUint64 &^ (usermem.PageSize - 1) +) + +// NewMemoryFile creates a MemoryFile backed by the given file. If +// NewMemoryFile succeeds, ownership of file is transferred to the returned +// MemoryFile. +func NewMemoryFile(file *os.File) (*MemoryFile, error) { + // Truncate the file to 0 bytes first to ensure that it's empty. + if err := file.Truncate(0); err != nil { + return nil, err + } + if err := file.Truncate(initialSize); err != nil { + return nil, err + } + f := &MemoryFile{ + fileSize: initialSize, + file: file, + // No pages are reclaimable. DecRef will always be able to + // decrease minReclaimablePage from this point. + minReclaimablePage: maxPage, + } + f.reclaimCond.L = &f.mu + f.mappings.Store(make([]uintptr, initialSize/chunkSize)) + go f.runReclaim() // S/R-SAFE: f.mu + + // The Linux kernel contains an optional feature called "Integrity + // Measurement Architecture" (IMA). If IMA is enabled, it will checksum + // binaries the first time they are mapped PROT_EXEC. This is bad news for + // executable pages mapped from our backing file, which can grow to + // terabytes in (sparse) size. If IMA attempts to checksum a file that + // large, it will allocate all of the sparse pages and quickly exhaust all + // memory. + // + // Work around IMA by immediately creating a temporary PROT_EXEC mapping, + // while the backing file is still small. IMA will ignore any future + // mappings. + m, _, errno := syscall.Syscall6( + syscall.SYS_MMAP, + 0, + usermem.PageSize, + syscall.PROT_EXEC, + syscall.MAP_SHARED, + file.Fd(), + 0) + if errno != 0 { + // This isn't fatal (IMA may not even be in use). Log the error, but + // don't return it. + log.Warningf("Failed to pre-map MemoryFile PROT_EXEC: %v", errno) + } else { + if _, _, errno := syscall.Syscall( + syscall.SYS_MUNMAP, + m, + usermem.PageSize, + 0); errno != 0 { + panic(fmt.Sprintf("failed to unmap PROT_EXEC MemoryFile mapping: %v", errno)) + } + } + + return f, nil +} + +// Destroy releases all resources used by f. +// +// Preconditions: All pages allocated by f have been freed. +// +// Postconditions: None of f's methods may be called after Destroy. +func (f *MemoryFile) Destroy() { + f.mu.Lock() + defer f.mu.Unlock() + f.destroyed = true + f.reclaimCond.Signal() +} + +// Allocate returns a range of initially-zeroed pages of the given length with +// the given accounting kind and a single reference held by the caller. When +// the last reference on an allocated page is released, ownership of the page +// is returned to the MemoryFile, allowing it to be returned by a future call +// to Allocate. +// +// Preconditions: length must be page-aligned and non-zero. +func (f *MemoryFile) Allocate(length uint64, kind usage.MemoryKind) (platform.FileRange, error) { + if length == 0 || length%usermem.PageSize != 0 { + panic(fmt.Sprintf("invalid allocation length: %#x", length)) + } + + f.mu.Lock() + defer f.mu.Unlock() + + // Align hugepage-and-larger allocations on hugepage boundaries to try + // to take advantage of hugetmpfs. + alignment := uint64(usermem.PageSize) + if length >= usermem.HugePageSize { + alignment = usermem.HugePageSize + } + + start, minUnallocatedPage := findUnallocatedRange(&f.usage, f.minUnallocatedPage, length, alignment) + end := start + length + // File offsets are int64s. Since length must be strictly positive, end + // cannot legitimately be 0. + if end < start || int64(end) <= 0 { + return platform.FileRange{}, syserror.ENOMEM + } + + // Expand the file if needed. Double the file size on each expansion; + // uncommitted pages have effectively no cost. + fileSize := f.fileSize + for int64(end) > fileSize { + if fileSize >= 2*fileSize { + // fileSize overflow. + return platform.FileRange{}, syserror.ENOMEM + } + fileSize *= 2 + } + if fileSize > f.fileSize { + if err := f.file.Truncate(fileSize); err != nil { + return platform.FileRange{}, err + } + f.fileSize = fileSize + f.mappingsMu.Lock() + oldMappings := f.mappings.Load().([]uintptr) + newMappings := make([]uintptr, fileSize>>chunkShift) + copy(newMappings, oldMappings) + f.mappings.Store(newMappings) + f.mappingsMu.Unlock() + } + + // Mark selected pages as in use. + fr := platform.FileRange{start, end} + if !f.usage.Add(fr, usageInfo{ + kind: kind, + refs: 1, + }) { + panic(fmt.Sprintf("allocating %v: failed to insert into usage set:\n%v", fr, &f.usage)) + } + + if minUnallocatedPage < start { + f.minUnallocatedPage = minUnallocatedPage + } else { + // start was the first unallocated page. The next must be + // somewhere beyond end. + f.minUnallocatedPage = end + } + + return fr, nil +} + +// findUnallocatedRange returns the first unallocated page in usage of the +// specified length and alignment beginning at page start and the first single +// unallocated page. +func findUnallocatedRange(usage *usageSet, start, length, alignment uint64) (uint64, uint64) { + // Only searched until the first page is found. + firstPage := start + foundFirstPage := false + alignMask := alignment - 1 + for seg := usage.LowerBoundSegment(start); seg.Ok(); seg = seg.NextSegment() { + r := seg.Range() + + if !foundFirstPage && r.Start > firstPage { + foundFirstPage = true + } + + if start >= r.End { + // start was rounded up to an alignment boundary from the end + // of a previous segment and is now beyond r.End. + continue + } + // This segment represents allocated or reclaimable pages; only the + // range from start to the segment's beginning is allocatable, and the + // next allocatable range begins after the segment. + if r.Start > start && r.Start-start >= length { + break + } + start = (r.End + alignMask) &^ alignMask + if !foundFirstPage { + firstPage = r.End + } + } + return start, firstPage +} + +// AllocateAndFill allocates memory of the given kind and fills it by calling +// r.ReadToBlocks() repeatedly until either length bytes are read or a non-nil +// error is returned. It returns the memory filled by r, truncated down to the +// nearest page. If this is shorter than length bytes due to an error returned +// by r.ReadToBlocks(), it returns that error. +// +// Preconditions: length > 0. length must be page-aligned. +func (f *MemoryFile) AllocateAndFill(length uint64, kind usage.MemoryKind, r safemem.Reader) (platform.FileRange, error) { + fr, err := f.Allocate(length, kind) + if err != nil { + return platform.FileRange{}, err + } + dsts, err := f.MapInternal(fr, usermem.Write) + if err != nil { + f.DecRef(fr) + return platform.FileRange{}, err + } + n, err := safemem.ReadFullToBlocks(r, dsts) + un := uint64(usermem.Addr(n).RoundDown()) + if un < length { + // Free unused memory and update fr to contain only the memory that is + // still allocated. + f.DecRef(platform.FileRange{fr.Start + un, fr.End}) + fr.End = fr.Start + un + } + return fr, err +} + +// fallocate(2) modes, defined in Linux's include/uapi/linux/falloc.h. +const ( + _FALLOC_FL_KEEP_SIZE = 1 + _FALLOC_FL_PUNCH_HOLE = 2 +) + +// Decommit releases resources associated with maintaining the contents of the +// given pages. If Decommit succeeds, future accesses of the decommitted pages +// will read zeroes. +// +// Preconditions: fr.Length() > 0. +func (f *MemoryFile) Decommit(fr platform.FileRange) error { + if !fr.WellFormed() || fr.Length() == 0 || fr.Start%usermem.PageSize != 0 || fr.End%usermem.PageSize != 0 { + panic(fmt.Sprintf("invalid range: %v", fr)) + } + + // "After a successful call, subsequent reads from this range will + // return zeroes. The FALLOC_FL_PUNCH_HOLE flag must be ORed with + // FALLOC_FL_KEEP_SIZE in mode ..." - fallocate(2) + err := syscall.Fallocate( + int(f.file.Fd()), + _FALLOC_FL_PUNCH_HOLE|_FALLOC_FL_KEEP_SIZE, + int64(fr.Start), + int64(fr.Length())) + if err != nil { + return err + } + f.markDecommitted(fr) + return nil +} + +func (f *MemoryFile) markDecommitted(fr platform.FileRange) { + f.mu.Lock() + defer f.mu.Unlock() + // Since we're changing the knownCommitted attribute, we need to merge + // across the entire range to ensure that the usage tree is minimal. + gap := f.usage.ApplyContiguous(fr, func(seg usageIterator) { + val := seg.ValuePtr() + if val.knownCommitted { + // Drop the usageExpected appropriately. + amount := seg.Range().Length() + usage.MemoryAccounting.Dec(amount, val.kind) + f.usageExpected -= amount + val.knownCommitted = false + } + }) + if gap.Ok() { + panic(fmt.Sprintf("Decommit(%v): attempted to decommit unallocated pages %v:\n%v", fr, gap.Range(), &f.usage)) + } + f.usage.MergeRange(fr) +} + +// runReclaim implements the reclaimer goroutine, which continuously decommits +// reclaimable pages in order to reduce memory usage and make them available +// for allocation. +func (f *MemoryFile) runReclaim() { + for { + fr, ok := f.findReclaimable() + if !ok { + break + } + + if err := f.Decommit(fr); err != nil { + log.Warningf("Reclaim failed to decommit %v: %v", fr, err) + // Zero the pages manually. This won't reduce memory usage, but at + // least ensures that the pages will be zero when reallocated. + f.forEachMappingSlice(fr, func(bs []byte) { + for i := range bs { + bs[i] = 0 + } + }) + // Pretend the pages were decommitted even though they weren't, + // since the memory accounting implementation has no idea how to + // deal with this. + f.markDecommitted(fr) + } + f.markReclaimed(fr) + } + // We only get here if findReclaimable finds f.destroyed set and returns + // false. + f.mu.Lock() + defer f.mu.Unlock() + if !f.destroyed { + panic("findReclaimable broke out of reclaim loop, but destroyed is no longer set") + } + f.file.Close() + // Ensure that any attempts to use f.file.Fd() fail instead of getting a fd + // that has possibly been reassigned. + f.file = nil + mappings := f.mappings.Load().([]uintptr) + for i, m := range mappings { + if m != 0 { + _, _, errno := syscall.Syscall(syscall.SYS_MUNMAP, m, chunkSize, 0) + if errno != 0 { + log.Warningf("Failed to unmap mapping %#x for MemoryFile chunk %d: %v", m, i, errno) + } + } + } + // Similarly, invalidate f.mappings. (atomic.Value.Store(nil) panics.) + f.mappings.Store([]uintptr{}) +} + +func (f *MemoryFile) findReclaimable() (platform.FileRange, bool) { + f.mu.Lock() + defer f.mu.Unlock() + for { + for { + if f.destroyed { + return platform.FileRange{}, false + } + if f.reclaimable { + break + } + f.reclaimCond.Wait() + } + // Allocate returns the first usable range in offset order and is + // currently a linear scan, so reclaiming from the beginning of the + // file minimizes the expected latency of Allocate. + for seg := f.usage.LowerBoundSegment(f.minReclaimablePage); seg.Ok(); seg = seg.NextSegment() { + if seg.ValuePtr().refs == 0 { + f.minReclaimablePage = seg.End() + return seg.Range(), true + } + } + // No pages are reclaimable. + f.reclaimable = false + f.minReclaimablePage = maxPage + } +} + +func (f *MemoryFile) markReclaimed(fr platform.FileRange) { + f.mu.Lock() + defer f.mu.Unlock() + seg := f.usage.FindSegment(fr.Start) + // All of fr should be mapped to a single uncommitted reclaimable segment + // accounted to System. + if !seg.Ok() { + panic(fmt.Sprintf("reclaimed pages %v include unreferenced pages:\n%v", fr, &f.usage)) + } + if !seg.Range().IsSupersetOf(fr) { + panic(fmt.Sprintf("reclaimed pages %v are not entirely contained in segment %v with state %v:\n%v", fr, seg.Range(), seg.Value(), &f.usage)) + } + if got, want := seg.Value(), (usageInfo{ + kind: usage.System, + knownCommitted: false, + refs: 0, + }); got != want { + panic(fmt.Sprintf("reclaimed pages %v in segment %v has incorrect state %v, wanted %v:\n%v", fr, seg.Range(), got, want, &f.usage)) + } + // Deallocate reclaimed pages. Even though all of seg is reclaimable, the + // caller of markReclaimed may not have decommitted it, so we can only mark + // fr as reclaimed. + f.usage.Remove(f.usage.Isolate(seg, fr)) + if fr.Start < f.minUnallocatedPage { + // We've deallocated at least one lower page. + f.minUnallocatedPage = fr.Start + } +} + +// IncRef implements platform.File.IncRef. +func (f *MemoryFile) IncRef(fr platform.FileRange) { + if !fr.WellFormed() || fr.Length() == 0 || fr.Start%usermem.PageSize != 0 || fr.End%usermem.PageSize != 0 { + panic(fmt.Sprintf("invalid range: %v", fr)) + } + + f.mu.Lock() + defer f.mu.Unlock() + + gap := f.usage.ApplyContiguous(fr, func(seg usageIterator) { + seg.ValuePtr().refs++ + }) + if gap.Ok() { + panic(fmt.Sprintf("IncRef(%v): attempted to IncRef on unallocated pages %v:\n%v", fr, gap.Range(), &f.usage)) + } + + f.usage.MergeAdjacent(fr) +} + +// DecRef implements platform.File.DecRef. +func (f *MemoryFile) DecRef(fr platform.FileRange) { + if !fr.WellFormed() || fr.Length() == 0 || fr.Start%usermem.PageSize != 0 || fr.End%usermem.PageSize != 0 { + panic(fmt.Sprintf("invalid range: %v", fr)) + } + + var freed bool + + f.mu.Lock() + defer f.mu.Unlock() + + for seg := f.usage.FindSegment(fr.Start); seg.Ok() && seg.Start() < fr.End; seg = seg.NextSegment() { + seg = f.usage.Isolate(seg, fr) + val := seg.ValuePtr() + if val.refs == 0 { + panic(fmt.Sprintf("DecRef(%v): 0 existing references on %v:\n%v", fr, seg.Range(), &f.usage)) + } + val.refs-- + if val.refs == 0 { + freed = true + // Reclassify memory as System, until it's freed by the reclaim + // goroutine. + if val.knownCommitted { + usage.MemoryAccounting.Move(seg.Range().Length(), usage.System, val.kind) + } + val.kind = usage.System + } + } + f.usage.MergeAdjacent(fr) + + if freed { + if fr.Start < f.minReclaimablePage { + // We've freed at least one lower page. + f.minReclaimablePage = fr.Start + } + f.reclaimable = true + f.reclaimCond.Signal() + } +} + +// MapInternal implements platform.File.MapInternal. +func (f *MemoryFile) MapInternal(fr platform.FileRange, at usermem.AccessType) (safemem.BlockSeq, error) { + if !fr.WellFormed() || fr.Length() == 0 { + panic(fmt.Sprintf("invalid range: %v", fr)) + } + if at.Execute { + return safemem.BlockSeq{}, syserror.EACCES + } + + chunks := ((fr.End + chunkMask) >> chunkShift) - (fr.Start >> chunkShift) + if chunks == 1 { + // Avoid an unnecessary slice allocation. + var seq safemem.BlockSeq + err := f.forEachMappingSlice(fr, func(bs []byte) { + seq = safemem.BlockSeqOf(safemem.BlockFromSafeSlice(bs)) + }) + return seq, err + } + blocks := make([]safemem.Block, 0, chunks) + err := f.forEachMappingSlice(fr, func(bs []byte) { + blocks = append(blocks, safemem.BlockFromSafeSlice(bs)) + }) + return safemem.BlockSeqFromSlice(blocks), err +} + +// forEachMappingSlice invokes fn on a sequence of byte slices that +// collectively map all bytes in fr. +func (f *MemoryFile) forEachMappingSlice(fr platform.FileRange, fn func([]byte)) error { + mappings := f.mappings.Load().([]uintptr) + for chunkStart := fr.Start &^ chunkMask; chunkStart < fr.End; chunkStart += chunkSize { + chunk := int(chunkStart >> chunkShift) + m := atomic.LoadUintptr(&mappings[chunk]) + if m == 0 { + var err error + mappings, m, err = f.getChunkMapping(chunk) + if err != nil { + return err + } + } + startOff := uint64(0) + if chunkStart < fr.Start { + startOff = fr.Start - chunkStart + } + endOff := uint64(chunkSize) + if chunkStart+chunkSize > fr.End { + endOff = fr.End - chunkStart + } + fn(unsafeSlice(m, chunkSize)[startOff:endOff]) + } + return nil +} + +func (f *MemoryFile) getChunkMapping(chunk int) ([]uintptr, uintptr, error) { + f.mappingsMu.Lock() + defer f.mappingsMu.Unlock() + // Another thread may have replaced f.mappings altogether due to file + // expansion. + mappings := f.mappings.Load().([]uintptr) + // Another thread may have already mapped the chunk. + if m := mappings[chunk]; m != 0 { + return mappings, m, nil + } + m, _, errno := syscall.Syscall6( + syscall.SYS_MMAP, + 0, + chunkSize, + syscall.PROT_READ|syscall.PROT_WRITE, + syscall.MAP_SHARED, + f.file.Fd(), + uintptr(chunk<<chunkShift)) + if errno != 0 { + return nil, 0, errno + } + atomic.StoreUintptr(&mappings[chunk], m) + return mappings, m, nil +} + +// FD implements platform.File.FD. +func (f *MemoryFile) FD() int { + return int(f.file.Fd()) +} + +// UpdateUsage ensures that the memory usage statistics in +// usage.MemoryAccounting are up to date. +func (f *MemoryFile) UpdateUsage() error { + f.mu.Lock() + defer f.mu.Unlock() + + // If the underlying usage matches where the usage tree already + // represents, then we can just avoid the entire scan (we know it's + // accurate). + currentUsage, err := f.TotalUsage() + if err != nil { + return err + } + if currentUsage == f.usageExpected && f.usageSwapped == 0 { + log.Debugf("UpdateUsage: skipped with usageSwapped=0.") + return nil + } + // If the current usage matches the expected but there's swap + // accounting, then ensure a scan takes place at least every second + // (when requested). + if currentUsage == f.usageExpected+f.usageSwapped && time.Now().Before(f.usageLast.Add(time.Second)) { + log.Debugf("UpdateUsage: skipped with usageSwapped!=0.") + return nil + } + + f.usageLast = time.Now() + err = f.updateUsageLocked(currentUsage, mincore) + log.Debugf("UpdateUsage: currentUsage=%d, usageExpected=%d, usageSwapped=%d.", + currentUsage, f.usageExpected, f.usageSwapped) + log.Debugf("UpdateUsage: took %v.", time.Since(f.usageLast)) + return err +} + +// updateUsageLocked attempts to detect commitment of previous-uncommitted +// pages by invoking checkCommitted, which is a function that, for each page i +// in bs, sets committed[i] to 1 if the page is committed and 0 otherwise. +// +// Precondition: f.mu must be held. +func (f *MemoryFile) updateUsageLocked(currentUsage uint64, checkCommitted func(bs []byte, committed []byte) error) error { + // Track if anything changed to elide the merge. In the common case, we + // expect all segments to be committed and no merge to occur. + changedAny := false + defer func() { + if changedAny { + f.usage.MergeAll() + } + + // Adjust the swap usage to reflect reality. + if f.usageExpected < currentUsage { + // Since no pages may be marked decommitted while we hold mu, we + // know that usage may have only increased since we got the last + // current usage. Therefore, if usageExpected is still short of + // currentUsage, we must assume that the difference is in pages + // that have been swapped. + newUsageSwapped := currentUsage - f.usageExpected + if f.usageSwapped < newUsageSwapped { + usage.MemoryAccounting.Inc(newUsageSwapped-f.usageSwapped, usage.System) + } else { + usage.MemoryAccounting.Dec(f.usageSwapped-newUsageSwapped, usage.System) + } + f.usageSwapped = newUsageSwapped + } else if f.usageSwapped != 0 { + // We have more usage accounted for than the file itself. + // That's fine, we probably caught a race where pages were + // being committed while the above loop was running. Just + // report the higher number that we found and ignore swap. + usage.MemoryAccounting.Dec(f.usageSwapped, usage.System) + f.usageSwapped = 0 + } + }() + + // Reused mincore buffer, will generally be <= 4096 bytes. + var buf []byte + + // Iterate over all usage data. There will only be usage segments + // present when there is an associated reference. + for seg := f.usage.FirstSegment(); seg.Ok(); seg = seg.NextSegment() { + val := seg.Value() + + // Already known to be committed; ignore. + if val.knownCommitted { + continue + } + + // Assume that reclaimable pages (that aren't already known to be + // committed) are not committed. This isn't necessarily true, even + // after the reclaimer does Decommit(), because the kernel may + // subsequently back the hugepage-sized region containing the + // decommitted page with a hugepage. However, it's consistent with our + // treatment of unallocated pages, which have the same property. + if val.refs == 0 { + continue + } + + // Get the range for this segment. As we touch slices, the + // Start value will be walked along. + r := seg.Range() + + var checkErr error + err := f.forEachMappingSlice(r, func(s []byte) { + if checkErr != nil { + return + } + + // Ensure that we have sufficient buffer for the call + // (one byte per page). The length of each slice must + // be page-aligned. + bufLen := len(s) / usermem.PageSize + if len(buf) < bufLen { + buf = make([]byte, bufLen) + } + + // Query for new pages in core. + if err := checkCommitted(s, buf); err != nil { + checkErr = err + return + } + + // Scan each page and switch out segments. + populatedRun := false + populatedRunStart := 0 + for i := 0; i <= bufLen; i++ { + // We run past the end of the slice here to + // simplify the logic and only set populated if + // we're still looking at elements. + populated := false + if i < bufLen { + populated = buf[i]&0x1 != 0 + } + + switch { + case populated == populatedRun: + // Keep the run going. + continue + case populated && !populatedRun: + // Begin the run. + populatedRun = true + populatedRunStart = i + // Keep going. + continue + case !populated && populatedRun: + // Finish the run by changing this segment. + runRange := platform.FileRange{ + Start: r.Start + uint64(populatedRunStart*usermem.PageSize), + End: r.Start + uint64(i*usermem.PageSize), + } + seg = f.usage.Isolate(seg, runRange) + seg.ValuePtr().knownCommitted = true + // Advance the segment only if we still + // have work to do in the context of + // the original segment from the for + // loop. Otherwise, the for loop itself + // will advance the segment + // appropriately. + if runRange.End != r.End { + seg = seg.NextSegment() + } + amount := runRange.Length() + usage.MemoryAccounting.Inc(amount, val.kind) + f.usageExpected += amount + changedAny = true + populatedRun = false + } + } + + // Advance r.Start. + r.Start += uint64(len(s)) + }) + if checkErr != nil { + return checkErr + } + if err != nil { + return err + } + } + + return nil +} + +// TotalUsage returns an aggregate usage for all memory statistics except +// Mapped (which is external to MemoryFile). This is generally much cheaper +// than UpdateUsage, but will not provide a fine-grained breakdown. +func (f *MemoryFile) TotalUsage() (uint64, error) { + // Stat the underlying file to discover the underlying usage. stat(2) + // always reports the allocated block count in units of 512 bytes. This + // includes pages in the page cache and swapped pages. + var stat syscall.Stat_t + if err := syscall.Fstat(int(f.file.Fd()), &stat); err != nil { + return 0, err + } + return uint64(stat.Blocks * 512), nil +} + +// TotalSize returns the current size of the backing file in bytes, which is an +// upper bound on the amount of memory that can currently be allocated from the +// MemoryFile. The value returned by TotalSize is permitted to change. +func (f *MemoryFile) TotalSize() uint64 { + f.mu.Lock() + defer f.mu.Unlock() + return uint64(f.fileSize) +} + +// File returns the backing file. +func (f *MemoryFile) File() *os.File { + return f.file +} + +// String implements fmt.Stringer.String. +// +// Note that because f.String locks f.mu, calling f.String internally +// (including indirectly through the fmt package) risks recursive locking. +// Within the pgalloc package, use f.usage directly instead. +func (f *MemoryFile) String() string { + f.mu.Lock() + defer f.mu.Unlock() + return f.usage.String() +} + +type usageSetFunctions struct{} + +func (usageSetFunctions) MinKey() uint64 { + return 0 +} + +func (usageSetFunctions) MaxKey() uint64 { + return math.MaxUint64 +} + +func (usageSetFunctions) ClearValue(val *usageInfo) { +} + +func (usageSetFunctions) Merge(_ platform.FileRange, val1 usageInfo, _ platform.FileRange, val2 usageInfo) (usageInfo, bool) { + return val1, val1 == val2 +} + +func (usageSetFunctions) Split(_ platform.FileRange, val usageInfo, _ uint64) (usageInfo, usageInfo) { + return val, val +} diff --git a/pkg/sentry/pgalloc/pgalloc_test.go b/pkg/sentry/pgalloc/pgalloc_test.go new file mode 100644 index 000000000..726623c1a --- /dev/null +++ b/pkg/sentry/pgalloc/pgalloc_test.go @@ -0,0 +1,168 @@ +// Copyright 2018 Google LLC +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package pgalloc + +import ( + "testing" + + "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" +) + +const ( + page = usermem.PageSize + hugepage = usermem.HugePageSize +) + +func TestFindUnallocatedRange(t *testing.T) { + for _, test := range []struct { + desc string + usage *usageSegmentDataSlices + start uint64 + length uint64 + alignment uint64 + unallocated uint64 + minUnallocated uint64 + }{ + { + desc: "Initial allocation succeeds", + usage: &usageSegmentDataSlices{}, + start: 0, + length: page, + alignment: page, + unallocated: 0, + minUnallocated: 0, + }, + { + desc: "Allocation begins at start of file", + usage: &usageSegmentDataSlices{ + Start: []uint64{page}, + End: []uint64{2 * page}, + Values: []usageInfo{{refs: 1}}, + }, + start: 0, + length: page, + alignment: page, + unallocated: 0, + minUnallocated: 0, + }, + { + desc: "In-use frames are not allocatable", + usage: &usageSegmentDataSlices{ + Start: []uint64{0, page}, + End: []uint64{page, 2 * page}, + Values: []usageInfo{{refs: 1}, {refs: 2}}, + }, + start: 0, + length: page, + alignment: page, + unallocated: 2 * page, + minUnallocated: 2 * page, + }, + { + desc: "Reclaimable frames are not allocatable", + usage: &usageSegmentDataSlices{ + Start: []uint64{0, page, 2 * page}, + End: []uint64{page, 2 * page, 3 * page}, + Values: []usageInfo{{refs: 1}, {refs: 0}, {refs: 1}}, + }, + start: 0, + length: page, + alignment: page, + unallocated: 3 * page, + minUnallocated: 3 * page, + }, + { + desc: "Gaps between in-use frames are allocatable", + usage: &usageSegmentDataSlices{ + Start: []uint64{0, 2 * page}, + End: []uint64{page, 3 * page}, + Values: []usageInfo{{refs: 1}, {refs: 1}}, + }, + start: 0, + length: page, + alignment: page, + unallocated: page, + minUnallocated: page, + }, + { + desc: "Inadequately-sized gaps are rejected", + usage: &usageSegmentDataSlices{ + Start: []uint64{0, 2 * page}, + End: []uint64{page, 3 * page}, + Values: []usageInfo{{refs: 1}, {refs: 1}}, + }, + start: 0, + length: 2 * page, + alignment: page, + unallocated: 3 * page, + minUnallocated: page, + }, + { + desc: "Hugepage alignment is honored", + usage: &usageSegmentDataSlices{ + Start: []uint64{0, hugepage + page}, + // Hugepage-sized gap here that shouldn't be allocated from + // since it's incorrectly aligned. + End: []uint64{page, hugepage + 2*page}, + Values: []usageInfo{{refs: 1}, {refs: 1}}, + }, + start: 0, + length: hugepage, + alignment: hugepage, + unallocated: 2 * hugepage, + minUnallocated: page, + }, + { + desc: "Pages before start ignored", + usage: &usageSegmentDataSlices{ + Start: []uint64{page, 3 * page}, + End: []uint64{2 * page, 4 * page}, + Values: []usageInfo{{refs: 1}, {refs: 2}}, + }, + start: page, + length: page, + alignment: page, + unallocated: 2 * page, + minUnallocated: 2 * page, + }, + { + desc: "start may be in the middle of segment", + usage: &usageSegmentDataSlices{ + Start: []uint64{0, 3 * page}, + End: []uint64{2 * page, 4 * page}, + Values: []usageInfo{{refs: 1}, {refs: 2}}, + }, + start: page, + length: page, + alignment: page, + unallocated: 2 * page, + minUnallocated: 2 * page, + }, + } { + t.Run(test.desc, func(t *testing.T) { + var usage usageSet + if err := usage.ImportSortedSlices(test.usage); err != nil { + t.Fatalf("Failed to initialize usage from %v: %v", test.usage, err) + } + unallocated, minUnallocated := findUnallocatedRange(&usage, test.start, test.length, test.alignment) + if unallocated != test.unallocated { + t.Errorf("findUnallocatedRange(%v, %x, %x, %x): got unallocated %x, wanted %x", test.usage, test.start, test.length, test.alignment, unallocated, test.unallocated) + } + if minUnallocated != test.minUnallocated { + t.Errorf("findUnallocatedRange(%v, %x, %x, %x): got minUnallocated %x, wanted %x", test.usage, test.start, test.length, test.alignment, minUnallocated, test.minUnallocated) + } + }) + } +} diff --git a/pkg/sentry/pgalloc/pgalloc_unsafe.go b/pkg/sentry/pgalloc/pgalloc_unsafe.go new file mode 100644 index 000000000..33b0a68a8 --- /dev/null +++ b/pkg/sentry/pgalloc/pgalloc_unsafe.go @@ -0,0 +1,40 @@ +// Copyright 2018 Google LLC +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package pgalloc + +import ( + "reflect" + "syscall" + "unsafe" +) + +func unsafeSlice(addr uintptr, length int) (slice []byte) { + sh := (*reflect.SliceHeader)(unsafe.Pointer(&slice)) + sh.Data = addr + sh.Len = length + sh.Cap = length + return +} + +func mincore(s []byte, buf []byte) error { + if _, _, errno := syscall.RawSyscall( + syscall.SYS_MINCORE, + uintptr(unsafe.Pointer(&s[0])), + uintptr(len(s)), + uintptr(unsafe.Pointer(&buf[0]))); errno != 0 { + return errno + } + return nil +} diff --git a/pkg/sentry/pgalloc/save_restore.go b/pkg/sentry/pgalloc/save_restore.go new file mode 100644 index 000000000..21024e656 --- /dev/null +++ b/pkg/sentry/pgalloc/save_restore.go @@ -0,0 +1,205 @@ +// Copyright 2018 Google LLC +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package pgalloc + +import ( + "bytes" + "fmt" + "io" + "runtime" + "sync/atomic" + "syscall" + + "gvisor.googlesource.com/gvisor/pkg/log" + "gvisor.googlesource.com/gvisor/pkg/sentry/usage" + "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" + "gvisor.googlesource.com/gvisor/pkg/state" +) + +// SaveTo writes f's state to the given stream. +func (f *MemoryFile) SaveTo(w io.Writer) error { + // Wait for reclaim. + f.mu.Lock() + defer f.mu.Unlock() + for f.reclaimable { + f.reclaimCond.Signal() + f.mu.Unlock() + runtime.Gosched() + f.mu.Lock() + } + + // Ensure that all pages that contain data have knownCommitted set, since + // we only store knownCommitted pages below. + zeroPage := make([]byte, usermem.PageSize) + err := f.updateUsageLocked(0, func(bs []byte, committed []byte) error { + for pgoff := 0; pgoff < len(bs); pgoff += usermem.PageSize { + i := pgoff / usermem.PageSize + pg := bs[pgoff : pgoff+usermem.PageSize] + if !bytes.Equal(pg, zeroPage) { + committed[i] = 1 + continue + } + committed[i] = 0 + // Reading the page caused it to be committed; decommit it to + // reduce memory usage. + // + // "MADV_REMOVE [...] Free up a given range of pages and its + // associated backing store. This is equivalent to punching a hole + // in the corresponding byte range of the backing store (see + // fallocate(2))." - madvise(2) + if err := syscall.Madvise(pg, syscall.MADV_REMOVE); err != nil { + // This doesn't impact the correctness of saved memory, it + // just means that we're incrementally more likely to OOM. + // Complain, but don't abort saving. + log.Warningf("Decommitting page %p while saving failed: %v", pg, err) + } + } + return nil + }) + if err != nil { + return err + } + + // Save metadata. + if err := state.Save(w, &f.fileSize, nil); err != nil { + return err + } + if err := state.Save(w, &f.usage, nil); err != nil { + return err + } + + // Dump out committed pages. + for seg := f.usage.FirstSegment(); seg.Ok(); seg = seg.NextSegment() { + if !seg.Value().knownCommitted { + continue + } + // Write a header to distinguish from objects. + if err := state.WriteHeader(w, uint64(seg.Range().Length()), false); err != nil { + return err + } + // Write out data. + var ioErr error + err := f.forEachMappingSlice(seg.Range(), func(s []byte) { + if ioErr != nil { + return + } + _, ioErr = w.Write(s) + }) + if ioErr != nil { + return ioErr + } + if err != nil { + return err + } + } + + return nil +} + +// LoadFrom loads MemoryFile state from the given stream. +func (f *MemoryFile) LoadFrom(r io.Reader) error { + // Load metadata. + if err := state.Load(r, &f.fileSize, nil); err != nil { + return err + } + if err := f.file.Truncate(f.fileSize); err != nil { + return err + } + newMappings := make([]uintptr, f.fileSize>>chunkShift) + f.mappings.Store(newMappings) + if err := state.Load(r, &f.usage, nil); err != nil { + return err + } + + // Try to map committed chunks concurrently: For any given chunk, either + // this loop or the following one will mmap the chunk first and cache it in + // f.mappings for the other, but this loop is likely to run ahead of the + // other since it doesn't do any work between mmaps. The rest of this + // function doesn't mutate f.usage, so it's safe to iterate concurrently. + mapperDone := make(chan struct{}) + mapperCanceled := int32(0) + go func() { // S/R-SAFE: see comment + defer func() { close(mapperDone) }() + for seg := f.usage.FirstSegment(); seg.Ok(); seg = seg.NextSegment() { + if atomic.LoadInt32(&mapperCanceled) != 0 { + return + } + if seg.Value().knownCommitted { + f.forEachMappingSlice(seg.Range(), func(s []byte) {}) + } + } + }() + defer func() { + atomic.StoreInt32(&mapperCanceled, 1) + <-mapperDone + }() + + // Load committed pages. + for seg := f.usage.FirstSegment(); seg.Ok(); seg = seg.NextSegment() { + if !seg.Value().knownCommitted { + continue + } + // Verify header. + length, object, err := state.ReadHeader(r) + if err != nil { + return err + } + if object { + // Not expected. + return fmt.Errorf("unexpected object") + } + if expected := uint64(seg.Range().Length()); length != expected { + // Size mismatch. + return fmt.Errorf("mismatched segment: expected %d, got %d", expected, length) + } + // Read data. + var ioErr error + err = f.forEachMappingSlice(seg.Range(), func(s []byte) { + if ioErr != nil { + return + } + _, ioErr = io.ReadFull(r, s) + }) + if ioErr != nil { + return ioErr + } + if err != nil { + return err + } + + // Update accounting for restored pages. We need to do this here since + // these segments are marked as "known committed", and will be skipped + // over on accounting scans. + usage.MemoryAccounting.Inc(seg.End()-seg.Start(), seg.Value().kind) + } + + return nil +} + +// MemoryFileProvider provides the MemoryFile method. +// +// This type exists to work around a save/restore defect. The only object in a +// saved object graph that S/R allows to be replaced at time of restore is the +// starting point of the restore, kernel.Kernel. However, the MemoryFile +// changes between save and restore as well, so objects that need persistent +// access to the MemoryFile must instead store a pointer to the Kernel and call +// Kernel.MemoryFile() as required. In most cases, depending on the kernel +// package directly would create a package dependency loop, so the stored +// pointer must instead be a MemoryProvider interface object. Correspondingly, +// kernel.Kernel is the only implementation of this interface. +type MemoryFileProvider interface { + // MemoryFile returns the Kernel MemoryFile. + MemoryFile() *MemoryFile +} |