summaryrefslogtreecommitdiffhomepage
path: root/pkg/sentry
diff options
context:
space:
mode:
authorgVisor bot <gvisor-bot@google.com>2020-05-04 18:04:29 +0000
committergVisor bot <gvisor-bot@google.com>2020-05-04 18:04:29 +0000
commit23c3b58f2d7ade5f9aa1e8c545223941ded690e4 (patch)
treef1a0dfdd7f921920e7e80ced792fd1b7a532752f /pkg/sentry
parent95cb42d3ad61bc1827f6be769ea66981414d5527 (diff)
parentcbc5bef2a66ece1f9e63b213d4dfa616db488df8 (diff)
Merge release-20200422.0-16-gcbc5bef (automated)
Diffstat (limited to 'pkg/sentry')
-rw-r--r--pkg/sentry/control/proc.go98
-rwxr-xr-xpkg/sentry/fdimport/fdimport.go129
-rwxr-xr-xpkg/sentry/fdimport/fdimport_state_autogen.go3
-rwxr-xr-xpkg/sentry/fsimpl/devpts/devpts.go209
-rwxr-xr-xpkg/sentry/fsimpl/devpts/devpts_state_autogen.go94
-rwxr-xr-xpkg/sentry/fsimpl/devpts/line_discipline.go449
-rwxr-xr-xpkg/sentry/fsimpl/devpts/master.go226
-rwxr-xr-xpkg/sentry/fsimpl/devpts/queue.go240
-rwxr-xr-xpkg/sentry/fsimpl/devpts/slave.go186
-rwxr-xr-xpkg/sentry/fsimpl/devpts/terminal.go124
-rwxr-xr-xpkg/sentry/fsimpl/host/host.go2
-rwxr-xr-xpkg/sentry/fsimpl/host/tty.go26
-rwxr-xr-xpkg/sentry/vfs/context.go24
-rwxr-xr-xpkg/sentry/vfs/epoll.go1
14 files changed, 1736 insertions, 75 deletions
diff --git a/pkg/sentry/control/proc.go b/pkg/sentry/control/proc.go
index b51fb3959..2ed17ee09 100644
--- a/pkg/sentry/control/proc.go
+++ b/pkg/sentry/control/proc.go
@@ -24,13 +24,16 @@ import (
"text/tabwriter"
"time"
+ "golang.org/x/sys/unix"
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
"gvisor.dev/gvisor/pkg/fspath"
"gvisor.dev/gvisor/pkg/log"
+ "gvisor.dev/gvisor/pkg/sentry/fdimport"
"gvisor.dev/gvisor/pkg/sentry/fs"
"gvisor.dev/gvisor/pkg/sentry/fs/host"
"gvisor.dev/gvisor/pkg/sentry/fsbridge"
+ hostvfs2 "gvisor.dev/gvisor/pkg/sentry/fsimpl/host"
"gvisor.dev/gvisor/pkg/sentry/kernel"
"gvisor.dev/gvisor/pkg/sentry/kernel/auth"
ktime "gvisor.dev/gvisor/pkg/sentry/kernel/time"
@@ -84,15 +87,13 @@ type ExecArgs struct {
// the root group if not set explicitly.
KGID auth.KGID
- // ExtraKGIDs is the list of additional groups to which the user
- // belongs.
+ // ExtraKGIDs is the list of additional groups to which the user belongs.
ExtraKGIDs []auth.KGID
// Capabilities is the list of capabilities to give to the process.
Capabilities *auth.TaskCapabilities
- // StdioIsPty indicates that FDs 0, 1, and 2 are connected to a host
- // pty FD.
+ // StdioIsPty indicates that FDs 0, 1, and 2 are connected to a host pty FD.
StdioIsPty bool
// FilePayload determines the files to give to the new process.
@@ -117,7 +118,7 @@ func (args ExecArgs) String() string {
// Exec runs a new task.
func (proc *Proc) Exec(args *ExecArgs, waitStatus *uint32) error {
- newTG, _, _, err := proc.execAsync(args)
+ newTG, _, _, _, err := proc.execAsync(args)
if err != nil {
return err
}
@@ -130,26 +131,18 @@ func (proc *Proc) Exec(args *ExecArgs, waitStatus *uint32) error {
// ExecAsync runs a new task, but doesn't wait for it to finish. It is defined
// as a function rather than a method to avoid exposing execAsync as an RPC.
-func ExecAsync(proc *Proc, args *ExecArgs) (*kernel.ThreadGroup, kernel.ThreadID, *host.TTYFileOperations, error) {
+func ExecAsync(proc *Proc, args *ExecArgs) (*kernel.ThreadGroup, kernel.ThreadID, *host.TTYFileOperations, *hostvfs2.TTYFileDescription, error) {
return proc.execAsync(args)
}
// execAsync runs a new task, but doesn't wait for it to finish. It returns the
// newly created thread group and its PID. If the stdio FDs are TTYs, then a
// TTYFileOperations that wraps the TTY is also returned.
-func (proc *Proc) execAsync(args *ExecArgs) (*kernel.ThreadGroup, kernel.ThreadID, *host.TTYFileOperations, error) {
+func (proc *Proc) execAsync(args *ExecArgs) (*kernel.ThreadGroup, kernel.ThreadID, *host.TTYFileOperations, *hostvfs2.TTYFileDescription, error) {
// Import file descriptors.
fdTable := proc.Kernel.NewFDTable()
defer fdTable.DecRef()
- // No matter what happens, we should close all files in the FilePayload
- // before returning. Any files that are imported will be duped.
- defer func() {
- for _, f := range args.FilePayload.Files {
- f.Close()
- }
- }()
-
creds := auth.NewUserCredentials(
args.KUID,
args.KGID,
@@ -202,7 +195,7 @@ func (proc *Proc) execAsync(args *ExecArgs) (*kernel.ThreadGroup, kernel.ThreadI
vfsObj := proc.Kernel.VFS()
file, err := ResolveExecutablePath(ctx, vfsObj, initArgs.WorkingDirectory, initArgs.Argv[0], paths)
if err != nil {
- return nil, 0, nil, fmt.Errorf("error finding executable %q in PATH %v: %v", initArgs.Argv[0], paths, err)
+ return nil, 0, nil, nil, fmt.Errorf("error finding executable %q in PATH %v: %v", initArgs.Argv[0], paths, err)
}
initArgs.File = fsbridge.NewVFSFile(file)
} else {
@@ -214,74 +207,57 @@ func (proc *Proc) execAsync(args *ExecArgs) (*kernel.ThreadGroup, kernel.ThreadI
// initArgs must hold a reference on MountNamespace, which will
// be donated to the new process in CreateProcess.
- initArgs.MountNamespaceVFS2.IncRef()
+ initArgs.MountNamespace.IncRef()
}
f, err := initArgs.MountNamespace.ResolveExecutablePath(ctx, initArgs.WorkingDirectory, initArgs.Argv[0], paths)
if err != nil {
- return nil, 0, nil, fmt.Errorf("error finding executable %q in PATH %v: %v", initArgs.Argv[0], paths, err)
+ return nil, 0, nil, nil, fmt.Errorf("error finding executable %q in PATH %v: %v", initArgs.Argv[0], paths, err)
}
initArgs.Filename = f
}
}
- // TODO(gvisor.dev/issue/1623): Use host FD when supported in VFS2.
- var ttyFile *fs.File
- for appFD, hostFile := range args.FilePayload.Files {
- var appFile *fs.File
-
- if args.StdioIsPty && appFD < 3 {
- // Import the file as a host TTY file.
- if ttyFile == nil {
- var err error
- appFile, err = host.ImportFile(ctx, int(hostFile.Fd()), true /* isTTY */)
- if err != nil {
- return nil, 0, nil, err
- }
- defer appFile.DecRef()
-
- // Remember this in the TTY file, as we will
- // use it for the other stdio FDs.
- ttyFile = appFile
- } else {
- // Re-use the existing TTY file, as all three
- // stdio FDs must point to the same fs.File in
- // order to share TTY state, specifically the
- // foreground process group id.
- appFile = ttyFile
- }
- } else {
- // Import the file as a regular host file.
- var err error
- appFile, err = host.ImportFile(ctx, int(hostFile.Fd()), false /* isTTY */)
+ fds := make([]int, len(args.FilePayload.Files))
+ for i, file := range args.FilePayload.Files {
+ if kernel.VFS2Enabled {
+ // Need to dup to remove ownership from os.File.
+ dup, err := unix.Dup(int(file.Fd()))
if err != nil {
- return nil, 0, nil, err
+ return nil, 0, nil, nil, fmt.Errorf("duplicating payload files: %w", err)
}
- defer appFile.DecRef()
+ fds[i] = dup
+ } else {
+ // VFS1 dups the file on import.
+ fds[i] = int(file.Fd())
}
-
- // Add the file to the FD map.
- if err := fdTable.NewFDAt(ctx, int32(appFD), appFile, kernel.FDFlags{}); err != nil {
- return nil, 0, nil, err
+ }
+ ttyFile, ttyFileVFS2, err := fdimport.Import(ctx, fdTable, args.StdioIsPty, fds)
+ if err != nil {
+ if kernel.VFS2Enabled {
+ for _, fd := range fds {
+ unix.Close(fd)
+ }
}
+ return nil, 0, nil, nil, err
}
tg, tid, err := proc.Kernel.CreateProcess(initArgs)
if err != nil {
- return nil, 0, nil, err
+ return nil, 0, nil, nil, err
}
- var ttyFileOps *host.TTYFileOperations
- if ttyFile != nil {
- // Set the foreground process group on the TTY before starting
- // the process.
- ttyFileOps = ttyFile.FileOperations.(*host.TTYFileOperations)
- ttyFileOps.InitForegroundProcessGroup(tg.ProcessGroup())
+ // Set the foreground process group on the TTY before starting the process.
+ switch {
+ case ttyFile != nil:
+ ttyFile.InitForegroundProcessGroup(tg.ProcessGroup())
+ case ttyFileVFS2 != nil:
+ ttyFileVFS2.InitForegroundProcessGroup(tg.ProcessGroup())
}
// Start the newly created process.
proc.Kernel.StartProcess(tg)
- return tg, tid, ttyFileOps, nil
+ return tg, tid, ttyFile, ttyFileVFS2, nil
}
// PsArgs is the set of arguments to ps.
diff --git a/pkg/sentry/fdimport/fdimport.go b/pkg/sentry/fdimport/fdimport.go
new file mode 100755
index 000000000..a4199f9e9
--- /dev/null
+++ b/pkg/sentry/fdimport/fdimport.go
@@ -0,0 +1,129 @@
+// Copyright 2020 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 fdimport
+
+import (
+ "gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/sentry/fs"
+ "gvisor.dev/gvisor/pkg/sentry/fs/host"
+ hostvfs2 "gvisor.dev/gvisor/pkg/sentry/fsimpl/host"
+ "gvisor.dev/gvisor/pkg/sentry/kernel"
+ "gvisor.dev/gvisor/pkg/sentry/vfs"
+)
+
+// Import imports a slice of FDs into the given FDTable. If console is true,
+// sets up TTY for the first 3 FDs in the slice representing stdin, stdout,
+// stderr. Upon success, Import takes ownership of all FDs.
+func Import(ctx context.Context, fdTable *kernel.FDTable, console bool, fds []int) (*host.TTYFileOperations, *hostvfs2.TTYFileDescription, error) {
+ if kernel.VFS2Enabled {
+ ttyFile, err := importVFS2(ctx, fdTable, console, fds)
+ return nil, ttyFile, err
+ }
+ ttyFile, err := importFS(ctx, fdTable, console, fds)
+ return ttyFile, nil, err
+}
+
+func importFS(ctx context.Context, fdTable *kernel.FDTable, console bool, fds []int) (*host.TTYFileOperations, error) {
+ var ttyFile *fs.File
+ for appFD, hostFD := range fds {
+ var appFile *fs.File
+
+ if console && appFD < 3 {
+ // Import the file as a host TTY file.
+ if ttyFile == nil {
+ var err error
+ appFile, err = host.ImportFile(ctx, hostFD, true /* isTTY */)
+ if err != nil {
+ return nil, err
+ }
+ defer appFile.DecRef()
+
+ // Remember this in the TTY file, as we will
+ // use it for the other stdio FDs.
+ ttyFile = appFile
+ } else {
+ // Re-use the existing TTY file, as all three
+ // stdio FDs must point to the same fs.File in
+ // order to share TTY state, specifically the
+ // foreground process group id.
+ appFile = ttyFile
+ }
+ } else {
+ // Import the file as a regular host file.
+ var err error
+ appFile, err = host.ImportFile(ctx, hostFD, false /* isTTY */)
+ if err != nil {
+ return nil, err
+ }
+ defer appFile.DecRef()
+ }
+
+ // Add the file to the FD map.
+ if err := fdTable.NewFDAt(ctx, int32(appFD), appFile, kernel.FDFlags{}); err != nil {
+ return nil, err
+ }
+ }
+
+ if ttyFile == nil {
+ return nil, nil
+ }
+ return ttyFile.FileOperations.(*host.TTYFileOperations), nil
+}
+
+func importVFS2(ctx context.Context, fdTable *kernel.FDTable, console bool, stdioFDs []int) (*hostvfs2.TTYFileDescription, error) {
+ k := kernel.KernelFromContext(ctx)
+
+ var ttyFile *vfs.FileDescription
+ for appFD, hostFD := range stdioFDs {
+ var appFile *vfs.FileDescription
+
+ if console && appFD < 3 {
+ // Import the file as a host TTY file.
+ if ttyFile == nil {
+ var err error
+ appFile, err = hostvfs2.ImportFD(ctx, k.HostMount(), hostFD, true /* isTTY */)
+ if err != nil {
+ return nil, err
+ }
+ defer appFile.DecRef()
+
+ // Remember this in the TTY file, as we will use it for the other stdio
+ // FDs.
+ ttyFile = appFile
+ } else {
+ // Re-use the existing TTY file, as all three stdio FDs must point to
+ // the same fs.File in order to share TTY state, specifically the
+ // foreground process group id.
+ appFile = ttyFile
+ }
+ } else {
+ var err error
+ appFile, err = hostvfs2.ImportFD(ctx, k.HostMount(), hostFD, false /* isTTY */)
+ if err != nil {
+ return nil, err
+ }
+ defer appFile.DecRef()
+ }
+
+ if err := fdTable.NewFDAtVFS2(ctx, int32(appFD), appFile, kernel.FDFlags{}); err != nil {
+ return nil, err
+ }
+ }
+
+ if ttyFile == nil {
+ return nil, nil
+ }
+ return ttyFile.Impl().(*hostvfs2.TTYFileDescription), nil
+}
diff --git a/pkg/sentry/fdimport/fdimport_state_autogen.go b/pkg/sentry/fdimport/fdimport_state_autogen.go
new file mode 100755
index 000000000..fbb89b276
--- /dev/null
+++ b/pkg/sentry/fdimport/fdimport_state_autogen.go
@@ -0,0 +1,3 @@
+// automatically generated by stateify.
+
+package fdimport
diff --git a/pkg/sentry/fsimpl/devpts/devpts.go b/pkg/sentry/fsimpl/devpts/devpts.go
new file mode 100755
index 000000000..181d765d3
--- /dev/null
+++ b/pkg/sentry/fsimpl/devpts/devpts.go
@@ -0,0 +1,209 @@
+// Copyright 2020 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 devpts provides a filesystem implementation that behaves like
+// devpts.
+package devpts
+
+import (
+ "fmt"
+ "math"
+ "sort"
+ "strconv"
+ "sync"
+
+ "gvisor.dev/gvisor/pkg/abi/linux"
+ "gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/sentry/fsimpl/kernfs"
+ "gvisor.dev/gvisor/pkg/sentry/kernel/auth"
+ "gvisor.dev/gvisor/pkg/sentry/vfs"
+ "gvisor.dev/gvisor/pkg/syserror"
+)
+
+// Name is the filesystem name.
+const Name = "devpts"
+
+// FilesystemType implements vfs.FilesystemType.
+type FilesystemType struct{}
+
+// Name implements vfs.FilesystemType.Name.
+func (FilesystemType) Name() string {
+ return Name
+}
+
+var _ vfs.FilesystemType = (*FilesystemType)(nil)
+
+// GetFilesystem implements vfs.FilesystemType.GetFilesystem.
+func (fstype FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.VirtualFilesystem, creds *auth.Credentials, source string, opts vfs.GetFilesystemOptions) (*vfs.Filesystem, *vfs.Dentry, error) {
+ // No data allowed.
+ if opts.Data != "" {
+ return nil, nil, syserror.EINVAL
+ }
+
+ fs, root := fstype.newFilesystem(vfsObj, creds)
+ return fs.VFSFilesystem(), root.VFSDentry(), nil
+}
+
+// newFilesystem creates a new devpts filesystem with root directory and ptmx
+// master inode. It returns the filesystem and root Dentry.
+func (fstype FilesystemType) newFilesystem(vfsObj *vfs.VirtualFilesystem, creds *auth.Credentials) (*kernfs.Filesystem, *kernfs.Dentry) {
+ fs := &kernfs.Filesystem{}
+ fs.Init(vfsObj, fstype)
+
+ // Construct the root directory. This is always inode id 1.
+ root := &rootInode{
+ slaves: make(map[uint32]*slaveInode),
+ }
+ root.InodeAttrs.Init(creds, 1, linux.ModeDirectory|0555)
+ root.OrderedChildren.Init(kernfs.OrderedChildrenOptions{})
+ root.dentry.Init(root)
+
+ // Construct the pts master inode and dentry. Linux always uses inode
+ // id 2 for ptmx. See fs/devpts/inode.c:mknod_ptmx.
+ master := &masterInode{
+ root: root,
+ }
+ master.InodeAttrs.Init(creds, 2, linux.ModeCharacterDevice|0666)
+ master.dentry.Init(master)
+
+ // Add the master as a child of the root.
+ links := root.OrderedChildren.Populate(&root.dentry, map[string]*kernfs.Dentry{
+ "ptmx": &master.dentry,
+ })
+ root.IncLinks(links)
+
+ return fs, &root.dentry
+}
+
+// rootInode is the root directory inode for the devpts mounts.
+type rootInode struct {
+ kernfs.AlwaysValid
+ kernfs.InodeAttrs
+ kernfs.InodeDirectoryNoNewChildren
+ kernfs.InodeNotSymlink
+ kernfs.OrderedChildren
+
+ // Keep a reference to this inode's dentry.
+ dentry kernfs.Dentry
+
+ // master is the master pty inode. Immutable.
+ master *masterInode
+
+ // root is the root directory inode for this filesystem. Immutable.
+ root *rootInode
+
+ // mu protects the fields below.
+ mu sync.Mutex
+
+ // slaves maps pty ids to slave inodes.
+ slaves map[uint32]*slaveInode
+
+ // nextIdx is the next pty index to use. Must be accessed atomically.
+ //
+ // TODO(b/29356795): reuse indices when ptys are closed.
+ nextIdx uint32
+}
+
+var _ kernfs.Inode = (*rootInode)(nil)
+
+// allocateTerminal creates a new Terminal and installs a pts node for it.
+func (i *rootInode) allocateTerminal(creds *auth.Credentials) (*Terminal, error) {
+ i.mu.Lock()
+ defer i.mu.Unlock()
+ if i.nextIdx == math.MaxUint32 {
+ return nil, syserror.ENOMEM
+ }
+ idx := i.nextIdx
+ i.nextIdx++
+
+ // Sanity check that slave with idx does not exist.
+ if _, ok := i.slaves[idx]; ok {
+ panic(fmt.Sprintf("pty index collision; index %d already exists", idx))
+ }
+
+ // Create the new terminal and slave.
+ t := newTerminal(idx)
+ slave := &slaveInode{
+ root: i,
+ t: t,
+ }
+ // Linux always uses pty index + 3 as the inode id. See
+ // fs/devpts/inode.c:devpts_pty_new().
+ slave.InodeAttrs.Init(creds, uint64(idx+3), linux.ModeCharacterDevice|0600)
+ slave.dentry.Init(slave)
+ i.slaves[idx] = slave
+
+ return t, nil
+}
+
+// masterClose is called when the master end of t is closed.
+func (i *rootInode) masterClose(t *Terminal) {
+ i.mu.Lock()
+ defer i.mu.Unlock()
+
+ // Sanity check that slave with idx exists.
+ if _, ok := i.slaves[t.n]; !ok {
+ panic(fmt.Sprintf("pty with index %d does not exist", t.n))
+ }
+ delete(i.slaves, t.n)
+}
+
+// Open implements kernfs.Inode.Open.
+func (i *rootInode) Open(ctx context.Context, rp *vfs.ResolvingPath, vfsd *vfs.Dentry, opts vfs.OpenOptions) (*vfs.FileDescription, error) {
+ fd, err := kernfs.NewGenericDirectoryFD(rp.Mount(), vfsd, &i.OrderedChildren, &opts)
+ if err != nil {
+ return nil, err
+ }
+ return fd.VFSFileDescription(), nil
+}
+
+// Lookup implements kernfs.Inode.Lookup.
+func (i *rootInode) Lookup(ctx context.Context, name string) (*vfs.Dentry, error) {
+ idx, err := strconv.ParseUint(name, 10, 32)
+ if err != nil {
+ return nil, syserror.ENOENT
+ }
+ i.mu.Lock()
+ defer i.mu.Unlock()
+ if si, ok := i.slaves[uint32(idx)]; ok {
+ si.dentry.IncRef()
+ return si.dentry.VFSDentry(), nil
+
+ }
+ return nil, syserror.ENOENT
+}
+
+// IterDirents implements kernfs.Inode.IterDirents.
+func (i *rootInode) IterDirents(ctx context.Context, cb vfs.IterDirentsCallback, offset, relOffset int64) (int64, error) {
+ i.mu.Lock()
+ defer i.mu.Unlock()
+ ids := make([]int, 0, len(i.slaves))
+ for id := range i.slaves {
+ ids = append(ids, int(id))
+ }
+ sort.Ints(ids)
+ for _, id := range ids[relOffset:] {
+ dirent := vfs.Dirent{
+ Name: strconv.FormatUint(uint64(id), 10),
+ Type: linux.DT_CHR,
+ Ino: i.slaves[uint32(id)].InodeAttrs.Ino(),
+ NextOff: offset + 1,
+ }
+ if err := cb.Handle(dirent); err != nil {
+ return offset, err
+ }
+ offset++
+ }
+ return offset, nil
+}
diff --git a/pkg/sentry/fsimpl/devpts/devpts_state_autogen.go b/pkg/sentry/fsimpl/devpts/devpts_state_autogen.go
new file mode 100755
index 000000000..90cd7937c
--- /dev/null
+++ b/pkg/sentry/fsimpl/devpts/devpts_state_autogen.go
@@ -0,0 +1,94 @@
+// automatically generated by stateify.
+
+package devpts
+
+import (
+ "gvisor.dev/gvisor/pkg/state"
+)
+
+func (x *lineDiscipline) beforeSave() {}
+func (x *lineDiscipline) save(m state.Map) {
+ x.beforeSave()
+ if !state.IsZeroValue(&x.masterWaiter) {
+ m.Failf("masterWaiter is %#v, expected zero", &x.masterWaiter)
+ }
+ if !state.IsZeroValue(&x.slaveWaiter) {
+ m.Failf("slaveWaiter is %#v, expected zero", &x.slaveWaiter)
+ }
+ m.Save("size", &x.size)
+ m.Save("inQueue", &x.inQueue)
+ m.Save("outQueue", &x.outQueue)
+ m.Save("termios", &x.termios)
+ m.Save("column", &x.column)
+}
+
+func (x *lineDiscipline) afterLoad() {}
+func (x *lineDiscipline) load(m state.Map) {
+ m.Load("size", &x.size)
+ m.Load("inQueue", &x.inQueue)
+ m.Load("outQueue", &x.outQueue)
+ m.Load("termios", &x.termios)
+ m.Load("column", &x.column)
+}
+
+func (x *outputQueueTransformer) beforeSave() {}
+func (x *outputQueueTransformer) save(m state.Map) {
+ x.beforeSave()
+}
+
+func (x *outputQueueTransformer) afterLoad() {}
+func (x *outputQueueTransformer) load(m state.Map) {
+}
+
+func (x *inputQueueTransformer) beforeSave() {}
+func (x *inputQueueTransformer) save(m state.Map) {
+ x.beforeSave()
+}
+
+func (x *inputQueueTransformer) afterLoad() {}
+func (x *inputQueueTransformer) load(m state.Map) {
+}
+
+func (x *queue) beforeSave() {}
+func (x *queue) save(m state.Map) {
+ x.beforeSave()
+ m.Save("readBuf", &x.readBuf)
+ m.Save("waitBuf", &x.waitBuf)
+ m.Save("waitBufLen", &x.waitBufLen)
+ m.Save("readable", &x.readable)
+ m.Save("transformer", &x.transformer)
+}
+
+func (x *queue) afterLoad() {}
+func (x *queue) load(m state.Map) {
+ m.Load("readBuf", &x.readBuf)
+ m.Load("waitBuf", &x.waitBuf)
+ m.Load("waitBufLen", &x.waitBufLen)
+ m.Load("readable", &x.readable)
+ m.Load("transformer", &x.transformer)
+}
+
+func (x *Terminal) beforeSave() {}
+func (x *Terminal) save(m state.Map) {
+ x.beforeSave()
+ m.Save("n", &x.n)
+ m.Save("ld", &x.ld)
+ m.Save("masterKTTY", &x.masterKTTY)
+ m.Save("slaveKTTY", &x.slaveKTTY)
+}
+
+func (x *Terminal) afterLoad() {}
+func (x *Terminal) load(m state.Map) {
+ m.Load("n", &x.n)
+ m.Load("ld", &x.ld)
+ m.Load("masterKTTY", &x.masterKTTY)
+ m.Load("slaveKTTY", &x.slaveKTTY)
+}
+
+func init() {
+ state.Register("pkg/sentry/fsimpl/devpts.lineDiscipline", (*lineDiscipline)(nil), state.Fns{Save: (*lineDiscipline).save, Load: (*lineDiscipline).load})
+ state.Register("pkg/sentry/fsimpl/devpts.outputQueueTransformer", (*outputQueueTransformer)(nil), state.Fns{Save: (*outputQueueTransformer).save, Load: (*outputQueueTransformer).load})
+ state.Register("pkg/sentry/fsimpl/devpts.inputQueueTransformer", (*inputQueueTransformer)(nil), state.Fns{Save: (*inputQueueTransformer).save, Load: (*inputQueueTransformer).load})
+ state.Register("pkg/sentry/fsimpl/devpts.queue", (*queue)(nil), state.Fns{Save: (*queue).save, Load: (*queue).load})
+ state.Register("pkg/sentry/fsimpl/devpts.Terminal", (*Terminal)(nil), state.Fns{Save: (*Terminal).save, Load: (*Terminal).load})
+}
diff --git a/pkg/sentry/fsimpl/devpts/line_discipline.go b/pkg/sentry/fsimpl/devpts/line_discipline.go
new file mode 100755
index 000000000..e201801d6
--- /dev/null
+++ b/pkg/sentry/fsimpl/devpts/line_discipline.go
@@ -0,0 +1,449 @@
+// 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 devpts
+
+import (
+ "bytes"
+ "unicode/utf8"
+
+ "gvisor.dev/gvisor/pkg/abi/linux"
+ "gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/sentry/arch"
+ "gvisor.dev/gvisor/pkg/sync"
+ "gvisor.dev/gvisor/pkg/syserror"
+ "gvisor.dev/gvisor/pkg/usermem"
+ "gvisor.dev/gvisor/pkg/waiter"
+)
+
+// LINT.IfChange
+
+const (
+ // canonMaxBytes is the number of bytes that fit into a single line of
+ // terminal input in canonical mode. This corresponds to N_TTY_BUF_SIZE
+ // in include/linux/tty.h.
+ canonMaxBytes = 4096
+
+ // nonCanonMaxBytes is the maximum number of bytes that can be read at
+ // a time in noncanonical mode.
+ nonCanonMaxBytes = canonMaxBytes - 1
+
+ spacesPerTab = 8
+)
+
+// lineDiscipline dictates how input and output are handled between the
+// pseudoterminal (pty) master and slave. It can be configured to alter I/O,
+// modify control characters (e.g. Ctrl-C for SIGINT), etc. The following man
+// pages are good resources for how to affect the line discipline:
+//
+// * termios(3)
+// * tty_ioctl(4)
+//
+// This file corresponds most closely to drivers/tty/n_tty.c.
+//
+// lineDiscipline has a simple structure but supports a multitude of options
+// (see the above man pages). It consists of two queues of bytes: one from the
+// terminal master to slave (the input queue) and one from slave to master (the
+// output queue). When bytes are written to one end of the pty, the line
+// discipline reads the bytes, modifies them or takes special action if
+// required, and enqueues them to be read by the other end of the pty:
+//
+// input from terminal +-------------+ input to process (e.g. bash)
+// +------------------------>| input queue |---------------------------+
+// | (inputQueueWrite) +-------------+ (inputQueueRead) |
+// | |
+// | v
+// masterFD slaveFD
+// ^ |
+// | |
+// | output to terminal +--------------+ output from process |
+// +------------------------| output queue |<--------------------------+
+// (outputQueueRead) +--------------+ (outputQueueWrite)
+//
+// Lock order:
+// termiosMu
+// inQueue.mu
+// outQueue.mu
+//
+// +stateify savable
+type lineDiscipline struct {
+ // sizeMu protects size.
+ sizeMu sync.Mutex `state:"nosave"`
+
+ // size is the terminal size (width and height).
+ size linux.WindowSize
+
+ // inQueue is the input queue of the terminal.
+ inQueue queue
+
+ // outQueue is the output queue of the terminal.
+ outQueue queue
+
+ // termiosMu protects termios.
+ termiosMu sync.RWMutex `state:"nosave"`
+
+ // termios is the terminal configuration used by the lineDiscipline.
+ termios linux.KernelTermios
+
+ // column is the location in a row of the cursor. This is important for
+ // handling certain special characters like backspace.
+ column int
+
+ // masterWaiter is used to wait on the master end of the TTY.
+ masterWaiter waiter.Queue `state:"zerovalue"`
+
+ // slaveWaiter is used to wait on the slave end of the TTY.
+ slaveWaiter waiter.Queue `state:"zerovalue"`
+}
+
+func newLineDiscipline(termios linux.KernelTermios) *lineDiscipline {
+ ld := lineDiscipline{termios: termios}
+ ld.inQueue.transformer = &inputQueueTransformer{}
+ ld.outQueue.transformer = &outputQueueTransformer{}
+ return &ld
+}
+
+// getTermios gets the linux.Termios for the tty.
+func (l *lineDiscipline) getTermios(ctx context.Context, io usermem.IO, args arch.SyscallArguments) (uintptr, error) {
+ l.termiosMu.RLock()
+ defer l.termiosMu.RUnlock()
+ // We must copy a Termios struct, not KernelTermios.
+ t := l.termios.ToTermios()
+ _, err := usermem.CopyObjectOut(ctx, io, args[2].Pointer(), t, usermem.IOOpts{
+ AddressSpaceActive: true,
+ })
+ return 0, err
+}
+
+// setTermios sets a linux.Termios for the tty.
+func (l *lineDiscipline) setTermios(ctx context.Context, io usermem.IO, args arch.SyscallArguments) (uintptr, error) {
+ l.termiosMu.Lock()
+ defer l.termiosMu.Unlock()
+ oldCanonEnabled := l.termios.LEnabled(linux.ICANON)
+ // We must copy a Termios struct, not KernelTermios.
+ var t linux.Termios
+ _, err := usermem.CopyObjectIn(ctx, io, args[2].Pointer(), &t, usermem.IOOpts{
+ AddressSpaceActive: true,
+ })
+ l.termios.FromTermios(t)
+
+ // If canonical mode is turned off, move bytes from inQueue's wait
+ // buffer to its read buffer. Anything already in the read buffer is
+ // now readable.
+ if oldCanonEnabled && !l.termios.LEnabled(linux.ICANON) {
+ l.inQueue.mu.Lock()
+ l.inQueue.pushWaitBufLocked(l)
+ l.inQueue.readable = true
+ l.inQueue.mu.Unlock()
+ l.slaveWaiter.Notify(waiter.EventIn)
+ }
+
+ return 0, err
+}
+
+func (l *lineDiscipline) windowSize(ctx context.Context, io usermem.IO, args arch.SyscallArguments) error {
+ l.sizeMu.Lock()
+ defer l.sizeMu.Unlock()
+ _, err := usermem.CopyObjectOut(ctx, io, args[2].Pointer(), l.size, usermem.IOOpts{
+ AddressSpaceActive: true,
+ })
+ return err
+}
+
+func (l *lineDiscipline) setWindowSize(ctx context.Context, io usermem.IO, args arch.SyscallArguments) error {
+ l.sizeMu.Lock()
+ defer l.sizeMu.Unlock()
+ _, err := usermem.CopyObjectIn(ctx, io, args[2].Pointer(), &l.size, usermem.IOOpts{
+ AddressSpaceActive: true,
+ })
+ return err
+}
+
+func (l *lineDiscipline) masterReadiness() waiter.EventMask {
+ // We don't have to lock a termios because the default master termios
+ // is immutable.
+ return l.inQueue.writeReadiness(&linux.MasterTermios) | l.outQueue.readReadiness(&linux.MasterTermios)
+}
+
+func (l *lineDiscipline) slaveReadiness() waiter.EventMask {
+ l.termiosMu.RLock()
+ defer l.termiosMu.RUnlock()
+ return l.outQueue.writeReadiness(&l.termios) | l.inQueue.readReadiness(&l.termios)
+}
+
+func (l *lineDiscipline) inputQueueReadSize(ctx context.Context, io usermem.IO, args arch.SyscallArguments) error {
+ return l.inQueue.readableSize(ctx, io, args)
+}
+
+func (l *lineDiscipline) inputQueueRead(ctx context.Context, dst usermem.IOSequence) (int64, error) {
+ l.termiosMu.RLock()
+ defer l.termiosMu.RUnlock()
+ n, pushed, err := l.inQueue.read(ctx, dst, l)
+ if err != nil {
+ return 0, err
+ }
+ if n > 0 {
+ l.masterWaiter.Notify(waiter.EventOut)
+ if pushed {
+ l.slaveWaiter.Notify(waiter.EventIn)
+ }
+ return n, nil
+ }
+ return 0, syserror.ErrWouldBlock
+}
+
+func (l *lineDiscipline) inputQueueWrite(ctx context.Context, src usermem.IOSequence) (int64, error) {
+ l.termiosMu.RLock()
+ defer l.termiosMu.RUnlock()
+ n, err := l.inQueue.write(ctx, src, l)
+ if err != nil {
+ return 0, err
+ }
+ if n > 0 {
+ l.slaveWaiter.Notify(waiter.EventIn)
+ return n, nil
+ }
+ return 0, syserror.ErrWouldBlock
+}
+
+func (l *lineDiscipline) outputQueueReadSize(ctx context.Context, io usermem.IO, args arch.SyscallArguments) error {
+ return l.outQueue.readableSize(ctx, io, args)
+}
+
+func (l *lineDiscipline) outputQueueRead(ctx context.Context, dst usermem.IOSequence) (int64, error) {
+ l.termiosMu.RLock()
+ defer l.termiosMu.RUnlock()
+ n, pushed, err := l.outQueue.read(ctx, dst, l)
+ if err != nil {
+ return 0, err
+ }
+ if n > 0 {
+ l.slaveWaiter.Notify(waiter.EventOut)
+ if pushed {
+ l.masterWaiter.Notify(waiter.EventIn)
+ }
+ return n, nil
+ }
+ return 0, syserror.ErrWouldBlock
+}
+
+func (l *lineDiscipline) outputQueueWrite(ctx context.Context, src usermem.IOSequence) (int64, error) {
+ l.termiosMu.RLock()
+ defer l.termiosMu.RUnlock()
+ n, err := l.outQueue.write(ctx, src, l)
+ if err != nil {
+ return 0, err
+ }
+ if n > 0 {
+ l.masterWaiter.Notify(waiter.EventIn)
+ return n, nil
+ }
+ return 0, syserror.ErrWouldBlock
+}
+
+// transformer is a helper interface to make it easier to stateify queue.
+type transformer interface {
+ // transform functions require queue's mutex to be held.
+ transform(*lineDiscipline, *queue, []byte) int
+}
+
+// outputQueueTransformer implements transformer. It performs line discipline
+// transformations on the output queue.
+//
+// +stateify savable
+type outputQueueTransformer struct{}
+
+// transform does output processing for one end of the pty. See
+// drivers/tty/n_tty.c:do_output_char for an analogous kernel function.
+//
+// Preconditions:
+// * l.termiosMu must be held for reading.
+// * q.mu must be held.
+func (*outputQueueTransformer) transform(l *lineDiscipline, q *queue, buf []byte) int {
+ // transformOutput is effectively always in noncanonical mode, as the
+ // master termios never has ICANON set.
+
+ if !l.termios.OEnabled(linux.OPOST) {
+ q.readBuf = append(q.readBuf, buf...)
+ if len(q.readBuf) > 0 {
+ q.readable = true
+ }
+ return len(buf)
+ }
+
+ var ret int
+ for len(buf) > 0 {
+ size := l.peek(buf)
+ cBytes := append([]byte{}, buf[:size]...)
+ ret += size
+ buf = buf[size:]
+ // We're guaranteed that cBytes has at least one element.
+ switch cBytes[0] {
+ case '\n':
+ if l.termios.OEnabled(linux.ONLRET) {
+ l.column = 0
+ }
+ if l.termios.OEnabled(linux.ONLCR) {
+ q.readBuf = append(q.readBuf, '\r', '\n')
+ continue
+ }
+ case '\r':
+ if l.termios.OEnabled(linux.ONOCR) && l.column == 0 {
+ continue
+ }
+ if l.termios.OEnabled(linux.OCRNL) {
+ cBytes[0] = '\n'
+ if l.termios.OEnabled(linux.ONLRET) {
+ l.column = 0
+ }
+ break
+ }
+ l.column = 0
+ case '\t':
+ spaces := spacesPerTab - l.column%spacesPerTab
+ if l.termios.OutputFlags&linux.TABDLY == linux.XTABS {
+ l.column += spaces
+ q.readBuf = append(q.readBuf, bytes.Repeat([]byte{' '}, spacesPerTab)...)
+ continue
+ }
+ l.column += spaces
+ case '\b':
+ if l.column > 0 {
+ l.column--
+ }
+ default:
+ l.column++
+ }
+ q.readBuf = append(q.readBuf, cBytes...)
+ }
+ if len(q.readBuf) > 0 {
+ q.readable = true
+ }
+ return ret
+}
+
+// inputQueueTransformer implements transformer. It performs line discipline
+// transformations on the input queue.
+//
+// +stateify savable
+type inputQueueTransformer struct{}
+
+// transform does input processing for one end of the pty. Characters read are
+// transformed according to flags set in the termios struct. See
+// drivers/tty/n_tty.c:n_tty_receive_char_special for an analogous kernel
+// function.
+//
+// Preconditions:
+// * l.termiosMu must be held for reading.
+// * q.mu must be held.
+func (*inputQueueTransformer) transform(l *lineDiscipline, q *queue, buf []byte) int {
+ // If there's a line waiting to be read in canonical mode, don't write
+ // anything else to the read buffer.
+ if l.termios.LEnabled(linux.ICANON) && q.readable {
+ return 0
+ }
+
+ maxBytes := nonCanonMaxBytes
+ if l.termios.LEnabled(linux.ICANON) {
+ maxBytes = canonMaxBytes
+ }
+
+ var ret int
+ for len(buf) > 0 && len(q.readBuf) < canonMaxBytes {
+ size := l.peek(buf)
+ cBytes := append([]byte{}, buf[:size]...)
+ // We're guaranteed that cBytes has at least one element.
+ switch cBytes[0] {
+ case '\r':
+ if l.termios.IEnabled(linux.IGNCR) {
+ buf = buf[size:]
+ ret += size
+ continue
+ }
+ if l.termios.IEnabled(linux.ICRNL) {
+ cBytes[0] = '\n'
+ }
+ case '\n':
+ if l.termios.IEnabled(linux.INLCR) {
+ cBytes[0] = '\r'
+ }
+ }
+
+ // In canonical mode, we discard non-terminating characters
+ // after the first 4095.
+ if l.shouldDiscard(q, cBytes) {
+ buf = buf[size:]
+ ret += size
+ continue
+ }
+
+ // Stop if the buffer would be overfilled.
+ if len(q.readBuf)+size > maxBytes {
+ break
+ }
+ buf = buf[size:]
+ ret += size
+
+ // If we get EOF, make the buffer available for reading.
+ if l.termios.LEnabled(linux.ICANON) && l.termios.IsEOF(cBytes[0]) {
+ q.readable = true
+ break
+ }
+
+ q.readBuf = append(q.readBuf, cBytes...)
+
+ // Anything written to the readBuf will have to be echoed.
+ if l.termios.LEnabled(linux.ECHO) {
+ l.outQueue.writeBytes(cBytes, l)
+ l.masterWaiter.Notify(waiter.EventIn)
+ }
+
+ // If we finish a line, make it available for reading.
+ if l.termios.LEnabled(linux.ICANON) && l.termios.IsTerminating(cBytes) {
+ q.readable = true
+ break
+ }
+ }
+
+ // In noncanonical mode, everything is readable.
+ if !l.termios.LEnabled(linux.ICANON) && len(q.readBuf) > 0 {
+ q.readable = true
+ }
+
+ return ret
+}
+
+// shouldDiscard returns whether c should be discarded. In canonical mode, if
+// too many bytes are enqueued, we keep reading input and discarding it until
+// we find a terminating character. Signal/echo processing still occurs.
+//
+// Precondition:
+// * l.termiosMu must be held for reading.
+// * q.mu must be held.
+func (l *lineDiscipline) shouldDiscard(q *queue, cBytes []byte) bool {
+ return l.termios.LEnabled(linux.ICANON) && len(q.readBuf)+len(cBytes) >= canonMaxBytes && !l.termios.IsTerminating(cBytes)
+}
+
+// peek returns the size in bytes of the next character to process. As long as
+// b isn't empty, peek returns a value of at least 1.
+func (l *lineDiscipline) peek(b []byte) int {
+ size := 1
+ // If UTF-8 support is enabled, runes might be multiple bytes.
+ if l.termios.IEnabled(linux.IUTF8) {
+ _, size = utf8.DecodeRune(b)
+ }
+ return size
+}
+
+// LINT.ThenChange(../../fs/tty/line_discipline.go)
diff --git a/pkg/sentry/fsimpl/devpts/master.go b/pkg/sentry/fsimpl/devpts/master.go
new file mode 100755
index 000000000..04a292927
--- /dev/null
+++ b/pkg/sentry/fsimpl/devpts/master.go
@@ -0,0 +1,226 @@
+// Copyright 2020 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 devpts
+
+import (
+ "gvisor.dev/gvisor/pkg/abi/linux"
+ "gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/sentry/arch"
+ "gvisor.dev/gvisor/pkg/sentry/fsimpl/kernfs"
+ "gvisor.dev/gvisor/pkg/sentry/kernel/auth"
+ "gvisor.dev/gvisor/pkg/sentry/unimpl"
+ "gvisor.dev/gvisor/pkg/sentry/vfs"
+ "gvisor.dev/gvisor/pkg/syserror"
+ "gvisor.dev/gvisor/pkg/usermem"
+ "gvisor.dev/gvisor/pkg/waiter"
+)
+
+// LINT.IfChange
+
+// masterInode is the inode for the master end of the Terminal.
+type masterInode struct {
+ kernfs.InodeAttrs
+ kernfs.InodeNoopRefCount
+ kernfs.InodeNotDirectory
+ kernfs.InodeNotSymlink
+
+ // Keep a reference to this inode's dentry.
+ dentry kernfs.Dentry
+
+ // root is the devpts root inode.
+ root *rootInode
+}
+
+var _ kernfs.Inode = (*masterInode)(nil)
+
+// Open implements kernfs.Inode.Open.
+func (mi *masterInode) Open(ctx context.Context, rp *vfs.ResolvingPath, vfsd *vfs.Dentry, opts vfs.OpenOptions) (*vfs.FileDescription, error) {
+ t, err := mi.root.allocateTerminal(rp.Credentials())
+ if err != nil {
+ return nil, err
+ }
+
+ mi.IncRef()
+ fd := &masterFileDescription{
+ inode: mi,
+ t: t,
+ }
+ if err := fd.vfsfd.Init(fd, opts.Flags, rp.Mount(), vfsd, &vfs.FileDescriptionOptions{}); err != nil {
+ mi.DecRef()
+ return nil, err
+ }
+ return &fd.vfsfd, nil
+}
+
+// Stat implements kernfs.Inode.Stat.
+func (mi *masterInode) Stat(vfsfs *vfs.Filesystem, opts vfs.StatOptions) (linux.Statx, error) {
+ statx, err := mi.InodeAttrs.Stat(vfsfs, opts)
+ if err != nil {
+ return linux.Statx{}, err
+ }
+ statx.Blksize = 1024
+ statx.RdevMajor = linux.TTYAUX_MAJOR
+ statx.RdevMinor = linux.PTMX_MINOR
+ return statx, nil
+}
+
+// SetStat implements kernfs.Inode.SetStat
+func (mi *masterInode) SetStat(ctx context.Context, vfsfs *vfs.Filesystem, creds *auth.Credentials, opts vfs.SetStatOptions) error {
+ if opts.Stat.Mask&linux.STATX_SIZE != 0 {
+ return syserror.EINVAL
+ }
+ return mi.InodeAttrs.SetStat(ctx, vfsfs, creds, opts)
+}
+
+type masterFileDescription struct {
+ vfsfd vfs.FileDescription
+ vfs.FileDescriptionDefaultImpl
+
+ inode *masterInode
+ t *Terminal
+}
+
+var _ vfs.FileDescriptionImpl = (*masterFileDescription)(nil)
+
+// Release implements vfs.FileDescriptionImpl.Release.
+func (mfd *masterFileDescription) Release() {
+ mfd.inode.root.masterClose(mfd.t)
+ mfd.inode.DecRef()
+}
+
+// EventRegister implements waiter.Waitable.EventRegister.
+func (mfd *masterFileDescription) EventRegister(e *waiter.Entry, mask waiter.EventMask) {
+ mfd.t.ld.masterWaiter.EventRegister(e, mask)
+}
+
+// EventUnregister implements waiter.Waitable.EventUnregister.
+func (mfd *masterFileDescription) EventUnregister(e *waiter.Entry) {
+ mfd.t.ld.masterWaiter.EventUnregister(e)
+}
+
+// Readiness implements waiter.Waitable.Readiness.
+func (mfd *masterFileDescription) Readiness(mask waiter.EventMask) waiter.EventMask {
+ return mfd.t.ld.masterReadiness()
+}
+
+// Read implements vfs.FileDescriptionImpl.Read.
+func (mfd *masterFileDescription) Read(ctx context.Context, dst usermem.IOSequence, _ vfs.ReadOptions) (int64, error) {
+ return mfd.t.ld.outputQueueRead(ctx, dst)
+}
+
+// Write implements vfs.FileDescriptionImpl.Write.
+func (mfd *masterFileDescription) Write(ctx context.Context, src usermem.IOSequence, _ vfs.WriteOptions) (int64, error) {
+ return mfd.t.ld.inputQueueWrite(ctx, src)
+}
+
+// Ioctl implements vfs.FileDescriptionImpl.Ioctl.
+func (mfd *masterFileDescription) Ioctl(ctx context.Context, io usermem.IO, args arch.SyscallArguments) (uintptr, error) {
+ switch cmd := args[1].Uint(); cmd {
+ case linux.FIONREAD: // linux.FIONREAD == linux.TIOCINQ
+ // Get the number of bytes in the output queue read buffer.
+ return 0, mfd.t.ld.outputQueueReadSize(ctx, io, args)
+ case linux.TCGETS:
+ // N.B. TCGETS on the master actually returns the configuration
+ // of the slave end.
+ return mfd.t.ld.getTermios(ctx, io, args)
+ case linux.TCSETS:
+ // N.B. TCSETS on the master actually affects the configuration
+ // of the slave end.
+ return mfd.t.ld.setTermios(ctx, io, args)
+ case linux.TCSETSW:
+ // TODO(b/29356795): This should drain the output queue first.
+ return mfd.t.ld.setTermios(ctx, io, args)
+ case linux.TIOCGPTN:
+ _, err := usermem.CopyObjectOut(ctx, io, args[2].Pointer(), uint32(mfd.t.n), usermem.IOOpts{
+ AddressSpaceActive: true,
+ })
+ return 0, err
+ case linux.TIOCSPTLCK:
+ // TODO(b/29356795): Implement pty locking. For now just pretend we do.
+ return 0, nil
+ case linux.TIOCGWINSZ:
+ return 0, mfd.t.ld.windowSize(ctx, io, args)
+ case linux.TIOCSWINSZ:
+ return 0, mfd.t.ld.setWindowSize(ctx, io, args)
+ case linux.TIOCSCTTY:
+ // Make the given terminal the controlling terminal of the
+ // calling process.
+ return 0, mfd.t.setControllingTTY(ctx, io, args, true /* isMaster */)
+ case linux.TIOCNOTTY:
+ // Release this process's controlling terminal.
+ return 0, mfd.t.releaseControllingTTY(ctx, io, args, true /* isMaster */)
+ case linux.TIOCGPGRP:
+ // Get the foreground process group.
+ return mfd.t.foregroundProcessGroup(ctx, io, args, true /* isMaster */)
+ case linux.TIOCSPGRP:
+ // Set the foreground process group.
+ return mfd.t.setForegroundProcessGroup(ctx, io, args, true /* isMaster */)
+ default:
+ maybeEmitUnimplementedEvent(ctx, cmd)
+ return 0, syserror.ENOTTY
+ }
+}
+
+// SetStat implements vfs.FileDescriptionImpl.SetStat.
+func (mfd *masterFileDescription) SetStat(ctx context.Context, opts vfs.SetStatOptions) error {
+ creds := auth.CredentialsFromContext(ctx)
+ fs := mfd.vfsfd.VirtualDentry().Mount().Filesystem()
+ return mfd.inode.SetStat(ctx, fs, creds, opts)
+}
+
+// Stat implements vfs.FileDescriptionImpl.Stat.
+func (mfd *masterFileDescription) Stat(ctx context.Context, opts vfs.StatOptions) (linux.Statx, error) {
+ fs := mfd.vfsfd.VirtualDentry().Mount().Filesystem()
+ return mfd.inode.Stat(fs, opts)
+}
+
+// maybeEmitUnimplementedEvent emits unimplemented event if cmd is valid.
+func maybeEmitUnimplementedEvent(ctx context.Context, cmd uint32) {
+ switch cmd {
+ case linux.TCGETS,
+ linux.TCSETS,
+ linux.TCSETSW,
+ linux.TCSETSF,
+ linux.TIOCGWINSZ,
+ linux.TIOCSWINSZ,
+ linux.TIOCSETD,
+ linux.TIOCSBRK,
+ linux.TIOCCBRK,
+ linux.TCSBRK,
+ linux.TCSBRKP,
+ linux.TIOCSTI,
+ linux.TIOCCONS,
+ linux.FIONBIO,
+ linux.TIOCEXCL,
+ linux.TIOCNXCL,
+ linux.TIOCGEXCL,
+ linux.TIOCGSID,
+ linux.TIOCGETD,
+ linux.TIOCVHANGUP,
+ linux.TIOCGDEV,
+ linux.TIOCMGET,
+ linux.TIOCMSET,
+ linux.TIOCMBIC,
+ linux.TIOCMBIS,
+ linux.TIOCGICOUNT,
+ linux.TCFLSH,
+ linux.TIOCSSERIAL,
+ linux.TIOCGPTPEER:
+
+ unimpl.EmitUnimplementedEvent(ctx)
+ }
+}
+
+// LINT.ThenChange(../../fs/tty/master.go)
diff --git a/pkg/sentry/fsimpl/devpts/queue.go b/pkg/sentry/fsimpl/devpts/queue.go
new file mode 100755
index 000000000..29a6be858
--- /dev/null
+++ b/pkg/sentry/fsimpl/devpts/queue.go
@@ -0,0 +1,240 @@
+// 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 devpts
+
+import (
+ "gvisor.dev/gvisor/pkg/abi/linux"
+ "gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/safemem"
+ "gvisor.dev/gvisor/pkg/sentry/arch"
+ "gvisor.dev/gvisor/pkg/sync"
+ "gvisor.dev/gvisor/pkg/syserror"
+ "gvisor.dev/gvisor/pkg/usermem"
+ "gvisor.dev/gvisor/pkg/waiter"
+)
+
+// LINT.IfChange
+
+// waitBufMaxBytes is the maximum size of a wait buffer. It is based on
+// TTYB_DEFAULT_MEM_LIMIT.
+const waitBufMaxBytes = 131072
+
+// queue represents one of the input or output queues between a pty master and
+// slave. Bytes written to a queue are added to the read buffer until it is
+// full, at which point they are written to the wait buffer. Bytes are
+// processed (i.e. undergo termios transformations) as they are added to the
+// read buffer. The read buffer is readable when its length is nonzero and
+// readable is true.
+//
+// +stateify savable
+type queue struct {
+ // mu protects everything in queue.
+ mu sync.Mutex `state:"nosave"`
+
+ // readBuf is buffer of data ready to be read when readable is true.
+ // This data has been processed.
+ readBuf []byte
+
+ // waitBuf contains data that can't fit into readBuf. It is put here
+ // until it can be loaded into the read buffer. waitBuf contains data
+ // that hasn't been processed.
+ waitBuf [][]byte
+ waitBufLen uint64
+
+ // readable indicates whether the read buffer can be read from. In
+ // canonical mode, there can be an unterminated line in the read buffer,
+ // so readable must be checked.
+ readable bool
+
+ // transform is the the queue's function for transforming bytes
+ // entering the queue. For example, transform might convert all '\r's
+ // entering the queue to '\n's.
+ transformer
+}
+
+// readReadiness returns whether q is ready to be read from.
+func (q *queue) readReadiness(t *linux.KernelTermios) waiter.EventMask {
+ q.mu.Lock()
+ defer q.mu.Unlock()
+ if len(q.readBuf) > 0 && q.readable {
+ return waiter.EventIn
+ }
+ return waiter.EventMask(0)
+}
+
+// writeReadiness returns whether q is ready to be written to.
+func (q *queue) writeReadiness(t *linux.KernelTermios) waiter.EventMask {
+ q.mu.Lock()
+ defer q.mu.Unlock()
+ if q.waitBufLen < waitBufMaxBytes {
+ return waiter.EventOut
+ }
+ return waiter.EventMask(0)
+}
+
+// readableSize writes the number of readable bytes to userspace.
+func (q *queue) readableSize(ctx context.Context, io usermem.IO, args arch.SyscallArguments) error {
+ q.mu.Lock()
+ defer q.mu.Unlock()
+ var size int32
+ if q.readable {
+ size = int32(len(q.readBuf))
+ }
+
+ _, err := usermem.CopyObjectOut(ctx, io, args[2].Pointer(), size, usermem.IOOpts{
+ AddressSpaceActive: true,
+ })
+ return err
+
+}
+
+// read reads from q to userspace. It returns the number of bytes read as well
+// as whether the read caused more readable data to become available (whether
+// data was pushed from the wait buffer to the read buffer).
+//
+// Preconditions:
+// * l.termiosMu must be held for reading.
+func (q *queue) read(ctx context.Context, dst usermem.IOSequence, l *lineDiscipline) (int64, bool, error) {
+ q.mu.Lock()
+ defer q.mu.Unlock()
+
+ if !q.readable {
+ return 0, false, syserror.ErrWouldBlock
+ }
+
+ if dst.NumBytes() > canonMaxBytes {
+ dst = dst.TakeFirst(canonMaxBytes)
+ }
+
+ n, err := dst.CopyOutFrom(ctx, safemem.ReaderFunc(func(dst safemem.BlockSeq) (uint64, error) {
+ src := safemem.BlockSeqOf(safemem.BlockFromSafeSlice(q.readBuf))
+ n, err := safemem.CopySeq(dst, src)
+ if err != nil {
+ return 0, err
+ }
+ q.readBuf = q.readBuf[n:]
+
+ // If we read everything, this queue is no longer readable.
+ if len(q.readBuf) == 0 {
+ q.readable = false
+ }
+
+ return n, nil
+ }))
+ if err != nil {
+ return 0, false, err
+ }
+
+ // Move data from the queue's wait buffer to its read buffer.
+ nPushed := q.pushWaitBufLocked(l)
+
+ return int64(n), nPushed > 0, nil
+}
+
+// write writes to q from userspace.
+//
+// Preconditions:
+// * l.termiosMu must be held for reading.
+func (q *queue) write(ctx context.Context, src usermem.IOSequence, l *lineDiscipline) (int64, error) {
+ q.mu.Lock()
+ defer q.mu.Unlock()
+
+ // Copy data into the wait buffer.
+ n, err := src.CopyInTo(ctx, safemem.WriterFunc(func(src safemem.BlockSeq) (uint64, error) {
+ copyLen := src.NumBytes()
+ room := waitBufMaxBytes - q.waitBufLen
+ // If out of room, return EAGAIN.
+ if room == 0 && copyLen > 0 {
+ return 0, syserror.ErrWouldBlock
+ }
+ // Cap the size of the wait buffer.
+ if copyLen > room {
+ copyLen = room
+ src = src.TakeFirst64(room)
+ }
+ buf := make([]byte, copyLen)
+
+ // Copy the data into the wait buffer.
+ dst := safemem.BlockSeqOf(safemem.BlockFromSafeSlice(buf))
+ n, err := safemem.CopySeq(dst, src)
+ if err != nil {
+ return 0, err
+ }
+ q.waitBufAppend(buf)
+
+ return n, nil
+ }))
+ if err != nil {
+ return 0, err
+ }
+
+ // Push data from the wait to the read buffer.
+ q.pushWaitBufLocked(l)
+
+ return n, nil
+}
+
+// writeBytes writes to q from b.
+//
+// Preconditions:
+// * l.termiosMu must be held for reading.
+func (q *queue) writeBytes(b []byte, l *lineDiscipline) {
+ q.mu.Lock()
+ defer q.mu.Unlock()
+
+ // Write to the wait buffer.
+ q.waitBufAppend(b)
+ q.pushWaitBufLocked(l)
+}
+
+// pushWaitBufLocked fills the queue's read buffer with data from the wait
+// buffer.
+//
+// Preconditions:
+// * l.termiosMu must be held for reading.
+// * q.mu must be locked.
+func (q *queue) pushWaitBufLocked(l *lineDiscipline) int {
+ if q.waitBufLen == 0 {
+ return 0
+ }
+
+ // Move data from the wait to the read buffer.
+ var total int
+ var i int
+ for i = 0; i < len(q.waitBuf); i++ {
+ n := q.transform(l, q, q.waitBuf[i])
+ total += n
+ if n != len(q.waitBuf[i]) {
+ // The read buffer filled up without consuming the
+ // entire buffer.
+ q.waitBuf[i] = q.waitBuf[i][n:]
+ break
+ }
+ }
+
+ // Update wait buffer based on consumed data.
+ q.waitBuf = q.waitBuf[i:]
+ q.waitBufLen -= uint64(total)
+
+ return total
+}
+
+// Precondition: q.mu must be locked.
+func (q *queue) waitBufAppend(b []byte) {
+ q.waitBuf = append(q.waitBuf, b)
+ q.waitBufLen += uint64(len(b))
+}
+
+// LINT.ThenChange(../../fs/tty/queue.go)
diff --git a/pkg/sentry/fsimpl/devpts/slave.go b/pkg/sentry/fsimpl/devpts/slave.go
new file mode 100755
index 000000000..0a98dc896
--- /dev/null
+++ b/pkg/sentry/fsimpl/devpts/slave.go
@@ -0,0 +1,186 @@
+// Copyright 2020 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 devpts
+
+import (
+ "gvisor.dev/gvisor/pkg/abi/linux"
+ "gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/sentry/arch"
+ "gvisor.dev/gvisor/pkg/sentry/fsimpl/kernfs"
+ "gvisor.dev/gvisor/pkg/sentry/kernel/auth"
+ "gvisor.dev/gvisor/pkg/sentry/vfs"
+ "gvisor.dev/gvisor/pkg/syserror"
+ "gvisor.dev/gvisor/pkg/usermem"
+ "gvisor.dev/gvisor/pkg/waiter"
+)
+
+// LINT.IfChange
+
+// slaveInode is the inode for the slave end of the Terminal.
+type slaveInode struct {
+ kernfs.InodeAttrs
+ kernfs.InodeNoopRefCount
+ kernfs.InodeNotDirectory
+ kernfs.InodeNotSymlink
+
+ // Keep a reference to this inode's dentry.
+ dentry kernfs.Dentry
+
+ // root is the devpts root inode.
+ root *rootInode
+
+ // t is the connected Terminal.
+ t *Terminal
+}
+
+var _ kernfs.Inode = (*slaveInode)(nil)
+
+// Open implements kernfs.Inode.Open.
+func (si *slaveInode) Open(ctx context.Context, rp *vfs.ResolvingPath, vfsd *vfs.Dentry, opts vfs.OpenOptions) (*vfs.FileDescription, error) {
+ si.IncRef()
+ fd := &slaveFileDescription{
+ inode: si,
+ }
+ if err := fd.vfsfd.Init(fd, opts.Flags, rp.Mount(), vfsd, &vfs.FileDescriptionOptions{}); err != nil {
+ si.DecRef()
+ return nil, err
+ }
+ return &fd.vfsfd, nil
+
+}
+
+// Valid implements kernfs.Inode.Valid.
+func (si *slaveInode) Valid(context.Context) bool {
+ // Return valid if the slave still exists.
+ si.root.mu.Lock()
+ defer si.root.mu.Unlock()
+ _, ok := si.root.slaves[si.t.n]
+ return ok
+}
+
+// Stat implements kernfs.Inode.Stat.
+func (si *slaveInode) Stat(vfsfs *vfs.Filesystem, opts vfs.StatOptions) (linux.Statx, error) {
+ statx, err := si.InodeAttrs.Stat(vfsfs, opts)
+ if err != nil {
+ return linux.Statx{}, err
+ }
+ statx.Blksize = 1024
+ statx.RdevMajor = linux.UNIX98_PTY_SLAVE_MAJOR
+ statx.RdevMinor = si.t.n
+ return statx, nil
+}
+
+// SetStat implements kernfs.Inode.SetStat
+func (si *slaveInode) SetStat(ctx context.Context, vfsfs *vfs.Filesystem, creds *auth.Credentials, opts vfs.SetStatOptions) error {
+ if opts.Stat.Mask&linux.STATX_SIZE != 0 {
+ return syserror.EINVAL
+ }
+ return si.InodeAttrs.SetStat(ctx, vfsfs, creds, opts)
+}
+
+type slaveFileDescription struct {
+ vfsfd vfs.FileDescription
+ vfs.FileDescriptionDefaultImpl
+
+ inode *slaveInode
+}
+
+var _ vfs.FileDescriptionImpl = (*slaveFileDescription)(nil)
+
+// Release implements fs.FileOperations.Release.
+func (sfd *slaveFileDescription) Release() {
+ sfd.inode.DecRef()
+}
+
+// EventRegister implements waiter.Waitable.EventRegister.
+func (sfd *slaveFileDescription) EventRegister(e *waiter.Entry, mask waiter.EventMask) {
+ sfd.inode.t.ld.slaveWaiter.EventRegister(e, mask)
+}
+
+// EventUnregister implements waiter.Waitable.EventUnregister.
+func (sfd *slaveFileDescription) EventUnregister(e *waiter.Entry) {
+ sfd.inode.t.ld.slaveWaiter.EventUnregister(e)
+}
+
+// Readiness implements waiter.Waitable.Readiness.
+func (sfd *slaveFileDescription) Readiness(mask waiter.EventMask) waiter.EventMask {
+ return sfd.inode.t.ld.slaveReadiness()
+}
+
+// Read implements vfs.FileDescriptionImpl.Read.
+func (sfd *slaveFileDescription) Read(ctx context.Context, dst usermem.IOSequence, _ vfs.ReadOptions) (int64, error) {
+ return sfd.inode.t.ld.inputQueueRead(ctx, dst)
+}
+
+// Write implements vfs.FileDescriptionImpl.Write.
+func (sfd *slaveFileDescription) Write(ctx context.Context, src usermem.IOSequence, _ vfs.WriteOptions) (int64, error) {
+ return sfd.inode.t.ld.outputQueueWrite(ctx, src)
+}
+
+// Ioctl implements vfs.FileDescripionImpl.Ioctl.
+func (sfd *slaveFileDescription) Ioctl(ctx context.Context, io usermem.IO, args arch.SyscallArguments) (uintptr, error) {
+ switch cmd := args[1].Uint(); cmd {
+ case linux.FIONREAD: // linux.FIONREAD == linux.TIOCINQ
+ // Get the number of bytes in the input queue read buffer.
+ return 0, sfd.inode.t.ld.inputQueueReadSize(ctx, io, args)
+ case linux.TCGETS:
+ return sfd.inode.t.ld.getTermios(ctx, io, args)
+ case linux.TCSETS:
+ return sfd.inode.t.ld.setTermios(ctx, io, args)
+ case linux.TCSETSW:
+ // TODO(b/29356795): This should drain the output queue first.
+ return sfd.inode.t.ld.setTermios(ctx, io, args)
+ case linux.TIOCGPTN:
+ _, err := usermem.CopyObjectOut(ctx, io, args[2].Pointer(), uint32(sfd.inode.t.n), usermem.IOOpts{
+ AddressSpaceActive: true,
+ })
+ return 0, err
+ case linux.TIOCGWINSZ:
+ return 0, sfd.inode.t.ld.windowSize(ctx, io, args)
+ case linux.TIOCSWINSZ:
+ return 0, sfd.inode.t.ld.setWindowSize(ctx, io, args)
+ case linux.TIOCSCTTY:
+ // Make the given terminal the controlling terminal of the
+ // calling process.
+ return 0, sfd.inode.t.setControllingTTY(ctx, io, args, false /* isMaster */)
+ case linux.TIOCNOTTY:
+ // Release this process's controlling terminal.
+ return 0, sfd.inode.t.releaseControllingTTY(ctx, io, args, false /* isMaster */)
+ case linux.TIOCGPGRP:
+ // Get the foreground process group.
+ return sfd.inode.t.foregroundProcessGroup(ctx, io, args, false /* isMaster */)
+ case linux.TIOCSPGRP:
+ // Set the foreground process group.
+ return sfd.inode.t.setForegroundProcessGroup(ctx, io, args, false /* isMaster */)
+ default:
+ maybeEmitUnimplementedEvent(ctx, cmd)
+ return 0, syserror.ENOTTY
+ }
+}
+
+// SetStat implements vfs.FileDescriptionImpl.SetStat.
+func (sfd *slaveFileDescription) SetStat(ctx context.Context, opts vfs.SetStatOptions) error {
+ creds := auth.CredentialsFromContext(ctx)
+ fs := sfd.vfsfd.VirtualDentry().Mount().Filesystem()
+ return sfd.inode.SetStat(ctx, fs, creds, opts)
+}
+
+// Stat implements vfs.FileDescriptionImpl.Stat.
+func (sfd *slaveFileDescription) Stat(ctx context.Context, opts vfs.StatOptions) (linux.Statx, error) {
+ fs := sfd.vfsfd.VirtualDentry().Mount().Filesystem()
+ return sfd.inode.Stat(fs, opts)
+}
+
+// LINT.ThenChange(../../fs/tty/slave.go)
diff --git a/pkg/sentry/fsimpl/devpts/terminal.go b/pkg/sentry/fsimpl/devpts/terminal.go
new file mode 100755
index 000000000..b44e673d8
--- /dev/null
+++ b/pkg/sentry/fsimpl/devpts/terminal.go
@@ -0,0 +1,124 @@
+// 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 devpts
+
+import (
+ "gvisor.dev/gvisor/pkg/abi/linux"
+ "gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/sentry/arch"
+ "gvisor.dev/gvisor/pkg/sentry/kernel"
+ "gvisor.dev/gvisor/pkg/usermem"
+)
+
+// LINT.IfChanges
+
+// Terminal is a pseudoterminal.
+//
+// +stateify savable
+type Terminal struct {
+ // n is the terminal index. It is immutable.
+ n uint32
+
+ // ld is the line discipline of the terminal. It is immutable.
+ ld *lineDiscipline
+
+ // masterKTTY contains the controlling process of the master end of
+ // this terminal. This field is immutable.
+ masterKTTY *kernel.TTY
+
+ // slaveKTTY contains the controlling process of the slave end of this
+ // terminal. This field is immutable.
+ slaveKTTY *kernel.TTY
+}
+
+func newTerminal(n uint32) *Terminal {
+ termios := linux.DefaultSlaveTermios
+ t := Terminal{
+ n: n,
+ ld: newLineDiscipline(termios),
+ masterKTTY: &kernel.TTY{Index: n},
+ slaveKTTY: &kernel.TTY{Index: n},
+ }
+ return &t
+}
+
+// setControllingTTY makes tm the controlling terminal of the calling thread
+// group.
+func (tm *Terminal) setControllingTTY(ctx context.Context, io usermem.IO, args arch.SyscallArguments, isMaster bool) error {
+ task := kernel.TaskFromContext(ctx)
+ if task == nil {
+ panic("setControllingTTY must be called from a task context")
+ }
+
+ return task.ThreadGroup().SetControllingTTY(tm.tty(isMaster), args[2].Int())
+}
+
+// releaseControllingTTY removes tm as the controlling terminal of the calling
+// thread group.
+func (tm *Terminal) releaseControllingTTY(ctx context.Context, io usermem.IO, args arch.SyscallArguments, isMaster bool) error {
+ task := kernel.TaskFromContext(ctx)
+ if task == nil {
+ panic("releaseControllingTTY must be called from a task context")
+ }
+
+ return task.ThreadGroup().ReleaseControllingTTY(tm.tty(isMaster))
+}
+
+// foregroundProcessGroup gets the process group ID of tm's foreground process.
+func (tm *Terminal) foregroundProcessGroup(ctx context.Context, io usermem.IO, args arch.SyscallArguments, isMaster bool) (uintptr, error) {
+ task := kernel.TaskFromContext(ctx)
+ if task == nil {
+ panic("foregroundProcessGroup must be called from a task context")
+ }
+
+ ret, err := task.ThreadGroup().ForegroundProcessGroup(tm.tty(isMaster))
+ if err != nil {
+ return 0, err
+ }
+
+ // Write it out to *arg.
+ _, err = usermem.CopyObjectOut(ctx, io, args[2].Pointer(), int32(ret), usermem.IOOpts{
+ AddressSpaceActive: true,
+ })
+ return 0, err
+}
+
+// foregroundProcessGroup sets tm's foreground process.
+func (tm *Terminal) setForegroundProcessGroup(ctx context.Context, io usermem.IO, args arch.SyscallArguments, isMaster bool) (uintptr, error) {
+ task := kernel.TaskFromContext(ctx)
+ if task == nil {
+ panic("setForegroundProcessGroup must be called from a task context")
+ }
+
+ // Read in the process group ID.
+ var pgid int32
+ if _, err := usermem.CopyObjectIn(ctx, io, args[2].Pointer(), &pgid, usermem.IOOpts{
+ AddressSpaceActive: true,
+ }); err != nil {
+ return 0, err
+ }
+
+ ret, err := task.ThreadGroup().SetForegroundProcessGroup(tm.tty(isMaster), kernel.ProcessGroupID(pgid))
+ return uintptr(ret), err
+}
+
+func (tm *Terminal) tty(isMaster bool) *kernel.TTY {
+ if isMaster {
+ return tm.masterKTTY
+ }
+ return tm.slaveKTTY
+}
+
+// LINT.ThenChange(../../fs/tty/terminal.go)
diff --git a/pkg/sentry/fsimpl/host/host.go b/pkg/sentry/fsimpl/host/host.go
index 34fbc69af..2be498afc 100755
--- a/pkg/sentry/fsimpl/host/host.go
+++ b/pkg/sentry/fsimpl/host/host.go
@@ -422,7 +422,7 @@ func (i *inode) open(ctx context.Context, d *vfs.Dentry, mnt *vfs.Mount) (*vfs.F
// TODO(gvisor.dev/issue/1672): Whitelist specific file types here, so that
// we don't allow importing arbitrary file types without proper support.
if i.isTTY {
- fd := &ttyFD{
+ fd := &TTYFileDescription{
fileDescription: fileDescription{inode: i},
termios: linux.DefaultSlaveTermios,
}
diff --git a/pkg/sentry/fsimpl/host/tty.go b/pkg/sentry/fsimpl/host/tty.go
index 8936afb06..68af6e5af 100755
--- a/pkg/sentry/fsimpl/host/tty.go
+++ b/pkg/sentry/fsimpl/host/tty.go
@@ -26,15 +26,15 @@ import (
"gvisor.dev/gvisor/pkg/usermem"
)
-// ttyFD implements vfs.FileDescriptionImpl for a host file descriptor
-// that wraps a TTY FD.
-type ttyFD struct {
+// TTYFileDescription implements vfs.FileDescriptionImpl for a host file
+// descriptor that wraps a TTY FD.
+type TTYFileDescription struct {
fileDescription
// mu protects the fields below.
mu sync.Mutex `state:"nosave"`
- // session is the session attached to this ttyFD.
+ // session is the session attached to this TTYFileDescription.
session *kernel.Session
// fgProcessGroup is the foreground process group that is currently
@@ -48,7 +48,7 @@ type ttyFD struct {
// InitForegroundProcessGroup sets the foreground process group and session for
// the TTY. This should only be called once, after the foreground process group
// has been created, but before it has started running.
-func (t *ttyFD) InitForegroundProcessGroup(pg *kernel.ProcessGroup) {
+func (t *TTYFileDescription) InitForegroundProcessGroup(pg *kernel.ProcessGroup) {
t.mu.Lock()
defer t.mu.Unlock()
if t.fgProcessGroup != nil {
@@ -59,14 +59,14 @@ func (t *ttyFD) InitForegroundProcessGroup(pg *kernel.ProcessGroup) {
}
// ForegroundProcessGroup returns the foreground process for the TTY.
-func (t *ttyFD) ForegroundProcessGroup() *kernel.ProcessGroup {
+func (t *TTYFileDescription) ForegroundProcessGroup() *kernel.ProcessGroup {
t.mu.Lock()
defer t.mu.Unlock()
return t.fgProcessGroup
}
// Release implements fs.FileOperations.Release.
-func (t *ttyFD) Release() {
+func (t *TTYFileDescription) Release() {
t.mu.Lock()
t.fgProcessGroup = nil
t.mu.Unlock()
@@ -78,7 +78,7 @@ func (t *ttyFD) Release() {
//
// Reading from a TTY is only allowed for foreground process groups. Background
// process groups will either get EIO or a SIGTTIN.
-func (t *ttyFD) PRead(ctx context.Context, dst usermem.IOSequence, offset int64, opts vfs.ReadOptions) (int64, error) {
+func (t *TTYFileDescription) PRead(ctx context.Context, dst usermem.IOSequence, offset int64, opts vfs.ReadOptions) (int64, error) {
t.mu.Lock()
defer t.mu.Unlock()
@@ -96,7 +96,7 @@ func (t *ttyFD) PRead(ctx context.Context, dst usermem.IOSequence, offset int64,
//
// Reading from a TTY is only allowed for foreground process groups. Background
// process groups will either get EIO or a SIGTTIN.
-func (t *ttyFD) Read(ctx context.Context, dst usermem.IOSequence, opts vfs.ReadOptions) (int64, error) {
+func (t *TTYFileDescription) Read(ctx context.Context, dst usermem.IOSequence, opts vfs.ReadOptions) (int64, error) {
t.mu.Lock()
defer t.mu.Unlock()
@@ -111,7 +111,7 @@ func (t *ttyFD) Read(ctx context.Context, dst usermem.IOSequence, opts vfs.ReadO
}
// PWrite implements vfs.FileDescriptionImpl.
-func (t *ttyFD) PWrite(ctx context.Context, src usermem.IOSequence, offset int64, opts vfs.WriteOptions) (int64, error) {
+func (t *TTYFileDescription) PWrite(ctx context.Context, src usermem.IOSequence, offset int64, opts vfs.WriteOptions) (int64, error) {
t.mu.Lock()
defer t.mu.Unlock()
@@ -126,7 +126,7 @@ func (t *ttyFD) PWrite(ctx context.Context, src usermem.IOSequence, offset int64
}
// Write implements vfs.FileDescriptionImpl.
-func (t *ttyFD) Write(ctx context.Context, src usermem.IOSequence, opts vfs.WriteOptions) (int64, error) {
+func (t *TTYFileDescription) Write(ctx context.Context, src usermem.IOSequence, opts vfs.WriteOptions) (int64, error) {
t.mu.Lock()
defer t.mu.Unlock()
@@ -141,7 +141,7 @@ func (t *ttyFD) Write(ctx context.Context, src usermem.IOSequence, opts vfs.Writ
}
// Ioctl implements vfs.FileDescriptionImpl.
-func (t *ttyFD) Ioctl(ctx context.Context, io usermem.IO, args arch.SyscallArguments) (uintptr, error) {
+func (t *TTYFileDescription) Ioctl(ctx context.Context, io usermem.IO, args arch.SyscallArguments) (uintptr, error) {
// Ignore arg[0]. This is the real FD:
fd := t.inode.hostFD
ioctl := args[1].Uint64()
@@ -321,7 +321,7 @@ func (t *ttyFD) Ioctl(ctx context.Context, io usermem.IO, args arch.SyscallArgum
// is a bit convoluted, but documented inline.
//
// Preconditions: t.mu must be held.
-func (t *ttyFD) checkChange(ctx context.Context, sig linux.Signal) error {
+func (t *TTYFileDescription) checkChange(ctx context.Context, sig linux.Signal) error {
task := kernel.TaskFromContext(ctx)
if task == nil {
// No task? Linux does not have an analog for this case, but
diff --git a/pkg/sentry/vfs/context.go b/pkg/sentry/vfs/context.go
index 82781e6d3..c9e724fef 100755
--- a/pkg/sentry/vfs/context.go
+++ b/pkg/sentry/vfs/context.go
@@ -49,3 +49,27 @@ func RootFromContext(ctx context.Context) VirtualDentry {
}
return VirtualDentry{}
}
+
+type rootContext struct {
+ context.Context
+ root VirtualDentry
+}
+
+// WithRoot returns a copy of ctx with the given root.
+func WithRoot(ctx context.Context, root VirtualDentry) context.Context {
+ return &rootContext{
+ Context: ctx,
+ root: root,
+ }
+}
+
+// Value implements Context.Value.
+func (rc rootContext) Value(key interface{}) interface{} {
+ switch key {
+ case CtxRoot:
+ rc.root.IncRef()
+ return rc.root
+ default:
+ return rc.Context.Value(key)
+ }
+}
diff --git a/pkg/sentry/vfs/epoll.go b/pkg/sentry/vfs/epoll.go
index 8e0b40841..8297f964b 100755
--- a/pkg/sentry/vfs/epoll.go
+++ b/pkg/sentry/vfs/epoll.go
@@ -192,6 +192,7 @@ func (ep *EpollInstance) AddInterest(file *FileDescription, num int32, event lin
mask: mask,
userData: event.Data,
}
+ epi.waiter.Callback = epi
ep.interest[key] = epi
wmask := waiter.EventMaskFromLinux(mask)
file.EventRegister(&epi.waiter, wmask)