diff options
author | Jamie Liu <jamieliu@google.com> | 2019-03-12 10:28:23 -0700 |
---|---|---|
committer | Shentubot <shentubot@google.com> | 2019-03-12 10:29:16 -0700 |
commit | 8930e79ebf72a0cc69e9b81af37bcbb57b115543 (patch) | |
tree | c71cae2be099a7923a1c7681ebe7c4322f189f4a /pkg/sentry/platform | |
parent | 6e6dbf0e566270ae96a4db81d9d04275d0fffb00 (diff) |
Clarify the platform.File interface.
- Redefine some memmap.Mappable, platform.File, and platform.Memory
semantics in terms of File reference counts (no functional change).
- Make AddressSpace.MapFile take a platform.File instead of a raw FD,
and replace platform.File.MapInto with platform.File.FD. This allows
kvm.AddressSpace.MapFile to always use platform.File.MapInternal instead
of maintaining its own (redundant) cache of file mappings in the sentry
address space.
PiperOrigin-RevId: 238044504
Change-Id: Ib73a11e4275c0da0126d0194aa6c6017a9cef64f
Diffstat (limited to 'pkg/sentry/platform')
-rw-r--r-- | pkg/sentry/platform/filemem/filemem.go | 65 | ||||
-rw-r--r-- | pkg/sentry/platform/kvm/BUILD | 24 | ||||
-rw-r--r-- | pkg/sentry/platform/kvm/address_space.go | 104 | ||||
-rw-r--r-- | pkg/sentry/platform/kvm/host_map.go | 184 | ||||
-rw-r--r-- | pkg/sentry/platform/platform.go | 95 | ||||
-rw-r--r-- | pkg/sentry/platform/ptrace/subprocess.go | 4 |
6 files changed, 100 insertions, 376 deletions
diff --git a/pkg/sentry/platform/filemem/filemem.go b/pkg/sentry/platform/filemem/filemem.go index 97da31e70..f41c70ba5 100644 --- a/pkg/sentry/platform/filemem/filemem.go +++ b/pkg/sentry/platform/filemem/filemem.go @@ -32,7 +32,6 @@ import ( "time" "gvisor.googlesource.com/gvisor/pkg/log" - "gvisor.googlesource.com/gvisor/pkg/sentry/context" "gvisor.googlesource.com/gvisor/pkg/sentry/memutil" "gvisor.googlesource.com/gvisor/pkg/sentry/platform" "gvisor.googlesource.com/gvisor/pkg/sentry/safemem" @@ -504,39 +503,6 @@ func (f *FileMem) markReclaimed(fr platform.FileRange) { } } -// MapInto implements platform.File.MapInto. -func (f *FileMem) MapInto(as platform.AddressSpace, addr usermem.Addr, fr platform.FileRange, at usermem.AccessType, precommit bool) error { - if !fr.WellFormed() || fr.Length() == 0 || fr.Start%usermem.PageSize != 0 || fr.End%usermem.PageSize != 0 { - panic(fmt.Sprintf("invalid range: %v", fr)) - } - return as.MapFile(addr, int(f.file.Fd()), fr, at, precommit) -} - -// 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 -} - // 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 { @@ -596,9 +562,29 @@ func (f *FileMem) DecRef(fr platform.FileRange) { } } -// Flush implements platform.Mappable.Flush. -func (f *FileMem) Flush(ctx context.Context) error { - return nil +// 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 @@ -653,6 +639,11 @@ func (f *FileMem) getChunkMapping(chunk int) ([]uintptr, uintptr, error) { 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() diff --git a/pkg/sentry/platform/kvm/BUILD b/pkg/sentry/platform/kvm/BUILD index 6e40b3177..b7bf88249 100644 --- a/pkg/sentry/platform/kvm/BUILD +++ b/pkg/sentry/platform/kvm/BUILD @@ -2,28 +2,6 @@ load("//tools/go_stateify:defs.bzl", "go_library", "go_test") package(licenses = ["notice"]) -load("//tools/go_generics:defs.bzl", "go_template_instance") - -go_template_instance( - name = "host_map_set", - out = "host_map_set.go", - consts = { - "minDegree": "15", - }, - imports = { - "usermem": "gvisor.googlesource.com/gvisor/pkg/sentry/usermem", - }, - package = "kvm", - prefix = "hostMap", - template = "//pkg/segment:generic_set", - types = { - "Key": "usermem.Addr", - "Range": "usermem.AddrRange", - "Value": "uintptr", - "Functions": "hostMapSetFunctions", - }, -) - go_library( name = "kvm", srcs = [ @@ -36,8 +14,6 @@ go_library( "bluepill_fault.go", "bluepill_unsafe.go", "context.go", - "host_map.go", - "host_map_set.go", "kvm.go", "kvm_amd64.go", "kvm_amd64_unsafe.go", diff --git a/pkg/sentry/platform/kvm/address_space.go b/pkg/sentry/platform/kvm/address_space.go index 72e897a9a..6d8d8e65b 100644 --- a/pkg/sentry/platform/kvm/address_space.go +++ b/pkg/sentry/platform/kvm/address_space.go @@ -15,7 +15,6 @@ package kvm import ( - "reflect" "sync" "sync/atomic" @@ -88,11 +87,6 @@ type addressSpace struct { // dirtySet is the set of dirty vCPUs. dirtySet *dirtySet - - // files contains files mapped in the host address space. - // - // See host_map.go for more information. - files hostMap } // invalidate is the implementation for Invalidate. @@ -118,6 +112,11 @@ func (as *addressSpace) Touch(c *vCPU) bool { return as.dirtySet.mark(c) } +type hostMapEntry struct { + addr uintptr + length uintptr +} + func (as *addressSpace) mapHost(addr usermem.Addr, m hostMapEntry, at usermem.AccessType) (inv bool) { for m.length > 0 { physical, length, ok := translateToPhysical(m.addr) @@ -158,100 +157,57 @@ func (as *addressSpace) mapHost(addr usermem.Addr, m hostMapEntry, at usermem.Ac return inv } -func (as *addressSpace) mapHostFile(addr usermem.Addr, fd int, fr platform.FileRange, at usermem.AccessType) error { - // Create custom host mappings. - ms, err := as.files.CreateMappings(usermem.AddrRange{ - Start: addr, - End: addr + usermem.Addr(fr.End-fr.Start), - }, at, fd, fr.Start) - if err != nil { - return err - } - - inv := false - for _, m := range ms { - // The host mapped slices are guaranteed to be aligned. - prev := as.mapHost(addr, m, at) - inv = inv || prev - addr += usermem.Addr(m.length) - } - if inv { - as.invalidate() - } - - return nil -} +// MapFile implements platform.AddressSpace.MapFile. +func (as *addressSpace) MapFile(addr usermem.Addr, f platform.File, fr platform.FileRange, at usermem.AccessType, precommit bool) error { + as.mu.Lock() + defer as.mu.Unlock() -func (as *addressSpace) mapFilemem(addr usermem.Addr, fr platform.FileRange, at usermem.AccessType, precommit bool) error { - // TODO: Lock order at the platform level is not sufficiently - // well-defined to guarantee that the caller (FileMem.MapInto) is not - // holding any locks that FileMem.MapInternal may take. - - // Retrieve mappings for the underlying filemem. Note that the - // permissions here are largely irrelevant, since it corresponds to - // physical memory for the guest. We enforce the given access type - // below, in the guest page tables. - bs, err := as.filemem.MapInternal(fr, usermem.AccessType{ - Read: true, - Write: true, + // Get mappings in the sentry's address space, which are guaranteed to be + // valid as long as a reference is held on the mapped pages (which is in + // turn required by AddressSpace.MapFile precondition). + // + // If precommit is true, we will touch mappings to commit them, so ensure + // that mappings are readable from sentry context. + // + // We don't execute from application file-mapped memory, and guest page + // tables don't care if we have execute permission (but they do need pages + // to be readable). + bs, err := f.MapInternal(fr, usermem.AccessType{ + Read: at.Read || at.Execute || precommit, + Write: at.Write, }) if err != nil { return err } - // Save the original range for invalidation. - orig := usermem.AddrRange{ - Start: addr, - End: addr + usermem.Addr(fr.End-fr.Start), - } - + // Map the mappings in the sentry's address space (guest physical memory) + // into the application's address space (guest virtual memory). inv := false for !bs.IsEmpty() { b := bs.Head() bs = bs.Tail() // Since fr was page-aligned, b should also be page-aligned. We do the // lookup in our host page tables for this translation. - s := b.ToSlice() if precommit { + s := b.ToSlice() for i := 0; i < len(s); i += usermem.PageSize { _ = s[i] // Touch to commit. } } prev := as.mapHost(addr, hostMapEntry{ - addr: reflect.ValueOf(&s[0]).Pointer(), - length: uintptr(len(s)), + addr: b.Addr(), + length: uintptr(b.Len()), }, at) inv = inv || prev - addr += usermem.Addr(len(s)) + addr += usermem.Addr(b.Len()) } if inv { as.invalidate() - as.files.DeleteMapping(orig) } return nil } -// MapFile implements platform.AddressSpace.MapFile. -func (as *addressSpace) MapFile(addr usermem.Addr, fd int, fr platform.FileRange, at usermem.AccessType, precommit bool) error { - as.mu.Lock() - defer as.mu.Unlock() - - // Create an appropriate mapping. If this is filemem, we don't create - // custom mappings for each in-application mapping. For files however, - // we create distinct mappings for each address space. Unfortunately, - // there's not a better way to manage this here. The file underlying - // this fd can change at any time, so we can't actually index the file - // and share between address space. Oh well. It's all referring to the - // same physical pages, hopefully we don't run out of address space. - if fd != int(as.filemem.File().Fd()) { - // N.B. precommit is ignored for host files. - return as.mapHostFile(addr, fd, fr, at) - } - - return as.mapFilemem(addr, fr, at, precommit) -} - // Unmap unmaps the given range by calling pagetables.PageTables.Unmap. func (as *addressSpace) Unmap(addr usermem.Addr, length uint64) { as.mu.Lock() @@ -264,10 +220,6 @@ func (as *addressSpace) Unmap(addr usermem.Addr, length uint64) { }) if prev { as.invalidate() - as.files.DeleteMapping(usermem.AddrRange{ - Start: addr, - End: addr + usermem.Addr(length), - }) // Recycle any freed intermediate pages. as.pageTables.Allocator.Recycle() diff --git a/pkg/sentry/platform/kvm/host_map.go b/pkg/sentry/platform/kvm/host_map.go deleted file mode 100644 index ee6a1a42d..000000000 --- a/pkg/sentry/platform/kvm/host_map.go +++ /dev/null @@ -1,184 +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 kvm - -import ( - "fmt" - "sync" - "syscall" - - "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" -) - -type hostMap struct { - // mu protects below. - mu sync.RWMutex - - // set contains host mappings. - set hostMapSet -} - -type hostMapEntry struct { - addr uintptr - length uintptr -} - -// forEach iterates over all mappings in the given range. -// -// Precondition: segFn and gapFn must be non-nil. -func (hm *hostMap) forEach( - r usermem.AddrRange, - segFn func(offset uint64, m hostMapEntry), - gapFn func(offset uint64, length uintptr) (uintptr, bool)) { - - seg, gap := hm.set.Find(r.Start) - for { - if seg.Ok() && seg.Start() < r.End { - // A valid segment: pass information. - overlap := seg.Range().Intersect(r) - segOffset := uintptr(overlap.Start - seg.Start()) - mapOffset := uint64(overlap.Start - r.Start) - segFn(mapOffset, hostMapEntry{ - addr: seg.Value() + segOffset, - length: uintptr(overlap.Length()), - }) - seg, gap = seg.NextNonEmpty() - } else if gap.Ok() && gap.Start() < r.End { - // A gap: pass gap information. - overlap := gap.Range().Intersect(r) - mapOffset := uint64(overlap.Start - r.Start) - addr, ok := gapFn(mapOffset, uintptr(overlap.Length())) - if ok { - seg = hm.set.Insert(gap, overlap, addr) - seg, gap = seg.NextNonEmpty() - } else { - seg = gap.NextSegment() - gap = hostMapGapIterator{} // Invalid. - } - } else { - // Terminal. - break - } - } -} - -func (hm *hostMap) createMappings(r usermem.AddrRange, at usermem.AccessType, fd int, offset uint64) (ms []hostMapEntry, err error) { - hm.forEach(r, func(mapOffset uint64, m hostMapEntry) { - // Replace any existing mappings. - _, _, errno := syscall.RawSyscall6( - syscall.SYS_MMAP, - m.addr, - m.length, - uintptr(at.Prot()), - syscall.MAP_FIXED|syscall.MAP_SHARED, - uintptr(fd), - uintptr(offset+mapOffset)) - if errno != 0 && err == nil { - err = errno - } - }, func(mapOffset uint64, length uintptr) (uintptr, bool) { - // Create a new mapping. - addr, _, errno := syscall.RawSyscall6( - syscall.SYS_MMAP, - 0, - length, - uintptr(at.Prot()), - syscall.MAP_SHARED, - uintptr(fd), - uintptr(offset+mapOffset)) - if errno != 0 { - err = errno - return 0, false - } - return addr, true - }) - if err != nil { - return nil, err - } - - // Collect all entries. - // - // We do this after the first iteration because some segments may have - // been merged in the above, and we'll return the simplest form. This - // also provides a basic sanity check in the form of no gaps. - hm.forEach(r, func(_ uint64, m hostMapEntry) { - ms = append(ms, m) - }, func(uint64, uintptr) (uintptr, bool) { - // Should not happen: we just mapped this above. - panic("unexpected gap") - }) - - return ms, nil -} - -// CreateMappings creates a new set of host mapping entries. -func (hm *hostMap) CreateMappings(r usermem.AddrRange, at usermem.AccessType, fd int, offset uint64) (ms []hostMapEntry, err error) { - hm.mu.Lock() - ms, err = hm.createMappings(r, at, fd, offset) - hm.mu.Unlock() - return -} - -func (hm *hostMap) deleteMapping(r usermem.AddrRange) { - // Remove all the existing mappings. - hm.forEach(r, func(_ uint64, m hostMapEntry) { - _, _, errno := syscall.RawSyscall( - syscall.SYS_MUNMAP, - m.addr, - m.length, - 0) - if errno != 0 { - // Should never happen. - panic(fmt.Sprintf("unmap error: %v", errno)) - } - }, func(uint64, uintptr) (uintptr, bool) { - // Sometimes deleteMapping will be called on a larger range - // than physical mappings are defined. That's okay. - return 0, false - }) - - // Knock the entire range out. - hm.set.RemoveRange(r) -} - -// DeleteMapping deletes the given range. -func (hm *hostMap) DeleteMapping(r usermem.AddrRange) { - hm.mu.Lock() - hm.deleteMapping(r) - hm.mu.Unlock() -} - -// hostMapSetFunctions is used in the implementation of mapSet. -type hostMapSetFunctions struct{} - -func (hostMapSetFunctions) MinKey() usermem.Addr { return 0 } -func (hostMapSetFunctions) MaxKey() usermem.Addr { return ^usermem.Addr(0) } -func (hostMapSetFunctions) ClearValue(val *uintptr) { *val = 0 } - -func (hostMapSetFunctions) Merge(r1 usermem.AddrRange, addr1 uintptr, r2 usermem.AddrRange, addr2 uintptr) (uintptr, bool) { - if addr1+uintptr(r1.Length()) != addr2 { - return 0, false - } - - // Since the two regions are contiguous in both the key space and the - // value space, we can just store a single segment with the first host - // virtual address; the logic above operates based on the size of the - // segments. - return addr1, true -} - -func (hostMapSetFunctions) Split(r usermem.AddrRange, hostAddr uintptr, split usermem.Addr) (uintptr, uintptr) { - return hostAddr, hostAddr + uintptr(split-r.Start) -} diff --git a/pkg/sentry/platform/platform.go b/pkg/sentry/platform/platform.go index a9e76bd45..b2ce851da 100644 --- a/pkg/sentry/platform/platform.go +++ b/pkg/sentry/platform/platform.go @@ -52,11 +52,11 @@ type Platform interface { DetectsCPUPreemption() bool // MapUnit returns the alignment used for optional mappings into this - // platform's AddressSpaces. Higher values indicate lower per-page - // costs for AddressSpace.MapInto. As a special case, a MapUnit of 0 - // indicates that the cost of AddressSpace.MapInto is effectively - // independent of the number of pages mapped. If MapUnit is non-zero, - // it must be a power-of-2 multiple of usermem.PageSize. + // platform's AddressSpaces. Higher values indicate lower per-page costs + // for AddressSpace.MapFile. As a special case, a MapUnit of 0 indicates + // that the cost of AddressSpace.MapFile is effectively independent of the + // number of pages mapped. If MapUnit is non-zero, it must be a power-of-2 + // multiple of usermem.PageSize. MapUnit() uint64 // MinUserAddress returns the minimum mappable address on this @@ -194,17 +194,17 @@ const SignalInterrupt = linux.SIGCHLD // AddressSpace represents a virtual address space in which a Context can // execute. type AddressSpace interface { - // MapFile creates a shared mapping of offsets in fr, from the file - // with file descriptor fd, at address addr. Any existing overlapping - // mappings are silently replaced. + // MapFile creates a shared mapping of offsets fr from f at address addr. + // Any existing overlapping mappings are silently replaced. // - // If precommit is true, host memory should be committed to the mapping - // when MapFile returns when possible. The precommit flag is advisory - // and implementations may choose to ignore it. + // If precommit is true, the platform should eagerly commit resources (e.g. + // physical memory) to the mapping. The precommit flag is advisory and + // implementations may choose to ignore it. // - // Preconditions: addr and fr must be page-aligned. length > 0. - // at.Any() == true. - MapFile(addr usermem.Addr, fd int, fr FileRange, at usermem.AccessType, precommit bool) error + // Preconditions: addr and fr must be page-aligned. fr.Length() > 0. + // at.Any() == true. At least one reference must be held on all pages in + // fr, and must continue to be held as long as pages are mapped. + MapFile(addr usermem.Addr, f File, fr FileRange, at usermem.AccessType, precommit bool) error // Unmap unmaps the given range. // @@ -309,44 +309,39 @@ func (f SegmentationFault) Error() string { // File represents a host file that may be mapped into an AddressSpace. type File interface { - // MapInto maps fr into as, starting at addr, for accesses of type at. - // - // If precommit is true, the platform should eagerly commit resources (e.g. - // physical memory) to the mapping. The precommit flag is advisory and - // implementations may choose to ignore it. + // All pages in a File are reference-counted. + + // IncRef increments the reference count on all pages in fr. // - // Note that there is no File.Unmap; clients should use as.Unmap directly. + // Preconditions: fr.Start and fr.End must be page-aligned. fr.Length() > + // 0. At least one reference must be held on all pages in fr. (The File + // interface does not provide a way to acquire an initial reference; + // implementors may define mechanisms for doing so.) + IncRef(fr FileRange) + + // DecRef decrements the reference count on all pages in fr. // - // Preconditions: fr.Start and fr.End must be page-aligned. - // fr.Length() > 0. at.Any() == true. Implementors may define - // additional requirements. - MapInto(as AddressSpace, addr usermem.Addr, fr FileRange, at usermem.AccessType, precommit bool) error + // Preconditions: fr.Start and fr.End must be page-aligned. fr.Length() > + // 0. At least one reference must be held on all pages in fr. + DecRef(fr FileRange) // MapInternal returns a mapping of the given file offsets in the invoking - // process' address space for reading and writing. The returned mapping is - // valid as long as a reference is held on the mapped range. + // process' address space for reading and writing. // // Note that fr.Start and fr.End need not be page-aligned. // - // Preconditions: fr.Length() > 0. Implementors may define additional - // requirements. - MapInternal(fr FileRange, at usermem.AccessType) (safemem.BlockSeq, error) - - // IncRef signals that a region in the file is actively referenced through a - // memory map. Implementors must ensure that the contents of a referenced - // region remain consistent. Specifically, mappings returned by MapInternal - // must refer to the same underlying contents. If the implementor also - // implements the Memory interface, the file range must not be reused in a - // different allocation while it has active references. + // Preconditions: fr.Length() > 0. At least one reference must be held on + // all pages in fr. // - // Preconditions: fr.Start and fr.End must be page-aligned. fr.Length() > 0. - IncRef(fr FileRange) + // Postconditions: The returned mapping is valid as long as at least one + // reference is held on the mapped pages. + MapInternal(fr FileRange, at usermem.AccessType) (safemem.BlockSeq, error) - // DecRef reduces the frame ref count on the range specified by fr. + // FD returns the file descriptor represented by the File. // - // Preconditions: fr.Start and fr.End must be page-aligned. fr.Length() > - // 0. DecRef()s on a region must match earlier IncRef()s. - DecRef(fr FileRange) + // The only permitted operation on the returned file descriptor is to map + // pages from it consistent with the requirements of AddressSpace.MapFile. + FD() int } // FileRange represents a range of uint64 offsets into a File. @@ -361,19 +356,13 @@ func (fr FileRange) String() string { // Memory represents an allocatable File that may be mapped into any // AddressSpace associated with the same Platform. type Memory interface { - // Memory implements File methods with the following properties: - // - // - Pages mapped by MapInto must be allocated, and must be unmapped from - // all AddressSpaces before they are freed. - // - // - Pages mapped by MapInternal must be allocated. Returned mappings are - // guaranteed to be valid until the mapped pages are freed. File - // Allocate returns a range of pages of the given length, owned by the - // caller and with the given accounting kind. Allocated memory initially has - // a single reference and will automatically be freed when no references to - // them remain. See File.IncRef and File.DecRef. + // 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) diff --git a/pkg/sentry/platform/ptrace/subprocess.go b/pkg/sentry/platform/ptrace/subprocess.go index a9d083f5a..82f125073 100644 --- a/pkg/sentry/platform/ptrace/subprocess.go +++ b/pkg/sentry/platform/ptrace/subprocess.go @@ -563,7 +563,7 @@ func (s *subprocess) syscall(sysno uintptr, args ...arch.SyscallArgument) (uintp } // MapFile implements platform.AddressSpace.MapFile. -func (s *subprocess) MapFile(addr usermem.Addr, fd int, fr platform.FileRange, at usermem.AccessType, precommit bool) error { +func (s *subprocess) MapFile(addr usermem.Addr, f platform.File, fr platform.FileRange, at usermem.AccessType, precommit bool) error { var flags int if precommit { flags |= syscall.MAP_POPULATE @@ -574,7 +574,7 @@ func (s *subprocess) MapFile(addr usermem.Addr, fd int, fr platform.FileRange, a arch.SyscallArgument{Value: uintptr(fr.Length())}, arch.SyscallArgument{Value: uintptr(at.Prot())}, arch.SyscallArgument{Value: uintptr(flags | syscall.MAP_SHARED | syscall.MAP_FIXED)}, - arch.SyscallArgument{Value: uintptr(fd)}, + arch.SyscallArgument{Value: uintptr(f.FD())}, arch.SyscallArgument{Value: uintptr(fr.Start)}) return err } |