summaryrefslogtreecommitdiffhomepage
path: root/pkg/sentry/platform
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/platform
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/platform')
-rw-r--r--pkg/sentry/platform/filemem/BUILD56
-rw-r--r--pkg/sentry/platform/filemem/filemem.go879
-rw-r--r--pkg/sentry/platform/filemem/filemem_state.go194
-rw-r--r--pkg/sentry/platform/filemem/filemem_test.go168
-rw-r--r--pkg/sentry/platform/filemem/filemem_unsafe.go40
-rw-r--r--pkg/sentry/platform/kvm/BUILD1
-rw-r--r--pkg/sentry/platform/kvm/address_space.go4
-rw-r--r--pkg/sentry/platform/kvm/kvm.go17
-rw-r--r--pkg/sentry/platform/kvm/kvm_test.go1
-rw-r--r--pkg/sentry/platform/platform.go90
-rw-r--r--pkg/sentry/platform/ptrace/BUILD1
-rw-r--r--pkg/sentry/platform/ptrace/ptrace.go14
12 files changed, 3 insertions, 1462 deletions
diff --git a/pkg/sentry/platform/filemem/BUILD b/pkg/sentry/platform/filemem/BUILD
deleted file mode 100644
index 1a61cfaa5..000000000
--- a/pkg/sentry/platform/filemem/BUILD
+++ /dev/null
@@ -1,56 +0,0 @@
-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 = "filemem",
- prefix = "usage",
- template = "//pkg/segment:generic_set",
- types = {
- "Key": "uint64",
- "Range": "platform.FileRange",
- "Value": "usageInfo",
- "Functions": "usageSetFunctions",
- },
-)
-
-go_library(
- name = "filemem",
- srcs = [
- "filemem.go",
- "filemem_state.go",
- "filemem_unsafe.go",
- "usage_set.go",
- ],
- importpath = "gvisor.googlesource.com/gvisor/pkg/sentry/platform/filemem",
- 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 = "filemem_test",
- size = "small",
- srcs = ["filemem_test.go"],
- embed = [":filemem"],
- deps = ["//pkg/sentry/usermem"],
-)
diff --git a/pkg/sentry/platform/filemem/filemem.go b/pkg/sentry/platform/filemem/filemem.go
deleted file mode 100644
index f41c70ba5..000000000
--- a/pkg/sentry/platform/filemem/filemem.go
+++ /dev/null
@@ -1,879 +0,0 @@
-// 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 filemem provides a reusable implementation of platform.Memory.
-//
-// It enables memory to be sourced from a memfd file.
-//
-// Lock order:
-//
-// filemem.FileMem.mu
-// filemem.FileMem.mappingsMu
-package filemem
-
-import (
- "fmt"
- "math"
- "os"
- "sync"
- "sync/atomic"
- "syscall"
- "time"
-
- "gvisor.googlesource.com/gvisor/pkg/log"
- "gvisor.googlesource.com/gvisor/pkg/sentry/memutil"
- "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"
-)
-
-// FileMem is a platform.Memory that allocates from a host file that it owns.
-type FileMem struct {
- // Filemem models the backing file 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) (FileMem.Decommit) causes committed
- // pages to be uncommitted. This is the only event that can cause a
- // committed page to be uncommitted.
- //
- // Filemem's accounting is based on identifying the set of committed pages.
- // Since filemem does 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,
- // filemem queries the host kernel to determine which pages are committed.
-
- // file is the backing memory 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
-
- // Filemem pages 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 filemem 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 FileMem.AccessPhysical 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 indicates whether this region is known to be
- // committed. If this is false, then the region may or may not have
- // been touched. If it is true however, then mincore (below) has
- // indicated that the page is present at least once.
- 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)
-)
-
-// newFromFile creates a FileMem backed by the given file.
-func newFromFile(file *os.File) (*FileMem, error) {
- if err := file.Truncate(initialSize); err != nil {
- return nil, err
- }
- f := &FileMem{
- 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 FileMem, 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 FileMem 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,
- f.file.Fd(),
- 0)
- if errno != 0 {
- // This isn't fatal to filemem (IMA may not even be in use). Log the
- // error, but don't return it.
- log.Warningf("Failed to pre-map FileMem PROT_EXEC: %v", errno)
- } else {
- syscall.Syscall(
- syscall.SYS_MUNMAP,
- m,
- usermem.PageSize,
- 0)
- }
-
- return f, nil
-}
-
-// New creates a FileMem backed by a memfd file.
-func New(name string) (*FileMem, error) {
- fd, err := memutil.CreateMemFD(name, 0)
- if err != nil {
- if e, ok := err.(syscall.Errno); ok && e == syscall.ENOSYS {
- return nil, fmt.Errorf("memfd_create(2) is not implemented. Check that you have Linux 3.17 or higher")
- }
- return nil, err
- }
- return newFromFile(os.NewFile(uintptr(fd), name))
-}
-
-// Destroy implements platform.Memory.Destroy.
-func (f *FileMem) Destroy() {
- f.mu.Lock()
- defer f.mu.Unlock()
- f.destroyed = true
- f.reclaimCond.Signal()
-}
-
-// Allocate implements platform.Memory.Allocate.
-func (f *FileMem) 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 f.usage:\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
-}
-
-// fallocate(2) modes, defined in Linux's include/uapi/linux/falloc.h.
-const (
- _FALLOC_FL_KEEP_SIZE = 1
- _FALLOC_FL_PUNCH_HOLE = 2
-)
-
-// Decommit implements platform.Memory.Decommit.
-func (f *FileMem) 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 *FileMem) 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 frames in order to reduce memory usage.
-func (f *FileMem) 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 frames manually. This won't reduce memory usage, but at
- // least ensures that the frames will be zero when reallocated.
- f.forEachMappingSlice(fr, func(bs []byte) {
- for i := range bs {
- bs[i] = 0
- }
- })
- // Pretend the frames 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 f.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 filemem chunk %d: %v", m, i, errno)
- }
- }
- }
- // Similarly, invalidate f.mappings. (atomic.Value.Store(nil) panics.)
- f.mappings.Store([]uintptr{})
-}
-
-func (f *FileMem) 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
- }
- }
- f.reclaimable = false
- // No pages are reclaimable.
- f.minReclaimablePage = maxPage
- }
-}
-
-func (f *FileMem) 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 *FileMem) 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 *FileMem) 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 *FileMem) 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 *FileMem) 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 *FileMem) 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 *FileMem) FD() int {
- return int(f.file.Fd())
-}
-
-// UpdateUsage implements platform.Memory.UpdateUsage.
-func (f *FileMem) 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 *FileMem) 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 decommitted while we hold usageMu, 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 implements platform.Memory.TotalUsage.
-func (f *FileMem) 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 implements platform.Memory.TotalSize.
-func (f *FileMem) TotalSize() uint64 {
- f.mu.Lock()
- defer f.mu.Unlock()
- return uint64(f.fileSize)
-}
-
-// File returns the memory file used by f.
-func (f *FileMem) 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 filemem package, use f.usage directly instead.
-func (f *FileMem) 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/platform/filemem/filemem_state.go b/pkg/sentry/platform/filemem/filemem_state.go
deleted file mode 100644
index 964e2aaaa..000000000
--- a/pkg/sentry/platform/filemem/filemem_state.go
+++ /dev/null
@@ -1,194 +0,0 @@
-// 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 filemem
-
-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 implements platform.Memory.SaveTo.
-func (f *FileMem) 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
- }
-
- // 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.Range().Length(), seg.Value().kind)
- }
-
- return nil
-}
-
-// LoadFrom implements platform.Memory.LoadFrom.
-func (f *FileMem) 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
-}
diff --git a/pkg/sentry/platform/filemem/filemem_test.go b/pkg/sentry/platform/filemem/filemem_test.go
deleted file mode 100644
index 9becec25f..000000000
--- a/pkg/sentry/platform/filemem/filemem_test.go
+++ /dev/null
@@ -1,168 +0,0 @@
-// 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 filemem
-
-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/platform/filemem/filemem_unsafe.go b/pkg/sentry/platform/filemem/filemem_unsafe.go
deleted file mode 100644
index 776aed74d..000000000
--- a/pkg/sentry/platform/filemem/filemem_unsafe.go
+++ /dev/null
@@ -1,40 +0,0 @@
-// 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 filemem
-
-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/platform/kvm/BUILD b/pkg/sentry/platform/kvm/BUILD
index b7bf88249..9999e58f4 100644
--- a/pkg/sentry/platform/kvm/BUILD
+++ b/pkg/sentry/platform/kvm/BUILD
@@ -34,7 +34,6 @@ go_library(
"//pkg/log",
"//pkg/sentry/arch",
"//pkg/sentry/platform",
- "//pkg/sentry/platform/filemem",
"//pkg/sentry/platform/interrupt",
"//pkg/sentry/platform/procid",
"//pkg/sentry/platform/ring0",
diff --git a/pkg/sentry/platform/kvm/address_space.go b/pkg/sentry/platform/kvm/address_space.go
index 6d8d8e65b..f2f7ab1e8 100644
--- a/pkg/sentry/platform/kvm/address_space.go
+++ b/pkg/sentry/platform/kvm/address_space.go
@@ -20,7 +20,6 @@ import (
"gvisor.googlesource.com/gvisor/pkg/atomicbitops"
"gvisor.googlesource.com/gvisor/pkg/sentry/platform"
- "gvisor.googlesource.com/gvisor/pkg/sentry/platform/filemem"
"gvisor.googlesource.com/gvisor/pkg/sentry/platform/ring0/pagetables"
"gvisor.googlesource.com/gvisor/pkg/sentry/usermem"
)
@@ -76,9 +75,6 @@ type addressSpace struct {
// Note that the page tables themselves are not locked.
mu sync.Mutex
- // filemem is the memory instance.
- filemem *filemem.FileMem
-
// machine is the underlying machine.
machine *machine
diff --git a/pkg/sentry/platform/kvm/kvm.go b/pkg/sentry/platform/kvm/kvm.go
index d4f50024d..c5a4435b1 100644
--- a/pkg/sentry/platform/kvm/kvm.go
+++ b/pkg/sentry/platform/kvm/kvm.go
@@ -23,7 +23,6 @@ import (
"gvisor.googlesource.com/gvisor/pkg/cpuid"
"gvisor.googlesource.com/gvisor/pkg/sentry/platform"
- "gvisor.googlesource.com/gvisor/pkg/sentry/platform/filemem"
"gvisor.googlesource.com/gvisor/pkg/sentry/platform/ring0"
"gvisor.googlesource.com/gvisor/pkg/sentry/platform/ring0/pagetables"
"gvisor.googlesource.com/gvisor/pkg/sentry/usermem"
@@ -33,9 +32,6 @@ import (
type KVM struct {
platform.NoCPUPreemptionDetection
- // filemem is our memory source.
- *filemem.FileMem
-
// machine is the backing VM.
machine *machine
}
@@ -56,12 +52,6 @@ func OpenDevice() (*os.File, error) {
// New returns a new KVM-based implementation of the platform interface.
func New(deviceFile *os.File) (*KVM, error) {
- // Allocate physical memory for the vCPUs.
- fm, err := filemem.New("kvm-memory")
- if err != nil {
- return nil, err
- }
-
fd := deviceFile.Fd()
// Ensure global initialization is done.
@@ -90,7 +80,6 @@ func New(deviceFile *os.File) (*KVM, error) {
// All set.
return &KVM{
- FileMem: fm,
machine: machine,
}, nil
}
@@ -140,7 +129,6 @@ func (k *KVM) NewAddressSpace(_ interface{}) (platform.AddressSpace, <-chan stru
// Return the new address space.
return &addressSpace{
- filemem: k.FileMem,
machine: k.machine,
pageTables: pageTables,
dirtySet: k.machine.newDirtySet(),
@@ -153,8 +141,3 @@ func (k *KVM) NewContext() platform.Context {
machine: k.machine,
}
}
-
-// Memory returns the platform memory used to do allocations.
-func (k *KVM) Memory() platform.Memory {
- return k.FileMem
-}
diff --git a/pkg/sentry/platform/kvm/kvm_test.go b/pkg/sentry/platform/kvm/kvm_test.go
index fff463a6e..361200622 100644
--- a/pkg/sentry/platform/kvm/kvm_test.go
+++ b/pkg/sentry/platform/kvm/kvm_test.go
@@ -48,7 +48,6 @@ func kvmTest(t testHarness, setup func(*KVM), fn func(*vCPU) bool) {
t.Fatalf("error creating KVM instance: %v", err)
}
defer k.machine.Destroy()
- defer k.FileMem.Destroy()
// Call additional setup.
if setup != nil {
diff --git a/pkg/sentry/platform/platform.go b/pkg/sentry/platform/platform.go
index b2ce851da..d1c9458ea 100644
--- a/pkg/sentry/platform/platform.go
+++ b/pkg/sentry/platform/platform.go
@@ -19,17 +19,15 @@ package platform
import (
"fmt"
- "io"
"gvisor.googlesource.com/gvisor/pkg/abi/linux"
"gvisor.googlesource.com/gvisor/pkg/sentry/arch"
"gvisor.googlesource.com/gvisor/pkg/sentry/safemem"
- "gvisor.googlesource.com/gvisor/pkg/sentry/usage"
"gvisor.googlesource.com/gvisor/pkg/sentry/usermem"
)
-// Platform provides abstractions for execution contexts (Context) and memory
-// management (Memory, AddressSpace).
+// Platform provides abstractions for execution contexts (Context,
+// AddressSpace).
type Platform interface {
// SupportsAddressSpaceIO returns true if AddressSpaces returned by this
// Platform support AddressSpaceIO methods.
@@ -87,9 +85,6 @@ type Platform interface {
// NewContext returns a new execution context.
NewContext() Context
- // Memory returns memory for allocations.
- Memory() Memory
-
// PreemptAllCPUs causes all concurrent calls to Context.Switch(), as well
// as the first following call to Context.Switch() for each Context, to
// return ErrContextCPUPreempted.
@@ -352,84 +347,3 @@ type File interface {
func (fr FileRange) String() string {
return fmt.Sprintf("[%#x, %#x)", fr.Start, fr.End)
}
-
-// Memory represents an allocatable File that may be mapped into any
-// AddressSpace associated with the same Platform.
-type Memory interface {
- File
-
- // 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 Memory, allowing it to be
- // returned by a future call to Allocate.
- //
- // Preconditions: length must be page-aligned and non-zero.
- Allocate(length uint64, kind usage.MemoryKind) (FileRange, error)
-
- // Decommit releases resources associated with maintaining the contents of
- // the given frames. If Decommit succeeds, future accesses of the
- // decommitted frames will read zeroes.
- //
- // Preconditions: fr.Length() > 0.
- Decommit(fr FileRange) error
-
- // UpdateUsage updates the memory usage statistics. This must be called
- // before the relevant memory statistics in usage.MemoryAccounting can
- // be considered accurate.
- UpdateUsage() error
-
- // TotalUsage returns an aggregate usage for all memory statistics
- // except Mapped (which is external to the Memory implementation). This
- // is generally much cheaper than UpdateUsage, but will not provide a
- // fine-grained breakdown.
- TotalUsage() (uint64, error)
-
- // TotalSize returns the current maximum size of the Memory in bytes. The
- // value returned by TotalSize is permitted to change.
- TotalSize() uint64
-
- // Destroy releases all resources associated with the Memory.
- //
- // Preconditions: There are no remaining uses of any of the freed memory's
- // frames.
- //
- // Postconditions: None of the Memory's methods may be called after Destroy.
- Destroy()
-
- // SaveTo saves the memory state to the given stream, which will
- // generally be a statefile.
- SaveTo(w io.Writer) error
-
- // LoadFrom loads the memory state from the given stream, which will
- // generally be a statefile.
- LoadFrom(r io.Reader) error
-}
-
-// AllocateAndFill allocates memory of the given kind from mem 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 AllocateAndFill(mem Memory, length uint64, kind usage.MemoryKind, r safemem.Reader) (FileRange, error) {
- fr, err := mem.Allocate(length, kind)
- if err != nil {
- return FileRange{}, err
- }
- dsts, err := mem.MapInternal(fr, usermem.Write)
- if err != nil {
- mem.DecRef(fr)
- return 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.
- mem.DecRef(FileRange{fr.Start + un, fr.End})
- fr.End = fr.Start + un
- }
- return fr, err
-}
diff --git a/pkg/sentry/platform/ptrace/BUILD b/pkg/sentry/platform/ptrace/BUILD
index f86790942..e9e4a0d16 100644
--- a/pkg/sentry/platform/ptrace/BUILD
+++ b/pkg/sentry/platform/ptrace/BUILD
@@ -23,7 +23,6 @@ go_library(
"//pkg/seccomp",
"//pkg/sentry/arch",
"//pkg/sentry/platform",
- "//pkg/sentry/platform/filemem",
"//pkg/sentry/platform/interrupt",
"//pkg/sentry/platform/procid",
"//pkg/sentry/platform/safecopy",
diff --git a/pkg/sentry/platform/ptrace/ptrace.go b/pkg/sentry/platform/ptrace/ptrace.go
index 8d3f6ac9a..3c0713e95 100644
--- a/pkg/sentry/platform/ptrace/ptrace.go
+++ b/pkg/sentry/platform/ptrace/ptrace.go
@@ -50,7 +50,6 @@ import (
"gvisor.googlesource.com/gvisor/pkg/abi/linux"
"gvisor.googlesource.com/gvisor/pkg/sentry/arch"
"gvisor.googlesource.com/gvisor/pkg/sentry/platform"
- "gvisor.googlesource.com/gvisor/pkg/sentry/platform/filemem"
"gvisor.googlesource.com/gvisor/pkg/sentry/platform/interrupt"
"gvisor.googlesource.com/gvisor/pkg/sentry/usermem"
)
@@ -181,7 +180,6 @@ func (c *context) Interrupt() {
type PTrace struct {
platform.MMapMinAddr
platform.NoCPUPreemptionDetection
- *filemem.FileMem
}
// New returns a new ptrace-based implementation of the platform interface.
@@ -202,12 +200,7 @@ func New() (*PTrace, error) {
globalPool.master = master
})
- fm, err := filemem.New("ptrace-memory")
- if err != nil {
- return nil, err
- }
-
- return &PTrace{FileMem: fm}, nil
+ return &PTrace{}, nil
}
// SupportsAddressSpaceIO implements platform.Platform.SupportsAddressSpaceIO.
@@ -243,8 +236,3 @@ func (p *PTrace) NewAddressSpace(_ interface{}) (platform.AddressSpace, <-chan s
func (*PTrace) NewContext() platform.Context {
return &context{}
}
-
-// Memory returns the platform memory used to do allocations.
-func (p *PTrace) Memory() platform.Memory {
- return p.FileMem
-}