summaryrefslogtreecommitdiffhomepage
diff options
context:
space:
mode:
-rw-r--r--pkg/sentry/fs/tty/BUILD1
-rw-r--r--pkg/sentry/fs/tty/dir.go3
-rw-r--r--pkg/sentry/fs/tty/master.go17
-rw-r--r--pkg/sentry/fs/tty/slave.go13
-rw-r--r--pkg/sentry/fs/tty/terminal.go92
-rw-r--r--pkg/sentry/kernel/BUILD1
-rw-r--r--pkg/sentry/kernel/sessions.go12
-rw-r--r--pkg/sentry/kernel/task_start.go3
-rw-r--r--pkg/sentry/kernel/thread_group.go179
-rw-r--r--pkg/sentry/kernel/tty.go28
-rw-r--r--test/syscalls/BUILD4
-rw-r--r--test/syscalls/linux/BUILD19
-rw-r--r--test/syscalls/linux/pty.cc359
-rw-r--r--test/syscalls/linux/pty_root.cc68
-rw-r--r--test/util/BUILD11
-rw-r--r--test/util/pty_util.cc45
-rw-r--r--test/util/pty_util.h30
17 files changed, 851 insertions, 34 deletions
diff --git a/pkg/sentry/fs/tty/BUILD b/pkg/sentry/fs/tty/BUILD
index 5e9327aec..291164986 100644
--- a/pkg/sentry/fs/tty/BUILD
+++ b/pkg/sentry/fs/tty/BUILD
@@ -23,6 +23,7 @@ go_library(
"//pkg/sentry/device",
"//pkg/sentry/fs",
"//pkg/sentry/fs/fsutil",
+ "//pkg/sentry/kernel",
"//pkg/sentry/kernel/auth",
"//pkg/sentry/safemem",
"//pkg/sentry/socket/unix/transport",
diff --git a/pkg/sentry/fs/tty/dir.go b/pkg/sentry/fs/tty/dir.go
index 1d128532b..2f639c823 100644
--- a/pkg/sentry/fs/tty/dir.go
+++ b/pkg/sentry/fs/tty/dir.go
@@ -129,6 +129,9 @@ func newDir(ctx context.Context, m *fs.MountSource) *fs.Inode {
// Release implements fs.InodeOperations.Release.
func (d *dirInodeOperations) Release(ctx context.Context) {
+ d.mu.Lock()
+ defer d.mu.Unlock()
+
d.master.DecRef()
if len(d.slaves) != 0 {
panic(fmt.Sprintf("devpts directory still contains active terminals: %+v", d))
diff --git a/pkg/sentry/fs/tty/master.go b/pkg/sentry/fs/tty/master.go
index 92ec1ca18..19b7557d5 100644
--- a/pkg/sentry/fs/tty/master.go
+++ b/pkg/sentry/fs/tty/master.go
@@ -172,6 +172,19 @@ func (mf *masterFileOperations) Ioctl(ctx context.Context, _ *fs.File, io userme
return 0, mf.t.ld.windowSize(ctx, io, args)
case linux.TIOCSWINSZ:
return 0, mf.t.ld.setWindowSize(ctx, io, args)
+ case linux.TIOCSCTTY:
+ // Make the given terminal the controlling terminal of the
+ // calling process.
+ return 0, mf.t.setControllingTTY(ctx, io, args, true /* isMaster */)
+ case linux.TIOCNOTTY:
+ // Release this process's controlling terminal.
+ return 0, mf.t.releaseControllingTTY(ctx, io, args, true /* isMaster */)
+ case linux.TIOCGPGRP:
+ // Get the foreground process group.
+ return mf.t.foregroundProcessGroup(ctx, io, args, true /* isMaster */)
+ case linux.TIOCSPGRP:
+ // Set the foreground process group.
+ return mf.t.setForegroundProcessGroup(ctx, io, args, true /* isMaster */)
default:
maybeEmitUnimplementedEvent(ctx, cmd)
return 0, syserror.ENOTTY
@@ -185,8 +198,6 @@ func maybeEmitUnimplementedEvent(ctx context.Context, cmd uint32) {
linux.TCSETS,
linux.TCSETSW,
linux.TCSETSF,
- linux.TIOCGPGRP,
- linux.TIOCSPGRP,
linux.TIOCGWINSZ,
linux.TIOCSWINSZ,
linux.TIOCSETD,
@@ -200,8 +211,6 @@ func maybeEmitUnimplementedEvent(ctx context.Context, cmd uint32) {
linux.TIOCEXCL,
linux.TIOCNXCL,
linux.TIOCGEXCL,
- linux.TIOCNOTTY,
- linux.TIOCSCTTY,
linux.TIOCGSID,
linux.TIOCGETD,
linux.TIOCVHANGUP,
diff --git a/pkg/sentry/fs/tty/slave.go b/pkg/sentry/fs/tty/slave.go
index e30266404..944c4ada1 100644
--- a/pkg/sentry/fs/tty/slave.go
+++ b/pkg/sentry/fs/tty/slave.go
@@ -152,9 +152,16 @@ func (sf *slaveFileOperations) Ioctl(ctx context.Context, _ *fs.File, io usermem
case linux.TIOCSCTTY:
// Make the given terminal the controlling terminal of the
// calling process.
- // TODO(b/129283598): Implement once we have support for job
- // control.
- return 0, nil
+ return 0, sf.si.t.setControllingTTY(ctx, io, args, false /* isMaster */)
+ case linux.TIOCNOTTY:
+ // Release this process's controlling terminal.
+ return 0, sf.si.t.releaseControllingTTY(ctx, io, args, false /* isMaster */)
+ case linux.TIOCGPGRP:
+ // Get the foreground process group.
+ return sf.si.t.foregroundProcessGroup(ctx, io, args, false /* isMaster */)
+ case linux.TIOCSPGRP:
+ // Set the foreground process group.
+ return sf.si.t.setForegroundProcessGroup(ctx, io, args, false /* isMaster */)
default:
maybeEmitUnimplementedEvent(ctx, cmd)
return 0, syserror.ENOTTY
diff --git a/pkg/sentry/fs/tty/terminal.go b/pkg/sentry/fs/tty/terminal.go
index b7cecb2ed..ff8138820 100644
--- a/pkg/sentry/fs/tty/terminal.go
+++ b/pkg/sentry/fs/tty/terminal.go
@@ -17,7 +17,10 @@ package tty
import (
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/refs"
+ "gvisor.dev/gvisor/pkg/sentry/arch"
"gvisor.dev/gvisor/pkg/sentry/context"
+ "gvisor.dev/gvisor/pkg/sentry/kernel"
+ "gvisor.dev/gvisor/pkg/sentry/usermem"
)
// Terminal is a pseudoterminal.
@@ -26,23 +29,100 @@ import (
type Terminal struct {
refs.AtomicRefCount
- // n is the terminal index.
+ // n is the terminal index. It is immutable.
n uint32
- // d is the containing directory.
+ // d is the containing directory. It is immutable.
d *dirInodeOperations
- // ld is the line discipline of the terminal.
+ // 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(ctx context.Context, d *dirInodeOperations, n uint32) *Terminal {
termios := linux.DefaultSlaveTermios
t := Terminal{
- d: d,
- n: n,
- ld: newLineDiscipline(termios),
+ d: d,
+ n: n,
+ ld: newLineDiscipline(termios),
+ masterKTTY: &kernel.TTY{},
+ slaveKTTY: &kernel.TTY{},
}
t.EnableLeakCheck("tty.Terminal")
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
+}
diff --git a/pkg/sentry/kernel/BUILD b/pkg/sentry/kernel/BUILD
index e61d39c82..41bee9a22 100644
--- a/pkg/sentry/kernel/BUILD
+++ b/pkg/sentry/kernel/BUILD
@@ -144,6 +144,7 @@ go_library(
"threads.go",
"timekeeper.go",
"timekeeper_state.go",
+ "tty.go",
"uts_namespace.go",
"vdso.go",
"version.go",
diff --git a/pkg/sentry/kernel/sessions.go b/pkg/sentry/kernel/sessions.go
index 81fcd8258..e5f297478 100644
--- a/pkg/sentry/kernel/sessions.go
+++ b/pkg/sentry/kernel/sessions.go
@@ -47,6 +47,11 @@ type Session struct {
// The id is immutable.
id SessionID
+ // foreground is the foreground process group.
+ //
+ // This is protected by TaskSet.mu.
+ foreground *ProcessGroup
+
// ProcessGroups is a list of process groups in this Session. This is
// protected by TaskSet.mu.
processGroups processGroupList
@@ -260,12 +265,14 @@ func (pg *ProcessGroup) SendSignal(info *arch.SignalInfo) error {
func (tg *ThreadGroup) CreateSession() error {
tg.pidns.owner.mu.Lock()
defer tg.pidns.owner.mu.Unlock()
+ tg.signalHandlers.mu.Lock()
+ defer tg.signalHandlers.mu.Unlock()
return tg.createSession()
}
// createSession creates a new session for a threadgroup.
//
-// Precondition: callers must hold TaskSet.mu for writing.
+// Precondition: callers must hold TaskSet.mu and the signal mutex for writing.
func (tg *ThreadGroup) createSession() error {
// Get the ID for this thread in the current namespace.
id := tg.pidns.tgids[tg]
@@ -346,6 +353,9 @@ func (tg *ThreadGroup) createSession() error {
ns.processGroups[ProcessGroupID(local)] = pg
}
+ // Disconnect from the controlling terminal.
+ tg.tty = nil
+
return nil
}
diff --git a/pkg/sentry/kernel/task_start.go b/pkg/sentry/kernel/task_start.go
index d60cd62c7..ae6fc4025 100644
--- a/pkg/sentry/kernel/task_start.go
+++ b/pkg/sentry/kernel/task_start.go
@@ -172,9 +172,10 @@ func (ts *TaskSet) newTask(cfg *TaskConfig) (*Task, error) {
if parentPG := tg.parentPG(); parentPG == nil {
tg.createSession()
} else {
- // Inherit the process group.
+ // Inherit the process group and terminal.
parentPG.incRefWithParent(parentPG)
tg.processGroup = parentPG
+ tg.tty = t.parent.tg.tty
}
}
tg.tasks.PushBack(t)
diff --git a/pkg/sentry/kernel/thread_group.go b/pkg/sentry/kernel/thread_group.go
index 2a97e3e8e..0eef24bfb 100644
--- a/pkg/sentry/kernel/thread_group.go
+++ b/pkg/sentry/kernel/thread_group.go
@@ -19,10 +19,13 @@ import (
"sync/atomic"
"gvisor.dev/gvisor/pkg/abi/linux"
+ "gvisor.dev/gvisor/pkg/sentry/arch"
"gvisor.dev/gvisor/pkg/sentry/fs"
+ "gvisor.dev/gvisor/pkg/sentry/kernel/auth"
ktime "gvisor.dev/gvisor/pkg/sentry/kernel/time"
"gvisor.dev/gvisor/pkg/sentry/limits"
"gvisor.dev/gvisor/pkg/sentry/usage"
+ "gvisor.dev/gvisor/pkg/syserror"
)
// A ThreadGroup is a logical grouping of tasks that has widespread
@@ -245,6 +248,12 @@ type ThreadGroup struct {
//
// mounts is immutable.
mounts *fs.MountNamespace
+
+ // tty is the thread group's controlling terminal. If nil, there is no
+ // controlling terminal.
+ //
+ // tty is protected by the signal mutex.
+ tty *TTY
}
// newThreadGroup returns a new, empty thread group in PID namespace ns. The
@@ -324,6 +333,176 @@ func (tg *ThreadGroup) forEachChildThreadGroupLocked(fn func(*ThreadGroup)) {
}
}
+// SetControllingTTY sets tty as the controlling terminal of tg.
+func (tg *ThreadGroup) SetControllingTTY(tty *TTY, arg int32) error {
+ tty.mu.Lock()
+ defer tty.mu.Unlock()
+
+ // We might be asked to set the controlling terminal of multiple
+ // processes, so we lock both the TaskSet and SignalHandlers.
+ tg.pidns.owner.mu.Lock()
+ defer tg.pidns.owner.mu.Unlock()
+ tg.signalHandlers.mu.Lock()
+ defer tg.signalHandlers.mu.Unlock()
+
+ // "The calling process must be a session leader and not have a
+ // controlling terminal already." - tty_ioctl(4)
+ if tg.processGroup.session.leader != tg || tg.tty != nil {
+ return syserror.EINVAL
+ }
+
+ // "If this terminal is already the controlling terminal of a different
+ // session group, then the ioctl fails with EPERM, unless the caller
+ // has the CAP_SYS_ADMIN capability and arg equals 1, in which case the
+ // terminal is stolen, and all processes that had it as controlling
+ // terminal lose it." - tty_ioctl(4)
+ if tty.tg != nil && tg.processGroup.session != tty.tg.processGroup.session {
+ if !auth.CredentialsFromContext(tg.leader).HasCapability(linux.CAP_SYS_ADMIN) || arg != 1 {
+ return syserror.EPERM
+ }
+ // Steal the TTY away. Unlike TIOCNOTTY, don't send signals.
+ for othertg := range tg.pidns.owner.Root.tgids {
+ // This won't deadlock by locking tg.signalHandlers
+ // because at this point:
+ // - We only lock signalHandlers if it's in the same
+ // session as the tty's controlling thread group.
+ // - We know that the calling thread group is not in
+ // the same session as the tty's controlling thread
+ // group.
+ if othertg.processGroup.session == tty.tg.processGroup.session {
+ othertg.signalHandlers.mu.Lock()
+ othertg.tty = nil
+ othertg.signalHandlers.mu.Unlock()
+ }
+ }
+ }
+
+ // Set the controlling terminal and foreground process group.
+ tg.tty = tty
+ tg.processGroup.session.foreground = tg.processGroup
+ // Set this as the controlling process of the terminal.
+ tty.tg = tg
+
+ return nil
+}
+
+// ReleaseControllingTTY gives up tty as the controlling tty of tg.
+func (tg *ThreadGroup) ReleaseControllingTTY(tty *TTY) error {
+ tty.mu.Lock()
+ defer tty.mu.Unlock()
+
+ // We might be asked to set the controlling terminal of multiple
+ // processes, so we lock both the TaskSet and SignalHandlers.
+ tg.pidns.owner.mu.RLock()
+ defer tg.pidns.owner.mu.RUnlock()
+
+ // Just below, we may re-lock signalHandlers in order to send signals.
+ // Thus we can't defer Unlock here.
+ tg.signalHandlers.mu.Lock()
+
+ if tg.tty == nil || tg.tty != tty {
+ tg.signalHandlers.mu.Unlock()
+ return syserror.ENOTTY
+ }
+
+ // "If the process was session leader, then send SIGHUP and SIGCONT to
+ // the foreground process group and all processes in the current
+ // session lose their controlling terminal." - tty_ioctl(4)
+ // Remove tty as the controlling tty for each process in the session,
+ // then send them SIGHUP and SIGCONT.
+
+ // If we're not the session leader, we don't have to do much.
+ if tty.tg != tg {
+ tg.tty = nil
+ tg.signalHandlers.mu.Unlock()
+ return nil
+ }
+
+ tg.signalHandlers.mu.Unlock()
+
+ // We're the session leader. SIGHUP and SIGCONT the foreground process
+ // group and remove all controlling terminals in the session.
+ var lastErr error
+ for othertg := range tg.pidns.owner.Root.tgids {
+ if othertg.processGroup.session == tg.processGroup.session {
+ othertg.signalHandlers.mu.Lock()
+ othertg.tty = nil
+ if othertg.processGroup == tg.processGroup.session.foreground {
+ if err := othertg.leader.sendSignalLocked(&arch.SignalInfo{Signo: int32(linux.SIGHUP)}, true /* group */); err != nil {
+ lastErr = err
+ }
+ if err := othertg.leader.sendSignalLocked(&arch.SignalInfo{Signo: int32(linux.SIGCONT)}, true /* group */); err != nil {
+ lastErr = err
+ }
+ }
+ othertg.signalHandlers.mu.Unlock()
+ }
+ }
+
+ return lastErr
+}
+
+// ForegroundProcessGroup returns the process group ID of the foreground
+// process group.
+func (tg *ThreadGroup) ForegroundProcessGroup(tty *TTY) (int32, error) {
+ tty.mu.Lock()
+ defer tty.mu.Unlock()
+
+ tg.pidns.owner.mu.Lock()
+ defer tg.pidns.owner.mu.Unlock()
+ tg.signalHandlers.mu.Lock()
+ defer tg.signalHandlers.mu.Unlock()
+
+ // "When fd does not refer to the controlling terminal of the calling
+ // process, -1 is returned" - tcgetpgrp(3)
+ if tg.tty != tty {
+ return -1, syserror.ENOTTY
+ }
+
+ return int32(tg.processGroup.session.foreground.id), nil
+}
+
+// SetForegroundProcessGroup sets the foreground process group of tty to pgid.
+func (tg *ThreadGroup) SetForegroundProcessGroup(tty *TTY, pgid ProcessGroupID) (int32, error) {
+ tty.mu.Lock()
+ defer tty.mu.Unlock()
+
+ tg.pidns.owner.mu.Lock()
+ defer tg.pidns.owner.mu.Unlock()
+ tg.signalHandlers.mu.Lock()
+ defer tg.signalHandlers.mu.Unlock()
+
+ // TODO(b/129283598): "If tcsetpgrp() is called by a member of a
+ // background process group in its session, and the calling process is
+ // not blocking or ignoring SIGTTOU, a SIGTTOU signal is sent to all
+ // members of this background process group."
+
+ // tty must be the controlling terminal.
+ if tg.tty != tty {
+ return -1, syserror.ENOTTY
+ }
+
+ // pgid must be positive.
+ if pgid < 0 {
+ return -1, syserror.EINVAL
+ }
+
+ // pg must not be empty. Empty process groups are removed from their
+ // pid namespaces.
+ pg, ok := tg.pidns.processGroups[pgid]
+ if !ok {
+ return -1, syserror.ESRCH
+ }
+
+ // pg must be part of this process's session.
+ if tg.processGroup.session != pg.session {
+ return -1, syserror.EPERM
+ }
+
+ tg.processGroup.session.foreground.id = pgid
+ return 0, nil
+}
+
// itimerRealListener implements ktime.Listener for ITIMER_REAL expirations.
//
// +stateify savable
diff --git a/pkg/sentry/kernel/tty.go b/pkg/sentry/kernel/tty.go
new file mode 100644
index 000000000..34f84487a
--- /dev/null
+++ b/pkg/sentry/kernel/tty.go
@@ -0,0 +1,28 @@
+// 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 "sync"
+
+// TTY defines the relationship between a thread group and its controlling
+// terminal.
+//
+// +stateify savable
+type TTY struct {
+ mu sync.Mutex `state:"nosave"`
+
+ // tg is protected by mu.
+ tg *ThreadGroup
+}
diff --git a/test/syscalls/BUILD b/test/syscalls/BUILD
index 841a0f2e1..a3e43cad2 100644
--- a/test/syscalls/BUILD
+++ b/test/syscalls/BUILD
@@ -310,6 +310,10 @@ syscall_test(
)
syscall_test(
+ test = "//test/syscalls/linux:pty_root_test",
+)
+
+syscall_test(
add_overlay = True,
test = "//test/syscalls/linux:pwritev2_test",
)
diff --git a/test/syscalls/linux/BUILD b/test/syscalls/linux/BUILD
index 40fc73812..16666e772 100644
--- a/test/syscalls/linux/BUILD
+++ b/test/syscalls/linux/BUILD
@@ -1214,8 +1214,10 @@ cc_binary(
srcs = ["pty.cc"],
linkstatic = 1,
deps = [
+ "//test/util:capability_util",
"//test/util:file_descriptor",
"//test/util:posix_error",
+ "//test/util:pty_util",
"//test/util:test_main",
"//test/util:test_util",
"//test/util:thread_util",
@@ -1228,6 +1230,23 @@ cc_binary(
)
cc_binary(
+ name = "pty_root_test",
+ testonly = 1,
+ srcs = ["pty_root.cc"],
+ linkstatic = 1,
+ deps = [
+ "//test/util:capability_util",
+ "//test/util:file_descriptor",
+ "//test/util:posix_error",
+ "//test/util:pty_util",
+ "//test/util:test_main",
+ "//test/util:thread_util",
+ "@com_google_absl//absl/base:core_headers",
+ "@com_google_googletest//:gtest",
+ ],
+)
+
+cc_binary(
name = "partial_bad_buffer_test",
testonly = 1,
srcs = ["partial_bad_buffer.cc"],
diff --git a/test/syscalls/linux/pty.cc b/test/syscalls/linux/pty.cc
index d1ab4703f..bd6907876 100644
--- a/test/syscalls/linux/pty.cc
+++ b/test/syscalls/linux/pty.cc
@@ -13,13 +13,17 @@
// limitations under the License.
#include <fcntl.h>
+#include <linux/capability.h>
#include <linux/major.h>
#include <poll.h>
+#include <sched.h>
+#include <signal.h>
#include <sys/ioctl.h>
#include <sys/mman.h>
#include <sys/stat.h>
#include <sys/sysmacros.h>
#include <sys/types.h>
+#include <sys/wait.h>
#include <termios.h>
#include <unistd.h>
@@ -31,8 +35,10 @@
#include "absl/synchronization/notification.h"
#include "absl/time/clock.h"
#include "absl/time/time.h"
+#include "test/util/capability_util.h"
#include "test/util/file_descriptor.h"
#include "test/util/posix_error.h"
+#include "test/util/pty_util.h"
#include "test/util/test_util.h"
#include "test/util/thread_util.h"
@@ -370,25 +376,6 @@ PosixErrorOr<size_t> PollAndReadFd(int fd, void* buf, size_t count,
return PosixError(ETIMEDOUT, "Poll timed out");
}
-// Opens the slave end of the passed master as R/W and nonblocking.
-PosixErrorOr<FileDescriptor> OpenSlave(const FileDescriptor& master) {
- // Get pty index.
- int n;
- int ret = ioctl(master.get(), TIOCGPTN, &n);
- if (ret < 0) {
- return PosixError(errno, "ioctl(TIOCGPTN) failed");
- }
-
- // Unlock pts.
- int unlock = 0;
- ret = ioctl(master.get(), TIOCSPTLCK, &unlock);
- if (ret < 0) {
- return PosixError(errno, "ioctl(TIOSPTLCK) failed");
- }
-
- return Open(absl::StrCat("/dev/pts/", n), O_RDWR | O_NONBLOCK);
-}
-
TEST(BasicPtyTest, StatUnopenedMaster) {
struct stat s;
ASSERT_THAT(stat("/dev/ptmx", &s), SyscallSucceeds());
@@ -1233,6 +1220,340 @@ TEST_F(PtyTest, SetMasterWindowSize) {
EXPECT_EQ(retrieved_ws.ws_col, kCols);
}
+class JobControlTest : public ::testing::Test {
+ protected:
+ void SetUp() override {
+ master_ = ASSERT_NO_ERRNO_AND_VALUE(Open("/dev/ptmx", O_RDWR | O_NONBLOCK));
+ slave_ = ASSERT_NO_ERRNO_AND_VALUE(OpenSlave(master_));
+
+ // Make this a session leader, which also drops the controlling terminal.
+ // In the gVisor test environment, this test will be run as the session
+ // leader already (as the sentry init process).
+ if (!IsRunningOnGvisor()) {
+ ASSERT_THAT(setsid(), SyscallSucceeds());
+ }
+ }
+
+ // Master and slave ends of the PTY. Non-blocking.
+ FileDescriptor master_;
+ FileDescriptor slave_;
+};
+
+TEST_F(JobControlTest, SetTTYMaster) {
+ ASSERT_THAT(ioctl(master_.get(), TIOCSCTTY, 0), SyscallSucceeds());
+}
+
+TEST_F(JobControlTest, SetTTY) {
+ ASSERT_THAT(ioctl(slave_.get(), TIOCSCTTY, 0), SyscallSucceeds());
+}
+
+TEST_F(JobControlTest, SetTTYNonLeader) {
+ // Fork a process that won't be the session leader.
+ pid_t child = fork();
+ if (!child) {
+ // We shouldn't be able to set the terminal.
+ TEST_PCHECK(ioctl(slave_.get(), TIOCSCTTY, 0));
+ _exit(0);
+ }
+
+ int wstatus;
+ ASSERT_THAT(waitpid(child, &wstatus, 0), SyscallSucceedsWithValue(child));
+ ASSERT_EQ(wstatus, 0);
+}
+
+TEST_F(JobControlTest, SetTTYBadArg) {
+ // Despite the man page saying arg should be 0 here, Linux doesn't actually
+ // check.
+ ASSERT_THAT(ioctl(slave_.get(), TIOCSCTTY, 1), SyscallSucceeds());
+}
+
+TEST_F(JobControlTest, SetTTYDifferentSession) {
+ SKIP_IF(ASSERT_NO_ERRNO_AND_VALUE(HaveCapability(CAP_SYS_ADMIN)));
+
+ ASSERT_THAT(ioctl(slave_.get(), TIOCSCTTY, 0), SyscallSucceeds());
+
+ // Fork, join a new session, and try to steal the parent's controlling
+ // terminal, which should fail.
+ pid_t child = fork();
+ if (!child) {
+ TEST_PCHECK(setsid() >= 0);
+ // We shouldn't be able to steal the terminal.
+ TEST_PCHECK(ioctl(slave_.get(), TIOCSCTTY, 1));
+ _exit(0);
+ }
+
+ int wstatus;
+ ASSERT_THAT(waitpid(child, &wstatus, 0), SyscallSucceedsWithValue(child));
+ ASSERT_EQ(wstatus, 0);
+}
+
+TEST_F(JobControlTest, ReleaseTTY) {
+ ASSERT_THAT(ioctl(slave_.get(), TIOCSCTTY, 0), SyscallSucceeds());
+
+ // Make sure we're ignoring SIGHUP, which will be sent to this process once we
+ // disconnect they TTY.
+ struct sigaction sa = {
+ .sa_handler = SIG_IGN,
+ .sa_flags = 0,
+ };
+ sigemptyset(&sa.sa_mask);
+ struct sigaction old_sa;
+ EXPECT_THAT(sigaction(SIGHUP, &sa, &old_sa), SyscallSucceeds());
+ EXPECT_THAT(ioctl(slave_.get(), TIOCNOTTY), SyscallSucceeds());
+ EXPECT_THAT(sigaction(SIGHUP, &old_sa, NULL), SyscallSucceeds());
+}
+
+TEST_F(JobControlTest, ReleaseUnsetTTY) {
+ ASSERT_THAT(ioctl(slave_.get(), TIOCNOTTY), SyscallFailsWithErrno(ENOTTY));
+}
+
+TEST_F(JobControlTest, ReleaseWrongTTY) {
+ ASSERT_THAT(ioctl(slave_.get(), TIOCSCTTY, 0), SyscallSucceeds());
+
+ ASSERT_THAT(ioctl(master_.get(), TIOCNOTTY), SyscallFailsWithErrno(ENOTTY));
+}
+
+TEST_F(JobControlTest, ReleaseTTYNonLeader) {
+ ASSERT_THAT(ioctl(slave_.get(), TIOCSCTTY, 0), SyscallSucceeds());
+
+ pid_t child = fork();
+ if (!child) {
+ TEST_PCHECK(!ioctl(slave_.get(), TIOCNOTTY));
+ _exit(0);
+ }
+
+ int wstatus;
+ ASSERT_THAT(waitpid(child, &wstatus, 0), SyscallSucceedsWithValue(child));
+ ASSERT_EQ(wstatus, 0);
+}
+
+TEST_F(JobControlTest, ReleaseTTYDifferentSession) {
+ ASSERT_THAT(ioctl(slave_.get(), TIOCSCTTY, 0), SyscallSucceeds());
+
+ pid_t child = fork();
+ if (!child) {
+ // Join a new session, then try to disconnect.
+ TEST_PCHECK(setsid() >= 0);
+ TEST_PCHECK(ioctl(slave_.get(), TIOCNOTTY));
+ _exit(0);
+ }
+
+ int wstatus;
+ ASSERT_THAT(waitpid(child, &wstatus, 0), SyscallSucceedsWithValue(child));
+ ASSERT_EQ(wstatus, 0);
+}
+
+// Used by the child process spawned in ReleaseTTYSignals to track received
+// signals.
+static int received;
+
+void sig_handler(int signum) { received |= signum; }
+
+// When the session leader releases its controlling terminal, the foreground
+// process group gets SIGHUP, then SIGCONT. This test:
+// - Spawns 2 threads
+// - Has thread 1 return 0 if it gets both SIGHUP and SIGCONT
+// - Has thread 2 leave the foreground process group, and return non-zero if it
+// receives any signals.
+// - Has the parent thread release its controlling terminal
+// - Checks that thread 1 got both signals
+// - Checks that thread 2 didn't get any signals.
+TEST_F(JobControlTest, ReleaseTTYSignals) {
+ ASSERT_THAT(ioctl(slave_.get(), TIOCSCTTY, 0), SyscallSucceeds());
+
+ received = 0;
+ struct sigaction sa = {
+ .sa_handler = sig_handler,
+ .sa_flags = 0,
+ };
+ sigemptyset(&sa.sa_mask);
+ sigaddset(&sa.sa_mask, SIGHUP);
+ sigaddset(&sa.sa_mask, SIGCONT);
+ sigprocmask(SIG_BLOCK, &sa.sa_mask, NULL);
+
+ pid_t same_pgrp_child = fork();
+ if (!same_pgrp_child) {
+ // The child will wait for SIGHUP and SIGCONT, then return 0. It begins with
+ // SIGHUP and SIGCONT blocked. We install signal handlers for those signals,
+ // then use sigsuspend to wait for those specific signals.
+ TEST_PCHECK(!sigaction(SIGHUP, &sa, NULL));
+ TEST_PCHECK(!sigaction(SIGCONT, &sa, NULL));
+ sigset_t mask;
+ sigfillset(&mask);
+ sigdelset(&mask, SIGHUP);
+ sigdelset(&mask, SIGCONT);
+ while (received != (SIGHUP | SIGCONT)) {
+ sigsuspend(&mask);
+ }
+ _exit(0);
+ }
+
+ // We don't want to block these anymore.
+ sigprocmask(SIG_UNBLOCK, &sa.sa_mask, NULL);
+
+ // This child will return non-zero if either SIGHUP or SIGCONT are received.
+ pid_t diff_pgrp_child = fork();
+ if (!diff_pgrp_child) {
+ TEST_PCHECK(!setpgid(0, 0));
+ TEST_PCHECK(pause());
+ _exit(1);
+ }
+
+ EXPECT_THAT(setpgid(diff_pgrp_child, diff_pgrp_child), SyscallSucceeds());
+
+ // Make sure we're ignoring SIGHUP, which will be sent to this process once we
+ // disconnect they TTY.
+ struct sigaction sighup_sa = {
+ .sa_handler = SIG_IGN,
+ .sa_flags = 0,
+ };
+ sigemptyset(&sighup_sa.sa_mask);
+ struct sigaction old_sa;
+ EXPECT_THAT(sigaction(SIGHUP, &sighup_sa, &old_sa), SyscallSucceeds());
+
+ // Release the controlling terminal, sending SIGHUP and SIGCONT to all other
+ // processes in this process group.
+ EXPECT_THAT(ioctl(slave_.get(), TIOCNOTTY), SyscallSucceeds());
+
+ EXPECT_THAT(sigaction(SIGHUP, &old_sa, NULL), SyscallSucceeds());
+
+ // The child in the same process group will get signaled.
+ int wstatus;
+ EXPECT_THAT(waitpid(same_pgrp_child, &wstatus, 0),
+ SyscallSucceedsWithValue(same_pgrp_child));
+ EXPECT_EQ(wstatus, 0);
+
+ // The other child will not get signaled.
+ EXPECT_THAT(waitpid(diff_pgrp_child, &wstatus, WNOHANG),
+ SyscallSucceedsWithValue(0));
+ EXPECT_THAT(kill(diff_pgrp_child, SIGKILL), SyscallSucceeds());
+}
+
+TEST_F(JobControlTest, GetForegroundProcessGroup) {
+ ASSERT_THAT(ioctl(slave_.get(), TIOCSCTTY, 0), SyscallSucceeds());
+ pid_t foreground_pgid;
+ pid_t pid;
+ ASSERT_THAT(ioctl(slave_.get(), TIOCGPGRP, &foreground_pgid),
+ SyscallSucceeds());
+ ASSERT_THAT(pid = getpid(), SyscallSucceeds());
+
+ ASSERT_EQ(foreground_pgid, pid);
+}
+
+TEST_F(JobControlTest, GetForegroundProcessGroupNonControlling) {
+ // At this point there's no controlling terminal, so TIOCGPGRP should fail.
+ pid_t foreground_pgid;
+ ASSERT_THAT(ioctl(slave_.get(), TIOCGPGRP, &foreground_pgid),
+ SyscallFailsWithErrno(ENOTTY));
+}
+
+// This test:
+// - sets itself as the foreground process group
+// - creates a child process in a new process group
+// - sets that child as the foreground process group
+// - kills its child and sets itself as the foreground process group.
+TEST_F(JobControlTest, SetForegroundProcessGroup) {
+ ASSERT_THAT(ioctl(slave_.get(), TIOCSCTTY, 0), SyscallSucceeds());
+
+ // Ignore SIGTTOU so that we don't stop ourself when calling tcsetpgrp.
+ struct sigaction sa = {
+ .sa_handler = SIG_IGN,
+ .sa_flags = 0,
+ };
+ sigemptyset(&sa.sa_mask);
+ sigaction(SIGTTOU, &sa, NULL);
+
+ // Set ourself as the foreground process group.
+ ASSERT_THAT(tcsetpgrp(slave_.get(), getpgid(0)), SyscallSucceeds());
+
+ // Create a new process that just waits to be signaled.
+ pid_t child = fork();
+ if (!child) {
+ TEST_PCHECK(!pause());
+ // We should never reach this.
+ _exit(1);
+ }
+
+ // Make the child its own process group, then make it the controlling process
+ // group of the terminal.
+ ASSERT_THAT(setpgid(child, child), SyscallSucceeds());
+ ASSERT_THAT(tcsetpgrp(slave_.get(), child), SyscallSucceeds());
+
+ // Sanity check - we're still the controlling session.
+ ASSERT_EQ(getsid(0), getsid(child));
+
+ // Signal the child, wait for it to exit, then retake the terminal.
+ ASSERT_THAT(kill(child, SIGTERM), SyscallSucceeds());
+ int wstatus;
+ ASSERT_THAT(waitpid(child, &wstatus, 0), SyscallSucceedsWithValue(child));
+ ASSERT_TRUE(WIFSIGNALED(wstatus));
+ ASSERT_EQ(WTERMSIG(wstatus), SIGTERM);
+
+ // Set ourself as the foreground process.
+ pid_t pgid;
+ ASSERT_THAT(pgid = getpgid(0), SyscallSucceeds());
+ ASSERT_THAT(tcsetpgrp(slave_.get(), pgid), SyscallSucceeds());
+}
+
+TEST_F(JobControlTest, SetForegroundProcessGroupWrongTTY) {
+ pid_t pid = getpid();
+ ASSERT_THAT(ioctl(slave_.get(), TIOCSPGRP, &pid),
+ SyscallFailsWithErrno(ENOTTY));
+}
+
+TEST_F(JobControlTest, SetForegroundProcessGroupNegPgid) {
+ ASSERT_THAT(ioctl(slave_.get(), TIOCSCTTY, 0), SyscallSucceeds());
+
+ pid_t pid = -1;
+ ASSERT_THAT(ioctl(slave_.get(), TIOCSPGRP, &pid),
+ SyscallFailsWithErrno(EINVAL));
+}
+
+TEST_F(JobControlTest, SetForegroundProcessGroupEmptyProcessGroup) {
+ ASSERT_THAT(ioctl(slave_.get(), TIOCSCTTY, 0), SyscallSucceeds());
+
+ // Create a new process, put it in a new process group, make that group the
+ // foreground process group, then have the process wait.
+ pid_t child = fork();
+ if (!child) {
+ TEST_PCHECK(!setpgid(0, 0));
+ _exit(0);
+ }
+
+ // Wait for the child to exit.
+ int wstatus;
+ EXPECT_THAT(waitpid(child, &wstatus, 0), SyscallSucceedsWithValue(child));
+ // The child's process group doesn't exist anymore - this should fail.
+ ASSERT_THAT(ioctl(slave_.get(), TIOCSPGRP, &child),
+ SyscallFailsWithErrno(ESRCH));
+}
+
+TEST_F(JobControlTest, SetForegroundProcessGroupDifferentSession) {
+ ASSERT_THAT(ioctl(slave_.get(), TIOCSCTTY, 0), SyscallSucceeds());
+
+ // Create a new process and put it in a new session.
+ pid_t child = fork();
+ if (!child) {
+ TEST_PCHECK(setsid() >= 0);
+ // Tell the parent we're in a new session.
+ TEST_PCHECK(!raise(SIGSTOP));
+ TEST_PCHECK(!pause());
+ _exit(1);
+ }
+
+ // Wait for the child to tell us it's in a new session.
+ int wstatus;
+ EXPECT_THAT(waitpid(child, &wstatus, WUNTRACED),
+ SyscallSucceedsWithValue(child));
+ EXPECT_TRUE(WSTOPSIG(wstatus));
+
+ // Child is in a new session, so we can't make it the foregroup process group.
+ EXPECT_THAT(ioctl(slave_.get(), TIOCSPGRP, &child),
+ SyscallFailsWithErrno(EPERM));
+
+ EXPECT_THAT(kill(child, SIGKILL), SyscallSucceeds());
+}
+
} // namespace
} // namespace testing
} // namespace gvisor
diff --git a/test/syscalls/linux/pty_root.cc b/test/syscalls/linux/pty_root.cc
new file mode 100644
index 000000000..14a4af980
--- /dev/null
+++ b/test/syscalls/linux/pty_root.cc
@@ -0,0 +1,68 @@
+// 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.
+
+#include <sys/ioctl.h>
+#include <termios.h>
+
+#include "gtest/gtest.h"
+#include "absl/base/macros.h"
+#include "test/util/capability_util.h"
+#include "test/util/file_descriptor.h"
+#include "test/util/posix_error.h"
+#include "test/util/pty_util.h"
+
+namespace gvisor {
+namespace testing {
+
+// These tests should be run as root.
+namespace {
+
+TEST(JobControlRootTest, StealTTY) {
+ SKIP_IF(!ASSERT_NO_ERRNO_AND_VALUE(HaveCapability(CAP_SYS_ADMIN)));
+
+ // Make this a session leader, which also drops the controlling terminal.
+ // In the gVisor test environment, this test will be run as the session
+ // leader already (as the sentry init process).
+ if (!IsRunningOnGvisor()) {
+ ASSERT_THAT(setsid(), SyscallSucceeds());
+ }
+
+ FileDescriptor master =
+ ASSERT_NO_ERRNO_AND_VALUE(Open("/dev/ptmx", O_RDWR | O_NONBLOCK));
+ FileDescriptor slave = ASSERT_NO_ERRNO_AND_VALUE(OpenSlave(master));
+
+ // Make slave the controlling terminal.
+ ASSERT_THAT(ioctl(slave.get(), TIOCSCTTY, 0), SyscallSucceeds());
+
+ // Fork, join a new session, and try to steal the parent's controlling
+ // terminal, which should succeed when we have CAP_SYS_ADMIN and pass an arg
+ // of 1.
+ pid_t child = fork();
+ if (!child) {
+ ASSERT_THAT(setsid(), SyscallSucceeds());
+ // We shouldn't be able to steal the terminal with the wrong arg value.
+ TEST_PCHECK(ioctl(slave.get(), TIOCSCTTY, 0));
+ // We should be able to steal it here.
+ TEST_PCHECK(!ioctl(slave.get(), TIOCSCTTY, 1));
+ _exit(0);
+ }
+
+ int wstatus;
+ ASSERT_THAT(waitpid(child, &wstatus, 0), SyscallSucceedsWithValue(child));
+ ASSERT_EQ(wstatus, 0);
+}
+
+} // namespace
+} // namespace testing
+} // namespace gvisor
diff --git a/test/util/BUILD b/test/util/BUILD
index a1b9ff526..c124cef34 100644
--- a/test/util/BUILD
+++ b/test/util/BUILD
@@ -184,6 +184,17 @@ cc_test(
)
cc_library(
+ name = "pty_util",
+ testonly = 1,
+ srcs = ["pty_util.cc"],
+ hdrs = ["pty_util.h"],
+ deps = [
+ ":file_descriptor",
+ ":posix_error",
+ ],
+)
+
+cc_library(
name = "signal_util",
testonly = 1,
srcs = ["signal_util.cc"],
diff --git a/test/util/pty_util.cc b/test/util/pty_util.cc
new file mode 100644
index 000000000..c0fd9a095
--- /dev/null
+++ b/test/util/pty_util.cc
@@ -0,0 +1,45 @@
+// Copyright 2019 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.
+
+#include "test/util/pty_util.h"
+
+#include <sys/ioctl.h>
+#include <termios.h>
+
+#include "test/util/file_descriptor.h"
+#include "test/util/posix_error.h"
+
+namespace gvisor {
+namespace testing {
+
+PosixErrorOr<FileDescriptor> OpenSlave(const FileDescriptor& master) {
+ // Get pty index.
+ int n;
+ int ret = ioctl(master.get(), TIOCGPTN, &n);
+ if (ret < 0) {
+ return PosixError(errno, "ioctl(TIOCGPTN) failed");
+ }
+
+ // Unlock pts.
+ int unlock = 0;
+ ret = ioctl(master.get(), TIOCSPTLCK, &unlock);
+ if (ret < 0) {
+ return PosixError(errno, "ioctl(TIOSPTLCK) failed");
+ }
+
+ return Open(absl::StrCat("/dev/pts/", n), O_RDWR | O_NONBLOCK);
+}
+
+} // namespace testing
+} // namespace gvisor
diff --git a/test/util/pty_util.h b/test/util/pty_util.h
new file mode 100644
index 000000000..367b14f15
--- /dev/null
+++ b/test/util/pty_util.h
@@ -0,0 +1,30 @@
+// Copyright 2019 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.
+
+#ifndef GVISOR_TEST_UTIL_PTY_UTIL_H_
+#define GVISOR_TEST_UTIL_PTY_UTIL_H_
+
+#include "test/util/file_descriptor.h"
+#include "test/util/posix_error.h"
+
+namespace gvisor {
+namespace testing {
+
+// Opens the slave end of the passed master as R/W and nonblocking.
+PosixErrorOr<FileDescriptor> OpenSlave(const FileDescriptor& master);
+
+} // namespace testing
+} // namespace gvisor
+
+#endif // GVISOR_TEST_UTIL_PTY_UTIL_H_