diff options
Diffstat (limited to 'pkg/sentry/fs/host')
-rw-r--r-- | pkg/sentry/fs/host/BUILD | 1 | ||||
-rw-r--r-- | pkg/sentry/fs/host/file.go | 144 | ||||
-rw-r--r-- | pkg/sentry/fs/host/inode.go | 18 | ||||
-rw-r--r-- | pkg/sentry/fs/host/tty.go | 185 |
4 files changed, 222 insertions, 126 deletions
diff --git a/pkg/sentry/fs/host/BUILD b/pkg/sentry/fs/host/BUILD index d1a6eaf6e..c34f1c26b 100644 --- a/pkg/sentry/fs/host/BUILD +++ b/pkg/sentry/fs/host/BUILD @@ -17,6 +17,7 @@ go_library( "socket.go", "socket_state.go", "socket_unsafe.go", + "tty.go", "util.go", "util_unsafe.go", ], diff --git a/pkg/sentry/fs/host/file.go b/pkg/sentry/fs/host/file.go index 6f469b5cc..22a5d9f12 100644 --- a/pkg/sentry/fs/host/file.go +++ b/pkg/sentry/fs/host/file.go @@ -18,15 +18,12 @@ import ( "fmt" "syscall" - "gvisor.googlesource.com/gvisor/pkg/abi/linux" "gvisor.googlesource.com/gvisor/pkg/fd" "gvisor.googlesource.com/gvisor/pkg/log" "gvisor.googlesource.com/gvisor/pkg/secio" - "gvisor.googlesource.com/gvisor/pkg/sentry/arch" "gvisor.googlesource.com/gvisor/pkg/sentry/context" "gvisor.googlesource.com/gvisor/pkg/sentry/fs" "gvisor.googlesource.com/gvisor/pkg/sentry/fs/fsutil" - "gvisor.googlesource.com/gvisor/pkg/sentry/kernel" "gvisor.googlesource.com/gvisor/pkg/sentry/memmap" "gvisor.googlesource.com/gvisor/pkg/sentry/safemem" "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" @@ -39,6 +36,7 @@ import ( // // +stateify savable type fileOperations struct { + fsutil.NoIoctl `state:"nosave"` fsutil.NoopRelease `state:"nosave"` // iops are the Inode operations for this file. @@ -49,49 +47,49 @@ type fileOperations struct { // dirCursor is the directory cursor. dirCursor string - - // allowIoctl determines whether ioctls should be passed through to the - // host. - allowIoctl bool } // fileOperations implements fs.FileOperations. var _ fs.FileOperations = (*fileOperations)(nil) // NewFile creates a new File backed by the provided host file descriptor. If -// NewFile succeeds, ownership of the fd is transferred to the returned File. +// NewFile succeeds, ownership of the FD is transferred to the returned File. // // The returned File cannot be saved, since there is no guarantee that the same -// fd will exist or represent the same file at time of restore. If such a +// FD will exist or represent the same file at time of restore. If such a // guarantee does exist, use ImportFile instead. func NewFile(ctx context.Context, fd int, mounter fs.FileOwner) (*fs.File, error) { return newFileFromDonatedFD(ctx, fd, mounter, false, false) } // ImportFile creates a new File backed by the provided host file descriptor. -// Unlike NewFile, the file descriptor used by the File is duped from fd to -// ensure that later changes to fd are not reflected by the fs.File. +// Unlike NewFile, the file descriptor used by the File is duped from FD to +// ensure that later changes to FD are not reflected by the fs.File. // -// If the returned file is saved, it will be restored by re-importing the fd +// If the returned file is saved, it will be restored by re-importing the FD // originally passed to ImportFile. It is the restorer's responsibility to -// ensure that the fd represents the same file. -func ImportFile(ctx context.Context, fd int, mounter fs.FileOwner, allowIoctl bool) (*fs.File, error) { - return newFileFromDonatedFD(ctx, fd, mounter, true, allowIoctl) +// ensure that the FD represents the same file. +func ImportFile(ctx context.Context, fd int, mounter fs.FileOwner, isTTY bool) (*fs.File, error) { + return newFileFromDonatedFD(ctx, fd, mounter, true, isTTY) } -// newFileFromDonatedFD returns an fs.File from a donated fd. If the fd is +// newFileFromDonatedFD returns an fs.File from a donated FD. If the FD is // saveable, then saveable is true. -func newFileFromDonatedFD(ctx context.Context, donated int, mounter fs.FileOwner, saveable, allowIoctl bool) (*fs.File, error) { +func newFileFromDonatedFD(ctx context.Context, donated int, mounter fs.FileOwner, saveable, isTTY bool) (*fs.File, error) { var s syscall.Stat_t if err := syscall.Fstat(donated, &s); err != nil { return nil, err } + flags, err := fileFlagsFromDonatedFD(donated) + if err != nil { + return nil, err + } switch s.Mode & syscall.S_IFMT { case syscall.S_IFSOCK: - flags, err := fileFlagsFromDonatedFD(donated) - if err != nil { - return nil, err + if isTTY { + return nil, fmt.Errorf("cannot import host socket as TTY") } + s, err := newSocket(ctx, donated, saveable) if err != nil { return nil, err @@ -101,10 +99,6 @@ func newFileFromDonatedFD(ctx context.Context, donated int, mounter fs.FileOwner }) return s, nil default: - flags, err := fileFlagsFromDonatedFD(donated) - if err != nil { - return nil, err - } msrc := newMountSource(ctx, "/", mounter, &Filesystem{}, fs.MountSourceFlags{}, false /* dontTranslateOwnership */) inode, err := newInode(ctx, msrc, donated, saveable, true /* donated */) if err != nil { @@ -116,14 +110,18 @@ func newFileFromDonatedFD(ctx context.Context, donated int, mounter fs.FileOwner dirent := fs.NewDirent(inode, name) defer dirent.DecRef() - return newFile(ctx, dirent, flags, iops, allowIoctl), nil + if isTTY { + return newTTYFile(ctx, dirent, flags, iops), nil + } + + return newFile(ctx, dirent, flags, iops), nil } } func fileFlagsFromDonatedFD(donated int) (fs.FileFlags, error) { flags, _, errno := syscall.Syscall(syscall.SYS_FCNTL, uintptr(donated), syscall.F_GETFL, 0) if errno != 0 { - log.Warningf("Failed to get file flags for donated fd %d (errno=%d)", donated, errno) + log.Warningf("Failed to get file flags for donated FD %d (errno=%d)", donated, errno) return fs.FileFlags{}, syscall.EIO } accmode := flags & syscall.O_ACCMODE @@ -138,17 +136,14 @@ func fileFlagsFromDonatedFD(donated int) (fs.FileFlags, error) { } // newFile returns a new fs.File. -func newFile(ctx context.Context, dirent *fs.Dirent, flags fs.FileFlags, iops *inodeOperations, allowIoctl bool) *fs.File { +func newFile(ctx context.Context, dirent *fs.Dirent, flags fs.FileFlags, iops *inodeOperations) *fs.File { if !iops.ReturnsWouldBlock() { // Allow reading/writing at an arbitrary offset for files // that support it. flags.Pread = true flags.Pwrite = true } - return fs.NewFile(ctx, dirent, flags, &fileOperations{ - iops: iops, - allowIoctl: allowIoctl, - }) + return fs.NewFile(ctx, dirent, flags, &fileOperations{iops: iops}) } // EventRegister implements waiter.Waitable.EventRegister. @@ -269,7 +264,7 @@ func (f *fileOperations) Fsync(ctx context.Context, file *fs.File, start int64, func (f *fileOperations) Flush(context.Context, *fs.File) error { // This is a no-op because flushing the resource backing this // file would mean closing it. We can't do that because other - // open files may depend on the backing host fd. + // open files may depend on the backing host FD. return nil } @@ -285,88 +280,3 @@ func (f *fileOperations) ConfigureMMap(ctx context.Context, file *fs.File, opts func (f *fileOperations) Seek(ctx context.Context, file *fs.File, whence fs.SeekWhence, offset int64) (int64, error) { return fsutil.SeekWithDirCursor(ctx, file, whence, offset, &f.dirCursor) } - -// Ioctl implements fs.FileOperations.Iocotl. -func (f *fileOperations) Ioctl(ctx context.Context, io usermem.IO, args arch.SyscallArguments) (uintptr, error) { - if !f.allowIoctl { - return 0, syserror.ENOTTY - } - // Ignore arg[0]. This is the real FD: - fd := f.iops.fileState.FD() - ioctl := args[1].Uint64() - switch ioctl { - case linux.TCGETS: - termios, err := ioctlGetTermios(fd) - if err != nil { - return 0, err - } - _, err = usermem.CopyObjectOut(ctx, io, args[2].Pointer(), termios, usermem.IOOpts{ - AddressSpaceActive: true, - }) - return 0, err - - case linux.TCSETS, linux.TCSETSW, linux.TCSETSF: - var termios linux.Termios - if _, err := usermem.CopyObjectIn(ctx, io, args[2].Pointer(), &termios, usermem.IOOpts{ - AddressSpaceActive: true, - }); err != nil { - return 0, err - } - err := ioctlSetTermios(fd, ioctl, &termios) - return 0, err - - case linux.TIOCGPGRP: - // Args: pid_t *argp - // When successful, equivalent to *argp = tcgetpgrp(fd). - // Get the process group ID of the foreground process group on - // this terminal. - - t := kernel.TaskFromContext(ctx) - if t == nil { - panic(fmt.Sprintf("cannot get thread group from context %v", ctx)) - } - tid := t.ThreadID() - _, err := usermem.CopyObjectOut(ctx, io, args[2].Pointer(), &tid, usermem.IOOpts{ - AddressSpaceActive: true, - }) - return 0, err - - case linux.TIOCSPGRP: - // Args: const pid_t *argp - // Equivalent to tcsetpgrp(fd, *argp). - // Set the foreground process group ID of this terminal. - - // Not much we can do with this one at the moment, so we just - // lie and pretend everything is great. Bash and Sh seem fine - // with this. - log.Warningf("Ignoring application ioctl(TIOCSPGRP) call") - return 0, nil - - case linux.TIOCGWINSZ: - // Args: struct winsize *argp - // Get window size. - winsize, err := ioctlGetWinsize(fd) - if err != nil { - return 0, err - } - _, err = usermem.CopyObjectOut(ctx, io, args[2].Pointer(), winsize, usermem.IOOpts{ - AddressSpaceActive: true, - }) - return 0, err - - case linux.TIOCSWINSZ: - // Args: const struct winsize *argp - // Set window size. - var winsize linux.Winsize - if _, err := usermem.CopyObjectIn(ctx, io, args[2].Pointer(), &winsize, usermem.IOOpts{ - AddressSpaceActive: true, - }); err != nil { - return 0, err - } - err := ioctlSetWinsize(fd, &winsize) - return 0, err - - default: - return 0, syserror.ENOTTY - } -} diff --git a/pkg/sentry/fs/host/inode.go b/pkg/sentry/fs/host/inode.go index e7254fa7d..c2e8ba62f 100644 --- a/pkg/sentry/fs/host/inode.go +++ b/pkg/sentry/fs/host/inode.go @@ -73,7 +73,7 @@ type inodeFileState struct { // Common file system state. mops *superOperations `state:"wait"` - // descriptor is the backing host fd. + // descriptor is the backing host FD. descriptor *descriptor `state:"wait"` // Event queue for blocking operations. @@ -167,7 +167,7 @@ func (i *inodeFileState) unstableAttr(ctx context.Context) (fs.UnstableAttr, err // inodeOperations implements fs.InodeOperations. var _ fs.InodeOperations = (*inodeOperations)(nil) -// newInode returns a new fs.Inode backed by the host fd. +// newInode returns a new fs.Inode backed by the host FD. func newInode(ctx context.Context, msrc *fs.MountSource, fd int, saveable bool, donated bool) (*fs.Inode, error) { // Retrieve metadata. var s syscall.Stat_t @@ -212,8 +212,8 @@ func (i *inodeOperations) Mappable(inode *fs.Inode) memmap.Mappable { return i.cachingInodeOps } -// ReturnsWouldBlock returns true if this host fd can return EWOULDBLOCK -// for operations that would block. +// ReturnsWouldBlock returns true if this host FD can return EWOULDBLOCK for +// operations that would block. func (i *inodeOperations) ReturnsWouldBlock() bool { return i.fileState.descriptor.wouldBlock } @@ -226,7 +226,7 @@ func (i *inodeOperations) Release(context.Context) { // Lookup implements fs.InodeOperations.Lookup. func (i *inodeOperations) Lookup(ctx context.Context, dir *fs.Inode, name string) (*fs.Dirent, error) { - // Get a new fd relative to i at name. + // Get a new FD relative to i at name. fd, err := open(i, name) if err != nil { if err == syserror.ENOENT { @@ -321,7 +321,7 @@ func (i *inodeOperations) BoundEndpoint(inode *fs.Inode, path string) unix.Bound // GetFile implements fs.InodeOperations.GetFile. func (i *inodeOperations) GetFile(ctx context.Context, d *fs.Dirent, flags fs.FileFlags) (*fs.File, error) { - return newFile(ctx, d, flags, i, false), nil + return newFile(ctx, d, flags, i), nil } // canMap returns true if this fs.Inode can be memory mapped. @@ -362,7 +362,7 @@ func (i *inodeOperations) SetOwner(context.Context, *fs.Inode, fs.FileOwner) err func (i *inodeOperations) SetPermissions(ctx context.Context, inode *fs.Inode, f fs.FilePermissions) bool { // Can we use host kernel metadata caches? if !inode.MountSource.Flags.ForcePageCache || !canMap(inode) { - // Then just change the timestamps on the fd, the host + // Then just change the timestamps on the FD, the host // will synchronize the metadata update with any host // inode and page cache. return syscall.Fchmod(i.fileState.FD(), uint32(f.LinuxMode())) == nil @@ -375,7 +375,7 @@ func (i *inodeOperations) SetPermissions(ctx context.Context, inode *fs.Inode, f func (i *inodeOperations) SetTimestamps(ctx context.Context, inode *fs.Inode, ts fs.TimeSpec) error { // Can we use host kernel metadata caches? if !inode.MountSource.Flags.ForcePageCache || !canMap(inode) { - // Then just change the timestamps on the fd, the host + // Then just change the timestamps on the FD, the host // will synchronize the metadata update with any host // inode and page cache. return setTimestamps(i.fileState.FD(), ts) @@ -388,7 +388,7 @@ func (i *inodeOperations) SetTimestamps(ctx context.Context, inode *fs.Inode, ts func (i *inodeOperations) Truncate(ctx context.Context, inode *fs.Inode, size int64) error { // Is the file not memory-mappable? if !canMap(inode) { - // Then just change the file size on the fd, the host + // Then just change the file size on the FD, the host // will synchronize the metadata update with any host // inode and page cache. return syscall.Ftruncate(i.fileState.FD(), size) diff --git a/pkg/sentry/fs/host/tty.go b/pkg/sentry/fs/host/tty.go new file mode 100644 index 000000000..ad1323610 --- /dev/null +++ b/pkg/sentry/fs/host/tty.go @@ -0,0 +1,185 @@ +// Copyright 2018 Google Inc. +// +// 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 host + +import ( + "sync" + + "gvisor.googlesource.com/gvisor/pkg/abi/linux" + "gvisor.googlesource.com/gvisor/pkg/sentry/arch" + "gvisor.googlesource.com/gvisor/pkg/sentry/context" + "gvisor.googlesource.com/gvisor/pkg/sentry/fs" + "gvisor.googlesource.com/gvisor/pkg/sentry/kernel" + "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" + "gvisor.googlesource.com/gvisor/pkg/syserror" +) + +// TTYFileOperations implements fs.FileOperations for a host file descriptor +// that wraps a TTY FD. +// +// +stateify savable +type TTYFileOperations struct { + fileOperations + + // mu protects the fields below. + mu sync.Mutex + + // FGProcessGroup is the foreground process group this TTY. Will be + // nil if not set or if this file has been released. + fgProcessGroup *kernel.ProcessGroup +} + +// newTTYFile returns a new fs.File that wraps a TTY FD. +func newTTYFile(ctx context.Context, dirent *fs.Dirent, flags fs.FileFlags, iops *inodeOperations) *fs.File { + return fs.NewFile(ctx, dirent, flags, &TTYFileOperations{ + fileOperations: fileOperations{iops: iops}, + }) +} + +// ForegroundProcessGroup returns the foreground process for the TTY. This will +// be nil if the foreground process has not been set or if the file has been +// released. +func (t *TTYFileOperations) ForegroundProcessGroup() *kernel.ProcessGroup { + t.mu.Lock() + defer t.mu.Unlock() + return t.fgProcessGroup +} + +// Release implements fs.FileOperations.Release. +func (t *TTYFileOperations) Release() { + t.mu.Lock() + t.fgProcessGroup = nil + t.mu.Unlock() + + t.fileOperations.Release() +} + +// Ioctl implements fs.FileOperations.Ioctl. +func (t *TTYFileOperations) Ioctl(ctx context.Context, io usermem.IO, args arch.SyscallArguments) (uintptr, error) { + // Ignore arg[0]. This is the real FD: + fd := t.fileOperations.iops.fileState.FD() + ioctl := args[1].Uint64() + switch ioctl { + case linux.TCGETS: + termios, err := ioctlGetTermios(fd) + if err != nil { + return 0, err + } + _, err = usermem.CopyObjectOut(ctx, io, args[2].Pointer(), termios, usermem.IOOpts{ + AddressSpaceActive: true, + }) + return 0, err + + case linux.TCSETS, linux.TCSETSW, linux.TCSETSF: + var termios linux.Termios + if _, err := usermem.CopyObjectIn(ctx, io, args[2].Pointer(), &termios, usermem.IOOpts{ + AddressSpaceActive: true, + }); err != nil { + return 0, err + } + err := ioctlSetTermios(fd, ioctl, &termios) + return 0, err + + case linux.TIOCGPGRP: + // Args: pid_t *argp + // When successful, equivalent to *argp = tcgetpgrp(fd). + // Get the process group ID of the foreground process group on + // this terminal. + + t.mu.Lock() + defer t.mu.Unlock() + + if t.fgProcessGroup == nil { + // No process group has been set yet. Let's just lie + // and tell it the process group from the current task. + // The app is probably going to set it to something + // else very soon anyways. + t.fgProcessGroup = kernel.TaskFromContext(ctx).ThreadGroup().ProcessGroup() + } + + // Map the ProcessGroup into a ProcessGroupID in the task's PID + // namespace. + pgID := kernel.TaskFromContext(ctx).ThreadGroup().PIDNamespace().IDOfProcessGroup(t.fgProcessGroup) + _, err := usermem.CopyObjectOut(ctx, io, args[2].Pointer(), &pgID, usermem.IOOpts{ + AddressSpaceActive: true, + }) + return 0, err + + case linux.TIOCSPGRP: + // Args: const pid_t *argp + // Equivalent to tcsetpgrp(fd, *argp). + // Set the foreground process group ID of this terminal. + + var pgID kernel.ProcessGroupID + if _, err := usermem.CopyObjectIn(ctx, io, args[2].Pointer(), &pgID, usermem.IOOpts{ + AddressSpaceActive: true, + }); err != nil { + return 0, err + } + + // pgID must be non-negative. + if pgID < 0 { + return 0, syserror.EINVAL + } + + // Process group with pgID must exist in this PID namespace. + task := kernel.TaskFromContext(ctx) + pidns := task.PIDNamespace() + pg := pidns.ProcessGroupWithID(pgID) + if pg == nil { + return 0, syserror.ESRCH + } + + // Process group must be in same session as calling task's + // process group. + curSession := task.ThreadGroup().ProcessGroup().Session() + curSessionID := pidns.IDOfSession(curSession) + if pidns.IDOfSession(pg.Session()) != curSessionID { + return 0, syserror.EPERM + } + + t.mu.Lock() + t.fgProcessGroup = pg + t.mu.Unlock() + return 0, nil + + case linux.TIOCGWINSZ: + // Args: struct winsize *argp + // Get window size. + winsize, err := ioctlGetWinsize(fd) + if err != nil { + return 0, err + } + _, err = usermem.CopyObjectOut(ctx, io, args[2].Pointer(), winsize, usermem.IOOpts{ + AddressSpaceActive: true, + }) + return 0, err + + case linux.TIOCSWINSZ: + // Args: const struct winsize *argp + // Set window size. + var winsize linux.Winsize + if _, err := usermem.CopyObjectIn(ctx, io, args[2].Pointer(), &winsize, usermem.IOOpts{ + AddressSpaceActive: true, + }); err != nil { + return 0, err + } + err := ioctlSetWinsize(fd, &winsize) + return 0, err + + default: + return 0, syserror.ENOTTY + } +} |