summaryrefslogtreecommitdiffhomepage
path: root/pkg/sentry/kernel
diff options
context:
space:
mode:
Diffstat (limited to 'pkg/sentry/kernel')
-rw-r--r--pkg/sentry/kernel/BUILD1
-rw-r--r--pkg/sentry/kernel/abstract_socket_namespace.go8
-rw-r--r--pkg/sentry/kernel/ipc/BUILD20
-rw-r--r--pkg/sentry/kernel/ipc/object.go115
-rw-r--r--pkg/sentry/kernel/ipc/registry.go196
-rw-r--r--pkg/sentry/kernel/ipc_namespace.go8
-rw-r--r--pkg/sentry/kernel/kernel.go6
-rw-r--r--pkg/sentry/kernel/kernel_opts.go3
-rw-r--r--pkg/sentry/kernel/msgqueue/BUILD36
-rw-r--r--pkg/sentry/kernel/msgqueue/msgqueue.go220
-rw-r--r--pkg/sentry/kernel/ptrace.go18
-rw-r--r--pkg/sentry/kernel/ptrace_amd64.go1
-rw-r--r--pkg/sentry/kernel/ptrace_arm64.go1
-rw-r--r--pkg/sentry/kernel/rseq.go5
-rw-r--r--pkg/sentry/kernel/semaphore/BUILD13
-rw-r--r--pkg/sentry/kernel/semaphore/semaphore.go272
-rw-r--r--pkg/sentry/kernel/semaphore/semaphore_test.go20
-rw-r--r--pkg/sentry/kernel/sessions.go3
-rw-r--r--pkg/sentry/kernel/shm/BUILD1
-rw-r--r--pkg/sentry/kernel/shm/shm.go261
-rw-r--r--pkg/sentry/kernel/task.go2
-rw-r--r--pkg/sentry/kernel/task_cgroup.go2
-rw-r--r--pkg/sentry/kernel/task_clone.go233
-rw-r--r--pkg/sentry/kernel/task_exit.go106
-rw-r--r--pkg/sentry/kernel/task_log.go2
-rw-r--r--pkg/sentry/kernel/task_run.go2
-rw-r--r--pkg/sentry/kernel/task_signals.go17
-rw-r--r--pkg/sentry/kernel/task_syscall.go6
-rw-r--r--pkg/sentry/kernel/task_usermem.go10
-rw-r--r--pkg/sentry/kernel/thread_group.go11
30 files changed, 968 insertions, 631 deletions
diff --git a/pkg/sentry/kernel/BUILD b/pkg/sentry/kernel/BUILD
index 26614b029..c613f4932 100644
--- a/pkg/sentry/kernel/BUILD
+++ b/pkg/sentry/kernel/BUILD
@@ -256,6 +256,7 @@ go_library(
"//pkg/sentry/kernel/auth",
"//pkg/sentry/kernel/epoll",
"//pkg/sentry/kernel/futex",
+ "//pkg/sentry/kernel/msgqueue",
"//pkg/sentry/kernel/sched",
"//pkg/sentry/kernel/semaphore",
"//pkg/sentry/kernel/shm",
diff --git a/pkg/sentry/kernel/abstract_socket_namespace.go b/pkg/sentry/kernel/abstract_socket_namespace.go
index d100e58d7..5d86a04f3 100644
--- a/pkg/sentry/kernel/abstract_socket_namespace.go
+++ b/pkg/sentry/kernel/abstract_socket_namespace.go
@@ -27,7 +27,7 @@ import (
// +stateify savable
type abstractEndpoint struct {
ep transport.BoundEndpoint
- socket refsvfs2.RefCounter
+ socket refsvfs2.TryRefCounter
name string
ns *AbstractSocketNamespace
}
@@ -57,7 +57,7 @@ func NewAbstractSocketNamespace() *AbstractSocketNamespace {
// its backing socket.
type boundEndpoint struct {
transport.BoundEndpoint
- socket refsvfs2.RefCounter
+ socket refsvfs2.TryRefCounter
}
// Release implements transport.BoundEndpoint.Release.
@@ -89,7 +89,7 @@ func (a *AbstractSocketNamespace) BoundEndpoint(name string) transport.BoundEndp
//
// When the last reference managed by socket is dropped, ep may be removed from the
// namespace.
-func (a *AbstractSocketNamespace) Bind(ctx context.Context, name string, ep transport.BoundEndpoint, socket refsvfs2.RefCounter) error {
+func (a *AbstractSocketNamespace) Bind(ctx context.Context, name string, ep transport.BoundEndpoint, socket refsvfs2.TryRefCounter) error {
a.mu.Lock()
defer a.mu.Unlock()
@@ -109,7 +109,7 @@ func (a *AbstractSocketNamespace) Bind(ctx context.Context, name string, ep tran
// Remove removes the specified socket at name from the abstract socket
// namespace, if it has not yet been replaced.
-func (a *AbstractSocketNamespace) Remove(name string, socket refsvfs2.RefCounter) {
+func (a *AbstractSocketNamespace) Remove(name string, socket refsvfs2.TryRefCounter) {
a.mu.Lock()
defer a.mu.Unlock()
diff --git a/pkg/sentry/kernel/ipc/BUILD b/pkg/sentry/kernel/ipc/BUILD
new file mode 100644
index 000000000..e42a94e15
--- /dev/null
+++ b/pkg/sentry/kernel/ipc/BUILD
@@ -0,0 +1,20 @@
+load("//tools:defs.bzl", "go_library")
+
+package(licenses = ["notice"])
+
+go_library(
+ name = "ipc",
+ srcs = [
+ "object.go",
+ "registry.go",
+ ],
+ visibility = ["//pkg/sentry:internal"],
+ deps = [
+ "//pkg/abi/linux",
+ "//pkg/context",
+ "//pkg/errors/linuxerr",
+ "//pkg/log",
+ "//pkg/sentry/fs",
+ "//pkg/sentry/kernel/auth",
+ ],
+)
diff --git a/pkg/sentry/kernel/ipc/object.go b/pkg/sentry/kernel/ipc/object.go
new file mode 100644
index 000000000..387b35e7e
--- /dev/null
+++ b/pkg/sentry/kernel/ipc/object.go
@@ -0,0 +1,115 @@
+// Copyright 2021 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 ipc defines functionality and utilities common to sysvipc mechanisms.
+//
+// Lock ordering: [shm/semaphore/msgqueue].Registry.mu -> Mechanism
+package ipc
+
+import (
+ "gvisor.dev/gvisor/pkg/abi/linux"
+ "gvisor.dev/gvisor/pkg/sentry/fs"
+ "gvisor.dev/gvisor/pkg/sentry/kernel/auth"
+)
+
+// Key is a user-provided identifier for IPC objects.
+type Key int32
+
+// ID is a kernel identifier for IPC objects.
+type ID int32
+
+// Object represents an abstract IPC object with fields common to all IPC
+// mechanisms.
+//
+// +stateify savable
+type Object struct {
+ // User namespace which owns the IPC namespace which owns the IPC object.
+ // Immutable.
+ UserNS *auth.UserNamespace
+
+ // ID is a kernel identifier for the IPC object. Immutable.
+ ID ID
+
+ // Key is a user-provided identifier for the IPC object. Immutable.
+ Key Key
+
+ // Creator is the user who created the IPC object. Immutable.
+ Creator fs.FileOwner
+
+ // Owner is the current owner of the IPC object.
+ Owner fs.FileOwner
+
+ // Perms is the access permissions the IPC object.
+ Perms fs.FilePermissions
+}
+
+// Mechanism represents a SysV mechanism that holds an IPC object. It can also
+// be looked at as a container for an ipc.Object, which is by definition a fully
+// functional SysV object.
+type Mechanism interface {
+ // Lock behaves the same as Mutex.Lock on the mechanism.
+ Lock()
+
+ // Unlock behaves the same as Mutex.Unlock on the mechanism.
+ Unlock()
+
+ // Object returns a pointer to the mechanism's ipc.Object. Mechanism.Lock,
+ // and Mechanism.Unlock should be used when the object is used.
+ Object() *Object
+
+ // Destroy destroys the mechanism.
+ Destroy()
+}
+
+// NewObject returns a new, initialized ipc.Object. The newly returned object
+// doesn't have a valid ID. When the object is registered, the registry assigns
+// it a new unique ID.
+func NewObject(un *auth.UserNamespace, key Key, creator, owner fs.FileOwner, perms fs.FilePermissions) *Object {
+ return &Object{
+ UserNS: un,
+ Key: key,
+ Creator: creator,
+ Owner: owner,
+ Perms: perms,
+ }
+}
+
+// CheckOwnership verifies whether an IPC object may be accessed using creds as
+// an owner. See ipc/util.c:ipcctl_obtain_check() in Linux.
+func (o *Object) CheckOwnership(creds *auth.Credentials) bool {
+ if o.Owner.UID == creds.EffectiveKUID || o.Creator.UID == creds.EffectiveKUID {
+ return true
+ }
+
+ // Tasks with CAP_SYS_ADMIN may bypass ownership checks. Strangely, Linux
+ // doesn't use CAP_IPC_OWNER for this despite CAP_IPC_OWNER being documented
+ // for use to "override IPC ownership checks".
+ return creds.HasCapabilityIn(linux.CAP_SYS_ADMIN, o.UserNS)
+}
+
+// CheckPermissions verifies whether an IPC object is accessible using creds for
+// access described by req. See ipc/util.c:ipcperms() in Linux.
+func (o *Object) CheckPermissions(creds *auth.Credentials, req fs.PermMask) bool {
+ p := o.Perms.Other
+ if o.Owner.UID == creds.EffectiveKUID {
+ p = o.Perms.User
+ } else if creds.InGroup(o.Owner.GID) {
+ p = o.Perms.Group
+ }
+
+ if p.SupersetOf(req) {
+ return true
+ }
+ return creds.HasCapabilityIn(linux.CAP_IPC_OWNER, o.UserNS)
+}
diff --git a/pkg/sentry/kernel/ipc/registry.go b/pkg/sentry/kernel/ipc/registry.go
new file mode 100644
index 000000000..91de19070
--- /dev/null
+++ b/pkg/sentry/kernel/ipc/registry.go
@@ -0,0 +1,196 @@
+// Copyright 2021 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 ipc
+
+import (
+ "gvisor.dev/gvisor/pkg/abi/linux"
+ "gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
+ "gvisor.dev/gvisor/pkg/log"
+ "gvisor.dev/gvisor/pkg/sentry/fs"
+ "gvisor.dev/gvisor/pkg/sentry/kernel/auth"
+)
+
+// Registry is similar to Object, but for registries. It represent an abstract
+// SysV IPC registry with fields common to all SysV registries. Registry is not
+// thread-safe, and should be protected using a mutex.
+//
+// +stateify savable
+type Registry struct {
+ // UserNS owning the IPC namespace this registry belongs to. Immutable.
+ UserNS *auth.UserNamespace
+
+ // objects is a map of IDs to IPC mechanisms.
+ objects map[ID]Mechanism
+
+ // KeysToIDs maps a lookup key to an ID.
+ keysToIDs map[Key]ID
+
+ // lastIDUsed is used to find the next available ID for object creation.
+ lastIDUsed ID
+}
+
+// NewRegistry return a new, initialized ipc.Registry.
+func NewRegistry(userNS *auth.UserNamespace) *Registry {
+ return &Registry{
+ UserNS: userNS,
+ objects: make(map[ID]Mechanism),
+ keysToIDs: make(map[Key]ID),
+ }
+}
+
+// Find uses key to search for and return a SysV mechanism. Find returns an
+// error if an object is found by shouldn't be, or if the user doesn't have
+// permission to use the object. If no object is found, Find checks create
+// flag, and returns an error only if it's false.
+func (r *Registry) Find(ctx context.Context, key Key, mode linux.FileMode, create, exclusive bool) (Mechanism, error) {
+ if id, ok := r.keysToIDs[key]; ok {
+ mech := r.objects[id]
+ mech.Lock()
+ defer mech.Unlock()
+
+ obj := mech.Object()
+ creds := auth.CredentialsFromContext(ctx)
+ if !obj.CheckPermissions(creds, fs.PermsFromMode(mode)) {
+ // The [calling process / user] does not have permission to access
+ // the set, and does not have the CAP_IPC_OWNER capability in the
+ // user namespace that governs its IPC namespace.
+ return nil, linuxerr.EACCES
+ }
+
+ if create && exclusive {
+ // IPC_CREAT and IPC_EXCL were specified, but an object already
+ // exists for key.
+ return nil, linuxerr.EEXIST
+ }
+ return mech, nil
+ }
+
+ if !create {
+ // No object exists for key and msgflg did not specify IPC_CREAT.
+ return nil, linuxerr.ENOENT
+ }
+
+ return nil, nil
+}
+
+// Register adds the given object into Registry.Objects, and assigns it a new
+// ID. It returns an error if all IDs are exhausted.
+func (r *Registry) Register(m Mechanism) error {
+ id, err := r.newID()
+ if err != nil {
+ return err
+ }
+
+ obj := m.Object()
+ obj.ID = id
+
+ r.objects[id] = m
+ r.keysToIDs[obj.Key] = id
+
+ return nil
+}
+
+// newID finds the first unused ID in the registry, and returns an error if
+// non is found.
+func (r *Registry) newID() (ID, error) {
+ // Find the next available ID.
+ for id := r.lastIDUsed + 1; id != r.lastIDUsed; id++ {
+ // Handle wrap around.
+ if id < 0 {
+ id = 0
+ continue
+ }
+ if r.objects[id] == nil {
+ r.lastIDUsed = id
+ return id, nil
+ }
+ }
+
+ log.Warningf("ids exhausted, they may be leaking")
+
+ // The man pages for shmget(2) mention that ENOSPC should be used if "All
+ // possible shared memory IDs have been taken (SHMMNI)". Other SysV
+ // mechanisms don't have a specific errno for running out of IDs, but they
+ // return ENOSPC if the max number of objects is exceeded, so we assume that
+ // it's the same case.
+ return 0, linuxerr.ENOSPC
+}
+
+// Remove removes the mechanism with the given id from the registry, and calls
+// mechanism.Destroy to perform mechanism-specific removal.
+func (r *Registry) Remove(id ID, creds *auth.Credentials) error {
+ mech := r.objects[id]
+ if mech == nil {
+ return linuxerr.EINVAL
+ }
+
+ mech.Lock()
+ defer mech.Unlock()
+
+ obj := mech.Object()
+
+ // The effective user ID of the calling process must match the creator or
+ // owner of the [mechanism], or the caller must be privileged.
+ if !obj.CheckOwnership(creds) {
+ return linuxerr.EPERM
+ }
+
+ delete(r.objects, obj.ID)
+ delete(r.keysToIDs, obj.Key)
+ mech.Destroy()
+
+ return nil
+}
+
+// ForAllObjects executes a given function for all given objects.
+func (r *Registry) ForAllObjects(f func(o Mechanism)) {
+ for _, o := range r.objects {
+ f(o)
+ }
+}
+
+// FindByID returns the mechanism with the given ID, nil if non exists.
+func (r *Registry) FindByID(id ID) Mechanism {
+ return r.objects[id]
+}
+
+// DissociateKey removes the association between a mechanism and its key
+// (deletes it from r.keysToIDs), preventing it from being discovered by any new
+// process, but not necessarily destroying it. If the given key doesn't exist,
+// nothing is changed.
+func (r *Registry) DissociateKey(key Key) {
+ delete(r.keysToIDs, key)
+}
+
+// DissociateID removes the association between a mechanism and its ID (deletes
+// it from r.objects). An ID can't be removed unless the associated key is
+// removed already, this is done to prevent the users from acquiring nil a
+// Mechanism.
+//
+// Precondition: must be preceded by a call to r.DissociateKey.
+func (r *Registry) DissociateID(id ID) {
+ delete(r.objects, id)
+}
+
+// ObjectCount returns the number of registered objects.
+func (r *Registry) ObjectCount() int {
+ return len(r.objects)
+}
+
+// LastIDUsed returns the last used ID.
+func (r *Registry) LastIDUsed() ID {
+ return r.lastIDUsed
+}
diff --git a/pkg/sentry/kernel/ipc_namespace.go b/pkg/sentry/kernel/ipc_namespace.go
index 9545bb5ef..0b101b1bb 100644
--- a/pkg/sentry/kernel/ipc_namespace.go
+++ b/pkg/sentry/kernel/ipc_namespace.go
@@ -17,6 +17,7 @@ package kernel
import (
"gvisor.dev/gvisor/pkg/context"
"gvisor.dev/gvisor/pkg/sentry/kernel/auth"
+ "gvisor.dev/gvisor/pkg/sentry/kernel/msgqueue"
"gvisor.dev/gvisor/pkg/sentry/kernel/semaphore"
"gvisor.dev/gvisor/pkg/sentry/kernel/shm"
)
@@ -30,6 +31,7 @@ type IPCNamespace struct {
// User namespace which owns this IPC namespace. Immutable.
userNS *auth.UserNamespace
+ queues *msgqueue.Registry
semaphores *semaphore.Registry
shms *shm.Registry
}
@@ -38,6 +40,7 @@ type IPCNamespace struct {
func NewIPCNamespace(userNS *auth.UserNamespace) *IPCNamespace {
ns := &IPCNamespace{
userNS: userNS,
+ queues: msgqueue.NewRegistry(userNS),
semaphores: semaphore.NewRegistry(userNS),
shms: shm.NewRegistry(userNS),
}
@@ -45,6 +48,11 @@ func NewIPCNamespace(userNS *auth.UserNamespace) *IPCNamespace {
return ns
}
+// MsgqueueRegistry returns the message queue registry for this namespace.
+func (i *IPCNamespace) MsgqueueRegistry() *msgqueue.Registry {
+ return i.queues
+}
+
// SemaphoreRegistry returns the semaphore set registry for this namespace.
func (i *IPCNamespace) SemaphoreRegistry() *semaphore.Registry {
return i.semaphores
diff --git a/pkg/sentry/kernel/kernel.go b/pkg/sentry/kernel/kernel.go
index 352c36ba9..df5160b67 100644
--- a/pkg/sentry/kernel/kernel.go
+++ b/pkg/sentry/kernel/kernel.go
@@ -1299,11 +1299,11 @@ func (k *Kernel) WaitExited() {
}
// Kill requests that all tasks in k immediately exit as if group exiting with
-// status es. Kill does not wait for tasks to exit.
-func (k *Kernel) Kill(es ExitStatus) {
+// status ws. Kill does not wait for tasks to exit.
+func (k *Kernel) Kill(ws linux.WaitStatus) {
k.extMu.Lock()
defer k.extMu.Unlock()
- k.tasks.Kill(es)
+ k.tasks.Kill(ws)
}
// Pause requests that all tasks in k temporarily stop executing, and blocks
diff --git a/pkg/sentry/kernel/kernel_opts.go b/pkg/sentry/kernel/kernel_opts.go
index 2e66ec587..5ffafb0d1 100644
--- a/pkg/sentry/kernel/kernel_opts.go
+++ b/pkg/sentry/kernel/kernel_opts.go
@@ -12,6 +12,9 @@
// See the License for the specific language governing permissions and
// limitations under the License.
+//go:build go1.1
+// +build go1.1
+
package kernel
// SpecialOpts contains non-standard options for the kernel.
diff --git a/pkg/sentry/kernel/msgqueue/BUILD b/pkg/sentry/kernel/msgqueue/BUILD
new file mode 100644
index 000000000..5ec11e1f6
--- /dev/null
+++ b/pkg/sentry/kernel/msgqueue/BUILD
@@ -0,0 +1,36 @@
+load("//tools:defs.bzl", "go_library")
+load("//tools/go_generics:defs.bzl", "go_template_instance")
+
+package(licenses = ["notice"])
+
+go_template_instance(
+ name = "message_list",
+ out = "message_list.go",
+ package = "msgqueue",
+ prefix = "msg",
+ template = "//pkg/ilist:generic_list",
+ types = {
+ "Element": "*Message",
+ "Linker": "*Message",
+ },
+)
+
+go_library(
+ name = "msgqueue",
+ srcs = [
+ "message_list.go",
+ "msgqueue.go",
+ ],
+ visibility = ["//pkg/sentry:internal"],
+ deps = [
+ "//pkg/abi/linux",
+ "//pkg/context",
+ "//pkg/errors/linuxerr",
+ "//pkg/sentry/fs",
+ "//pkg/sentry/kernel/auth",
+ "//pkg/sentry/kernel/ipc",
+ "//pkg/sentry/kernel/time",
+ "//pkg/sync",
+ "//pkg/waiter",
+ ],
+)
diff --git a/pkg/sentry/kernel/msgqueue/msgqueue.go b/pkg/sentry/kernel/msgqueue/msgqueue.go
new file mode 100644
index 000000000..3ce926950
--- /dev/null
+++ b/pkg/sentry/kernel/msgqueue/msgqueue.go
@@ -0,0 +1,220 @@
+// Copyright 2021 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 msgqueue implements System V message queues.
+package msgqueue
+
+import (
+ "gvisor.dev/gvisor/pkg/abi/linux"
+ "gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/errors/linuxerr"
+ "gvisor.dev/gvisor/pkg/sentry/fs"
+ "gvisor.dev/gvisor/pkg/sentry/kernel/auth"
+ "gvisor.dev/gvisor/pkg/sentry/kernel/ipc"
+ ktime "gvisor.dev/gvisor/pkg/sentry/kernel/time"
+ "gvisor.dev/gvisor/pkg/sync"
+ "gvisor.dev/gvisor/pkg/waiter"
+)
+
+const (
+ // System-wide limit for maximum number of queues.
+ maxQueues = linux.MSGMNI
+
+ // Maximum size of a queue in bytes.
+ maxQueueBytes = linux.MSGMNB
+
+ // Maximum size of a message in bytes.
+ maxMessageBytes = linux.MSGMAX
+)
+
+// Registry contains a set of message queues that can be referenced using keys
+// or IDs.
+//
+// +stateify savable
+type Registry struct {
+ // mu protects all the fields below.
+ mu sync.Mutex `state:"nosave"`
+
+ // reg defines basic fields and operations needed for all SysV registries.
+ reg *ipc.Registry
+}
+
+// NewRegistry returns a new Registry ready to be used.
+func NewRegistry(userNS *auth.UserNamespace) *Registry {
+ return &Registry{
+ reg: ipc.NewRegistry(userNS),
+ }
+}
+
+// Queue represents a SysV message queue, described by sysvipc(7).
+//
+// +stateify savable
+type Queue struct {
+ // registry is the registry owning this queue. Immutable.
+ registry *Registry
+
+ // mu protects all the fields below.
+ mu sync.Mutex `state:"nosave"`
+
+ // dead is set to true when a queue is removed from the registry and should
+ // not be used. Operations on the queue should check dead, and return
+ // EIDRM if set to true.
+ dead bool
+
+ // obj defines basic fields that should be included in all SysV IPC objects.
+ obj *ipc.Object
+
+ // senders holds a queue of blocked message senders. Senders are notified
+ // when enough space is available in the queue to insert their message.
+ senders waiter.Queue
+
+ // receivers holds a queue of blocked receivers. Receivers are notified
+ // when a new message is inserted into the queue and can be received.
+ receivers waiter.Queue
+
+ // messages is a list of sent messages.
+ messages msgList
+
+ // sendTime is the last time a msgsnd was perfomed.
+ sendTime ktime.Time
+
+ // receiveTime is the last time a msgrcv was performed.
+ receiveTime ktime.Time
+
+ // changeTime is the last time the queue was modified using msgctl.
+ changeTime ktime.Time
+
+ // byteCount is the current number of message bytes in the queue.
+ byteCount uint64
+
+ // messageCount is the current number of messages in the queue.
+ messageCount uint64
+
+ // maxBytes is the maximum allowed number of bytes in the queue, and is also
+ // used as a limit for the number of total possible messages.
+ maxBytes uint64
+
+ // sendPID is the PID of the process that performed the last msgsnd.
+ sendPID int32
+
+ // receivePID is the PID of the process that performed the last msgrcv.
+ receivePID int32
+}
+
+// Message represents a message exchanged through a Queue via msgsnd(2) and
+// msgrcv(2).
+//
+// +stateify savable
+type Message struct {
+ msgEntry
+
+ // mType is an integer representing the type of the sent message.
+ mType int64
+
+ // mText is an untyped block of memory.
+ mText []byte
+
+ // mSize is the size of mText.
+ mSize uint64
+}
+
+// FindOrCreate creates a new message queue or returns an existing one. See
+// msgget(2).
+func (r *Registry) FindOrCreate(ctx context.Context, key ipc.Key, mode linux.FileMode, private, create, exclusive bool) (*Queue, error) {
+ r.mu.Lock()
+ defer r.mu.Unlock()
+
+ if !private {
+ queue, err := r.reg.Find(ctx, key, mode, create, exclusive)
+ if err != nil {
+ return nil, err
+ }
+
+ if queue != nil {
+ return queue.(*Queue), nil
+ }
+ }
+
+ // Check system-wide limits.
+ if r.reg.ObjectCount() >= maxQueues {
+ return nil, linuxerr.ENOSPC
+ }
+
+ return r.newQueueLocked(ctx, key, fs.FileOwnerFromContext(ctx), fs.FilePermsFromMode(mode))
+}
+
+// newQueueLocked creates a new queue using the given fields. An error is
+// returned if there're no more available identifiers.
+//
+// Precondition: r.mu must be held.
+func (r *Registry) newQueueLocked(ctx context.Context, key ipc.Key, creator fs.FileOwner, perms fs.FilePermissions) (*Queue, error) {
+ q := &Queue{
+ registry: r,
+ obj: ipc.NewObject(r.reg.UserNS, key, creator, creator, perms),
+ sendTime: ktime.ZeroTime,
+ receiveTime: ktime.ZeroTime,
+ changeTime: ktime.NowFromContext(ctx),
+ maxBytes: maxQueueBytes,
+ }
+
+ err := r.reg.Register(q)
+ if err != nil {
+ return nil, err
+ }
+ return q, nil
+}
+
+// Remove removes the queue with specified ID. All waiters (readers and
+// writers) and writers will be awakened and fail. Remove will return an error
+// if the ID is invalid, or the the user doesn't have privileges.
+func (r *Registry) Remove(id ipc.ID, creds *auth.Credentials) error {
+ r.mu.Lock()
+ defer r.mu.Unlock()
+
+ r.reg.Remove(id, creds)
+ return nil
+}
+
+// Lock implements ipc.Mechanism.Lock.
+func (q *Queue) Lock() {
+ q.mu.Lock()
+}
+
+// Unlock implements ipc.mechanism.Unlock.
+//
+// +checklocksignore
+func (q *Queue) Unlock() {
+ q.mu.Unlock()
+}
+
+// Object implements ipc.Mechanism.Object.
+func (q *Queue) Object() *ipc.Object {
+ return q.obj
+}
+
+// Destroy implements ipc.Mechanism.Destroy.
+func (q *Queue) Destroy() {
+ q.dead = true
+
+ // Notify waiters. Senders and receivers will try to run, and return an
+ // error (EIDRM). Waiters should remove themselves from the queue after
+ // waking up.
+ q.senders.Notify(waiter.EventOut)
+ q.receivers.Notify(waiter.EventIn)
+}
+
+// ID returns queue's ID.
+func (q *Queue) ID() ipc.ID {
+ return q.obj.ID
+}
diff --git a/pkg/sentry/kernel/ptrace.go b/pkg/sentry/kernel/ptrace.go
index 161140980..079294f81 100644
--- a/pkg/sentry/kernel/ptrace.go
+++ b/pkg/sentry/kernel/ptrace.go
@@ -768,14 +768,14 @@ const (
// ptraceClone is called at the end of a clone or fork syscall to check if t
// should enter PTRACE_EVENT_CLONE, PTRACE_EVENT_FORK, or PTRACE_EVENT_VFORK
// stop. child is the new task.
-func (t *Task) ptraceClone(kind ptraceCloneKind, child *Task, opts *CloneOptions) bool {
+func (t *Task) ptraceClone(kind ptraceCloneKind, child *Task, args *linux.CloneArgs) bool {
if !t.hasTracer() {
return false
}
t.tg.pidns.owner.mu.Lock()
defer t.tg.pidns.owner.mu.Unlock()
event := false
- if !opts.Untraced {
+ if args.Flags&linux.CLONE_UNTRACED == 0 {
switch kind {
case ptraceCloneKindClone:
if t.ptraceOpts.TraceClone {
@@ -810,7 +810,7 @@ func (t *Task) ptraceClone(kind ptraceCloneKind, child *Task, opts *CloneOptions
// clone(2)'s documentation of CLONE_UNTRACED and CLONE_PTRACE is
// confusingly wrong; see kernel/fork.c:_do_fork() => copy_process() =>
// include/linux/ptrace.h:ptrace_init_task().
- if event || opts.InheritTracer {
+ if event || args.Flags&linux.CLONE_PTRACE != 0 {
tracer := t.Tracer()
if tracer != nil {
child.ptraceTracer.Store(tracer)
@@ -912,7 +912,7 @@ func (t *Task) ptraceExit() {
return
}
t.tg.signalHandlers.mu.Lock()
- status := t.exitStatus.Status()
+ status := t.exitStatus
t.tg.signalHandlers.mu.Unlock()
t.Debugf("Entering PTRACE_EVENT_EXIT stop")
t.ptraceEventLocked(linux.PTRACE_EVENT_EXIT, uint64(status))
@@ -940,7 +940,7 @@ func (t *Task) ptraceKill(target *Task) error {
t.tg.pidns.owner.mu.Lock()
defer t.tg.pidns.owner.mu.Unlock()
if target.Tracer() != t {
- return syserror.ESRCH
+ return linuxerr.ESRCH
}
target.tg.signalHandlers.mu.Lock()
defer target.tg.signalHandlers.mu.Unlock()
@@ -964,7 +964,7 @@ func (t *Task) ptraceInterrupt(target *Task) error {
t.tg.pidns.owner.mu.Lock()
defer t.tg.pidns.owner.mu.Unlock()
if target.Tracer() != t {
- return syserror.ESRCH
+ return linuxerr.ESRCH
}
if !target.ptraceSeized {
return syserror.EIO
@@ -1022,7 +1022,7 @@ func (t *Task) Ptrace(req int64, pid ThreadID, addr, data hostarch.Addr) error {
// specified by pid.
target := t.tg.pidns.TaskWithID(pid)
if target == nil {
- return syserror.ESRCH
+ return linuxerr.ESRCH
}
// PTRACE_ATTACH and PTRACE_SEIZE do not require that target is not already
@@ -1047,7 +1047,7 @@ func (t *Task) Ptrace(req int64, pid ThreadID, addr, data hostarch.Addr) error {
t.tg.pidns.owner.mu.RLock()
if target.Tracer() != t {
t.tg.pidns.owner.mu.RUnlock()
- return syserror.ESRCH
+ return linuxerr.ESRCH
}
if !target.ptraceFreeze() {
t.tg.pidns.owner.mu.RUnlock()
@@ -1055,7 +1055,7 @@ func (t *Task) Ptrace(req int64, pid ThreadID, addr, data hostarch.Addr) error {
// PTRACE_TRACEME, PTRACE_INTERRUPT, and PTRACE_KILL) require the
// tracee to be in a ptrace-stop, otherwise they fail with ESRCH." -
// ptrace(2)
- return syserror.ESRCH
+ return linuxerr.ESRCH
}
t.tg.pidns.owner.mu.RUnlock()
// Even if the target has a ptrace-stop active, the tracee's task goroutine
diff --git a/pkg/sentry/kernel/ptrace_amd64.go b/pkg/sentry/kernel/ptrace_amd64.go
index 5ae05b5c3..63422e155 100644
--- a/pkg/sentry/kernel/ptrace_amd64.go
+++ b/pkg/sentry/kernel/ptrace_amd64.go
@@ -12,6 +12,7 @@
// See the License for the specific language governing permissions and
// limitations under the License.
+//go:build amd64
// +build amd64
package kernel
diff --git a/pkg/sentry/kernel/ptrace_arm64.go b/pkg/sentry/kernel/ptrace_arm64.go
index 46dd84cbc..27514d67b 100644
--- a/pkg/sentry/kernel/ptrace_arm64.go
+++ b/pkg/sentry/kernel/ptrace_arm64.go
@@ -12,6 +12,7 @@
// See the License for the specific language governing permissions and
// limitations under the License.
+//go:build arm64
// +build arm64
package kernel
diff --git a/pkg/sentry/kernel/rseq.go b/pkg/sentry/kernel/rseq.go
index 5e0618e44..de352f4f2 100644
--- a/pkg/sentry/kernel/rseq.go
+++ b/pkg/sentry/kernel/rseq.go
@@ -21,7 +21,6 @@ import (
"gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/hostarch"
"gvisor.dev/gvisor/pkg/sentry/hostcpu"
- "gvisor.dev/gvisor/pkg/syserror"
"gvisor.dev/gvisor/pkg/usermem"
)
@@ -76,7 +75,7 @@ func (t *Task) SetRSeq(addr hostarch.Addr, length, signature uint32) error {
return linuxerr.EINVAL
}
if _, ok := t.MemoryManager().CheckIORange(addr, linux.SizeOfRSeq); !ok {
- return syserror.EFAULT
+ return linuxerr.EFAULT
}
t.rseqAddr = addr
@@ -93,7 +92,7 @@ func (t *Task) SetRSeq(addr hostarch.Addr, length, signature uint32) error {
t.Debugf("Failed to copy CPU to %#x for rseq: %v", t.rseqAddr, err)
t.forceSignal(linux.SIGSEGV, false /* unconditional */)
t.SendSignal(SignalInfoPriv(linux.SIGSEGV))
- return syserror.EFAULT
+ return linuxerr.EFAULT
}
return nil
diff --git a/pkg/sentry/kernel/semaphore/BUILD b/pkg/sentry/kernel/semaphore/BUILD
index a787c00a8..2ae08ed12 100644
--- a/pkg/sentry/kernel/semaphore/BUILD
+++ b/pkg/sentry/kernel/semaphore/BUILD
@@ -26,9 +26,9 @@ go_library(
"//pkg/abi/linux",
"//pkg/context",
"//pkg/errors/linuxerr",
- "//pkg/log",
"//pkg/sentry/fs",
"//pkg/sentry/kernel/auth",
+ "//pkg/sentry/kernel/ipc",
"//pkg/sentry/kernel/time",
"//pkg/sync",
"//pkg/syserror",
@@ -41,10 +41,11 @@ go_test(
srcs = ["semaphore_test.go"],
library = ":semaphore",
deps = [
- "//pkg/abi/linux",
- "//pkg/context",
- "//pkg/sentry/contexttest",
- "//pkg/sentry/kernel/auth",
- "//pkg/syserror",
+ "//pkg/abi/linux", # keep
+ "//pkg/context", # keep
+ "//pkg/sentry/contexttest", # keep
+ "//pkg/sentry/kernel/auth", # keep
+ "//pkg/sentry/kernel/ipc", # keep
+ "//pkg/syserror", # keep
],
)
diff --git a/pkg/sentry/kernel/semaphore/semaphore.go b/pkg/sentry/kernel/semaphore/semaphore.go
index 067cc75c1..8610d3fc1 100644
--- a/pkg/sentry/kernel/semaphore/semaphore.go
+++ b/pkg/sentry/kernel/semaphore/semaphore.go
@@ -21,9 +21,9 @@ import (
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/context"
"gvisor.dev/gvisor/pkg/errors/linuxerr"
- "gvisor.dev/gvisor/pkg/log"
"gvisor.dev/gvisor/pkg/sentry/fs"
"gvisor.dev/gvisor/pkg/sentry/kernel/auth"
+ "gvisor.dev/gvisor/pkg/sentry/kernel/ipc"
ktime "gvisor.dev/gvisor/pkg/sentry/kernel/time"
"gvisor.dev/gvisor/pkg/sync"
"gvisor.dev/gvisor/pkg/syserror"
@@ -47,15 +47,15 @@ const (
//
// +stateify savable
type Registry struct {
- // userNS owning the ipc name this registry belongs to. Immutable.
- userNS *auth.UserNamespace
// mu protects all fields below.
- mu sync.Mutex `state:"nosave"`
- semaphores map[int32]*Set
- lastIDUsed int32
+ mu sync.Mutex `state:"nosave"`
+
+ // reg defines basic fields and operations needed for all SysV registries.
+ reg *ipc.Registry
+
// indexes maintains a mapping between a set's index in virtual array and
// its identifier.
- indexes map[int32]int32
+ indexes map[int32]ipc.ID
}
// Set represents a set of semaphores that can be operated atomically.
@@ -65,19 +65,11 @@ type Set struct {
// registry owning this sem set. Immutable.
registry *Registry
- // Id is a handle that identifies the set.
- ID int32
-
- // key is an user provided key that can be shared between processes.
- key int32
+ // mu protects all fields below.
+ mu sync.Mutex `state:"nosave"`
- // creator is the user that created the set. Immutable.
- creator fs.FileOwner
+ obj *ipc.Object
- // mu protects all fields below.
- mu sync.Mutex `state:"nosave"`
- owner fs.FileOwner
- perms fs.FilePermissions
opTime ktime.Time
changeTime ktime.Time
@@ -115,9 +107,8 @@ type waiter struct {
// NewRegistry creates a new semaphore set registry.
func NewRegistry(userNS *auth.UserNamespace) *Registry {
return &Registry{
- userNS: userNS,
- semaphores: make(map[int32]*Set),
- indexes: make(map[int32]int32),
+ reg: ipc.NewRegistry(userNS),
+ indexes: make(map[int32]ipc.ID),
}
}
@@ -126,7 +117,7 @@ func NewRegistry(userNS *auth.UserNamespace) *Registry {
// a new set is always created. If create is false, it fails if a set cannot
// be found. If exclusive is true, it fails if a set with the same key already
// exists.
-func (r *Registry) FindOrCreate(ctx context.Context, key, nsems int32, mode linux.FileMode, private, create, exclusive bool) (*Set, error) {
+func (r *Registry) FindOrCreate(ctx context.Context, key ipc.Key, nsems int32, mode linux.FileMode, private, create, exclusive bool) (*Set, error) {
if nsems < 0 || nsems > semsMax {
return nil, linuxerr.EINVAL
}
@@ -135,31 +126,19 @@ func (r *Registry) FindOrCreate(ctx context.Context, key, nsems int32, mode linu
defer r.mu.Unlock()
if !private {
- // Look up an existing semaphore.
- if set := r.findByKey(key); set != nil {
- set.mu.Lock()
- defer set.mu.Unlock()
-
- // Check that caller can access semaphore set.
- creds := auth.CredentialsFromContext(ctx)
- if !set.checkPerms(creds, fs.PermsFromMode(mode)) {
- return nil, linuxerr.EACCES
- }
+ set, err := r.reg.Find(ctx, key, mode, create, exclusive)
+ if err != nil {
+ return nil, err
+ }
- // Validate parameters.
+ // Validate semaphore-specific parameters.
+ if set != nil {
+ set := set.(*Set)
if nsems > int32(set.Size()) {
return nil, linuxerr.EINVAL
}
- if create && exclusive {
- return nil, syserror.EEXIST
- }
return set, nil
}
-
- if !create {
- // Semaphore not found and should not be created.
- return nil, syserror.ENOENT
- }
}
// Zero is only valid if an existing set is found.
@@ -169,9 +148,9 @@ func (r *Registry) FindOrCreate(ctx context.Context, key, nsems int32, mode linu
// Apply system limits.
//
- // Map semaphores and map indexes in a registry are of the same size,
- // check map semaphores only here for the system limit.
- if len(r.semaphores) >= setsMax {
+ // Map reg.objects and map indexes in a registry are of the same size,
+ // check map reg.objects only here for the system limit.
+ if r.reg.ObjectCount() >= setsMax {
return nil, syserror.ENOSPC
}
if r.totalSems() > int(semsTotalMax-nsems) {
@@ -179,9 +158,7 @@ func (r *Registry) FindOrCreate(ctx context.Context, key, nsems int32, mode linu
}
// Finally create a new set.
- owner := fs.FileOwnerFromContext(ctx)
- perms := fs.FilePermsFromMode(mode)
- return r.newSet(ctx, key, owner, owner, perms, nsems)
+ return r.newSetLocked(ctx, key, fs.FileOwnerFromContext(ctx), fs.FilePermsFromMode(mode), nsems)
}
// IPCInfo returns information about system-wide semaphore limits and parameters.
@@ -208,7 +185,7 @@ func (r *Registry) SemInfo() *linux.SemInfo {
defer r.mu.Unlock()
info := r.IPCInfo()
- info.SemUsz = uint32(len(r.semaphores))
+ info.SemUsz = uint32(r.reg.ObjectCount())
info.SemAem = uint32(r.totalSems())
return info
@@ -231,77 +208,59 @@ func (r *Registry) HighestIndex() int32 {
return highestIndex
}
-// RemoveID removes set with give 'id' from the registry and marks the set as
+// Remove removes set with give 'id' from the registry and marks the set as
// dead. All waiters will be awakened and fail.
-func (r *Registry) RemoveID(id int32, creds *auth.Credentials) error {
+func (r *Registry) Remove(id ipc.ID, creds *auth.Credentials) error {
r.mu.Lock()
defer r.mu.Unlock()
- set := r.semaphores[id]
- if set == nil {
- return linuxerr.EINVAL
- }
index, found := r.findIndexByID(id)
if !found {
- // Inconsistent state.
- panic(fmt.Sprintf("unable to find an index for ID: %d", id))
+ return linuxerr.EINVAL
}
+ delete(r.indexes, index)
- set.mu.Lock()
- defer set.mu.Unlock()
+ r.reg.Remove(id, creds)
- // "The effective user ID of the calling process must match the creator or
- // owner of the semaphore set, or the caller must be privileged."
- if !set.checkCredentials(creds) && !set.checkCapability(creds) {
- return linuxerr.EACCES
- }
-
- delete(r.semaphores, set.ID)
- delete(r.indexes, index)
- set.destroy()
return nil
}
-func (r *Registry) newSet(ctx context.Context, key int32, owner, creator fs.FileOwner, perms fs.FilePermissions, nsems int32) (*Set, error) {
+// newSetLocked creates a new Set using given fields. An error is returned if there
+// are no more available identifiers.
+//
+// Precondition: r.mu must be held.
+func (r *Registry) newSetLocked(ctx context.Context, key ipc.Key, creator fs.FileOwner, perms fs.FilePermissions, nsems int32) (*Set, error) {
set := &Set{
registry: r,
- key: key,
- owner: owner,
- creator: owner,
- perms: perms,
+ obj: ipc.NewObject(r.reg.UserNS, ipc.Key(key), creator, creator, perms),
changeTime: ktime.NowFromContext(ctx),
sems: make([]sem, nsems),
}
- // Find the next available ID.
- for id := r.lastIDUsed + 1; id != r.lastIDUsed; id++ {
- // Handle wrap around.
- if id < 0 {
- id = 0
- continue
- }
- if r.semaphores[id] == nil {
- index, found := r.findFirstAvailableIndex()
- if !found {
- panic("unable to find an available index")
- }
- r.indexes[index] = id
- r.lastIDUsed = id
- r.semaphores[id] = set
- set.ID = id
- return set, nil
- }
+ err := r.reg.Register(set)
+ if err != nil {
+ return nil, err
}
- log.Warningf("Semaphore map is full, they must be leaking")
- return nil, syserror.ENOMEM
+ index, found := r.findFirstAvailableIndex()
+ if !found {
+ // See linux, ipc/sem.c:newary().
+ return nil, linuxerr.ENOSPC
+ }
+ r.indexes[index] = set.obj.ID
+
+ return set, nil
}
// FindByID looks up a set given an ID.
-func (r *Registry) FindByID(id int32) *Set {
+func (r *Registry) FindByID(id ipc.ID) *Set {
r.mu.Lock()
defer r.mu.Unlock()
- return r.semaphores[id]
+ mech := r.reg.FindByID(id)
+ if mech == nil {
+ return nil
+ }
+ return mech.(*Set)
}
// FindByIndex looks up a set given an index.
@@ -313,19 +272,10 @@ func (r *Registry) FindByIndex(index int32) *Set {
if !present {
return nil
}
- return r.semaphores[id]
+ return r.reg.FindByID(id).(*Set)
}
-func (r *Registry) findByKey(key int32) *Set {
- for _, v := range r.semaphores {
- if v.key == key {
- return v
- }
- }
- return nil
-}
-
-func (r *Registry) findIndexByID(id int32) (int32, bool) {
+func (r *Registry) findIndexByID(id ipc.ID) (int32, bool) {
for k, v := range r.indexes {
if v == id {
return k, true
@@ -345,12 +295,36 @@ func (r *Registry) findFirstAvailableIndex() (int32, bool) {
func (r *Registry) totalSems() int {
totalSems := 0
- for _, v := range r.semaphores {
- totalSems += v.Size()
- }
+ r.reg.ForAllObjects(
+ func(o ipc.Mechanism) {
+ totalSems += o.(*Set).Size()
+ },
+ )
return totalSems
}
+// ID returns semaphore's ID.
+func (s *Set) ID() ipc.ID {
+ return s.obj.ID
+}
+
+// Object implements ipc.Mechanism.Object.
+func (s *Set) Object() *ipc.Object {
+ return s.obj
+}
+
+// Lock implements ipc.Mechanism.Lock.
+func (s *Set) Lock() {
+ s.mu.Lock()
+}
+
+// Unlock implements ipc.mechanism.Unlock.
+//
+// +checklocksignore
+func (s *Set) Unlock() {
+ s.mu.Unlock()
+}
+
func (s *Set) findSem(num int32) *sem {
if num < 0 || int(num) >= s.Size() {
return nil
@@ -370,12 +344,12 @@ func (s *Set) Change(ctx context.Context, creds *auth.Credentials, owner fs.File
// "The effective UID of the calling process must match the owner or creator
// of the semaphore set, or the caller must be privileged."
- if !s.checkCredentials(creds) && !s.checkCapability(creds) {
+ if !s.obj.CheckOwnership(creds) {
return linuxerr.EACCES
}
- s.owner = owner
- s.perms = perms
+ s.obj.Owner = owner
+ s.obj.Perms = perms
s.changeTime = ktime.NowFromContext(ctx)
return nil
}
@@ -395,18 +369,18 @@ func (s *Set) semStat(creds *auth.Credentials, permMask fs.PermMask) (*linux.Sem
s.mu.Lock()
defer s.mu.Unlock()
- if !s.checkPerms(creds, permMask) {
+ if !s.obj.CheckPermissions(creds, permMask) {
return nil, linuxerr.EACCES
}
return &linux.SemidDS{
SemPerm: linux.IPCPerm{
- Key: uint32(s.key),
- UID: uint32(creds.UserNamespace.MapFromKUID(s.owner.UID)),
- GID: uint32(creds.UserNamespace.MapFromKGID(s.owner.GID)),
- CUID: uint32(creds.UserNamespace.MapFromKUID(s.creator.UID)),
- CGID: uint32(creds.UserNamespace.MapFromKGID(s.creator.GID)),
- Mode: uint16(s.perms.LinuxMode()),
+ Key: uint32(s.obj.Key),
+ UID: uint32(creds.UserNamespace.MapFromKUID(s.obj.Owner.UID)),
+ GID: uint32(creds.UserNamespace.MapFromKGID(s.obj.Owner.GID)),
+ CUID: uint32(creds.UserNamespace.MapFromKUID(s.obj.Creator.UID)),
+ CGID: uint32(creds.UserNamespace.MapFromKGID(s.obj.Creator.GID)),
+ Mode: uint16(s.obj.Perms.LinuxMode()),
Seq: 0, // IPC sequence not supported.
},
SemOTime: s.opTime.TimeT(),
@@ -418,20 +392,20 @@ func (s *Set) semStat(creds *auth.Credentials, permMask fs.PermMask) (*linux.Sem
// SetVal overrides a semaphore value, waking up waiters as needed.
func (s *Set) SetVal(ctx context.Context, num int32, val int16, creds *auth.Credentials, pid int32) error {
if val < 0 || val > valueMax {
- return syserror.ERANGE
+ return linuxerr.ERANGE
}
s.mu.Lock()
defer s.mu.Unlock()
// "The calling process must have alter permission on the semaphore set."
- if !s.checkPerms(creds, fs.PermMask{Write: true}) {
+ if !s.obj.CheckPermissions(creds, fs.PermMask{Write: true}) {
return linuxerr.EACCES
}
sem := s.findSem(num)
if sem == nil {
- return syserror.ERANGE
+ return linuxerr.ERANGE
}
// TODO(gvisor.dev/issue/137): Clear undo entries in all processes.
@@ -453,7 +427,7 @@ func (s *Set) SetValAll(ctx context.Context, vals []uint16, creds *auth.Credenti
for _, val := range vals {
if val > valueMax {
- return syserror.ERANGE
+ return linuxerr.ERANGE
}
}
@@ -461,7 +435,7 @@ func (s *Set) SetValAll(ctx context.Context, vals []uint16, creds *auth.Credenti
defer s.mu.Unlock()
// "The calling process must have alter permission on the semaphore set."
- if !s.checkPerms(creds, fs.PermMask{Write: true}) {
+ if !s.obj.CheckPermissions(creds, fs.PermMask{Write: true}) {
return linuxerr.EACCES
}
@@ -483,13 +457,13 @@ func (s *Set) GetVal(num int32, creds *auth.Credentials) (int16, error) {
defer s.mu.Unlock()
// "The calling process must have read permission on the semaphore set."
- if !s.checkPerms(creds, fs.PermMask{Read: true}) {
+ if !s.obj.CheckPermissions(creds, fs.PermMask{Read: true}) {
return 0, linuxerr.EACCES
}
sem := s.findSem(num)
if sem == nil {
- return 0, syserror.ERANGE
+ return 0, linuxerr.ERANGE
}
return sem.value, nil
}
@@ -500,7 +474,7 @@ func (s *Set) GetValAll(creds *auth.Credentials) ([]uint16, error) {
defer s.mu.Unlock()
// "The calling process must have read permission on the semaphore set."
- if !s.checkPerms(creds, fs.PermMask{Read: true}) {
+ if !s.obj.CheckPermissions(creds, fs.PermMask{Read: true}) {
return nil, linuxerr.EACCES
}
@@ -517,13 +491,13 @@ func (s *Set) GetPID(num int32, creds *auth.Credentials) (int32, error) {
defer s.mu.Unlock()
// "The calling process must have read permission on the semaphore set."
- if !s.checkPerms(creds, fs.PermMask{Read: true}) {
+ if !s.obj.CheckPermissions(creds, fs.PermMask{Read: true}) {
return 0, linuxerr.EACCES
}
sem := s.findSem(num)
if sem == nil {
- return 0, syserror.ERANGE
+ return 0, linuxerr.ERANGE
}
return sem.pid, nil
}
@@ -533,13 +507,13 @@ func (s *Set) countWaiters(num int32, creds *auth.Credentials, pred func(w *wait
defer s.mu.Unlock()
// The calling process must have read permission on the semaphore set.
- if !s.checkPerms(creds, fs.PermMask{Read: true}) {
+ if !s.obj.CheckPermissions(creds, fs.PermMask{Read: true}) {
return 0, linuxerr.EACCES
}
sem := s.findSem(num)
if sem == nil {
- return 0, syserror.ERANGE
+ return 0, linuxerr.ERANGE
}
var cnt uint16
for w := sem.waiters.Front(); w != nil; w = w.Next() {
@@ -589,7 +563,7 @@ func (s *Set) ExecuteOps(ctx context.Context, ops []linux.Sembuf, creds *auth.Cr
}
}
- if !s.checkPerms(creds, fs.PermMask{Read: readOnly, Write: !readOnly}) {
+ if !s.obj.CheckPermissions(creds, fs.PermMask{Read: readOnly, Write: !readOnly}) {
return nil, 0, linuxerr.EACCES
}
@@ -625,7 +599,7 @@ func (s *Set) executeOps(ctx context.Context, ops []linux.Sembuf, pid int32) (ch
if op.SemOp < 0 {
// Handle 'wait' operation.
if -op.SemOp > valueMax {
- return nil, 0, syserror.ERANGE
+ return nil, 0, linuxerr.ERANGE
}
if -op.SemOp > tmpVals[op.SemNum] {
// Not enough resources, must wait.
@@ -640,7 +614,7 @@ func (s *Set) executeOps(ctx context.Context, ops []linux.Sembuf, pid int32) (ch
} else {
// op.SemOp > 0: Handle 'signal' operation.
if tmpVals[op.SemNum] > valueMax-op.SemOp {
- return nil, 0, syserror.ERANGE
+ return nil, 0, linuxerr.ERANGE
}
}
@@ -675,38 +649,10 @@ func (s *Set) AbortWait(num int32, ch chan struct{}) {
// Waiter may not be found in case it raced with wakeWaiters().
}
-func (s *Set) checkCredentials(creds *auth.Credentials) bool {
- return s.owner.UID == creds.EffectiveKUID ||
- s.owner.GID == creds.EffectiveKGID ||
- s.creator.UID == creds.EffectiveKUID ||
- s.creator.GID == creds.EffectiveKGID
-}
-
-func (s *Set) checkCapability(creds *auth.Credentials) bool {
- return creds.HasCapabilityIn(linux.CAP_IPC_OWNER, s.registry.userNS) && creds.UserNamespace.MapFromKUID(s.owner.UID).Ok()
-}
-
-func (s *Set) checkPerms(creds *auth.Credentials, reqPerms fs.PermMask) bool {
- // Are we owner, or in group, or other?
- p := s.perms.Other
- if s.owner.UID == creds.EffectiveKUID {
- p = s.perms.User
- } else if creds.InGroup(s.owner.GID) {
- p = s.perms.Group
- }
-
- // Are permissions satisfied without capability checks?
- if p.SupersetOf(reqPerms) {
- return true
- }
-
- return s.checkCapability(creds)
-}
-
-// destroy destroys the set.
+// Destroy implements ipc.Mechanism.Destroy.
//
// Preconditions: Caller must hold 's.mu'.
-func (s *Set) destroy() {
+func (s *Set) Destroy() {
// Notify all waiters. They will fail on the next attempt to execute
// operations and return error.
s.dead = true
diff --git a/pkg/sentry/kernel/semaphore/semaphore_test.go b/pkg/sentry/kernel/semaphore/semaphore_test.go
index e47acefdf..2e4ab8121 100644
--- a/pkg/sentry/kernel/semaphore/semaphore_test.go
+++ b/pkg/sentry/kernel/semaphore/semaphore_test.go
@@ -21,6 +21,7 @@ import (
"gvisor.dev/gvisor/pkg/context"
"gvisor.dev/gvisor/pkg/sentry/contexttest"
"gvisor.dev/gvisor/pkg/sentry/kernel/auth"
+ "gvisor.dev/gvisor/pkg/sentry/kernel/ipc"
"gvisor.dev/gvisor/pkg/syserror"
)
@@ -55,7 +56,7 @@ func signalled(ch chan struct{}) bool {
func TestBasic(t *testing.T) {
ctx := contexttest.Context(t)
- set := &Set{ID: 123, sems: make([]sem, 1)}
+ set := &Set{obj: &ipc.Object{ID: 123}, sems: make([]sem, 1)}
ops := []linux.Sembuf{
{SemOp: 1},
}
@@ -76,7 +77,7 @@ func TestBasic(t *testing.T) {
func TestWaitForZero(t *testing.T) {
ctx := contexttest.Context(t)
- set := &Set{ID: 123, sems: make([]sem, 1)}
+ set := &Set{obj: &ipc.Object{ID: 123}, sems: make([]sem, 1)}
ops := []linux.Sembuf{
{SemOp: 0},
}
@@ -115,7 +116,7 @@ func TestWaitForZero(t *testing.T) {
func TestNoWait(t *testing.T) {
ctx := contexttest.Context(t)
- set := &Set{ID: 123, sems: make([]sem, 1)}
+ set := &Set{obj: &ipc.Object{ID: 123}, sems: make([]sem, 1)}
ops := []linux.Sembuf{
{SemOp: 1},
}
@@ -138,11 +139,12 @@ func TestUnregister(t *testing.T) {
ctx := contexttest.Context(t)
r := NewRegistry(auth.NewRootUserNamespace())
set, err := r.FindOrCreate(ctx, 123, 2, linux.FileMode(0x600), true, true, true)
+
if err != nil {
t.Fatalf("FindOrCreate() failed, err: %v", err)
}
- if got := r.FindByID(set.ID); got.ID != set.ID {
- t.Fatalf("FindById(%d) failed, got: %+v, expected: %+v", set.ID, got, set)
+ if got := r.FindByID(set.obj.ID); got.obj.ID != set.obj.ID {
+ t.Fatalf("FindById(%d) failed, got: %+v, expected: %+v", set.obj.ID, got, set)
}
ops := []linux.Sembuf{
@@ -155,14 +157,14 @@ func TestUnregister(t *testing.T) {
}
creds := auth.CredentialsFromContext(ctx)
- if err := r.RemoveID(set.ID, creds); err != nil {
- t.Fatalf("RemoveID(%d) failed, err: %v", set.ID, err)
+ if err := r.Remove(set.obj.ID, creds); err != nil {
+ t.Fatalf("Remove(%d) failed, err: %v", set.obj.ID, err)
}
if !set.dead {
t.Fatalf("set is not dead: %+v", set)
}
- if got := r.FindByID(set.ID); got != nil {
- t.Fatalf("FindById(%d) failed, got: %+v, expected: nil", set.ID, got)
+ if got := r.FindByID(set.obj.ID); got != nil {
+ t.Fatalf("FindById(%d) failed, got: %+v, expected: nil", set.obj.ID, got)
}
for i, ch := range chs {
if !signalled(ch) {
diff --git a/pkg/sentry/kernel/sessions.go b/pkg/sentry/kernel/sessions.go
index ae21a55da..f9f872522 100644
--- a/pkg/sentry/kernel/sessions.go
+++ b/pkg/sentry/kernel/sessions.go
@@ -17,7 +17,6 @@ package kernel
import (
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/errors/linuxerr"
- "gvisor.dev/gvisor/pkg/syserror"
)
// SessionID is the public identifier.
@@ -373,7 +372,7 @@ func (tg *ThreadGroup) CreateProcessGroup() error {
// Check whether a process still exists or not.
if id == 0 {
- return syserror.ESRCH
+ return linuxerr.ESRCH
}
// Per above, check for a Session leader or existing group.
diff --git a/pkg/sentry/kernel/shm/BUILD b/pkg/sentry/kernel/shm/BUILD
index 5b69333fe..4e8deac4c 100644
--- a/pkg/sentry/kernel/shm/BUILD
+++ b/pkg/sentry/kernel/shm/BUILD
@@ -36,6 +36,7 @@ go_library(
"//pkg/sentry/device",
"//pkg/sentry/fs",
"//pkg/sentry/kernel/auth",
+ "//pkg/sentry/kernel/ipc",
"//pkg/sentry/kernel/time",
"//pkg/sentry/memmap",
"//pkg/sentry/pgalloc",
diff --git a/pkg/sentry/kernel/shm/shm.go b/pkg/sentry/kernel/shm/shm.go
index 9c66b90ef..2abf467d7 100644
--- a/pkg/sentry/kernel/shm/shm.go
+++ b/pkg/sentry/kernel/shm/shm.go
@@ -43,6 +43,7 @@ import (
"gvisor.dev/gvisor/pkg/log"
"gvisor.dev/gvisor/pkg/sentry/fs"
"gvisor.dev/gvisor/pkg/sentry/kernel/auth"
+ "gvisor.dev/gvisor/pkg/sentry/kernel/ipc"
ktime "gvisor.dev/gvisor/pkg/sentry/kernel/time"
"gvisor.dev/gvisor/pkg/sentry/memmap"
"gvisor.dev/gvisor/pkg/sentry/pgalloc"
@@ -51,12 +52,6 @@ import (
"gvisor.dev/gvisor/pkg/syserror"
)
-// Key represents a shm segment key. Analogous to a file name.
-type Key int32
-
-// ID represents the opaque handle for a shm segment. Analogous to an fd.
-type ID int32
-
// Registry tracks all shared memory segments in an IPC namespace. The registry
// provides the mechanisms for creating and finding segments, and reporting
// global shm parameters.
@@ -69,50 +64,51 @@ type Registry struct {
// mu protects all fields below.
mu sync.Mutex `state:"nosave"`
- // shms maps segment ids to segments.
+ // reg defines basic fields and operations needed for all SysV registries.
//
- // shms holds all referenced segments, which are removed on the last
+ // Withing reg, there are two maps, Objects and KeysToIDs.
+ //
+ // reg.objects holds all referenced segments, which are removed on the last
// DecRef. Thus, it cannot itself hold a reference on the Shm.
//
// Since removal only occurs after the last (unlocked) DecRef, there
// exists a short window during which a Shm still exists in Shm, but is
// unreferenced. Users must use TryIncRef to determine if the Shm is
// still valid.
- shms map[ID]*Shm
-
- // keysToShms maps segment keys to segments.
//
- // Shms in keysToShms are guaranteed to be referenced, as they are
+ // keysToIDs maps segment keys to IDs.
+ //
+ // Shms in keysToIDs are guaranteed to be referenced, as they are
// removed by disassociateKey before the last DecRef.
- keysToShms map[Key]*Shm
+ reg *ipc.Registry
// Sum of the sizes of all existing segments rounded up to page size, in
// units of page size.
totalPages uint64
-
- // ID assigned to the last created segment. Used to quickly find the next
- // unused ID.
- lastIDUsed ID
}
// NewRegistry creates a new shm registry.
func NewRegistry(userNS *auth.UserNamespace) *Registry {
return &Registry{
- userNS: userNS,
- shms: make(map[ID]*Shm),
- keysToShms: make(map[Key]*Shm),
+ userNS: userNS,
+ reg: ipc.NewRegistry(userNS),
}
}
// FindByID looks up a segment given an ID.
//
// FindByID returns a reference on Shm.
-func (r *Registry) FindByID(id ID) *Shm {
+func (r *Registry) FindByID(id ipc.ID) *Shm {
r.mu.Lock()
defer r.mu.Unlock()
- s := r.shms[id]
+ mech := r.reg.FindByID(id)
+ if mech == nil {
+ return nil
+ }
+ s := mech.(*Shm)
+
// Take a reference on s. If TryIncRef fails, s has reached the last
- // DecRef, but hasn't quite been removed from r.shms yet.
+ // DecRef, but hasn't quite been removed from r.reg.objects yet.
if s != nil && s.TryIncRef() {
return s
}
@@ -129,9 +125,9 @@ func (r *Registry) dissociateKey(s *Shm) {
defer r.mu.Unlock()
s.mu.Lock()
defer s.mu.Unlock()
- if s.key != linux.IPC_PRIVATE {
- delete(r.keysToShms, s.key)
- s.key = linux.IPC_PRIVATE
+ if s.obj.Key != linux.IPC_PRIVATE {
+ r.reg.DissociateKey(s.obj.Key)
+ s.obj.Key = linux.IPC_PRIVATE
}
}
@@ -139,7 +135,7 @@ func (r *Registry) dissociateKey(s *Shm) {
// analogous to open(2).
//
// FindOrCreate returns a reference on Shm.
-func (r *Registry) FindOrCreate(ctx context.Context, pid int32, key Key, size uint64, mode linux.FileMode, private, create, exclusive bool) (*Shm, error) {
+func (r *Registry) FindOrCreate(ctx context.Context, pid int32, key ipc.Key, size uint64, mode linux.FileMode, private, create, exclusive bool) (*Shm, error) {
if (create || private) && (size < linux.SHMMIN || size > linux.SHMMAX) {
// "A new segment was to be created and size is less than SHMMIN or
// greater than SHMMAX." - man shmget(2)
@@ -152,49 +148,29 @@ func (r *Registry) FindOrCreate(ctx context.Context, pid int32, key Key, size ui
r.mu.Lock()
defer r.mu.Unlock()
- if len(r.shms) >= linux.SHMMNI {
+ if r.reg.ObjectCount() >= linux.SHMMNI {
// "All possible shared memory IDs have been taken (SHMMNI) ..."
// - man shmget(2)
return nil, syserror.ENOSPC
}
if !private {
- // Look up an existing segment.
- if shm := r.keysToShms[key]; shm != nil {
- shm.mu.Lock()
- defer shm.mu.Unlock()
-
- // Check that caller can access the segment.
- if !shm.checkPermissions(ctx, fs.PermsFromMode(mode)) {
- // "The user does not have permission to access the shared
- // memory segment, and does not have the CAP_IPC_OWNER
- // capability in the user namespace that governs its IPC
- // namespace." - man shmget(2)
- return nil, linuxerr.EACCES
- }
+ shm, err := r.reg.Find(ctx, key, mode, create, exclusive)
+ if err != nil {
+ return nil, err
+ }
+ // Validate shm-specific parameters.
+ if shm != nil {
+ shm := shm.(*Shm)
if size > shm.size {
// "A segment for the given key exists, but size is greater than
// the size of that segment." - man shmget(2)
return nil, linuxerr.EINVAL
}
-
- if create && exclusive {
- // "IPC_CREAT and IPC_EXCL were specified in shmflg, but a
- // shared memory segment already exists for key."
- // - man shmget(2)
- return nil, syserror.EEXIST
- }
-
shm.IncRef()
return shm, nil
}
-
- if !create {
- // "No segment exists for the given key, and IPC_CREAT was not
- // specified." - man shmget(2)
- return nil, syserror.ENOENT
- }
}
var sizeAligned uint64
@@ -212,9 +188,7 @@ func (r *Registry) FindOrCreate(ctx context.Context, pid int32, key Key, size ui
}
// Need to create a new segment.
- creator := fs.FileOwnerFromContext(ctx)
- perms := fs.FilePermsFromMode(mode)
- s, err := r.newShm(ctx, pid, key, creator, perms, size)
+ s, err := r.newShmLocked(ctx, pid, key, fs.FileOwnerFromContext(ctx), fs.FilePermsFromMode(mode), size)
if err != nil {
return nil, err
}
@@ -224,10 +198,10 @@ func (r *Registry) FindOrCreate(ctx context.Context, pid int32, key Key, size ui
return s, nil
}
-// newShm creates a new segment in the registry.
+// newShmLocked creates a new segment in the registry.
//
// Precondition: Caller must hold r.mu.
-func (r *Registry) newShm(ctx context.Context, pid int32, key Key, creator fs.FileOwner, perms fs.FilePermissions, size uint64) (*Shm, error) {
+func (r *Registry) newShmLocked(ctx context.Context, pid int32, key ipc.Key, creator fs.FileOwner, perms fs.FilePermissions, size uint64) (*Shm, error) {
mfp := pgalloc.MemoryFileProviderFromContext(ctx)
if mfp == nil {
panic(fmt.Sprintf("context.Context %T lacks non-nil value for key %T", ctx, pgalloc.CtxMemoryFileProvider))
@@ -242,40 +216,21 @@ func (r *Registry) newShm(ctx context.Context, pid int32, key Key, creator fs.Fi
shm := &Shm{
mfp: mfp,
registry: r,
- creator: creator,
size: size,
effectiveSize: effectiveSize,
+ obj: ipc.NewObject(r.reg.UserNS, ipc.Key(key), creator, creator, perms),
fr: fr,
- key: key,
- perms: perms,
- owner: creator,
creatorPID: pid,
changeTime: ktime.NowFromContext(ctx),
}
shm.InitRefs()
- // Find the next available ID.
- for id := r.lastIDUsed + 1; id != r.lastIDUsed; id++ {
- // Handle wrap around.
- if id < 0 {
- id = 0
- continue
- }
- if r.shms[id] == nil {
- r.lastIDUsed = id
-
- shm.ID = id
- r.shms[id] = shm
- r.keysToShms[key] = shm
-
- r.totalPages += effectiveSize / hostarch.PageSize
-
- return shm, nil
- }
+ if err := r.reg.Register(shm); err != nil {
+ return nil, err
}
+ r.totalPages += effectiveSize / hostarch.PageSize
- log.Warningf("Shm ids exhuasted, they may be leaking")
- return nil, syserror.ENOSPC
+ return shm, nil
}
// IPCInfo reports global parameters for sysv shared memory segments on this
@@ -297,7 +252,7 @@ func (r *Registry) ShmInfo() *linux.ShmInfo {
defer r.mu.Unlock()
return &linux.ShmInfo{
- UsedIDs: int32(r.lastIDUsed),
+ UsedIDs: int32(r.reg.LastIDUsed()),
ShmTot: r.totalPages,
ShmRss: r.totalPages, // We could probably get a better estimate from memory accounting.
ShmSwp: 0, // No reclaim at the moment.
@@ -314,11 +269,11 @@ func (r *Registry) remove(s *Shm) {
s.mu.Lock()
defer s.mu.Unlock()
- if s.key != linux.IPC_PRIVATE {
+ if s.obj.Key != linux.IPC_PRIVATE {
panic(fmt.Sprintf("Attempted to remove %s from the registry whose key is still associated", s.debugLocked()))
}
- delete(r.shms, s.ID)
+ r.reg.DissociateID(s.obj.ID)
r.totalPages -= s.effectiveSize / hostarch.PageSize
}
@@ -330,13 +285,16 @@ func (r *Registry) Release(ctx context.Context) {
// the IPC namespace containing it has no more references.
toRelease := make([]*Shm, 0)
r.mu.Lock()
- for _, s := range r.keysToShms {
- s.mu.Lock()
- if !s.pendingDestruction {
- toRelease = append(toRelease, s)
- }
- s.mu.Unlock()
- }
+ r.reg.ForAllObjects(
+ func(o ipc.Mechanism) {
+ s := o.(*Shm)
+ s.mu.Lock()
+ if !s.pendingDestruction {
+ toRelease = append(toRelease, s)
+ }
+ s.mu.Unlock()
+ },
+ )
r.mu.Unlock()
for _, s := range toRelease {
@@ -374,12 +332,6 @@ type Shm struct {
// registry points to the shm registry containing this segment. Immutable.
registry *Registry
- // ID is the kernel identifier for this segment. Immutable.
- ID ID
-
- // creator is the user that created the segment. Immutable.
- creator fs.FileOwner
-
// size is the requested size of the segment at creation, in
// bytes. Immutable.
size uint64
@@ -397,14 +349,8 @@ type Shm struct {
// mu protects all fields below.
mu sync.Mutex `state:"nosave"`
- // key is the public identifier for this segment.
- key Key
-
- // perms is the access permissions for the segment.
- perms fs.FilePermissions
+ obj *ipc.Object
- // owner of this segment.
- owner fs.FileOwner
// attachTime is updated on every successful shmat.
attachTime ktime.Time
// detachTime is updated on every successful shmdt.
@@ -426,17 +372,44 @@ type Shm struct {
pendingDestruction bool
}
+// ID returns object's ID.
+func (s *Shm) ID() ipc.ID {
+ return s.obj.ID
+}
+
+// Object implements ipc.Mechanism.Object.
+func (s *Shm) Object() *ipc.Object {
+ return s.obj
+}
+
+// Destroy implements ipc.Mechanism.Destroy. No work is performed on shm.Destroy
+// because a different removal mechanism is used in shm. See Shm.MarkDestroyed.
+func (s *Shm) Destroy() {
+}
+
+// Lock implements ipc.Mechanism.Lock.
+func (s *Shm) Lock() {
+ s.mu.Lock()
+}
+
+// Unlock implements ipc.mechanism.Unlock.
+//
+// +checklocksignore
+func (s *Shm) Unlock() {
+ s.mu.Unlock()
+}
+
// Precondition: Caller must hold s.mu.
func (s *Shm) debugLocked() string {
return fmt.Sprintf("Shm{id: %d, key: %d, size: %d bytes, refs: %d, destroyed: %v}",
- s.ID, s.key, s.size, s.ReadRefs(), s.pendingDestruction)
+ s.obj.ID, s.obj.Key, s.size, s.ReadRefs(), s.pendingDestruction)
}
// MappedName implements memmap.MappingIdentity.MappedName.
func (s *Shm) MappedName(ctx context.Context) string {
s.mu.Lock()
defer s.mu.Unlock()
- return fmt.Sprintf("SYSV%08d", s.key)
+ return fmt.Sprintf("SYSV%08d", s.obj.Key)
}
// DeviceID implements memmap.MappingIdentity.DeviceID.
@@ -448,7 +421,7 @@ func (s *Shm) DeviceID() uint64 {
func (s *Shm) InodeID() uint64 {
// "shmid gets reported as "inode#" in /proc/pid/maps. proc-ps tools use
// this. Changing this will break them." -- Linux, ipc/shm.c:newseg()
- return uint64(s.ID)
+ return uint64(s.obj.ID)
}
// DecRef drops a reference on s.
@@ -512,7 +485,7 @@ func (*Shm) CopyMapping(context.Context, memmap.MappingSpace, hostarch.AddrRange
func (s *Shm) Translate(ctx context.Context, required, optional memmap.MappableRange, at hostarch.AccessType) ([]memmap.Translation, error) {
var err error
if required.End > s.fr.Length() {
- err = &memmap.BusError{syserror.EFAULT}
+ err = &memmap.BusError{linuxerr.EFAULT}
}
if source := optional.Intersect(memmap.MappableRange{0, s.fr.Length()}); source.Length() != 0 {
return []memmap.Translation{
@@ -551,7 +524,8 @@ func (s *Shm) ConfigureAttach(ctx context.Context, addr hostarch.Addr, opts Atta
return memmap.MMapOpts{}, syserror.EIDRM
}
- if !s.checkPermissions(ctx, fs.PermMask{
+ creds := auth.CredentialsFromContext(ctx)
+ if !s.obj.CheckPermissions(creds, fs.PermMask{
Read: true,
Write: !opts.Readonly,
Execute: opts.Execute,
@@ -591,7 +565,8 @@ func (s *Shm) IPCStat(ctx context.Context) (*linux.ShmidDS, error) {
// "The caller must have read permission on the shared memory segment."
// - man shmctl(2)
- if !s.checkPermissions(ctx, fs.PermMask{Read: true}) {
+ creds := auth.CredentialsFromContext(ctx)
+ if !s.obj.CheckPermissions(creds, fs.PermMask{Read: true}) {
// "IPC_STAT or SHM_STAT is requested and shm_perm.mode does not allow
// read access for shmid, and the calling process does not have the
// CAP_IPC_OWNER capability in the user namespace that governs its IPC
@@ -603,7 +578,6 @@ func (s *Shm) IPCStat(ctx context.Context) (*linux.ShmidDS, error) {
if s.pendingDestruction {
mode |= linux.SHM_DEST
}
- creds := auth.CredentialsFromContext(ctx)
// Use the reference count as a rudimentary count of the number of
// attaches. We exclude:
@@ -620,12 +594,12 @@ func (s *Shm) IPCStat(ctx context.Context) (*linux.ShmidDS, error) {
ds := &linux.ShmidDS{
ShmPerm: linux.IPCPerm{
- Key: uint32(s.key),
- UID: uint32(creds.UserNamespace.MapFromKUID(s.owner.UID)),
- GID: uint32(creds.UserNamespace.MapFromKGID(s.owner.GID)),
- CUID: uint32(creds.UserNamespace.MapFromKUID(s.creator.UID)),
- CGID: uint32(creds.UserNamespace.MapFromKGID(s.creator.GID)),
- Mode: mode | uint16(s.perms.LinuxMode()),
+ Key: uint32(s.obj.Key),
+ UID: uint32(creds.UserNamespace.MapFromKUID(s.obj.Owner.UID)),
+ GID: uint32(creds.UserNamespace.MapFromKGID(s.obj.Owner.GID)),
+ CUID: uint32(creds.UserNamespace.MapFromKUID(s.obj.Creator.UID)),
+ CGID: uint32(creds.UserNamespace.MapFromKGID(s.obj.Creator.GID)),
+ Mode: mode | uint16(s.obj.Perms.LinuxMode()),
Seq: 0, // IPC sequences not supported.
},
ShmSegsz: s.size,
@@ -645,11 +619,11 @@ func (s *Shm) Set(ctx context.Context, ds *linux.ShmidDS) error {
s.mu.Lock()
defer s.mu.Unlock()
- if !s.checkOwnership(ctx) {
+ creds := auth.CredentialsFromContext(ctx)
+ if !s.obj.CheckOwnership(creds) {
return linuxerr.EPERM
}
- creds := auth.CredentialsFromContext(ctx)
uid := creds.UserNamespace.MapToKUID(auth.UID(ds.ShmPerm.UID))
gid := creds.UserNamespace.MapToKGID(auth.GID(ds.ShmPerm.GID))
if !uid.Ok() || !gid.Ok() {
@@ -659,10 +633,10 @@ func (s *Shm) Set(ctx context.Context, ds *linux.ShmidDS) error {
// User may only modify the lower 9 bits of the mode. All the other bits are
// always 0 for the underlying inode.
mode := linux.FileMode(ds.ShmPerm.Mode & 0x1ff)
- s.perms = fs.FilePermsFromMode(mode)
+ s.obj.Perms = fs.FilePermsFromMode(mode)
- s.owner.UID = uid
- s.owner.GID = gid
+ s.obj.Owner.UID = uid
+ s.obj.Owner.GID = gid
s.changeTime = ktime.NowFromContext(ctx)
return nil
@@ -691,40 +665,3 @@ func (s *Shm) MarkDestroyed(ctx context.Context) {
s.DecRef(ctx)
return
}
-
-// checkOwnership verifies whether a segment may be accessed by ctx as an
-// owner. See ipc/util.c:ipcctl_pre_down_nolock() in Linux.
-//
-// Precondition: Caller must hold s.mu.
-func (s *Shm) checkOwnership(ctx context.Context) bool {
- creds := auth.CredentialsFromContext(ctx)
- if s.owner.UID == creds.EffectiveKUID || s.creator.UID == creds.EffectiveKUID {
- return true
- }
-
- // Tasks with CAP_SYS_ADMIN may bypass ownership checks. Strangely, Linux
- // doesn't use CAP_IPC_OWNER for this despite CAP_IPC_OWNER being documented
- // for use to "override IPC ownership checks".
- return creds.HasCapabilityIn(linux.CAP_SYS_ADMIN, s.registry.userNS)
-}
-
-// checkPermissions verifies whether a segment is accessible by ctx for access
-// described by req. See ipc/util.c:ipcperms() in Linux.
-//
-// Precondition: Caller must hold s.mu.
-func (s *Shm) checkPermissions(ctx context.Context, req fs.PermMask) bool {
- creds := auth.CredentialsFromContext(ctx)
-
- p := s.perms.Other
- if s.owner.UID == creds.EffectiveKUID {
- p = s.perms.User
- } else if creds.InGroup(s.owner.GID) {
- p = s.perms.Group
- }
- if p.SupersetOf(req) {
- return true
- }
-
- // Tasks with CAP_IPC_OWNER may bypass permission checks.
- return creds.HasCapabilityIn(linux.CAP_IPC_OWNER, s.registry.userNS)
-}
diff --git a/pkg/sentry/kernel/task.go b/pkg/sentry/kernel/task.go
index d211e4d82..59eeb253d 100644
--- a/pkg/sentry/kernel/task.go
+++ b/pkg/sentry/kernel/task.go
@@ -232,7 +232,7 @@ type Task struct {
// exitStatus is the task's exit status.
//
// exitStatus is protected by the signal mutex.
- exitStatus ExitStatus
+ exitStatus linux.WaitStatus
// syscallRestartBlock represents a custom restart function to run in
// restart_syscall(2) to resume an interrupted syscall.
diff --git a/pkg/sentry/kernel/task_cgroup.go b/pkg/sentry/kernel/task_cgroup.go
index 7dd6d3108..828b90014 100644
--- a/pkg/sentry/kernel/task_cgroup.go
+++ b/pkg/sentry/kernel/task_cgroup.go
@@ -27,8 +27,6 @@ import (
// EnterInitialCgroups moves t into an initial set of cgroups.
//
// Precondition: t isn't in any cgroups yet, t.cgs is empty.
-//
-// +checklocksignore parent.mu is conditionally acquired.
func (t *Task) EnterInitialCgroups(parent *Task) {
var inherit map[Cgroup]struct{}
if parent != nil {
diff --git a/pkg/sentry/kernel/task_clone.go b/pkg/sentry/kernel/task_clone.go
index 7e1347aa6..da4b77ca2 100644
--- a/pkg/sentry/kernel/task_clone.go
+++ b/pkg/sentry/kernel/task_clone.go
@@ -26,140 +26,39 @@ import (
"gvisor.dev/gvisor/pkg/usermem"
)
-// SharingOptions controls what resources are shared by a new task created by
-// Task.Clone, or an existing task affected by Task.Unshare.
-type SharingOptions struct {
- // If NewAddressSpace is true, the task should have an independent virtual
- // address space.
- NewAddressSpace bool
-
- // If NewSignalHandlers is true, the task should use an independent set of
- // signal handlers.
- NewSignalHandlers bool
-
- // If NewThreadGroup is true, the task should be the leader of its own
- // thread group. TerminationSignal is the signal that the thread group
- // will send to its parent when it exits. If NewThreadGroup is false,
- // TerminationSignal is ignored.
- NewThreadGroup bool
- TerminationSignal linux.Signal
-
- // If NewPIDNamespace is true:
- //
- // - In the context of Task.Clone, the new task should be the init task
- // (TID 1) in a new PID namespace.
- //
- // - In the context of Task.Unshare, the task should create a new PID
- // namespace, and all subsequent clones of the task should be members of
- // the new PID namespace.
- NewPIDNamespace bool
-
- // If NewUserNamespace is true, the task should have an independent user
- // namespace.
- NewUserNamespace bool
-
- // If NewNetworkNamespace is true, the task should have an independent
- // network namespace.
- NewNetworkNamespace bool
-
- // If NewFiles is true, the task should use an independent file descriptor
- // table.
- NewFiles bool
-
- // If NewFSContext is true, the task should have an independent FSContext.
- NewFSContext bool
-
- // If NewUTSNamespace is true, the task should have an independent UTS
- // namespace.
- NewUTSNamespace bool
-
- // If NewIPCNamespace is true, the task should have an independent IPC
- // namespace.
- NewIPCNamespace bool
-}
-
-// CloneOptions controls the behavior of Task.Clone.
-type CloneOptions struct {
- // SharingOptions defines the set of resources that the new task will share
- // with its parent.
- SharingOptions
-
- // Stack is the initial stack pointer of the new task. If Stack is 0, the
- // new task will start with the same stack pointer as its parent.
- Stack hostarch.Addr
-
- // If SetTLS is true, set the new task's TLS (thread-local storage)
- // descriptor to TLS. If SetTLS is false, TLS is ignored.
- SetTLS bool
- TLS hostarch.Addr
-
- // If ChildClearTID is true, when the child exits, 0 is written to the
- // address ChildTID in the child's memory, and if the write is successful a
- // futex wake on the same address is performed.
- //
- // If ChildSetTID is true, the child's thread ID (in the child's PID
- // namespace) is written to address ChildTID in the child's memory. (As in
- // Linux, failed writes are silently ignored.)
- ChildClearTID bool
- ChildSetTID bool
- ChildTID hostarch.Addr
-
- // If ParentSetTID is true, the child's thread ID (in the parent's PID
- // namespace) is written to address ParentTID in the parent's memory. (As
- // in Linux, failed writes are silently ignored.)
- //
- // Older versions of the clone(2) man page state that CLONE_PARENT_SETTID
- // causes the child's thread ID to be written to ptid in both the parent
- // and child's memory, but this is a documentation error fixed by
- // 87ab04792ced ("clone.2: Fix description of CLONE_PARENT_SETTID").
- ParentSetTID bool
- ParentTID hostarch.Addr
-
- // If Vfork is true, place the parent in vforkStop until the cloned task
- // releases its TaskImage.
- Vfork bool
-
- // If Untraced is true, do not report PTRACE_EVENT_CLONE/FORK/VFORK for
- // this clone(), and do not ptrace-attach the caller's tracer to the new
- // task. (PTRACE_EVENT_VFORK_DONE will still be reported if appropriate).
- Untraced bool
-
- // If InheritTracer is true, ptrace-attach the caller's tracer to the new
- // task, even if no PTRACE_EVENT_CLONE/FORK/VFORK event would be reported
- // for it. If both Untraced and InheritTracer are true, no event will be
- // reported, but tracer inheritance will still occur.
- InheritTracer bool
-}
-
// Clone implements the clone(2) syscall and returns the thread ID of the new
// task in t's PID namespace. Clone may return both a non-zero thread ID and a
// non-nil error.
//
// Preconditions: The caller must be running Task.doSyscallInvoke on the task
// goroutine.
-func (t *Task) Clone(opts *CloneOptions) (ThreadID, *SyscallControl, error) {
+func (t *Task) Clone(args *linux.CloneArgs) (ThreadID, *SyscallControl, error) {
// Since signal actions may refer to application signal handlers by virtual
// address, any set of signal handlers must refer to the same address
// space.
- if !opts.NewSignalHandlers && opts.NewAddressSpace {
+ if args.Flags&(linux.CLONE_SIGHAND|linux.CLONE_VM) == linux.CLONE_SIGHAND {
return 0, nil, linuxerr.EINVAL
}
// In order for the behavior of thread-group-directed signals to be sane,
// all tasks in a thread group must share signal handlers.
- if !opts.NewThreadGroup && opts.NewSignalHandlers {
+ if args.Flags&(linux.CLONE_THREAD|linux.CLONE_SIGHAND) == linux.CLONE_THREAD {
return 0, nil, linuxerr.EINVAL
}
// All tasks in a thread group must be in the same PID namespace.
- if !opts.NewThreadGroup && (opts.NewPIDNamespace || t.childPIDNamespace != nil) {
+ if (args.Flags&linux.CLONE_THREAD != 0) && (args.Flags&linux.CLONE_NEWPID != 0 || t.childPIDNamespace != nil) {
return 0, nil, linuxerr.EINVAL
}
// The two different ways of specifying a new PID namespace are
// incompatible.
- if opts.NewPIDNamespace && t.childPIDNamespace != nil {
+ if args.Flags&linux.CLONE_NEWPID != 0 && t.childPIDNamespace != nil {
return 0, nil, linuxerr.EINVAL
}
// Thread groups and FS contexts cannot span user namespaces.
- if opts.NewUserNamespace && (!opts.NewThreadGroup || !opts.NewFSContext) {
+ if args.Flags&linux.CLONE_NEWUSER != 0 && args.Flags&(linux.CLONE_THREAD|linux.CLONE_FS) != 0 {
+ return 0, nil, linuxerr.EINVAL
+ }
+ // args.ExitSignal must be a valid signal.
+ if args.ExitSignal != 0 && !linux.Signal(args.ExitSignal).IsValid() {
return 0, nil, linuxerr.EINVAL
}
@@ -174,7 +73,7 @@ func (t *Task) Clone(opts *CloneOptions) (ThreadID, *SyscallControl, error) {
// user_namespaces(7)
creds := t.Credentials()
userns := creds.UserNamespace
- if opts.NewUserNamespace {
+ if args.Flags&linux.CLONE_NEWUSER != 0 {
var err error
// "EPERM (since Linux 3.9): CLONE_NEWUSER was specified in flags and
// the caller is in a chroot environment (i.e., the caller's root
@@ -189,21 +88,19 @@ func (t *Task) Clone(opts *CloneOptions) (ThreadID, *SyscallControl, error) {
return 0, nil, err
}
}
- if (opts.NewPIDNamespace || opts.NewNetworkNamespace || opts.NewUTSNamespace) && !creds.HasCapabilityIn(linux.CAP_SYS_ADMIN, userns) {
+ if args.Flags&(linux.CLONE_NEWPID|linux.CLONE_NEWNET|linux.CLONE_NEWUTS|linux.CLONE_NEWIPC) != 0 && !creds.HasCapabilityIn(linux.CAP_SYS_ADMIN, userns) {
return 0, nil, linuxerr.EPERM
}
utsns := t.UTSNamespace()
- if opts.NewUTSNamespace {
+ if args.Flags&linux.CLONE_NEWUTS != 0 {
// Note that this must happen after NewUserNamespace so we get
// the new userns if there is one.
utsns = t.UTSNamespace().Clone(userns)
}
ipcns := t.IPCNamespace()
- if opts.NewIPCNamespace {
- // Note that "If CLONE_NEWIPC is set, then create the process in a new IPC
- // namespace"
+ if args.Flags&linux.CLONE_NEWIPC != 0 {
ipcns = NewIPCNamespace(userns)
} else {
ipcns.IncRef()
@@ -214,7 +111,7 @@ func (t *Task) Clone(opts *CloneOptions) (ThreadID, *SyscallControl, error) {
defer cu.Clean()
netns := t.NetworkNamespace()
- if opts.NewNetworkNamespace {
+ if args.Flags&linux.CLONE_NEWNET != 0 {
netns = inet.NewNamespace(netns)
}
@@ -227,7 +124,7 @@ func (t *Task) Clone(opts *CloneOptions) (ThreadID, *SyscallControl, error) {
})
}
- image, err := t.image.Fork(t, t.k, !opts.NewAddressSpace)
+ image, err := t.image.Fork(t, t.k, args.Flags&linux.CLONE_VM != 0)
if err != nil {
return 0, nil, err
}
@@ -236,17 +133,17 @@ func (t *Task) Clone(opts *CloneOptions) (ThreadID, *SyscallControl, error) {
})
// clone() returns 0 in the child.
image.Arch.SetReturn(0)
- if opts.Stack != 0 {
- image.Arch.SetStack(uintptr(opts.Stack))
+ if args.Stack != 0 {
+ image.Arch.SetStack(uintptr(args.Stack))
}
- if opts.SetTLS {
- if !image.Arch.SetTLS(uintptr(opts.TLS)) {
+ if args.Flags&linux.CLONE_SETTLS != 0 {
+ if !image.Arch.SetTLS(uintptr(args.TLS)) {
return 0, nil, linuxerr.EPERM
}
}
var fsContext *FSContext
- if opts.NewFSContext {
+ if args.Flags&linux.CLONE_FS == 0 {
fsContext = t.fsContext.Fork()
} else {
fsContext = t.fsContext
@@ -254,7 +151,7 @@ func (t *Task) Clone(opts *CloneOptions) (ThreadID, *SyscallControl, error) {
}
var fdTable *FDTable
- if opts.NewFiles {
+ if args.Flags&linux.CLONE_FILES == 0 {
fdTable = t.fdTable.Fork(t)
} else {
fdTable = t.fdTable
@@ -264,22 +161,22 @@ func (t *Task) Clone(opts *CloneOptions) (ThreadID, *SyscallControl, error) {
pidns := t.tg.pidns
if t.childPIDNamespace != nil {
pidns = t.childPIDNamespace
- } else if opts.NewPIDNamespace {
+ } else if args.Flags&linux.CLONE_NEWPID != 0 {
pidns = pidns.NewChild(userns)
}
tg := t.tg
rseqAddr := hostarch.Addr(0)
rseqSignature := uint32(0)
- if opts.NewThreadGroup {
+ if args.Flags&linux.CLONE_THREAD == 0 {
if tg.mounts != nil {
tg.mounts.IncRef()
}
sh := t.tg.signalHandlers
- if opts.NewSignalHandlers {
+ if args.Flags&linux.CLONE_SIGHAND == 0 {
sh = sh.Fork()
}
- tg = t.k.NewThreadGroup(tg.mounts, pidns, sh, opts.TerminationSignal, tg.limits.GetCopy())
+ tg = t.k.NewThreadGroup(tg.mounts, pidns, sh, linux.Signal(args.ExitSignal), tg.limits.GetCopy())
tg.oomScoreAdj = atomic.LoadInt32(&t.tg.oomScoreAdj)
rseqAddr = t.rseqAddr
rseqSignature = t.rseqSignature
@@ -304,7 +201,7 @@ func (t *Task) Clone(opts *CloneOptions) (ThreadID, *SyscallControl, error) {
RSeqSignature: rseqSignature,
ContainerID: t.ContainerID(),
}
- if opts.NewThreadGroup {
+ if args.Flags&linux.CLONE_THREAD == 0 {
cfg.Parent = t
} else {
cfg.InheritParent = t
@@ -322,7 +219,7 @@ func (t *Task) Clone(opts *CloneOptions) (ThreadID, *SyscallControl, error) {
//
// However kernel/fork.c:copy_process() adds a limitation to this:
// "sigaltstack should be cleared when sharing the same VM".
- if opts.NewAddressSpace || opts.Vfork {
+ if args.Flags&linux.CLONE_VM == 0 || args.Flags&linux.CLONE_VFORK != 0 {
nt.SetSignalStack(t.SignalStack())
}
@@ -347,35 +244,35 @@ func (t *Task) Clone(opts *CloneOptions) (ThreadID, *SyscallControl, error) {
copiedFilters := append([]bpf.Program(nil), f.([]bpf.Program)...)
nt.syscallFilters.Store(copiedFilters)
}
- if opts.Vfork {
+ if args.Flags&linux.CLONE_VFORK != 0 {
nt.vforkParent = t
}
- if opts.ChildClearTID {
- nt.SetClearTID(opts.ChildTID)
+ if args.Flags&linux.CLONE_CHILD_CLEARTID != 0 {
+ nt.SetClearTID(hostarch.Addr(args.ChildTID))
}
- if opts.ChildSetTID {
+ if args.Flags&linux.CLONE_CHILD_SETTID != 0 {
ctid := nt.ThreadID()
- ctid.CopyOut(nt.CopyContext(t, usermem.IOOpts{AddressSpaceActive: false}), opts.ChildTID)
+ ctid.CopyOut(nt.CopyContext(t, usermem.IOOpts{AddressSpaceActive: false}), hostarch.Addr(args.ChildTID))
}
ntid := t.tg.pidns.IDOfTask(nt)
- if opts.ParentSetTID {
- ntid.CopyOut(t, opts.ParentTID)
+ if args.Flags&linux.CLONE_PARENT_SETTID != 0 {
+ ntid.CopyOut(t, hostarch.Addr(args.ParentTID))
}
kind := ptraceCloneKindClone
- if opts.Vfork {
+ if args.Flags&linux.CLONE_VFORK != 0 {
kind = ptraceCloneKindVfork
- } else if opts.TerminationSignal == linux.SIGCHLD {
+ } else if linux.Signal(args.ExitSignal) == linux.SIGCHLD {
kind = ptraceCloneKindFork
}
- if t.ptraceClone(kind, nt, opts) {
- if opts.Vfork {
+ if t.ptraceClone(kind, nt, args) {
+ if args.Flags&linux.CLONE_VFORK != 0 {
return ntid, &SyscallControl{next: &runSyscallAfterPtraceEventClone{vforkChild: nt, vforkChildTID: ntid}}, nil
}
return ntid, &SyscallControl{next: &runSyscallAfterPtraceEventClone{}}, nil
}
- if opts.Vfork {
+ if args.Flags&linux.CLONE_VFORK != 0 {
t.maybeBeginVforkStop(nt)
return ntid, &SyscallControl{next: &runSyscallAfterVforkStop{childTID: ntid}}, nil
}
@@ -446,27 +343,35 @@ func (r *runSyscallAfterVforkStop) execute(t *Task) taskRunState {
}
// Unshare changes the set of resources t shares with other tasks, as specified
-// by opts.
+// by flags.
//
// Preconditions: The caller must be running on the task goroutine.
-func (t *Task) Unshare(opts *SharingOptions) error {
- // In Linux unshare(2), NewThreadGroup implies NewSignalHandlers and
- // NewSignalHandlers implies NewAddressSpace. All three flags are no-ops if
- // t is the only task using its MM, which due to clone(2)'s rules imply
- // that it is also the only task using its signal handlers / in its thread
- // group, and cause EINVAL to be returned otherwise.
+func (t *Task) Unshare(flags int32) error {
+ // "CLONE_THREAD, CLONE_SIGHAND, and CLONE_VM can be specified in flags if
+ // the caller is single threaded (i.e., it is not sharing its address space
+ // with another process or thread). In this case, these flags have no
+ // effect. (Note also that specifying CLONE_THREAD automatically implies
+ // CLONE_VM, and specifying CLONE_VM automatically implies CLONE_SIGHAND.)
+ // If the process is multithreaded, then the use of these flags results in
+ // an error." - unshare(2). This is incorrect (cf.
+ // kernel/fork.c:ksys_unshare()):
+ //
+ // - CLONE_THREAD does not imply CLONE_VM.
+ //
+ // - CLONE_SIGHAND implies CLONE_THREAD.
+ //
+ // - Only CLONE_VM requires that the caller is not sharing its address
+ // space with another thread. CLONE_SIGHAND requires that the caller is not
+ // sharing its signal handlers, and CLONE_THREAD requires that the caller
+ // is the only thread in its thread group.
//
// Since we don't count the number of tasks using each address space or set
- // of signal handlers, we reject NewSignalHandlers and NewAddressSpace
- // altogether, and interpret NewThreadGroup as requiring that t be the only
- // member of its thread group. This seems to be logically coherent, in the
- // sense that clone(2) allows a task to share signal handlers and address
- // spaces with tasks in other thread groups.
- if opts.NewAddressSpace || opts.NewSignalHandlers {
+ // of signal handlers, we reject CLONE_VM and CLONE_SIGHAND altogether.
+ if flags&(linux.CLONE_VM|linux.CLONE_SIGHAND) != 0 {
return linuxerr.EINVAL
}
creds := t.Credentials()
- if opts.NewThreadGroup {
+ if flags&linux.CLONE_THREAD != 0 {
t.tg.signalHandlers.mu.Lock()
if t.tg.tasksCount != 1 {
t.tg.signalHandlers.mu.Unlock()
@@ -476,7 +381,7 @@ func (t *Task) Unshare(opts *SharingOptions) error {
// This isn't racy because we're the only living task, and therefore
// the only task capable of creating new ones, in our thread group.
}
- if opts.NewUserNamespace {
+ if flags&linux.CLONE_NEWUSER != 0 {
if t.IsChrooted() {
return linuxerr.EPERM
}
@@ -492,7 +397,7 @@ func (t *Task) Unshare(opts *SharingOptions) error {
creds = t.Credentials()
}
haveCapSysAdmin := t.HasCapability(linux.CAP_SYS_ADMIN)
- if opts.NewPIDNamespace {
+ if flags&linux.CLONE_NEWPID != 0 {
if !haveCapSysAdmin {
return linuxerr.EPERM
}
@@ -500,14 +405,14 @@ func (t *Task) Unshare(opts *SharingOptions) error {
}
t.mu.Lock()
// Can't defer unlock: DecRefs must occur without holding t.mu.
- if opts.NewNetworkNamespace {
+ if flags&linux.CLONE_NEWNET != 0 {
if !haveCapSysAdmin {
t.mu.Unlock()
return linuxerr.EPERM
}
t.netns = inet.NewNamespace(t.netns)
}
- if opts.NewUTSNamespace {
+ if flags&linux.CLONE_NEWUTS != 0 {
if !haveCapSysAdmin {
t.mu.Unlock()
return linuxerr.EPERM
@@ -516,7 +421,7 @@ func (t *Task) Unshare(opts *SharingOptions) error {
// new user namespace is used if there is one.
t.utsns = t.utsns.Clone(creds.UserNamespace)
}
- if opts.NewIPCNamespace {
+ if flags&linux.CLONE_NEWIPC != 0 {
if !haveCapSysAdmin {
t.mu.Unlock()
return linuxerr.EPERM
@@ -527,12 +432,12 @@ func (t *Task) Unshare(opts *SharingOptions) error {
t.ipcns = NewIPCNamespace(creds.UserNamespace)
}
var oldFDTable *FDTable
- if opts.NewFiles {
+ if flags&linux.CLONE_FILES != 0 {
oldFDTable = t.fdTable
t.fdTable = oldFDTable.Fork(t)
}
var oldFSContext *FSContext
- if opts.NewFSContext {
+ if flags&linux.CLONE_FS != 0 {
oldFSContext = t.fsContext
t.fsContext = oldFSContext.Fork()
}
diff --git a/pkg/sentry/kernel/task_exit.go b/pkg/sentry/kernel/task_exit.go
index fe08c7519..fbfcc19e5 100644
--- a/pkg/sentry/kernel/task_exit.go
+++ b/pkg/sentry/kernel/task_exit.go
@@ -28,7 +28,6 @@ import (
"errors"
"fmt"
"strconv"
- "strings"
"gvisor.dev/gvisor/pkg/abi/linux"
"gvisor.dev/gvisor/pkg/errors/linuxerr"
@@ -37,58 +36,6 @@ import (
"gvisor.dev/gvisor/pkg/waiter"
)
-// An ExitStatus is a value communicated from an exiting task or thread group
-// to the party that reaps it.
-//
-// +stateify savable
-type ExitStatus struct {
- // Code is the numeric value passed to the call to exit or exit_group that
- // caused the exit. If the exit was not caused by such a call, Code is 0.
- Code int
-
- // Signo is the signal that caused the exit. If the exit was not caused by
- // a signal, Signo is 0.
- Signo int
-}
-
-func (es ExitStatus) String() string {
- var b strings.Builder
- if code := es.Code; code != 0 {
- if b.Len() != 0 {
- b.WriteByte(' ')
- }
- _, _ = fmt.Fprintf(&b, "Code=%d", code)
- }
- if signal := es.Signo; signal != 0 {
- if b.Len() != 0 {
- b.WriteByte(' ')
- }
- _, _ = fmt.Fprintf(&b, "Signal=%d", signal)
- }
- return b.String()
-}
-
-// Signaled returns true if the ExitStatus indicates that the exiting task or
-// thread group was killed by a signal.
-func (es ExitStatus) Signaled() bool {
- return es.Signo != 0
-}
-
-// Status returns the numeric representation of the ExitStatus returned by e.g.
-// the wait4() system call.
-func (es ExitStatus) Status() uint32 {
- return ((uint32(es.Code) & 0xff) << 8) | (uint32(es.Signo) & 0xff)
-}
-
-// ShellExitCode returns the numeric exit code that Bash would return for an
-// exit status of es.
-func (es ExitStatus) ShellExitCode() int {
- if es.Signaled() {
- return 128 + es.Signo
- }
- return es.Code
-}
-
// TaskExitState represents a step in the task exit path.
//
// "Exiting" and "exited" are often ambiguous; prefer to name specific states.
@@ -164,13 +111,13 @@ func (t *Task) killedLocked() bool {
return t.pendingSignals.pendingSet&linux.SignalSetOf(linux.SIGKILL) != 0
}
-// PrepareExit indicates an exit with status es.
+// PrepareExit indicates an exit with the given status.
//
// Preconditions: The caller must be running on the task goroutine.
-func (t *Task) PrepareExit(es ExitStatus) {
+func (t *Task) PrepareExit(ws linux.WaitStatus) {
t.tg.signalHandlers.mu.Lock()
defer t.tg.signalHandlers.mu.Unlock()
- t.exitStatus = es
+ t.exitStatus = ws
}
// PrepareGroupExit indicates a group exit with status es to t's thread group.
@@ -181,7 +128,7 @@ func (t *Task) PrepareExit(es ExitStatus) {
// ptrace.)
//
// Preconditions: The caller must be running on the task goroutine.
-func (t *Task) PrepareGroupExit(es ExitStatus) {
+func (t *Task) PrepareGroupExit(ws linux.WaitStatus) {
t.tg.signalHandlers.mu.Lock()
defer t.tg.signalHandlers.mu.Unlock()
if t.tg.exiting || t.tg.execing != nil {
@@ -199,8 +146,8 @@ func (t *Task) PrepareGroupExit(es ExitStatus) {
return
}
t.tg.exiting = true
- t.tg.exitStatus = es
- t.exitStatus = es
+ t.tg.exitStatus = ws
+ t.exitStatus = ws
for sibling := t.tg.tasks.Front(); sibling != nil; sibling = sibling.Next() {
if sibling != t {
sibling.killLocked()
@@ -208,11 +155,11 @@ func (t *Task) PrepareGroupExit(es ExitStatus) {
}
}
-// Kill requests that all tasks in ts exit as if group exiting with status es.
+// Kill requests that all tasks in ts exit as if group exiting with status ws.
// Kill does not wait for tasks to exit.
//
// Kill has no analogue in Linux; it's provided for save/restore only.
-func (ts *TaskSet) Kill(es ExitStatus) {
+func (ts *TaskSet) Kill(ws linux.WaitStatus) {
ts.mu.Lock()
defer ts.mu.Unlock()
ts.Root.exiting = true
@@ -220,7 +167,7 @@ func (ts *TaskSet) Kill(es ExitStatus) {
t.tg.signalHandlers.mu.Lock()
if !t.tg.exiting {
t.tg.exiting = true
- t.tg.exitStatus = es
+ t.tg.exitStatus = ws
}
t.killLocked()
t.tg.signalHandlers.mu.Unlock()
@@ -731,10 +678,10 @@ func (t *Task) exitNotificationSignal(sig linux.Signal, receiver *Task) *linux.S
info.SetUID(int32(t.Credentials().RealKUID.In(receiver.UserNamespace()).OrOverflow()))
if t.exitStatus.Signaled() {
info.Code = linux.CLD_KILLED
- info.SetStatus(int32(t.exitStatus.Signo))
+ info.SetStatus(int32(t.exitStatus.TerminationSignal()))
} else {
info.Code = linux.CLD_EXITED
- info.SetStatus(int32(t.exitStatus.Code))
+ info.SetStatus(int32(t.exitStatus.ExitStatus()))
}
// TODO(b/72102453): Set utime, stime.
return info
@@ -742,7 +689,7 @@ func (t *Task) exitNotificationSignal(sig linux.Signal, receiver *Task) *linux.S
// ExitStatus returns t's exit status, which is only guaranteed to be
// meaningful if t.ExitState() != TaskExitNone.
-func (t *Task) ExitStatus() ExitStatus {
+func (t *Task) ExitStatus() linux.WaitStatus {
t.tg.pidns.owner.mu.RLock()
defer t.tg.pidns.owner.mu.RUnlock()
t.tg.signalHandlers.mu.Lock()
@@ -752,7 +699,7 @@ func (t *Task) ExitStatus() ExitStatus {
// ExitStatus returns the exit status that would be returned by a consuming
// wait*() on tg.
-func (tg *ThreadGroup) ExitStatus() ExitStatus {
+func (tg *ThreadGroup) ExitStatus() linux.WaitStatus {
tg.pidns.owner.mu.RLock()
defer tg.pidns.owner.mu.RUnlock()
tg.signalHandlers.mu.Lock()
@@ -763,7 +710,9 @@ func (tg *ThreadGroup) ExitStatus() ExitStatus {
return tg.leader.exitStatus
}
-// TerminationSignal returns the thread group's termination signal.
+// TerminationSignal returns the thread group's termination signal, which is
+// the signal that will be sent to its leader's parent when all threads have
+// exited.
func (tg *ThreadGroup) TerminationSignal() linux.Signal {
tg.pidns.owner.mu.RLock()
defer tg.pidns.owner.mu.RUnlock()
@@ -889,8 +838,8 @@ type WaitResult struct {
// Event is exactly one of the events defined above.
Event waiter.EventMask
- // Status is the numeric status associated with the event.
- Status uint32
+ // Status is the wait status associated with the event.
+ Status linux.WaitStatus
}
// Wait waits for an event from a thread group that is a child of t's thread
@@ -1043,7 +992,7 @@ func (t *Task) waitCollectZombieLocked(target *Task, opts *WaitOptions, asPtrace
}
pid := t.tg.pidns.tids[target]
uid := target.Credentials().RealKUID.In(t.UserNamespace()).OrOverflow()
- status := target.exitStatus.Status()
+ status := target.exitStatus
if !opts.ConsumeEvent {
return &WaitResult{
Task: target,
@@ -1057,7 +1006,7 @@ func (t *Task) waitCollectZombieLocked(target *Task, opts *WaitOptions, asPtrace
// differ from that reported by a consuming wait; the latter will return
// the group exit code if one is available.
if target.tg.exiting {
- status = target.tg.exitStatus.Status()
+ status = target.tg.exitStatus
}
// t may be (in the thread group of) target's parent, tracer, or both. We
// don't need to check for !exitTracerAcked because tracees are detached
@@ -1123,12 +1072,11 @@ func (t *Task) waitCollectChildGroupStopLocked(target *Task, opts *WaitOptions)
target.tg.groupStopWaitable = false
}
return &WaitResult{
- Task: target,
- TID: pid,
- UID: uid,
- Event: EventChildGroupStop,
- // There is no name for these status constants.
- Status: (uint32(sig)&0xff)<<8 | 0x7f,
+ Task: target,
+ TID: pid,
+ UID: uid,
+ Event: EventChildGroupStop,
+ Status: linux.WaitStatusStopped(uint32(sig)),
}
}
@@ -1149,7 +1097,7 @@ func (t *Task) waitCollectGroupContinueLocked(target *Task, opts *WaitOptions) *
TID: pid,
UID: uid,
Event: EventGroupContinue,
- Status: 0xffff,
+ Status: linux.WaitStatusContinued(),
}
}
@@ -1177,7 +1125,7 @@ func (t *Task) waitCollectTraceeStopLocked(target *Task, opts *WaitOptions) *Wai
TID: pid,
UID: uid,
Event: EventTraceeStop,
- Status: uint32(code)<<8 | 0x7f,
+ Status: linux.WaitStatusStopped(uint32(code)),
}
}
diff --git a/pkg/sentry/kernel/task_log.go b/pkg/sentry/kernel/task_log.go
index 72b9a0384..8de08151a 100644
--- a/pkg/sentry/kernel/task_log.go
+++ b/pkg/sentry/kernel/task_log.go
@@ -235,7 +235,7 @@ func (t *Task) traceExitEvent() {
if !trace.IsEnabled() {
return
}
- trace.Logf(t.traceContext, traceCategory, "exit status: 0x%x", t.exitStatus.Status())
+ trace.Logf(t.traceContext, traceCategory, "exit status: %s", t.exitStatus)
}
// traceExecEvent is called when a task calls exec.
diff --git a/pkg/sentry/kernel/task_run.go b/pkg/sentry/kernel/task_run.go
index 068f25af1..054ff212f 100644
--- a/pkg/sentry/kernel/task_run.go
+++ b/pkg/sentry/kernel/task_run.go
@@ -377,7 +377,7 @@ func (app *runApp) execute(t *Task) taskRunState {
default:
// What happened? Can't continue.
t.Warningf("Unexpected SwitchToApp error: %v", err)
- t.PrepareExit(ExitStatus{Code: ExtractErrno(err, -1)})
+ t.PrepareExit(linux.WaitStatusExit(int32(ExtractErrno(err, -1))))
return (*runExit)(nil)
}
}
diff --git a/pkg/sentry/kernel/task_signals.go b/pkg/sentry/kernel/task_signals.go
index 72dce7cd9..7065ac79c 100644
--- a/pkg/sentry/kernel/task_signals.go
+++ b/pkg/sentry/kernel/task_signals.go
@@ -157,7 +157,8 @@ func (t *Task) PendingSignals() linux.SignalSet {
// deliverSignal delivers the given signal and returns the following run state.
func (t *Task) deliverSignal(info *linux.SignalInfo, act linux.SigAction) taskRunState {
- sigact := computeAction(linux.Signal(info.Signo), act)
+ sig := linux.Signal(info.Signo)
+ sigact := computeAction(sig, act)
if t.haveSyscallReturn {
if sre, ok := syserror.SyscallRestartErrnoFromReturn(t.Arch().Return()); ok {
@@ -198,14 +199,14 @@ func (t *Task) deliverSignal(info *linux.SignalInfo, act linux.SigAction) taskRu
}
// Attach an fault address if appropriate.
- switch linux.Signal(info.Signo) {
+ switch sig {
case linux.SIGSEGV, linux.SIGFPE, linux.SIGILL, linux.SIGTRAP, linux.SIGBUS:
ucs.FaultAddr = info.Addr()
}
eventchannel.Emit(ucs)
- t.PrepareGroupExit(ExitStatus{Signo: int(info.Signo)})
+ t.PrepareGroupExit(linux.WaitStatusTerminationSignal(sig))
return (*runExit)(nil)
case SignalActionStop:
@@ -225,12 +226,12 @@ func (t *Task) deliverSignal(info *linux.SignalInfo, act linux.SigAction) taskRu
// Send a forced SIGSEGV. If the signal that couldn't be delivered
// was a SIGSEGV, force the handler to SIG_DFL.
- t.forceSignal(linux.SIGSEGV, linux.Signal(info.Signo) == linux.SIGSEGV /* unconditional */)
+ t.forceSignal(linux.SIGSEGV, sig == linux.SIGSEGV /* unconditional */)
t.SendSignal(SignalInfoPriv(linux.SIGSEGV))
}
default:
- panic(fmt.Sprintf("Unknown signal action %+v, %d?", info, computeAction(linux.Signal(info.Signo), act)))
+ panic(fmt.Sprintf("Unknown signal action %+v, %d?", info, computeAction(sig, act)))
}
return (*runInterrupt)(nil)
}
@@ -370,7 +371,7 @@ func (t *Task) Sigtimedwait(set linux.SignalSet, timeout time.Duration) (*linux.
//
// The following errors may be returned:
//
-// syserror.ESRCH - The task has exited.
+// linuxerr.ESRCH - The task has exited.
// linuxerr.EINVAL - The signal is not valid.
// linuxerr.EAGAIN - THe signal is realtime, and cannot be queued.
//
@@ -407,7 +408,7 @@ func (t *Task) sendSignalLocked(info *linux.SignalInfo, group bool) error {
func (t *Task) sendSignalTimerLocked(info *linux.SignalInfo, group bool, timer *IntervalTimer) error {
if t.exitState == TaskExitDead {
- return syserror.ESRCH
+ return linuxerr.ESRCH
}
sig := linux.Signal(info.Signo)
if sig == 0 {
@@ -506,7 +507,7 @@ func (tg *ThreadGroup) applySignalSideEffectsLocked(sig linux.Signal) {
// ignores tg.execing.
if !tg.exiting {
tg.exiting = true
- tg.exitStatus = ExitStatus{Signo: int(linux.SIGKILL)}
+ tg.exitStatus = linux.WaitStatusTerminationSignal(linux.SIGKILL)
}
for t := tg.tasks.Front(); t != nil; t = t.Next() {
t.killLocked()
diff --git a/pkg/sentry/kernel/task_syscall.go b/pkg/sentry/kernel/task_syscall.go
index 409b712d8..0586c9def 100644
--- a/pkg/sentry/kernel/task_syscall.go
+++ b/pkg/sentry/kernel/task_syscall.go
@@ -161,7 +161,7 @@ func (t *Task) doSyscall() taskRunState {
// ok
case linux.SECCOMP_RET_KILL_THREAD:
t.Debugf("Syscall %d: killed by seccomp", sysno)
- t.PrepareExit(ExitStatus{Signo: int(linux.SIGSYS)})
+ t.PrepareExit(linux.WaitStatusTerminationSignal(linux.SIGSYS))
return (*runExit)(nil)
case linux.SECCOMP_RET_TRACE:
t.Debugf("Syscall %d: stopping for PTRACE_EVENT_SECCOMP", sysno)
@@ -311,7 +311,7 @@ func (t *Task) doVsyscall(addr hostarch.Addr, sysno uintptr) taskRunState {
return &runVsyscallAfterPtraceEventSeccomp{addr, sysno, caller}
case linux.SECCOMP_RET_KILL_THREAD:
t.Debugf("vsyscall %d: killed by seccomp", sysno)
- t.PrepareExit(ExitStatus{Signo: int(linux.SIGSYS)})
+ t.PrepareExit(linux.WaitStatusTerminationSignal(linux.SIGSYS))
return (*runExit)(nil)
default:
panic(fmt.Sprintf("Unknown seccomp result %d", r))
@@ -338,7 +338,7 @@ func (r *runVsyscallAfterPtraceEventSeccomp) execute(t *Task) taskRunState {
// Documentation/prctl/seccomp_filter.txt. On Linux, changing orig_ax or ip
// causes do_exit(SIGSYS), and changing sp is ignored.
if (sysno != ^uintptr(0) && sysno != r.sysno) || hostarch.Addr(t.Arch().IP()) != r.addr {
- t.PrepareExit(ExitStatus{Signo: int(linux.SIGSYS)})
+ t.PrepareExit(linux.WaitStatusTerminationSignal(linux.SIGSYS))
return (*runExit)(nil)
}
if sysno == ^uintptr(0) {
diff --git a/pkg/sentry/kernel/task_usermem.go b/pkg/sentry/kernel/task_usermem.go
index 7935d15a6..8e2c36598 100644
--- a/pkg/sentry/kernel/task_usermem.go
+++ b/pkg/sentry/kernel/task_usermem.go
@@ -133,7 +133,7 @@ func (t *Task) CopyOutIovecs(addr hostarch.Addr, src hostarch.AddrRangeSeq) erro
case 8:
const itemLen = 16
if _, ok := addr.AddLength(uint64(src.NumRanges()) * itemLen); !ok {
- return syserror.EFAULT
+ return linuxerr.EFAULT
}
b := t.CopyScratchBuffer(itemLen)
@@ -191,7 +191,7 @@ func (t *Task) CopyInIovecs(addr hostarch.Addr, numIovecs int) (hostarch.AddrRan
case 8:
const itemLen = 16
if _, ok := addr.AddLength(uint64(numIovecs) * itemLen); !ok {
- return hostarch.AddrRangeSeq{}, syserror.EFAULT
+ return hostarch.AddrRangeSeq{}, linuxerr.EFAULT
}
b := t.CopyScratchBuffer(itemLen)
@@ -207,7 +207,7 @@ func (t *Task) CopyInIovecs(addr hostarch.Addr, numIovecs int) (hostarch.AddrRan
}
ar, ok := t.MemoryManager().CheckIORange(base, int64(length))
if !ok {
- return hostarch.AddrRangeSeq{}, syserror.EFAULT
+ return hostarch.AddrRangeSeq{}, linuxerr.EFAULT
}
if numIovecs == 1 {
@@ -253,7 +253,7 @@ func (t *Task) SingleIOSequence(addr hostarch.Addr, length int, opts usermem.IOO
}
ar, ok := t.MemoryManager().CheckIORange(addr, int64(length))
if !ok {
- return usermem.IOSequence{}, syserror.EFAULT
+ return usermem.IOSequence{}, linuxerr.EFAULT
}
return usermem.IOSequence{
IO: t.MemoryManager(),
@@ -313,7 +313,7 @@ func (cc *taskCopyContext) getMemoryManager() (*mm.MemoryManager, error) {
tmm := cc.t.MemoryManager()
cc.t.mu.Unlock()
if !tmm.IncUsers() {
- return nil, syserror.EFAULT
+ return nil, linuxerr.EFAULT
}
return tmm, nil
}
diff --git a/pkg/sentry/kernel/thread_group.go b/pkg/sentry/kernel/thread_group.go
index 3ce11f542..2eda15303 100644
--- a/pkg/sentry/kernel/thread_group.go
+++ b/pkg/sentry/kernel/thread_group.go
@@ -26,7 +26,6 @@ import (
"gvisor.dev/gvisor/pkg/sentry/limits"
"gvisor.dev/gvisor/pkg/sentry/usage"
"gvisor.dev/gvisor/pkg/sync"
- "gvisor.dev/gvisor/pkg/syserror"
)
// A ThreadGroup is a logical grouping of tasks that has widespread
@@ -144,7 +143,7 @@ type ThreadGroup struct {
//
// While exiting is false, exitStatus is protected by the signal mutex.
// When exiting becomes true, exitStatus becomes immutable.
- exitStatus ExitStatus
+ exitStatus linux.WaitStatus
// terminationSignal is the signal that this thread group's leader will
// send to its parent when it exits.
@@ -420,7 +419,7 @@ func (tg *ThreadGroup) ReleaseControllingTTY(tty *TTY) error {
if tg.tty == nil || tg.tty != tty {
tg.signalHandlers.mu.Unlock()
- return syserror.ENOTTY
+ return linuxerr.ENOTTY
}
// "If the process was session leader, then send SIGHUP and SIGCONT to
@@ -474,7 +473,7 @@ func (tg *ThreadGroup) ForegroundProcessGroup(tty *TTY) (int32, error) {
// "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 -1, linuxerr.ENOTTY
}
return int32(tg.processGroup.session.foreground.id), nil
@@ -497,7 +496,7 @@ func (tg *ThreadGroup) SetForegroundProcessGroup(tty *TTY, pgid ProcessGroupID)
// tty must be the controlling terminal.
if tg.tty != tty {
- return -1, syserror.ENOTTY
+ return -1, linuxerr.ENOTTY
}
// pgid must be positive.
@@ -509,7 +508,7 @@ func (tg *ThreadGroup) SetForegroundProcessGroup(tty *TTY, pgid ProcessGroupID)
// pid namespaces.
pg, ok := tg.pidns.processGroups[pgid]
if !ok {
- return -1, syserror.ESRCH
+ return -1, linuxerr.ESRCH
}
// pg must be part of this process's session.