summaryrefslogtreecommitdiffhomepage
path: root/pkg/sentry/pgalloc
diff options
context:
space:
mode:
authorJamie Liu <jamieliu@google.com>2019-03-14 08:11:36 -0700
committerShentubot <shentubot@google.com>2019-03-14 08:12:48 -0700
commit8f4634997bd97810a85a70b71f000378d9db2e55 (patch)
tree903096f91ee8f201fa622296e0f04cf7c7cd9013 /pkg/sentry/pgalloc
parentfb9919881c7dc98eaf97cad2a70d187bd78f1566 (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/BUILD57
-rw-r--r--pkg/sentry/pgalloc/context.go48
-rw-r--r--pkg/sentry/pgalloc/pgalloc.go922
-rw-r--r--pkg/sentry/pgalloc/pgalloc_test.go168
-rw-r--r--pkg/sentry/pgalloc/pgalloc_unsafe.go40
-rw-r--r--pkg/sentry/pgalloc/save_restore.go205
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
+}