summaryrefslogtreecommitdiffhomepage
diff options
context:
space:
mode:
-rw-r--r--pkg/abi/linux/BUILD1
-rw-r--r--pkg/abi/linux/signalfd.go45
-rw-r--r--pkg/sentry/kernel/signalfd/BUILD22
-rw-r--r--pkg/sentry/kernel/signalfd/signalfd.go137
-rw-r--r--pkg/sentry/kernel/task.go8
-rw-r--r--pkg/sentry/kernel/task_signals.go18
-rw-r--r--pkg/sentry/syscalls/linux/BUILD1
-rw-r--r--pkg/sentry/syscalls/linux/linux64.go4
-rw-r--r--pkg/sentry/syscalls/linux/sys_signal.go77
-rw-r--r--test/syscalls/linux/BUILD18
-rw-r--r--test/syscalls/linux/signalfd.cc333
11 files changed, 662 insertions, 2 deletions
diff --git a/pkg/abi/linux/BUILD b/pkg/abi/linux/BUILD
index 39c92bb33..f45934466 100644
--- a/pkg/abi/linux/BUILD
+++ b/pkg/abi/linux/BUILD
@@ -46,6 +46,7 @@ go_library(
"sem.go",
"shm.go",
"signal.go",
+ "signalfd.go",
"socket.go",
"splice.go",
"tcp.go",
diff --git a/pkg/abi/linux/signalfd.go b/pkg/abi/linux/signalfd.go
new file mode 100644
index 000000000..85fad9956
--- /dev/null
+++ b/pkg/abi/linux/signalfd.go
@@ -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.
+
+package linux
+
+const (
+ // SFD_NONBLOCK is a signalfd(2) flag.
+ SFD_NONBLOCK = 00004000
+
+ // SFD_CLOEXEC is a signalfd(2) flag.
+ SFD_CLOEXEC = 02000000
+)
+
+// SignalfdSiginfo is the siginfo encoding for signalfds.
+type SignalfdSiginfo struct {
+ Signo uint32
+ Errno int32
+ Code int32
+ PID uint32
+ UID uint32
+ FD int32
+ TID uint32
+ Band uint32
+ Overrun uint32
+ TrapNo uint32
+ Status int32
+ Int int32
+ Ptr uint64
+ UTime uint64
+ STime uint64
+ Addr uint64
+ AddrLSB uint16
+ _ [48]uint8
+}
diff --git a/pkg/sentry/kernel/signalfd/BUILD b/pkg/sentry/kernel/signalfd/BUILD
new file mode 100644
index 000000000..50b69d154
--- /dev/null
+++ b/pkg/sentry/kernel/signalfd/BUILD
@@ -0,0 +1,22 @@
+package(licenses = ["notice"])
+
+load("//tools/go_stateify:defs.bzl", "go_library")
+
+go_library(
+ name = "signalfd",
+ srcs = ["signalfd.go"],
+ importpath = "gvisor.dev/gvisor/pkg/sentry/kernel/signalfd",
+ visibility = ["//pkg/sentry:internal"],
+ deps = [
+ "//pkg/abi/linux",
+ "//pkg/binary",
+ "//pkg/sentry/context",
+ "//pkg/sentry/fs",
+ "//pkg/sentry/fs/anon",
+ "//pkg/sentry/fs/fsutil",
+ "//pkg/sentry/kernel",
+ "//pkg/sentry/usermem",
+ "//pkg/syserror",
+ "//pkg/waiter",
+ ],
+)
diff --git a/pkg/sentry/kernel/signalfd/signalfd.go b/pkg/sentry/kernel/signalfd/signalfd.go
new file mode 100644
index 000000000..06fd5ec88
--- /dev/null
+++ b/pkg/sentry/kernel/signalfd/signalfd.go
@@ -0,0 +1,137 @@
+// 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.
+
+// Package signalfd provides an implementation of signal file descriptors.
+package signalfd
+
+import (
+ "sync"
+
+ "gvisor.dev/gvisor/pkg/abi/linux"
+ "gvisor.dev/gvisor/pkg/binary"
+ "gvisor.dev/gvisor/pkg/sentry/context"
+ "gvisor.dev/gvisor/pkg/sentry/fs"
+ "gvisor.dev/gvisor/pkg/sentry/fs/anon"
+ "gvisor.dev/gvisor/pkg/sentry/fs/fsutil"
+ "gvisor.dev/gvisor/pkg/sentry/kernel"
+ "gvisor.dev/gvisor/pkg/sentry/usermem"
+ "gvisor.dev/gvisor/pkg/syserror"
+ "gvisor.dev/gvisor/pkg/waiter"
+)
+
+// SignalOperations represent a file with signalfd semantics.
+//
+// +stateify savable
+type SignalOperations struct {
+ fsutil.FileNoopRelease `state:"nosave"`
+ fsutil.FilePipeSeek `state:"nosave"`
+ fsutil.FileNotDirReaddir `state:"nosave"`
+ fsutil.FileNoIoctl `state:"nosave"`
+ fsutil.FileNoFsync `state:"nosave"`
+ fsutil.FileNoMMap `state:"nosave"`
+ fsutil.FileNoSplice `state:"nosave"`
+ fsutil.FileNoWrite `state:"nosave"`
+ fsutil.FileNoopFlush `state:"nosave"`
+ fsutil.FileUseInodeUnstableAttr `state:"nosave"`
+
+ // target is the original task target.
+ //
+ // The semantics here are a bit broken. Linux will always use current
+ // for all reads, regardless of where the signalfd originated. We can't
+ // do exactly that because we need to plumb the context through
+ // EventRegister in order to support proper blocking behavior. This
+ // will undoubtedly become very complicated quickly.
+ target *kernel.Task
+
+ // mu protects below.
+ mu sync.Mutex `state:"nosave"`
+
+ // mask is the signal mask. Protected by mu.
+ mask linux.SignalSet
+}
+
+// New creates a new signalfd object with the supplied mask.
+func New(ctx context.Context, mask linux.SignalSet) (*fs.File, error) {
+ t := kernel.TaskFromContext(ctx)
+ if t == nil {
+ // No task context? Not valid.
+ return nil, syserror.EINVAL
+ }
+ // name matches fs/signalfd.c:signalfd4.
+ dirent := fs.NewDirent(ctx, anon.NewInode(ctx), "anon_inode:[signalfd]")
+ return fs.NewFile(ctx, dirent, fs.FileFlags{Read: true, Write: true}, &SignalOperations{
+ target: t,
+ mask: mask,
+ }), nil
+}
+
+// Release implements fs.FileOperations.Release.
+func (s *SignalOperations) Release() {}
+
+// Mask returns the signal mask.
+func (s *SignalOperations) Mask() linux.SignalSet {
+ s.mu.Lock()
+ mask := s.mask
+ s.mu.Unlock()
+ return mask
+}
+
+// SetMask sets the signal mask.
+func (s *SignalOperations) SetMask(mask linux.SignalSet) {
+ s.mu.Lock()
+ s.mask = mask
+ s.mu.Unlock()
+}
+
+// Read implements fs.FileOperations.Read.
+func (s *SignalOperations) Read(ctx context.Context, _ *fs.File, dst usermem.IOSequence, _ int64) (int64, error) {
+ // Attempt to dequeue relevant signals.
+ info, err := s.target.Sigtimedwait(s.Mask(), 0)
+ if err != nil {
+ // There must be no signal available.
+ return 0, syserror.ErrWouldBlock
+ }
+
+ // Copy out the signal info using the specified format.
+ var buf [128]byte
+ binary.Marshal(buf[:0], usermem.ByteOrder, &linux.SignalfdSiginfo{
+ Signo: uint32(info.Signo),
+ Errno: info.Errno,
+ Code: info.Code,
+ PID: uint32(info.Pid()),
+ UID: uint32(info.Uid()),
+ Status: info.Status(),
+ Overrun: uint32(info.Overrun()),
+ Addr: info.Addr(),
+ })
+ n, err := dst.CopyOut(ctx, buf[:])
+ return int64(n), err
+}
+
+// Readiness implements waiter.Waitable.Readiness.
+func (s *SignalOperations) Readiness(mask waiter.EventMask) waiter.EventMask {
+ return mask & waiter.EventIn
+}
+
+// EventRegister implements waiter.Waitable.EventRegister.
+func (s *SignalOperations) EventRegister(entry *waiter.Entry, _ waiter.EventMask) {
+ // Register for the signal set; ignore the passed events.
+ s.target.SignalRegister(entry, waiter.EventMask(s.Mask()))
+}
+
+// EventUnregister implements waiter.Waitable.EventUnregister.
+func (s *SignalOperations) EventUnregister(entry *waiter.Entry) {
+ // Unregister the original entry.
+ s.target.SignalUnregister(entry)
+}
diff --git a/pkg/sentry/kernel/task.go b/pkg/sentry/kernel/task.go
index e91f82bb3..c82ef5486 100644
--- a/pkg/sentry/kernel/task.go
+++ b/pkg/sentry/kernel/task.go
@@ -35,6 +35,7 @@ import (
"gvisor.dev/gvisor/pkg/sentry/uniqueid"
"gvisor.dev/gvisor/pkg/sentry/usage"
"gvisor.dev/gvisor/pkg/sentry/usermem"
+ "gvisor.dev/gvisor/pkg/waiter"
"gvisor.dev/gvisor/third_party/gvsync"
)
@@ -133,6 +134,13 @@ type Task struct {
// signalStack is exclusive to the task goroutine.
signalStack arch.SignalStack
+ // signalQueue is a set of registered waiters for signal-related events.
+ //
+ // signalQueue is protected by the signalMutex. Note that the task does
+ // not implement all queue methods, specifically the readiness checks.
+ // The task only broadcast a notification on signal delivery.
+ signalQueue waiter.Queue `state:"zerovalue"`
+
// If groupStopPending is true, the task should participate in a group
// stop in the interrupt path.
//
diff --git a/pkg/sentry/kernel/task_signals.go b/pkg/sentry/kernel/task_signals.go
index 266959a07..39cd1340d 100644
--- a/pkg/sentry/kernel/task_signals.go
+++ b/pkg/sentry/kernel/task_signals.go
@@ -28,6 +28,7 @@ import (
ucspb "gvisor.dev/gvisor/pkg/sentry/kernel/uncaught_signal_go_proto"
"gvisor.dev/gvisor/pkg/sentry/usermem"
"gvisor.dev/gvisor/pkg/syserror"
+ "gvisor.dev/gvisor/pkg/waiter"
)
// SignalAction is an internal signal action.
@@ -497,6 +498,9 @@ func (tg *ThreadGroup) applySignalSideEffectsLocked(sig linux.Signal) {
//
// Preconditions: The signal mutex must be locked.
func (t *Task) canReceiveSignalLocked(sig linux.Signal) bool {
+ // Notify that the signal is queued.
+ t.signalQueue.Notify(waiter.EventMask(linux.MakeSignalSet(sig)))
+
// - Do not choose tasks that are blocking the signal.
if linux.SignalSetOf(sig)&t.signalMask != 0 {
return false
@@ -1108,3 +1112,17 @@ func (*runInterruptAfterSignalDeliveryStop) execute(t *Task) taskRunState {
t.tg.signalHandlers.mu.Unlock()
return t.deliverSignal(info, act)
}
+
+// SignalRegister registers a waiter for pending signals.
+func (t *Task) SignalRegister(e *waiter.Entry, mask waiter.EventMask) {
+ t.tg.signalHandlers.mu.Lock()
+ t.signalQueue.EventRegister(e, mask)
+ t.tg.signalHandlers.mu.Unlock()
+}
+
+// SignalUnregister unregisters a waiter for pending signals.
+func (t *Task) SignalUnregister(e *waiter.Entry) {
+ t.tg.signalHandlers.mu.Lock()
+ t.signalQueue.EventUnregister(e)
+ t.tg.signalHandlers.mu.Unlock()
+}
diff --git a/pkg/sentry/syscalls/linux/BUILD b/pkg/sentry/syscalls/linux/BUILD
index 33a40b9c6..e76ee27d2 100644
--- a/pkg/sentry/syscalls/linux/BUILD
+++ b/pkg/sentry/syscalls/linux/BUILD
@@ -74,6 +74,7 @@ go_library(
"//pkg/sentry/kernel/pipe",
"//pkg/sentry/kernel/sched",
"//pkg/sentry/kernel/shm",
+ "//pkg/sentry/kernel/signalfd",
"//pkg/sentry/kernel/time",
"//pkg/sentry/limits",
"//pkg/sentry/memmap",
diff --git a/pkg/sentry/syscalls/linux/linux64.go b/pkg/sentry/syscalls/linux/linux64.go
index 150999fb8..18d24ab61 100644
--- a/pkg/sentry/syscalls/linux/linux64.go
+++ b/pkg/sentry/syscalls/linux/linux64.go
@@ -327,14 +327,14 @@ var AMD64 = &kernel.SyscallTable{
279: syscalls.CapError("move_pages", linux.CAP_SYS_NICE, "", nil), // requires cap_sys_nice (mostly)
280: syscalls.Supported("utimensat", Utimensat),
281: syscalls.Supported("epoll_pwait", EpollPwait),
- 282: syscalls.ErrorWithEvent("signalfd", syserror.ENOSYS, "", []string{"gvisor.dev/issue/139"}), // TODO(b/19846426)
+ 282: syscalls.PartiallySupported("signalfd", Signalfd, "Semantics are slightly different.", []string{"gvisor.dev/issue/139"}),
283: syscalls.Supported("timerfd_create", TimerfdCreate),
284: syscalls.Supported("eventfd", Eventfd),
285: syscalls.PartiallySupported("fallocate", Fallocate, "Not all options are supported.", nil),
286: syscalls.Supported("timerfd_settime", TimerfdSettime),
287: syscalls.Supported("timerfd_gettime", TimerfdGettime),
288: syscalls.Supported("accept4", Accept4),
- 289: syscalls.ErrorWithEvent("signalfd4", syserror.ENOSYS, "", []string{"gvisor.dev/issue/139"}), // TODO(b/19846426)
+ 289: syscalls.PartiallySupported("signalfd4", Signalfd4, "Semantics are slightly different.", []string{"gvisor.dev/issue/139"}),
290: syscalls.Supported("eventfd2", Eventfd2),
291: syscalls.Supported("epoll_create1", EpollCreate1),
292: syscalls.Supported("dup3", Dup3),
diff --git a/pkg/sentry/syscalls/linux/sys_signal.go b/pkg/sentry/syscalls/linux/sys_signal.go
index 0104a94c0..fb6efd5d8 100644
--- a/pkg/sentry/syscalls/linux/sys_signal.go
+++ b/pkg/sentry/syscalls/linux/sys_signal.go
@@ -20,7 +20,10 @@ import (
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/sentry/arch"
+ "gvisor.dev/gvisor/pkg/sentry/fs"
"gvisor.dev/gvisor/pkg/sentry/kernel"
+ "gvisor.dev/gvisor/pkg/sentry/kernel/signalfd"
+ "gvisor.dev/gvisor/pkg/sentry/usermem"
"gvisor.dev/gvisor/pkg/syserror"
)
@@ -506,3 +509,77 @@ func RestartSyscall(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kerne
t.Debugf("Restart block missing in restart_syscall(2). Did ptrace inject a return value of ERESTART_RESTARTBLOCK?")
return 0, nil, syserror.EINTR
}
+
+// sharedSignalfd is shared between the two calls.
+func sharedSignalfd(t *kernel.Task, fd int32, sigset usermem.Addr, sigsetsize uint, flags int32) (uintptr, *kernel.SyscallControl, error) {
+ // Copy in the signal mask.
+ mask, err := copyInSigSet(t, sigset, sigsetsize)
+ if err != nil {
+ return 0, nil, err
+ }
+
+ // Always check for valid flags, even if not creating.
+ if flags&^(linux.SFD_NONBLOCK|linux.SFD_CLOEXEC) != 0 {
+ return 0, nil, syserror.EINVAL
+ }
+
+ // Is this a change to an existing signalfd?
+ //
+ // The spec indicates that this should adjust the mask.
+ if fd != -1 {
+ file := t.GetFile(fd)
+ if file == nil {
+ return 0, nil, syserror.EBADF
+ }
+ defer file.DecRef()
+
+ // Is this a signalfd?
+ if s, ok := file.FileOperations.(*signalfd.SignalOperations); ok {
+ s.SetMask(mask)
+ return 0, nil, nil
+ }
+
+ // Not a signalfd.
+ return 0, nil, syserror.EINVAL
+ }
+
+ // Create a new file.
+ file, err := signalfd.New(t, mask)
+ if err != nil {
+ return 0, nil, err
+ }
+ defer file.DecRef()
+
+ // Set appropriate flags.
+ file.SetFlags(fs.SettableFileFlags{
+ NonBlocking: flags&linux.SFD_NONBLOCK != 0,
+ })
+
+ // Create a new descriptor.
+ fd, err = t.NewFDFrom(0, file, kernel.FDFlags{
+ CloseOnExec: flags&linux.SFD_CLOEXEC != 0,
+ })
+ if err != nil {
+ return 0, nil, err
+ }
+
+ // Done.
+ return uintptr(fd), nil, nil
+}
+
+// Signalfd implements the linux syscall signalfd(2).
+func Signalfd(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel.SyscallControl, error) {
+ fd := args[0].Int()
+ sigset := args[1].Pointer()
+ sigsetsize := args[2].SizeT()
+ return sharedSignalfd(t, fd, sigset, sigsetsize, 0)
+}
+
+// Signalfd4 implements the linux syscall signalfd4(2).
+func Signalfd4(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel.SyscallControl, error) {
+ fd := args[0].Int()
+ sigset := args[1].Pointer()
+ sigsetsize := args[2].SizeT()
+ flags := args[3].Int()
+ return sharedSignalfd(t, fd, sigset, sigsetsize, flags)
+}
diff --git a/test/syscalls/linux/BUILD b/test/syscalls/linux/BUILD
index eac32850d..56fe7be37 100644
--- a/test/syscalls/linux/BUILD
+++ b/test/syscalls/linux/BUILD
@@ -1964,6 +1964,24 @@ cc_binary(
)
cc_binary(
+ name = "signalfd_test",
+ testonly = 1,
+ srcs = ["signalfd.cc"],
+ linkstatic = 1,
+ deps = [
+ "//test/util:file_descriptor",
+ "//test/util:logging",
+ "//test/util:posix_error",
+ "//test/util:signal_util",
+ "//test/util:test_main",
+ "//test/util:test_util",
+ "//test/util:thread_util",
+ "@com_google_absl//absl/synchronization",
+ "@com_google_googletest//:gtest",
+ ],
+)
+
+cc_binary(
name = "sigprocmask_test",
testonly = 1,
srcs = ["sigprocmask.cc"],
diff --git a/test/syscalls/linux/signalfd.cc b/test/syscalls/linux/signalfd.cc
new file mode 100644
index 000000000..54c598627
--- /dev/null
+++ b/test/syscalls/linux/signalfd.cc
@@ -0,0 +1,333 @@
+// 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 <errno.h>
+#include <poll.h>
+#include <signal.h>
+#include <stdio.h>
+#include <string.h>
+#include <sys/signalfd.h>
+#include <unistd.h>
+
+#include <functional>
+#include <vector>
+
+#include "gtest/gtest.h"
+#include "gtest/gtest.h"
+#include "absl/synchronization/mutex.h"
+#include "test/util/file_descriptor.h"
+#include "test/util/posix_error.h"
+#include "test/util/signal_util.h"
+#include "test/util/test_util.h"
+#include "test/util/thread_util.h"
+
+using ::testing::KilledBySignal;
+
+namespace gvisor {
+namespace testing {
+
+namespace {
+
+constexpr int kSigno = SIGUSR1;
+constexpr int kSignoAlt = SIGUSR2;
+
+// Returns a new signalfd.
+inline PosixErrorOr<FileDescriptor> NewSignalFD(sigset_t* mask, int flags = 0) {
+ int fd = signalfd(-1, mask, flags);
+ MaybeSave();
+ if (fd < 0) {
+ return PosixError(errno, "signalfd");
+ }
+ return FileDescriptor(fd);
+}
+
+TEST(Signalfd, Basic) {
+ // Create the signalfd.
+ sigset_t mask;
+ sigemptyset(&mask);
+ sigaddset(&mask, kSigno);
+ FileDescriptor fd = ASSERT_NO_ERRNO_AND_VALUE(NewSignalFD(&mask, 0));
+
+ // Deliver the blocked signal.
+ const auto scoped_sigmask =
+ ASSERT_NO_ERRNO_AND_VALUE(ScopedSignalMask(SIG_BLOCK, kSigno));
+ ASSERT_THAT(tgkill(getpid(), gettid(), kSigno), SyscallSucceeds());
+
+ // We should now read the signal.
+ struct signalfd_siginfo rbuf;
+ ASSERT_THAT(read(fd.get(), &rbuf, sizeof(rbuf)),
+ SyscallSucceedsWithValue(sizeof(rbuf)));
+ EXPECT_EQ(rbuf.ssi_signo, kSigno);
+}
+
+TEST(Signalfd, MaskWorks) {
+ // Create two signalfds with different masks.
+ sigset_t mask1, mask2;
+ sigemptyset(&mask1);
+ sigemptyset(&mask2);
+ sigaddset(&mask1, kSigno);
+ sigaddset(&mask2, kSignoAlt);
+ FileDescriptor fd1 = ASSERT_NO_ERRNO_AND_VALUE(NewSignalFD(&mask1, 0));
+ FileDescriptor fd2 = ASSERT_NO_ERRNO_AND_VALUE(NewSignalFD(&mask2, 0));
+
+ // Deliver the two signals.
+ const auto scoped_sigmask1 =
+ ASSERT_NO_ERRNO_AND_VALUE(ScopedSignalMask(SIG_BLOCK, kSigno));
+ const auto scoped_sigmask2 =
+ ASSERT_NO_ERRNO_AND_VALUE(ScopedSignalMask(SIG_BLOCK, kSignoAlt));
+ ASSERT_THAT(tgkill(getpid(), gettid(), kSigno), SyscallSucceeds());
+ ASSERT_THAT(tgkill(getpid(), gettid(), kSignoAlt), SyscallSucceeds());
+
+ // We should see the signals on the appropriate signalfds.
+ //
+ // We read in the opposite order as the signals deliver above, to ensure that
+ // we don't happen to read the correct signal from the correct signalfd.
+ struct signalfd_siginfo rbuf1, rbuf2;
+ ASSERT_THAT(read(fd2.get(), &rbuf2, sizeof(rbuf2)),
+ SyscallSucceedsWithValue(sizeof(rbuf2)));
+ EXPECT_EQ(rbuf2.ssi_signo, kSignoAlt);
+ ASSERT_THAT(read(fd1.get(), &rbuf1, sizeof(rbuf1)),
+ SyscallSucceedsWithValue(sizeof(rbuf1)));
+ EXPECT_EQ(rbuf1.ssi_signo, kSigno);
+}
+
+TEST(Signalfd, Cloexec) {
+ // Exec tests confirm that O_CLOEXEC has the intended effect. We just create a
+ // signalfd with the appropriate flag here and assert that the FD has it set.
+ sigset_t mask;
+ sigemptyset(&mask);
+ FileDescriptor fd =
+ ASSERT_NO_ERRNO_AND_VALUE(NewSignalFD(&mask, SFD_CLOEXEC));
+ EXPECT_THAT(fcntl(fd.get(), F_GETFD), SyscallSucceedsWithValue(FD_CLOEXEC));
+}
+
+TEST(Signalfd, Blocking) {
+ // Create the signalfd in blocking mode.
+ sigset_t mask;
+ sigemptyset(&mask);
+ sigaddset(&mask, kSigno);
+ FileDescriptor fd = ASSERT_NO_ERRNO_AND_VALUE(NewSignalFD(&mask, 0));
+
+ // Shared tid variable.
+ absl::Mutex mu;
+ bool has_tid;
+ pid_t tid;
+
+ // Start a thread reading.
+ ScopedThread t([&] {
+ // Copy the tid and notify the caller.
+ {
+ absl::MutexLock ml(&mu);
+ tid = gettid();
+ has_tid = true;
+ }
+
+ // Read the signal from the signalfd.
+ struct signalfd_siginfo rbuf;
+ ASSERT_THAT(read(fd.get(), &rbuf, sizeof(rbuf)),
+ SyscallSucceedsWithValue(sizeof(rbuf)));
+ EXPECT_EQ(rbuf.ssi_signo, kSigno);
+ });
+
+ // Wait until blocked.
+ absl::MutexLock ml(&mu);
+ mu.Await(absl::Condition(&has_tid));
+
+ // Deliver the signal to either the waiting thread, or
+ // to this thread. N.B. this is a bug in the core gVisor
+ // behavior for signalfd, and needs to be fixed.
+ //
+ // See gvisor.dev/issue/139.
+ if (IsRunningOnGvisor()) {
+ ASSERT_THAT(tgkill(getpid(), gettid(), kSigno), SyscallSucceeds());
+ } else {
+ ASSERT_THAT(tgkill(getpid(), tid, kSigno), SyscallSucceeds());
+ }
+
+ // Ensure that it was received.
+ t.Join();
+}
+
+TEST(Signalfd, ThreadGroup) {
+ // Create the signalfd in blocking mode.
+ sigset_t mask;
+ sigemptyset(&mask);
+ sigaddset(&mask, kSigno);
+ FileDescriptor fd = ASSERT_NO_ERRNO_AND_VALUE(NewSignalFD(&mask, 0));
+
+ // Shared variable.
+ absl::Mutex mu;
+ bool first = false;
+ bool second = false;
+
+ // Start a thread reading.
+ ScopedThread t([&] {
+ // Read the signal from the signalfd.
+ struct signalfd_siginfo rbuf;
+ ASSERT_THAT(read(fd.get(), &rbuf, sizeof(rbuf)),
+ SyscallSucceedsWithValue(sizeof(rbuf)));
+ EXPECT_EQ(rbuf.ssi_signo, kSigno);
+
+ // Wait for the other thread.
+ absl::MutexLock ml(&mu);
+ first = true;
+ mu.Await(absl::Condition(&second));
+ });
+
+ // Deliver the signal to the threadgroup.
+ ASSERT_THAT(kill(getpid(), kSigno), SyscallSucceeds());
+
+ // Wait for the first thread to process.
+ {
+ absl::MutexLock ml(&mu);
+ mu.Await(absl::Condition(&first));
+ }
+
+ // Deliver to the thread group again (other thread still exists).
+ ASSERT_THAT(kill(getpid(), kSigno), SyscallSucceeds());
+
+ // Ensure that we can also receive it.
+ struct signalfd_siginfo rbuf;
+ ASSERT_THAT(read(fd.get(), &rbuf, sizeof(rbuf)),
+ SyscallSucceedsWithValue(sizeof(rbuf)));
+ EXPECT_EQ(rbuf.ssi_signo, kSigno);
+
+ // Mark the test as done.
+ {
+ absl::MutexLock ml(&mu);
+ second = true;
+ }
+
+ // The other thread should be joinable.
+ t.Join();
+}
+
+TEST(Signalfd, Nonblock) {
+ // Create the signalfd in non-blocking mode.
+ sigset_t mask;
+ sigemptyset(&mask);
+ sigaddset(&mask, kSigno);
+ FileDescriptor fd =
+ ASSERT_NO_ERRNO_AND_VALUE(NewSignalFD(&mask, SFD_NONBLOCK));
+
+ // We should return if we attempt to read.
+ struct signalfd_siginfo rbuf;
+ ASSERT_THAT(read(fd.get(), &rbuf, sizeof(rbuf)),
+ SyscallFailsWithErrno(EWOULDBLOCK));
+
+ // Block and deliver the signal.
+ const auto scoped_sigmask =
+ ASSERT_NO_ERRNO_AND_VALUE(ScopedSignalMask(SIG_BLOCK, kSigno));
+ ASSERT_THAT(tgkill(getpid(), gettid(), kSigno), SyscallSucceeds());
+
+ // Ensure that a read actually works.
+ ASSERT_THAT(read(fd.get(), &rbuf, sizeof(rbuf)),
+ SyscallSucceedsWithValue(sizeof(rbuf)));
+ EXPECT_EQ(rbuf.ssi_signo, kSigno);
+
+ // Should block again.
+ EXPECT_THAT(read(fd.get(), &rbuf, sizeof(rbuf)),
+ SyscallFailsWithErrno(EWOULDBLOCK));
+}
+
+TEST(Signalfd, SetMask) {
+ // Create the signalfd matching nothing.
+ sigset_t mask;
+ sigemptyset(&mask);
+ FileDescriptor fd =
+ ASSERT_NO_ERRNO_AND_VALUE(NewSignalFD(&mask, SFD_NONBLOCK));
+
+ // Block and deliver a signal.
+ const auto scoped_sigmask =
+ ASSERT_NO_ERRNO_AND_VALUE(ScopedSignalMask(SIG_BLOCK, kSigno));
+ ASSERT_THAT(tgkill(getpid(), gettid(), kSigno), SyscallSucceeds());
+
+ // We should have nothing.
+ struct signalfd_siginfo rbuf;
+ ASSERT_THAT(read(fd.get(), &rbuf, sizeof(rbuf)),
+ SyscallFailsWithErrno(EWOULDBLOCK));
+
+ // Change the signal mask.
+ sigaddset(&mask, kSigno);
+ ASSERT_THAT(signalfd(fd.get(), &mask, 0), SyscallSucceeds());
+
+ // We should now have the signal.
+ ASSERT_THAT(read(fd.get(), &rbuf, sizeof(rbuf)),
+ SyscallSucceedsWithValue(sizeof(rbuf)));
+ EXPECT_EQ(rbuf.ssi_signo, kSigno);
+}
+
+TEST(Signalfd, Poll) {
+ // Create the signalfd.
+ sigset_t mask;
+ sigemptyset(&mask);
+ sigaddset(&mask, kSigno);
+ FileDescriptor fd = ASSERT_NO_ERRNO_AND_VALUE(NewSignalFD(&mask, 0));
+
+ // Block the signal, and start a thread to deliver it.
+ const auto scoped_sigmask =
+ ASSERT_NO_ERRNO_AND_VALUE(ScopedSignalMask(SIG_BLOCK, kSigno));
+ pid_t orig_tid = gettid();
+ ScopedThread t([&] {
+ absl::SleepFor(absl::Seconds(5));
+ ASSERT_THAT(tgkill(getpid(), orig_tid, kSigno), SyscallSucceeds());
+ });
+
+ // Start polling for the signal. We expect that it is not available at the
+ // outset, but then becomes available when the signal is sent. We give a
+ // timeout of 10000ms (or the delay above + 5 seconds of additional grace
+ // time).
+ struct pollfd poll_fd = {fd.get(), POLLIN, 0};
+ EXPECT_THAT(RetryEINTR(poll)(&poll_fd, 1, 10000),
+ SyscallSucceedsWithValue(1));
+
+ // Actually read the signal to prevent delivery.
+ struct signalfd_siginfo rbuf;
+ EXPECT_THAT(read(fd.get(), &rbuf, sizeof(rbuf)),
+ SyscallSucceedsWithValue(sizeof(rbuf)));
+}
+
+TEST(Signalfd, KillStillKills) {
+ sigset_t mask;
+ sigemptyset(&mask);
+ sigaddset(&mask, SIGKILL);
+ FileDescriptor fd =
+ ASSERT_NO_ERRNO_AND_VALUE(NewSignalFD(&mask, SFD_CLOEXEC));
+
+ // Just because there is a signalfd, we shouldn't see any change in behavior
+ // for unblockable signals. It's easier to test this with SIGKILL.
+ const auto scoped_sigmask =
+ ASSERT_NO_ERRNO_AND_VALUE(ScopedSignalMask(SIG_BLOCK, SIGKILL));
+ EXPECT_EXIT(tgkill(getpid(), gettid(), SIGKILL), KilledBySignal(SIGKILL), "");
+}
+
+} // namespace
+
+} // namespace testing
+} // namespace gvisor
+
+int main(int argc, char** argv) {
+ // These tests depend on delivering signals. Block them up front so that all
+ // other threads created by TestInit will also have them blocked, and they
+ // will not interface with the rest of the test.
+ sigset_t set;
+ sigemptyset(&set);
+ sigaddset(&set, gvisor::testing::kSigno);
+ sigaddset(&set, gvisor::testing::kSignoAlt);
+ TEST_PCHECK(sigprocmask(SIG_BLOCK, &set, nullptr) == 0);
+
+ gvisor::testing::TestInit(&argc, &argv);
+
+ return RUN_ALL_TESTS();
+}