summaryrefslogtreecommitdiffhomepage
path: root/pkg/sentry/kernel
diff options
context:
space:
mode:
Diffstat (limited to 'pkg/sentry/kernel')
-rw-r--r--pkg/sentry/kernel/BUILD9
-rw-r--r--pkg/sentry/kernel/epoll/BUILD1
-rw-r--r--pkg/sentry/kernel/epoll/epoll.go3
-rw-r--r--pkg/sentry/kernel/fd_map.go366
-rw-r--r--pkg/sentry/kernel/fd_map_test.go136
-rw-r--r--pkg/sentry/kernel/fd_table.go380
-rw-r--r--pkg/sentry/kernel/fd_table_test.go192
-rw-r--r--pkg/sentry/kernel/fd_table_unsafe.go103
-rw-r--r--pkg/sentry/kernel/kdefs/BUILD10
-rw-r--r--pkg/sentry/kernel/kdefs/kdefs.go20
-rw-r--r--pkg/sentry/kernel/kernel.go76
-rw-r--r--pkg/sentry/kernel/task.go65
-rw-r--r--pkg/sentry/kernel/task_clone.go40
-rw-r--r--pkg/sentry/kernel/task_exec.go2
-rw-r--r--pkg/sentry/kernel/task_exit.go4
-rw-r--r--pkg/sentry/kernel/task_log.go2
-rw-r--r--pkg/sentry/kernel/task_start.go13
17 files changed, 802 insertions, 620 deletions
diff --git a/pkg/sentry/kernel/BUILD b/pkg/sentry/kernel/BUILD
index c172d399e..7b92f1b8d 100644
--- a/pkg/sentry/kernel/BUILD
+++ b/pkg/sentry/kernel/BUILD
@@ -96,7 +96,8 @@ go_library(
srcs = [
"abstract_socket_namespace.go",
"context.go",
- "fd_map.go",
+ "fd_table.go",
+ "fd_table_unsafe.go",
"fs_context.go",
"ipc_namespace.go",
"kernel.go",
@@ -179,7 +180,6 @@ go_library(
"//pkg/sentry/kernel/auth",
"//pkg/sentry/kernel/epoll",
"//pkg/sentry/kernel/futex",
- "//pkg/sentry/kernel/kdefs",
"//pkg/sentry/kernel/sched",
"//pkg/sentry/kernel/semaphore",
"//pkg/sentry/kernel/shm",
@@ -214,7 +214,7 @@ go_test(
name = "kernel_test",
size = "small",
srcs = [
- "fd_map_test.go",
+ "fd_table_test.go",
"table_test.go",
"task_test.go",
"timekeeper_test.go",
@@ -223,9 +223,10 @@ go_test(
deps = [
"//pkg/abi",
"//pkg/sentry/arch",
+ "//pkg/sentry/context",
"//pkg/sentry/context/contexttest",
+ "//pkg/sentry/fs",
"//pkg/sentry/fs/filetest",
- "//pkg/sentry/kernel/kdefs",
"//pkg/sentry/kernel/sched",
"//pkg/sentry/limits",
"//pkg/sentry/pgalloc",
diff --git a/pkg/sentry/kernel/epoll/BUILD b/pkg/sentry/kernel/epoll/BUILD
index fb99cfc8f..f46c43128 100644
--- a/pkg/sentry/kernel/epoll/BUILD
+++ b/pkg/sentry/kernel/epoll/BUILD
@@ -30,7 +30,6 @@ go_library(
"//pkg/sentry/fs",
"//pkg/sentry/fs/anon",
"//pkg/sentry/fs/fsutil",
- "//pkg/sentry/kernel/kdefs",
"//pkg/sentry/usermem",
"//pkg/waiter",
],
diff --git a/pkg/sentry/kernel/epoll/epoll.go b/pkg/sentry/kernel/epoll/epoll.go
index 33c7dccae..9c0a4e1b4 100644
--- a/pkg/sentry/kernel/epoll/epoll.go
+++ b/pkg/sentry/kernel/epoll/epoll.go
@@ -26,7 +26,6 @@ import (
"gvisor.dev/gvisor/pkg/sentry/fs"
"gvisor.dev/gvisor/pkg/sentry/fs/anon"
"gvisor.dev/gvisor/pkg/sentry/fs/fsutil"
- "gvisor.dev/gvisor/pkg/sentry/kernel/kdefs"
"gvisor.dev/gvisor/pkg/sentry/usermem"
"gvisor.dev/gvisor/pkg/waiter"
)
@@ -61,7 +60,7 @@ const (
// +stateify savable
type FileIdentifier struct {
File *fs.File `state:"wait"`
- Fd kdefs.FD
+ Fd int32
}
// pollEntry holds all the state associated with an event poll entry, that is,
diff --git a/pkg/sentry/kernel/fd_map.go b/pkg/sentry/kernel/fd_map.go
deleted file mode 100644
index 1b84bfe14..000000000
--- a/pkg/sentry/kernel/fd_map.go
+++ /dev/null
@@ -1,366 +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 kernel
-
-import (
- "bytes"
- "fmt"
- "sort"
- "sync"
- "sync/atomic"
- "syscall"
-
- "gvisor.dev/gvisor/pkg/abi/linux"
- "gvisor.dev/gvisor/pkg/refs"
- "gvisor.dev/gvisor/pkg/sentry/fs"
- "gvisor.dev/gvisor/pkg/sentry/fs/lock"
- "gvisor.dev/gvisor/pkg/sentry/kernel/kdefs"
- "gvisor.dev/gvisor/pkg/sentry/limits"
-)
-
-// FDs is an ordering of FD's that can be made stable.
-type FDs []kdefs.FD
-
-func (f FDs) Len() int {
- return len(f)
-}
-
-func (f FDs) Swap(i, j int) {
- f[i], f[j] = f[j], f[i]
-}
-
-func (f FDs) Less(i, j int) bool {
- return f[i] < f[j]
-}
-
-// FDFlags define flags for an individual descriptor.
-//
-// +stateify savable
-type FDFlags struct {
- // CloseOnExec indicates the descriptor should be closed on exec.
- CloseOnExec bool
-}
-
-// ToLinuxFileFlags converts a kernel.FDFlags object to a Linux file flags
-// representation.
-func (f FDFlags) ToLinuxFileFlags() (mask uint) {
- if f.CloseOnExec {
- mask |= linux.O_CLOEXEC
- }
- return
-}
-
-// ToLinuxFDFlags converts a kernel.FDFlags object to a Linux descriptor flags
-// representation.
-func (f FDFlags) ToLinuxFDFlags() (mask uint) {
- if f.CloseOnExec {
- mask |= linux.FD_CLOEXEC
- }
- return
-}
-
-// descriptor holds the details about a file descriptor, namely a pointer the
-// file itself and the descriptor flags.
-//
-// +stateify savable
-type descriptor struct {
- file *fs.File
- flags FDFlags
-}
-
-// FDMap is used to manage File references and flags.
-//
-// +stateify savable
-type FDMap struct {
- refs.AtomicRefCount
- k *Kernel
- files map[kdefs.FD]descriptor
- mu sync.RWMutex `state:"nosave"`
- uid uint64
-}
-
-// ID returns a unique identifier for this FDMap.
-func (f *FDMap) ID() uint64 {
- return f.uid
-}
-
-// NewFDMap allocates a new FDMap that may be used by tasks in k.
-func (k *Kernel) NewFDMap() *FDMap {
- f := FDMap{
- k: k,
- files: make(map[kdefs.FD]descriptor),
- uid: atomic.AddUint64(&k.fdMapUids, 1),
- }
- f.EnableLeakCheck("kernel.FDMap")
- return &f
-}
-
-// destroy removes all of the file descriptors from the map.
-func (f *FDMap) destroy() {
- f.RemoveIf(func(*fs.File, FDFlags) bool {
- return true
- })
-}
-
-// DecRef implements RefCounter.DecRef with destructor f.destroy.
-func (f *FDMap) DecRef() {
- f.DecRefWithDestructor(f.destroy)
-}
-
-// Size returns the number of file descriptor slots currently allocated.
-func (f *FDMap) Size() int {
- f.mu.RLock()
- defer f.mu.RUnlock()
-
- return len(f.files)
-}
-
-// String is a stringer for FDMap.
-func (f *FDMap) String() string {
- f.mu.RLock()
- defer f.mu.RUnlock()
-
- var b bytes.Buffer
- for k, v := range f.files {
- n, _ := v.file.Dirent.FullName(nil /* root */)
- b.WriteString(fmt.Sprintf("\tfd:%d => name %s\n", k, n))
- }
- return b.String()
-}
-
-// NewFDFrom allocates a new FD guaranteed to be the lowest number available
-// greater than or equal to from. This property is important as Unix programs
-// tend to count on this allocation order.
-func (f *FDMap) NewFDFrom(fd kdefs.FD, file *fs.File, flags FDFlags, limitSet *limits.LimitSet) (kdefs.FD, error) {
- if fd < 0 {
- // Don't accept negative FDs.
- return 0, syscall.EINVAL
- }
-
- f.mu.Lock()
- defer f.mu.Unlock()
-
- // Finds the lowest fd not in the handles map.
- lim := limitSet.Get(limits.NumberOfFiles)
- for i := fd; lim.Cur == limits.Infinity || i < kdefs.FD(lim.Cur); i++ {
- if _, ok := f.files[i]; !ok {
- file.IncRef()
- f.files[i] = descriptor{file, flags}
- return i, nil
- }
- }
-
- return -1, syscall.EMFILE
-}
-
-// NewFDAt sets the file reference for the given FD. If there is an
-// active reference for that FD, the ref count for that existing reference
-// is decremented.
-func (f *FDMap) NewFDAt(fd kdefs.FD, file *fs.File, flags FDFlags, limitSet *limits.LimitSet) error {
- if fd < 0 {
- // Don't accept negative FDs.
- return syscall.EBADF
- }
-
- // In this one case we do not do a defer of the Unlock. The
- // reason is that we must have done all the work needed for
- // discarding any old open file before we return to the
- // caller. In other words, the DecRef(), below, must have
- // completed by the time we return to the caller to ensure
- // side effects are, in fact, effected. A classic example is
- // dup2(fd1, fd2); if fd2 was already open, it must be closed,
- // and we don't want to resume the caller until it is; we have
- // to block on the DecRef(). Hence we can not just do a 'go
- // oldfile.DecRef()', since there would be no guarantee that
- // it would be done before we the caller resumed. Since we
- // must wait for the DecRef() to finish, and that could take
- // time, it's best to first call f.muUnlock beore so we are
- // not blocking other uses of this FDMap on the DecRef() call.
- f.mu.Lock()
- oldDesc, oldExists := f.files[fd]
- lim := limitSet.Get(limits.NumberOfFiles).Cur
- // if we're closing one then the effective limit is one
- // more than the actual limit.
- if oldExists && lim != limits.Infinity {
- lim++
- }
- if lim != limits.Infinity && fd >= kdefs.FD(lim) {
- f.mu.Unlock()
- return syscall.EMFILE
- }
-
- file.IncRef()
- f.files[fd] = descriptor{file, flags}
- f.mu.Unlock()
-
- if oldExists {
- oldDesc.file.DecRef()
- }
- return nil
-}
-
-// SetFlags sets the flags for the given file descriptor, if it is valid.
-func (f *FDMap) SetFlags(fd kdefs.FD, flags FDFlags) {
- f.mu.Lock()
- defer f.mu.Unlock()
-
- desc, ok := f.files[fd]
- if !ok {
- return
- }
-
- f.files[fd] = descriptor{desc.file, flags}
-}
-
-// GetDescriptor returns a reference to the file and the flags for the FD. It
-// bumps its reference count as well. It returns nil if there is no File
-// for the FD, i.e. if the FD is invalid. The caller must use DecRef
-// when they are done.
-func (f *FDMap) GetDescriptor(fd kdefs.FD) (*fs.File, FDFlags) {
- f.mu.RLock()
- defer f.mu.RUnlock()
-
- if desc, ok := f.files[fd]; ok {
- desc.file.IncRef()
- return desc.file, desc.flags
- }
- return nil, FDFlags{}
-}
-
-// GetFile returns a reference to the File for the FD and bumps
-// its reference count as well. It returns nil if there is no File
-// for the FD, i.e. if the FD is invalid. The caller must use DecRef
-// when they are done.
-func (f *FDMap) GetFile(fd kdefs.FD) *fs.File {
- f.mu.RLock()
- if desc, ok := f.files[fd]; ok {
- desc.file.IncRef()
- f.mu.RUnlock()
- return desc.file
- }
- f.mu.RUnlock()
- return nil
-}
-
-// fds returns an ordering of FDs.
-func (f *FDMap) fds() FDs {
- fds := make(FDs, 0, len(f.files))
- for fd := range f.files {
- fds = append(fds, fd)
- }
- sort.Sort(fds)
- return fds
-}
-
-// GetFDs returns a list of valid fds.
-func (f *FDMap) GetFDs() FDs {
- f.mu.RLock()
- defer f.mu.RUnlock()
- return f.fds()
-}
-
-// GetRefs returns a stable slice of references to all files and bumps the
-// reference count on each. The caller must use DecRef on each reference when
-// they're done using the slice.
-func (f *FDMap) GetRefs() []*fs.File {
- f.mu.RLock()
- defer f.mu.RUnlock()
-
- fds := f.fds()
- fs := make([]*fs.File, 0, len(fds))
- for _, fd := range fds {
- desc := f.files[fd]
- desc.file.IncRef()
- fs = append(fs, desc.file)
- }
- return fs
-}
-
-// Fork returns an independent FDMap pointing to the same descriptors.
-func (f *FDMap) Fork() *FDMap {
- f.mu.RLock()
- defer f.mu.RUnlock()
-
- clone := f.k.NewFDMap()
-
- // Grab a extra reference for every file.
- for fd, desc := range f.files {
- desc.file.IncRef()
- clone.files[fd] = desc
- }
-
- // That's it!
- return clone
-}
-
-// unlock releases all file locks held by this FDMap's uid. Must only be
-// called on a non-nil *fs.File.
-func (f *FDMap) unlock(file *fs.File) {
- id := lock.UniqueID(f.ID())
- file.Dirent.Inode.LockCtx.Posix.UnlockRegion(id, lock.LockRange{0, lock.LockEOF})
-}
-
-// inotifyFileClose generates the appropriate inotify events for f being closed.
-func inotifyFileClose(f *fs.File) {
- var ev uint32
- d := f.Dirent
-
- if fs.IsDir(d.Inode.StableAttr) {
- ev |= linux.IN_ISDIR
- }
-
- if f.Flags().Write {
- ev |= linux.IN_CLOSE_WRITE
- } else {
- ev |= linux.IN_CLOSE_NOWRITE
- }
-
- d.InotifyEvent(ev, 0)
-}
-
-// Remove removes an FD from the FDMap, and returns (File, true) if a File
-// one was found. Callers are expected to decrement the reference count on
-// the File. Otherwise returns (nil, false).
-func (f *FDMap) Remove(fd kdefs.FD) (*fs.File, bool) {
- f.mu.Lock()
- desc := f.files[fd]
- delete(f.files, fd)
- f.mu.Unlock()
- if desc.file != nil {
- f.unlock(desc.file)
- inotifyFileClose(desc.file)
- return desc.file, true
- }
- return nil, false
-}
-
-// RemoveIf removes all FDs where cond is true.
-func (f *FDMap) RemoveIf(cond func(*fs.File, FDFlags) bool) {
- var removed []*fs.File
- f.mu.Lock()
- for fd, desc := range f.files {
- if desc.file != nil && cond(desc.file, desc.flags) {
- delete(f.files, fd)
- removed = append(removed, desc.file)
- }
- }
- f.mu.Unlock()
-
- for _, file := range removed {
- f.unlock(file)
- inotifyFileClose(file)
- file.DecRef()
- }
-}
diff --git a/pkg/sentry/kernel/fd_map_test.go b/pkg/sentry/kernel/fd_map_test.go
deleted file mode 100644
index 8571dbe59..000000000
--- a/pkg/sentry/kernel/fd_map_test.go
+++ /dev/null
@@ -1,136 +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 kernel
-
-import (
- "testing"
-
- "gvisor.dev/gvisor/pkg/sentry/fs/filetest"
- "gvisor.dev/gvisor/pkg/sentry/kernel/kdefs"
- "gvisor.dev/gvisor/pkg/sentry/limits"
-)
-
-const (
- // maxFD is the maximum FD to try to create in the map.
- // This number of open files has been seen in the wild.
- maxFD = 2 * 1024
-)
-
-func newTestFDMap() *FDMap {
- return &FDMap{
- files: make(map[kdefs.FD]descriptor),
- }
-}
-
-// TestFDMapMany allocates maxFD FDs, i.e. maxes out the FDMap,
-// until there is no room, then makes sure that NewFDAt works
-// and also that if we remove one and add one that works too.
-func TestFDMapMany(t *testing.T) {
- file := filetest.NewTestFile(t)
- limitSet := limits.NewLimitSet()
- limitSet.Set(limits.NumberOfFiles, limits.Limit{maxFD, maxFD}, true /* privileged */)
-
- f := newTestFDMap()
- for i := 0; i < maxFD; i++ {
- if _, err := f.NewFDFrom(0, file, FDFlags{}, limitSet); err != nil {
- t.Fatalf("Allocated %v FDs but wanted to allocate %v", i, maxFD)
- }
- }
-
- if _, err := f.NewFDFrom(0, file, FDFlags{}, limitSet); err == nil {
- t.Fatalf("f.NewFDFrom(0, r) in full map: got nil, wanted error")
- }
-
- if err := f.NewFDAt(1, file, FDFlags{}, limitSet); err != nil {
- t.Fatalf("f.NewFDAt(1, r, FDFlags{}): got %v, wanted nil", err)
- }
-}
-
-// TestFDMap does a set of simple tests to make sure simple adds,
-// removes, GetRefs, and DecRefs work. The ordering is just weird
-// enough that a table-driven approach seemed clumsy.
-func TestFDMap(t *testing.T) {
- file := filetest.NewTestFile(t)
- limitSet := limits.NewLimitSet()
- limitSet.Set(limits.NumberOfFiles, limits.Limit{1, maxFD}, true /* privileged */)
-
- f := newTestFDMap()
- if _, err := f.NewFDFrom(0, file, FDFlags{}, limitSet); err != nil {
- t.Fatalf("Adding an FD to an empty 1-size map: got %v, want nil", err)
- }
-
- if _, err := f.NewFDFrom(0, file, FDFlags{}, limitSet); err == nil {
- t.Fatalf("Adding an FD to a filled 1-size map: got nil, wanted an error")
- }
-
- largeLimit := limits.Limit{maxFD, maxFD}
- limitSet.Set(limits.NumberOfFiles, largeLimit, true /* privileged */)
-
- if fd, err := f.NewFDFrom(0, file, FDFlags{}, limitSet); err != nil {
- t.Fatalf("Adding an FD to a resized map: got %v, want nil", err)
- } else if fd != kdefs.FD(1) {
- t.Fatalf("Added an FD to a resized map: got %v, want 1", fd)
- }
-
- if err := f.NewFDAt(1, file, FDFlags{}, limitSet); err != nil {
- t.Fatalf("Replacing FD 1 via f.NewFDAt(1, r, FDFlags{}): got %v, wanted nil", err)
- }
-
- if err := f.NewFDAt(maxFD+1, file, FDFlags{}, limitSet); err == nil {
- t.Fatalf("Using an FD that was too large via f.NewFDAt(%v, r, FDFlags{}): got nil, wanted an error", maxFD+1)
- }
-
- if ref := f.GetFile(1); ref == nil {
- t.Fatalf("f.GetFile(1): got nil, wanted %v", file)
- }
-
- if ref := f.GetFile(2); ref != nil {
- t.Fatalf("f.GetFile(2): got a %v, wanted nil", ref)
- }
-
- ref, ok := f.Remove(1)
- if !ok {
- t.Fatalf("f.Remove(1) for an existing FD: failed, want success")
- }
- ref.DecRef()
-
- if ref, ok := f.Remove(1); ok {
- ref.DecRef()
- t.Fatalf("r.Remove(1) for a removed FD: got success, want failure")
- }
-
-}
-
-func TestDescriptorFlags(t *testing.T) {
- file := filetest.NewTestFile(t)
- f := newTestFDMap()
- limitSet := limits.NewLimitSet()
- limitSet.Set(limits.NumberOfFiles, limits.Limit{maxFD, maxFD}, true /* privileged */)
-
- origFlags := FDFlags{CloseOnExec: true}
-
- if err := f.NewFDAt(2, file, origFlags, limitSet); err != nil {
- t.Fatalf("f.NewFDAt(2, r, FDFlags{}): got %v, wanted nil", err)
- }
-
- newFile, newFlags := f.GetDescriptor(2)
- if newFile == nil {
- t.Fatalf("f.GetFile(2): got a %v, wanted nil", newFile)
- }
-
- if newFlags != origFlags {
- t.Fatalf("new File flags %+v don't match original %+v", newFlags, origFlags)
- }
-}
diff --git a/pkg/sentry/kernel/fd_table.go b/pkg/sentry/kernel/fd_table.go
new file mode 100644
index 000000000..1f3a57dc1
--- /dev/null
+++ b/pkg/sentry/kernel/fd_table.go
@@ -0,0 +1,380 @@
+// Copyright 2018 Google LLC
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package kernel
+
+import (
+ "bytes"
+ "fmt"
+ "math"
+ "sync"
+ "sync/atomic"
+ "syscall"
+
+ "gvisor.dev/gvisor/pkg/abi/linux"
+ "gvisor.dev/gvisor/pkg/refs"
+ "gvisor.dev/gvisor/pkg/sentry/context"
+ "gvisor.dev/gvisor/pkg/sentry/fs"
+ "gvisor.dev/gvisor/pkg/sentry/fs/lock"
+ "gvisor.dev/gvisor/pkg/sentry/limits"
+)
+
+// FDFlags define flags for an individual descriptor.
+//
+// +stateify savable
+type FDFlags struct {
+ // CloseOnExec indicates the descriptor should be closed on exec.
+ CloseOnExec bool
+}
+
+// ToLinuxFileFlags converts a kernel.FDFlags object to a Linux file flags
+// representation.
+func (f FDFlags) ToLinuxFileFlags() (mask uint) {
+ if f.CloseOnExec {
+ mask |= linux.O_CLOEXEC
+ }
+ return
+}
+
+// ToLinuxFDFlags converts a kernel.FDFlags object to a Linux descriptor flags
+// representation.
+func (f FDFlags) ToLinuxFDFlags() (mask uint) {
+ if f.CloseOnExec {
+ mask |= linux.FD_CLOEXEC
+ }
+ return
+}
+
+// descriptor holds the details about a file descriptor, namely a pointer to
+// the file itself and the descriptor flags.
+//
+// Note that this is immutable and can only be changed via operations on the
+// descriptorTable.
+//
+// +stateify savable
+type descriptor struct {
+ file *fs.File
+ flags FDFlags
+}
+
+// FDTable is used to manage File references and flags.
+//
+// +stateify savable
+type FDTable struct {
+ refs.AtomicRefCount
+ k *Kernel
+
+ // uid is a unique identifier.
+ uid uint64
+
+ // mu protects below.
+ mu sync.Mutex `state:"nosave"`
+
+ // used contains the number of non-nil entries.
+ used int32
+
+ // descriptorTable holds descriptors.
+ descriptorTable `state:".(map[int32]descriptor)"`
+}
+
+func (f *FDTable) saveDescriptorTable() map[int32]descriptor {
+ m := make(map[int32]descriptor)
+ f.forEach(func(fd int32, file *fs.File, flags FDFlags) {
+ m[fd] = descriptor{
+ file: file,
+ flags: flags,
+ }
+ })
+ return m
+}
+
+func (f *FDTable) loadDescriptorTable(m map[int32]descriptor) {
+ f.init() // Initialize table.
+ for fd, d := range m {
+ f.set(fd, d.file, d.flags)
+
+ // Note that we do _not_ need to acquire a extra table
+ // reference here. The table reference will already be
+ // accounted for in the file, so we drop the reference taken by
+ // set above.
+ d.file.DecRef()
+ }
+}
+
+// drop drops the table reference.
+func (f *FDTable) drop(file *fs.File) {
+ // Release locks.
+ file.Dirent.Inode.LockCtx.Posix.UnlockRegion(lock.UniqueID(f.uid), lock.LockRange{0, lock.LockEOF})
+
+ // Send inotify events.
+ d := file.Dirent
+ var ev uint32
+ if fs.IsDir(d.Inode.StableAttr) {
+ ev |= linux.IN_ISDIR
+ }
+ if file.Flags().Write {
+ ev |= linux.IN_CLOSE_WRITE
+ } else {
+ ev |= linux.IN_CLOSE_NOWRITE
+ }
+ d.InotifyEvent(ev, 0)
+
+ // Drop the table reference.
+ file.DecRef()
+}
+
+// ID returns a unique identifier for this FDTable.
+func (f *FDTable) ID() uint64 {
+ return f.uid
+}
+
+// NewFDTable allocates a new FDTable that may be used by tasks in k.
+func (k *Kernel) NewFDTable() *FDTable {
+ f := &FDTable{
+ k: k,
+ uid: atomic.AddUint64(&k.fdMapUids, 1),
+ }
+ f.init()
+ return f
+}
+
+// destroy removes all of the file descriptors from the map.
+func (f *FDTable) destroy() {
+ f.RemoveIf(func(*fs.File, FDFlags) bool {
+ return true
+ })
+}
+
+// DecRef implements RefCounter.DecRef with destructor f.destroy.
+func (f *FDTable) DecRef() {
+ f.DecRefWithDestructor(f.destroy)
+}
+
+// Size returns the number of file descriptor slots currently allocated.
+func (f *FDTable) Size() int {
+ size := atomic.LoadInt32(&f.used)
+ return int(size)
+}
+
+// forEach iterates over all non-nil files.
+//
+// It is the caller's responsibility to acquire an appropriate lock.
+func (f *FDTable) forEach(fn func(fd int32, file *fs.File, flags FDFlags)) {
+ fd := int32(0)
+ for {
+ file, flags, ok := f.get(fd)
+ if !ok {
+ break
+ }
+ if file != nil {
+ if !file.TryIncRef() {
+ continue // Race caught.
+ }
+ fn(int32(fd), file, flags)
+ file.DecRef()
+ }
+ fd++
+ }
+}
+
+// String is a stringer for FDTable.
+func (f *FDTable) String() string {
+ var b bytes.Buffer
+ f.forEach(func(fd int32, file *fs.File, flags FDFlags) {
+ n, _ := file.Dirent.FullName(nil /* root */)
+ b.WriteString(fmt.Sprintf("\tfd:%d => name %s\n", fd, n))
+ })
+ return b.String()
+}
+
+// NewFDs allocates new FDs guaranteed to be the lowest number available
+// greater than or equal to the fd parameter. All files will share the set
+// flags. Success is guaranteed to be all or none.
+func (f *FDTable) NewFDs(ctx context.Context, fd int32, files []*fs.File, flags FDFlags) (fds []int32, err error) {
+ if fd < 0 {
+ // Don't accept negative FDs.
+ return nil, syscall.EINVAL
+ }
+
+ // Default limit.
+ end := int32(math.MaxInt32)
+
+ // Ensure we don't get past the provided limit.
+ if limitSet := limits.FromContext(ctx); limitSet != nil {
+ lim := limitSet.Get(limits.NumberOfFiles)
+ if lim.Cur != limits.Infinity {
+ end = int32(lim.Cur)
+ }
+ if fd >= end {
+ return nil, syscall.EMFILE
+ }
+ }
+
+ f.mu.Lock()
+ defer f.mu.Unlock()
+
+ // Install all entries.
+ for i := fd; i < end && len(fds) < len(files); i++ {
+ if d, _, _ := f.get(i); d == nil {
+ f.set(i, files[len(fds)], flags) // Set the descriptor.
+ fds = append(fds, i) // Record the file descriptor.
+ }
+ }
+
+ // Failure? Unwind existing FDs.
+ if len(fds) < len(files) {
+ for _, i := range fds {
+ f.set(i, nil, FDFlags{}) // Zap entry.
+ }
+ return nil, syscall.EMFILE
+ }
+
+ return fds, nil
+}
+
+// NewFDAt sets the file reference for the given FD. If there is an active
+// reference for that FD, the ref count for that existing reference is
+// decremented.
+func (f *FDTable) NewFDAt(ctx context.Context, fd int32, file *fs.File, flags FDFlags) error {
+ if fd < 0 {
+ // Don't accept negative FDs.
+ return syscall.EBADF
+ }
+
+ f.mu.Lock()
+ defer f.mu.Unlock()
+
+ // Check the limit for the provided file.
+ if limitSet := limits.FromContext(ctx); limitSet != nil {
+ if lim := limitSet.Get(limits.NumberOfFiles); lim.Cur != limits.Infinity && uint64(fd) >= lim.Cur {
+ return syscall.EMFILE
+ }
+ }
+
+ // Install the entry.
+ f.set(fd, file, flags)
+ return nil
+}
+
+// SetFlags sets the flags for the given file descriptor.
+//
+// True is returned iff flags were changed.
+func (f *FDTable) SetFlags(fd int32, flags FDFlags) error {
+ if fd < 0 {
+ // Don't accept negative FDs.
+ return syscall.EBADF
+ }
+
+ f.mu.Lock()
+ defer f.mu.Unlock()
+
+ file, _, _ := f.get(fd)
+ if file == nil {
+ // No file found.
+ return syscall.EBADF
+ }
+
+ // Update the flags.
+ f.set(fd, file, flags)
+ return nil
+}
+
+// Get returns a reference to the file and the flags for the FD or nil if no
+// file is defined for the given fd.
+//
+// N.B. Callers are required to use DecRef when they are done.
+//
+//go:nosplit
+func (f *FDTable) Get(fd int32) (*fs.File, FDFlags) {
+ if fd < 0 {
+ return nil, FDFlags{}
+ }
+
+ for {
+ file, flags, _ := f.get(fd)
+ if file != nil {
+ if !file.TryIncRef() {
+ continue // Race caught.
+ }
+ // Reference acquired.
+ return file, flags
+ }
+ // No file available.
+ return nil, FDFlags{}
+ }
+}
+
+// GetFDs returns a list of valid fds.
+func (f *FDTable) GetFDs() []int32 {
+ fds := make([]int32, 0, f.used)
+ f.forEach(func(fd int32, file *fs.File, flags FDFlags) {
+ fds = append(fds, fd)
+ })
+ return fds
+}
+
+// GetRefs returns a stable slice of references to all files and bumps the
+// reference count on each. The caller must use DecRef on each reference when
+// they're done using the slice.
+func (f *FDTable) GetRefs() []*fs.File {
+ files := make([]*fs.File, 0, f.Size())
+ f.forEach(func(_ int32, file *fs.File, flags FDFlags) {
+ file.IncRef() // Acquire a reference for caller.
+ files = append(files, file)
+ })
+ return files
+}
+
+// Fork returns an independent FDTable.
+func (f *FDTable) Fork() *FDTable {
+ clone := f.k.NewFDTable()
+
+ f.forEach(func(fd int32, file *fs.File, flags FDFlags) {
+ // The set function here will acquire an appropriate table
+ // reference for the clone. We don't need anything else.
+ clone.set(fd, file, flags)
+ })
+ return clone
+}
+
+// Remove removes an FD from and returns a non-file iff successful.
+//
+// N.B. Callers are required to use DecRef when they are done.
+func (f *FDTable) Remove(fd int32) *fs.File {
+ if fd < 0 {
+ return nil
+ }
+
+ f.mu.Lock()
+ defer f.mu.Unlock()
+
+ orig, _, _ := f.get(fd)
+ if orig != nil {
+ orig.IncRef() // Reference for caller.
+ f.set(fd, nil, FDFlags{}) // Zap entry.
+ }
+ return orig
+}
+
+// RemoveIf removes all FDs where cond is true.
+func (f *FDTable) RemoveIf(cond func(*fs.File, FDFlags) bool) {
+ f.mu.Lock()
+ defer f.mu.Unlock()
+
+ f.forEach(func(fd int32, file *fs.File, flags FDFlags) {
+ if cond(file, flags) {
+ f.set(fd, nil, FDFlags{}) // Clear from table.
+ }
+ })
+}
diff --git a/pkg/sentry/kernel/fd_table_test.go b/pkg/sentry/kernel/fd_table_test.go
new file mode 100644
index 000000000..2413788e7
--- /dev/null
+++ b/pkg/sentry/kernel/fd_table_test.go
@@ -0,0 +1,192 @@
+// Copyright 2018 Google LLC
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package kernel
+
+import (
+ "runtime"
+ "sync"
+ "testing"
+
+ "gvisor.dev/gvisor/pkg/sentry/context"
+ "gvisor.dev/gvisor/pkg/sentry/context/contexttest"
+ "gvisor.dev/gvisor/pkg/sentry/fs"
+ "gvisor.dev/gvisor/pkg/sentry/fs/filetest"
+ "gvisor.dev/gvisor/pkg/sentry/limits"
+)
+
+const (
+ // maxFD is the maximum FD to try to create in the map.
+ //
+ // This number of open files has been seen in the wild.
+ maxFD = 2 * 1024
+)
+
+func runTest(t testing.TB, fn func(ctx context.Context, fdTable *FDTable, file *fs.File, limitSet *limits.LimitSet)) {
+ t.Helper() // Don't show in stacks.
+
+ // Create the limits and context.
+ limitSet := limits.NewLimitSet()
+ limitSet.Set(limits.NumberOfFiles, limits.Limit{maxFD, maxFD}, true)
+ ctx := contexttest.WithLimitSet(contexttest.Context(t), limitSet)
+
+ // Create a test file.;
+ file := filetest.NewTestFile(t)
+
+ // Create the table.
+ fdTable := new(FDTable)
+ fdTable.init()
+
+ // Run the test.
+ fn(ctx, fdTable, file, limitSet)
+}
+
+// TestFDTableMany allocates maxFD FDs, i.e. maxes out the FDTable, until there
+// is no room, then makes sure that NewFDAt works and also that if we remove
+// one and add one that works too.
+func TestFDTableMany(t *testing.T) {
+ runTest(t, func(ctx context.Context, fdTable *FDTable, file *fs.File, _ *limits.LimitSet) {
+ for i := 0; i < maxFD; i++ {
+ if _, err := fdTable.NewFDs(ctx, 0, []*fs.File{file}, FDFlags{}); err != nil {
+ t.Fatalf("Allocated %v FDs but wanted to allocate %v", i, maxFD)
+ }
+ }
+
+ if _, err := fdTable.NewFDs(ctx, 0, []*fs.File{file}, FDFlags{}); err == nil {
+ t.Fatalf("fdTable.NewFDs(0, r) in full map: got nil, wanted error")
+ }
+
+ if err := fdTable.NewFDAt(ctx, 1, file, FDFlags{}); err != nil {
+ t.Fatalf("fdTable.NewFDAt(1, r, FDFlags{}): got %v, wanted nil", err)
+ }
+ })
+}
+
+// TestFDTable does a set of simple tests to make sure simple adds, removes,
+// GetRefs, and DecRefs work. The ordering is just weird enough that a
+// table-driven approach seemed clumsy.
+func TestFDTable(t *testing.T) {
+ runTest(t, func(ctx context.Context, fdTable *FDTable, file *fs.File, limitSet *limits.LimitSet) {
+ // Cap the limit at one.
+ limitSet.Set(limits.NumberOfFiles, limits.Limit{1, maxFD}, true)
+
+ if _, err := fdTable.NewFDs(ctx, 0, []*fs.File{file}, FDFlags{}); err != nil {
+ t.Fatalf("Adding an FD to an empty 1-size map: got %v, want nil", err)
+ }
+
+ if _, err := fdTable.NewFDs(ctx, 0, []*fs.File{file}, FDFlags{}); err == nil {
+ t.Fatalf("Adding an FD to a filled 1-size map: got nil, wanted an error")
+ }
+
+ // Remove the previous limit.
+ limitSet.Set(limits.NumberOfFiles, limits.Limit{maxFD, maxFD}, true)
+
+ if fds, err := fdTable.NewFDs(ctx, 0, []*fs.File{file}, FDFlags{}); err != nil {
+ t.Fatalf("Adding an FD to a resized map: got %v, want nil", err)
+ } else if len(fds) != 1 || fds[0] != 1 {
+ t.Fatalf("Added an FD to a resized map: got %v, want {1}", fds)
+ }
+
+ if err := fdTable.NewFDAt(ctx, 1, file, FDFlags{}); err != nil {
+ t.Fatalf("Replacing FD 1 via fdTable.NewFDAt(1, r, FDFlags{}): got %v, wanted nil", err)
+ }
+
+ if err := fdTable.NewFDAt(ctx, maxFD+1, file, FDFlags{}); err == nil {
+ t.Fatalf("Using an FD that was too large via fdTable.NewFDAt(%v, r, FDFlags{}): got nil, wanted an error", maxFD+1)
+ }
+
+ if ref, _ := fdTable.Get(1); ref == nil {
+ t.Fatalf("fdTable.Get(1): got nil, wanted %v", file)
+ }
+
+ if ref, _ := fdTable.Get(2); ref != nil {
+ t.Fatalf("fdTable.Get(2): got a %v, wanted nil", ref)
+ }
+
+ ref := fdTable.Remove(1)
+ if ref == nil {
+ t.Fatalf("fdTable.Remove(1) for an existing FD: failed, want success")
+ }
+ ref.DecRef()
+
+ if ref := fdTable.Remove(1); ref != nil {
+ t.Fatalf("r.Remove(1) for a removed FD: got success, want failure")
+ }
+ })
+}
+
+func TestDescriptorFlags(t *testing.T) {
+ runTest(t, func(ctx context.Context, fdTable *FDTable, file *fs.File, _ *limits.LimitSet) {
+ if err := fdTable.NewFDAt(ctx, 2, file, FDFlags{CloseOnExec: true}); err != nil {
+ t.Fatalf("fdTable.NewFDAt(2, r, FDFlags{}): got %v, wanted nil", err)
+ }
+
+ newFile, flags := fdTable.Get(2)
+ if newFile == nil {
+ t.Fatalf("fdTable.Get(2): got a %v, wanted nil", newFile)
+ }
+
+ if !flags.CloseOnExec {
+ t.Fatalf("new File flags %v don't match original %d\n", flags, 0)
+ }
+ })
+}
+
+func BenchmarkFDLookupAndDecRef(b *testing.B) {
+ b.StopTimer() // Setup.
+
+ runTest(b, func(ctx context.Context, fdTable *FDTable, file *fs.File, _ *limits.LimitSet) {
+ fds, err := fdTable.NewFDs(ctx, 0, []*fs.File{file, file, file, file, file}, FDFlags{})
+ if err != nil {
+ b.Fatalf("fdTable.NewFDs: got %v, wanted nil", err)
+ }
+
+ b.StartTimer() // Benchmark.
+ for i := 0; i < b.N; i++ {
+ tf, _ := fdTable.Get(fds[i%len(fds)])
+ tf.DecRef()
+ }
+ })
+}
+
+func BenchmarkFDLookupAndDecRefConcurrent(b *testing.B) {
+ b.StopTimer() // Setup.
+
+ runTest(b, func(ctx context.Context, fdTable *FDTable, file *fs.File, _ *limits.LimitSet) {
+ fds, err := fdTable.NewFDs(ctx, 0, []*fs.File{file, file, file, file, file}, FDFlags{})
+ if err != nil {
+ b.Fatalf("fdTable.NewFDs: got %v, wanted nil", err)
+ }
+
+ concurrency := runtime.GOMAXPROCS(0)
+ if concurrency < 4 {
+ concurrency = 4
+ }
+ each := b.N / concurrency
+
+ b.StartTimer() // Benchmark.
+ var wg sync.WaitGroup
+ for i := 0; i < concurrency; i++ {
+ wg.Add(1)
+ go func() {
+ defer wg.Done()
+ for i := 0; i < each; i++ {
+ tf, _ := fdTable.Get(fds[i%len(fds)])
+ tf.DecRef()
+ }
+ }()
+ }
+ wg.Wait()
+ })
+}
diff --git a/pkg/sentry/kernel/fd_table_unsafe.go b/pkg/sentry/kernel/fd_table_unsafe.go
new file mode 100644
index 000000000..e009df974
--- /dev/null
+++ b/pkg/sentry/kernel/fd_table_unsafe.go
@@ -0,0 +1,103 @@
+// Copyright 2018 Google LLC
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package kernel
+
+import (
+ "sync/atomic"
+ "unsafe"
+
+ "gvisor.dev/gvisor/pkg/sentry/fs"
+)
+
+type descriptorTable struct {
+ // slice is a *[]unsafe.Pointer, where each element is actually
+ // *descriptor object, updated atomically.
+ //
+ // Changes to the slice itself requiring holding FDTable.mu.
+ slice unsafe.Pointer `state:".(map[int32]*descriptor)"`
+}
+
+// init initializes the table.
+func (f *FDTable) init() {
+ var slice []unsafe.Pointer // Empty slice.
+ atomic.StorePointer(&f.slice, unsafe.Pointer(&slice))
+}
+
+// get gets a file entry.
+//
+// The boolean indicates whether this was in range.
+//
+//go:nosplit
+func (f *FDTable) get(fd int32) (*fs.File, FDFlags, bool) {
+ slice := *(*[]unsafe.Pointer)(atomic.LoadPointer(&f.slice))
+ if fd >= int32(len(slice)) {
+ return nil, FDFlags{}, false
+ }
+ d := (*descriptor)(atomic.LoadPointer(&slice[fd]))
+ if d == nil {
+ return nil, FDFlags{}, true
+ }
+ return d.file, d.flags, true
+}
+
+// set sets an entry.
+//
+// This handles accounting changes, as well as acquiring and releasing the
+// reference needed by the table iff the file is different.
+//
+// Precondition: mu must be held.
+func (f *FDTable) set(fd int32, file *fs.File, flags FDFlags) {
+ slice := *(*[]unsafe.Pointer)(atomic.LoadPointer(&f.slice))
+
+ // Grow the table as required.
+ if last := int32(len(slice)); fd >= last {
+ end := fd + 1
+ if end < 2*last {
+ end = 2 * last
+ }
+ slice = append(slice, make([]unsafe.Pointer, end-last)...)
+ atomic.StorePointer(&f.slice, unsafe.Pointer(&slice))
+ }
+
+ // Create the new element.
+ var d *descriptor
+ if file != nil {
+ d = &descriptor{
+ file: file,
+ flags: flags,
+ }
+ }
+
+ // Update the single element.
+ orig := (*descriptor)(atomic.SwapPointer(&slice[fd], unsafe.Pointer(d)))
+
+ // Acquire a table reference.
+ if file != nil && (orig == nil || file != orig.file) {
+ file.IncRef()
+ }
+
+ // Drop the table reference.
+ if orig != nil && file != orig.file {
+ f.drop(orig.file)
+ }
+
+ // Adjust used.
+ switch {
+ case orig == nil && file != nil:
+ atomic.AddInt32(&f.used, 1)
+ case orig != nil && file == nil:
+ atomic.AddInt32(&f.used, -1)
+ }
+}
diff --git a/pkg/sentry/kernel/kdefs/BUILD b/pkg/sentry/kernel/kdefs/BUILD
deleted file mode 100644
index 5d62f406a..000000000
--- a/pkg/sentry/kernel/kdefs/BUILD
+++ /dev/null
@@ -1,10 +0,0 @@
-load("//tools/go_stateify:defs.bzl", "go_library")
-
-package(licenses = ["notice"])
-
-go_library(
- name = "kdefs",
- srcs = ["kdefs.go"],
- importpath = "gvisor.dev/gvisor/pkg/sentry/kernel/kdefs",
- visibility = ["//:sandbox"],
-)
diff --git a/pkg/sentry/kernel/kdefs/kdefs.go b/pkg/sentry/kernel/kdefs/kdefs.go
deleted file mode 100644
index 304da2032..000000000
--- a/pkg/sentry/kernel/kdefs/kdefs.go
+++ /dev/null
@@ -1,20 +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 kdefs defines common kernel definitions.
-//
-package kdefs
-
-// FD is a File Descriptor.
-type FD int32
diff --git a/pkg/sentry/kernel/kernel.go b/pkg/sentry/kernel/kernel.go
index 47dadc43a..38b49cba2 100644
--- a/pkg/sentry/kernel/kernel.go
+++ b/pkg/sentry/kernel/kernel.go
@@ -155,7 +155,7 @@ type Kernel struct {
// cpuClockTicker increments cpuClock.
cpuClockTicker *ktime.Timer `state:"nosave"`
- // fdMapUids is an ever-increasing counter for generating FDMap uids.
+ // fdMapUids is an ever-increasing counter for generating FDTable uids.
//
// fdMapUids is mutable, and is accessed using atomic memory operations.
fdMapUids uint64
@@ -400,8 +400,8 @@ func (k *Kernel) flushMountSourceRefs() error {
// There may be some open FDs whose filesystems have been unmounted. We
// must flush those as well.
- return k.tasks.forEachFDPaused(func(desc descriptor) error {
- desc.file.Dirent.Inode.MountSource.FlushDirentRefs()
+ return k.tasks.forEachFDPaused(func(file *fs.File) error {
+ file.Dirent.Inode.MountSource.FlushDirentRefs()
return nil
})
}
@@ -410,35 +410,35 @@ func (k *Kernel) flushMountSourceRefs() error {
// task.
//
// Precondition: Must be called with the kernel paused.
-func (ts *TaskSet) forEachFDPaused(f func(descriptor) error) error {
+func (ts *TaskSet) forEachFDPaused(f func(*fs.File) error) (err error) {
ts.mu.RLock()
defer ts.mu.RUnlock()
for t := range ts.Root.tids {
// We can skip locking Task.mu here since the kernel is paused.
- if t.fds == nil {
+ if t.fdTable == nil {
continue
}
- for _, desc := range t.fds.files {
- if err := f(desc); err != nil {
- return err
+ t.fdTable.forEach(func(_ int32, file *fs.File, _ FDFlags) {
+ if lastErr := f(file); lastErr != nil && err == nil {
+ err = lastErr
}
- }
+ })
}
- return nil
+ return err
}
func (ts *TaskSet) flushWritesToFiles(ctx context.Context) error {
- return ts.forEachFDPaused(func(desc descriptor) error {
- if flags := desc.file.Flags(); !flags.Write {
+ return ts.forEachFDPaused(func(file *fs.File) error {
+ if flags := file.Flags(); !flags.Write {
return nil
}
- if sattr := desc.file.Dirent.Inode.StableAttr; !fs.IsFile(sattr) && !fs.IsDir(sattr) {
+ if sattr := file.Dirent.Inode.StableAttr; !fs.IsFile(sattr) && !fs.IsDir(sattr) {
return nil
}
// Here we need all metadata synced.
- syncErr := desc.file.Fsync(ctx, 0, fs.FileMaxOffset, fs.SyncAll)
+ syncErr := file.Fsync(ctx, 0, fs.FileMaxOffset, fs.SyncAll)
if err := fs.SaveFileFsyncError(syncErr); err != nil {
- name, _ := desc.file.Dirent.FullName(nil /* root */)
+ name, _ := file.Dirent.FullName(nil /* root */)
// Wrap this error in ErrSaveRejection
// so that it will trigger a save
// error, rather than a panic. This
@@ -483,14 +483,12 @@ func (ts *TaskSet) unregisterEpollWaiters() {
defer ts.mu.RUnlock()
for t := range ts.Root.tids {
// We can skip locking Task.mu here since the kernel is paused.
- if fdmap := t.fds; fdmap != nil {
- for _, desc := range fdmap.files {
- if desc.file != nil {
- if e, ok := desc.file.FileOperations.(*epoll.EventPoll); ok {
- e.UnregisterEpollWaiters()
- }
+ if t.fdTable != nil {
+ t.fdTable.forEach(func(_ int32, file *fs.File, _ FDFlags) {
+ if e, ok := file.FileOperations.(*epoll.EventPoll); ok {
+ e.UnregisterEpollWaiters()
}
- }
+ })
}
}
}
@@ -538,6 +536,8 @@ func (k *Kernel) LoadFrom(r io.Reader, net inet.Stack) error {
}
log.Infof("Memory load took [%s].", time.Since(memoryStart))
+ log.Infof("Overall load took [%s]", time.Since(loadStart))
+
// Ensure that all pending asynchronous work is complete:
// - namedpipe opening
// - inode file opening
@@ -602,9 +602,9 @@ type CreateProcessArgs struct {
// Credentials is the initial credentials.
Credentials *auth.Credentials
- // FDMap is the initial set of file descriptors. If CreateProcess succeeds,
- // it takes a reference on FDMap.
- FDMap *FDMap
+ // FDTable is the initial set of file descriptors. If CreateProcess succeeds,
+ // it takes a reference on FDTable.
+ FDTable *FDTable
// Umask is the initial umask.
Umask uint
@@ -789,9 +789,9 @@ func (k *Kernel) CreateProcess(args CreateProcessArgs) (*ThreadGroup, ThreadID,
return nil, 0, errors.New(se.String())
}
- // Take a reference on the FDMap, which will be transferred to
+ // Take a reference on the FDTable, which will be transferred to
// TaskSet.NewTask().
- args.FDMap.IncRef()
+ args.FDTable.IncRef()
// Create the task.
config := &TaskConfig{
@@ -799,7 +799,7 @@ func (k *Kernel) CreateProcess(args CreateProcessArgs) (*ThreadGroup, ThreadID,
ThreadGroup: tg,
TaskContext: tc,
FSContext: newFSContext(root, wd, args.Umask),
- FDMap: args.FDMap,
+ FDTable: args.FDTable,
Credentials: args.Credentials,
AllowedCPUMask: sched.NewFullCPUSet(k.applicationCores),
UTSNamespace: args.UTSNamespace,
@@ -871,7 +871,7 @@ func (k *Kernel) pauseTimeLocked() {
}
// By precondition, nothing else can be interacting with PIDNamespace.tids
- // or FDMap.files, so we can iterate them without synchronization. (We
+ // or FDTable.files, so we can iterate them without synchronization. (We
// can't hold the TaskSet mutex when pausing thread group timers because
// thread group timers call ThreadGroup.SendSignal, which takes the TaskSet
// mutex, while holding the Timer mutex.)
@@ -882,14 +882,14 @@ func (k *Kernel) pauseTimeLocked() {
it.PauseTimer()
}
}
- // This means we'll iterate FDMaps shared by multiple tasks repeatedly,
+ // This means we'll iterate FDTables shared by multiple tasks repeatedly,
// but ktime.Timer.Pause is idempotent so this is harmless.
- if fdm := t.fds; fdm != nil {
- for _, desc := range fdm.files {
- if tfd, ok := desc.file.FileOperations.(*timerfd.TimerOperations); ok {
+ if t.fdTable != nil {
+ t.fdTable.forEach(func(_ int32, file *fs.File, _ FDFlags) {
+ if tfd, ok := file.FileOperations.(*timerfd.TimerOperations); ok {
tfd.PauseTimer()
}
- }
+ })
}
}
k.timekeeper.PauseUpdates()
@@ -914,12 +914,12 @@ func (k *Kernel) resumeTimeLocked() {
it.ResumeTimer()
}
}
- if fdm := t.fds; fdm != nil {
- for _, desc := range fdm.files {
- if tfd, ok := desc.file.FileOperations.(*timerfd.TimerOperations); ok {
+ if t.fdTable != nil {
+ t.fdTable.forEach(func(_ int32, file *fs.File, _ FDFlags) {
+ if tfd, ok := file.FileOperations.(*timerfd.TimerOperations); ok {
tfd.ResumeTimer()
}
- }
+ })
}
}
}
diff --git a/pkg/sentry/kernel/task.go b/pkg/sentry/kernel/task.go
index 2e3a39d3b..e91f82bb3 100644
--- a/pkg/sentry/kernel/task.go
+++ b/pkg/sentry/kernel/task.go
@@ -236,15 +236,15 @@ type Task struct {
// tc is protected by mu, and is owned by the task goroutine.
tc TaskContext
- // fsc is the task's filesystem context.
+ // fsContext is the task's filesystem context.
//
- // fsc is protected by mu, and is owned by the task goroutine.
- fsc *FSContext
+ // fsContext is protected by mu, and is owned by the task goroutine.
+ fsContext *FSContext
- // fds is the task's file descriptor table.
+ // fdTable is the task's file descriptor table.
//
- // fds is protected by mu, and is owned by the task goroutine.
- fds *FDMap
+ // fdTable is protected by mu, and is owned by the task goroutine.
+ fdTable *FDTable
// If vforkParent is not nil, it is the task that created this task with
// vfork() or clone(CLONE_VFORK), and should have its vforkStop ended when
@@ -602,7 +602,7 @@ func (t *Task) Value(key interface{}) interface{} {
case context.CtxThreadGroupID:
return int32(t.ThreadGroup().ID())
case fs.CtxRoot:
- return t.fsc.RootDirectory()
+ return t.fsContext.RootDirectory()
case fs.CtxDirentCacheLimiter:
return t.k.DirentCacheLimiter
case inet.CtxStack:
@@ -668,7 +668,7 @@ func (t *Task) SyscallRestartBlock() SyscallRestartBlock {
func (t *Task) IsChrooted() bool {
realRoot := t.tg.mounts.Root()
defer realRoot.DecRef()
- root := t.fsc.RootDirectory()
+ root := t.fsContext.RootDirectory()
if root != nil {
defer root.DecRef()
}
@@ -689,16 +689,55 @@ func (t *Task) TaskContext() *TaskContext {
// Precondition: The caller must be running on the task goroutine, or t.mu must
// be locked.
func (t *Task) FSContext() *FSContext {
- return t.fsc
+ return t.fsContext
}
-// FDMap returns t's FDMap. FDMap does not take an additional reference on the
-// returned FDMap.
+// FDTable returns t's FDTable. FDMTable does not take an additional reference
+// on the returned FDMap.
//
// Precondition: The caller must be running on the task goroutine, or t.mu must
// be locked.
-func (t *Task) FDMap() *FDMap {
- return t.fds
+func (t *Task) FDTable() *FDTable {
+ return t.fdTable
+}
+
+// GetFile is a convenience wrapper t.FDTable().GetFile.
+//
+// Precondition: same as FDTable.
+func (t *Task) GetFile(fd int32) *fs.File {
+ f, _ := t.fdTable.Get(fd)
+ return f
+}
+
+// NewFDs is a convenience wrapper for t.FDTable().NewFDs.
+//
+// This automatically passes the task as the context.
+//
+// Precondition: same as FDTable.
+func (t *Task) NewFDs(fd int32, files []*fs.File, flags FDFlags) ([]int32, error) {
+ return t.fdTable.NewFDs(t, fd, files, flags)
+}
+
+// NewFDFrom is a convenience wrapper for t.FDTable().NewFDs with a single file.
+//
+// This automatically passes the task as the context.
+//
+// Precondition: same as FDTable.
+func (t *Task) NewFDFrom(fd int32, file *fs.File, flags FDFlags) (int32, error) {
+ fds, err := t.fdTable.NewFDs(t, fd, []*fs.File{file}, flags)
+ if err != nil {
+ return 0, err
+ }
+ return fds[0], nil
+}
+
+// NewFDAt is a convenience wrapper for t.FDTable().NewFDAt.
+//
+// This automatically passes the task as the context.
+//
+// Precondition: same as FDTable.
+func (t *Task) NewFDAt(fd int32, file *fs.File, flags FDFlags) error {
+ return t.fdTable.NewFDAt(t, fd, file, flags)
}
// WithMuLocked executes f with t.mu locked.
diff --git a/pkg/sentry/kernel/task_clone.go b/pkg/sentry/kernel/task_clone.go
index b5cc3860d..0916fd658 100644
--- a/pkg/sentry/kernel/task_clone.go
+++ b/pkg/sentry/kernel/task_clone.go
@@ -214,20 +214,20 @@ func (t *Task) Clone(opts *CloneOptions) (ThreadID, *SyscallControl, error) {
}
}
- var fsc *FSContext
+ var fsContext *FSContext
if opts.NewFSContext {
- fsc = t.fsc.Fork()
+ fsContext = t.fsContext.Fork()
} else {
- fsc = t.fsc
- fsc.IncRef()
+ fsContext = t.fsContext
+ fsContext.IncRef()
}
- var fds *FDMap
+ var fdTable *FDTable
if opts.NewFiles {
- fds = t.fds.Fork()
+ fdTable = t.fdTable.Fork()
} else {
- fds = t.fds
- fds.IncRef()
+ fdTable = t.fdTable
+ fdTable.IncRef()
}
pidns := t.tg.pidns
@@ -251,8 +251,8 @@ func (t *Task) Clone(opts *CloneOptions) (ThreadID, *SyscallControl, error) {
ThreadGroup: tg,
SignalMask: t.SignalMask(),
TaskContext: tc,
- FSContext: fsc,
- FDMap: fds,
+ FSContext: fsContext,
+ FDTable: fdTable,
Credentials: creds,
Niceness: t.Niceness(),
NetworkNamespaced: t.netns,
@@ -485,22 +485,22 @@ func (t *Task) Unshare(opts *SharingOptions) error {
// namespace"
t.ipcns = NewIPCNamespace(creds.UserNamespace)
}
- var oldfds *FDMap
+ var oldFDTable *FDTable
if opts.NewFiles {
- oldfds = t.fds
- t.fds = oldfds.Fork()
+ oldFDTable = t.fdTable
+ t.fdTable = oldFDTable.Fork()
}
- var oldfsc *FSContext
+ var oldFSContext *FSContext
if opts.NewFSContext {
- oldfsc = t.fsc
- t.fsc = oldfsc.Fork()
+ oldFSContext = t.fsContext
+ t.fsContext = oldFSContext.Fork()
}
t.mu.Unlock()
- if oldfds != nil {
- oldfds.DecRef()
+ if oldFDTable != nil {
+ oldFDTable.DecRef()
}
- if oldfsc != nil {
- oldfsc.DecRef()
+ if oldFSContext != nil {
+ oldFSContext.DecRef()
}
return nil
}
diff --git a/pkg/sentry/kernel/task_exec.go b/pkg/sentry/kernel/task_exec.go
index cd85acaef..17a089b90 100644
--- a/pkg/sentry/kernel/task_exec.go
+++ b/pkg/sentry/kernel/task_exec.go
@@ -195,7 +195,7 @@ func (r *runSyscallAfterExecStop) execute(t *Task) taskRunState {
t.tg.pidns.owner.mu.Unlock()
// Remove FDs with the CloseOnExec flag set.
- t.fds.RemoveIf(func(file *fs.File, flags FDFlags) bool {
+ t.fdTable.RemoveIf(func(file *fs.File, flags FDFlags) bool {
return flags.CloseOnExec
})
diff --git a/pkg/sentry/kernel/task_exit.go b/pkg/sentry/kernel/task_exit.go
index b97d65185..535f03e50 100644
--- a/pkg/sentry/kernel/task_exit.go
+++ b/pkg/sentry/kernel/task_exit.go
@@ -265,8 +265,8 @@ func (*runExitMain) execute(t *Task) taskRunState {
// Releasing the MM unblocks a blocked CLONE_VFORK parent.
t.unstopVforkParent()
- t.fsc.DecRef()
- t.fds.DecRef()
+ t.fsContext.DecRef()
+ t.fdTable.DecRef()
// If this is the last task to exit from the thread group, release the
// thread group's resources.
diff --git a/pkg/sentry/kernel/task_log.go b/pkg/sentry/kernel/task_log.go
index cf48663b6..a29e9b9eb 100644
--- a/pkg/sentry/kernel/task_log.go
+++ b/pkg/sentry/kernel/task_log.go
@@ -63,7 +63,7 @@ func (t *Task) DebugDumpState() {
if mm := t.MemoryManager(); mm != nil {
t.Debugf("Mappings:\n%s", mm)
}
- t.Debugf("FDMap:\n%s", t.fds)
+ t.Debugf("FDTable:\n%s", t.fdTable)
}
// debugDumpRegisters logs register state at log level debug.
diff --git a/pkg/sentry/kernel/task_start.go b/pkg/sentry/kernel/task_start.go
index 72caae537..a88bf3951 100644
--- a/pkg/sentry/kernel/task_start.go
+++ b/pkg/sentry/kernel/task_start.go
@@ -52,9 +52,10 @@ type TaskConfig struct {
// succeeds.
FSContext *FSContext
- // FDMap is the FDMap of the new task. A reference must be held on FDMap,
- // which is transferred to TaskSet.NewTask whether or not it succeeds.
- FDMap *FDMap
+ // FDTable is the FDTableof the new task. A reference must be held on
+ // FDMap, which is transferred to TaskSet.NewTask whether or not it
+ // succeeds.
+ FDTable *FDTable
// Credentials is the Credentials of the new task.
Credentials *auth.Credentials
@@ -90,7 +91,7 @@ func (ts *TaskSet) NewTask(cfg *TaskConfig) (*Task, error) {
if err != nil {
cfg.TaskContext.release()
cfg.FSContext.DecRef()
- cfg.FDMap.DecRef()
+ cfg.FDTable.DecRef()
return nil, err
}
return t, nil
@@ -112,8 +113,8 @@ func (ts *TaskSet) newTask(cfg *TaskConfig) (*Task, error) {
signalMask: cfg.SignalMask,
signalStack: arch.SignalStack{Flags: arch.SignalStackFlagDisable},
tc: *tc,
- fsc: cfg.FSContext,
- fds: cfg.FDMap,
+ fsContext: cfg.FSContext,
+ fdTable: cfg.FDTable,
p: cfg.Kernel.Platform.NewContext(),
k: cfg.Kernel,
ptraceTracees: make(map[*Task]struct{}),