Signalfd support

Note that the exact semantics for these signalfds are slightly different from
Linux. These signalfds are bound to the process at creation time. Reads, polls,
etc. are all associated with signals directed at that task. In Linux, all
signalfd operations are associated with current, regardless of where the
signalfd originated.

In practice, this should not be an issue given how signalfds are used. In order
to fix this however, we will need to plumb the context through all the event
APIs. This gets complicated really quickly, because the waiter APIs are all
netstack-specific, and not generally exposed to the context.  Probably not
worthwhile fixing immediately.

PiperOrigin-RevId: 269901749
This commit is contained in:
Adin Scannell 2019-09-18 15:15:16 -07:00 committed by gVisor bot
parent 461123ea35
commit c98e7f0d19
11 changed files with 662 additions and 2 deletions

View File

@ -46,6 +46,7 @@ go_library(
"sem.go", "sem.go",
"shm.go", "shm.go",
"signal.go", "signal.go",
"signalfd.go",
"socket.go", "socket.go",
"splice.go", "splice.go",
"tcp.go", "tcp.go",

45
pkg/abi/linux/signalfd.go Normal file
View File

@ -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
}

View File

@ -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",
],
)

View File

@ -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)
}

View File

@ -35,6 +35,7 @@ import (
"gvisor.dev/gvisor/pkg/sentry/uniqueid" "gvisor.dev/gvisor/pkg/sentry/uniqueid"
"gvisor.dev/gvisor/pkg/sentry/usage" "gvisor.dev/gvisor/pkg/sentry/usage"
"gvisor.dev/gvisor/pkg/sentry/usermem" "gvisor.dev/gvisor/pkg/sentry/usermem"
"gvisor.dev/gvisor/pkg/waiter"
"gvisor.dev/gvisor/third_party/gvsync" "gvisor.dev/gvisor/third_party/gvsync"
) )
@ -133,6 +134,13 @@ type Task struct {
// signalStack is exclusive to the task goroutine. // signalStack is exclusive to the task goroutine.
signalStack arch.SignalStack 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 // If groupStopPending is true, the task should participate in a group
// stop in the interrupt path. // stop in the interrupt path.
// //

View File

@ -28,6 +28,7 @@ import (
ucspb "gvisor.dev/gvisor/pkg/sentry/kernel/uncaught_signal_go_proto" ucspb "gvisor.dev/gvisor/pkg/sentry/kernel/uncaught_signal_go_proto"
"gvisor.dev/gvisor/pkg/sentry/usermem" "gvisor.dev/gvisor/pkg/sentry/usermem"
"gvisor.dev/gvisor/pkg/syserror" "gvisor.dev/gvisor/pkg/syserror"
"gvisor.dev/gvisor/pkg/waiter"
) )
// SignalAction is an internal signal action. // SignalAction is an internal signal action.
@ -497,6 +498,9 @@ func (tg *ThreadGroup) applySignalSideEffectsLocked(sig linux.Signal) {
// //
// Preconditions: The signal mutex must be locked. // Preconditions: The signal mutex must be locked.
func (t *Task) canReceiveSignalLocked(sig linux.Signal) bool { 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. // - Do not choose tasks that are blocking the signal.
if linux.SignalSetOf(sig)&t.signalMask != 0 { if linux.SignalSetOf(sig)&t.signalMask != 0 {
return false return false
@ -1108,3 +1112,17 @@ func (*runInterruptAfterSignalDeliveryStop) execute(t *Task) taskRunState {
t.tg.signalHandlers.mu.Unlock() t.tg.signalHandlers.mu.Unlock()
return t.deliverSignal(info, act) 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()
}

View File

@ -74,6 +74,7 @@ go_library(
"//pkg/sentry/kernel/pipe", "//pkg/sentry/kernel/pipe",
"//pkg/sentry/kernel/sched", "//pkg/sentry/kernel/sched",
"//pkg/sentry/kernel/shm", "//pkg/sentry/kernel/shm",
"//pkg/sentry/kernel/signalfd",
"//pkg/sentry/kernel/time", "//pkg/sentry/kernel/time",
"//pkg/sentry/limits", "//pkg/sentry/limits",
"//pkg/sentry/memmap", "//pkg/sentry/memmap",

View File

@ -327,14 +327,14 @@ var AMD64 = &kernel.SyscallTable{
279: syscalls.CapError("move_pages", linux.CAP_SYS_NICE, "", nil), // requires cap_sys_nice (mostly) 279: syscalls.CapError("move_pages", linux.CAP_SYS_NICE, "", nil), // requires cap_sys_nice (mostly)
280: syscalls.Supported("utimensat", Utimensat), 280: syscalls.Supported("utimensat", Utimensat),
281: syscalls.Supported("epoll_pwait", EpollPwait), 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), 283: syscalls.Supported("timerfd_create", TimerfdCreate),
284: syscalls.Supported("eventfd", Eventfd), 284: syscalls.Supported("eventfd", Eventfd),
285: syscalls.PartiallySupported("fallocate", Fallocate, "Not all options are supported.", nil), 285: syscalls.PartiallySupported("fallocate", Fallocate, "Not all options are supported.", nil),
286: syscalls.Supported("timerfd_settime", TimerfdSettime), 286: syscalls.Supported("timerfd_settime", TimerfdSettime),
287: syscalls.Supported("timerfd_gettime", TimerfdGettime), 287: syscalls.Supported("timerfd_gettime", TimerfdGettime),
288: syscalls.Supported("accept4", Accept4), 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), 290: syscalls.Supported("eventfd2", Eventfd2),
291: syscalls.Supported("epoll_create1", EpollCreate1), 291: syscalls.Supported("epoll_create1", EpollCreate1),
292: syscalls.Supported("dup3", Dup3), 292: syscalls.Supported("dup3", Dup3),

View File

@ -20,7 +20,10 @@ import (
"gvisor.dev/gvisor/pkg/abi/linux" "gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/sentry/arch" "gvisor.dev/gvisor/pkg/sentry/arch"
"gvisor.dev/gvisor/pkg/sentry/fs"
"gvisor.dev/gvisor/pkg/sentry/kernel" "gvisor.dev/gvisor/pkg/sentry/kernel"
"gvisor.dev/gvisor/pkg/sentry/kernel/signalfd"
"gvisor.dev/gvisor/pkg/sentry/usermem"
"gvisor.dev/gvisor/pkg/syserror" "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?") t.Debugf("Restart block missing in restart_syscall(2). Did ptrace inject a return value of ERESTART_RESTARTBLOCK?")
return 0, nil, syserror.EINTR 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)
}

View File

@ -1963,6 +1963,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( cc_binary(
name = "sigprocmask_test", name = "sigprocmask_test",
testonly = 1, testonly = 1,

View File

@ -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();
}