summaryrefslogtreecommitdiffhomepage
path: root/pkg/sentry
diff options
context:
space:
mode:
Diffstat (limited to 'pkg/sentry')
-rw-r--r--pkg/sentry/control/pprof.go30
-rw-r--r--pkg/sentry/fs/fdpipe/pipe.go3
-rw-r--r--pkg/sentry/fs/file_overlay.go12
-rw-r--r--pkg/sentry/fs/fsutil/file.go4
-rw-r--r--pkg/sentry/fs/fsutil/host_file_mapper.go21
-rw-r--r--pkg/sentry/fs/host/inode.go2
-rw-r--r--pkg/sentry/fs/inotify.go2
-rw-r--r--pkg/sentry/fs/lock/lock.go2
-rw-r--r--pkg/sentry/fs/proc/sys.go3
-rw-r--r--pkg/sentry/fs/timerfd/timerfd.go2
-rw-r--r--pkg/sentry/fs/tty/line_discipline.go4
-rw-r--r--pkg/sentry/fsimpl/cgroupfs/BUILD13
-rw-r--r--pkg/sentry/fsimpl/cgroupfs/bitmap.go139
-rw-r--r--pkg/sentry/fsimpl/cgroupfs/bitmap_test.go99
-rw-r--r--pkg/sentry/fsimpl/cgroupfs/cgroupfs.go2
-rw-r--r--pkg/sentry/fsimpl/cgroupfs/cpuset.go114
-rw-r--r--pkg/sentry/fsimpl/gofer/BUILD3
-rw-r--r--pkg/sentry/fsimpl/gofer/directory.go101
-rw-r--r--pkg/sentry/fsimpl/gofer/filesystem.go515
-rw-r--r--pkg/sentry/fsimpl/gofer/gofer.go677
-rw-r--r--pkg/sentry/fsimpl/gofer/gofer_test.go1
-rw-r--r--pkg/sentry/fsimpl/gofer/handle.go80
-rw-r--r--pkg/sentry/fsimpl/gofer/p9file.go8
-rw-r--r--pkg/sentry/fsimpl/gofer/regular_file.go26
-rw-r--r--pkg/sentry/fsimpl/gofer/revalidate.go50
-rw-r--r--pkg/sentry/fsimpl/gofer/save_restore.go151
-rw-r--r--pkg/sentry/fsimpl/gofer/socket.go45
-rw-r--r--pkg/sentry/fsimpl/gofer/special_file.go18
-rw-r--r--pkg/sentry/fsimpl/gofer/symlink.go8
-rw-r--r--pkg/sentry/fsimpl/gofer/time.go5
-rw-r--r--pkg/sentry/fsimpl/mqfs/BUILD2
-rw-r--r--pkg/sentry/fsimpl/mqfs/mqfs.go4
-rw-r--r--pkg/sentry/fsimpl/mqfs/registry.go6
-rw-r--r--pkg/sentry/fsimpl/overlay/filesystem.go2
-rw-r--r--pkg/sentry/fsimpl/proc/tasks.go2
-rw-r--r--pkg/sentry/fsimpl/proc/tasks_files.go5
-rw-r--r--pkg/sentry/fsimpl/proc/tasks_sys.go9
-rw-r--r--pkg/sentry/fsimpl/sys/sys.go14
-rw-r--r--pkg/sentry/fsimpl/sys/sys_test.go14
-rw-r--r--pkg/sentry/fsimpl/tmpfs/regular_file.go2
-rw-r--r--pkg/sentry/fsimpl/tmpfs/tmpfs.go14
-rw-r--r--pkg/sentry/fsimpl/verity/filesystem.go2
-rw-r--r--pkg/sentry/hostmm/BUILD3
-rw-r--r--pkg/sentry/hostmm/hostmm.go42
-rw-r--r--pkg/sentry/inet/BUILD13
-rw-r--r--pkg/sentry/kernel/BUILD3
-rw-r--r--pkg/sentry/kernel/epoll/epoll.go15
-rw-r--r--pkg/sentry/kernel/epoll/epoll_state.go2
-rw-r--r--pkg/sentry/kernel/eventfd/eventfd.go2
-rw-r--r--pkg/sentry/kernel/ipc/BUILD2
-rw-r--r--pkg/sentry/kernel/kernel.go46
-rw-r--r--pkg/sentry/kernel/mq/mq.go4
-rw-r--r--pkg/sentry/kernel/pipe/pipe.go2
-rw-r--r--pkg/sentry/kernel/task.go6
-rw-r--r--pkg/sentry/kernel/task_clone.go2
-rw-r--r--pkg/sentry/kernel/task_log.go12
-rw-r--r--pkg/sentry/kernel/task_net.go12
-rw-r--r--pkg/sentry/kernel/task_start.go2
-rw-r--r--pkg/sentry/kernel/threads.go6
-rw-r--r--pkg/sentry/mm/syscalls.go2
-rw-r--r--pkg/sentry/platform/kvm/BUILD24
-rw-r--r--pkg/sentry/platform/kvm/bluepill.go7
-rw-r--r--pkg/sentry/platform/kvm/bluepill_amd64.go8
-rw-r--r--pkg/sentry/platform/kvm/bluepill_amd64.s27
-rw-r--r--pkg/sentry/platform/kvm/bluepill_arm64.go10
-rw-r--r--pkg/sentry/platform/kvm/bluepill_arm64.s34
-rw-r--r--pkg/sentry/platform/kvm/bluepill_unsafe.go32
-rw-r--r--pkg/sentry/platform/kvm/kvm.go6
-rw-r--r--pkg/sentry/platform/kvm/kvm_safecopy_test.go104
-rw-r--r--pkg/sentry/platform/kvm/machine.go136
-rw-r--r--pkg/sentry/platform/kvm/machine_amd64.go36
-rw-r--r--pkg/sentry/platform/kvm/machine_amd64_unsafe.go12
-rw-r--r--pkg/sentry/platform/kvm/machine_arm64.go125
-rw-r--r--pkg/sentry/platform/kvm/machine_arm64_unsafe.go15
-rw-r--r--pkg/sentry/platform/kvm/machine_unsafe.go43
-rw-r--r--pkg/sentry/platform/kvm/physical_map.go3
-rw-r--r--pkg/sentry/platform/kvm/testutil/testutil_arm64.go4
-rw-r--r--pkg/sentry/platform/kvm/testutil/testutil_arm64.s35
-rw-r--r--pkg/sentry/seccheck/BUILD4
-rw-r--r--pkg/sentry/seccheck/execve.go65
-rw-r--r--pkg/sentry/seccheck/exit.go57
-rw-r--r--pkg/sentry/seccheck/seccheck.go26
-rw-r--r--pkg/sentry/sighandling/BUILD16
-rw-r--r--pkg/sentry/sighandling/sighandling.go102
-rw-r--r--pkg/sentry/sighandling/sighandling_unsafe.go39
-rw-r--r--pkg/sentry/socket/BUILD5
-rw-r--r--pkg/sentry/socket/control/control.go32
-rw-r--r--pkg/sentry/socket/control/control_test.go2
-rw-r--r--pkg/sentry/socket/hostinet/socket.go28
-rw-r--r--pkg/sentry/socket/netfilter/netfilter.go4
-rw-r--r--pkg/sentry/socket/netfilter/targets.go2
-rw-r--r--pkg/sentry/socket/netstack/BUILD1
-rw-r--r--pkg/sentry/socket/netstack/netstack.go137
-rw-r--r--pkg/sentry/socket/netstack/netstack_state.go31
-rw-r--r--pkg/sentry/socket/netstack/stack.go2
-rw-r--r--pkg/sentry/socket/socket.go37
-rw-r--r--pkg/sentry/socket/socket_state.go27
-rw-r--r--pkg/sentry/socket/unix/transport/queue.go8
-rw-r--r--pkg/sentry/strace/strace.go4
-rw-r--r--pkg/sentry/time/sampler_arm64.go4
-rw-r--r--pkg/sentry/usage/memory.go6
-rw-r--r--pkg/sentry/vfs/epoll.go6
-rw-r--r--pkg/sentry/vfs/file_description_impl_util.go4
-rw-r--r--pkg/sentry/vfs/resolving_path.go6
-rw-r--r--pkg/sentry/vfs/save_restore.go29
105 files changed, 2830 insertions, 907 deletions
diff --git a/pkg/sentry/control/pprof.go b/pkg/sentry/control/pprof.go
index 2f3664c57..f721b7236 100644
--- a/pkg/sentry/control/pprof.go
+++ b/pkg/sentry/control/pprof.go
@@ -26,6 +26,23 @@ import (
"gvisor.dev/gvisor/pkg/urpc"
)
+const (
+ // DefaultBlockProfileRate is the default profiling rate for block
+ // profiles.
+ //
+ // The default here is 10%, which will record a stacktrace 10% of the
+ // time when blocking occurs. Since these events should not be super
+ // frequent, we expect this to achieve a reasonable balance between
+ // collecting the data we need and imposing a high performance cost
+ // (e.g. skewing even the CPU profile).
+ DefaultBlockProfileRate = 10
+
+ // DefaultMutexProfileRate is the default profiling rate for mutex
+ // profiles. Like the block rate above, we use a default rate of 10%
+ // for the same reasons.
+ DefaultMutexProfileRate = 10
+)
+
// Profile includes profile-related RPC stubs. It provides a way to
// control the built-in runtime profiling facilities.
//
@@ -175,12 +192,8 @@ func (p *Profile) Block(o *BlockProfileOpts, _ *struct{}) error {
defer p.blockMu.Unlock()
// Always set the rate. We then wait to collect a profile at this rate,
- // and disable when we're done. Note that the default here is 10%, which
- // will record a stacktrace 10% of the time when blocking occurs. Since
- // these events should not be super frequent, we expect this to achieve
- // a reasonable balance between collecting the data we need and imposing
- // a high performance cost (e.g. skewing even the CPU profile).
- rate := 10
+ // and disable when we're done.
+ rate := DefaultBlockProfileRate
if o.Rate != 0 {
rate = o.Rate
}
@@ -220,9 +233,8 @@ func (p *Profile) Mutex(o *MutexProfileOpts, _ *struct{}) error {
p.mutexMu.Lock()
defer p.mutexMu.Unlock()
- // Always set the fraction. Like the block rate above, we use
- // a default rate of 10% for the same reasons.
- fraction := 10
+ // Always set the fraction.
+ fraction := DefaultMutexProfileRate
if o.Fraction != 0 {
fraction = o.Fraction
}
diff --git a/pkg/sentry/fs/fdpipe/pipe.go b/pkg/sentry/fs/fdpipe/pipe.go
index 4370cce33..d2eb03bb7 100644
--- a/pkg/sentry/fs/fdpipe/pipe.go
+++ b/pkg/sentry/fs/fdpipe/pipe.go
@@ -45,7 +45,8 @@ type pipeOperations struct {
fsutil.FileNoIoctl `state:"nosave"`
fsutil.FileNoSplice `state:"nosave"`
fsutil.FileUseInodeUnstableAttr `state:"nosave"`
- waiter.Queue `state:"nosave"`
+
+ waiter.Queue
// flags are the flags used to open the pipe.
flags fs.FileFlags `state:".(fs.FileFlags)"`
diff --git a/pkg/sentry/fs/file_overlay.go b/pkg/sentry/fs/file_overlay.go
index 031cd33ce..a27dd0b9a 100644
--- a/pkg/sentry/fs/file_overlay.go
+++ b/pkg/sentry/fs/file_overlay.go
@@ -16,6 +16,7 @@ package fs
import (
"io"
+ "math"
"gvisor.dev/gvisor/pkg/context"
"gvisor.dev/gvisor/pkg/errors/linuxerr"
@@ -360,10 +361,13 @@ func (*overlayFileOperations) ConfigureMMap(ctx context.Context, file *File, opt
return linuxerr.ENODEV
}
- // FIXME(jamieliu): This is a copy/paste of fsutil.GenericConfigureMMap,
- // which we can't use because the overlay implementation is in package fs,
- // so depending on fs/fsutil would create a circular dependency. Move
- // overlay to fs/overlay.
+ // TODO(gvisor.dev/issue/1624): This is a copy/paste of
+ // fsutil.GenericConfigureMMap, which we can't use because the overlay
+ // implementation is in package fs, so depending on fs/fsutil would create
+ // a circular dependency. VFS2 overlay doesn't have this issue.
+ if opts.Offset+opts.Length > math.MaxInt64 {
+ return linuxerr.EOVERFLOW
+ }
opts.Mappable = o
opts.MappingIdentity = file
file.IncRef()
diff --git a/pkg/sentry/fs/fsutil/file.go b/pkg/sentry/fs/fsutil/file.go
index 3ece73b81..38e3ed42d 100644
--- a/pkg/sentry/fs/fsutil/file.go
+++ b/pkg/sentry/fs/fsutil/file.go
@@ -16,6 +16,7 @@ package fsutil
import (
"io"
+ "math"
"gvisor.dev/gvisor/pkg/context"
"gvisor.dev/gvisor/pkg/errors/linuxerr"
@@ -210,6 +211,9 @@ func (FileNoMMap) ConfigureMMap(context.Context, *fs.File, *memmap.MMapOpts) err
// GenericConfigureMMap implements fs.FileOperations.ConfigureMMap for most
// filesystems that support memory mapping.
func GenericConfigureMMap(file *fs.File, m memmap.Mappable, opts *memmap.MMapOpts) error {
+ if opts.Offset+opts.Length > math.MaxInt64 {
+ return linuxerr.EOVERFLOW
+ }
opts.Mappable = m
opts.MappingIdentity = file
file.IncRef()
diff --git a/pkg/sentry/fs/fsutil/host_file_mapper.go b/pkg/sentry/fs/fsutil/host_file_mapper.go
index 23528bf25..37ddb1a3c 100644
--- a/pkg/sentry/fs/fsutil/host_file_mapper.go
+++ b/pkg/sentry/fs/fsutil/host_file_mapper.go
@@ -93,7 +93,8 @@ func NewHostFileMapper() *HostFileMapper {
func (f *HostFileMapper) IncRefOn(mr memmap.MappableRange) {
f.refsMu.Lock()
defer f.refsMu.Unlock()
- for chunkStart := mr.Start &^ chunkMask; chunkStart < mr.End; chunkStart += chunkSize {
+ chunkStart := mr.Start &^ chunkMask
+ for {
refs := f.refs[chunkStart]
pgs := pagesInChunk(mr, chunkStart)
if refs+pgs < refs {
@@ -101,6 +102,10 @@ func (f *HostFileMapper) IncRefOn(mr memmap.MappableRange) {
panic(fmt.Sprintf("HostFileMapper.IncRefOn(%v): adding %d page references to chunk %#x, which has %d page references", mr, pgs, chunkStart, refs))
}
f.refs[chunkStart] = refs + pgs
+ chunkStart += chunkSize
+ if chunkStart >= mr.End || chunkStart == 0 {
+ break
+ }
}
}
@@ -112,7 +117,8 @@ func (f *HostFileMapper) IncRefOn(mr memmap.MappableRange) {
func (f *HostFileMapper) DecRefOn(mr memmap.MappableRange) {
f.refsMu.Lock()
defer f.refsMu.Unlock()
- for chunkStart := mr.Start &^ chunkMask; chunkStart < mr.End; chunkStart += chunkSize {
+ chunkStart := mr.Start &^ chunkMask
+ for {
refs := f.refs[chunkStart]
pgs := pagesInChunk(mr, chunkStart)
switch {
@@ -128,6 +134,10 @@ func (f *HostFileMapper) DecRefOn(mr memmap.MappableRange) {
case refs < pgs:
panic(fmt.Sprintf("HostFileMapper.DecRefOn(%v): removing %d page references from chunk %#x, which has %d page references", mr, pgs, chunkStart, refs))
}
+ chunkStart += chunkSize
+ if chunkStart >= mr.End || chunkStart == 0 {
+ break
+ }
}
}
@@ -161,7 +171,8 @@ func (f *HostFileMapper) forEachMappingBlockLocked(fr memmap.FileRange, fd int,
if write {
prot |= unix.PROT_WRITE
}
- for chunkStart := fr.Start &^ chunkMask; chunkStart < fr.End; chunkStart += chunkSize {
+ chunkStart := fr.Start &^ chunkMask
+ for {
m, ok := f.mappings[chunkStart]
if !ok {
addr, _, errno := unix.Syscall6(
@@ -201,6 +212,10 @@ func (f *HostFileMapper) forEachMappingBlockLocked(fr memmap.FileRange, fd int,
endOff = fr.End - chunkStart
}
fn(f.unsafeBlockFromChunkMapping(m.addr).TakeFirst64(endOff).DropFirst64(startOff))
+ chunkStart += chunkSize
+ if chunkStart >= fr.End || chunkStart == 0 {
+ break
+ }
}
return nil
}
diff --git a/pkg/sentry/fs/host/inode.go b/pkg/sentry/fs/host/inode.go
index 92d58e3e9..99c37291e 100644
--- a/pkg/sentry/fs/host/inode.go
+++ b/pkg/sentry/fs/host/inode.go
@@ -70,7 +70,7 @@ type inodeFileState struct {
descriptor *descriptor `state:"wait"`
// Event queue for blocking operations.
- queue waiter.Queue `state:"zerovalue"`
+ queue waiter.Queue
// sattr is used to restore the inodeOperations.
sattr fs.StableAttr `state:"wait"`
diff --git a/pkg/sentry/fs/inotify.go b/pkg/sentry/fs/inotify.go
index 51cd6cd37..941f37116 100644
--- a/pkg/sentry/fs/inotify.go
+++ b/pkg/sentry/fs/inotify.go
@@ -43,7 +43,7 @@ type Inotify struct {
// user, since we may aggressively reuse an id on S/R.
id uint64
- waiter.Queue `state:"nosave"`
+ waiter.Queue
// evMu *only* protects the events list. We need a separate lock because
// while queuing events, a watch needs to lock the event queue, and using mu
diff --git a/pkg/sentry/fs/lock/lock.go b/pkg/sentry/fs/lock/lock.go
index 7d7a207cc..e39d340fe 100644
--- a/pkg/sentry/fs/lock/lock.go
+++ b/pkg/sentry/fs/lock/lock.go
@@ -132,7 +132,7 @@ type Locks struct {
locks LockSet
// blockedQueue is the queue of waiters that are waiting on a lock.
- blockedQueue waiter.Queue `state:"zerovalue"`
+ blockedQueue waiter.Queue
}
// Blocker is the interface used for blocking locks. Passing a nil Blocker
diff --git a/pkg/sentry/fs/proc/sys.go b/pkg/sentry/fs/proc/sys.go
index 085aa6d61..443b9a94c 100644
--- a/pkg/sentry/fs/proc/sys.go
+++ b/pkg/sentry/fs/proc/sys.go
@@ -109,6 +109,9 @@ func (p *proc) newKernelDir(ctx context.Context, msrc *fs.MountSource) *fs.Inode
"shmall": newStaticProcInode(ctx, msrc, []byte(strconv.FormatUint(linux.SHMALL, 10))),
"shmmax": newStaticProcInode(ctx, msrc, []byte(strconv.FormatUint(linux.SHMMAX, 10))),
"shmmni": newStaticProcInode(ctx, msrc, []byte(strconv.FormatUint(linux.SHMMNI, 10))),
+ "msgmni": newStaticProcInode(ctx, msrc, []byte(strconv.FormatUint(linux.MSGMNI, 10))),
+ "msgmax": newStaticProcInode(ctx, msrc, []byte(strconv.FormatUint(linux.MSGMAX, 10))),
+ "msgmnb": newStaticProcInode(ctx, msrc, []byte(strconv.FormatUint(linux.MSGMNB, 10))),
}
d := ramfs.NewDir(ctx, children, fs.RootOwner, fs.FilePermsFromMode(0555))
diff --git a/pkg/sentry/fs/timerfd/timerfd.go b/pkg/sentry/fs/timerfd/timerfd.go
index 1c8518d71..ca8be8683 100644
--- a/pkg/sentry/fs/timerfd/timerfd.go
+++ b/pkg/sentry/fs/timerfd/timerfd.go
@@ -43,7 +43,7 @@ type TimerOperations struct {
fsutil.FileNoopFlush `state:"nosave"`
fsutil.FileUseInodeUnstableAttr `state:"nosave"`
- events waiter.Queue `state:"zerovalue"`
+ events waiter.Queue
timer *ktime.Timer
// val is the number of timer expirations since the last successful call to
diff --git a/pkg/sentry/fs/tty/line_discipline.go b/pkg/sentry/fs/tty/line_discipline.go
index f9fca6d8e..f2c9e9668 100644
--- a/pkg/sentry/fs/tty/line_discipline.go
+++ b/pkg/sentry/fs/tty/line_discipline.go
@@ -102,10 +102,10 @@ type lineDiscipline struct {
column int
// masterWaiter is used to wait on the master end of the TTY.
- masterWaiter waiter.Queue `state:"zerovalue"`
+ masterWaiter waiter.Queue
// replicaWaiter is used to wait on the replica end of the TTY.
- replicaWaiter waiter.Queue `state:"zerovalue"`
+ replicaWaiter waiter.Queue
}
func newLineDiscipline(termios linux.KernelTermios) *lineDiscipline {
diff --git a/pkg/sentry/fsimpl/cgroupfs/BUILD b/pkg/sentry/fsimpl/cgroupfs/BUILD
index e5fdcc776..60ee5ede2 100644
--- a/pkg/sentry/fsimpl/cgroupfs/BUILD
+++ b/pkg/sentry/fsimpl/cgroupfs/BUILD
@@ -1,4 +1,4 @@
-load("//tools:defs.bzl", "go_library")
+load("//tools:defs.bzl", "go_library", "go_test")
load("//tools/go_generics:defs.bzl", "go_template_instance")
licenses(["notice"])
@@ -18,6 +18,7 @@ go_library(
name = "cgroupfs",
srcs = [
"base.go",
+ "bitmap.go",
"cgroupfs.go",
"cpu.go",
"cpuacct.go",
@@ -29,10 +30,12 @@ go_library(
visibility = ["//pkg/sentry:internal"],
deps = [
"//pkg/abi/linux",
+ "//pkg/bitmap",
"//pkg/context",
"//pkg/coverage",
"//pkg/errors/linuxerr",
"//pkg/fspath",
+ "//pkg/hostarch",
"//pkg/log",
"//pkg/refs",
"//pkg/refsvfs2",
@@ -47,3 +50,11 @@ go_library(
"//pkg/usermem",
],
)
+
+go_test(
+ name = "cgroupfs_test",
+ size = "small",
+ srcs = ["bitmap_test.go"],
+ library = ":cgroupfs",
+ deps = ["//pkg/bitmap"],
+)
diff --git a/pkg/sentry/fsimpl/cgroupfs/bitmap.go b/pkg/sentry/fsimpl/cgroupfs/bitmap.go
new file mode 100644
index 000000000..8074641db
--- /dev/null
+++ b/pkg/sentry/fsimpl/cgroupfs/bitmap.go
@@ -0,0 +1,139 @@
+// Copyright 2021 The gVisor Authors.
+//
+// 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 cgroupfs
+
+import (
+ "fmt"
+ "strconv"
+ "strings"
+
+ "gvisor.dev/gvisor/pkg/bitmap"
+)
+
+// formatBitmap produces a string representation of b, which lists the indicies
+// of set bits in the bitmap. Indicies are separated by commas and ranges of
+// set bits are abbreviated. Example outputs: "0,2,4", "0,3-7,10", "0-10".
+//
+// Inverse of parseBitmap.
+func formatBitmap(b *bitmap.Bitmap) string {
+ ones := b.ToSlice()
+ if len(ones) == 0 {
+ return ""
+ }
+
+ elems := make([]string, 0, len(ones))
+ runStart := ones[0]
+ lastVal := ones[0]
+ inRun := false
+
+ for _, v := range ones[1:] {
+ last := lastVal
+ lastVal = v
+
+ if last+1 == v {
+ // In a contiguous block of ones.
+ if !inRun {
+ runStart = last
+ inRun = true
+ }
+
+ continue
+ }
+
+ // Non-contiguous bit.
+ if inRun {
+ // Render a run
+ elems = append(elems, fmt.Sprintf("%d-%d", runStart, last))
+ inRun = false
+ continue
+ }
+
+ // Lone non-contiguous bit.
+ elems = append(elems, fmt.Sprintf("%d", last))
+
+ }
+
+ // Process potential final run
+ if inRun {
+ elems = append(elems, fmt.Sprintf("%d-%d", runStart, lastVal))
+ } else {
+ elems = append(elems, fmt.Sprintf("%d", lastVal))
+ }
+
+ return strings.Join(elems, ",")
+}
+
+func parseToken(token string) (start, end uint32, err error) {
+ ts := strings.SplitN(token, "-", 2)
+ switch len(ts) {
+ case 0:
+ return 0, 0, fmt.Errorf("invalid token %q", token)
+ case 1:
+ val, err := strconv.ParseUint(ts[0], 10, 32)
+ if err != nil {
+ return 0, 0, err
+ }
+ return uint32(val), uint32(val), nil
+ case 2:
+ val1, err := strconv.ParseUint(ts[0], 10, 32)
+ if err != nil {
+ return 0, 0, err
+ }
+ val2, err := strconv.ParseUint(ts[1], 10, 32)
+ if err != nil {
+ return 0, 0, err
+ }
+ if val1 >= val2 {
+ return 0, 0, fmt.Errorf("start (%v) must be less than end (%v)", val1, val2)
+ }
+ return uint32(val1), uint32(val2), nil
+ default:
+ panic(fmt.Sprintf("Unreachable: got %d substrs", len(ts)))
+ }
+}
+
+// parseBitmap parses input as a bitmap. input should be a comma separated list
+// of indices, and ranges of set bits may be abbreviated. Examples: "0,2,4",
+// "0,3-7,10", "0-10". Input after the first newline or null byte is discarded.
+//
+// sizeHint sets the initial size of the bitmap, which may prevent reallocation
+// when growing the bitmap during parsing. Ideally sizeHint should be at least
+// as large as the bitmap represented by input, but this is not required.
+//
+// Inverse of formatBitmap.
+func parseBitmap(input string, sizeHint uint32) (*bitmap.Bitmap, error) {
+ b := bitmap.New(sizeHint)
+
+ if termIdx := strings.IndexAny(input, "\n\000"); termIdx != -1 {
+ input = input[:termIdx]
+ }
+ input = strings.TrimSpace(input)
+
+ if len(input) == 0 {
+ return &b, nil
+ }
+ tokens := strings.Split(input, ",")
+
+ for _, t := range tokens {
+ start, end, err := parseToken(strings.TrimSpace(t))
+ if err != nil {
+ return nil, err
+ }
+ for i := start; i <= end; i++ {
+ b.Add(i)
+ }
+ }
+ return &b, nil
+}
diff --git a/pkg/sentry/fsimpl/cgroupfs/bitmap_test.go b/pkg/sentry/fsimpl/cgroupfs/bitmap_test.go
new file mode 100644
index 000000000..5cc56de3b
--- /dev/null
+++ b/pkg/sentry/fsimpl/cgroupfs/bitmap_test.go
@@ -0,0 +1,99 @@
+// Copyright 2021 The gVisor Authors.
+//
+// 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 cgroupfs
+
+import (
+ "fmt"
+ "reflect"
+ "testing"
+
+ "gvisor.dev/gvisor/pkg/bitmap"
+)
+
+func TestFormat(t *testing.T) {
+ tests := []struct {
+ input []uint32
+ output string
+ }{
+ {[]uint32{1, 2, 3, 4, 7}, "1-4,7"},
+ {[]uint32{2}, "2"},
+ {[]uint32{0, 1, 2}, "0-2"},
+ {[]uint32{}, ""},
+ {[]uint32{1, 3, 4, 5, 6, 9, 11, 13, 14, 15, 16, 17}, "1,3-6,9,11,13-17"},
+ {[]uint32{2, 3, 10, 12, 13, 14, 15, 16, 20, 21, 33, 34, 47}, "2-3,10,12-16,20-21,33-34,47"},
+ }
+ for i, tt := range tests {
+ t.Run(fmt.Sprintf("case-%d", i), func(t *testing.T) {
+ b := bitmap.New(64)
+ for _, v := range tt.input {
+ b.Add(v)
+ }
+ s := formatBitmap(&b)
+ if s != tt.output {
+ t.Errorf("Expected %q, got %q", tt.output, s)
+ }
+ b1, err := parseBitmap(s, 64)
+ if err != nil {
+ t.Fatalf("Failed to parse formatted bitmap: %v", err)
+ }
+ if got, want := b1.ToSlice(), b.ToSlice(); !reflect.DeepEqual(got, want) {
+ t.Errorf("Parsing formatted output doesn't result in the original bitmap. Got %v, want %v", got, want)
+ }
+ })
+ }
+}
+
+func TestParse(t *testing.T) {
+ tests := []struct {
+ input string
+ output []uint32
+ shouldFail bool
+ }{
+ {"1", []uint32{1}, false},
+ {"", []uint32{}, false},
+ {"1,2,3,4", []uint32{1, 2, 3, 4}, false},
+ {"1-4", []uint32{1, 2, 3, 4}, false},
+ {"1,2-4", []uint32{1, 2, 3, 4}, false},
+ {"1,2-3,4", []uint32{1, 2, 3, 4}, false},
+ {"1-2,3,4,10,11", []uint32{1, 2, 3, 4, 10, 11}, false},
+ {"1,2-4,5,16", []uint32{1, 2, 3, 4, 5, 16}, false},
+ {"abc", []uint32{}, true},
+ {"1,3-2,4", []uint32{}, true},
+ {"1,3-3,4", []uint32{}, true},
+ {"1,2,3\000,4", []uint32{1, 2, 3}, false},
+ {"1,2,3\n,4", []uint32{1, 2, 3}, false},
+ }
+ for i, tt := range tests {
+ t.Run(fmt.Sprintf("case-%d", i), func(t *testing.T) {
+ b, err := parseBitmap(tt.input, 64)
+ if tt.shouldFail {
+ if err == nil {
+ t.Fatalf("Expected parsing of %q to fail, but it didn't", tt.input)
+ }
+ return
+ }
+ if err != nil {
+ t.Fatalf("Failed to parse bitmap: %v", err)
+ return
+ }
+
+ got := b.ToSlice()
+ if !reflect.DeepEqual(got, tt.output) {
+ t.Errorf("Parsed bitmap doesn't match what we expected. Got %v, want %v", got, tt.output)
+ }
+
+ })
+ }
+}
diff --git a/pkg/sentry/fsimpl/cgroupfs/cgroupfs.go b/pkg/sentry/fsimpl/cgroupfs/cgroupfs.go
index edc3b50b9..e089b2c28 100644
--- a/pkg/sentry/fsimpl/cgroupfs/cgroupfs.go
+++ b/pkg/sentry/fsimpl/cgroupfs/cgroupfs.go
@@ -269,7 +269,7 @@ func (fsType FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.Virt
case controllerCPUAcct:
c = newCPUAcctController(fs)
case controllerCPUSet:
- c = newCPUSetController(fs)
+ c = newCPUSetController(k, fs)
case controllerJob:
c = newJobController(fs)
case controllerMemory:
diff --git a/pkg/sentry/fsimpl/cgroupfs/cpuset.go b/pkg/sentry/fsimpl/cgroupfs/cpuset.go
index ac547f8e2..62e7029da 100644
--- a/pkg/sentry/fsimpl/cgroupfs/cpuset.go
+++ b/pkg/sentry/fsimpl/cgroupfs/cpuset.go
@@ -15,25 +15,133 @@
package cgroupfs
import (
+ "bytes"
+ "fmt"
+
+ "gvisor.dev/gvisor/pkg/bitmap"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
+ "gvisor.dev/gvisor/pkg/hostarch"
+ "gvisor.dev/gvisor/pkg/log"
"gvisor.dev/gvisor/pkg/sentry/fsimpl/kernfs"
+ "gvisor.dev/gvisor/pkg/sentry/kernel"
"gvisor.dev/gvisor/pkg/sentry/kernel/auth"
+ "gvisor.dev/gvisor/pkg/usermem"
)
// +stateify savable
type cpusetController struct {
controllerCommon
+
+ maxCpus uint32
+ maxMems uint32
+
+ cpus *bitmap.Bitmap
+ mems *bitmap.Bitmap
}
var _ controller = (*cpusetController)(nil)
-func newCPUSetController(fs *filesystem) *cpusetController {
- c := &cpusetController{}
+func newCPUSetController(k *kernel.Kernel, fs *filesystem) *cpusetController {
+ cores := uint32(k.ApplicationCores())
+ cpus := bitmap.New(cores)
+ cpus.FlipRange(0, cores)
+ mems := bitmap.New(1)
+ mems.FlipRange(0, 1)
+ c := &cpusetController{
+ cpus: &cpus,
+ mems: &mems,
+ maxCpus: uint32(k.ApplicationCores()),
+ maxMems: 1, // We always report a single NUMA node.
+ }
c.controllerCommon.init(controllerCPUSet, fs)
return c
}
// AddControlFiles implements controller.AddControlFiles.
func (c *cpusetController) AddControlFiles(ctx context.Context, creds *auth.Credentials, _ *cgroupInode, contents map[string]kernfs.Inode) {
- // This controller is currently intentionally empty.
+ contents["cpuset.cpus"] = c.fs.newControllerWritableFile(ctx, creds, &cpusData{c: c})
+ contents["cpuset.mems"] = c.fs.newControllerWritableFile(ctx, creds, &memsData{c: c})
+}
+
+// +stateify savable
+type cpusData struct {
+ c *cpusetController
+}
+
+// Generate implements vfs.DynamicBytesSource.Generate.
+func (d *cpusData) Generate(ctx context.Context, buf *bytes.Buffer) error {
+ fmt.Fprintf(buf, "%s\n", formatBitmap(d.c.cpus))
+ return nil
+}
+
+// Write implements vfs.WritableDynamicBytesSource.Write.
+func (d *cpusData) Write(ctx context.Context, src usermem.IOSequence, offset int64) (int64, error) {
+ src = src.DropFirst64(offset)
+ if src.NumBytes() > hostarch.PageSize {
+ return 0, linuxerr.EINVAL
+ }
+
+ t := kernel.TaskFromContext(ctx)
+ buf := t.CopyScratchBuffer(hostarch.PageSize)
+ n, err := src.CopyIn(ctx, buf)
+ if err != nil {
+ return 0, err
+ }
+ buf = buf[:n]
+
+ b, err := parseBitmap(string(buf), d.c.maxCpus)
+ if err != nil {
+ log.Warningf("cgroupfs cpuset controller: Failed to parse bitmap: %v", err)
+ return 0, linuxerr.EINVAL
+ }
+
+ if got, want := b.Maximum(), d.c.maxCpus; got > want {
+ log.Warningf("cgroupfs cpuset controller: Attempted to specify cpuset.cpus beyond highest available cpu: got %d, want %d", got, want)
+ return 0, linuxerr.EINVAL
+ }
+
+ d.c.cpus = b
+ return int64(n), nil
+}
+
+// +stateify savable
+type memsData struct {
+ c *cpusetController
+}
+
+// Generate implements vfs.DynamicBytesSource.Generate.
+func (d *memsData) Generate(ctx context.Context, buf *bytes.Buffer) error {
+ fmt.Fprintf(buf, "%s\n", formatBitmap(d.c.mems))
+ return nil
+}
+
+// Write implements vfs.WritableDynamicBytesSource.Write.
+func (d *memsData) Write(ctx context.Context, src usermem.IOSequence, offset int64) (int64, error) {
+ src = src.DropFirst64(offset)
+ if src.NumBytes() > hostarch.PageSize {
+ return 0, linuxerr.EINVAL
+ }
+
+ t := kernel.TaskFromContext(ctx)
+ buf := t.CopyScratchBuffer(hostarch.PageSize)
+ n, err := src.CopyIn(ctx, buf)
+ if err != nil {
+ return 0, err
+ }
+ buf = buf[:n]
+
+ b, err := parseBitmap(string(buf), d.c.maxMems)
+ if err != nil {
+ log.Warningf("cgroupfs cpuset controller: Failed to parse bitmap: %v", err)
+ return 0, linuxerr.EINVAL
+ }
+
+ if got, want := b.Maximum(), d.c.maxMems; got > want {
+ log.Warningf("cgroupfs cpuset controller: Attempted to specify cpuset.mems beyond highest available node: got %d, want %d", got, want)
+ return 0, linuxerr.EINVAL
+ }
+
+ d.c.mems = b
+ return int64(n), nil
}
diff --git a/pkg/sentry/fsimpl/gofer/BUILD b/pkg/sentry/fsimpl/gofer/BUILD
index 4244f2cf5..509dd0e1a 100644
--- a/pkg/sentry/fsimpl/gofer/BUILD
+++ b/pkg/sentry/fsimpl/gofer/BUILD
@@ -54,7 +54,10 @@ go_library(
"//pkg/fdnotifier",
"//pkg/fspath",
"//pkg/hostarch",
+ "//pkg/lisafs",
"//pkg/log",
+ "//pkg/marshal",
+ "//pkg/marshal/primitive",
"//pkg/metric",
"//pkg/p9",
"//pkg/refs",
diff --git a/pkg/sentry/fsimpl/gofer/directory.go b/pkg/sentry/fsimpl/gofer/directory.go
index 5c48a9fee..d99a6112c 100644
--- a/pkg/sentry/fsimpl/gofer/directory.go
+++ b/pkg/sentry/fsimpl/gofer/directory.go
@@ -222,47 +222,88 @@ func (d *dentry) getDirents(ctx context.Context) ([]vfs.Dirent, error) {
off := uint64(0)
const count = 64 * 1024 // for consistency with the vfs1 client
d.handleMu.RLock()
- if d.readFile.isNil() {
+ if !d.isReadFileOk() {
// This should not be possible because a readable handle should
// have been opened when the calling directoryFD was opened.
d.handleMu.RUnlock()
panic("gofer.dentry.getDirents called without a readable handle")
}
+ // shouldSeek0 indicates whether the server should SEEK to 0 before reading
+ // directory entries.
+ shouldSeek0 := true
for {
- p9ds, err := d.readFile.readdir(ctx, off, count)
- if err != nil {
- d.handleMu.RUnlock()
- return nil, err
- }
- if len(p9ds) == 0 {
- d.handleMu.RUnlock()
- break
- }
- for _, p9d := range p9ds {
- if p9d.Name == "." || p9d.Name == ".." {
- continue
+ if d.fs.opts.lisaEnabled {
+ countLisa := int32(count)
+ if shouldSeek0 {
+ // See lisafs.Getdents64Req.Count.
+ countLisa = -countLisa
+ shouldSeek0 = false
+ }
+ lisafsDs, err := d.readFDLisa.Getdents64(ctx, countLisa)
+ if err != nil {
+ d.handleMu.RUnlock()
+ return nil, err
+ }
+ if len(lisafsDs) == 0 {
+ d.handleMu.RUnlock()
+ break
+ }
+ for i := range lisafsDs {
+ name := string(lisafsDs[i].Name)
+ if name == "." || name == ".." {
+ continue
+ }
+ dirent := vfs.Dirent{
+ Name: name,
+ Ino: d.fs.inoFromKey(inoKey{
+ ino: uint64(lisafsDs[i].Ino),
+ devMinor: uint32(lisafsDs[i].DevMinor),
+ devMajor: uint32(lisafsDs[i].DevMajor),
+ }),
+ NextOff: int64(len(dirents) + 1),
+ Type: uint8(lisafsDs[i].Type),
+ }
+ dirents = append(dirents, dirent)
+ if realChildren != nil {
+ realChildren[name] = struct{}{}
+ }
}
- dirent := vfs.Dirent{
- Name: p9d.Name,
- Ino: d.fs.inoFromQIDPath(p9d.QID.Path),
- NextOff: int64(len(dirents) + 1),
+ } else {
+ p9ds, err := d.readFile.readdir(ctx, off, count)
+ if err != nil {
+ d.handleMu.RUnlock()
+ return nil, err
}
- // p9 does not expose 9P2000.U's DMDEVICE, DMNAMEDPIPE, or
- // DMSOCKET.
- switch p9d.Type {
- case p9.TypeSymlink:
- dirent.Type = linux.DT_LNK
- case p9.TypeDir:
- dirent.Type = linux.DT_DIR
- default:
- dirent.Type = linux.DT_REG
+ if len(p9ds) == 0 {
+ d.handleMu.RUnlock()
+ break
}
- dirents = append(dirents, dirent)
- if realChildren != nil {
- realChildren[p9d.Name] = struct{}{}
+ for _, p9d := range p9ds {
+ if p9d.Name == "." || p9d.Name == ".." {
+ continue
+ }
+ dirent := vfs.Dirent{
+ Name: p9d.Name,
+ Ino: d.fs.inoFromQIDPath(p9d.QID.Path),
+ NextOff: int64(len(dirents) + 1),
+ }
+ // p9 does not expose 9P2000.U's DMDEVICE, DMNAMEDPIPE, or
+ // DMSOCKET.
+ switch p9d.Type {
+ case p9.TypeSymlink:
+ dirent.Type = linux.DT_LNK
+ case p9.TypeDir:
+ dirent.Type = linux.DT_DIR
+ default:
+ dirent.Type = linux.DT_REG
+ }
+ dirents = append(dirents, dirent)
+ if realChildren != nil {
+ realChildren[p9d.Name] = struct{}{}
+ }
}
+ off = p9ds[len(p9ds)-1].Offset
}
- off = p9ds[len(p9ds)-1].Offset
}
}
// Emit entries for synthetic children.
diff --git a/pkg/sentry/fsimpl/gofer/filesystem.go b/pkg/sentry/fsimpl/gofer/filesystem.go
index 00228c469..23c8b8ce3 100644
--- a/pkg/sentry/fsimpl/gofer/filesystem.go
+++ b/pkg/sentry/fsimpl/gofer/filesystem.go
@@ -21,10 +21,12 @@ import (
"sync"
"sync/atomic"
+ "golang.org/x/sys/unix"
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
"gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/fspath"
+ "gvisor.dev/gvisor/pkg/lisafs"
"gvisor.dev/gvisor/pkg/p9"
"gvisor.dev/gvisor/pkg/sentry/fsimpl/host"
"gvisor.dev/gvisor/pkg/sentry/fsmetric"
@@ -53,9 +55,47 @@ func (fs *filesystem) Sync(ctx context.Context) error {
// regardless.
var retErr error
+ if fs.opts.lisaEnabled {
+ // Try accumulating all FDIDs to fsync and fsync then via one RPC as
+ // opposed to making an RPC per FDID. Passing a non-nil accFsyncFDIDs to
+ // dentry.syncCachedFile() and specialFileFD.sync() will cause them to not
+ // make an RPC, instead accumulate syncable FDIDs in the passed slice.
+ accFsyncFDIDs := make([]lisafs.FDID, 0, len(ds)+len(sffds))
+
+ // Sync syncable dentries.
+ for _, d := range ds {
+ if err := d.syncCachedFile(ctx, true /* forFilesystemSync */, &accFsyncFDIDs); err != nil {
+ ctx.Infof("gofer.filesystem.Sync: dentry.syncCachedFile failed: %v", err)
+ if retErr == nil {
+ retErr = err
+ }
+ }
+ }
+
+ // Sync special files, which may be writable but do not use dentry shared
+ // handles (so they won't be synced by the above).
+ for _, sffd := range sffds {
+ if err := sffd.sync(ctx, true /* forFilesystemSync */, &accFsyncFDIDs); err != nil {
+ ctx.Infof("gofer.filesystem.Sync: specialFileFD.sync failed: %v", err)
+ if retErr == nil {
+ retErr = err
+ }
+ }
+ }
+
+ if err := fs.clientLisa.SyncFDs(ctx, accFsyncFDIDs); err != nil {
+ ctx.Infof("gofer.filesystem.Sync: fs.fsyncMultipleFDLisa failed: %v", err)
+ if retErr == nil {
+ retErr = err
+ }
+ }
+
+ return retErr
+ }
+
// Sync syncable dentries.
for _, d := range ds {
- if err := d.syncCachedFile(ctx, true /* forFilesystemSync */); err != nil {
+ if err := d.syncCachedFile(ctx, true /* forFilesystemSync */, nil /* accFsyncFDIDsLisa */); err != nil {
ctx.Infof("gofer.filesystem.Sync: dentry.syncCachedFile failed: %v", err)
if retErr == nil {
retErr = err
@@ -66,7 +106,7 @@ func (fs *filesystem) Sync(ctx context.Context) error {
// Sync special files, which may be writable but do not use dentry shared
// handles (so they won't be synced by the above).
for _, sffd := range sffds {
- if err := sffd.sync(ctx, true /* forFilesystemSync */); err != nil {
+ if err := sffd.sync(ctx, true /* forFilesystemSync */, nil /* accFsyncFDIDsLisa */); err != nil {
ctx.Infof("gofer.filesystem.Sync: specialFileFD.sync failed: %v", err)
if retErr == nil {
retErr = err
@@ -130,7 +170,7 @@ func putDentrySlice(ds *[]*dentry) {
// but dentry slices are allocated lazily, and it's much easier to say "defer
// fs.renameMuRUnlockAndCheckCaching(&ds)" than "defer func() {
// fs.renameMuRUnlockAndCheckCaching(ds) }()" to work around this.
-// +checklocksrelease:fs.renameMu
+// +checklocksreleaseread:fs.renameMu
func (fs *filesystem) renameMuRUnlockAndCheckCaching(ctx context.Context, dsp **[]*dentry) {
fs.renameMu.RUnlock()
if *dsp == nil {
@@ -197,7 +237,13 @@ afterSymlink:
rp.Advance()
return d.parent, followedSymlink, nil
}
- child, err := fs.getChildLocked(ctx, d, name, ds)
+ var child *dentry
+ var err error
+ if fs.opts.lisaEnabled {
+ child, err = fs.getChildAndWalkPathLocked(ctx, d, rp, ds)
+ } else {
+ child, err = fs.getChildLocked(ctx, d, name, ds)
+ }
if err != nil {
return nil, false, err
}
@@ -219,6 +265,99 @@ afterSymlink:
return child, followedSymlink, nil
}
+// Preconditions:
+// * fs.opts.lisaEnabled.
+// * fs.renameMu must be locked.
+// * parent.dirMu must be locked.
+// * parent.isDir().
+// * parent and the dentry at name have been revalidated.
+func (fs *filesystem) getChildAndWalkPathLocked(ctx context.Context, parent *dentry, rp *vfs.ResolvingPath, ds **[]*dentry) (*dentry, error) {
+ // Note that pit is a copy of the iterator that does not affect rp.
+ pit := rp.Pit()
+ first := pit.String()
+ if len(first) > maxFilenameLen {
+ return nil, linuxerr.ENAMETOOLONG
+ }
+ if child, ok := parent.children[first]; ok || parent.isSynthetic() {
+ if child == nil {
+ return nil, linuxerr.ENOENT
+ }
+ return child, nil
+ }
+
+ // Walk as much of the path as possible in 1 RPC.
+ names := []string{first}
+ for pit = pit.Next(); pit.Ok(); pit = pit.Next() {
+ name := pit.String()
+ if name == "." {
+ continue
+ }
+ if name == ".." {
+ break
+ }
+ names = append(names, name)
+ }
+ status, inodes, err := parent.controlFDLisa.WalkMultiple(ctx, names)
+ if err != nil {
+ return nil, err
+ }
+ if len(inodes) == 0 {
+ parent.cacheNegativeLookupLocked(first)
+ return nil, linuxerr.ENOENT
+ }
+
+ // Add the walked inodes into the dentry tree.
+ curParent := parent
+ curParentDirMuLock := func() {
+ if curParent != parent {
+ curParent.dirMu.Lock()
+ }
+ }
+ curParentDirMuUnlock := func() {
+ if curParent != parent {
+ curParent.dirMu.Unlock() // +checklocksforce: locked via curParentDirMuLock().
+ }
+ }
+ var ret *dentry
+ var dentryCreationErr error
+ for i := range inodes {
+ if dentryCreationErr != nil {
+ fs.clientLisa.CloseFDBatched(ctx, inodes[i].ControlFD)
+ continue
+ }
+
+ child, err := fs.newDentryLisa(ctx, &inodes[i])
+ if err != nil {
+ fs.clientLisa.CloseFDBatched(ctx, inodes[i].ControlFD)
+ dentryCreationErr = err
+ continue
+ }
+ curParentDirMuLock()
+ curParent.cacheNewChildLocked(child, names[i])
+ curParentDirMuUnlock()
+ // For now, child has 0 references, so our caller should call
+ // child.checkCachingLocked(). curParent gained a ref so we should also
+ // call curParent.checkCachingLocked() so it can be removed from the cache
+ // if needed. We only do that for the first iteration because all
+ // subsequent parents would have already been added to ds.
+ if i == 0 {
+ *ds = appendDentry(*ds, curParent)
+ }
+ *ds = appendDentry(*ds, child)
+ curParent = child
+ if i == 0 {
+ ret = child
+ }
+ }
+
+ if status == lisafs.WalkComponentDoesNotExist && curParent.isDir() {
+ curParentDirMuLock()
+ curParent.cacheNegativeLookupLocked(names[len(inodes)])
+ curParentDirMuUnlock()
+ }
+ return ret, dentryCreationErr
+}
+
// getChildLocked returns a dentry representing the child of parent with the
// given name. Returns ENOENT if the child doesn't exist.
//
@@ -227,7 +366,7 @@ afterSymlink:
// * parent.dirMu must be locked.
// * parent.isDir().
// * name is not "." or "..".
-// * dentry at name has been revalidated
+// * parent and the dentry at name have been revalidated.
func (fs *filesystem) getChildLocked(ctx context.Context, parent *dentry, name string, ds **[]*dentry) (*dentry, error) {
if len(name) > maxFilenameLen {
return nil, linuxerr.ENAMETOOLONG
@@ -239,20 +378,35 @@ func (fs *filesystem) getChildLocked(ctx context.Context, parent *dentry, name s
return child, nil
}
- qid, file, attrMask, attr, err := parent.file.walkGetAttrOne(ctx, name)
- if err != nil {
- if linuxerr.Equals(linuxerr.ENOENT, err) {
- parent.cacheNegativeLookupLocked(name)
+ var child *dentry
+ if fs.opts.lisaEnabled {
+ childInode, err := parent.controlFDLisa.Walk(ctx, name)
+ if err != nil {
+ if linuxerr.Equals(linuxerr.ENOENT, err) {
+ parent.cacheNegativeLookupLocked(name)
+ }
+ return nil, err
+ }
+ // Create a new dentry representing the file.
+ child, err = fs.newDentryLisa(ctx, childInode)
+ if err != nil {
+ fs.clientLisa.CloseFDBatched(ctx, childInode.ControlFD)
+ return nil, err
+ }
+ } else {
+ qid, file, attrMask, attr, err := parent.file.walkGetAttrOne(ctx, name)
+ if err != nil {
+ if linuxerr.Equals(linuxerr.ENOENT, err) {
+ parent.cacheNegativeLookupLocked(name)
+ }
+ return nil, err
+ }
+ // Create a new dentry representing the file.
+ child, err = fs.newDentry(ctx, file, qid, attrMask, &attr)
+ if err != nil {
+ file.close(ctx)
+ return nil, err
}
- return nil, err
- }
-
- // Create a new dentry representing the file.
- child, err := fs.newDentry(ctx, file, qid, attrMask, &attr)
- if err != nil {
- file.close(ctx)
- delete(parent.children, name)
- return nil, err
}
parent.cacheNewChildLocked(child, name)
appendNewChildDentry(ds, parent, child)
@@ -328,7 +482,7 @@ func (fs *filesystem) resolveLocked(ctx context.Context, rp *vfs.ResolvingPath,
// Preconditions:
// * !rp.Done().
// * For the final path component in rp, !rp.ShouldFollowSymlink().
-func (fs *filesystem) doCreateAt(ctx context.Context, rp *vfs.ResolvingPath, dir bool, createInRemoteDir func(parent *dentry, name string, ds **[]*dentry) error, createInSyntheticDir func(parent *dentry, name string) error) error {
+func (fs *filesystem) doCreateAt(ctx context.Context, rp *vfs.ResolvingPath, dir bool, createInRemoteDir func(parent *dentry, name string, ds **[]*dentry) (*lisafs.Inode, error), createInSyntheticDir func(parent *dentry, name string) error, updateChild func(child *dentry)) error {
var ds *[]*dentry
fs.renameMu.RLock()
defer fs.renameMuRUnlockAndCheckCaching(ctx, &ds)
@@ -415,9 +569,26 @@ func (fs *filesystem) doCreateAt(ctx context.Context, rp *vfs.ResolvingPath, dir
// No cached dentry exists; however, in InteropModeShared there might still be
// an existing file at name. Just attempt the file creation RPC anyways. If a
// file does exist, the RPC will fail with EEXIST like we would have.
- if err := createInRemoteDir(parent, name, &ds); err != nil {
+ lisaInode, err := createInRemoteDir(parent, name, &ds)
+ if err != nil {
return err
}
+ // lisafs may aggresively cache newly created inodes. This has helped reduce
+ // Walk RPCs in practice.
+ if lisaInode != nil {
+ child, err := fs.newDentryLisa(ctx, lisaInode)
+ if err != nil {
+ fs.clientLisa.CloseFDBatched(ctx, lisaInode.ControlFD)
+ return err
+ }
+ parent.cacheNewChildLocked(child, name)
+ appendNewChildDentry(&ds, parent, child)
+
+ // lisafs may update dentry properties upon successful creation.
+ if updateChild != nil {
+ updateChild(child)
+ }
+ }
if fs.opts.interop != InteropModeShared {
if child, ok := parent.children[name]; ok && child == nil {
// Delete the now-stale negative dentry.
@@ -565,7 +736,11 @@ func (fs *filesystem) unlinkAt(ctx context.Context, rp *vfs.ResolvingPath, dir b
return linuxerr.ENOENT
}
} else if child == nil || !child.isSynthetic() {
- err = parent.file.unlinkAt(ctx, name, flags)
+ if fs.opts.lisaEnabled {
+ err = parent.controlFDLisa.UnlinkAt(ctx, name, flags)
+ } else {
+ err = parent.file.unlinkAt(ctx, name, flags)
+ }
if err != nil {
if child != nil {
vfsObj.AbortDeleteDentry(&child.vfsd) // +checklocksforce: see above.
@@ -658,40 +833,43 @@ func (fs *filesystem) GetParentDentryAt(ctx context.Context, rp *vfs.ResolvingPa
// LinkAt implements vfs.FilesystemImpl.LinkAt.
func (fs *filesystem) LinkAt(ctx context.Context, rp *vfs.ResolvingPath, vd vfs.VirtualDentry) error {
- return fs.doCreateAt(ctx, rp, false /* dir */, func(parent *dentry, childName string, _ **[]*dentry) error {
+ err := fs.doCreateAt(ctx, rp, false /* dir */, func(parent *dentry, childName string, ds **[]*dentry) (*lisafs.Inode, error) {
if rp.Mount() != vd.Mount() {
- return linuxerr.EXDEV
+ return nil, linuxerr.EXDEV
}
d := vd.Dentry().Impl().(*dentry)
if d.isDir() {
- return linuxerr.EPERM
+ return nil, linuxerr.EPERM
}
gid := auth.KGID(atomic.LoadUint32(&d.gid))
uid := auth.KUID(atomic.LoadUint32(&d.uid))
mode := linux.FileMode(atomic.LoadUint32(&d.mode))
if err := vfs.MayLink(rp.Credentials(), mode, uid, gid); err != nil {
- return err
+ return nil, err
}
if d.nlink == 0 {
- return linuxerr.ENOENT
+ return nil, linuxerr.ENOENT
}
if d.nlink == math.MaxUint32 {
- return linuxerr.EMLINK
+ return nil, linuxerr.EMLINK
}
- if err := parent.file.link(ctx, d.file, childName); err != nil {
- return err
+ if fs.opts.lisaEnabled {
+ return parent.controlFDLisa.LinkAt(ctx, d.controlFDLisa.ID(), childName)
}
+ return nil, parent.file.link(ctx, d.file, childName)
+ }, nil, nil)
+ if err == nil {
// Success!
- atomic.AddUint32(&d.nlink, 1)
- return nil
- }, nil)
+ vd.Dentry().Impl().(*dentry).incLinks()
+ }
+ return err
}
// MkdirAt implements vfs.FilesystemImpl.MkdirAt.
func (fs *filesystem) MkdirAt(ctx context.Context, rp *vfs.ResolvingPath, opts vfs.MkdirOptions) error {
creds := rp.Credentials()
- return fs.doCreateAt(ctx, rp, true /* dir */, func(parent *dentry, name string, ds **[]*dentry) error {
+ return fs.doCreateAt(ctx, rp, true /* dir */, func(parent *dentry, name string, ds **[]*dentry) (*lisafs.Inode, error) {
// If the parent is a setgid directory, use the parent's GID
// rather than the caller's and enable setgid.
kgid := creds.EffectiveKGID
@@ -700,23 +878,37 @@ func (fs *filesystem) MkdirAt(ctx context.Context, rp *vfs.ResolvingPath, opts v
kgid = auth.KGID(atomic.LoadUint32(&parent.gid))
mode |= linux.S_ISGID
}
- if _, err := parent.file.mkdir(ctx, name, p9.FileMode(mode), (p9.UID)(creds.EffectiveKUID), p9.GID(kgid)); err != nil {
- if !opts.ForSyntheticMountpoint || linuxerr.Equals(linuxerr.EEXIST, err) {
- return err
+ var (
+ childDirInode *lisafs.Inode
+ err error
+ )
+ if fs.opts.lisaEnabled {
+ childDirInode, err = parent.controlFDLisa.MkdirAt(ctx, name, mode, lisafs.UID(creds.EffectiveKUID), lisafs.GID(kgid))
+ } else {
+ _, err = parent.file.mkdir(ctx, name, p9.FileMode(mode), (p9.UID)(creds.EffectiveKUID), p9.GID(kgid))
+ }
+ if err == nil {
+ if fs.opts.interop != InteropModeShared {
+ parent.incLinks()
}
- ctx.Infof("Failed to create remote directory %q: %v; falling back to synthetic directory", name, err)
- parent.createSyntheticChildLocked(&createSyntheticOpts{
- name: name,
- mode: linux.S_IFDIR | opts.Mode,
- kuid: creds.EffectiveKUID,
- kgid: creds.EffectiveKGID,
- })
- *ds = appendDentry(*ds, parent)
+ return childDirInode, nil
+ }
+
+ if !opts.ForSyntheticMountpoint || linuxerr.Equals(linuxerr.EEXIST, err) {
+ return nil, err
}
+ ctx.Infof("Failed to create remote directory %q: %v; falling back to synthetic directory", name, err)
+ parent.createSyntheticChildLocked(&createSyntheticOpts{
+ name: name,
+ mode: linux.S_IFDIR | opts.Mode,
+ kuid: creds.EffectiveKUID,
+ kgid: creds.EffectiveKGID,
+ })
+ *ds = appendDentry(*ds, parent)
if fs.opts.interop != InteropModeShared {
parent.incLinks()
}
- return nil
+ return nil, nil
}, func(parent *dentry, name string) error {
if !opts.ForSyntheticMountpoint {
// Can't create non-synthetic files in synthetic directories.
@@ -730,16 +922,26 @@ func (fs *filesystem) MkdirAt(ctx context.Context, rp *vfs.ResolvingPath, opts v
})
parent.incLinks()
return nil
- })
+ }, nil)
}
// MknodAt implements vfs.FilesystemImpl.MknodAt.
func (fs *filesystem) MknodAt(ctx context.Context, rp *vfs.ResolvingPath, opts vfs.MknodOptions) error {
- return fs.doCreateAt(ctx, rp, false /* dir */, func(parent *dentry, name string, ds **[]*dentry) error {
+ return fs.doCreateAt(ctx, rp, false /* dir */, func(parent *dentry, name string, ds **[]*dentry) (*lisafs.Inode, error) {
creds := rp.Credentials()
- _, err := parent.file.mknod(ctx, name, (p9.FileMode)(opts.Mode), opts.DevMajor, opts.DevMinor, (p9.UID)(creds.EffectiveKUID), (p9.GID)(creds.EffectiveKGID))
- if !linuxerr.Equals(linuxerr.EPERM, err) {
- return err
+ var (
+ childInode *lisafs.Inode
+ err error
+ )
+ if fs.opts.lisaEnabled {
+ childInode, err = parent.controlFDLisa.MknodAt(ctx, name, opts.Mode, lisafs.UID(creds.EffectiveKUID), lisafs.GID(creds.EffectiveKGID), opts.DevMinor, opts.DevMajor)
+ } else {
+ _, err = parent.file.mknod(ctx, name, (p9.FileMode)(opts.Mode), opts.DevMajor, opts.DevMinor, (p9.UID)(creds.EffectiveKUID), (p9.GID)(creds.EffectiveKGID))
+ }
+ if err == nil {
+ return childInode, nil
+ } else if !linuxerr.Equals(linuxerr.EPERM, err) {
+ return nil, err
}
// EPERM means that gofer does not allow creating a socket or pipe. Fallback
@@ -750,10 +952,10 @@ func (fs *filesystem) MknodAt(ctx context.Context, rp *vfs.ResolvingPath, opts v
switch {
case err == nil:
// Step succeeded, another file exists.
- return linuxerr.EEXIST
+ return nil, linuxerr.EEXIST
case !linuxerr.Equals(linuxerr.ENOENT, err):
// Unexpected error.
- return err
+ return nil, err
}
switch opts.Mode.FileType() {
@@ -766,7 +968,7 @@ func (fs *filesystem) MknodAt(ctx context.Context, rp *vfs.ResolvingPath, opts v
endpoint: opts.Endpoint,
})
*ds = appendDentry(*ds, parent)
- return nil
+ return nil, nil
case linux.S_IFIFO:
parent.createSyntheticChildLocked(&createSyntheticOpts{
name: name,
@@ -776,11 +978,11 @@ func (fs *filesystem) MknodAt(ctx context.Context, rp *vfs.ResolvingPath, opts v
pipe: pipe.NewVFSPipe(true /* isNamed */, pipe.DefaultPipeSize),
})
*ds = appendDentry(*ds, parent)
- return nil
+ return nil, nil
}
// Retain error from gofer if synthetic file cannot be created internally.
- return linuxerr.EPERM
- }, nil)
+ return nil, linuxerr.EPERM
+ }, nil, nil)
}
// OpenAt implements vfs.FilesystemImpl.OpenAt.
@@ -986,6 +1188,23 @@ func (d *dentry) openSocketByConnecting(ctx context.Context, opts *vfs.OpenOptio
if opts.Flags&linux.O_DIRECT != 0 {
return nil, linuxerr.EINVAL
}
+ if d.fs.opts.lisaEnabled {
+ // Note that special value of linux.SockType = 0 is interpreted by lisafs
+ // as "do not care about the socket type". Analogous to p9.AnonymousSocket.
+ sockFD, err := d.controlFDLisa.Connect(ctx, 0 /* sockType */)
+ if err != nil {
+ return nil, err
+ }
+ fd, err := host.NewFD(ctx, kernel.KernelFromContext(ctx).HostMount(), sockFD, &host.NewFDOptions{
+ HaveFlags: true,
+ Flags: opts.Flags,
+ })
+ if err != nil {
+ unix.Close(sockFD)
+ return nil, err
+ }
+ return fd, nil
+ }
fdObj, err := d.file.connect(ctx, p9.AnonymousSocket)
if err != nil {
return nil, err
@@ -998,6 +1217,7 @@ func (d *dentry) openSocketByConnecting(ctx context.Context, opts *vfs.OpenOptio
fdObj.Close()
return nil, err
}
+ // Ownership has been transferred to fd.
fdObj.Release()
return fd, nil
}
@@ -1017,7 +1237,13 @@ func (d *dentry) openSpecialFile(ctx context.Context, mnt *vfs.Mount, opts *vfs.
// since closed its end.
isBlockingOpenOfNamedPipe := d.fileType() == linux.S_IFIFO && opts.Flags&linux.O_NONBLOCK == 0
retry:
- h, err := openHandle(ctx, d.file, ats.MayRead(), ats.MayWrite(), opts.Flags&linux.O_TRUNC != 0)
+ var h handle
+ var err error
+ if d.fs.opts.lisaEnabled {
+ h, err = openHandleLisa(ctx, d.controlFDLisa, ats.MayRead(), ats.MayWrite(), opts.Flags&linux.O_TRUNC != 0)
+ } else {
+ h, err = openHandle(ctx, d.file, ats.MayRead(), ats.MayWrite(), opts.Flags&linux.O_TRUNC != 0)
+ }
if err != nil {
if isBlockingOpenOfNamedPipe && ats == vfs.MayWrite && linuxerr.Equals(linuxerr.ENXIO, err) {
// An attempt to open a named pipe with O_WRONLY|O_NONBLOCK fails
@@ -1061,18 +1287,8 @@ func (d *dentry) createAndOpenChildLocked(ctx context.Context, rp *vfs.Resolving
}
defer mnt.EndWrite()
- // 9P2000.L's lcreate takes a fid representing the parent directory, and
- // converts it into an open fid representing the created file, so we need
- // to duplicate the directory fid first.
- _, dirfile, err := d.file.walk(ctx, nil)
- if err != nil {
- return nil, err
- }
creds := rp.Credentials()
name := rp.Component()
- // We only want the access mode for creating the file.
- createFlags := p9.OpenFlags(opts.Flags) & p9.OpenFlagsModeMask
-
// If the parent is a setgid directory, use the parent's GID rather
// than the caller's.
kgid := creds.EffectiveKGID
@@ -1080,51 +1296,87 @@ func (d *dentry) createAndOpenChildLocked(ctx context.Context, rp *vfs.Resolving
kgid = auth.KGID(atomic.LoadUint32(&d.gid))
}
- fdobj, openFile, createQID, _, err := dirfile.create(ctx, name, createFlags, p9.FileMode(opts.Mode), (p9.UID)(creds.EffectiveKUID), p9.GID(kgid))
- if err != nil {
- dirfile.close(ctx)
- return nil, err
- }
- // Then we need to walk to the file we just created to get a non-open fid
- // representing it, and to get its metadata. This must use d.file since, as
- // explained above, dirfile was invalidated by dirfile.Create().
- _, nonOpenFile, attrMask, attr, err := d.file.walkGetAttrOne(ctx, name)
- if err != nil {
- openFile.close(ctx)
- if fdobj != nil {
- fdobj.Close()
+ var child *dentry
+ var openP9File p9file
+ openLisaFD := lisafs.InvalidFDID
+ openHostFD := int32(-1)
+ if d.fs.opts.lisaEnabled {
+ ino, openFD, hostFD, err := d.controlFDLisa.OpenCreateAt(ctx, name, opts.Flags&linux.O_ACCMODE, opts.Mode, lisafs.UID(creds.EffectiveKUID), lisafs.GID(kgid))
+ if err != nil {
+ return nil, err
+ }
+ openHostFD = int32(hostFD)
+ openLisaFD = openFD
+
+ child, err = d.fs.newDentryLisa(ctx, &ino)
+ if err != nil {
+ d.fs.clientLisa.CloseFDBatched(ctx, ino.ControlFD)
+ d.fs.clientLisa.CloseFDBatched(ctx, openFD)
+ if hostFD >= 0 {
+ unix.Close(hostFD)
+ }
+ return nil, err
+ }
+ } else {
+ // 9P2000.L's lcreate takes a fid representing the parent directory, and
+ // converts it into an open fid representing the created file, so we need
+ // to duplicate the directory fid first.
+ _, dirfile, err := d.file.walk(ctx, nil)
+ if err != nil {
+ return nil, err
+ }
+ // We only want the access mode for creating the file.
+ createFlags := p9.OpenFlags(opts.Flags) & p9.OpenFlagsModeMask
+
+ fdobj, openFile, createQID, _, err := dirfile.create(ctx, name, createFlags, p9.FileMode(opts.Mode), (p9.UID)(creds.EffectiveKUID), p9.GID(kgid))
+ if err != nil {
+ dirfile.close(ctx)
+ return nil, err
+ }
+ // Then we need to walk to the file we just created to get a non-open fid
+ // representing it, and to get its metadata. This must use d.file since, as
+ // explained above, dirfile was invalidated by dirfile.Create().
+ _, nonOpenFile, attrMask, attr, err := d.file.walkGetAttrOne(ctx, name)
+ if err != nil {
+ openFile.close(ctx)
+ if fdobj != nil {
+ fdobj.Close()
+ }
+ return nil, err
+ }
+
+ // Construct the new dentry.
+ child, err = d.fs.newDentry(ctx, nonOpenFile, createQID, attrMask, &attr)
+ if err != nil {
+ nonOpenFile.close(ctx)
+ openFile.close(ctx)
+ if fdobj != nil {
+ fdobj.Close()
+ }
+ return nil, err
}
- return nil, err
- }
- // Construct the new dentry.
- child, err := d.fs.newDentry(ctx, nonOpenFile, createQID, attrMask, &attr)
- if err != nil {
- nonOpenFile.close(ctx)
- openFile.close(ctx)
if fdobj != nil {
- fdobj.Close()
+ openHostFD = int32(fdobj.Release())
}
- return nil, err
+ openP9File = openFile
}
// Incorporate the fid that was opened by lcreate.
useRegularFileFD := child.fileType() == linux.S_IFREG && !d.fs.opts.regularFilesUseSpecialFileFD
if useRegularFileFD {
- openFD := int32(-1)
- if fdobj != nil {
- openFD = int32(fdobj.Release())
- }
child.handleMu.Lock()
if vfs.MayReadFileWithOpenFlags(opts.Flags) {
- child.readFile = openFile
- if fdobj != nil {
- child.readFD = openFD
- child.mmapFD = openFD
+ child.readFile = openP9File
+ child.readFDLisa = d.fs.clientLisa.NewFD(openLisaFD)
+ if openHostFD != -1 {
+ child.readFD = openHostFD
+ child.mmapFD = openHostFD
}
}
if vfs.MayWriteFileWithOpenFlags(opts.Flags) {
- child.writeFile = openFile
- child.writeFD = openFD
+ child.writeFile = openP9File
+ child.writeFDLisa = d.fs.clientLisa.NewFD(openLisaFD)
+ child.writeFD = openHostFD
}
child.handleMu.Unlock()
}
@@ -1146,11 +1398,9 @@ func (d *dentry) createAndOpenChildLocked(ctx context.Context, rp *vfs.Resolving
childVFSFD = &fd.vfsfd
} else {
h := handle{
- file: openFile,
- fd: -1,
- }
- if fdobj != nil {
- h.fd = int32(fdobj.Release())
+ file: openP9File,
+ fdLisa: d.fs.clientLisa.NewFD(openLisaFD),
+ fd: openHostFD,
}
fd, err := newSpecialFileFD(h, mnt, child, opts.Flags)
if err != nil {
@@ -1304,7 +1554,12 @@ func (fs *filesystem) RenameAt(ctx context.Context, rp *vfs.ResolvingPath, oldPa
// Update the remote filesystem.
if !renamed.isSynthetic() {
- if err := renamed.file.rename(ctx, newParent.file, newName); err != nil {
+ if fs.opts.lisaEnabled {
+ err = renamed.controlFDLisa.RenameTo(ctx, newParent.controlFDLisa.ID(), newName)
+ } else {
+ err = renamed.file.rename(ctx, newParent.file, newName)
+ }
+ if err != nil {
vfsObj.AbortRenameDentry(&renamed.vfsd, replacedVFSD)
return err
}
@@ -1315,7 +1570,12 @@ func (fs *filesystem) RenameAt(ctx context.Context, rp *vfs.ResolvingPath, oldPa
if replaced.isDir() {
flags = linux.AT_REMOVEDIR
}
- if err := newParent.file.unlinkAt(ctx, newName, flags); err != nil {
+ if fs.opts.lisaEnabled {
+ err = newParent.controlFDLisa.UnlinkAt(ctx, newName, flags)
+ } else {
+ err = newParent.file.unlinkAt(ctx, newName, flags)
+ }
+ if err != nil {
vfsObj.AbortRenameDentry(&renamed.vfsd, replacedVFSD)
return err
}
@@ -1431,6 +1691,28 @@ func (fs *filesystem) StatFSAt(ctx context.Context, rp *vfs.ResolvingPath) (linu
for d.isSynthetic() {
d = d.parent
}
+ if fs.opts.lisaEnabled {
+ var statFS lisafs.StatFS
+ if err := d.controlFDLisa.StatFSTo(ctx, &statFS); err != nil {
+ return linux.Statfs{}, err
+ }
+ if statFS.NameLength > maxFilenameLen {
+ statFS.NameLength = maxFilenameLen
+ }
+ return linux.Statfs{
+ // This is primarily for distinguishing a gofer file system in
+ // tests. Testing is important, so instead of defining
+ // something completely random, use a standard value.
+ Type: linux.V9FS_MAGIC,
+ BlockSize: statFS.BlockSize,
+ Blocks: statFS.Blocks,
+ BlocksFree: statFS.BlocksFree,
+ BlocksAvailable: statFS.BlocksAvailable,
+ Files: statFS.Files,
+ FilesFree: statFS.FilesFree,
+ NameLength: statFS.NameLength,
+ }, nil
+ }
fsstat, err := d.file.statFS(ctx)
if err != nil {
return linux.Statfs{}, err
@@ -1456,11 +1738,21 @@ func (fs *filesystem) StatFSAt(ctx context.Context, rp *vfs.ResolvingPath) (linu
// SymlinkAt implements vfs.FilesystemImpl.SymlinkAt.
func (fs *filesystem) SymlinkAt(ctx context.Context, rp *vfs.ResolvingPath, target string) error {
- return fs.doCreateAt(ctx, rp, false /* dir */, func(parent *dentry, name string, _ **[]*dentry) error {
+ return fs.doCreateAt(ctx, rp, false /* dir */, func(parent *dentry, name string, ds **[]*dentry) (*lisafs.Inode, error) {
creds := rp.Credentials()
+ if fs.opts.lisaEnabled {
+ return parent.controlFDLisa.SymlinkAt(ctx, name, target, lisafs.UID(creds.EffectiveKUID), lisafs.GID(creds.EffectiveKGID))
+ }
_, err := parent.file.symlink(ctx, target, name, (p9.UID)(creds.EffectiveKUID), (p9.GID)(creds.EffectiveKGID))
- return err
- }, nil)
+ return nil, err
+ }, nil, func(child *dentry) {
+ if fs.opts.interop != InteropModeShared {
+ // lisafs caches the symlink target on creation. In practice, this
+ // helps avoid a lot of ReadLink RPCs.
+ child.haveTarget = true
+ child.target = target
+ }
+ })
}
// UnlinkAt implements vfs.FilesystemImpl.UnlinkAt.
@@ -1505,7 +1797,7 @@ func (fs *filesystem) ListXattrAt(ctx context.Context, rp *vfs.ResolvingPath, si
if err != nil {
return nil, err
}
- return d.listXattr(ctx, rp.Credentials(), size)
+ return d.listXattr(ctx, size)
}
// GetXattrAt implements vfs.FilesystemImpl.GetXattrAt.
@@ -1612,6 +1904,9 @@ func (fs *filesystem) MountOptions() string {
if fs.opts.overlayfsStaleRead {
optsKV = append(optsKV, mopt{moptOverlayfsStaleRead, nil})
}
+ if fs.opts.lisaEnabled {
+ optsKV = append(optsKV, mopt{moptLisafs, nil})
+ }
opts := make([]string, 0, len(optsKV))
for _, opt := range optsKV {
diff --git a/pkg/sentry/fsimpl/gofer/gofer.go b/pkg/sentry/fsimpl/gofer/gofer.go
index 43440ec19..b98825e26 100644
--- a/pkg/sentry/fsimpl/gofer/gofer.go
+++ b/pkg/sentry/fsimpl/gofer/gofer.go
@@ -48,6 +48,7 @@ import (
"gvisor.dev/gvisor/pkg/context"
"gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/hostarch"
+ "gvisor.dev/gvisor/pkg/lisafs"
"gvisor.dev/gvisor/pkg/log"
"gvisor.dev/gvisor/pkg/p9"
refs_vfs1 "gvisor.dev/gvisor/pkg/refs"
@@ -83,6 +84,7 @@ const (
moptForcePageCache = "force_page_cache"
moptLimitHostFDTranslation = "limit_host_fd_translation"
moptOverlayfsStaleRead = "overlayfs_stale_read"
+ moptLisafs = "lisafs"
)
// Valid values for the "cache" mount option.
@@ -118,6 +120,10 @@ type filesystem struct {
// client is the client used by this filesystem. client is immutable.
client *p9.Client `state:"nosave"`
+ // clientLisa is the client used for communicating with the server when
+ // lisafs is enabled. lisafsCient is immutable.
+ clientLisa *lisafs.Client `state:"nosave"`
+
// clock is a realtime clock used to set timestamps in file operations.
clock ktime.Clock
@@ -161,6 +167,12 @@ type filesystem struct {
inoMu sync.Mutex `state:"nosave"`
inoByQIDPath map[uint64]uint64 `state:"nosave"`
+ // inoByKey is the same as inoByQIDPath but only used by lisafs. It helps
+ // identify inodes based on the device ID and host inode number provided
+ // by the gofer process. It is not preserved across checkpoint/restore for
+ // the same reason as above. inoByKey is protected by inoMu.
+ inoByKey map[inoKey]uint64 `state:"nosave"`
+
// lastIno is the last inode number assigned to a file. lastIno is accessed
// using atomic memory operations.
lastIno uint64
@@ -214,6 +226,10 @@ type filesystemOptions struct {
// way that application FDs representing "special files" such as sockets
// do. Note that this disables client caching and mmap for regular files.
regularFilesUseSpecialFileFD bool
+
+ // lisaEnabled indicates whether the client will use lisafs protocol to
+ // communicate with the server instead of 9P.
+ lisaEnabled bool
}
// InteropMode controls the client's interaction with other remote filesystem
@@ -427,6 +443,14 @@ func (fstype FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.Virt
delete(mopts, moptOverlayfsStaleRead)
fsopts.overlayfsStaleRead = true
}
+ if lisafs, ok := mopts[moptLisafs]; ok {
+ delete(mopts, moptLisafs)
+ fsopts.lisaEnabled, err = strconv.ParseBool(lisafs)
+ if err != nil {
+ ctx.Warningf("gofer.FilesystemType.GetFilesystem: invalid lisafs option: %s", lisafs)
+ return nil, nil, linuxerr.EINVAL
+ }
+ }
// fsopts.regularFilesUseSpecialFileFD can only be enabled by specifying
// "cache=none".
@@ -458,44 +482,83 @@ func (fstype FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.Virt
syncableDentries: make(map[*dentry]struct{}),
specialFileFDs: make(map[*specialFileFD]struct{}),
inoByQIDPath: make(map[uint64]uint64),
+ inoByKey: make(map[inoKey]uint64),
}
fs.vfsfs.Init(vfsObj, &fstype, fs)
+ if err := fs.initClientAndRoot(ctx); err != nil {
+ fs.vfsfs.DecRef(ctx)
+ return nil, nil, err
+ }
+
+ return &fs.vfsfs, &fs.root.vfsd, nil
+}
+
+func (fs *filesystem) initClientAndRoot(ctx context.Context) error {
+ var err error
+ if fs.opts.lisaEnabled {
+ var rootInode *lisafs.Inode
+ rootInode, err = fs.initClientLisa(ctx)
+ if err != nil {
+ return err
+ }
+ fs.root, err = fs.newDentryLisa(ctx, rootInode)
+ if err != nil {
+ fs.clientLisa.CloseFDBatched(ctx, rootInode.ControlFD)
+ }
+ } else {
+ fs.root, err = fs.initClient(ctx)
+ }
+
+ // Set the root's reference count to 2. One reference is returned to the
+ // caller, and the other is held by fs to prevent the root from being "cached"
+ // and subsequently evicted.
+ if err == nil {
+ fs.root.refs = 2
+ }
+ return err
+}
+
+func (fs *filesystem) initClientLisa(ctx context.Context) (*lisafs.Inode, error) {
+ sock, err := unet.NewSocket(fs.opts.fd)
+ if err != nil {
+ return nil, err
+ }
+
+ var rootInode *lisafs.Inode
+ ctx.UninterruptibleSleepStart(false)
+ fs.clientLisa, rootInode, err = lisafs.NewClient(sock, fs.opts.aname)
+ ctx.UninterruptibleSleepFinish(false)
+ return rootInode, err
+}
+
+func (fs *filesystem) initClient(ctx context.Context) (*dentry, error) {
// Connect to the server.
if err := fs.dial(ctx); err != nil {
- return nil, nil, err
+ return nil, err
}
// Perform attach to obtain the filesystem root.
ctx.UninterruptibleSleepStart(false)
- attached, err := fs.client.Attach(fsopts.aname)
+ attached, err := fs.client.Attach(fs.opts.aname)
ctx.UninterruptibleSleepFinish(false)
if err != nil {
- fs.vfsfs.DecRef(ctx)
- return nil, nil, err
+ return nil, err
}
attachFile := p9file{attached}
qid, attrMask, attr, err := attachFile.getAttr(ctx, dentryAttrMask())
if err != nil {
attachFile.close(ctx)
- fs.vfsfs.DecRef(ctx)
- return nil, nil, err
+ return nil, err
}
// Construct the root dentry.
root, err := fs.newDentry(ctx, attachFile, qid, attrMask, &attr)
if err != nil {
attachFile.close(ctx)
- fs.vfsfs.DecRef(ctx)
- return nil, nil, err
+ return nil, err
}
- // Set the root's reference count to 2. One reference is returned to the
- // caller, and the other is held by fs to prevent the root from being "cached"
- // and subsequently evicted.
- root.refs = 2
- fs.root = root
-
- return &fs.vfsfs, &root.vfsd, nil
+ return root, nil
}
func getFDFromMountOptionsMap(ctx context.Context, mopts map[string]string) (int, error) {
@@ -613,7 +676,11 @@ func (fs *filesystem) Release(ctx context.Context) {
if !fs.iopts.LeakConnection {
// Close the connection to the server. This implicitly clunks all fids.
- fs.client.Close()
+ if fs.opts.lisaEnabled {
+ fs.clientLisa.Close()
+ } else {
+ fs.client.Close()
+ }
}
fs.vfsfs.VirtualFilesystem().PutAnonBlockDevMinor(fs.devMinor)
@@ -644,6 +711,23 @@ func (d *dentry) releaseSyntheticRecursiveLocked(ctx context.Context) {
}
}
+// inoKey is the key used to identify the inode backed by this dentry.
+//
+// +stateify savable
+type inoKey struct {
+ ino uint64
+ devMinor uint32
+ devMajor uint32
+}
+
+func inoKeyFromStat(stat *linux.Statx) inoKey {
+ return inoKey{
+ ino: stat.Ino,
+ devMinor: stat.DevMinor,
+ devMajor: stat.DevMajor,
+ }
+}
+
// dentry implements vfs.DentryImpl.
//
// +stateify savable
@@ -674,6 +758,9 @@ type dentry struct {
// qidPath is the p9.QID.Path for this file. qidPath is immutable.
qidPath uint64
+ // inoKey is used to identify this dentry's inode.
+ inoKey inoKey
+
// file is the unopened p9.File that backs this dentry. file is immutable.
//
// If file.isNil(), this dentry represents a synthetic file, i.e. a file
@@ -681,6 +768,14 @@ type dentry struct {
// only files that can be synthetic are sockets, pipes, and directories.
file p9file `state:"nosave"`
+ // controlFDLisa is used by lisafs to perform path based operations on this
+ // dentry.
+ //
+ // if !controlFDLisa.Ok(), this dentry represents a synthetic file, i.e. a
+ // file that does not exist on the remote filesystem. As of this writing, the
+ // only files that can be synthetic are sockets, pipes, and directories.
+ controlFDLisa lisafs.ClientFD `state:"nosave"`
+
// If deleted is non-zero, the file represented by this dentry has been
// deleted. deleted is accessed using atomic memory operations.
deleted uint32
@@ -791,12 +886,14 @@ type dentry struct {
// always either -1 or equal to readFD; if !writeFile.isNil() (the file has
// been opened for writing), it is additionally either -1 or equal to
// writeFD.
- handleMu sync.RWMutex `state:"nosave"`
- readFile p9file `state:"nosave"`
- writeFile p9file `state:"nosave"`
- readFD int32 `state:"nosave"`
- writeFD int32 `state:"nosave"`
- mmapFD int32 `state:"nosave"`
+ handleMu sync.RWMutex `state:"nosave"`
+ readFile p9file `state:"nosave"`
+ writeFile p9file `state:"nosave"`
+ readFDLisa lisafs.ClientFD `state:"nosave"`
+ writeFDLisa lisafs.ClientFD `state:"nosave"`
+ readFD int32 `state:"nosave"`
+ writeFD int32 `state:"nosave"`
+ mmapFD int32 `state:"nosave"`
dataMu sync.RWMutex `state:"nosave"`
@@ -920,6 +1017,79 @@ func (fs *filesystem) newDentry(ctx context.Context, file p9file, qid p9.QID, ma
return d, nil
}
+func (fs *filesystem) newDentryLisa(ctx context.Context, ino *lisafs.Inode) (*dentry, error) {
+ if ino.Stat.Mask&linux.STATX_TYPE == 0 {
+ ctx.Warningf("can't create gofer.dentry without file type")
+ return nil, linuxerr.EIO
+ }
+ if ino.Stat.Mode&linux.FileTypeMask == linux.ModeRegular && ino.Stat.Mask&linux.STATX_SIZE == 0 {
+ ctx.Warningf("can't create regular file gofer.dentry without file size")
+ return nil, linuxerr.EIO
+ }
+
+ inoKey := inoKeyFromStat(&ino.Stat)
+ d := &dentry{
+ fs: fs,
+ inoKey: inoKey,
+ ino: fs.inoFromKey(inoKey),
+ mode: uint32(ino.Stat.Mode),
+ uid: uint32(fs.opts.dfltuid),
+ gid: uint32(fs.opts.dfltgid),
+ blockSize: hostarch.PageSize,
+ readFD: -1,
+ writeFD: -1,
+ mmapFD: -1,
+ controlFDLisa: fs.clientLisa.NewFD(ino.ControlFD),
+ }
+
+ d.pf.dentry = d
+ if ino.Stat.Mask&linux.STATX_UID != 0 {
+ d.uid = dentryUIDFromLisaUID(lisafs.UID(ino.Stat.UID))
+ }
+ if ino.Stat.Mask&linux.STATX_GID != 0 {
+ d.gid = dentryGIDFromLisaGID(lisafs.GID(ino.Stat.GID))
+ }
+ if ino.Stat.Mask&linux.STATX_SIZE != 0 {
+ d.size = ino.Stat.Size
+ }
+ if ino.Stat.Blksize != 0 {
+ d.blockSize = ino.Stat.Blksize
+ }
+ if ino.Stat.Mask&linux.STATX_ATIME != 0 {
+ d.atime = dentryTimestampFromLisa(ino.Stat.Atime)
+ }
+ if ino.Stat.Mask&linux.STATX_MTIME != 0 {
+ d.mtime = dentryTimestampFromLisa(ino.Stat.Mtime)
+ }
+ if ino.Stat.Mask&linux.STATX_CTIME != 0 {
+ d.ctime = dentryTimestampFromLisa(ino.Stat.Ctime)
+ }
+ if ino.Stat.Mask&linux.STATX_BTIME != 0 {
+ d.btime = dentryTimestampFromLisa(ino.Stat.Btime)
+ }
+ if ino.Stat.Mask&linux.STATX_NLINK != 0 {
+ d.nlink = ino.Stat.Nlink
+ }
+ d.vfsd.Init(d)
+ refsvfs2.Register(d)
+ fs.syncMu.Lock()
+ fs.syncableDentries[d] = struct{}{}
+ fs.syncMu.Unlock()
+ return d, nil
+}
+
+func (fs *filesystem) inoFromKey(key inoKey) uint64 {
+ fs.inoMu.Lock()
+ defer fs.inoMu.Unlock()
+
+ if ino, ok := fs.inoByKey[key]; ok {
+ return ino
+ }
+ ino := fs.nextIno()
+ fs.inoByKey[key] = ino
+ return ino
+}
+
func (fs *filesystem) inoFromQIDPath(qidPath uint64) uint64 {
fs.inoMu.Lock()
defer fs.inoMu.Unlock()
@@ -936,7 +1106,7 @@ func (fs *filesystem) nextIno() uint64 {
}
func (d *dentry) isSynthetic() bool {
- return d.file.isNil()
+ return !d.isControlFileOk()
}
func (d *dentry) cachedMetadataAuthoritative() bool {
@@ -986,6 +1156,50 @@ func (d *dentry) updateFromP9AttrsLocked(mask p9.AttrMask, attr *p9.Attr) {
}
}
+// updateFromLisaStatLocked is called to update d's metadata after an update
+// from the remote filesystem.
+// Precondition: d.metadataMu must be locked.
+// +checklocks:d.metadataMu
+func (d *dentry) updateFromLisaStatLocked(stat *linux.Statx) {
+ if stat.Mask&linux.STATX_TYPE != 0 {
+ if got, want := stat.Mode&linux.FileTypeMask, d.fileType(); uint32(got) != want {
+ panic(fmt.Sprintf("gofer.dentry file type changed from %#o to %#o", want, got))
+ }
+ }
+ if stat.Mask&linux.STATX_MODE != 0 {
+ atomic.StoreUint32(&d.mode, uint32(stat.Mode))
+ }
+ if stat.Mask&linux.STATX_UID != 0 {
+ atomic.StoreUint32(&d.uid, dentryUIDFromLisaUID(lisafs.UID(stat.UID)))
+ }
+ if stat.Mask&linux.STATX_GID != 0 {
+ atomic.StoreUint32(&d.uid, dentryGIDFromLisaGID(lisafs.GID(stat.GID)))
+ }
+ if stat.Blksize != 0 {
+ atomic.StoreUint32(&d.blockSize, stat.Blksize)
+ }
+ // Don't override newer client-defined timestamps with old server-defined
+ // ones.
+ if stat.Mask&linux.STATX_ATIME != 0 && atomic.LoadUint32(&d.atimeDirty) == 0 {
+ atomic.StoreInt64(&d.atime, dentryTimestampFromLisa(stat.Atime))
+ }
+ if stat.Mask&linux.STATX_MTIME != 0 && atomic.LoadUint32(&d.mtimeDirty) == 0 {
+ atomic.StoreInt64(&d.mtime, dentryTimestampFromLisa(stat.Mtime))
+ }
+ if stat.Mask&linux.STATX_CTIME != 0 {
+ atomic.StoreInt64(&d.ctime, dentryTimestampFromLisa(stat.Ctime))
+ }
+ if stat.Mask&linux.STATX_BTIME != 0 {
+ atomic.StoreInt64(&d.btime, dentryTimestampFromLisa(stat.Btime))
+ }
+ if stat.Mask&linux.STATX_NLINK != 0 {
+ atomic.StoreUint32(&d.nlink, stat.Nlink)
+ }
+ if stat.Mask&linux.STATX_SIZE != 0 {
+ d.updateSizeLocked(stat.Size)
+ }
+}
+
// Preconditions: !d.isSynthetic().
// Preconditions: d.metadataMu is locked.
// +checklocks:d.metadataMu
@@ -995,6 +1209,9 @@ func (d *dentry) refreshSizeLocked(ctx context.Context) error {
if d.writeFD < 0 {
d.handleMu.RUnlock()
// Ask the gofer if we don't have a host FD.
+ if d.fs.opts.lisaEnabled {
+ return d.updateFromStatLisaLocked(ctx, nil)
+ }
return d.updateFromGetattrLocked(ctx, p9file{})
}
@@ -1014,6 +1231,9 @@ func (d *dentry) updateFromGetattr(ctx context.Context) error {
// updating stale attributes in d.updateFromP9AttrsLocked().
d.metadataMu.Lock()
defer d.metadataMu.Unlock()
+ if d.fs.opts.lisaEnabled {
+ return d.updateFromStatLisaLocked(ctx, nil)
+ }
return d.updateFromGetattrLocked(ctx, p9file{})
}
@@ -1021,6 +1241,45 @@ func (d *dentry) updateFromGetattr(ctx context.Context) error {
// * !d.isSynthetic().
// * d.metadataMu is locked.
// +checklocks:d.metadataMu
+func (d *dentry) updateFromStatLisaLocked(ctx context.Context, fdLisa *lisafs.ClientFD) error {
+ handleMuRLocked := false
+ if fdLisa == nil {
+ // Use open FDs in preferenece to the control FD. This may be significantly
+ // more efficient in some implementations. Prefer a writable FD over a
+ // readable one since some filesystem implementations may update a writable
+ // FD's metadata after writes, without making metadata updates immediately
+ // visible to read-only FDs representing the same file.
+ d.handleMu.RLock()
+ switch {
+ case d.writeFDLisa.Ok():
+ fdLisa = &d.writeFDLisa
+ handleMuRLocked = true
+ case d.readFDLisa.Ok():
+ fdLisa = &d.readFDLisa
+ handleMuRLocked = true
+ default:
+ fdLisa = &d.controlFDLisa
+ d.handleMu.RUnlock()
+ }
+ }
+
+ var stat linux.Statx
+ err := fdLisa.StatTo(ctx, &stat)
+ if handleMuRLocked {
+ // handleMu must be released before updateFromLisaStatLocked().
+ d.handleMu.RUnlock() // +checklocksforce: complex case.
+ }
+ if err != nil {
+ return err
+ }
+ d.updateFromLisaStatLocked(&stat)
+ return nil
+}
+
+// Preconditions:
+// * !d.isSynthetic().
+// * d.metadataMu is locked.
+// +checklocks:d.metadataMu
func (d *dentry) updateFromGetattrLocked(ctx context.Context, file p9file) error {
handleMuRLocked := false
if file.isNil() {
@@ -1160,6 +1419,13 @@ func (d *dentry) setStat(ctx context.Context, creds *auth.Credentials, opts *vfs
}
}
+ // failureMask indicates which attributes could not be set on the remote
+ // filesystem. p9 returns an error if any of the attributes could not be set
+ // but that leads to inconsistency as the server could have set a few
+ // attributes successfully but a later failure will cause the successful ones
+ // to not be updated in the dentry cache.
+ var failureMask uint32
+ var failureErr error
if !d.isSynthetic() {
if stat.Mask != 0 {
if stat.Mask&linux.STATX_SIZE != 0 {
@@ -1169,35 +1435,50 @@ func (d *dentry) setStat(ctx context.Context, creds *auth.Credentials, opts *vfs
// the remote file has been truncated).
d.dataMu.Lock()
}
- if err := d.file.setAttr(ctx, p9.SetAttrMask{
- Permissions: stat.Mask&linux.STATX_MODE != 0,
- UID: stat.Mask&linux.STATX_UID != 0,
- GID: stat.Mask&linux.STATX_GID != 0,
- Size: stat.Mask&linux.STATX_SIZE != 0,
- ATime: stat.Mask&linux.STATX_ATIME != 0,
- MTime: stat.Mask&linux.STATX_MTIME != 0,
- ATimeNotSystemTime: stat.Mask&linux.STATX_ATIME != 0 && stat.Atime.Nsec != linux.UTIME_NOW,
- MTimeNotSystemTime: stat.Mask&linux.STATX_MTIME != 0 && stat.Mtime.Nsec != linux.UTIME_NOW,
- }, p9.SetAttr{
- Permissions: p9.FileMode(stat.Mode),
- UID: p9.UID(stat.UID),
- GID: p9.GID(stat.GID),
- Size: stat.Size,
- ATimeSeconds: uint64(stat.Atime.Sec),
- ATimeNanoSeconds: uint64(stat.Atime.Nsec),
- MTimeSeconds: uint64(stat.Mtime.Sec),
- MTimeNanoSeconds: uint64(stat.Mtime.Nsec),
- }); err != nil {
- if stat.Mask&linux.STATX_SIZE != 0 {
- d.dataMu.Unlock() // +checklocksforce: locked conditionally above
+ if d.fs.opts.lisaEnabled {
+ var err error
+ failureMask, failureErr, err = d.controlFDLisa.SetStat(ctx, stat)
+ if err != nil {
+ if stat.Mask&linux.STATX_SIZE != 0 {
+ d.dataMu.Unlock() // +checklocksforce: locked conditionally above
+ }
+ return err
+ }
+ } else {
+ if err := d.file.setAttr(ctx, p9.SetAttrMask{
+ Permissions: stat.Mask&linux.STATX_MODE != 0,
+ UID: stat.Mask&linux.STATX_UID != 0,
+ GID: stat.Mask&linux.STATX_GID != 0,
+ Size: stat.Mask&linux.STATX_SIZE != 0,
+ ATime: stat.Mask&linux.STATX_ATIME != 0,
+ MTime: stat.Mask&linux.STATX_MTIME != 0,
+ ATimeNotSystemTime: stat.Mask&linux.STATX_ATIME != 0 && stat.Atime.Nsec != linux.UTIME_NOW,
+ MTimeNotSystemTime: stat.Mask&linux.STATX_MTIME != 0 && stat.Mtime.Nsec != linux.UTIME_NOW,
+ }, p9.SetAttr{
+ Permissions: p9.FileMode(stat.Mode),
+ UID: p9.UID(stat.UID),
+ GID: p9.GID(stat.GID),
+ Size: stat.Size,
+ ATimeSeconds: uint64(stat.Atime.Sec),
+ ATimeNanoSeconds: uint64(stat.Atime.Nsec),
+ MTimeSeconds: uint64(stat.Mtime.Sec),
+ MTimeNanoSeconds: uint64(stat.Mtime.Nsec),
+ }); err != nil {
+ if stat.Mask&linux.STATX_SIZE != 0 {
+ d.dataMu.Unlock() // +checklocksforce: locked conditionally above
+ }
+ return err
}
- return err
}
if stat.Mask&linux.STATX_SIZE != 0 {
- // d.size should be kept up to date, and privatized
- // copy-on-write mappings of truncated pages need to be
- // invalidated, even if InteropModeShared is in effect.
- d.updateSizeAndUnlockDataMuLocked(stat.Size) // +checklocksforce: locked conditionally above
+ if failureMask&linux.STATX_SIZE == 0 {
+ // d.size should be kept up to date, and privatized
+ // copy-on-write mappings of truncated pages need to be
+ // invalidated, even if InteropModeShared is in effect.
+ d.updateSizeAndUnlockDataMuLocked(stat.Size) // +checklocksforce: locked conditionally above
+ } else {
+ d.dataMu.Unlock() // +checklocksforce: locked conditionally above
+ }
}
}
if d.fs.opts.interop == InteropModeShared {
@@ -1208,13 +1489,13 @@ func (d *dentry) setStat(ctx context.Context, creds *auth.Credentials, opts *vfs
return nil
}
}
- if stat.Mask&linux.STATX_MODE != 0 {
+ if stat.Mask&linux.STATX_MODE != 0 && failureMask&linux.STATX_MODE == 0 {
atomic.StoreUint32(&d.mode, d.fileType()|uint32(stat.Mode))
}
- if stat.Mask&linux.STATX_UID != 0 {
+ if stat.Mask&linux.STATX_UID != 0 && failureMask&linux.STATX_UID == 0 {
atomic.StoreUint32(&d.uid, stat.UID)
}
- if stat.Mask&linux.STATX_GID != 0 {
+ if stat.Mask&linux.STATX_GID != 0 && failureMask&linux.STATX_GID == 0 {
atomic.StoreUint32(&d.gid, stat.GID)
}
// Note that stat.Atime.Nsec and stat.Mtime.Nsec can't be UTIME_NOW because
@@ -1222,15 +1503,19 @@ func (d *dentry) setStat(ctx context.Context, creds *auth.Credentials, opts *vfs
// stat.Mtime to client-local timestamps above, and if
// !d.cachedMetadataAuthoritative() then we returned after calling
// d.file.setAttr(). For the same reason, now must have been initialized.
- if stat.Mask&linux.STATX_ATIME != 0 {
+ if stat.Mask&linux.STATX_ATIME != 0 && failureMask&linux.STATX_ATIME == 0 {
atomic.StoreInt64(&d.atime, stat.Atime.ToNsec())
atomic.StoreUint32(&d.atimeDirty, 0)
}
- if stat.Mask&linux.STATX_MTIME != 0 {
+ if stat.Mask&linux.STATX_MTIME != 0 && failureMask&linux.STATX_MTIME == 0 {
atomic.StoreInt64(&d.mtime, stat.Mtime.ToNsec())
atomic.StoreUint32(&d.mtimeDirty, 0)
}
atomic.StoreInt64(&d.ctime, now)
+ if failureMask != 0 {
+ // Setting some attribute failed on the remote filesystem.
+ return failureErr
+ }
return nil
}
@@ -1310,7 +1595,10 @@ func (d *dentry) checkXattrPermissions(creds *auth.Credentials, name string, ats
// (b/148380782). Allow all other extended attributes to be passed through
// to the remote filesystem. This is inconsistent with Linux's 9p client,
// but consistent with other filesystems (e.g. FUSE).
- if strings.HasPrefix(name, linux.XATTR_SECURITY_PREFIX) || strings.HasPrefix(name, linux.XATTR_SYSTEM_PREFIX) {
+ //
+ // NOTE(b/202533394): Also disallow "trusted" namespace for now. This is
+ // consistent with the VFS1 gofer client.
+ if strings.HasPrefix(name, linux.XATTR_SECURITY_PREFIX) || strings.HasPrefix(name, linux.XATTR_SYSTEM_PREFIX) || strings.HasPrefix(name, linux.XATTR_TRUSTED_PREFIX) {
return linuxerr.EOPNOTSUPP
}
mode := linux.FileMode(atomic.LoadUint32(&d.mode))
@@ -1346,6 +1634,20 @@ func dentryGIDFromP9GID(gid p9.GID) uint32 {
return uint32(gid)
}
+func dentryUIDFromLisaUID(uid lisafs.UID) uint32 {
+ if !uid.Ok() {
+ return uint32(auth.OverflowUID)
+ }
+ return uint32(uid)
+}
+
+func dentryGIDFromLisaGID(gid lisafs.GID) uint32 {
+ if !gid.Ok() {
+ return uint32(auth.OverflowGID)
+ }
+ return uint32(gid)
+}
+
// IncRef implements vfs.DentryImpl.IncRef.
func (d *dentry) IncRef() {
// d.refs may be 0 if d.fs.renameMu is locked, which serializes against
@@ -1654,15 +1956,24 @@ func (d *dentry) destroyLocked(ctx context.Context) {
d.dirty.RemoveAll()
}
d.dataMu.Unlock()
- // Clunk open fids and close open host FDs.
- if !d.readFile.isNil() {
- _ = d.readFile.close(ctx)
- }
- if !d.writeFile.isNil() && d.readFile != d.writeFile {
- _ = d.writeFile.close(ctx)
+ if d.fs.opts.lisaEnabled {
+ if d.readFDLisa.Ok() && d.readFDLisa.ID() != d.writeFDLisa.ID() {
+ d.readFDLisa.CloseBatched(ctx)
+ }
+ if d.writeFDLisa.Ok() {
+ d.writeFDLisa.CloseBatched(ctx)
+ }
+ } else {
+ // Clunk open fids and close open host FDs.
+ if !d.readFile.isNil() {
+ _ = d.readFile.close(ctx)
+ }
+ if !d.writeFile.isNil() && d.readFile != d.writeFile {
+ _ = d.writeFile.close(ctx)
+ }
+ d.readFile = p9file{}
+ d.writeFile = p9file{}
}
- d.readFile = p9file{}
- d.writeFile = p9file{}
if d.readFD >= 0 {
_ = unix.Close(int(d.readFD))
}
@@ -1674,7 +1985,7 @@ func (d *dentry) destroyLocked(ctx context.Context) {
d.mmapFD = -1
d.handleMu.Unlock()
- if !d.file.isNil() {
+ if d.isControlFileOk() {
// Note that it's possible that d.atimeDirty or d.mtimeDirty are true,
// i.e. client and server timestamps may differ (because e.g. a client
// write was serviced by the page cache, and only written back to the
@@ -1683,10 +1994,16 @@ func (d *dentry) destroyLocked(ctx context.Context) {
// instantiated for the same file would remain coherent. Unfortunately,
// this turns out to be too expensive in many cases, so for now we
// don't do this.
- if err := d.file.close(ctx); err != nil {
- log.Warningf("gofer.dentry.destroyLocked: failed to close file: %v", err)
+
+ // Close the control FD.
+ if d.fs.opts.lisaEnabled {
+ d.controlFDLisa.CloseBatched(ctx)
+ } else {
+ if err := d.file.close(ctx); err != nil {
+ log.Warningf("gofer.dentry.destroyLocked: failed to close file: %v", err)
+ }
+ d.file = p9file{}
}
- d.file = p9file{}
// Remove d from the set of syncable dentries.
d.fs.syncMu.Lock()
@@ -1712,10 +2029,29 @@ func (d *dentry) setDeleted() {
atomic.StoreUint32(&d.deleted, 1)
}
-func (d *dentry) listXattr(ctx context.Context, creds *auth.Credentials, size uint64) ([]string, error) {
- if d.file.isNil() {
+func (d *dentry) isControlFileOk() bool {
+ if d.fs.opts.lisaEnabled {
+ return d.controlFDLisa.Ok()
+ }
+ return !d.file.isNil()
+}
+
+func (d *dentry) isReadFileOk() bool {
+ if d.fs.opts.lisaEnabled {
+ return d.readFDLisa.Ok()
+ }
+ return !d.readFile.isNil()
+}
+
+func (d *dentry) listXattr(ctx context.Context, size uint64) ([]string, error) {
+ if !d.isControlFileOk() {
return nil, nil
}
+
+ if d.fs.opts.lisaEnabled {
+ return d.controlFDLisa.ListXattr(ctx, size)
+ }
+
xattrMap, err := d.file.listXattr(ctx, size)
if err != nil {
return nil, err
@@ -1728,32 +2064,41 @@ func (d *dentry) listXattr(ctx context.Context, creds *auth.Credentials, size ui
}
func (d *dentry) getXattr(ctx context.Context, creds *auth.Credentials, opts *vfs.GetXattrOptions) (string, error) {
- if d.file.isNil() {
+ if !d.isControlFileOk() {
return "", linuxerr.ENODATA
}
if err := d.checkXattrPermissions(creds, opts.Name, vfs.MayRead); err != nil {
return "", err
}
+ if d.fs.opts.lisaEnabled {
+ return d.controlFDLisa.GetXattr(ctx, opts.Name, opts.Size)
+ }
return d.file.getXattr(ctx, opts.Name, opts.Size)
}
func (d *dentry) setXattr(ctx context.Context, creds *auth.Credentials, opts *vfs.SetXattrOptions) error {
- if d.file.isNil() {
+ if !d.isControlFileOk() {
return linuxerr.EPERM
}
if err := d.checkXattrPermissions(creds, opts.Name, vfs.MayWrite); err != nil {
return err
}
+ if d.fs.opts.lisaEnabled {
+ return d.controlFDLisa.SetXattr(ctx, opts.Name, opts.Value, opts.Flags)
+ }
return d.file.setXattr(ctx, opts.Name, opts.Value, opts.Flags)
}
func (d *dentry) removeXattr(ctx context.Context, creds *auth.Credentials, name string) error {
- if d.file.isNil() {
+ if !d.isControlFileOk() {
return linuxerr.EPERM
}
if err := d.checkXattrPermissions(creds, name, vfs.MayWrite); err != nil {
return err
}
+ if d.fs.opts.lisaEnabled {
+ return d.controlFDLisa.RemoveXattr(ctx, name)
+ }
return d.file.removeXattr(ctx, name)
}
@@ -1765,19 +2110,30 @@ func (d *dentry) ensureSharedHandle(ctx context.Context, read, write, trunc bool
// O_TRUNC).
if !trunc {
d.handleMu.RLock()
- if (!read || !d.readFile.isNil()) && (!write || !d.writeFile.isNil()) {
+ var canReuseCurHandle bool
+ if d.fs.opts.lisaEnabled {
+ canReuseCurHandle = (!read || d.readFDLisa.Ok()) && (!write || d.writeFDLisa.Ok())
+ } else {
+ canReuseCurHandle = (!read || !d.readFile.isNil()) && (!write || !d.writeFile.isNil())
+ }
+ d.handleMu.RUnlock()
+ if canReuseCurHandle {
// Current handles are sufficient.
- d.handleMu.RUnlock()
return nil
}
- d.handleMu.RUnlock()
}
var fdsToCloseArr [2]int32
fdsToClose := fdsToCloseArr[:0]
invalidateTranslations := false
d.handleMu.Lock()
- if (read && d.readFile.isNil()) || (write && d.writeFile.isNil()) || trunc {
+ var needNewHandle bool
+ if d.fs.opts.lisaEnabled {
+ needNewHandle = (read && !d.readFDLisa.Ok()) || (write && !d.writeFDLisa.Ok()) || trunc
+ } else {
+ needNewHandle = (read && d.readFile.isNil()) || (write && d.writeFile.isNil()) || trunc
+ }
+ if needNewHandle {
// Get a new handle. If this file has been opened for both reading and
// writing, try to get a single handle that is usable for both:
//
@@ -1786,9 +2142,21 @@ func (d *dentry) ensureSharedHandle(ctx context.Context, read, write, trunc bool
//
// - NOTE(b/141991141): Some filesystems may not ensure coherence
// between multiple handles for the same file.
- openReadable := !d.readFile.isNil() || read
- openWritable := !d.writeFile.isNil() || write
- h, err := openHandle(ctx, d.file, openReadable, openWritable, trunc)
+ var (
+ openReadable bool
+ openWritable bool
+ h handle
+ err error
+ )
+ if d.fs.opts.lisaEnabled {
+ openReadable = d.readFDLisa.Ok() || read
+ openWritable = d.writeFDLisa.Ok() || write
+ h, err = openHandleLisa(ctx, d.controlFDLisa, openReadable, openWritable, trunc)
+ } else {
+ openReadable = !d.readFile.isNil() || read
+ openWritable = !d.writeFile.isNil() || write
+ h, err = openHandle(ctx, d.file, openReadable, openWritable, trunc)
+ }
if linuxerr.Equals(linuxerr.EACCES, err) && (openReadable != read || openWritable != write) {
// It may not be possible to use a single handle for both
// reading and writing, since permissions on the file may have
@@ -1798,7 +2166,11 @@ func (d *dentry) ensureSharedHandle(ctx context.Context, read, write, trunc bool
ctx.Debugf("gofer.dentry.ensureSharedHandle: bifurcating read/write handles for dentry %p", d)
openReadable = read
openWritable = write
- h, err = openHandle(ctx, d.file, openReadable, openWritable, trunc)
+ if d.fs.opts.lisaEnabled {
+ h, err = openHandleLisa(ctx, d.controlFDLisa, openReadable, openWritable, trunc)
+ } else {
+ h, err = openHandle(ctx, d.file, openReadable, openWritable, trunc)
+ }
}
if err != nil {
d.handleMu.Unlock()
@@ -1860,9 +2232,16 @@ func (d *dentry) ensureSharedHandle(ctx context.Context, read, write, trunc bool
// previously opened for reading (without an FD), then existing
// translations of the file may use the internal page cache;
// invalidate those mappings.
- if d.writeFile.isNil() {
- invalidateTranslations = !d.readFile.isNil()
- atomic.StoreInt32(&d.mmapFD, h.fd)
+ if d.fs.opts.lisaEnabled {
+ if !d.writeFDLisa.Ok() {
+ invalidateTranslations = d.readFDLisa.Ok()
+ atomic.StoreInt32(&d.mmapFD, h.fd)
+ }
+ } else {
+ if d.writeFile.isNil() {
+ invalidateTranslations = !d.readFile.isNil()
+ atomic.StoreInt32(&d.mmapFD, h.fd)
+ }
}
} else if openWritable && d.writeFD < 0 {
atomic.StoreInt32(&d.writeFD, h.fd)
@@ -1889,24 +2268,45 @@ func (d *dentry) ensureSharedHandle(ctx context.Context, read, write, trunc bool
atomic.StoreInt32(&d.mmapFD, -1)
}
- // Switch to new fids.
- var oldReadFile p9file
- if openReadable {
- oldReadFile = d.readFile
- d.readFile = h.file
- }
- var oldWriteFile p9file
- if openWritable {
- oldWriteFile = d.writeFile
- d.writeFile = h.file
- }
- // NOTE(b/141991141): Clunk old fids before making new fids visible (by
- // unlocking d.handleMu).
- if !oldReadFile.isNil() {
- oldReadFile.close(ctx)
- }
- if !oldWriteFile.isNil() && oldReadFile != oldWriteFile {
- oldWriteFile.close(ctx)
+ // Switch to new fids/FDs.
+ if d.fs.opts.lisaEnabled {
+ oldReadFD := lisafs.InvalidFDID
+ if openReadable {
+ oldReadFD = d.readFDLisa.ID()
+ d.readFDLisa = h.fdLisa
+ }
+ oldWriteFD := lisafs.InvalidFDID
+ if openWritable {
+ oldWriteFD = d.writeFDLisa.ID()
+ d.writeFDLisa = h.fdLisa
+ }
+ // NOTE(b/141991141): Close old FDs before making new fids visible (by
+ // unlocking d.handleMu).
+ if oldReadFD.Ok() {
+ d.fs.clientLisa.CloseFDBatched(ctx, oldReadFD)
+ }
+ if oldWriteFD.Ok() && oldReadFD != oldWriteFD {
+ d.fs.clientLisa.CloseFDBatched(ctx, oldWriteFD)
+ }
+ } else {
+ var oldReadFile p9file
+ if openReadable {
+ oldReadFile = d.readFile
+ d.readFile = h.file
+ }
+ var oldWriteFile p9file
+ if openWritable {
+ oldWriteFile = d.writeFile
+ d.writeFile = h.file
+ }
+ // NOTE(b/141991141): Clunk old fids before making new fids visible (by
+ // unlocking d.handleMu).
+ if !oldReadFile.isNil() {
+ oldReadFile.close(ctx)
+ }
+ if !oldWriteFile.isNil() && oldReadFile != oldWriteFile {
+ oldWriteFile.close(ctx)
+ }
}
}
d.handleMu.Unlock()
@@ -1930,27 +2330,29 @@ func (d *dentry) ensureSharedHandle(ctx context.Context, read, write, trunc bool
// Preconditions: d.handleMu must be locked.
func (d *dentry) readHandleLocked() handle {
return handle{
- file: d.readFile,
- fd: d.readFD,
+ fdLisa: d.readFDLisa,
+ file: d.readFile,
+ fd: d.readFD,
}
}
// Preconditions: d.handleMu must be locked.
func (d *dentry) writeHandleLocked() handle {
return handle{
- file: d.writeFile,
- fd: d.writeFD,
+ fdLisa: d.writeFDLisa,
+ file: d.writeFile,
+ fd: d.writeFD,
}
}
func (d *dentry) syncRemoteFile(ctx context.Context) error {
d.handleMu.RLock()
defer d.handleMu.RUnlock()
- return d.syncRemoteFileLocked(ctx)
+ return d.syncRemoteFileLocked(ctx, nil /* accFsyncFDIDsLisa */)
}
// Preconditions: d.handleMu must be locked.
-func (d *dentry) syncRemoteFileLocked(ctx context.Context) error {
+func (d *dentry) syncRemoteFileLocked(ctx context.Context, accFsyncFDIDsLisa *[]lisafs.FDID) error {
// If we have a host FD, fsyncing it is likely to be faster than an fsync
// RPC. Prefer syncing write handles over read handles, since some remote
// filesystem implementations may not sync changes made through write
@@ -1961,7 +2363,13 @@ func (d *dentry) syncRemoteFileLocked(ctx context.Context) error {
ctx.UninterruptibleSleepFinish(false)
return err
}
- if !d.writeFile.isNil() {
+ if d.fs.opts.lisaEnabled && d.writeFDLisa.Ok() {
+ if accFsyncFDIDsLisa != nil {
+ *accFsyncFDIDsLisa = append(*accFsyncFDIDsLisa, d.writeFDLisa.ID())
+ return nil
+ }
+ return d.writeFDLisa.Sync(ctx)
+ } else if !d.fs.opts.lisaEnabled && !d.writeFile.isNil() {
return d.writeFile.fsync(ctx)
}
if d.readFD >= 0 {
@@ -1970,13 +2378,19 @@ func (d *dentry) syncRemoteFileLocked(ctx context.Context) error {
ctx.UninterruptibleSleepFinish(false)
return err
}
- if !d.readFile.isNil() {
+ if d.fs.opts.lisaEnabled && d.readFDLisa.Ok() {
+ if accFsyncFDIDsLisa != nil {
+ *accFsyncFDIDsLisa = append(*accFsyncFDIDsLisa, d.readFDLisa.ID())
+ return nil
+ }
+ return d.readFDLisa.Sync(ctx)
+ } else if !d.fs.opts.lisaEnabled && !d.readFile.isNil() {
return d.readFile.fsync(ctx)
}
return nil
}
-func (d *dentry) syncCachedFile(ctx context.Context, forFilesystemSync bool) error {
+func (d *dentry) syncCachedFile(ctx context.Context, forFilesystemSync bool, accFsyncFDIDsLisa *[]lisafs.FDID) error {
d.handleMu.RLock()
defer d.handleMu.RUnlock()
h := d.writeHandleLocked()
@@ -1989,7 +2403,7 @@ func (d *dentry) syncCachedFile(ctx context.Context, forFilesystemSync bool) err
return err
}
}
- if err := d.syncRemoteFileLocked(ctx); err != nil {
+ if err := d.syncRemoteFileLocked(ctx, accFsyncFDIDsLisa); err != nil {
if !forFilesystemSync {
return err
}
@@ -2046,18 +2460,33 @@ func (fd *fileDescription) Stat(ctx context.Context, opts vfs.StatOptions) (linu
d := fd.dentry()
const validMask = uint32(linux.STATX_MODE | linux.STATX_UID | linux.STATX_GID | linux.STATX_ATIME | linux.STATX_MTIME | linux.STATX_CTIME | linux.STATX_SIZE | linux.STATX_BLOCKS | linux.STATX_BTIME)
if !d.cachedMetadataAuthoritative() && opts.Mask&validMask != 0 && opts.Sync != linux.AT_STATX_DONT_SYNC {
- // Use specialFileFD.handle.file for the getattr if available, for the
- // same reason that we try to use open file handles in
- // dentry.updateFromGetattrLocked().
- var file p9file
- if sffd, ok := fd.vfsfd.Impl().(*specialFileFD); ok {
- file = sffd.handle.file
- }
- d.metadataMu.Lock()
- err := d.updateFromGetattrLocked(ctx, file)
- d.metadataMu.Unlock()
- if err != nil {
- return linux.Statx{}, err
+ if d.fs.opts.lisaEnabled {
+ // Use specialFileFD.handle.fileLisa for the Stat if available, for the
+ // same reason that we try to use open FD in updateFromStatLisaLocked().
+ var fdLisa *lisafs.ClientFD
+ if sffd, ok := fd.vfsfd.Impl().(*specialFileFD); ok {
+ fdLisa = &sffd.handle.fdLisa
+ }
+ d.metadataMu.Lock()
+ err := d.updateFromStatLisaLocked(ctx, fdLisa)
+ d.metadataMu.Unlock()
+ if err != nil {
+ return linux.Statx{}, err
+ }
+ } else {
+ // Use specialFileFD.handle.file for the getattr if available, for the
+ // same reason that we try to use open file handles in
+ // dentry.updateFromGetattrLocked().
+ var file p9file
+ if sffd, ok := fd.vfsfd.Impl().(*specialFileFD); ok {
+ file = sffd.handle.file
+ }
+ d.metadataMu.Lock()
+ err := d.updateFromGetattrLocked(ctx, file)
+ d.metadataMu.Unlock()
+ if err != nil {
+ return linux.Statx{}, err
+ }
}
}
var stat linux.Statx
@@ -2078,7 +2507,7 @@ func (fd *fileDescription) SetStat(ctx context.Context, opts vfs.SetStatOptions)
// ListXattr implements vfs.FileDescriptionImpl.ListXattr.
func (fd *fileDescription) ListXattr(ctx context.Context, size uint64) ([]string, error) {
- return fd.dentry().listXattr(ctx, auth.CredentialsFromContext(ctx), size)
+ return fd.dentry().listXattr(ctx, size)
}
// GetXattr implements vfs.FileDescriptionImpl.GetXattr.
diff --git a/pkg/sentry/fsimpl/gofer/gofer_test.go b/pkg/sentry/fsimpl/gofer/gofer_test.go
index 806392d50..d5cc73f33 100644
--- a/pkg/sentry/fsimpl/gofer/gofer_test.go
+++ b/pkg/sentry/fsimpl/gofer/gofer_test.go
@@ -33,6 +33,7 @@ func TestDestroyIdempotent(t *testing.T) {
},
syncableDentries: make(map[*dentry]struct{}),
inoByQIDPath: make(map[uint64]uint64),
+ inoByKey: make(map[inoKey]uint64),
}
attr := &p9.Attr{
diff --git a/pkg/sentry/fsimpl/gofer/handle.go b/pkg/sentry/fsimpl/gofer/handle.go
index 02540a754..394aecd62 100644
--- a/pkg/sentry/fsimpl/gofer/handle.go
+++ b/pkg/sentry/fsimpl/gofer/handle.go
@@ -17,6 +17,7 @@ package gofer
import (
"golang.org/x/sys/unix"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/lisafs"
"gvisor.dev/gvisor/pkg/p9"
"gvisor.dev/gvisor/pkg/safemem"
"gvisor.dev/gvisor/pkg/sentry/hostfd"
@@ -26,10 +27,13 @@ import (
// handle represents a remote "open file descriptor", consisting of an opened
// fid (p9.File) and optionally a host file descriptor.
//
+// If lisafs is being used, fdLisa points to an open file on the server.
+//
// These are explicitly not savable.
type handle struct {
- file p9file
- fd int32 // -1 if unavailable
+ fdLisa lisafs.ClientFD
+ file p9file
+ fd int32 // -1 if unavailable
}
// Preconditions: read || write.
@@ -65,13 +69,47 @@ func openHandle(ctx context.Context, file p9file, read, write, trunc bool) (hand
}, nil
}
+// Preconditions: read || write.
+func openHandleLisa(ctx context.Context, fdLisa lisafs.ClientFD, read, write, trunc bool) (handle, error) {
+ var flags uint32
+ switch {
+ case read && write:
+ flags = unix.O_RDWR
+ case read:
+ flags = unix.O_RDONLY
+ case write:
+ flags = unix.O_WRONLY
+ default:
+ panic("tried to open unreadable and unwritable handle")
+ }
+ if trunc {
+ flags |= unix.O_TRUNC
+ }
+ openFD, hostFD, err := fdLisa.OpenAt(ctx, flags)
+ if err != nil {
+ return handle{fd: -1}, err
+ }
+ h := handle{
+ fdLisa: fdLisa.Client().NewFD(openFD),
+ fd: int32(hostFD),
+ }
+ return h, nil
+}
+
func (h *handle) isOpen() bool {
+ if h.fdLisa.Client() != nil {
+ return h.fdLisa.Ok()
+ }
return !h.file.isNil()
}
func (h *handle) close(ctx context.Context) {
- h.file.close(ctx)
- h.file = p9file{}
+ if h.fdLisa.Client() != nil {
+ h.fdLisa.CloseBatched(ctx)
+ } else {
+ h.file.close(ctx)
+ h.file = p9file{}
+ }
if h.fd >= 0 {
unix.Close(int(h.fd))
h.fd = -1
@@ -89,19 +127,27 @@ func (h *handle) readToBlocksAt(ctx context.Context, dsts safemem.BlockSeq, offs
return n, err
}
if dsts.NumBlocks() == 1 && !dsts.Head().NeedSafecopy() {
- n, err := h.file.readAt(ctx, dsts.Head().ToSlice(), offset)
- return uint64(n), err
+ if h.fdLisa.Client() != nil {
+ return h.fdLisa.Read(ctx, dsts.Head().ToSlice(), offset)
+ }
+ return h.file.readAt(ctx, dsts.Head().ToSlice(), offset)
}
// Buffer the read since p9.File.ReadAt() takes []byte.
buf := make([]byte, dsts.NumBytes())
- n, err := h.file.readAt(ctx, buf, offset)
+ var n uint64
+ var err error
+ if h.fdLisa.Client() != nil {
+ n, err = h.fdLisa.Read(ctx, buf, offset)
+ } else {
+ n, err = h.file.readAt(ctx, buf, offset)
+ }
if n == 0 {
return 0, err
}
if cp, cperr := safemem.CopySeq(dsts, safemem.BlockSeqOf(safemem.BlockFromSafeSlice(buf[:n]))); cperr != nil {
return cp, cperr
}
- return uint64(n), err
+ return n, err
}
func (h *handle) writeFromBlocksAt(ctx context.Context, srcs safemem.BlockSeq, offset uint64) (uint64, error) {
@@ -115,8 +161,10 @@ func (h *handle) writeFromBlocksAt(ctx context.Context, srcs safemem.BlockSeq, o
return n, err
}
if srcs.NumBlocks() == 1 && !srcs.Head().NeedSafecopy() {
- n, err := h.file.writeAt(ctx, srcs.Head().ToSlice(), offset)
- return uint64(n), err
+ if h.fdLisa.Client() != nil {
+ return h.fdLisa.Write(ctx, srcs.Head().ToSlice(), offset)
+ }
+ return h.file.writeAt(ctx, srcs.Head().ToSlice(), offset)
}
// Buffer the write since p9.File.WriteAt() takes []byte.
buf := make([]byte, srcs.NumBytes())
@@ -124,12 +172,18 @@ func (h *handle) writeFromBlocksAt(ctx context.Context, srcs safemem.BlockSeq, o
if cp == 0 {
return 0, cperr
}
- n, err := h.file.writeAt(ctx, buf[:cp], offset)
+ var n uint64
+ var err error
+ if h.fdLisa.Client() != nil {
+ n, err = h.fdLisa.Write(ctx, buf[:cp], offset)
+ } else {
+ n, err = h.file.writeAt(ctx, buf[:cp], offset)
+ }
// err takes precedence over cperr.
if err != nil {
- return uint64(n), err
+ return n, err
}
- return uint64(n), cperr
+ return n, cperr
}
type handleReadWriter struct {
diff --git a/pkg/sentry/fsimpl/gofer/p9file.go b/pkg/sentry/fsimpl/gofer/p9file.go
index 5a3ddfc9d..0d97b60fd 100644
--- a/pkg/sentry/fsimpl/gofer/p9file.go
+++ b/pkg/sentry/fsimpl/gofer/p9file.go
@@ -141,18 +141,18 @@ func (f p9file) open(ctx context.Context, flags p9.OpenFlags) (*fd.FD, p9.QID, u
return fdobj, qid, iounit, err
}
-func (f p9file) readAt(ctx context.Context, p []byte, offset uint64) (int, error) {
+func (f p9file) readAt(ctx context.Context, p []byte, offset uint64) (uint64, error) {
ctx.UninterruptibleSleepStart(false)
n, err := f.file.ReadAt(p, offset)
ctx.UninterruptibleSleepFinish(false)
- return n, err
+ return uint64(n), err
}
-func (f p9file) writeAt(ctx context.Context, p []byte, offset uint64) (int, error) {
+func (f p9file) writeAt(ctx context.Context, p []byte, offset uint64) (uint64, error) {
ctx.UninterruptibleSleepStart(false)
n, err := f.file.WriteAt(p, offset)
ctx.UninterruptibleSleepFinish(false)
- return n, err
+ return uint64(n), err
}
func (f p9file) fsync(ctx context.Context) error {
diff --git a/pkg/sentry/fsimpl/gofer/regular_file.go b/pkg/sentry/fsimpl/gofer/regular_file.go
index 947dbe05f..874f9873d 100644
--- a/pkg/sentry/fsimpl/gofer/regular_file.go
+++ b/pkg/sentry/fsimpl/gofer/regular_file.go
@@ -98,6 +98,12 @@ func (fd *regularFileFD) OnClose(ctx context.Context) error {
}
d.handleMu.RLock()
defer d.handleMu.RUnlock()
+ if d.fs.opts.lisaEnabled {
+ if !d.writeFDLisa.Ok() {
+ return nil
+ }
+ return d.writeFDLisa.Flush(ctx)
+ }
if d.writeFile.isNil() {
return nil
}
@@ -110,6 +116,9 @@ func (fd *regularFileFD) Allocate(ctx context.Context, mode, offset, length uint
return d.doAllocate(ctx, offset, length, func() error {
d.handleMu.RLock()
defer d.handleMu.RUnlock()
+ if d.fs.opts.lisaEnabled {
+ return d.writeFDLisa.Allocate(ctx, mode, offset, length)
+ }
return d.writeFile.allocate(ctx, p9.ToAllocateMode(mode), offset, length)
})
}
@@ -282,8 +291,19 @@ func (fd *regularFileFD) pwrite(ctx context.Context, src usermem.IOSequence, off
// changes to the host.
if newMode := vfs.ClearSUIDAndSGID(oldMode); newMode != oldMode {
atomic.StoreUint32(&d.mode, newMode)
- if err := d.file.setAttr(ctx, p9.SetAttrMask{Permissions: true}, p9.SetAttr{Permissions: p9.FileMode(newMode)}); err != nil {
- return 0, offset, err
+ if d.fs.opts.lisaEnabled {
+ stat := linux.Statx{Mask: linux.STATX_MODE, Mode: uint16(newMode)}
+ failureMask, failureErr, err := d.controlFDLisa.SetStat(ctx, &stat)
+ if err != nil {
+ return 0, offset, err
+ }
+ if failureMask != 0 {
+ return 0, offset, failureErr
+ }
+ } else {
+ if err := d.file.setAttr(ctx, p9.SetAttrMask{Permissions: true}, p9.SetAttr{Permissions: p9.FileMode(newMode)}); err != nil {
+ return 0, offset, err
+ }
}
}
}
@@ -677,7 +697,7 @@ func regularFileSeekLocked(ctx context.Context, d *dentry, fdOffset, offset int6
// Sync implements vfs.FileDescriptionImpl.Sync.
func (fd *regularFileFD) Sync(ctx context.Context) error {
- return fd.dentry().syncCachedFile(ctx, false /* lowSyncExpectations */)
+ return fd.dentry().syncCachedFile(ctx, false /* forFilesystemSync */, nil /* accFsyncFDIDsLisa */)
}
// ConfigureMMap implements vfs.FileDescriptionImpl.ConfigureMMap.
diff --git a/pkg/sentry/fsimpl/gofer/revalidate.go b/pkg/sentry/fsimpl/gofer/revalidate.go
index 226790a11..5d4009832 100644
--- a/pkg/sentry/fsimpl/gofer/revalidate.go
+++ b/pkg/sentry/fsimpl/gofer/revalidate.go
@@ -15,7 +15,9 @@
package gofer
import (
+ "gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/p9"
"gvisor.dev/gvisor/pkg/sentry/vfs"
"gvisor.dev/gvisor/pkg/sync"
)
@@ -234,28 +236,54 @@ func (fs *filesystem) revalidateHelper(ctx context.Context, vfsObj *vfs.VirtualF
}
// Lock metadata on all dentries *before* getting attributes for them.
state.lockAllMetadata()
- stats, err := state.start.file.multiGetAttr(ctx, state.names)
- if err != nil {
- return err
+
+ var (
+ stats []p9.FullStat
+ statsLisa []linux.Statx
+ numStats int
+ )
+ if fs.opts.lisaEnabled {
+ var err error
+ statsLisa, err = state.start.controlFDLisa.WalkStat(ctx, state.names)
+ if err != nil {
+ return err
+ }
+ numStats = len(statsLisa)
+ } else {
+ var err error
+ stats, err = state.start.file.multiGetAttr(ctx, state.names)
+ if err != nil {
+ return err
+ }
+ numStats = len(stats)
}
i := -1
for d := state.popFront(); d != nil; d = state.popFront() {
i++
- found := i < len(stats)
+ found := i < numStats
if i == 0 && len(state.names[0]) == 0 {
if found && !d.isSynthetic() {
// First dentry is where the search is starting, just update attributes
// since it cannot be replaced.
- d.updateFromP9AttrsLocked(stats[i].Valid, &stats[i].Attr) // +checklocksforce: acquired by lockAllMetadata.
+ if fs.opts.lisaEnabled {
+ d.updateFromLisaStatLocked(&statsLisa[i]) // +checklocksforce: acquired by lockAllMetadata.
+ } else {
+ d.updateFromP9AttrsLocked(stats[i].Valid, &stats[i].Attr) // +checklocksforce: acquired by lockAllMetadata.
+ }
}
d.metadataMu.Unlock() // +checklocksforce: see above.
continue
}
- // Note that synthetic dentries will always fails the comparison check
- // below.
- if !found || d.qidPath != stats[i].QID.Path {
+ // Note that synthetic dentries will always fail this comparison check.
+ var shouldInvalidate bool
+ if fs.opts.lisaEnabled {
+ shouldInvalidate = !found || d.inoKey != inoKeyFromStat(&statsLisa[i])
+ } else {
+ shouldInvalidate = !found || d.qidPath != stats[i].QID.Path
+ }
+ if shouldInvalidate {
d.metadataMu.Unlock() // +checklocksforce: see above.
if !found && d.isSynthetic() {
// We have a synthetic file, and no remote file has arisen to replace
@@ -298,7 +326,11 @@ func (fs *filesystem) revalidateHelper(ctx context.Context, vfsObj *vfs.VirtualF
}
// The file at this path hasn't changed. Just update cached metadata.
- d.updateFromP9AttrsLocked(stats[i].Valid, &stats[i].Attr) // +checklocksforce: see above.
+ if fs.opts.lisaEnabled {
+ d.updateFromLisaStatLocked(&statsLisa[i]) // +checklocksforce: see above.
+ } else {
+ d.updateFromP9AttrsLocked(stats[i].Valid, &stats[i].Attr) // +checklocksforce: see above.
+ }
d.metadataMu.Unlock()
}
diff --git a/pkg/sentry/fsimpl/gofer/save_restore.go b/pkg/sentry/fsimpl/gofer/save_restore.go
index 8dcbc61ed..82878c056 100644
--- a/pkg/sentry/fsimpl/gofer/save_restore.go
+++ b/pkg/sentry/fsimpl/gofer/save_restore.go
@@ -24,6 +24,7 @@ import (
"gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/fdnotifier"
"gvisor.dev/gvisor/pkg/hostarch"
+ "gvisor.dev/gvisor/pkg/lisafs"
"gvisor.dev/gvisor/pkg/p9"
"gvisor.dev/gvisor/pkg/refsvfs2"
"gvisor.dev/gvisor/pkg/safemem"
@@ -112,10 +113,19 @@ func (d *dentry) prepareSaveRecursive(ctx context.Context) error {
return err
}
}
- if !d.readFile.isNil() || !d.writeFile.isNil() {
- d.fs.savedDentryRW[d] = savedDentryRW{
- read: !d.readFile.isNil(),
- write: !d.writeFile.isNil(),
+ if d.fs.opts.lisaEnabled {
+ if d.readFDLisa.Ok() || d.writeFDLisa.Ok() {
+ d.fs.savedDentryRW[d] = savedDentryRW{
+ read: d.readFDLisa.Ok(),
+ write: d.writeFDLisa.Ok(),
+ }
+ }
+ } else {
+ if !d.readFile.isNil() || !d.writeFile.isNil() {
+ d.fs.savedDentryRW[d] = savedDentryRW{
+ read: !d.readFile.isNil(),
+ write: !d.writeFile.isNil(),
+ }
}
}
d.dirMu.Lock()
@@ -177,25 +187,37 @@ func (fs *filesystem) CompleteRestore(ctx context.Context, opts vfs.CompleteRest
return fmt.Errorf("no server FD available for filesystem with unique ID %q", fs.iopts.UniqueID)
}
fs.opts.fd = fd
- if err := fs.dial(ctx); err != nil {
- return err
- }
fs.inoByQIDPath = make(map[uint64]uint64)
+ fs.inoByKey = make(map[inoKey]uint64)
- // Restore the filesystem root.
- ctx.UninterruptibleSleepStart(false)
- attached, err := fs.client.Attach(fs.opts.aname)
- ctx.UninterruptibleSleepFinish(false)
- if err != nil {
- return err
- }
- attachFile := p9file{attached}
- qid, attrMask, attr, err := attachFile.getAttr(ctx, dentryAttrMask())
- if err != nil {
- return err
- }
- if err := fs.root.restoreFile(ctx, attachFile, qid, attrMask, &attr, &opts); err != nil {
- return err
+ if fs.opts.lisaEnabled {
+ rootInode, err := fs.initClientLisa(ctx)
+ if err != nil {
+ return err
+ }
+ if err := fs.root.restoreFileLisa(ctx, rootInode, &opts); err != nil {
+ return err
+ }
+ } else {
+ if err := fs.dial(ctx); err != nil {
+ return err
+ }
+
+ // Restore the filesystem root.
+ ctx.UninterruptibleSleepStart(false)
+ attached, err := fs.client.Attach(fs.opts.aname)
+ ctx.UninterruptibleSleepFinish(false)
+ if err != nil {
+ return err
+ }
+ attachFile := p9file{attached}
+ qid, attrMask, attr, err := attachFile.getAttr(ctx, dentryAttrMask())
+ if err != nil {
+ return err
+ }
+ if err := fs.root.restoreFile(ctx, attachFile, qid, attrMask, &attr, &opts); err != nil {
+ return err
+ }
}
// Restore remaining dentries.
@@ -255,18 +277,18 @@ func (d *dentry) restoreFile(ctx context.Context, file p9file, qid p9.QID, attrM
if d.isRegularFile() {
if opts.ValidateFileSizes {
if !attrMask.Size {
- return fmt.Errorf("gofer.dentry(%q).restoreFile: file size validation failed: file size not available", genericDebugPathname(d))
+ return vfs.ErrCorruption{fmt.Errorf("gofer.dentry(%q).restoreFile: file size validation failed: file size not available", genericDebugPathname(d))}
}
if d.size != attr.Size {
- return fmt.Errorf("gofer.dentry(%q).restoreFile: file size validation failed: size changed from %d to %d", genericDebugPathname(d), d.size, attr.Size)
+ return vfs.ErrCorruption{fmt.Errorf("gofer.dentry(%q).restoreFile: file size validation failed: size changed from %d to %d", genericDebugPathname(d), d.size, attr.Size)}
}
}
if opts.ValidateFileModificationTimestamps {
if !attrMask.MTime {
- return fmt.Errorf("gofer.dentry(%q).restoreFile: mtime validation failed: mtime not available", genericDebugPathname(d))
+ return vfs.ErrCorruption{fmt.Errorf("gofer.dentry(%q).restoreFile: mtime validation failed: mtime not available", genericDebugPathname(d))}
}
if want := dentryTimestampFromP9(attr.MTimeSeconds, attr.MTimeNanoSeconds); d.mtime != want {
- return fmt.Errorf("gofer.dentry(%q).restoreFile: mtime validation failed: mtime changed from %+v to %+v", genericDebugPathname(d), linux.NsecToStatxTimestamp(d.mtime), linux.NsecToStatxTimestamp(want))
+ return vfs.ErrCorruption{fmt.Errorf("gofer.dentry(%q).restoreFile: mtime validation failed: mtime changed from %+v to %+v", genericDebugPathname(d), linux.NsecToStatxTimestamp(d.mtime), linux.NsecToStatxTimestamp(want))}
}
}
}
@@ -283,6 +305,55 @@ func (d *dentry) restoreFile(ctx context.Context, file p9file, qid p9.QID, attrM
return nil
}
+func (d *dentry) restoreFileLisa(ctx context.Context, inode *lisafs.Inode, opts *vfs.CompleteRestoreOptions) error {
+ d.controlFDLisa = d.fs.clientLisa.NewFD(inode.ControlFD)
+
+ // Gofers do not preserve inoKey across checkpoint/restore, so:
+ //
+ // - We must assume that the remote filesystem did not change in a way that
+ // would invalidate dentries, since we can't revalidate dentries by
+ // checking inoKey.
+ //
+ // - We need to associate the new inoKey with the existing d.ino.
+ d.inoKey = inoKeyFromStat(&inode.Stat)
+ d.fs.inoMu.Lock()
+ d.fs.inoByKey[d.inoKey] = d.ino
+ d.fs.inoMu.Unlock()
+
+ // Check metadata stability before updating metadata.
+ d.metadataMu.Lock()
+ defer d.metadataMu.Unlock()
+ if d.isRegularFile() {
+ if opts.ValidateFileSizes {
+ if inode.Stat.Mask&linux.STATX_SIZE != 0 {
+ return vfs.ErrCorruption{fmt.Errorf("gofer.dentry(%q).restoreFile: file size validation failed: file size not available", genericDebugPathname(d))}
+ }
+ if d.size != inode.Stat.Size {
+ return vfs.ErrCorruption{fmt.Errorf("gofer.dentry(%q).restoreFile: file size validation failed: size changed from %d to %d", genericDebugPathname(d), d.size, inode.Stat.Size)}
+ }
+ }
+ if opts.ValidateFileModificationTimestamps {
+ if inode.Stat.Mask&linux.STATX_MTIME != 0 {
+ return vfs.ErrCorruption{fmt.Errorf("gofer.dentry(%q).restoreFile: mtime validation failed: mtime not available", genericDebugPathname(d))}
+ }
+ if want := dentryTimestampFromLisa(inode.Stat.Mtime); d.mtime != want {
+ return vfs.ErrCorruption{fmt.Errorf("gofer.dentry(%q).restoreFile: mtime validation failed: mtime changed from %+v to %+v", genericDebugPathname(d), linux.NsecToStatxTimestamp(d.mtime), linux.NsecToStatxTimestamp(want))}
+ }
+ }
+ }
+ if !d.cachedMetadataAuthoritative() {
+ d.updateFromLisaStatLocked(&inode.Stat)
+ }
+
+ if rw, ok := d.fs.savedDentryRW[d]; ok {
+ if err := d.ensureSharedHandle(ctx, rw.read, rw.write, false /* trunc */); err != nil {
+ return err
+ }
+ }
+
+ return nil
+}
+
// Preconditions: d is not synthetic.
func (d *dentry) restoreDescendantsRecursive(ctx context.Context, opts *vfs.CompleteRestoreOptions) error {
for _, child := range d.children {
@@ -305,19 +376,35 @@ func (d *dentry) restoreDescendantsRecursive(ctx context.Context, opts *vfs.Comp
// only be detected by checking filesystem.syncableDentries). d.parent has been
// restored.
func (d *dentry) restoreRecursive(ctx context.Context, opts *vfs.CompleteRestoreOptions) error {
- qid, file, attrMask, attr, err := d.parent.file.walkGetAttrOne(ctx, d.name)
- if err != nil {
- return err
- }
- if err := d.restoreFile(ctx, file, qid, attrMask, &attr, opts); err != nil {
- return err
+ if d.fs.opts.lisaEnabled {
+ inode, err := d.parent.controlFDLisa.Walk(ctx, d.name)
+ if err != nil {
+ return err
+ }
+ if err := d.restoreFileLisa(ctx, inode, opts); err != nil {
+ return err
+ }
+ } else {
+ qid, file, attrMask, attr, err := d.parent.file.walkGetAttrOne(ctx, d.name)
+ if err != nil {
+ return err
+ }
+ if err := d.restoreFile(ctx, file, qid, attrMask, &attr, opts); err != nil {
+ return err
+ }
}
return d.restoreDescendantsRecursive(ctx, opts)
}
func (fd *specialFileFD) completeRestore(ctx context.Context) error {
d := fd.dentry()
- h, err := openHandle(ctx, d.file, fd.vfsfd.IsReadable(), fd.vfsfd.IsWritable(), false /* trunc */)
+ var h handle
+ var err error
+ if d.fs.opts.lisaEnabled {
+ h, err = openHandleLisa(ctx, d.controlFDLisa, fd.vfsfd.IsReadable(), fd.vfsfd.IsWritable(), false /* trunc */)
+ } else {
+ h, err = openHandle(ctx, d.file, fd.vfsfd.IsReadable(), fd.vfsfd.IsWritable(), false /* trunc */)
+ }
if err != nil {
return err
}
diff --git a/pkg/sentry/fsimpl/gofer/socket.go b/pkg/sentry/fsimpl/gofer/socket.go
index fe15f8583..86ab70453 100644
--- a/pkg/sentry/fsimpl/gofer/socket.go
+++ b/pkg/sentry/fsimpl/gofer/socket.go
@@ -59,11 +59,6 @@ func sockTypeToP9(t linux.SockType) (p9.ConnectFlags, bool) {
// BidirectionalConnect implements ConnectableEndpoint.BidirectionalConnect.
func (e *endpoint) BidirectionalConnect(ctx context.Context, ce transport.ConnectingEndpoint, returnConnect func(transport.Receiver, transport.ConnectedEndpoint)) *syserr.Error {
- cf, ok := sockTypeToP9(ce.Type())
- if !ok {
- return syserr.ErrConnectionRefused
- }
-
// No lock ordering required as only the ConnectingEndpoint has a mutex.
ce.Lock()
@@ -77,7 +72,7 @@ func (e *endpoint) BidirectionalConnect(ctx context.Context, ce transport.Connec
return syserr.ErrInvalidEndpointState
}
- c, err := e.newConnectedEndpoint(ctx, cf, ce.WaiterQueue())
+ c, err := e.newConnectedEndpoint(ctx, ce.Type(), ce.WaiterQueue())
if err != nil {
ce.Unlock()
return err
@@ -95,7 +90,7 @@ func (e *endpoint) BidirectionalConnect(ctx context.Context, ce transport.Connec
// UnidirectionalConnect implements
// transport.BoundEndpoint.UnidirectionalConnect.
func (e *endpoint) UnidirectionalConnect(ctx context.Context) (transport.ConnectedEndpoint, *syserr.Error) {
- c, err := e.newConnectedEndpoint(ctx, p9.DgramSocket, &waiter.Queue{})
+ c, err := e.newConnectedEndpoint(ctx, linux.SOCK_DGRAM, &waiter.Queue{})
if err != nil {
return nil, err
}
@@ -111,25 +106,39 @@ func (e *endpoint) UnidirectionalConnect(ctx context.Context) (transport.Connect
return c, nil
}
-func (e *endpoint) newConnectedEndpoint(ctx context.Context, flags p9.ConnectFlags, queue *waiter.Queue) (*host.SCMConnectedEndpoint, *syserr.Error) {
- hostFile, err := e.dentry.file.connect(ctx, flags)
- if err != nil {
+func (e *endpoint) newConnectedEndpoint(ctx context.Context, sockType linux.SockType, queue *waiter.Queue) (*host.SCMConnectedEndpoint, *syserr.Error) {
+ if e.dentry.fs.opts.lisaEnabled {
+ hostSockFD, err := e.dentry.controlFDLisa.Connect(ctx, sockType)
+ if err != nil {
+ return nil, syserr.ErrConnectionRefused
+ }
+
+ c, serr := host.NewSCMEndpoint(ctx, hostSockFD, queue, e.path)
+ if serr != nil {
+ unix.Close(hostSockFD)
+ log.Warningf("Gofer returned invalid host socket for BidirectionalConnect; file %+v sockType %d: %v", e.dentry.file, sockType, serr)
+ return nil, serr
+ }
+ return c, nil
+ }
+
+ flags, ok := sockTypeToP9(sockType)
+ if !ok {
return nil, syserr.ErrConnectionRefused
}
- // Dup the fd so that the new endpoint can manage its lifetime.
- hostFD, err := unix.Dup(hostFile.FD())
+ hostFile, err := e.dentry.file.connect(ctx, flags)
if err != nil {
- log.Warningf("Could not dup host socket fd %d: %v", hostFile.FD(), err)
- return nil, syserr.FromError(err)
+ return nil, syserr.ErrConnectionRefused
}
- // After duplicating, we no longer need hostFile.
- hostFile.Close()
- c, serr := host.NewSCMEndpoint(ctx, hostFD, queue, e.path)
+ c, serr := host.NewSCMEndpoint(ctx, hostFile.FD(), queue, e.path)
if serr != nil {
- log.Warningf("Gofer returned invalid host socket for BidirectionalConnect; file %+v flags %+v: %v", e.dentry.file, flags, serr)
+ hostFile.Close()
+ log.Warningf("Gofer returned invalid host socket for BidirectionalConnect; file %+v sockType %d: %v", e.dentry.file, sockType, serr)
return nil, serr
}
+ // Ownership has been transferred to c.
+ hostFile.Release()
return c, nil
}
diff --git a/pkg/sentry/fsimpl/gofer/special_file.go b/pkg/sentry/fsimpl/gofer/special_file.go
index a8d47b65b..c568bbfd2 100644
--- a/pkg/sentry/fsimpl/gofer/special_file.go
+++ b/pkg/sentry/fsimpl/gofer/special_file.go
@@ -23,6 +23,7 @@ import (
"gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/fdnotifier"
"gvisor.dev/gvisor/pkg/hostarch"
+ "gvisor.dev/gvisor/pkg/lisafs"
"gvisor.dev/gvisor/pkg/metric"
"gvisor.dev/gvisor/pkg/p9"
"gvisor.dev/gvisor/pkg/safemem"
@@ -149,6 +150,9 @@ func (fd *specialFileFD) OnClose(ctx context.Context) error {
if !fd.vfsfd.IsWritable() {
return nil
}
+ if fs := fd.filesystem(); fs.opts.lisaEnabled {
+ return fd.handle.fdLisa.Flush(ctx)
+ }
return fd.handle.file.flush(ctx)
}
@@ -184,6 +188,9 @@ func (fd *specialFileFD) Allocate(ctx context.Context, mode, offset, length uint
if fd.isRegularFile {
d := fd.dentry()
return d.doAllocate(ctx, offset, length, func() error {
+ if d.fs.opts.lisaEnabled {
+ return fd.handle.fdLisa.Allocate(ctx, mode, offset, length)
+ }
return fd.handle.file.allocate(ctx, p9.ToAllocateMode(mode), offset, length)
})
}
@@ -371,10 +378,10 @@ func (fd *specialFileFD) Seek(ctx context.Context, offset int64, whence int32) (
// Sync implements vfs.FileDescriptionImpl.Sync.
func (fd *specialFileFD) Sync(ctx context.Context) error {
- return fd.sync(ctx, false /* forFilesystemSync */)
+ return fd.sync(ctx, false /* forFilesystemSync */, nil /* accFsyncFDIDsLisa */)
}
-func (fd *specialFileFD) sync(ctx context.Context, forFilesystemSync bool) error {
+func (fd *specialFileFD) sync(ctx context.Context, forFilesystemSync bool, accFsyncFDIDsLisa *[]lisafs.FDID) error {
// Locks to ensure it didn't race with fd.Release().
fd.releaseMu.RLock()
defer fd.releaseMu.RUnlock()
@@ -391,6 +398,13 @@ func (fd *specialFileFD) sync(ctx context.Context, forFilesystemSync bool) error
ctx.UninterruptibleSleepFinish(false)
return err
}
+ if fs := fd.filesystem(); fs.opts.lisaEnabled {
+ if accFsyncFDIDsLisa != nil {
+ *accFsyncFDIDsLisa = append(*accFsyncFDIDsLisa, fd.handle.fdLisa.ID())
+ return nil
+ }
+ return fd.handle.fdLisa.Sync(ctx)
+ }
return fd.handle.file.fsync(ctx)
}()
if err != nil {
diff --git a/pkg/sentry/fsimpl/gofer/symlink.go b/pkg/sentry/fsimpl/gofer/symlink.go
index dbd834c67..27d9be5c4 100644
--- a/pkg/sentry/fsimpl/gofer/symlink.go
+++ b/pkg/sentry/fsimpl/gofer/symlink.go
@@ -35,7 +35,13 @@ func (d *dentry) readlink(ctx context.Context, mnt *vfs.Mount) (string, error) {
return target, nil
}
}
- target, err := d.file.readlink(ctx)
+ var target string
+ var err error
+ if d.fs.opts.lisaEnabled {
+ target, err = d.controlFDLisa.ReadLinkAt(ctx)
+ } else {
+ target, err = d.file.readlink(ctx)
+ }
if d.fs.opts.interop != InteropModeShared {
if err == nil {
d.haveTarget = true
diff --git a/pkg/sentry/fsimpl/gofer/time.go b/pkg/sentry/fsimpl/gofer/time.go
index 9cbe805b9..07940b225 100644
--- a/pkg/sentry/fsimpl/gofer/time.go
+++ b/pkg/sentry/fsimpl/gofer/time.go
@@ -17,6 +17,7 @@ package gofer
import (
"sync/atomic"
+ "gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/sentry/vfs"
)
@@ -24,6 +25,10 @@ func dentryTimestampFromP9(s, ns uint64) int64 {
return int64(s*1e9 + ns)
}
+func dentryTimestampFromLisa(t linux.StatxTimestamp) int64 {
+ return t.Sec*1e9 + int64(t.Nsec)
+}
+
// Preconditions: d.cachedMetadataAuthoritative() == true.
func (d *dentry) touchAtime(mnt *vfs.Mount) {
if mnt.Flags.NoATime || mnt.ReadOnly() {
diff --git a/pkg/sentry/fsimpl/mqfs/BUILD b/pkg/sentry/fsimpl/mqfs/BUILD
index e1a38686b..332c9b504 100644
--- a/pkg/sentry/fsimpl/mqfs/BUILD
+++ b/pkg/sentry/fsimpl/mqfs/BUILD
@@ -18,9 +18,9 @@ go_library(
name = "mqfs",
srcs = [
"mqfs.go",
- "root.go",
"queue.go",
"registry.go",
+ "root.go",
"root_inode_refs.go",
],
visibility = ["//pkg/sentry:internal"],
diff --git a/pkg/sentry/fsimpl/mqfs/mqfs.go b/pkg/sentry/fsimpl/mqfs/mqfs.go
index ed559cd13..c2b53c9d0 100644
--- a/pkg/sentry/fsimpl/mqfs/mqfs.go
+++ b/pkg/sentry/fsimpl/mqfs/mqfs.go
@@ -30,6 +30,7 @@ import (
)
const (
+ // Name is the user-visible filesystem name.
Name = "mqueue"
defaultMaxCachedDentries = uint64(1000)
)
@@ -73,7 +74,7 @@ func (ft FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.VirtualF
}
impl.fs.MaxCachedDentries = maxCachedDentries
- impl.root.IncRef()
+ impl.fs.VFSFilesystem().IncRef()
return impl.fs.VFSFilesystem(), impl.root.VFSDentry(), nil
}
@@ -109,7 +110,6 @@ type filesystem struct {
func (fs *filesystem) Release(ctx context.Context) {
fs.Filesystem.VFSFilesystem().VirtualFilesystem().PutAnonBlockDevMinor(fs.devMinor)
fs.Filesystem.Release(ctx)
- fs.root.DecRef(ctx)
}
// MountOptions implements vfs.FilesystemImpl.MountOptions.
diff --git a/pkg/sentry/fsimpl/mqfs/registry.go b/pkg/sentry/fsimpl/mqfs/registry.go
index 2c9c79f01..c8fbe4d33 100644
--- a/pkg/sentry/fsimpl/mqfs/registry.go
+++ b/pkg/sentry/fsimpl/mqfs/registry.go
@@ -63,11 +63,12 @@ func NewRegistryImpl(ctx context.Context, vfsObj *vfs.VirtualFilesystem, creds *
root: &dentry,
}
fs.VFSFilesystem().Init(vfsObj, &FilesystemType{}, fs)
+ vfsfs := fs.VFSFilesystem()
dentry.InitRoot(&fs.Filesystem, fs.newRootInode(ctx, creds))
- dentry.IncRef()
+ defer vfsfs.DecRef(ctx) // NewDisconnectedMount will obtain a ref on success.
- mount, err := vfsObj.NewDisconnectedMount(fs.VFSFilesystem(), dentry.VFSDentry(), &vfs.MountOptions{})
+ mount, err := vfsObj.NewDisconnectedMount(vfsfs, dentry.VFSDentry(), &vfs.MountOptions{})
if err != nil {
return nil, err
}
@@ -129,6 +130,7 @@ func (r *RegistryImpl) Unlink(ctx context.Context, name string) error {
// Destroy implements mq.RegistryImpl.Destroy.
func (r *RegistryImpl) Destroy(ctx context.Context) {
r.root.DecRef(ctx)
+ r.mount.DecRef(ctx)
}
// lookup retreives a kernfs.Inode using a name.
diff --git a/pkg/sentry/fsimpl/overlay/filesystem.go b/pkg/sentry/fsimpl/overlay/filesystem.go
index 3b3dcf836..044902241 100644
--- a/pkg/sentry/fsimpl/overlay/filesystem.go
+++ b/pkg/sentry/fsimpl/overlay/filesystem.go
@@ -86,7 +86,7 @@ func putDentrySlice(ds *[]*dentry) {
// fs.renameMuRUnlockAndCheckDrop(&ds)" than "defer func() {
// fs.renameMuRUnlockAndCheckDrop(ds) }()" to work around this.
//
-// +checklocksrelease:fs.renameMu
+// +checklocksreleaseread:fs.renameMu
func (fs *filesystem) renameMuRUnlockAndCheckDrop(ctx context.Context, dsp **[]*dentry) {
fs.renameMu.RUnlock()
if *dsp == nil {
diff --git a/pkg/sentry/fsimpl/proc/tasks.go b/pkg/sentry/fsimpl/proc/tasks.go
index 26d44744b..7b0be9c14 100644
--- a/pkg/sentry/fsimpl/proc/tasks.go
+++ b/pkg/sentry/fsimpl/proc/tasks.go
@@ -268,6 +268,6 @@ func cpuInfoData(k *kernel.Kernel) string {
return buf.String()
}
-func shmData(v uint64) dynamicInode {
+func ipcData(v uint64) dynamicInode {
return newStaticFile(strconv.FormatUint(v, 10))
}
diff --git a/pkg/sentry/fsimpl/proc/tasks_files.go b/pkg/sentry/fsimpl/proc/tasks_files.go
index 4d3a2f7e6..faec36d8d 100644
--- a/pkg/sentry/fsimpl/proc/tasks_files.go
+++ b/pkg/sentry/fsimpl/proc/tasks_files.go
@@ -262,9 +262,8 @@ var _ dynamicInode = (*meminfoData)(nil)
// Generate implements vfs.DynamicBytesSource.Generate.
func (*meminfoData) Generate(ctx context.Context, buf *bytes.Buffer) error {
- k := kernel.KernelFromContext(ctx)
- mf := k.MemoryFile()
- mf.UpdateUsage()
+ mf := kernel.KernelFromContext(ctx).MemoryFile()
+ _ = mf.UpdateUsage() // Best effort
snapshot, totalUsage := usage.MemoryAccounting.Copy()
totalSize := usage.TotalMemory(mf.TotalSize(), totalUsage)
anon := snapshot.Anonymous + snapshot.Tmpfs
diff --git a/pkg/sentry/fsimpl/proc/tasks_sys.go b/pkg/sentry/fsimpl/proc/tasks_sys.go
index 99f64a9d8..82e2857b3 100644
--- a/pkg/sentry/fsimpl/proc/tasks_sys.go
+++ b/pkg/sentry/fsimpl/proc/tasks_sys.go
@@ -47,9 +47,12 @@ func (fs *filesystem) newSysDir(ctx context.Context, root *auth.Credentials, k *
"kernel": fs.newStaticDir(ctx, root, map[string]kernfs.Inode{
"hostname": fs.newInode(ctx, root, 0444, &hostnameData{}),
"sem": fs.newInode(ctx, root, 0444, newStaticFile(fmt.Sprintf("%d\t%d\t%d\t%d\n", linux.SEMMSL, linux.SEMMNS, linux.SEMOPM, linux.SEMMNI))),
- "shmall": fs.newInode(ctx, root, 0444, shmData(linux.SHMALL)),
- "shmmax": fs.newInode(ctx, root, 0444, shmData(linux.SHMMAX)),
- "shmmni": fs.newInode(ctx, root, 0444, shmData(linux.SHMMNI)),
+ "shmall": fs.newInode(ctx, root, 0444, ipcData(linux.SHMALL)),
+ "shmmax": fs.newInode(ctx, root, 0444, ipcData(linux.SHMMAX)),
+ "shmmni": fs.newInode(ctx, root, 0444, ipcData(linux.SHMMNI)),
+ "msgmni": fs.newInode(ctx, root, 0444, ipcData(linux.MSGMNI)),
+ "msgmax": fs.newInode(ctx, root, 0444, ipcData(linux.MSGMAX)),
+ "msgmnb": fs.newInode(ctx, root, 0444, ipcData(linux.MSGMNB)),
"yama": fs.newStaticDir(ctx, root, map[string]kernfs.Inode{
"ptrace_scope": fs.newYAMAPtraceScopeFile(ctx, k, root),
}),
diff --git a/pkg/sentry/fsimpl/sys/sys.go b/pkg/sentry/fsimpl/sys/sys.go
index f322d2747..7fcb2d26b 100644
--- a/pkg/sentry/fsimpl/sys/sys.go
+++ b/pkg/sentry/fsimpl/sys/sys.go
@@ -84,6 +84,18 @@ func (fsType FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.Virt
fs.MaxCachedDentries = maxCachedDentries
fs.VFSFilesystem().Init(vfsObj, &fsType, fs)
+ k := kernel.KernelFromContext(ctx)
+ fsDirChildren := make(map[string]kernfs.Inode)
+ // Create an empty directory to serve as the mount point for cgroupfs when
+ // cgroups are available. This emulates Linux behaviour, see
+ // kernel/cgroup.c:cgroup_init(). Note that in Linux, userspace (typically
+ // the init process) is ultimately responsible for actually mounting
+ // cgroupfs, but the kernel creates the mountpoint. For the sentry, the
+ // launcher mounts cgroupfs.
+ if k.CgroupRegistry() != nil {
+ fsDirChildren["cgroup"] = fs.newDir(ctx, creds, defaultSysDirMode, nil)
+ }
+
root := fs.newDir(ctx, creds, defaultSysDirMode, map[string]kernfs.Inode{
"block": fs.newDir(ctx, creds, defaultSysDirMode, nil),
"bus": fs.newDir(ctx, creds, defaultSysDirMode, nil),
@@ -97,7 +109,7 @@ func (fsType FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.Virt
}),
}),
"firmware": fs.newDir(ctx, creds, defaultSysDirMode, nil),
- "fs": fs.newDir(ctx, creds, defaultSysDirMode, nil),
+ "fs": fs.newDir(ctx, creds, defaultSysDirMode, fsDirChildren),
"kernel": kernelDir(ctx, fs, creds),
"module": fs.newDir(ctx, creds, defaultSysDirMode, nil),
"power": fs.newDir(ctx, creds, defaultSysDirMode, nil),
diff --git a/pkg/sentry/fsimpl/sys/sys_test.go b/pkg/sentry/fsimpl/sys/sys_test.go
index 0a0d914cc..0c46a3a13 100644
--- a/pkg/sentry/fsimpl/sys/sys_test.go
+++ b/pkg/sentry/fsimpl/sys/sys_test.go
@@ -87,3 +87,17 @@ func TestSysRootContainsExpectedEntries(t *testing.T) {
"power": linux.DT_DIR,
})
}
+
+func TestCgroupMountpointExists(t *testing.T) {
+ // Note: The mountpoint is only created if cgroups are available. This is
+ // the VFS2 implementation of sysfs and the test runs with VFS2 enabled, so
+ // we expect to see the mount point unconditionally.
+ s := newTestSystem(t)
+ defer s.Destroy()
+ pop := s.PathOpAtRoot("/fs")
+ s.AssertAllDirentTypes(s.ListDirents(pop), map[string]testutil.DirentType{
+ "cgroup": linux.DT_DIR,
+ })
+ pop = s.PathOpAtRoot("/fs/cgroup")
+ s.AssertAllDirentTypes(s.ListDirents(pop), map[string]testutil.DirentType{ /*empty*/ })
+}
diff --git a/pkg/sentry/fsimpl/tmpfs/regular_file.go b/pkg/sentry/fsimpl/tmpfs/regular_file.go
index 0f2ac6144..453e1aa61 100644
--- a/pkg/sentry/fsimpl/tmpfs/regular_file.go
+++ b/pkg/sentry/fsimpl/tmpfs/regular_file.go
@@ -95,7 +95,7 @@ type regularFile struct {
func (fs *filesystem) newRegularFile(kuid auth.KUID, kgid auth.KGID, mode linux.FileMode, parentDir *directory) *inode {
file := &regularFile{
memFile: fs.mfp.MemoryFile(),
- memoryUsageKind: usage.Tmpfs,
+ memoryUsageKind: fs.usage,
seals: linux.F_SEAL_SEAL,
}
file.inode.init(file, fs, kuid, kgid, linux.S_IFREG|mode, parentDir)
diff --git a/pkg/sentry/fsimpl/tmpfs/tmpfs.go b/pkg/sentry/fsimpl/tmpfs/tmpfs.go
index feafb06e4..f84165aba 100644
--- a/pkg/sentry/fsimpl/tmpfs/tmpfs.go
+++ b/pkg/sentry/fsimpl/tmpfs/tmpfs.go
@@ -41,6 +41,7 @@ import (
"gvisor.dev/gvisor/pkg/sentry/kernel/auth"
"gvisor.dev/gvisor/pkg/sentry/kernel/time"
"gvisor.dev/gvisor/pkg/sentry/pgalloc"
+ "gvisor.dev/gvisor/pkg/sentry/usage"
"gvisor.dev/gvisor/pkg/sentry/vfs"
"gvisor.dev/gvisor/pkg/sentry/vfs/memxattr"
"gvisor.dev/gvisor/pkg/sync"
@@ -74,6 +75,10 @@ type filesystem struct {
// filesystem. Immutable.
mopts string
+ // usage is the memory accounting category under which pages backing
+ // files in this filesystem are accounted.
+ usage usage.MemoryKind
+
// mu serializes changes to the Dentry tree.
mu sync.RWMutex `state:"nosave"`
@@ -106,6 +111,10 @@ type FilesystemOpts struct {
// tmpfs filesystem. This allows tmpfs to "impersonate" other
// filesystems, like ramdiskfs and cgroupfs.
FilesystemType vfs.FilesystemType
+
+ // Usage is the memory accounting category under which pages backing files in
+ // the filesystem are accounted.
+ Usage *usage.MemoryKind
}
// GetFilesystem implements vfs.FilesystemType.GetFilesystem.
@@ -184,11 +193,16 @@ func (fstype FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.Virt
return nil, nil, err
}
clock := time.RealtimeClockFromContext(ctx)
+ memUsage := usage.Tmpfs
+ if tmpfsOpts.Usage != nil {
+ memUsage = *tmpfsOpts.Usage
+ }
fs := filesystem{
mfp: mfp,
clock: clock,
devMinor: devMinor,
mopts: opts.Data,
+ usage: memUsage,
}
fs.vfsfs.Init(vfsObj, newFSType, &fs)
diff --git a/pkg/sentry/fsimpl/verity/filesystem.go b/pkg/sentry/fsimpl/verity/filesystem.go
index 52d47994d..8b059aa7d 100644
--- a/pkg/sentry/fsimpl/verity/filesystem.go
+++ b/pkg/sentry/fsimpl/verity/filesystem.go
@@ -74,7 +74,7 @@ func putDentrySlice(ds *[]*dentry) {
// but dentry slices are allocated lazily, and it's much easier to say "defer
// fs.renameMuRUnlockAndCheckCaching(&ds)" than "defer func() {
// fs.renameMuRUnlockAndCheckCaching(ds) }()" to work around this.
-// +checklocksrelease:fs.renameMu
+// +checklocksreleaseread:fs.renameMu
func (fs *filesystem) renameMuRUnlockAndCheckCaching(ctx context.Context, ds **[]*dentry) {
fs.renameMu.RUnlock()
if *ds == nil {
diff --git a/pkg/sentry/hostmm/BUILD b/pkg/sentry/hostmm/BUILD
index 66fa1ad40..03c8e2f38 100644
--- a/pkg/sentry/hostmm/BUILD
+++ b/pkg/sentry/hostmm/BUILD
@@ -12,8 +12,7 @@ go_library(
visibility = ["//pkg/sentry:internal"],
deps = [
"//pkg/abi/linux",
- "//pkg/fd",
- "//pkg/hostarch",
+ "//pkg/eventfd",
"//pkg/log",
"@org_golang_x_sys//unix:go_default_library",
],
diff --git a/pkg/sentry/hostmm/hostmm.go b/pkg/sentry/hostmm/hostmm.go
index 285ea9050..5df06a60f 100644
--- a/pkg/sentry/hostmm/hostmm.go
+++ b/pkg/sentry/hostmm/hostmm.go
@@ -21,9 +21,7 @@ import (
"os"
"path"
- "golang.org/x/sys/unix"
- "gvisor.dev/gvisor/pkg/fd"
- "gvisor.dev/gvisor/pkg/hostarch"
+ "gvisor.dev/gvisor/pkg/eventfd"
"gvisor.dev/gvisor/pkg/log"
)
@@ -54,7 +52,7 @@ func NotifyCurrentMemcgPressureCallback(f func(), level string) (func(), error)
}
defer eventControlFile.Close()
- eventFD, err := newEventFD()
+ eventFD, err := eventfd.Create()
if err != nil {
return nil, err
}
@@ -75,20 +73,11 @@ func NotifyCurrentMemcgPressureCallback(f func(), level string) (func(), error)
const stopVal = 1 << 63
stopCh := make(chan struct{})
go func() { // S/R-SAFE: f provides synchronization if necessary
- rw := fd.NewReadWriter(eventFD.FD())
- var buf [sizeofUint64]byte
for {
- n, err := rw.Read(buf[:])
+ val, err := eventFD.Read()
if err != nil {
- if err == unix.EINTR {
- continue
- }
panic(fmt.Sprintf("failed to read from memory pressure level eventfd: %v", err))
}
- if n != sizeofUint64 {
- panic(fmt.Sprintf("short read from memory pressure level eventfd: got %d bytes, wanted %d", n, sizeofUint64))
- }
- val := hostarch.ByteOrder.Uint64(buf[:])
if val >= stopVal {
// Assume this was due to the notifier's "destructor" (the
// function returned by NotifyCurrentMemcgPressureCallback
@@ -101,30 +90,7 @@ func NotifyCurrentMemcgPressureCallback(f func(), level string) (func(), error)
}
}()
return func() {
- rw := fd.NewReadWriter(eventFD.FD())
- var buf [sizeofUint64]byte
- hostarch.ByteOrder.PutUint64(buf[:], stopVal)
- for {
- n, err := rw.Write(buf[:])
- if err != nil {
- if err == unix.EINTR {
- continue
- }
- panic(fmt.Sprintf("failed to write to memory pressure level eventfd: %v", err))
- }
- if n != sizeofUint64 {
- panic(fmt.Sprintf("short write to memory pressure level eventfd: got %d bytes, wanted %d", n, sizeofUint64))
- }
- break
- }
+ eventFD.Write(stopVal)
<-stopCh
}, nil
}
-
-func newEventFD() (*fd.FD, error) {
- f, _, e := unix.Syscall(unix.SYS_EVENTFD2, 0, 0, 0)
- if e != 0 {
- return nil, fmt.Errorf("failed to create eventfd: %v", e)
- }
- return fd.New(int(f)), nil
-}
diff --git a/pkg/sentry/inet/BUILD b/pkg/sentry/inet/BUILD
index 5bba9de0b..2363cec5f 100644
--- a/pkg/sentry/inet/BUILD
+++ b/pkg/sentry/inet/BUILD
@@ -1,13 +1,26 @@
load("//tools:defs.bzl", "go_library")
+load("//tools/go_generics:defs.bzl", "go_template_instance")
package(
default_visibility = ["//:sandbox"],
licenses = ["notice"],
)
+go_template_instance(
+ name = "atomicptr_netns",
+ out = "atomicptr_netns_unsafe.go",
+ package = "inet",
+ prefix = "Namespace",
+ template = "//pkg/sync/atomicptr:generic_atomicptr",
+ types = {
+ "Value": "Namespace",
+ },
+)
+
go_library(
name = "inet",
srcs = [
+ "atomicptr_netns_unsafe.go",
"context.go",
"inet.go",
"namespace.go",
diff --git a/pkg/sentry/kernel/BUILD b/pkg/sentry/kernel/BUILD
index 9f30a7706..f3f16eb7a 100644
--- a/pkg/sentry/kernel/BUILD
+++ b/pkg/sentry/kernel/BUILD
@@ -216,7 +216,6 @@ go_library(
visibility = ["//:sandbox"],
deps = [
":uncaught_signal_go_proto",
- "//pkg/sentry/kernel/ipc",
"//pkg/abi",
"//pkg/abi/linux",
"//pkg/abi/linux/errno",
@@ -257,8 +256,8 @@ go_library(
"//pkg/sentry/hostcpu",
"//pkg/sentry/inet",
"//pkg/sentry/kernel/auth",
- "//pkg/sentry/kernel/epoll",
"//pkg/sentry/kernel/futex",
+ "//pkg/sentry/kernel/ipc",
"//pkg/sentry/kernel/mq",
"//pkg/sentry/kernel/msgqueue",
"//pkg/sentry/kernel/sched",
diff --git a/pkg/sentry/kernel/epoll/epoll.go b/pkg/sentry/kernel/epoll/epoll.go
index 6006c46a9..8d0a21baf 100644
--- a/pkg/sentry/kernel/epoll/epoll.go
+++ b/pkg/sentry/kernel/epoll/epoll.go
@@ -66,7 +66,7 @@ type pollEntry struct {
file *refs.WeakRef `state:"manual"`
id FileIdentifier `state:"wait"`
userData [2]int32
- waiter waiter.Entry `state:"manual"`
+ waiter waiter.Entry
mask waiter.EventMask
flags EntryFlags
@@ -102,7 +102,7 @@ type EventPoll struct {
// Wait queue is used to notify interested parties when the event poll
// object itself becomes readable or writable.
- waiter.Queue `state:"zerovalue"`
+ waiter.Queue
// files is the map of all the files currently being observed, it is
// protected by mu.
@@ -454,14 +454,3 @@ func (e *EventPoll) RemoveEntry(ctx context.Context, id FileIdentifier) error {
return nil
}
-
-// UnregisterEpollWaiters removes the epoll waiter objects from the waiting
-// queues. This is different from Release() as the file is not dereferenced.
-func (e *EventPoll) UnregisterEpollWaiters() {
- e.mu.Lock()
- defer e.mu.Unlock()
-
- for _, entry := range e.files {
- entry.id.File.EventUnregister(&entry.waiter)
- }
-}
diff --git a/pkg/sentry/kernel/epoll/epoll_state.go b/pkg/sentry/kernel/epoll/epoll_state.go
index e08d6287f..135a6d72c 100644
--- a/pkg/sentry/kernel/epoll/epoll_state.go
+++ b/pkg/sentry/kernel/epoll/epoll_state.go
@@ -21,9 +21,7 @@ import (
// afterLoad is invoked by stateify.
func (p *pollEntry) afterLoad() {
- p.waiter.Callback = p
p.file = refs.NewWeakRef(p.id.File, p)
- p.id.File.EventRegister(&p.waiter, p.mask)
}
// afterLoad is invoked by stateify.
diff --git a/pkg/sentry/kernel/eventfd/eventfd.go b/pkg/sentry/kernel/eventfd/eventfd.go
index 5ea44a2c2..bf625dede 100644
--- a/pkg/sentry/kernel/eventfd/eventfd.go
+++ b/pkg/sentry/kernel/eventfd/eventfd.go
@@ -54,7 +54,7 @@ type EventOperations struct {
// Queue is used to notify interested parties when the event object
// becomes readable or writable.
- wq waiter.Queue `state:"zerovalue"`
+ wq waiter.Queue
// val is the current value of the event counter.
val uint64
diff --git a/pkg/sentry/kernel/ipc/BUILD b/pkg/sentry/kernel/ipc/BUILD
index a5cbb2b51..bb5cf1c17 100644
--- a/pkg/sentry/kernel/ipc/BUILD
+++ b/pkg/sentry/kernel/ipc/BUILD
@@ -5,9 +5,9 @@ package(licenses = ["notice"])
go_library(
name = "ipc",
srcs = [
+ "ns.go",
"object.go",
"registry.go",
- "ns.go",
],
visibility = ["//pkg/sentry:internal"],
deps = [
diff --git a/pkg/sentry/kernel/kernel.go b/pkg/sentry/kernel/kernel.go
index 04b24369a..d4851ccda 100644
--- a/pkg/sentry/kernel/kernel.go
+++ b/pkg/sentry/kernel/kernel.go
@@ -57,7 +57,6 @@ import (
"gvisor.dev/gvisor/pkg/sentry/hostcpu"
"gvisor.dev/gvisor/pkg/sentry/inet"
"gvisor.dev/gvisor/pkg/sentry/kernel/auth"
- "gvisor.dev/gvisor/pkg/sentry/kernel/epoll"
"gvisor.dev/gvisor/pkg/sentry/kernel/futex"
"gvisor.dev/gvisor/pkg/sentry/kernel/ipc"
"gvisor.dev/gvisor/pkg/sentry/kernel/sched"
@@ -79,11 +78,19 @@ import (
"gvisor.dev/gvisor/pkg/tcpip"
)
-// VFS2Enabled is set to true when VFS2 is enabled. Added as a global for allow
-// easy access everywhere. To be removed once VFS2 becomes the default.
+// VFS2Enabled is set to true when VFS2 is enabled. Added as a global to allow
+// easy access everywhere.
+//
+// TODO(gvisor.dev/issue/1624): Remove when VFS1 is no longer used.
var VFS2Enabled = false
-// FUSEEnabled is set to true when FUSE is enabled. Added as a global for allow
+// LISAFSEnabled is set to true when lisafs protocol is enabled. Added as a
+// global to allow easy access everywhere.
+//
+// TODO(gvisor.dev/issue/6319): Remove when lisafs is default.
+var LISAFSEnabled = false
+
+// FUSEEnabled is set to true when FUSE is enabled. Added as a global to allow
// easy access everywhere. To be removed once FUSE is completed.
var FUSEEnabled = false
@@ -484,11 +491,6 @@ func (k *Kernel) SaveTo(ctx context.Context, w wire.Writer) error {
return err
}
- // Remove all epoll waiter objects from underlying wait queues.
- // NOTE: for programs to resume execution in future snapshot scenarios,
- // we will need to re-establish these waiter objects after saving.
- k.tasks.unregisterEpollWaiters(ctx)
-
// Clear the dirent cache before saving because Dirents must be Loaded in a
// particular order (parents before children), and Loading dirents from a cache
// breaks that order.
@@ -621,32 +623,6 @@ func (k *Kernel) flushWritesToFiles(ctx context.Context) error {
})
}
-// Preconditions: !VFS2Enabled.
-func (ts *TaskSet) unregisterEpollWaiters(ctx context.Context) {
- ts.mu.RLock()
- defer ts.mu.RUnlock()
-
- // Tasks that belong to the same process could potentially point to the
- // same FDTable. So we retain a map of processed ones to avoid
- // processing the same FDTable multiple times.
- processed := make(map[*FDTable]struct{})
- for t := range ts.Root.tids {
- // We can skip locking Task.mu here since the kernel is paused.
- if t.fdTable == nil {
- continue
- }
- if _, ok := processed[t.fdTable]; ok {
- continue
- }
- t.fdTable.forEach(ctx, func(_ int32, file *fs.File, _ *vfs.FileDescription, _ FDFlags) {
- if e, ok := file.FileOperations.(*epoll.EventPoll); ok {
- e.UnregisterEpollWaiters()
- }
- })
- processed[t.fdTable] = struct{}{}
- }
-}
-
// Preconditions: The kernel must be paused.
func (k *Kernel) invalidateUnsavableMappings(ctx context.Context) error {
invalidated := make(map[*mm.MemoryManager]struct{})
diff --git a/pkg/sentry/kernel/mq/mq.go b/pkg/sentry/kernel/mq/mq.go
index a7c787081..50ca6d34a 100644
--- a/pkg/sentry/kernel/mq/mq.go
+++ b/pkg/sentry/kernel/mq/mq.go
@@ -40,8 +40,10 @@ const (
ReadWrite
)
+// MaxName is the maximum size for a queue name.
+const MaxName = 255
+
const (
- MaxName = 255 // Maximum size for a queue name.
maxPriority = linux.MQ_PRIO_MAX - 1 // Highest possible message priority.
maxQueuesDefault = linux.DFLT_QUEUESMAX // Default max number of queues.
diff --git a/pkg/sentry/kernel/pipe/pipe.go b/pkg/sentry/kernel/pipe/pipe.go
index 86beee6fe..8345473f3 100644
--- a/pkg/sentry/kernel/pipe/pipe.go
+++ b/pkg/sentry/kernel/pipe/pipe.go
@@ -55,7 +55,7 @@ const (
//
// +stateify savable
type Pipe struct {
- waiter.Queue `state:"nosave"`
+ waiter.Queue
// isNamed indicates whether this is a named pipe.
//
diff --git a/pkg/sentry/kernel/task.go b/pkg/sentry/kernel/task.go
index 9a95bf44c..1ea3c1bf7 100644
--- a/pkg/sentry/kernel/task.go
+++ b/pkg/sentry/kernel/task.go
@@ -158,7 +158,7 @@ type Task struct {
// signalQueue is protected by the signalMutex. Note that the task does
// not implement all queue methods, specifically the readiness checks.
// The task only broadcast a notification on signal delivery.
- signalQueue waiter.Queue `state:"zerovalue"`
+ signalQueue waiter.Queue
// If groupStopPending is true, the task should participate in a group
// stop in the interrupt path.
@@ -511,9 +511,7 @@ type Task struct {
numaNodeMask uint64
// netns is the task's network namespace. netns is never nil.
- //
- // netns is protected by mu.
- netns *inet.Namespace
+ netns inet.NamespaceAtomicPtr
// If rseqPreempted is true, before the next call to p.Switch(),
// interrupt rseq critical regions as defined by rseqAddr and
diff --git a/pkg/sentry/kernel/task_clone.go b/pkg/sentry/kernel/task_clone.go
index e174913d1..69a3227f0 100644
--- a/pkg/sentry/kernel/task_clone.go
+++ b/pkg/sentry/kernel/task_clone.go
@@ -447,7 +447,7 @@ func (t *Task) Unshare(flags int32) error {
t.mu.Unlock()
return linuxerr.EPERM
}
- t.netns = inet.NewNamespace(t.netns)
+ t.netns.Store(inet.NewNamespace(t.netns.Load()))
}
if flags&linux.CLONE_NEWUTS != 0 {
if !haveCapSysAdmin {
diff --git a/pkg/sentry/kernel/task_log.go b/pkg/sentry/kernel/task_log.go
index 8de08151a..f0c168ecc 100644
--- a/pkg/sentry/kernel/task_log.go
+++ b/pkg/sentry/kernel/task_log.go
@@ -191,9 +191,11 @@ const (
//
// Preconditions: The task's owning TaskSet.mu must be locked.
func (t *Task) updateInfoLocked() {
- // Use the task's TID in the root PID namespace for logging.
+ // Use the task's TID and PID in the root PID namespace for logging.
+ pid := t.tg.pidns.owner.Root.tgids[t.tg]
tid := t.tg.pidns.owner.Root.tids[t]
- t.logPrefix.Store(fmt.Sprintf("[% 4d] ", tid))
+ t.logPrefix.Store(fmt.Sprintf("[% 4d:% 4d] ", pid, tid))
+
t.rebuildTraceContext(tid)
}
@@ -249,5 +251,9 @@ func (t *Task) traceExecEvent(image *TaskImage) {
return
}
defer file.DecRef(t)
- trace.Logf(t.traceContext, traceCategory, "exec: %s", file.PathnameWithDeleted(t))
+
+ // traceExecEvent function may be called before the task goroutine
+ // starts, so we must use the async context.
+ name := file.PathnameWithDeleted(t.AsyncContext())
+ trace.Logf(t.traceContext, traceCategory, "exec: %s", name)
}
diff --git a/pkg/sentry/kernel/task_net.go b/pkg/sentry/kernel/task_net.go
index f7711232c..e31e2b2e8 100644
--- a/pkg/sentry/kernel/task_net.go
+++ b/pkg/sentry/kernel/task_net.go
@@ -20,9 +20,7 @@ import (
// IsNetworkNamespaced returns true if t is in a non-root network namespace.
func (t *Task) IsNetworkNamespaced() bool {
- t.mu.Lock()
- defer t.mu.Unlock()
- return !t.netns.IsRoot()
+ return !t.netns.Load().IsRoot()
}
// NetworkContext returns the network stack used by the task. NetworkContext
@@ -31,14 +29,10 @@ func (t *Task) IsNetworkNamespaced() bool {
// TODO(gvisor.dev/issue/1833): Migrate callers of this method to
// NetworkNamespace().
func (t *Task) NetworkContext() inet.Stack {
- t.mu.Lock()
- defer t.mu.Unlock()
- return t.netns.Stack()
+ return t.netns.Load().Stack()
}
// NetworkNamespace returns the network namespace observed by the task.
func (t *Task) NetworkNamespace() *inet.Namespace {
- t.mu.Lock()
- defer t.mu.Unlock()
- return t.netns
+ return t.netns.Load()
}
diff --git a/pkg/sentry/kernel/task_start.go b/pkg/sentry/kernel/task_start.go
index 217c6f531..4919dea7c 100644
--- a/pkg/sentry/kernel/task_start.go
+++ b/pkg/sentry/kernel/task_start.go
@@ -140,7 +140,6 @@ func (ts *TaskSet) newTask(cfg *TaskConfig) (*Task, error) {
allowedCPUMask: cfg.AllowedCPUMask.Copy(),
ioUsage: &usage.IO{},
niceness: cfg.Niceness,
- netns: cfg.NetworkNamespace,
utsns: cfg.UTSNamespace,
ipcns: cfg.IPCNamespace,
abstractSockets: cfg.AbstractSocketNamespace,
@@ -152,6 +151,7 @@ func (ts *TaskSet) newTask(cfg *TaskConfig) (*Task, error) {
containerID: cfg.ContainerID,
cgroups: make(map[Cgroup]struct{}),
}
+ t.netns.Store(cfg.NetworkNamespace)
t.creds.Store(cfg.Credentials)
t.endStopCond.L = &t.tg.signalHandlers.mu
t.ptraceTracer.Store((*Task)(nil))
diff --git a/pkg/sentry/kernel/threads.go b/pkg/sentry/kernel/threads.go
index 77ad62445..e38b723ce 100644
--- a/pkg/sentry/kernel/threads.go
+++ b/pkg/sentry/kernel/threads.go
@@ -324,11 +324,7 @@ type threadGroupNode struct {
// eventQueue is notified whenever a event of interest to Task.Wait occurs
// in a child of this thread group, or a ptrace tracee of a task in this
// thread group. Events are defined in task_exit.go.
- //
- // Note that we cannot check and save this wait queue similarly to other
- // wait queues, as the queue will not be empty by the time of saving, due
- // to the wait sourced from Exec().
- eventQueue waiter.Queue `state:"nosave"`
+ eventQueue waiter.Queue
// leader is the thread group's leader, which is the oldest task in the
// thread group; usually the last task in the thread group to call
diff --git a/pkg/sentry/mm/syscalls.go b/pkg/sentry/mm/syscalls.go
index 9e00c2cec..dc12ad357 100644
--- a/pkg/sentry/mm/syscalls.go
+++ b/pkg/sentry/mm/syscalls.go
@@ -89,7 +89,7 @@ func (mm *MemoryManager) MMap(ctx context.Context, opts memmap.MMapOpts) (hostar
}
// Offset + length must not overflow.
if end := opts.Offset + opts.Length; end < opts.Offset {
- return 0, linuxerr.ENOMEM
+ return 0, linuxerr.EOVERFLOW
}
} else {
opts.Offset = 0
diff --git a/pkg/sentry/platform/kvm/BUILD b/pkg/sentry/platform/kvm/BUILD
index 8a490b3de..834d72408 100644
--- a/pkg/sentry/platform/kvm/BUILD
+++ b/pkg/sentry/platform/kvm/BUILD
@@ -1,13 +1,26 @@
load("//tools:defs.bzl", "go_library", "go_test")
+load("//tools/go_generics:defs.bzl", "go_template_instance")
package(licenses = ["notice"])
+go_template_instance(
+ name = "atomicptr_machine",
+ out = "atomicptr_machine_unsafe.go",
+ package = "kvm",
+ prefix = "machine",
+ template = "//pkg/sync/atomicptr:generic_atomicptr",
+ types = {
+ "Value": "machine",
+ },
+)
+
go_library(
name = "kvm",
srcs = [
"address_space.go",
"address_space_amd64.go",
"address_space_arm64.go",
+ "atomicptr_machine_unsafe.go",
"bluepill.go",
"bluepill_allocator.go",
"bluepill_amd64.go",
@@ -50,7 +63,6 @@ go_library(
"//pkg/procid",
"//pkg/ring0",
"//pkg/ring0/pagetables",
- "//pkg/safecopy",
"//pkg/seccomp",
"//pkg/sentry/arch",
"//pkg/sentry/arch/fpu",
@@ -58,6 +70,7 @@ go_library(
"//pkg/sentry/platform",
"//pkg/sentry/platform/interrupt",
"//pkg/sentry/time",
+ "//pkg/sighandling",
"//pkg/sync",
"@org_golang_x_sys//unix:go_default_library",
],
@@ -69,10 +82,17 @@ go_test(
"kvm_amd64_test.go",
"kvm_amd64_test.s",
"kvm_arm64_test.go",
+ "kvm_safecopy_test.go",
"kvm_test.go",
"virtual_map_test.go",
],
library = ":kvm",
+ # FIXME(gvisor.dev/issue/3374): Not working with all build systems.
+ nogo = False,
+ # cgo has to be disabled. We have seen libc that blocks all signals and
+ # calls mmap from pthread_create, but we use SIGSYS to trap mmap system
+ # calls.
+ pure = True,
tags = [
"manual",
"nogotsan",
@@ -81,8 +101,10 @@ go_test(
deps = [
"//pkg/abi/linux",
"//pkg/hostarch",
+ "//pkg/memutil",
"//pkg/ring0",
"//pkg/ring0/pagetables",
+ "//pkg/safecopy",
"//pkg/sentry/arch",
"//pkg/sentry/arch/fpu",
"//pkg/sentry/platform",
diff --git a/pkg/sentry/platform/kvm/bluepill.go b/pkg/sentry/platform/kvm/bluepill.go
index bb9967b9f..5be2215ed 100644
--- a/pkg/sentry/platform/kvm/bluepill.go
+++ b/pkg/sentry/platform/kvm/bluepill.go
@@ -19,8 +19,8 @@ import (
"golang.org/x/sys/unix"
"gvisor.dev/gvisor/pkg/ring0"
- "gvisor.dev/gvisor/pkg/safecopy"
"gvisor.dev/gvisor/pkg/sentry/arch"
+ "gvisor.dev/gvisor/pkg/sighandling"
)
// bluepill enters guest mode.
@@ -61,6 +61,9 @@ var (
// This is called by bluepillHandler.
savedHandler uintptr
+ // savedSigsysHandler is a pointer to the previos handler of the SIGSYS signals.
+ savedSigsysHandler uintptr
+
// dieTrampolineAddr is the address of dieTrampoline.
dieTrampolineAddr uintptr
)
@@ -94,7 +97,7 @@ func (c *vCPU) die(context *arch.SignalContext64, msg string) {
func init() {
// Install the handler.
- if err := safecopy.ReplaceSignalHandler(bluepillSignal, addrOfSighandler(), &savedHandler); err != nil {
+ if err := sighandling.ReplaceSignalHandler(bluepillSignal, addrOfSighandler(), &savedHandler); err != nil {
panic(fmt.Sprintf("Unable to set handler for signal %d: %v", bluepillSignal, err))
}
diff --git a/pkg/sentry/platform/kvm/bluepill_amd64.go b/pkg/sentry/platform/kvm/bluepill_amd64.go
index 0567c8d32..b2db2bb9f 100644
--- a/pkg/sentry/platform/kvm/bluepill_amd64.go
+++ b/pkg/sentry/platform/kvm/bluepill_amd64.go
@@ -71,10 +71,6 @@ func (c *vCPU) KernelSyscall() {
if regs.Rax != ^uint64(0) {
regs.Rip -= 2 // Rewind.
}
- // We only trigger a bluepill entry in the bluepill function, and can
- // therefore be guaranteed that there is no floating point state to be
- // loaded on resuming from halt. We only worry about saving on exit.
- ring0.SaveFloatingPoint(c.floatingPointState.BytePointer()) // escapes: no.
// N.B. Since KernelSyscall is called when the kernel makes a syscall,
// FS_BASE is already set for correct execution of this function.
//
@@ -112,8 +108,6 @@ func (c *vCPU) KernelException(vector ring0.Vector) {
regs.Rip = 0
}
// See above.
- ring0.SaveFloatingPoint(c.floatingPointState.BytePointer()) // escapes: no.
- // See above.
ring0.HaltAndWriteFSBase(regs) // escapes: no, reload host segment.
}
@@ -144,5 +138,5 @@ func bluepillArchExit(c *vCPU, context *arch.SignalContext64) {
// Set the context pointer to the saved floating point state. This is
// where the guest data has been serialized, the kernel will restore
// from this new pointer value.
- context.Fpstate = uint64(uintptrValue(c.floatingPointState.BytePointer()))
+ context.Fpstate = uint64(uintptrValue(c.FloatingPointState().BytePointer())) // escapes: no.
}
diff --git a/pkg/sentry/platform/kvm/bluepill_amd64.s b/pkg/sentry/platform/kvm/bluepill_amd64.s
index c2a1dca11..5d8358f64 100644
--- a/pkg/sentry/platform/kvm/bluepill_amd64.s
+++ b/pkg/sentry/platform/kvm/bluepill_amd64.s
@@ -32,6 +32,8 @@
// This is checked as the source of the fault.
#define CLI $0xfa
+#define SYS_MMAP 9
+
// See bluepill.go.
TEXT ·bluepill(SB),NOSPLIT,$0
begin:
@@ -95,6 +97,31 @@ TEXT ·addrOfSighandler(SB), $0-8
MOVQ AX, ret+0(FP)
RET
+TEXT ·sigsysHandler(SB),NOSPLIT,$0
+ // Check if the signal is from the kernel.
+ MOVQ $1, CX
+ CMPL CX, 0x8(SI)
+ JNE fallback
+
+ MOVL CONTEXT_RAX(DX), CX
+ CMPL CX, $SYS_MMAP
+ JNE fallback
+ PUSHQ DX // First argument (context).
+ CALL ·seccompMmapHandler(SB) // Call the handler.
+ POPQ DX // Discard the argument.
+ RET
+fallback:
+ // Jump to the previous signal handler.
+ XORQ CX, CX
+ MOVQ ·savedSigsysHandler(SB), AX
+ JMP AX
+
+// func addrOfSighandler() uintptr
+TEXT ·addrOfSigsysHandler(SB), $0-8
+ MOVQ $·sigsysHandler(SB), AX
+ MOVQ AX, ret+0(FP)
+ RET
+
// dieTrampoline: see bluepill.go, bluepill_amd64_unsafe.go for documentation.
TEXT ·dieTrampoline(SB),NOSPLIT,$0
PUSHQ BX // First argument (vCPU).
diff --git a/pkg/sentry/platform/kvm/bluepill_arm64.go b/pkg/sentry/platform/kvm/bluepill_arm64.go
index acb0cb05f..df772d620 100644
--- a/pkg/sentry/platform/kvm/bluepill_arm64.go
+++ b/pkg/sentry/platform/kvm/bluepill_arm64.go
@@ -70,7 +70,7 @@ func bluepillArchExit(c *vCPU, context *arch.SignalContext64) {
lazyVfp := c.GetLazyVFP()
if lazyVfp != 0 {
- fpsimd := fpsimdPtr(c.floatingPointState.BytePointer())
+ fpsimd := fpsimdPtr(c.FloatingPointState().BytePointer()) // escapes: no
context.Fpsimd64.Fpsr = fpsimd.Fpsr
context.Fpsimd64.Fpcr = fpsimd.Fpcr
context.Fpsimd64.Vregs = fpsimd.Vregs
@@ -90,12 +90,12 @@ func (c *vCPU) KernelSyscall() {
fpDisableTrap := ring0.CPACREL1()
if fpDisableTrap != 0 {
- fpsimd := fpsimdPtr(c.floatingPointState.BytePointer())
+ fpsimd := fpsimdPtr(c.FloatingPointState().BytePointer()) // escapes: no
fpcr := ring0.GetFPCR()
fpsr := ring0.GetFPSR()
fpsimd.Fpcr = uint32(fpcr)
fpsimd.Fpsr = uint32(fpsr)
- ring0.SaveVRegs(c.floatingPointState.BytePointer())
+ ring0.SaveVRegs(c.FloatingPointState().BytePointer()) // escapes: no
}
ring0.Halt()
@@ -114,12 +114,12 @@ func (c *vCPU) KernelException(vector ring0.Vector) {
fpDisableTrap := ring0.CPACREL1()
if fpDisableTrap != 0 {
- fpsimd := fpsimdPtr(c.floatingPointState.BytePointer())
+ fpsimd := fpsimdPtr(c.FloatingPointState().BytePointer()) // escapes: no
fpcr := ring0.GetFPCR()
fpsr := ring0.GetFPSR()
fpsimd.Fpcr = uint32(fpcr)
fpsimd.Fpsr = uint32(fpsr)
- ring0.SaveVRegs(c.floatingPointState.BytePointer())
+ ring0.SaveVRegs(c.FloatingPointState().BytePointer()) // escapes: no
}
ring0.Halt()
diff --git a/pkg/sentry/platform/kvm/bluepill_arm64.s b/pkg/sentry/platform/kvm/bluepill_arm64.s
index 308f2a951..9690e3772 100644
--- a/pkg/sentry/platform/kvm/bluepill_arm64.s
+++ b/pkg/sentry/platform/kvm/bluepill_arm64.s
@@ -29,9 +29,12 @@
// Only limited use of the context is done in the assembly stub below, most is
// done in the Go handlers.
#define SIGINFO_SIGNO 0x0
+#define SIGINFO_CODE 0x8
#define CONTEXT_PC 0x1B8
#define CONTEXT_R0 0xB8
+#define SYS_MMAP 222
+
// getTLS returns the value of TPIDR_EL0 register.
TEXT ·getTLS(SB),NOSPLIT,$0-8
MRS TPIDR_EL0, R1
@@ -98,6 +101,37 @@ TEXT ·addrOfSighandler(SB), $0-8
MOVD R0, ret+0(FP)
RET
+// The arguments are the following:
+//
+// R0 - The signal number.
+// R1 - Pointer to siginfo_t structure.
+// R2 - Pointer to ucontext structure.
+//
+TEXT ·sigsysHandler(SB),NOSPLIT,$0
+ // si_code should be SYS_SECCOMP.
+ MOVD SIGINFO_CODE(R1), R7
+ CMPW $1, R7
+ BNE fallback
+
+ CMPW $SYS_MMAP, R8
+ BNE fallback
+
+ MOVD R2, 8(RSP)
+ BL ·seccompMmapHandler(SB) // Call the handler.
+
+ RET
+
+fallback:
+ // Jump to the previous signal handler.
+ MOVD ·savedHandler(SB), R7
+ B (R7)
+
+// func addrOfSighandler() uintptr
+TEXT ·addrOfSigsysHandler(SB), $0-8
+ MOVD $·sigsysHandler(SB), R0
+ MOVD R0, ret+0(FP)
+ RET
+
// dieTrampoline: see bluepill.go, bluepill_arm64_unsafe.go for documentation.
TEXT ·dieTrampoline(SB),NOSPLIT,$0
// R0: Fake the old PC as caller
diff --git a/pkg/sentry/platform/kvm/bluepill_unsafe.go b/pkg/sentry/platform/kvm/bluepill_unsafe.go
index 0f0c1e73b..e38ca05c0 100644
--- a/pkg/sentry/platform/kvm/bluepill_unsafe.go
+++ b/pkg/sentry/platform/kvm/bluepill_unsafe.go
@@ -193,36 +193,8 @@ func bluepillHandler(context unsafe.Pointer) {
return
}
- // Increment the fault count.
- atomic.AddUint32(&c.faults, 1)
-
- // For MMIO, the physical address is the first data item.
- physical = uintptr(c.runData.data[0])
- virtual, ok := handleBluepillFault(c.machine, physical, physicalRegions, _KVM_MEM_FLAGS_NONE)
- if !ok {
- c.die(bluepillArchContext(context), "invalid physical address")
- return
- }
-
- // We now need to fill in the data appropriately. KVM
- // expects us to provide the result of the given MMIO
- // operation in the runData struct. This is safe
- // because, if a fault occurs here, the same fault
- // would have occurred in guest mode. The kernel should
- // not create invalid page table mappings.
- data := (*[8]byte)(unsafe.Pointer(&c.runData.data[1]))
- length := (uintptr)((uint32)(c.runData.data[2]))
- write := (uint8)(((c.runData.data[2] >> 32) & 0xff)) != 0
- for i := uintptr(0); i < length; i++ {
- b := bytePtr(uintptr(virtual) + i)
- if write {
- // Write to the given address.
- *b = data[i]
- } else {
- // Read from the given address.
- data[i] = *b
- }
- }
+ c.die(bluepillArchContext(context), "exit_mmio")
+ return
case _KVM_EXIT_IRQ_WINDOW_OPEN:
bluepillStopGuest(c)
case _KVM_EXIT_SHUTDOWN:
diff --git a/pkg/sentry/platform/kvm/kvm.go b/pkg/sentry/platform/kvm/kvm.go
index aac0fdffe..ad6863646 100644
--- a/pkg/sentry/platform/kvm/kvm.go
+++ b/pkg/sentry/platform/kvm/kvm.go
@@ -77,7 +77,11 @@ var (
// OpenDevice opens the KVM device at /dev/kvm and returns the File.
func OpenDevice() (*os.File, error) {
- f, err := os.OpenFile("/dev/kvm", unix.O_RDWR, 0)
+ dev, ok := os.LookupEnv("GVISOR_KVM_DEV")
+ if !ok {
+ dev = "/dev/kvm"
+ }
+ f, err := os.OpenFile(dev, unix.O_RDWR, 0)
if err != nil {
return nil, fmt.Errorf("error opening /dev/kvm: %v", err)
}
diff --git a/pkg/sentry/platform/kvm/kvm_safecopy_test.go b/pkg/sentry/platform/kvm/kvm_safecopy_test.go
new file mode 100644
index 000000000..fe488e707
--- /dev/null
+++ b/pkg/sentry/platform/kvm/kvm_safecopy_test.go
@@ -0,0 +1,104 @@
+// Copyright 2021 The gVisor Authors.
+//
+// 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.
+
+// FIXME(gvisor.dev/issue/6629): These tests don't pass on ARM64.
+//
+//go:build amd64
+// +build amd64
+
+package kvm
+
+import (
+ "fmt"
+ "os"
+ "testing"
+ "unsafe"
+
+ "golang.org/x/sys/unix"
+ "gvisor.dev/gvisor/pkg/hostarch"
+ "gvisor.dev/gvisor/pkg/memutil"
+ "gvisor.dev/gvisor/pkg/safecopy"
+)
+
+func testSafecopy(t *testing.T, mapSize uintptr, fileSize uintptr, testFunc func(t *testing.T, c *vCPU, addr uintptr)) {
+ memfd, err := memutil.CreateMemFD(fmt.Sprintf("kvm_test_%d", os.Getpid()), 0)
+ if err != nil {
+ t.Errorf("error creating memfd: %v", err)
+ }
+
+ memfile := os.NewFile(uintptr(memfd), "kvm_test")
+ memfile.Truncate(int64(fileSize))
+ kvmTest(t, nil, func(c *vCPU) bool {
+ const n = 10
+ mappings := make([]uintptr, n)
+ defer func() {
+ for i := 0; i < n && mappings[i] != 0; i++ {
+ unix.RawSyscall(
+ unix.SYS_MUNMAP,
+ mappings[i], mapSize, 0)
+ }
+ }()
+ for i := 0; i < n; i++ {
+ addr, _, errno := unix.RawSyscall6(
+ unix.SYS_MMAP,
+ 0,
+ mapSize,
+ unix.PROT_READ|unix.PROT_WRITE,
+ unix.MAP_SHARED|unix.MAP_FILE,
+ uintptr(memfile.Fd()),
+ 0)
+ if errno != 0 {
+ t.Errorf("error mapping file: %v", errno)
+ }
+ mappings[i] = addr
+ testFunc(t, c, addr)
+ }
+ return false
+ })
+}
+
+func TestSafecopySigbus(t *testing.T) {
+ mapSize := uintptr(faultBlockSize)
+ fileSize := mapSize - hostarch.PageSize
+ buf := make([]byte, hostarch.PageSize)
+ testSafecopy(t, mapSize, fileSize, func(t *testing.T, c *vCPU, addr uintptr) {
+ want := safecopy.BusError{addr + fileSize}
+ bluepill(c)
+ _, err := safecopy.CopyIn(buf, unsafe.Pointer(addr+fileSize))
+ if err != want {
+ t.Errorf("expected error: got %v, want %v", err, want)
+ }
+ })
+}
+
+func TestSafecopy(t *testing.T) {
+ mapSize := uintptr(faultBlockSize)
+ fileSize := mapSize
+ testSafecopy(t, mapSize, fileSize, func(t *testing.T, c *vCPU, addr uintptr) {
+ want := uint32(0x12345678)
+ bluepill(c)
+ _, err := safecopy.SwapUint32(unsafe.Pointer(addr+fileSize-8), want)
+ if err != nil {
+ t.Errorf("unexpected error: %v", err)
+ }
+ bluepill(c)
+ val, err := safecopy.LoadUint32(unsafe.Pointer(addr + fileSize - 8))
+ if err != nil {
+ t.Errorf("unexpected error: %v", err)
+ }
+ if val != want {
+ t.Errorf("incorrect value: got %x, want %x", val, want)
+ }
+ })
+}
diff --git a/pkg/sentry/platform/kvm/machine.go b/pkg/sentry/platform/kvm/machine.go
index d67563958..f1f7e4ea4 100644
--- a/pkg/sentry/platform/kvm/machine.go
+++ b/pkg/sentry/platform/kvm/machine.go
@@ -17,16 +17,20 @@ package kvm
import (
"fmt"
"runtime"
+ gosync "sync"
"sync/atomic"
"golang.org/x/sys/unix"
+ "gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/atomicbitops"
"gvisor.dev/gvisor/pkg/hostarch"
"gvisor.dev/gvisor/pkg/log"
"gvisor.dev/gvisor/pkg/procid"
"gvisor.dev/gvisor/pkg/ring0"
"gvisor.dev/gvisor/pkg/ring0/pagetables"
+ "gvisor.dev/gvisor/pkg/seccomp"
ktime "gvisor.dev/gvisor/pkg/sentry/time"
+ "gvisor.dev/gvisor/pkg/sighandling"
"gvisor.dev/gvisor/pkg/sync"
)
@@ -35,6 +39,9 @@ type machine struct {
// fd is the vm fd.
fd int
+ // machinePoolIndex is the index in the machinePool array.
+ machinePoolIndex uint32
+
// nextSlot is the next slot for setMemoryRegion.
//
// This must be accessed atomically. If nextSlot is ^uint32(0), then
@@ -192,6 +199,10 @@ func (m *machine) newVCPU() *vCPU {
return c // Done.
}
+// readOnlyGuestRegions contains regions that have to be mapped read-only into
+// the guest physical address space. Right now, it is used on arm64 only.
+var readOnlyGuestRegions []region
+
// newMachine returns a new VM context.
func newMachine(vm int) (*machine, error) {
// Create the machine.
@@ -227,6 +238,10 @@ func newMachine(vm int) (*machine, error) {
m.upperSharedPageTables.MarkReadOnlyShared()
m.kernel.PageTables = pagetables.NewWithUpper(newAllocator(), m.upperSharedPageTables, ring0.KernelStartAddress)
+ // Install seccomp rules to trap runtime mmap system calls. They will
+ // be handled by seccompMmapHandler.
+ seccompMmapRules(m)
+
// Apply the physical mappings. Note that these mappings may point to
// guest physical addresses that are not actually available. These
// physical pages are mapped on demand, see kernel_unsafe.go.
@@ -241,32 +256,11 @@ func newMachine(vm int) (*machine, error) {
return true // Keep iterating.
})
- var physicalRegionsReadOnly []physicalRegion
- var physicalRegionsAvailable []physicalRegion
-
- physicalRegionsReadOnly = rdonlyRegionsForSetMem()
- physicalRegionsAvailable = availableRegionsForSetMem()
-
- // Map all read-only regions.
- for _, r := range physicalRegionsReadOnly {
- m.mapPhysical(r.physical, r.length, physicalRegionsReadOnly, _KVM_MEM_READONLY)
- }
-
// Ensure that the currently mapped virtual regions are actually
// available in the VM. Note that this doesn't guarantee no future
// faults, however it should guarantee that everything is available to
// ensure successful vCPU entry.
- applyVirtualRegions(func(vr virtualRegion) {
- if excludeVirtualRegion(vr) {
- return // skip region.
- }
-
- for _, r := range physicalRegionsReadOnly {
- if vr.virtual == r.virtual {
- return
- }
- }
-
+ mapRegion := func(vr region, flags uint32) {
for virtual := vr.virtual; virtual < vr.virtual+vr.length; {
physical, length, ok := translateToPhysical(virtual)
if !ok {
@@ -280,9 +274,32 @@ func newMachine(vm int) (*machine, error) {
}
// Ensure the physical range is mapped.
- m.mapPhysical(physical, length, physicalRegionsAvailable, _KVM_MEM_FLAGS_NONE)
+ m.mapPhysical(physical, length, physicalRegions, flags)
virtual += length
}
+ }
+
+ for _, vr := range readOnlyGuestRegions {
+ mapRegion(vr, _KVM_MEM_READONLY)
+ }
+
+ applyVirtualRegions(func(vr virtualRegion) {
+ if excludeVirtualRegion(vr) {
+ return // skip region.
+ }
+ for _, r := range readOnlyGuestRegions {
+ if vr.virtual == r.virtual {
+ return
+ }
+ }
+ // Take into account that the stack can grow down.
+ if vr.filename == "[stack]" {
+ vr.virtual -= 1 << 20
+ vr.length += 1 << 20
+ }
+
+ mapRegion(vr.region, 0)
+
})
// Initialize architecture state.
@@ -352,6 +369,10 @@ func (m *machine) mapPhysical(physical, length uintptr, phyRegions []physicalReg
func (m *machine) Destroy() {
runtime.SetFinalizer(m, nil)
+ machinePoolMu.Lock()
+ machinePool[m.machinePoolIndex].Store(nil)
+ machinePoolMu.Unlock()
+
// Destroy vCPUs.
for _, c := range m.vCPUsByID {
if c == nil {
@@ -683,3 +704,72 @@ func (c *vCPU) setSystemTimeLegacy() error {
}
}
}
+
+const machinePoolSize = 16
+
+// machinePool is enumerated from the seccompMmapHandler signal handler
+var (
+ machinePool [machinePoolSize]machineAtomicPtr
+ machinePoolLen uint32
+ machinePoolMu sync.Mutex
+ seccompMmapRulesOnce gosync.Once
+)
+
+func sigsysHandler()
+func addrOfSigsysHandler() uintptr
+
+// seccompMmapRules adds seccomp rules to trap mmap system calls that will be
+// handled in seccompMmapHandler.
+func seccompMmapRules(m *machine) {
+ seccompMmapRulesOnce.Do(func() {
+ // Install the handler.
+ if err := sighandling.ReplaceSignalHandler(unix.SIGSYS, addrOfSigsysHandler(), &savedSigsysHandler); err != nil {
+ panic(fmt.Sprintf("Unable to set handler for signal %d: %v", bluepillSignal, err))
+ }
+ rules := []seccomp.RuleSet{}
+ rules = append(rules, []seccomp.RuleSet{
+ // Trap mmap system calls and handle them in sigsysGoHandler
+ {
+ Rules: seccomp.SyscallRules{
+ unix.SYS_MMAP: {
+ {
+ seccomp.MatchAny{},
+ seccomp.MatchAny{},
+ seccomp.MatchAny{},
+ /* MAP_DENYWRITE is ignored and used only for filtering. */
+ seccomp.MaskedEqual(unix.MAP_DENYWRITE, 0),
+ },
+ },
+ },
+ Action: linux.SECCOMP_RET_TRAP,
+ },
+ }...)
+ instrs, err := seccomp.BuildProgram(rules, linux.SECCOMP_RET_ALLOW, linux.SECCOMP_RET_ALLOW)
+ if err != nil {
+ panic(fmt.Sprintf("failed to build rules: %v", err))
+ }
+ // Perform the actual installation.
+ if err := seccomp.SetFilter(instrs); err != nil {
+ panic(fmt.Sprintf("failed to set filter: %v", err))
+ }
+ })
+
+ machinePoolMu.Lock()
+ n := atomic.LoadUint32(&machinePoolLen)
+ i := uint32(0)
+ for ; i < n; i++ {
+ if machinePool[i].Load() == nil {
+ break
+ }
+ }
+ if i == n {
+ if i == machinePoolSize {
+ machinePoolMu.Unlock()
+ panic("machinePool is full")
+ }
+ atomic.AddUint32(&machinePoolLen, 1)
+ }
+ machinePool[i].Store(m)
+ m.machinePoolIndex = i
+ machinePoolMu.Unlock()
+}
diff --git a/pkg/sentry/platform/kvm/machine_amd64.go b/pkg/sentry/platform/kvm/machine_amd64.go
index a96634381..5bc023899 100644
--- a/pkg/sentry/platform/kvm/machine_amd64.go
+++ b/pkg/sentry/platform/kvm/machine_amd64.go
@@ -29,7 +29,6 @@ import (
"gvisor.dev/gvisor/pkg/hostarch"
"gvisor.dev/gvisor/pkg/ring0"
"gvisor.dev/gvisor/pkg/ring0/pagetables"
- "gvisor.dev/gvisor/pkg/sentry/arch/fpu"
"gvisor.dev/gvisor/pkg/sentry/platform"
ktime "gvisor.dev/gvisor/pkg/sentry/time"
)
@@ -72,10 +71,6 @@ type vCPUArchState struct {
//
// This starts above fixedKernelPCID.
PCIDs *pagetables.PCIDs
-
- // floatingPointState is the floating point state buffer used in guest
- // to host transitions. See usage in bluepill_amd64.go.
- floatingPointState fpu.State
}
const (
@@ -152,12 +147,6 @@ func (c *vCPU) initArchState() error {
return fmt.Errorf("error setting user registers: %v", errno)
}
- // Allocate some floating point state save area for the local vCPU.
- // This will be saved prior to leaving the guest, and we restore from
- // this always. We cannot use the pointer in the context alone because
- // we don't know how large the area there is in reality.
- c.floatingPointState = fpu.NewState()
-
// Set the time offset to the host native time.
return c.setSystemTime()
}
@@ -309,22 +298,6 @@ func loadByte(ptr *byte) byte {
return *ptr
}
-// prefaultFloatingPointState touches each page of the floating point state to
-// be sure that its physical pages are mapped.
-//
-// Otherwise the kernel can trigger KVM_EXIT_MMIO and an instruction that
-// triggered a fault will be emulated by the kvm kernel code, but it can't
-// emulate instructions like xsave and xrstor.
-//
-//go:nosplit
-func prefaultFloatingPointState(data *fpu.State) {
- size := len(*data)
- for i := 0; i < size; i += hostarch.PageSize {
- loadByte(&(*data)[i])
- }
- loadByte(&(*data)[size-1])
-}
-
// SwitchToUser unpacks architectural-details.
func (c *vCPU) SwitchToUser(switchOpts ring0.SwitchOpts, info *linux.SignalInfo) (hostarch.AccessType, error) {
// Check for canonical addresses.
@@ -355,11 +328,6 @@ func (c *vCPU) SwitchToUser(switchOpts ring0.SwitchOpts, info *linux.SignalInfo)
// allocations occur.
entersyscall()
bluepill(c)
- // The root table physical page has to be mapped to not fault in iret
- // or sysret after switching into a user address space. sysret and
- // iret are in the upper half that is global and already mapped.
- switchOpts.PageTables.PrefaultRootTable()
- prefaultFloatingPointState(switchOpts.FloatingPointState)
vector = c.CPU.SwitchToUser(switchOpts)
exitsyscall()
@@ -522,3 +490,7 @@ func (m *machine) getNewVCPU() *vCPU {
}
return nil
}
+
+func archPhysicalRegions(physicalRegions []physicalRegion) []physicalRegion {
+ return physicalRegions
+}
diff --git a/pkg/sentry/platform/kvm/machine_amd64_unsafe.go b/pkg/sentry/platform/kvm/machine_amd64_unsafe.go
index de798bb2c..fbacea9ad 100644
--- a/pkg/sentry/platform/kvm/machine_amd64_unsafe.go
+++ b/pkg/sentry/platform/kvm/machine_amd64_unsafe.go
@@ -161,3 +161,15 @@ func (c *vCPU) getSystemRegisters(sregs *systemRegs) unix.Errno {
}
return 0
}
+
+//go:nosplit
+func seccompMmapSyscall(context unsafe.Pointer) (uintptr, uintptr, unix.Errno) {
+ ctx := bluepillArchContext(context)
+
+ // MAP_DENYWRITE is deprecated and ignored by kernel. We use it only for seccomp filters.
+ addr, _, e := unix.RawSyscall6(uintptr(ctx.Rax), uintptr(ctx.Rdi), uintptr(ctx.Rsi),
+ uintptr(ctx.Rdx), uintptr(ctx.R10)|unix.MAP_DENYWRITE, uintptr(ctx.R8), uintptr(ctx.R9))
+ ctx.Rax = uint64(addr)
+
+ return addr, uintptr(ctx.Rsi), e
+}
diff --git a/pkg/sentry/platform/kvm/machine_arm64.go b/pkg/sentry/platform/kvm/machine_arm64.go
index 7937a8481..31998a600 100644
--- a/pkg/sentry/platform/kvm/machine_arm64.go
+++ b/pkg/sentry/platform/kvm/machine_arm64.go
@@ -26,7 +26,6 @@ import (
"gvisor.dev/gvisor/pkg/hostarch"
"gvisor.dev/gvisor/pkg/ring0"
"gvisor.dev/gvisor/pkg/ring0/pagetables"
- "gvisor.dev/gvisor/pkg/sentry/arch/fpu"
"gvisor.dev/gvisor/pkg/sentry/platform"
)
@@ -40,10 +39,6 @@ type vCPUArchState struct {
//
// This starts above fixedKernelPCID.
PCIDs *pagetables.PCIDs
-
- // floatingPointState is the floating point state buffer used in guest
- // to host transitions. See usage in bluepill_arm64.go.
- floatingPointState fpu.State
}
const (
@@ -110,18 +105,128 @@ func rdonlyRegionsForSetMem() (phyRegions []physicalRegion) {
return phyRegions
}
+// archPhysicalRegions fills readOnlyGuestRegions and allocates separate
+// physical regions form them.
+func archPhysicalRegions(physicalRegions []physicalRegion) []physicalRegion {
+ applyVirtualRegions(func(vr virtualRegion) {
+ if excludeVirtualRegion(vr) {
+ return // skip region.
+ }
+ if !vr.accessType.Write {
+ readOnlyGuestRegions = append(readOnlyGuestRegions, vr.region)
+ }
+ })
+
+ rdRegions := readOnlyGuestRegions[:]
+
+ // Add an unreachable region.
+ rdRegions = append(rdRegions, region{
+ virtual: 0xffffffffffffffff,
+ length: 0,
+ })
+
+ var regions []physicalRegion
+ addValidRegion := func(r *physicalRegion, virtual, length uintptr) {
+ if length == 0 {
+ return
+ }
+ regions = append(regions, physicalRegion{
+ region: region{
+ virtual: virtual,
+ length: length,
+ },
+ physical: r.physical + (virtual - r.virtual),
+ })
+ }
+ i := 0
+ for _, pr := range physicalRegions {
+ start := pr.virtual
+ end := pr.virtual + pr.length
+ for start < end {
+ rdRegion := rdRegions[i]
+ rdStart := rdRegion.virtual
+ rdEnd := rdRegion.virtual + rdRegion.length
+ if rdEnd <= start {
+ i++
+ continue
+ }
+ if rdStart > start {
+ newEnd := rdStart
+ if end < rdStart {
+ newEnd = end
+ }
+ addValidRegion(&pr, start, newEnd-start)
+ start = rdStart
+ continue
+ }
+ if rdEnd < end {
+ addValidRegion(&pr, start, rdEnd-start)
+ start = rdEnd
+ continue
+ }
+ addValidRegion(&pr, start, end-start)
+ start = end
+ }
+ }
+
+ return regions
+}
+
// Get all available physicalRegions.
-func availableRegionsForSetMem() (phyRegions []physicalRegion) {
- var excludeRegions []region
+func availableRegionsForSetMem() []physicalRegion {
+ var excludedRegions []region
applyVirtualRegions(func(vr virtualRegion) {
if !vr.accessType.Write {
- excludeRegions = append(excludeRegions, vr.region)
+ excludedRegions = append(excludedRegions, vr.region)
}
})
- phyRegions = computePhysicalRegions(excludeRegions)
+ // Add an unreachable region.
+ excludedRegions = append(excludedRegions, region{
+ virtual: 0xffffffffffffffff,
+ length: 0,
+ })
- return phyRegions
+ var regions []physicalRegion
+ addValidRegion := func(r *physicalRegion, virtual, length uintptr) {
+ if length == 0 {
+ return
+ }
+ regions = append(regions, physicalRegion{
+ region: region{
+ virtual: virtual,
+ length: length,
+ },
+ physical: r.physical + (virtual - r.virtual),
+ })
+ }
+ i := 0
+ for _, pr := range physicalRegions {
+ start := pr.virtual
+ end := pr.virtual + pr.length
+ for start < end {
+ er := excludedRegions[i]
+ excludeEnd := er.virtual + er.length
+ excludeStart := er.virtual
+ if excludeEnd < start {
+ i++
+ continue
+ }
+ if excludeStart < start {
+ start = excludeEnd
+ i++
+ continue
+ }
+ rend := excludeStart
+ if rend > end {
+ rend = end
+ }
+ addValidRegion(&pr, start, rend-start)
+ start = excludeEnd
+ }
+ }
+
+ return regions
}
// nonCanonical generates a canonical address return.
diff --git a/pkg/sentry/platform/kvm/machine_arm64_unsafe.go b/pkg/sentry/platform/kvm/machine_arm64_unsafe.go
index 1a4a9ce7d..e73d5c544 100644
--- a/pkg/sentry/platform/kvm/machine_arm64_unsafe.go
+++ b/pkg/sentry/platform/kvm/machine_arm64_unsafe.go
@@ -28,7 +28,6 @@ import (
"gvisor.dev/gvisor/pkg/hostarch"
"gvisor.dev/gvisor/pkg/ring0"
"gvisor.dev/gvisor/pkg/ring0/pagetables"
- "gvisor.dev/gvisor/pkg/sentry/arch/fpu"
"gvisor.dev/gvisor/pkg/sentry/platform"
ktime "gvisor.dev/gvisor/pkg/sentry/time"
)
@@ -159,8 +158,6 @@ func (c *vCPU) initArchState() error {
c.PCIDs = pagetables.NewPCIDs(fixedKernelPCID+1, poolPCIDs)
}
- c.floatingPointState = fpu.NewState()
-
return c.setSystemTime()
}
@@ -333,3 +330,15 @@ func (c *vCPU) SwitchToUser(switchOpts ring0.SwitchOpts, info *linux.SignalInfo)
}
}
+
+//go:nosplit
+func seccompMmapSyscall(context unsafe.Pointer) (uintptr, uintptr, unix.Errno) {
+ ctx := bluepillArchContext(context)
+
+ // MAP_DENYWRITE is deprecated and ignored by kernel. We use it only for seccomp filters.
+ addr, _, e := unix.RawSyscall6(uintptr(ctx.Regs[8]), uintptr(ctx.Regs[0]), uintptr(ctx.Regs[1]),
+ uintptr(ctx.Regs[2]), uintptr(ctx.Regs[3])|unix.MAP_DENYWRITE, uintptr(ctx.Regs[4]), uintptr(ctx.Regs[5]))
+ ctx.Regs[0] = uint64(addr)
+
+ return addr, uintptr(ctx.Regs[1]), e
+}
diff --git a/pkg/sentry/platform/kvm/machine_unsafe.go b/pkg/sentry/platform/kvm/machine_unsafe.go
index cc3a1253b..cf3a4e7c9 100644
--- a/pkg/sentry/platform/kvm/machine_unsafe.go
+++ b/pkg/sentry/platform/kvm/machine_unsafe.go
@@ -171,3 +171,46 @@ func (c *vCPU) setSignalMask() error {
return nil
}
+
+// seccompMmapHandler is a signal handler for runtime mmap system calls
+// that are trapped by seccomp.
+//
+// It executes the mmap syscall with specified arguments and maps a new region
+// to the guest.
+//
+//go:nosplit
+func seccompMmapHandler(context unsafe.Pointer) {
+ addr, length, errno := seccompMmapSyscall(context)
+ if errno != 0 {
+ return
+ }
+
+ for i := uint32(0); i < atomic.LoadUint32(&machinePoolLen); i++ {
+ m := machinePool[i].Load()
+ if m == nil {
+ continue
+ }
+
+ // Map the new region to the guest.
+ vr := region{
+ virtual: addr,
+ length: length,
+ }
+ for virtual := vr.virtual; virtual < vr.virtual+vr.length; {
+ physical, length, ok := translateToPhysical(virtual)
+ if !ok {
+ // This must be an invalid region that was
+ // knocked out by creation of the physical map.
+ return
+ }
+ if virtual+length > vr.virtual+vr.length {
+ // Cap the length to the end of the area.
+ length = vr.virtual + vr.length - virtual
+ }
+
+ // Ensure the physical range is mapped.
+ m.mapPhysical(physical, length, physicalRegions, _KVM_MEM_FLAGS_NONE)
+ virtual += length
+ }
+ }
+}
diff --git a/pkg/sentry/platform/kvm/physical_map.go b/pkg/sentry/platform/kvm/physical_map.go
index d812e6c26..9864d1258 100644
--- a/pkg/sentry/platform/kvm/physical_map.go
+++ b/pkg/sentry/platform/kvm/physical_map.go
@@ -168,6 +168,9 @@ func computePhysicalRegions(excludedRegions []region) (physicalRegions []physica
}
addValidRegion(lastExcludedEnd, ring0.MaximumUserAddress-lastExcludedEnd)
+ // Do arch-specific actions on physical regions.
+ physicalRegions = archPhysicalRegions(physicalRegions)
+
// Dump our all physical regions.
for _, r := range physicalRegions {
log.Infof("physicalRegion: virtual [%x,%x) => physical [%x,%x)",
diff --git a/pkg/sentry/platform/kvm/testutil/testutil_arm64.go b/pkg/sentry/platform/kvm/testutil/testutil_arm64.go
index 6d0ba8252..346a10043 100644
--- a/pkg/sentry/platform/kvm/testutil/testutil_arm64.go
+++ b/pkg/sentry/platform/kvm/testutil/testutil_arm64.go
@@ -30,8 +30,8 @@ import (
func TLSWorks() bool
// SetTestTarget sets the rip appropriately.
-func SetTestTarget(regs *arch.Registers, fn func()) {
- regs.Pc = uint64(reflect.ValueOf(fn).Pointer())
+func SetTestTarget(regs *arch.Registers, fn uintptr) {
+ regs.Pc = uint64(fn)
}
// SetTouchTarget sets rax appropriately.
diff --git a/pkg/sentry/platform/kvm/testutil/testutil_arm64.s b/pkg/sentry/platform/kvm/testutil/testutil_arm64.s
index 7348c29a5..42876245a 100644
--- a/pkg/sentry/platform/kvm/testutil/testutil_arm64.s
+++ b/pkg/sentry/platform/kvm/testutil/testutil_arm64.s
@@ -28,6 +28,11 @@ TEXT ·Getpid(SB),NOSPLIT,$0
SVC
RET
+TEXT ·AddrOfGetpid(SB),NOSPLIT,$0-8
+ MOVD $·Getpid(SB), R0
+ MOVD R0, ret+0(FP)
+ RET
+
TEXT ·Touch(SB),NOSPLIT,$0
start:
MOVD 0(R8), R1
@@ -35,21 +40,41 @@ start:
SVC
B start
+TEXT ·AddrOfTouch(SB),NOSPLIT,$0-8
+ MOVD $·Touch(SB), R0
+ MOVD R0, ret+0(FP)
+ RET
+
TEXT ·HaltLoop(SB),NOSPLIT,$0
start:
HLT
B start
+TEXT ·AddOfHaltLoop(SB),NOSPLIT,$0-8
+ MOVD $·HaltLoop(SB), R0
+ MOVD R0, ret+0(FP)
+ RET
+
// This function simulates a loop of syscall.
TEXT ·SyscallLoop(SB),NOSPLIT,$0
start:
SVC
B start
+TEXT ·AddrOfSyscallLoop(SB),NOSPLIT,$0-8
+ MOVD $·SyscallLoop(SB), R0
+ MOVD R0, ret+0(FP)
+ RET
+
TEXT ·SpinLoop(SB),NOSPLIT,$0
start:
B start
+TEXT ·AddrOfSpinLoop(SB),NOSPLIT,$0-8
+ MOVD $·SpinLoop(SB), R0
+ MOVD R0, ret+0(FP)
+ RET
+
TEXT ·TLSWorks(SB),NOSPLIT,$0-8
NO_LOCAL_POINTERS
MOVD $0x6789, R5
@@ -125,6 +150,11 @@ TEXT ·TwiddleRegsSyscall(SB),NOSPLIT,$0
SVC
RET // never reached
+TEXT ·AddrOfTwiddleRegsSyscall(SB),NOSPLIT,$0-8
+ MOVD $·TwiddleRegsSyscall(SB), R0
+ MOVD R0, ret+0(FP)
+ RET
+
TEXT ·TwiddleRegsFault(SB),NOSPLIT,$0
TWIDDLE_REGS()
MSR R10, TPIDR_EL0
@@ -132,3 +162,8 @@ TEXT ·TwiddleRegsFault(SB),NOSPLIT,$0
// Branch to Register branches unconditionally to an address in <Rn>.
JMP (R6) // <=> br x6, must fault
RET // never reached
+
+TEXT ·AddrOfTwiddleRegsFault(SB),NOSPLIT,$0-8
+ MOVD $·TwiddleRegsFault(SB), R0
+ MOVD R0, ret+0(FP)
+ RET
diff --git a/pkg/sentry/seccheck/BUILD b/pkg/sentry/seccheck/BUILD
index 943fa180d..35feb969f 100644
--- a/pkg/sentry/seccheck/BUILD
+++ b/pkg/sentry/seccheck/BUILD
@@ -8,6 +8,8 @@ go_fieldenum(
name = "seccheck_fieldenum",
srcs = [
"clone.go",
+ "execve.go",
+ "exit.go",
"task.go",
],
out = "seccheck_fieldenum.go",
@@ -29,6 +31,8 @@ go_library(
name = "seccheck",
srcs = [
"clone.go",
+ "execve.go",
+ "exit.go",
"seccheck.go",
"seccheck_fieldenum.go",
"seqatomic_checkerslice_unsafe.go",
diff --git a/pkg/sentry/seccheck/execve.go b/pkg/sentry/seccheck/execve.go
new file mode 100644
index 000000000..f36e0730e
--- /dev/null
+++ b/pkg/sentry/seccheck/execve.go
@@ -0,0 +1,65 @@
+// Copyright 2021 The gVisor Authors.
+//
+// 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 seccheck
+
+import (
+ "gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/sentry/kernel/auth"
+)
+
+// ExecveInfo contains information used by the Execve checkpoint.
+//
+// +fieldenum Execve
+type ExecveInfo struct {
+ // Invoker identifies the invoking thread.
+ Invoker TaskInfo
+
+ // Credentials are the invoking thread's credentials.
+ Credentials *auth.Credentials
+
+ // BinaryPath is a path to the executable binary file being switched to in
+ // the mount namespace in which it was opened.
+ BinaryPath string
+
+ // Argv is the new process image's argument vector.
+ Argv []string
+
+ // Env is the new process image's environment variables.
+ Env []string
+
+ // BinaryMode is the executable binary file's mode.
+ BinaryMode uint16
+
+ // BinarySHA256 is the SHA-256 hash of the executable binary file.
+ //
+ // Note that this requires reading the entire file into memory, which is
+ // likely to be extremely slow.
+ BinarySHA256 [32]byte
+}
+
+// ExecveReq returns fields required by the Execve checkpoint.
+func (s *state) ExecveReq() ExecveFieldSet {
+ return s.execveReq.Load()
+}
+
+// Execve is called at the Execve checkpoint.
+func (s *state) Execve(ctx context.Context, mask ExecveFieldSet, info *ExecveInfo) error {
+ for _, c := range s.getCheckers() {
+ if err := c.Execve(ctx, mask, *info); err != nil {
+ return err
+ }
+ }
+ return nil
+}
diff --git a/pkg/sentry/seccheck/exit.go b/pkg/sentry/seccheck/exit.go
new file mode 100644
index 000000000..69cb6911c
--- /dev/null
+++ b/pkg/sentry/seccheck/exit.go
@@ -0,0 +1,57 @@
+// Copyright 2021 The gVisor Authors.
+//
+// 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 seccheck
+
+import (
+ "gvisor.dev/gvisor/pkg/abi/linux"
+ "gvisor.dev/gvisor/pkg/context"
+)
+
+// ExitNotifyParentInfo contains information used by the ExitNotifyParent
+// checkpoint.
+//
+// +fieldenum ExitNotifyParent
+type ExitNotifyParentInfo struct {
+ // Exiter identifies the exiting thread. Note that by the checkpoint's
+ // definition, Exiter.ThreadID == Exiter.ThreadGroupID and
+ // Exiter.ThreadStartTime == Exiter.ThreadGroupStartTime, so requesting
+ // ThreadGroup* fields is redundant.
+ Exiter TaskInfo
+
+ // ExitStatus is the exiting thread group's exit status, as reported
+ // by wait*().
+ ExitStatus linux.WaitStatus
+}
+
+// ExitNotifyParentReq returns fields required by the ExitNotifyParent
+// checkpoint.
+func (s *state) ExitNotifyParentReq() ExitNotifyParentFieldSet {
+ return s.exitNotifyParentReq.Load()
+}
+
+// ExitNotifyParent is called at the ExitNotifyParent checkpoint.
+//
+// The ExitNotifyParent checkpoint occurs when a zombied thread group leader,
+// not waiting for exit acknowledgement from a non-parent ptracer, becomes the
+// last non-dead thread in its thread group and notifies its parent of its
+// exiting.
+func (s *state) ExitNotifyParent(ctx context.Context, mask ExitNotifyParentFieldSet, info *ExitNotifyParentInfo) error {
+ for _, c := range s.getCheckers() {
+ if err := c.ExitNotifyParent(ctx, mask, *info); err != nil {
+ return err
+ }
+ }
+ return nil
+}
diff --git a/pkg/sentry/seccheck/seccheck.go b/pkg/sentry/seccheck/seccheck.go
index b6c9d44ce..e13274096 100644
--- a/pkg/sentry/seccheck/seccheck.go
+++ b/pkg/sentry/seccheck/seccheck.go
@@ -29,6 +29,8 @@ type Point uint
// PointX represents the checkpoint X.
const (
PointClone Point = iota
+ PointExecve
+ PointExitNotifyParent
// Add new Points above this line.
pointLength
@@ -47,6 +49,8 @@ const (
// registered concurrently with invocations of checkpoints).
type Checker interface {
Clone(ctx context.Context, mask CloneFieldSet, info CloneInfo) error
+ Execve(ctx context.Context, mask ExecveFieldSet, info ExecveInfo) error
+ ExitNotifyParent(ctx context.Context, mask ExitNotifyParentFieldSet, info ExitNotifyParentInfo) error
}
// CheckerDefaults may be embedded by implementations of Checker to obtain
@@ -58,6 +62,16 @@ func (CheckerDefaults) Clone(ctx context.Context, mask CloneFieldSet, info Clone
return nil
}
+// Execve implements Checker.Execve.
+func (CheckerDefaults) Execve(ctx context.Context, mask ExecveFieldSet, info ExecveInfo) error {
+ return nil
+}
+
+// ExitNotifyParent implements Checker.ExitNotifyParent.
+func (CheckerDefaults) ExitNotifyParent(ctx context.Context, mask ExitNotifyParentFieldSet, info ExitNotifyParentInfo) error {
+ return nil
+}
+
// CheckerReq indicates what checkpoints a corresponding Checker runs at, and
// what information it requires at those checkpoints.
type CheckerReq struct {
@@ -69,7 +83,9 @@ type CheckerReq struct {
// All of the following fields indicate what fields in the corresponding
// XInfo struct will be requested at the corresponding checkpoint.
- Clone CloneFields
+ Clone CloneFields
+ Execve ExecveFields
+ ExitNotifyParent ExitNotifyParentFields
}
// Global is the method receiver of all seccheck functions.
@@ -101,7 +117,9 @@ type state struct {
// corresponding XInfo struct have been requested by any registered
// checker, are accessed using atomic memory operations, and are mutated
// with registrationMu locked.
- cloneReq CloneFieldSet
+ cloneReq CloneFieldSet
+ execveReq ExecveFieldSet
+ exitNotifyParentReq ExitNotifyParentFieldSet
}
// AppendChecker registers the given Checker to execute at checkpoints. The
@@ -110,7 +128,11 @@ type state struct {
func (s *state) AppendChecker(c Checker, req *CheckerReq) {
s.registrationMu.Lock()
defer s.registrationMu.Unlock()
+
s.cloneReq.AddFieldsLoadable(req.Clone)
+ s.execveReq.AddFieldsLoadable(req.Execve)
+ s.exitNotifyParentReq.AddFieldsLoadable(req.ExitNotifyParent)
+
s.appendCheckerLocked(c)
for _, p := range req.Points {
word, bit := p/32, p%32
diff --git a/pkg/sentry/sighandling/BUILD b/pkg/sentry/sighandling/BUILD
deleted file mode 100644
index 1790d57c9..000000000
--- a/pkg/sentry/sighandling/BUILD
+++ /dev/null
@@ -1,16 +0,0 @@
-load("//tools:defs.bzl", "go_library")
-
-package(licenses = ["notice"])
-
-go_library(
- name = "sighandling",
- srcs = [
- "sighandling.go",
- "sighandling_unsafe.go",
- ],
- visibility = ["//pkg/sentry:internal"],
- deps = [
- "//pkg/abi/linux",
- "@org_golang_x_sys//unix:go_default_library",
- ],
-)
diff --git a/pkg/sentry/sighandling/sighandling.go b/pkg/sentry/sighandling/sighandling.go
deleted file mode 100644
index bdaf8af29..000000000
--- a/pkg/sentry/sighandling/sighandling.go
+++ /dev/null
@@ -1,102 +0,0 @@
-// Copyright 2018 The gVisor Authors.
-//
-// 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 sighandling contains helpers for handling signals to applications.
-package sighandling
-
-import (
- "os"
- "os/signal"
- "reflect"
-
- "golang.org/x/sys/unix"
- "gvisor.dev/gvisor/pkg/abi/linux"
-)
-
-// numSignals is the number of normal (non-realtime) signals on Linux.
-const numSignals = 32
-
-// handleSignals listens for incoming signals and calls the given handler
-// function.
-//
-// It stops when the stop channel is closed. The done channel is closed once it
-// will no longer deliver signals to k.
-func handleSignals(sigchans []chan os.Signal, handler func(linux.Signal), stop, done chan struct{}) {
- // Build a select case.
- sc := []reflect.SelectCase{{Dir: reflect.SelectRecv, Chan: reflect.ValueOf(stop)}}
- for _, sigchan := range sigchans {
- sc = append(sc, reflect.SelectCase{Dir: reflect.SelectRecv, Chan: reflect.ValueOf(sigchan)})
- }
-
- for {
- // Wait for a notification.
- index, _, ok := reflect.Select(sc)
-
- // Was it the stop channel?
- if index == 0 {
- if !ok {
- // Stop forwarding and notify that it's done.
- close(done)
- return
- }
- continue
- }
-
- // How about a different close?
- if !ok {
- panic("signal channel closed unexpectedly")
- }
-
- // Otherwise, it was a signal on channel N. Index 0 represents the stop
- // channel, so index N represents the channel for signal N.
- handler(linux.Signal(index))
- }
-}
-
-// StartSignalForwarding ensures that synchronous signals are passed to the
-// given handler function and returns a callback that stops signal delivery.
-//
-// Note that this function permanently takes over signal handling. After the
-// stop callback, signals revert to the default Go runtime behavior, which
-// cannot be overridden with external calls to signal.Notify.
-func StartSignalForwarding(handler func(linux.Signal)) func() {
- stop := make(chan struct{})
- done := make(chan struct{})
-
- // Register individual channels. One channel per standard signal is
- // required as os.Notify() is non-blocking and may drop signals. To avoid
- // this, standard signals have to be queued separately. Channel size 1 is
- // enough for standard signals as their semantics allow de-duplication.
- //
- // External real-time signals are not supported. We rely on the go-runtime
- // for their handling.
- var sigchans []chan os.Signal
- for sig := 1; sig <= numSignals+1; sig++ {
- sigchan := make(chan os.Signal, 1)
- sigchans = append(sigchans, sigchan)
-
- // SIGURG is used by Go's runtime scheduler.
- if sig == int(linux.SIGURG) {
- continue
- }
- signal.Notify(sigchan, unix.Signal(sig))
- }
- // Start up our listener.
- go handleSignals(sigchans, handler, stop, done) // S/R-SAFE: synchronized by Kernel.extMu.
-
- return func() {
- close(stop)
- <-done
- }
-}
diff --git a/pkg/sentry/sighandling/sighandling_unsafe.go b/pkg/sentry/sighandling/sighandling_unsafe.go
deleted file mode 100644
index 3fe5c6770..000000000
--- a/pkg/sentry/sighandling/sighandling_unsafe.go
+++ /dev/null
@@ -1,39 +0,0 @@
-// Copyright 2018 The gVisor Authors.
-//
-// 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 sighandling
-
-import (
- "unsafe"
-
- "golang.org/x/sys/unix"
- "gvisor.dev/gvisor/pkg/abi/linux"
-)
-
-// IgnoreChildStop sets the SA_NOCLDSTOP flag, causing child processes to not
-// generate SIGCHLD when they stop.
-func IgnoreChildStop() error {
- var sa linux.SigAction
-
- // Get the existing signal handler information, and set the flag.
- if _, _, e := unix.RawSyscall6(unix.SYS_RT_SIGACTION, uintptr(unix.SIGCHLD), 0, uintptr(unsafe.Pointer(&sa)), linux.SignalSetSize, 0, 0); e != 0 {
- return e
- }
- sa.Flags |= linux.SA_NOCLDSTOP
- if _, _, e := unix.RawSyscall6(unix.SYS_RT_SIGACTION, uintptr(unix.SIGCHLD), uintptr(unsafe.Pointer(&sa)), 0, linux.SignalSetSize, 0, 0); e != 0 {
- return e
- }
-
- return nil
-}
diff --git a/pkg/sentry/socket/BUILD b/pkg/sentry/socket/BUILD
index 7ee89a735..00f925166 100644
--- a/pkg/sentry/socket/BUILD
+++ b/pkg/sentry/socket/BUILD
@@ -4,7 +4,10 @@ package(licenses = ["notice"])
go_library(
name = "socket",
- srcs = ["socket.go"],
+ srcs = [
+ "socket.go",
+ "socket_state.go",
+ ],
visibility = ["//pkg/sentry:internal"],
deps = [
"//pkg/abi/linux",
diff --git a/pkg/sentry/socket/control/control.go b/pkg/sentry/socket/control/control.go
index 00a5e729a..6077b2150 100644
--- a/pkg/sentry/socket/control/control.go
+++ b/pkg/sentry/socket/control/control.go
@@ -29,10 +29,9 @@ import (
"gvisor.dev/gvisor/pkg/sentry/kernel/auth"
"gvisor.dev/gvisor/pkg/sentry/socket"
"gvisor.dev/gvisor/pkg/sentry/socket/unix/transport"
+ "time"
)
-const maxInt = int(^uint(0) >> 1)
-
// SCMCredentials represents a SCM_CREDENTIALS socket control message.
type SCMCredentials interface {
transport.CredentialsControlMessage
@@ -78,7 +77,7 @@ func NewSCMRights(t *kernel.Task, fds []int32) (SCMRights, error) {
}
// Files implements SCMRights.Files.
-func (fs *RightsFiles) Files(ctx context.Context, max int) (RightsFiles, bool) {
+func (fs *RightsFiles) Files(_ context.Context, max int) (RightsFiles, bool) {
n := max
var trunc bool
if l := len(*fs); n > l {
@@ -124,7 +123,7 @@ func rightsFDs(t *kernel.Task, rights SCMRights, cloexec bool, max int) ([]int32
break
}
- fds = append(fds, int32(fd))
+ fds = append(fds, fd)
}
return fds, trunc
}
@@ -300,8 +299,8 @@ func alignSlice(buf []byte, align uint) []byte {
}
// PackTimestamp packs a SO_TIMESTAMP socket control message.
-func PackTimestamp(t *kernel.Task, timestamp int64, buf []byte) []byte {
- timestampP := linux.NsecToTimeval(timestamp)
+func PackTimestamp(t *kernel.Task, timestamp time.Time, buf []byte) []byte {
+ timestampP := linux.NsecToTimeval(timestamp.UnixNano())
return putCmsgStruct(
buf,
linux.SOL_SOCKET,
@@ -355,6 +354,17 @@ func PackIPPacketInfo(t *kernel.Task, packetInfo *linux.ControlMessageIPPacketIn
)
}
+// PackIPv6PacketInfo packs an IPV6_PKTINFO socket control message.
+func PackIPv6PacketInfo(t *kernel.Task, packetInfo *linux.ControlMessageIPv6PacketInfo, buf []byte) []byte {
+ return putCmsgStruct(
+ buf,
+ linux.SOL_IPV6,
+ linux.IPV6_PKTINFO,
+ t.Arch().Width(),
+ packetInfo,
+ )
+}
+
// PackOriginalDstAddress packs an IP_RECVORIGINALDSTADDR socket control message.
func PackOriginalDstAddress(t *kernel.Task, originalDstAddress linux.SockAddr, buf []byte) []byte {
var level uint32
@@ -412,6 +422,10 @@ func PackControlMessages(t *kernel.Task, cmsgs socket.ControlMessages, buf []byt
buf = PackIPPacketInfo(t, &cmsgs.IP.PacketInfo, buf)
}
+ if cmsgs.IP.HasIPv6PacketInfo {
+ buf = PackIPv6PacketInfo(t, &cmsgs.IP.IPv6PacketInfo, buf)
+ }
+
if cmsgs.IP.OriginalDstAddress != nil {
buf = PackOriginalDstAddress(t, cmsgs.IP.OriginalDstAddress, buf)
}
@@ -453,6 +467,10 @@ func CmsgsSpace(t *kernel.Task, cmsgs socket.ControlMessages) int {
space += cmsgSpace(t, linux.SizeOfControlMessageIPPacketInfo)
}
+ if cmsgs.IP.HasIPv6PacketInfo {
+ space += cmsgSpace(t, linux.SizeOfControlMessageIPv6PacketInfo)
+ }
+
if cmsgs.IP.OriginalDstAddress != nil {
space += cmsgSpace(t, cmsgs.IP.OriginalDstAddress.SizeBytes())
}
@@ -526,7 +544,7 @@ func Parse(t *kernel.Task, socketOrEndpoint interface{}, buf []byte, width uint)
}
var ts linux.Timeval
ts.UnmarshalUnsafe(buf[i : i+linux.SizeOfTimeval])
- cmsgs.IP.Timestamp = ts.ToNsecCapped()
+ cmsgs.IP.Timestamp = ts.ToTime()
cmsgs.IP.HasTimestamp = true
i += bits.AlignUp(length, width)
diff --git a/pkg/sentry/socket/control/control_test.go b/pkg/sentry/socket/control/control_test.go
index 7e28a0cef..1b04e1bbc 100644
--- a/pkg/sentry/socket/control/control_test.go
+++ b/pkg/sentry/socket/control/control_test.go
@@ -50,7 +50,7 @@ func TestParse(t *testing.T) {
want := socket.ControlMessages{
IP: socket.IPControlMessages{
HasTimestamp: true,
- Timestamp: ts.ToNsecCapped(),
+ Timestamp: ts.ToTime(),
},
}
if diff := cmp.Diff(want, cmsg); diff != "" {
diff --git a/pkg/sentry/socket/hostinet/socket.go b/pkg/sentry/socket/hostinet/socket.go
index 1c1e501ba..6e2318f75 100644
--- a/pkg/sentry/socket/hostinet/socket.go
+++ b/pkg/sentry/socket/hostinet/socket.go
@@ -111,7 +111,7 @@ func (s *socketOperations) Read(ctx context.Context, _ *fs.File, dst usermem.IOS
}
return readv(s.fd, safemem.IovecsFromBlockSeq(dsts))
}))
- return int64(n), err
+ return n, err
}
// Write implements fs.FileOperations.Write.
@@ -134,7 +134,7 @@ func (s *socketOperations) Write(ctx context.Context, _ *fs.File, src usermem.IO
}
return writev(s.fd, safemem.IovecsFromBlockSeq(srcs))
}))
- return int64(n), err
+ return n, err
}
// Socket implements socket.Provider.Socket.
@@ -180,7 +180,7 @@ func (p *socketProvider) Socket(t *kernel.Task, stypeflags linux.SockType, proto
}
// Pair implements socket.Provider.Pair.
-func (p *socketProvider) Pair(t *kernel.Task, stype linux.SockType, protocol int) (*fs.File, *fs.File, *syserr.Error) {
+func (p *socketProvider) Pair(*kernel.Task, linux.SockType, int) (*fs.File, *fs.File, *syserr.Error) {
// Not supported by AF_INET/AF_INET6.
return nil, nil, nil
}
@@ -207,7 +207,7 @@ type socketOpsCommon struct {
// Release implements fs.FileOperations.Release.
func (s *socketOpsCommon) Release(context.Context) {
fdnotifier.RemoveFD(int32(s.fd))
- unix.Close(s.fd)
+ _ = unix.Close(s.fd)
}
// Readiness implements waiter.Waitable.Readiness.
@@ -218,13 +218,13 @@ func (s *socketOpsCommon) Readiness(mask waiter.EventMask) waiter.EventMask {
// EventRegister implements waiter.Waitable.EventRegister.
func (s *socketOpsCommon) EventRegister(e *waiter.Entry, mask waiter.EventMask) {
s.queue.EventRegister(e, mask)
- fdnotifier.UpdateFD(int32(s.fd))
+ _ = fdnotifier.UpdateFD(int32(s.fd))
}
// EventUnregister implements waiter.Waitable.EventUnregister.
func (s *socketOpsCommon) EventUnregister(e *waiter.Entry) {
s.queue.EventUnregister(e)
- fdnotifier.UpdateFD(int32(s.fd))
+ _ = fdnotifier.UpdateFD(int32(s.fd))
}
// Connect implements socket.Socket.Connect.
@@ -316,7 +316,7 @@ func (s *socketOpsCommon) Accept(t *kernel.Task, peerRequested bool, flags int,
if kernel.VFS2Enabled {
f, err := newVFS2Socket(t, s.family, s.stype, s.protocol, fd, uint32(flags&unix.SOCK_NONBLOCK))
if err != nil {
- unix.Close(fd)
+ _ = unix.Close(fd)
return 0, nil, 0, err
}
defer f.DecRef(t)
@@ -328,7 +328,7 @@ func (s *socketOpsCommon) Accept(t *kernel.Task, peerRequested bool, flags int,
} else {
f, err := newSocketFile(t, s.family, s.stype, s.protocol, fd, flags&unix.SOCK_NONBLOCK != 0)
if err != nil {
- unix.Close(fd)
+ _ = unix.Close(fd)
return 0, nil, 0, err
}
defer f.DecRef(t)
@@ -343,7 +343,7 @@ func (s *socketOpsCommon) Accept(t *kernel.Task, peerRequested bool, flags int,
}
// Bind implements socket.Socket.Bind.
-func (s *socketOpsCommon) Bind(t *kernel.Task, sockaddr []byte) *syserr.Error {
+func (s *socketOpsCommon) Bind(_ *kernel.Task, sockaddr []byte) *syserr.Error {
if len(sockaddr) > sizeofSockaddr {
sockaddr = sockaddr[:sizeofSockaddr]
}
@@ -356,12 +356,12 @@ func (s *socketOpsCommon) Bind(t *kernel.Task, sockaddr []byte) *syserr.Error {
}
// Listen implements socket.Socket.Listen.
-func (s *socketOpsCommon) Listen(t *kernel.Task, backlog int) *syserr.Error {
+func (s *socketOpsCommon) Listen(_ *kernel.Task, backlog int) *syserr.Error {
return syserr.FromError(unix.Listen(s.fd, backlog))
}
// Shutdown implements socket.Socket.Shutdown.
-func (s *socketOpsCommon) Shutdown(t *kernel.Task, how int) *syserr.Error {
+func (s *socketOpsCommon) Shutdown(_ *kernel.Task, how int) *syserr.Error {
switch how {
case unix.SHUT_RD, unix.SHUT_WR, unix.SHUT_RDWR:
return syserr.FromError(unix.Shutdown(s.fd, how))
@@ -371,7 +371,7 @@ func (s *socketOpsCommon) Shutdown(t *kernel.Task, how int) *syserr.Error {
}
// GetSockOpt implements socket.Socket.GetSockOpt.
-func (s *socketOpsCommon) GetSockOpt(t *kernel.Task, level int, name int, outPtr hostarch.Addr, outLen int) (marshal.Marshallable, *syserr.Error) {
+func (s *socketOpsCommon) GetSockOpt(t *kernel.Task, level int, name int, _ hostarch.Addr, outLen int) (marshal.Marshallable, *syserr.Error) {
if outLen < 0 {
return nil, syserr.ErrInvalidArgument
}
@@ -401,7 +401,7 @@ func (s *socketOpsCommon) GetSockOpt(t *kernel.Task, level int, name int, outPtr
case linux.TCP_NODELAY:
optlen = sizeofInt32
case linux.TCP_INFO:
- optlen = int(linux.SizeOfTCPInfo)
+ optlen = linux.SizeOfTCPInfo
}
}
@@ -579,7 +579,7 @@ func parseUnixControlMessages(unixControlMessages []unix.SocketControlMessage) s
controlMessages.IP.HasTimestamp = true
ts := linux.Timeval{}
ts.UnmarshalUnsafe(unixCmsg.Data[:linux.SizeOfTimeval])
- controlMessages.IP.Timestamp = ts.ToNsecCapped()
+ controlMessages.IP.Timestamp = ts.ToTime()
}
case linux.SOL_IP:
diff --git a/pkg/sentry/socket/netfilter/netfilter.go b/pkg/sentry/socket/netfilter/netfilter.go
index e3eade180..8d9e73243 100644
--- a/pkg/sentry/socket/netfilter/netfilter.go
+++ b/pkg/sentry/socket/netfilter/netfilter.go
@@ -58,8 +58,8 @@ var nameToID = map[string]stack.TableID{
// DefaultLinuxTables returns the rules of stack.DefaultTables() wrapped for
// compatibility with netfilter extensions.
-func DefaultLinuxTables(seed uint32) *stack.IPTables {
- tables := stack.DefaultTables(seed)
+func DefaultLinuxTables(seed uint32, clock tcpip.Clock) *stack.IPTables {
+ tables := stack.DefaultTables(seed, clock)
tables.VisitTargets(func(oldTarget stack.Target) stack.Target {
switch val := oldTarget.(type) {
case *stack.AcceptTarget:
diff --git a/pkg/sentry/socket/netfilter/targets.go b/pkg/sentry/socket/netfilter/targets.go
index ea56f39c1..b9c15daab 100644
--- a/pkg/sentry/socket/netfilter/targets.go
+++ b/pkg/sentry/socket/netfilter/targets.go
@@ -647,7 +647,7 @@ func (jt *JumpTarget) id() targetID {
}
// Action implements stack.Target.Action.
-func (jt *JumpTarget) Action(*stack.PacketBuffer, *stack.ConnTrack, stack.Hook, *stack.Route, tcpip.Address) (stack.RuleVerdict, int) {
+func (jt *JumpTarget) Action(*stack.PacketBuffer, stack.Hook, *stack.Route, stack.AddressableEndpoint) (stack.RuleVerdict, int) {
return stack.RuleJump, jt.RuleNum
}
diff --git a/pkg/sentry/socket/netstack/BUILD b/pkg/sentry/socket/netstack/BUILD
index bf5ec4558..075f61cda 100644
--- a/pkg/sentry/socket/netstack/BUILD
+++ b/pkg/sentry/socket/netstack/BUILD
@@ -7,6 +7,7 @@ go_library(
srcs = [
"device.go",
"netstack.go",
+ "netstack_state.go",
"netstack_vfs2.go",
"provider.go",
"provider_vfs2.go",
diff --git a/pkg/sentry/socket/netstack/netstack.go b/pkg/sentry/socket/netstack/netstack.go
index f79bda922..030c6c8e4 100644
--- a/pkg/sentry/socket/netstack/netstack.go
+++ b/pkg/sentry/socket/netstack/netstack.go
@@ -274,6 +274,7 @@ var Metrics = tcpip.Stats{
ChecksumErrors: mustCreateMetric("/netstack/tcp/checksum_errors", "Number of segments dropped due to bad checksums."),
FailedPortReservations: mustCreateMetric("/netstack/tcp/failed_port_reservations", "Number of time TCP failed to reserve a port."),
SegmentsAckedWithDSACK: mustCreateMetric("/netstack/tcp/segments_acked_with_dsack", "Number of segments for which DSACK was received."),
+ SpuriousRecovery: mustCreateMetric("/netstack/tcp/spurious_recovery", "Number of times the connection entered loss recovery spuriously."),
},
UDP: tcpip.UDPStats{
PacketsReceived: mustCreateMetric("/netstack/udp/packets_received", "Number of UDP datagrams received via HandlePacket."),
@@ -378,9 +379,9 @@ type socketOpsCommon struct {
// timestampValid indicates whether timestamp for SIOCGSTAMP has been
// set. It is protected by readMu.
timestampValid bool
- // timestampNS holds the timestamp to use with SIOCTSTAMP. It is only
+ // timestamp holds the timestamp to use with SIOCTSTAMP. It is only
// valid when timestampValid is true. It is protected by readMu.
- timestampNS int64
+ timestamp time.Time `state:".(int64)"`
// TODO(b/153685824): Move this to SocketOptions.
// sockOptInq corresponds to TCP_INQ.
@@ -410,15 +411,6 @@ var sockAddrInetSize = (*linux.SockAddrInet)(nil).SizeBytes()
var sockAddrInet6Size = (*linux.SockAddrInet6)(nil).SizeBytes()
var sockAddrLinkSize = (*linux.SockAddrLink)(nil).SizeBytes()
-// bytesToIPAddress converts an IPv4 or IPv6 address from the user to the
-// netstack representation taking any addresses into account.
-func bytesToIPAddress(addr []byte) tcpip.Address {
- if bytes.Equal(addr, make([]byte, 4)) || bytes.Equal(addr, make([]byte, 16)) {
- return ""
- }
- return tcpip.Address(addr)
-}
-
// minSockAddrLen returns the minimum length in bytes of a socket address for
// the socket's family.
func (s *socketOpsCommon) minSockAddrLen() int {
@@ -468,7 +460,7 @@ func (s *socketOpsCommon) Release(ctx context.Context) {
t := kernel.TaskFromContext(ctx)
start := t.Kernel().MonotonicClock().Now()
deadline := start.Add(v.Timeout)
- t.BlockWithDeadline(ch, true, deadline)
+ _ = t.BlockWithDeadline(ch, true, deadline)
}
}
@@ -488,7 +480,7 @@ func (s *SocketOperations) Read(ctx context.Context, _ *fs.File, dst usermem.IOS
}
// WriteTo implements fs.FileOperations.WriteTo.
-func (s *SocketOperations) WriteTo(ctx context.Context, _ *fs.File, dst io.Writer, count int64, dup bool) (int64, error) {
+func (s *SocketOperations) WriteTo(_ context.Context, _ *fs.File, dst io.Writer, count int64, dup bool) (int64, error) {
s.readMu.Lock()
defer s.readMu.Unlock()
@@ -543,7 +535,7 @@ func (l *limitedPayloader) Len() int {
}
// ReadFrom implements fs.FileOperations.ReadFrom.
-func (s *SocketOperations) ReadFrom(ctx context.Context, _ *fs.File, r io.Reader, count int64) (int64, error) {
+func (s *SocketOperations) ReadFrom(_ context.Context, _ *fs.File, r io.Reader, count int64) (int64, error) {
f := limitedPayloader{
inner: io.LimitedReader{
R: r,
@@ -654,7 +646,7 @@ func (s *socketOpsCommon) Connect(t *kernel.Task, sockaddr []byte, blocking bool
// Bind implements the linux syscall bind(2) for sockets backed by
// tcpip.Endpoint.
-func (s *socketOpsCommon) Bind(t *kernel.Task, sockaddr []byte) *syserr.Error {
+func (s *socketOpsCommon) Bind(_ *kernel.Task, sockaddr []byte) *syserr.Error {
if len(sockaddr) < 2 {
return syserr.ErrInvalidArgument
}
@@ -672,13 +664,10 @@ func (s *socketOpsCommon) Bind(t *kernel.Task, sockaddr []byte) *syserr.Error {
}
a.UnmarshalBytes(sockaddr[:sockAddrLinkSize])
- if a.Protocol != uint16(s.protocol) {
- return syserr.ErrInvalidArgument
- }
-
addr = tcpip.FullAddress{
NIC: tcpip.NICID(a.InterfaceIndex),
Addr: tcpip.Address(a.HardwareAddr[:header.EthernetAddressSize]),
+ Port: socket.Ntohs(a.Protocol),
}
} else {
if s.minSockAddrLen() > len(sockaddr) {
@@ -717,7 +706,7 @@ func (s *socketOpsCommon) Bind(t *kernel.Task, sockaddr []byte) *syserr.Error {
// Listen implements the linux syscall listen(2) for sockets backed by
// tcpip.Endpoint.
-func (s *socketOpsCommon) Listen(t *kernel.Task, backlog int) *syserr.Error {
+func (s *socketOpsCommon) Listen(_ *kernel.Task, backlog int) *syserr.Error {
return syserr.TranslateNetstackError(s.Endpoint.Listen(backlog))
}
@@ -808,7 +797,7 @@ func ConvertShutdown(how int) (tcpip.ShutdownFlags, *syserr.Error) {
// Shutdown implements the linux syscall shutdown(2) for sockets backed by
// tcpip.Endpoint.
-func (s *socketOpsCommon) Shutdown(t *kernel.Task, how int) *syserr.Error {
+func (s *socketOpsCommon) Shutdown(_ *kernel.Task, how int) *syserr.Error {
f, err := ConvertShutdown(how)
if err != nil {
return err
@@ -889,7 +878,7 @@ func boolToInt32(v bool) int32 {
}
// getSockOptSocket implements GetSockOpt when level is SOL_SOCKET.
-func getSockOptSocket(t *kernel.Task, s socket.SocketOps, ep commonEndpoint, family int, skType linux.SockType, name, outLen int) (marshal.Marshallable, *syserr.Error) {
+func getSockOptSocket(t *kernel.Task, s socket.SocketOps, ep commonEndpoint, family int, _ linux.SockType, name, outLen int) (marshal.Marshallable, *syserr.Error) {
// TODO(b/124056281): Stop rejecting short optLen values in getsockopt.
switch name {
case linux.SO_ERROR:
@@ -1374,6 +1363,14 @@ func getSockOptIPv6(t *kernel.Task, s socket.SocketOps, ep commonEndpoint, name
v := primitive.Int32(boolToInt32(ep.SocketOptions().GetReceiveOriginalDstAddress()))
return &v, nil
+ case linux.IPV6_RECVPKTINFO:
+ if outLen < sizeOfInt32 {
+ return nil, syserr.ErrInvalidArgument
+ }
+
+ v := primitive.Int32(boolToInt32(ep.SocketOptions().GetIPv6ReceivePacketInfo()))
+ return &v, nil
+
case linux.IP6T_ORIGINAL_DST:
if outLen < sockAddrInet6Size {
return nil, syserr.ErrInvalidArgument
@@ -1397,11 +1394,11 @@ func getSockOptIPv6(t *kernel.Task, s socket.SocketOps, ep commonEndpoint, name
return nil, syserr.ErrProtocolNotAvailable
}
- stack := inet.StackFromContext(t)
- if stack == nil {
+ stk := inet.StackFromContext(t)
+ if stk == nil {
return nil, syserr.ErrNoDevice
}
- info, err := netfilter.GetInfo(t, stack.(*Stack).Stack, outPtr, true)
+ info, err := netfilter.GetInfo(t, stk.(*Stack).Stack, outPtr, true)
if err != nil {
return nil, err
}
@@ -1417,11 +1414,11 @@ func getSockOptIPv6(t *kernel.Task, s socket.SocketOps, ep commonEndpoint, name
return nil, syserr.ErrProtocolNotAvailable
}
- stack := inet.StackFromContext(t)
- if stack == nil {
+ stk := inet.StackFromContext(t)
+ if stk == nil {
return nil, syserr.ErrNoDevice
}
- entries, err := netfilter.GetEntries6(t, stack.(*Stack).Stack, outPtr, outLen)
+ entries, err := netfilter.GetEntries6(t, stk.(*Stack).Stack, outPtr, outLen)
if err != nil {
return nil, err
}
@@ -1437,8 +1434,8 @@ func getSockOptIPv6(t *kernel.Task, s socket.SocketOps, ep commonEndpoint, name
return nil, syserr.ErrProtocolNotAvailable
}
- stack := inet.StackFromContext(t)
- if stack == nil {
+ stk := inet.StackFromContext(t)
+ if stk == nil {
return nil, syserr.ErrNoDevice
}
ret, err := netfilter.TargetRevision(t, outPtr, header.IPv6ProtocolNumber)
@@ -1454,7 +1451,7 @@ func getSockOptIPv6(t *kernel.Task, s socket.SocketOps, ep commonEndpoint, name
}
// getSockOptIP implements GetSockOpt when level is SOL_IP.
-func getSockOptIP(t *kernel.Task, s socket.SocketOps, ep commonEndpoint, name int, outPtr hostarch.Addr, outLen int, family int) (marshal.Marshallable, *syserr.Error) {
+func getSockOptIP(t *kernel.Task, s socket.SocketOps, ep commonEndpoint, name int, outPtr hostarch.Addr, outLen int, _ int) (marshal.Marshallable, *syserr.Error) {
if _, ok := ep.(tcpip.Endpoint); !ok {
log.Warningf("SOL_IP options not supported on endpoints other than tcpip.Endpoint: option = %d", name)
return nil, syserr.ErrUnknownProtocolOption
@@ -1594,11 +1591,11 @@ func getSockOptIP(t *kernel.Task, s socket.SocketOps, ep commonEndpoint, name in
return nil, syserr.ErrProtocolNotAvailable
}
- stack := inet.StackFromContext(t)
- if stack == nil {
+ stk := inet.StackFromContext(t)
+ if stk == nil {
return nil, syserr.ErrNoDevice
}
- info, err := netfilter.GetInfo(t, stack.(*Stack).Stack, outPtr, false)
+ info, err := netfilter.GetInfo(t, stk.(*Stack).Stack, outPtr, false)
if err != nil {
return nil, err
}
@@ -1614,11 +1611,11 @@ func getSockOptIP(t *kernel.Task, s socket.SocketOps, ep commonEndpoint, name in
return nil, syserr.ErrProtocolNotAvailable
}
- stack := inet.StackFromContext(t)
- if stack == nil {
+ stk := inet.StackFromContext(t)
+ if stk == nil {
return nil, syserr.ErrNoDevice
}
- entries, err := netfilter.GetEntries4(t, stack.(*Stack).Stack, outPtr, outLen)
+ entries, err := netfilter.GetEntries4(t, stk.(*Stack).Stack, outPtr, outLen)
if err != nil {
return nil, err
}
@@ -1634,8 +1631,8 @@ func getSockOptIP(t *kernel.Task, s socket.SocketOps, ep commonEndpoint, name in
return nil, syserr.ErrProtocolNotAvailable
}
- stack := inet.StackFromContext(t)
- if stack == nil {
+ stk := inet.StackFromContext(t)
+ if stk == nil {
return nil, syserr.ErrNoDevice
}
ret, err := netfilter.TargetRevision(t, outPtr, header.IPv4ProtocolNumber)
@@ -2130,6 +2127,15 @@ func setSockOptIPv6(t *kernel.Task, s socket.SocketOps, ep commonEndpoint, name
ep.SocketOptions().SetReceiveOriginalDstAddress(v != 0)
return nil
+ case linux.IPV6_RECVPKTINFO:
+ if len(optVal) < sizeOfInt32 {
+ return syserr.ErrInvalidArgument
+ }
+ v := int32(hostarch.ByteOrder.Uint32(optVal))
+
+ ep.SocketOptions().SetIPv6ReceivePacketInfo(v != 0)
+ return nil
+
case linux.IPV6_TCLASS:
if len(optVal) < sizeOfInt32 {
return syserr.ErrInvalidArgument
@@ -2172,12 +2178,12 @@ func setSockOptIPv6(t *kernel.Task, s socket.SocketOps, ep commonEndpoint, name
return syserr.ErrProtocolNotAvailable
}
- stack := inet.StackFromContext(t)
- if stack == nil {
+ stk := inet.StackFromContext(t)
+ if stk == nil {
return syserr.ErrNoDevice
}
// Stack must be a netstack stack.
- return netfilter.SetEntries(t, stack.(*Stack).Stack, optVal, true)
+ return netfilter.SetEntries(t, stk.(*Stack).Stack, optVal, true)
case linux.IP6T_SO_SET_ADD_COUNTERS:
log.Infof("IP6T_SO_SET_ADD_COUNTERS is not supported")
@@ -2415,12 +2421,12 @@ func setSockOptIP(t *kernel.Task, s socket.SocketOps, ep commonEndpoint, name in
return syserr.ErrProtocolNotAvailable
}
- stack := inet.StackFromContext(t)
- if stack == nil {
+ stk := inet.StackFromContext(t)
+ if stk == nil {
return syserr.ErrNoDevice
}
// Stack must be a netstack stack.
- return netfilter.SetEntries(t, stack.(*Stack).Stack, optVal, false)
+ return netfilter.SetEntries(t, stk.(*Stack).Stack, optVal, false)
case linux.IPT_SO_SET_ADD_COUNTERS:
log.Infof("IPT_SO_SET_ADD_COUNTERS is not supported")
@@ -2519,7 +2525,6 @@ func emitUnimplementedEventIPv6(t *kernel.Task, name int) {
linux.IPV6_RECVHOPLIMIT,
linux.IPV6_RECVHOPOPTS,
linux.IPV6_RECVPATHMTU,
- linux.IPV6_RECVPKTINFO,
linux.IPV6_RECVRTHDR,
linux.IPV6_RTHDR,
linux.IPV6_RTHDRDSTOPTS,
@@ -2588,7 +2593,7 @@ func emitUnimplementedEventIP(t *kernel.Task, name int) {
// GetSockName implements the linux syscall getsockname(2) for sockets backed by
// tcpip.Endpoint.
-func (s *socketOpsCommon) GetSockName(t *kernel.Task) (linux.SockAddr, uint32, *syserr.Error) {
+func (s *socketOpsCommon) GetSockName(*kernel.Task) (linux.SockAddr, uint32, *syserr.Error) {
addr, err := s.Endpoint.GetLocalAddress()
if err != nil {
return nil, 0, syserr.TranslateNetstackError(err)
@@ -2600,7 +2605,7 @@ func (s *socketOpsCommon) GetSockName(t *kernel.Task) (linux.SockAddr, uint32, *
// GetPeerName implements the linux syscall getpeername(2) for sockets backed by
// tcpip.Endpoint.
-func (s *socketOpsCommon) GetPeerName(t *kernel.Task) (linux.SockAddr, uint32, *syserr.Error) {
+func (s *socketOpsCommon) GetPeerName(*kernel.Task) (linux.SockAddr, uint32, *syserr.Error) {
addr, err := s.Endpoint.GetRemoteAddress()
if err != nil {
return nil, 0, syserr.TranslateNetstackError(err)
@@ -2745,6 +2750,8 @@ func (s *socketOpsCommon) controlMessages(cm tcpip.ControlMessages) socket.Contr
TClass: readCM.TClass,
HasIPPacketInfo: readCM.HasIPPacketInfo,
PacketInfo: readCM.PacketInfo,
+ HasIPv6PacketInfo: readCM.HasIPv6PacketInfo,
+ IPv6PacketInfo: readCM.IPv6PacketInfo,
OriginalDstAddress: readCM.OriginalDstAddress,
SockErr: readCM.SockErr,
},
@@ -2759,7 +2766,7 @@ func (s *socketOpsCommon) updateTimestamp(cm tcpip.ControlMessages) {
// Save the SIOCGSTAMP timestamp only if SO_TIMESTAMP is disabled.
if !s.sockOptTimestamp {
s.timestampValid = true
- s.timestampNS = cm.Timestamp
+ s.timestamp = cm.Timestamp
}
}
@@ -2818,7 +2825,7 @@ func (s *socketOpsCommon) recvErr(t *kernel.Task, dst usermem.IOSequence) (int,
// RecvMsg implements the linux syscall recvmsg(2) for sockets backed by
// tcpip.Endpoint.
-func (s *socketOpsCommon) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags int, haveDeadline bool, deadline ktime.Time, senderRequested bool, controlDataLen uint64) (n int, msgFlags int, senderAddr linux.SockAddr, senderAddrLen uint32, controlMessages socket.ControlMessages, err *syserr.Error) {
+func (s *socketOpsCommon) RecvMsg(t *kernel.Task, dst usermem.IOSequence, flags int, haveDeadline bool, deadline ktime.Time, senderRequested bool, _ uint64) (n int, msgFlags int, senderAddr linux.SockAddr, senderAddrLen uint32, controlMessages socket.ControlMessages, err *syserr.Error) {
if flags&linux.MSG_ERRQUEUE != 0 {
return s.recvErr(t, dst)
}
@@ -2983,7 +2990,7 @@ func (s *socketOpsCommon) ioctl(ctx context.Context, io usermem.IO, args arch.Sy
return 0, linuxerr.ENOENT
}
- tv := linux.NsecToTimeval(s.timestampNS)
+ tv := linux.NsecToTimeval(s.timestamp.UnixNano())
_, err := tv.CopyOut(t, args[2].Pointer())
return 0, err
@@ -3090,7 +3097,7 @@ func Ioctl(ctx context.Context, ep commonEndpoint, io usermem.IO, args arch.Sysc
}
// interfaceIoctl implements interface requests.
-func interfaceIoctl(ctx context.Context, io usermem.IO, arg int, ifr *linux.IFReq) *syserr.Error {
+func interfaceIoctl(ctx context.Context, _ usermem.IO, arg int, ifr *linux.IFReq) *syserr.Error {
var (
iface inet.Interface
index int32
@@ -3098,8 +3105,8 @@ func interfaceIoctl(ctx context.Context, io usermem.IO, arg int, ifr *linux.IFRe
)
// Find the relevant device.
- stack := inet.StackFromContext(ctx)
- if stack == nil {
+ stk := inet.StackFromContext(ctx)
+ if stk == nil {
return syserr.ErrNoDevice
}
@@ -3109,7 +3116,7 @@ func interfaceIoctl(ctx context.Context, io usermem.IO, arg int, ifr *linux.IFRe
// Gets the name of the interface given the interface index
// stored in ifr_ifindex.
index = int32(hostarch.ByteOrder.Uint32(ifr.Data[:4]))
- if iface, ok := stack.Interfaces()[index]; ok {
+ if iface, ok := stk.Interfaces()[index]; ok {
ifr.SetName(iface.Name)
return nil
}
@@ -3117,7 +3124,7 @@ func interfaceIoctl(ctx context.Context, io usermem.IO, arg int, ifr *linux.IFRe
}
// Find the relevant device.
- for index, iface = range stack.Interfaces() {
+ for index, iface = range stk.Interfaces() {
if iface.Name == ifr.Name() {
found = true
break
@@ -3150,7 +3157,7 @@ func interfaceIoctl(ctx context.Context, io usermem.IO, arg int, ifr *linux.IFRe
}
case linux.SIOCGIFFLAGS:
- f, err := interfaceStatusFlags(stack, iface.Name)
+ f, err := interfaceStatusFlags(stk, iface.Name)
if err != nil {
return err
}
@@ -3160,7 +3167,7 @@ func interfaceIoctl(ctx context.Context, io usermem.IO, arg int, ifr *linux.IFRe
case linux.SIOCGIFADDR:
// Copy the IPv4 address out.
- for _, addr := range stack.InterfaceAddrs()[index] {
+ for _, addr := range stk.InterfaceAddrs()[index] {
// This ioctl is only compatible with AF_INET addresses.
if addr.Family != linux.AF_INET {
continue
@@ -3196,7 +3203,7 @@ func interfaceIoctl(ctx context.Context, io usermem.IO, arg int, ifr *linux.IFRe
case linux.SIOCGIFNETMASK:
// Gets the network mask of a device.
- for _, addr := range stack.InterfaceAddrs()[index] {
+ for _, addr := range stk.InterfaceAddrs()[index] {
// This ioctl is only compatible with AF_INET addresses.
if addr.Family != linux.AF_INET {
continue
@@ -3228,24 +3235,24 @@ func interfaceIoctl(ctx context.Context, io usermem.IO, arg int, ifr *linux.IFRe
}
// ifconfIoctl populates a struct ifconf for the SIOCGIFCONF ioctl.
-func ifconfIoctl(ctx context.Context, t *kernel.Task, io usermem.IO, ifc *linux.IFConf) error {
+func ifconfIoctl(ctx context.Context, t *kernel.Task, _ usermem.IO, ifc *linux.IFConf) error {
// If Ptr is NULL, return the necessary buffer size via Len.
// Otherwise, write up to Len bytes starting at Ptr containing ifreq
// structs.
- stack := inet.StackFromContext(ctx)
- if stack == nil {
+ stk := inet.StackFromContext(ctx)
+ if stk == nil {
return syserr.ErrNoDevice.ToError()
}
if ifc.Ptr == 0 {
- ifc.Len = int32(len(stack.Interfaces())) * int32(linux.SizeOfIFReq)
+ ifc.Len = int32(len(stk.Interfaces())) * int32(linux.SizeOfIFReq)
return nil
}
max := ifc.Len
ifc.Len = 0
- for key, ifaceAddrs := range stack.InterfaceAddrs() {
- iface := stack.Interfaces()[key]
+ for key, ifaceAddrs := range stk.InterfaceAddrs() {
+ iface := stk.Interfaces()[key]
for _, ifaceAddr := range ifaceAddrs {
// Don't write past the end of the buffer.
if ifc.Len+int32(linux.SizeOfIFReq) > max {
diff --git a/pkg/sentry/socket/netstack/netstack_state.go b/pkg/sentry/socket/netstack/netstack_state.go
new file mode 100644
index 000000000..591e00d42
--- /dev/null
+++ b/pkg/sentry/socket/netstack/netstack_state.go
@@ -0,0 +1,31 @@
+// Copyright 2021 The gVisor Authors.
+//
+// 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 netstack
+
+import (
+ "time"
+)
+
+func (s *socketOpsCommon) saveTimestamp() int64 {
+ s.readMu.Lock()
+ defer s.readMu.Unlock()
+ return s.timestamp.UnixNano()
+}
+
+func (s *socketOpsCommon) loadTimestamp(nsec int64) {
+ s.readMu.Lock()
+ defer s.readMu.Unlock()
+ s.timestamp = time.Unix(0, nsec)
+}
diff --git a/pkg/sentry/socket/netstack/stack.go b/pkg/sentry/socket/netstack/stack.go
index 208ab9909..ea199f223 100644
--- a/pkg/sentry/socket/netstack/stack.go
+++ b/pkg/sentry/socket/netstack/stack.go
@@ -155,7 +155,7 @@ func (s *Stack) AddInterfaceAddr(idx int32, addr inet.InterfaceAddr) error {
// Attach address to interface.
nicID := tcpip.NICID(idx)
- if err := s.Stack.AddProtocolAddressWithOptions(nicID, protocolAddress, stack.CanBePrimaryEndpoint); err != nil {
+ if err := s.Stack.AddProtocolAddress(nicID, protocolAddress, stack.AddressProperties{}); err != nil {
return syserr.TranslateNetstackError(err).ToError()
}
diff --git a/pkg/sentry/socket/socket.go b/pkg/sentry/socket/socket.go
index 841d5bd55..d4b80a39d 100644
--- a/pkg/sentry/socket/socket.go
+++ b/pkg/sentry/socket/socket.go
@@ -21,6 +21,7 @@ import (
"bytes"
"fmt"
"sync/atomic"
+ "time"
"golang.org/x/sys/unix"
"gvisor.dev/gvisor/pkg/abi/linux"
@@ -51,8 +52,19 @@ type ControlMessages struct {
func packetInfoToLinux(packetInfo tcpip.IPPacketInfo) linux.ControlMessageIPPacketInfo {
var p linux.ControlMessageIPPacketInfo
p.NIC = int32(packetInfo.NIC)
- copy(p.LocalAddr[:], []byte(packetInfo.LocalAddr))
- copy(p.DestinationAddr[:], []byte(packetInfo.DestinationAddr))
+ copy(p.LocalAddr[:], packetInfo.LocalAddr)
+ copy(p.DestinationAddr[:], packetInfo.DestinationAddr)
+ return p
+}
+
+// ipv6PacketInfoToLinux converts IPv6PacketInfo from tcpip format to Linux
+// format.
+func ipv6PacketInfoToLinux(packetInfo tcpip.IPv6PacketInfo) linux.ControlMessageIPv6PacketInfo {
+ var p linux.ControlMessageIPv6PacketInfo
+ if n := copy(p.Addr[:], packetInfo.Addr); n != len(p.Addr) {
+ panic(fmt.Sprintf("got copy(%x, %x) = %d, want = %d", p.Addr, packetInfo.Addr, n, len(p.Addr)))
+ }
+ p.NIC = uint32(packetInfo.NIC)
return p
}
@@ -114,7 +126,7 @@ func NewIPControlMessages(family int, cmgs tcpip.ControlMessages) IPControlMessa
if cmgs.HasOriginalDstAddress {
orgDstAddr, _ = ConvertAddress(family, cmgs.OriginalDstAddress)
}
- return IPControlMessages{
+ cm := IPControlMessages{
HasTimestamp: cmgs.HasTimestamp,
Timestamp: cmgs.Timestamp,
HasInq: cmgs.HasInq,
@@ -125,9 +137,16 @@ func NewIPControlMessages(family int, cmgs tcpip.ControlMessages) IPControlMessa
TClass: cmgs.TClass,
HasIPPacketInfo: cmgs.HasIPPacketInfo,
PacketInfo: packetInfoToLinux(cmgs.PacketInfo),
+ HasIPv6PacketInfo: cmgs.HasIPv6PacketInfo,
OriginalDstAddress: orgDstAddr,
SockErr: sockErrCmsgToLinux(cmgs.SockErr),
}
+
+ if cm.HasIPv6PacketInfo {
+ cm.IPv6PacketInfo = ipv6PacketInfoToLinux(cmgs.IPv6PacketInfo)
+ }
+
+ return cm
}
// IPControlMessages contains socket control messages for IP sockets.
@@ -138,9 +157,9 @@ type IPControlMessages struct {
// HasTimestamp indicates whether Timestamp is valid/set.
HasTimestamp bool
- // Timestamp is the time (in ns) that the last packet used to create
- // the read data was received.
- Timestamp int64
+ // Timestamp is the time that the last packet used to create the read data
+ // was received.
+ Timestamp time.Time `state:".(int64)"`
// HasInq indicates whether Inq is valid/set.
HasInq bool
@@ -166,6 +185,12 @@ type IPControlMessages struct {
// PacketInfo holds interface and address data on an incoming packet.
PacketInfo linux.ControlMessageIPPacketInfo
+ // HasIPv6PacketInfo indicates whether IPv6PacketInfo is set.
+ HasIPv6PacketInfo bool
+
+ // PacketInfo holds interface and address data on an incoming packet.
+ IPv6PacketInfo linux.ControlMessageIPv6PacketInfo
+
// OriginalDestinationAddress holds the original destination address
// and port of the incoming packet.
OriginalDstAddress linux.SockAddr
diff --git a/pkg/sentry/socket/socket_state.go b/pkg/sentry/socket/socket_state.go
new file mode 100644
index 000000000..32e12b238
--- /dev/null
+++ b/pkg/sentry/socket/socket_state.go
@@ -0,0 +1,27 @@
+// Copyright 2021 The gVisor Authors.
+//
+// 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 socket
+
+import (
+ "time"
+)
+
+func (i *IPControlMessages) saveTimestamp() int64 {
+ return i.Timestamp.UnixNano()
+}
+
+func (i *IPControlMessages) loadTimestamp(nsec int64) {
+ i.Timestamp = time.Unix(0, nsec)
+}
diff --git a/pkg/sentry/socket/unix/transport/queue.go b/pkg/sentry/socket/unix/transport/queue.go
index a9cedcf5f..188ad3bd9 100644
--- a/pkg/sentry/socket/unix/transport/queue.go
+++ b/pkg/sentry/socket/unix/transport/queue.go
@@ -59,12 +59,14 @@ func (q *queue) Close() {
// q.WriterQueue.Notify(waiter.WritableEvents)
func (q *queue) Reset(ctx context.Context) {
q.mu.Lock()
- for cur := q.dataList.Front(); cur != nil; cur = cur.Next() {
- cur.Release(ctx)
- }
+ dataList := q.dataList
q.dataList.Reset()
q.used = 0
q.mu.Unlock()
+
+ for cur := dataList.Front(); cur != nil; cur = cur.Next() {
+ cur.Release(ctx)
+ }
}
// DecRef implements RefCounter.DecRef.
diff --git a/pkg/sentry/strace/strace.go b/pkg/sentry/strace/strace.go
index 757ff2a40..4d3f4d556 100644
--- a/pkg/sentry/strace/strace.go
+++ b/pkg/sentry/strace/strace.go
@@ -610,9 +610,9 @@ func (i *SyscallInfo) printExit(t *kernel.Task, elapsed time.Duration, output []
if err == nil {
// Fill in the output after successful execution.
i.post(t, args, retval, output, LogMaximumSize)
- rval = fmt.Sprintf("%#x (%v)", retval, elapsed)
+ rval = fmt.Sprintf("%d (%#x) (%v)", retval, retval, elapsed)
} else {
- rval = fmt.Sprintf("%#x errno=%d (%s) (%v)", retval, errno, err, elapsed)
+ rval = fmt.Sprintf("%d (%#x) errno=%d (%s) (%v)", retval, retval, errno, err, elapsed)
}
switch len(output) {
diff --git a/pkg/sentry/time/sampler_arm64.go b/pkg/sentry/time/sampler_arm64.go
index 3560e66ae..9b8c9a480 100644
--- a/pkg/sentry/time/sampler_arm64.go
+++ b/pkg/sentry/time/sampler_arm64.go
@@ -30,9 +30,9 @@ func getDefaultArchOverheadCycles() TSCValue {
// frqRatio. defaultOverheadCycles of ARM equals to that on
// x86 devided by frqRatio
cntfrq := getCNTFRQ()
- frqRatio := 1000000000 / cntfrq
+ frqRatio := 1000000000 / float64(cntfrq)
overheadCycles := (1 * 1000) / frqRatio
- return overheadCycles
+ return TSCValue(overheadCycles)
}
// defaultOverheadTSC is the default estimated syscall overhead in TSC cycles.
diff --git a/pkg/sentry/usage/memory.go b/pkg/sentry/usage/memory.go
index e7073ec87..d9df890c4 100644
--- a/pkg/sentry/usage/memory.go
+++ b/pkg/sentry/usage/memory.go
@@ -252,9 +252,9 @@ func (m *MemoryLocked) Copy() (MemoryStats, uint64) {
return ms, m.totalLocked()
}
-// These options control how much total memory the is reported to the application.
-// They may only be set before the application starts executing, and must not
-// be modified.
+// These options control how much total memory the is reported to the
+// application. They may only be set before the application starts executing,
+// and must not be modified.
var (
// MinimumTotalMemoryBytes is the minimum reported total system memory.
MinimumTotalMemoryBytes uint64 = 2 << 30 // 2 GB
diff --git a/pkg/sentry/vfs/epoll.go b/pkg/sentry/vfs/epoll.go
index 04bc4d10c..fefd0fc9c 100644
--- a/pkg/sentry/vfs/epoll.go
+++ b/pkg/sentry/vfs/epoll.go
@@ -135,12 +135,16 @@ func (ep *EpollInstance) Readiness(mask waiter.EventMask) waiter.EventMask {
return 0
}
ep.mu.Lock()
- for epi := ep.ready.Front(); epi != nil; epi = epi.Next() {
+ var next *epollInterest
+ for epi := ep.ready.Front(); epi != nil; epi = next {
+ next = epi.Next()
wmask := waiter.EventMaskFromLinux(epi.mask)
if epi.key.file.Readiness(wmask)&wmask != 0 {
ep.mu.Unlock()
return waiter.ReadableEvents
}
+ ep.ready.Remove(epi)
+ epi.ready = false
}
ep.mu.Unlock()
return 0
diff --git a/pkg/sentry/vfs/file_description_impl_util.go b/pkg/sentry/vfs/file_description_impl_util.go
index 5dab069ed..452f5f1f9 100644
--- a/pkg/sentry/vfs/file_description_impl_util.go
+++ b/pkg/sentry/vfs/file_description_impl_util.go
@@ -17,6 +17,7 @@ package vfs
import (
"bytes"
"io"
+ "math"
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
@@ -399,6 +400,9 @@ func (fd *DynamicBytesFileDescriptionImpl) Write(ctx context.Context, src userme
// GenericConfigureMMap may be used by most implementations of
// FileDescriptionImpl.ConfigureMMap.
func GenericConfigureMMap(fd *FileDescription, m memmap.Mappable, opts *memmap.MMapOpts) error {
+ if opts.Offset+opts.Length > math.MaxInt64 {
+ return linuxerr.EOVERFLOW
+ }
opts.Mappable = m
opts.MappingIdentity = fd
fd.IncRef()
diff --git a/pkg/sentry/vfs/resolving_path.go b/pkg/sentry/vfs/resolving_path.go
index 7fd7f000d..40aff2927 100644
--- a/pkg/sentry/vfs/resolving_path.go
+++ b/pkg/sentry/vfs/resolving_path.go
@@ -223,6 +223,12 @@ func (rp *ResolvingPath) Final() bool {
return rp.curPart == 0 && !rp.pit.NextOk()
}
+// Pit returns a copy of rp's current path iterator. Modifying the iterator
+// does not change rp.
+func (rp *ResolvingPath) Pit() fspath.Iterator {
+ return rp.pit
+}
+
// Component returns the current path component in the stream represented by
// rp.
//
diff --git a/pkg/sentry/vfs/save_restore.go b/pkg/sentry/vfs/save_restore.go
index 8998a82dd..8a6ced365 100644
--- a/pkg/sentry/vfs/save_restore.go
+++ b/pkg/sentry/vfs/save_restore.go
@@ -15,7 +15,6 @@
package vfs
import (
- "fmt"
"sync/atomic"
"gvisor.dev/gvisor/pkg/abi/linux"
@@ -24,6 +23,18 @@ import (
"gvisor.dev/gvisor/pkg/waiter"
)
+// ErrCorruption indicates a failed restore due to external file system state in
+// corruption.
+type ErrCorruption struct {
+ // Err is the wrapped error.
+ Err error
+}
+
+// Error returns a sensible description of the restore error.
+func (e ErrCorruption) Error() string {
+ return "restore failed due to external file system state in corruption: " + e.Err.Error()
+}
+
// FilesystemImplSaveRestoreExtension is an optional extension to
// FilesystemImpl.
type FilesystemImplSaveRestoreExtension interface {
@@ -37,38 +48,30 @@ type FilesystemImplSaveRestoreExtension interface {
// PrepareSave prepares all filesystems for serialization.
func (vfs *VirtualFilesystem) PrepareSave(ctx context.Context) error {
- failures := 0
for fs := range vfs.getFilesystems() {
if ext, ok := fs.impl.(FilesystemImplSaveRestoreExtension); ok {
if err := ext.PrepareSave(ctx); err != nil {
- ctx.Warningf("%T.PrepareSave failed: %v", fs.impl, err)
- failures++
+ fs.DecRef(ctx)
+ return err
}
}
fs.DecRef(ctx)
}
- if failures != 0 {
- return fmt.Errorf("%d filesystems failed to prepare for serialization", failures)
- }
return nil
}
// CompleteRestore completes restoration from checkpoint for all filesystems
// after deserialization.
func (vfs *VirtualFilesystem) CompleteRestore(ctx context.Context, opts *CompleteRestoreOptions) error {
- failures := 0
for fs := range vfs.getFilesystems() {
if ext, ok := fs.impl.(FilesystemImplSaveRestoreExtension); ok {
if err := ext.CompleteRestore(ctx, *opts); err != nil {
- ctx.Warningf("%T.CompleteRestore failed: %v", fs.impl, err)
- failures++
+ fs.DecRef(ctx)
+ return err
}
}
fs.DecRef(ctx)
}
- if failures != 0 {
- return fmt.Errorf("%d filesystems failed to complete restore after deserialization", failures)
- }
return nil
}