diff options
Diffstat (limited to 'pkg/sentry/platform')
-rw-r--r-- | pkg/sentry/platform/filemem/BUILD | 56 | ||||
-rw-r--r-- | pkg/sentry/platform/filemem/filemem.go | 879 | ||||
-rw-r--r-- | pkg/sentry/platform/filemem/filemem_state.go | 194 | ||||
-rw-r--r-- | pkg/sentry/platform/filemem/filemem_test.go | 168 | ||||
-rw-r--r-- | pkg/sentry/platform/filemem/filemem_unsafe.go | 40 | ||||
-rw-r--r-- | pkg/sentry/platform/kvm/BUILD | 1 | ||||
-rw-r--r-- | pkg/sentry/platform/kvm/address_space.go | 4 | ||||
-rw-r--r-- | pkg/sentry/platform/kvm/kvm.go | 17 | ||||
-rw-r--r-- | pkg/sentry/platform/kvm/kvm_test.go | 1 | ||||
-rw-r--r-- | pkg/sentry/platform/platform.go | 90 | ||||
-rw-r--r-- | pkg/sentry/platform/ptrace/BUILD | 1 | ||||
-rw-r--r-- | pkg/sentry/platform/ptrace/ptrace.go | 14 |
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 -} |