summaryrefslogtreecommitdiffhomepage
diff options
context:
space:
mode:
-rw-r--r--pkg/abi/linux/BUILD1
-rw-r--r--pkg/abi/linux/shm.go75
-rw-r--r--pkg/refs/refcounter.go8
-rw-r--r--pkg/sentry/context/context.go20
-rw-r--r--pkg/sentry/fs/dirent_refs_test.go62
-rw-r--r--pkg/sentry/kernel/BUILD1
-rw-r--r--pkg/sentry/kernel/ipc_namespace.go15
-rw-r--r--pkg/sentry/kernel/shm/BUILD40
-rw-r--r--pkg/sentry/kernel/shm/device.go20
-rw-r--r--pkg/sentry/kernel/shm/shm.go630
-rw-r--r--pkg/sentry/kernel/task.go3
-rw-r--r--pkg/sentry/kernel/task_clone.go4
-rw-r--r--pkg/sentry/mm/BUILD2
-rw-r--r--pkg/sentry/mm/shm.go66
-rw-r--r--pkg/sentry/syscalls/linux/BUILD2
-rw-r--r--pkg/sentry/syscalls/linux/linux64.go8
-rw-r--r--pkg/sentry/syscalls/linux/sys_shm.go155
-rw-r--r--runsc/boot/loader.go2
18 files changed, 1072 insertions, 42 deletions
diff --git a/pkg/abi/linux/BUILD b/pkg/abi/linux/BUILD
index a428e61a3..693ce0fdd 100644
--- a/pkg/abi/linux/BUILD
+++ b/pkg/abi/linux/BUILD
@@ -51,6 +51,7 @@ go_library(
"sched.go",
"seccomp.go",
"sem.go",
+ "shm.go",
"signal.go",
"socket.go",
"time.go",
diff --git a/pkg/abi/linux/shm.go b/pkg/abi/linux/shm.go
new file mode 100644
index 000000000..9149ed094
--- /dev/null
+++ b/pkg/abi/linux/shm.go
@@ -0,0 +1,75 @@
+// Copyright 2018 Google Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package linux
+
+// shmat(2) flags. Source: include/uapi/linux/shm.h
+const (
+ SHM_RDONLY = 010000 // Read-only access.
+ SHM_RND = 020000 // Round attach address to SHMLBA boundary.
+ SHM_REMAP = 040000 // Take-over region on attach.
+ SHM_EXEC = 0100000 // Execution access.
+)
+
+// IPCPerm.Mode upper byte flags. Source: include/linux/shm.h
+const (
+ SHM_DEST = 01000 // Segment will be destroyed on last detach.
+ SHM_LOCKED = 02000 // Segment will not be swapped.
+ SHM_HUGETLB = 04000 // Segment will use huge TLB pages.
+ SHM_NORESERVE = 010000 // Don't check for reservations.
+)
+
+// Additional Linux-only flags for shmctl(2). Source: include/uapi/linux/shm.h
+const (
+ SHM_LOCK = 11
+ SHM_UNLOCK = 12
+ SHM_STAT = 13
+ SHM_INFO = 14
+)
+
+// ShmidDS is equivalent to struct shmid64_ds. Source:
+// include/uapi/asm-generic/shmbuf.h
+type ShmidDS struct {
+ ShmPerm IPCPerm
+ ShmSegsz uint64
+ ShmAtime TimeT
+ ShmDtime TimeT
+ ShmCtime TimeT
+ ShmCpid int32
+ ShmLpid int32
+ ShmNattach uint64
+
+ Unused4 uint64
+ Unused5 uint64
+}
+
+// ShmParams is equivalent to struct shminfo. Source: include/uapi/linux/shm.h
+type ShmParams struct {
+ ShmMax uint64
+ ShmMin uint64
+ ShmMni uint64
+ ShmSeg uint64
+ ShmAll uint64
+}
+
+// ShmInfo is equivalent to struct shm_info. Source: include/uapi/linux/shm.h
+type ShmInfo struct {
+ UsedIDs int32 // Number of currently existing segments.
+ _ [4]byte
+ ShmTot uint64 // Total number of shared memory pages.
+ ShmRss uint64 // Number of resident shared memory pages.
+ ShmSwp uint64 // Number of swapped shared memory pages.
+ SwapAttempts uint64 // Unused since Linux 2.4.
+ SwapSuccesses uint64 // Unused since Linux 2.4.
+}
diff --git a/pkg/refs/refcounter.go b/pkg/refs/refcounter.go
index 1036553c7..3162001e1 100644
--- a/pkg/refs/refcounter.go
+++ b/pkg/refs/refcounter.go
@@ -194,9 +194,11 @@ type AtomicRefCount struct {
weakRefs ilist.List `state:"nosave"`
}
-// TestReadRefs returns the current reference count of r. Use only for tests.
-func (r *AtomicRefCount) TestReadRefs() int64 {
- return atomic.LoadInt64(&r.refCount)
+// ReadRefs returns the current number of references. The returned count is
+// inherently racy and is unsafe to use without external synchronization.
+func (r *AtomicRefCount) ReadRefs() int64 {
+ // Account for the internal -1 offset on refcounts.
+ return atomic.LoadInt64(&r.refCount) + 1
}
// IncRef increments this object's reference count. While the count is kept
diff --git a/pkg/sentry/context/context.go b/pkg/sentry/context/context.go
index e0dffafba..598c5b4ff 100644
--- a/pkg/sentry/context/context.go
+++ b/pkg/sentry/context/context.go
@@ -20,6 +20,26 @@ import (
"gvisor.googlesource.com/gvisor/pkg/log"
)
+type contextID int
+
+// Globally accessible values from a context. These keys are defined in the
+// context package to resolve dependency cycles by not requiring the caller to
+// import packages usually required to get these information.
+const (
+ // CtxThreadGroupID is the current thread group ID when a context represents
+ // a task context. The value is represented as an int32.
+ CtxThreadGroupID contextID = iota
+)
+
+// ThreadGroupIDFromContext returns the current thread group ID when ctx
+// represents a task context.
+func ThreadGroupIDFromContext(ctx Context) (tgid int32, ok bool) {
+ if tgid := ctx.Value(CtxThreadGroupID); tgid != nil {
+ return tgid.(int32), true
+ }
+ return 0, false
+}
+
// A Context represents a thread of execution (hereafter "goroutine" to reflect
// Go idiosyncrasy). It carries state associated with the goroutine across API
// boundaries.
diff --git a/pkg/sentry/fs/dirent_refs_test.go b/pkg/sentry/fs/dirent_refs_test.go
index 8ce9ba02d..f9dcba316 100644
--- a/pkg/sentry/fs/dirent_refs_test.go
+++ b/pkg/sentry/fs/dirent_refs_test.go
@@ -33,8 +33,8 @@ func TestWalkPositive(t *testing.T) {
ctx := contexttest.Context(t)
root := NewDirent(newMockDirInode(ctx, nil), "root")
- if got := root.TestReadRefs(); got != 0 {
- t.Fatalf("root has a ref count of %d, want %d", got, 0)
+ if got := root.ReadRefs(); got != 1 {
+ t.Fatalf("root has a ref count of %d, want %d", got, 1)
}
name := "d"
@@ -43,22 +43,22 @@ func TestWalkPositive(t *testing.T) {
t.Fatalf("root.walk(root, %q) got %v, want nil", name, err)
}
- if got := root.TestReadRefs(); got != 1 {
- t.Fatalf("root has a ref count of %d, want %d", got, 1)
+ if got := root.ReadRefs(); got != 2 {
+ t.Fatalf("root has a ref count of %d, want %d", got, 2)
}
- if got := d.TestReadRefs(); got != 0 {
- t.Fatalf("child name = %q has a ref count of %d, want %d", d.name, got, 0)
+ if got := d.ReadRefs(); got != 1 {
+ t.Fatalf("child name = %q has a ref count of %d, want %d", d.name, got, 1)
}
d.DecRef()
- if got := root.TestReadRefs(); got != 0 {
- t.Fatalf("root has a ref count of %d, want %d", got, 0)
+ if got := root.ReadRefs(); got != 1 {
+ t.Fatalf("root has a ref count of %d, want %d", got, 1)
}
- if got := d.TestReadRefs(); got != -1 {
- t.Fatalf("child name = %q has a ref count of %d, want %d", d.name, got, -1)
+ if got := d.ReadRefs(); got != 0 {
+ t.Fatalf("child name = %q has a ref count of %d, want %d", d.name, got, 0)
}
root.flush()
@@ -76,8 +76,8 @@ func TestWalkNegative(t *testing.T) {
root := NewDirent(NewEmptyDir(ctx, nil), "root")
mn := root.Inode.InodeOperations.(*mockInodeOperationsLookupNegative)
- if got := root.TestReadRefs(); got != 0 {
- t.Fatalf("root has a ref count of %d, want %d", got, 0)
+ if got := root.ReadRefs(); got != 1 {
+ t.Fatalf("root has a ref count of %d, want %d", got, 1)
}
name := "d"
@@ -88,7 +88,7 @@ func TestWalkNegative(t *testing.T) {
}
}
- if got := root.TestReadRefs(); got != 0 {
+ if got := root.ReadRefs(); got != 1 {
t.Fatalf("root has a ref count of %d, want %d", got, 1)
}
@@ -110,14 +110,14 @@ func TestWalkNegative(t *testing.T) {
t.Fatalf("root found positive child at %q, want negative", name)
}
- if got := child.(*Dirent).TestReadRefs(); got != 1 {
- t.Fatalf("child has a ref count of %d, want %d", got, 1)
+ if got := child.(*Dirent).ReadRefs(); got != 2 {
+ t.Fatalf("child has a ref count of %d, want %d", got, 2)
}
child.DecRef()
- if got := child.(*Dirent).TestReadRefs(); got != 0 {
- t.Fatalf("child has a ref count of %d, want %d", got, 0)
+ if got := child.(*Dirent).ReadRefs(); got != 1 {
+ t.Fatalf("child has a ref count of %d, want %d", got, 1)
}
if got := len(root.children); got != 1 {
@@ -126,7 +126,7 @@ func TestWalkNegative(t *testing.T) {
root.DecRef()
- if got := root.TestReadRefs(); got != -1 {
+ if got := root.ReadRefs(); got != 0 {
t.Fatalf("root has a ref count of %d, want %d", got, 0)
}
@@ -184,12 +184,12 @@ func TestHashNegativeToPositive(t *testing.T) {
t.Fatalf("got negative Dirent, want positive")
}
- if got := d.TestReadRefs(); got != 0 {
- t.Fatalf("child %q has a ref count of %d, want %d", name, got, 0)
+ if got := d.ReadRefs(); got != 1 {
+ t.Fatalf("child %q has a ref count of %d, want %d", name, got, 1)
}
- if got := root.TestReadRefs(); got != 1 {
- t.Fatalf("root has a ref count of %d, want %d", got, 1)
+ if got := root.ReadRefs(); got != 2 {
+ t.Fatalf("root has a ref count of %d, want %d", got, 2)
}
if got := len(root.children); got != 1 {
@@ -291,12 +291,12 @@ func TestCreateExtraRefs(t *testing.T) {
{
desc: "Create caching",
root: NewDirent(NewEmptyDir(ctx, NewDirentCache(1)), "root"),
- refs: 1,
+ refs: 2,
},
{
desc: "Create not caching",
root: NewDirent(NewEmptyDir(ctx, nil), "root"),
- refs: 0,
+ refs: 1,
},
} {
t.Run(test.desc, func(t *testing.T) {
@@ -307,7 +307,7 @@ func TestCreateExtraRefs(t *testing.T) {
}
d := f.Dirent
- if got := d.TestReadRefs(); got != test.refs {
+ if got := d.ReadRefs(); got != test.refs {
t.Errorf("dirent has a ref count of %d, want %d", got, test.refs)
}
})
@@ -347,8 +347,8 @@ func TestRemoveExtraRefs(t *testing.T) {
t.Fatalf("root.Remove(root, %q) failed: %v", name, err)
}
- if got := d.TestReadRefs(); got != 0 {
- t.Fatalf("dirent has a ref count of %d, want %d", got, 0)
+ if got := d.ReadRefs(); got != 1 {
+ t.Fatalf("dirent has a ref count of %d, want %d", got, 1)
}
d.DecRef()
@@ -406,11 +406,11 @@ func TestRenameExtraRefs(t *testing.T) {
newParent.flush()
// Expect to have only active references.
- if got := renamed.TestReadRefs(); got != 0 {
- t.Errorf("renamed has ref count %d, want only active references %d", got, 0)
+ if got := renamed.ReadRefs(); got != 1 {
+ t.Errorf("renamed has ref count %d, want only active references %d", got, 1)
}
- if got := replaced.TestReadRefs(); got != 0 {
- t.Errorf("replaced has ref count %d, want only active references %d", got, 0)
+ if got := replaced.ReadRefs(); got != 1 {
+ t.Errorf("replaced has ref count %d, want only active references %d", got, 1)
}
})
}
diff --git a/pkg/sentry/kernel/BUILD b/pkg/sentry/kernel/BUILD
index 62794cff5..377c94e4c 100644
--- a/pkg/sentry/kernel/BUILD
+++ b/pkg/sentry/kernel/BUILD
@@ -184,6 +184,7 @@ go_library(
"//pkg/sentry/kernel/kdefs",
"//pkg/sentry/kernel/sched",
"//pkg/sentry/kernel/semaphore",
+ "//pkg/sentry/kernel/shm",
"//pkg/sentry/kernel/time",
"//pkg/sentry/limits",
"//pkg/sentry/loader",
diff --git a/pkg/sentry/kernel/ipc_namespace.go b/pkg/sentry/kernel/ipc_namespace.go
index 78737f58f..3049fead4 100644
--- a/pkg/sentry/kernel/ipc_namespace.go
+++ b/pkg/sentry/kernel/ipc_namespace.go
@@ -15,18 +15,26 @@
package kernel
import (
+ "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/auth"
"gvisor.googlesource.com/gvisor/pkg/sentry/kernel/semaphore"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/shm"
)
// IPCNamespace represents an IPC namespace.
type IPCNamespace struct {
+ // User namespace which owns this IPC namespace. Immutable.
+ userNS *auth.UserNamespace
+
semaphores *semaphore.Registry
+ shms *shm.Registry
}
// NewIPCNamespace creates a new IPC namespace.
-func NewIPCNamespace() *IPCNamespace {
+func NewIPCNamespace(userNS *auth.UserNamespace) *IPCNamespace {
return &IPCNamespace{
+ userNS: userNS,
semaphores: semaphore.NewRegistry(),
+ shms: shm.NewRegistry(userNS),
}
}
@@ -35,6 +43,11 @@ func (i *IPCNamespace) SemaphoreRegistry() *semaphore.Registry {
return i.semaphores
}
+// ShmRegistry returns the shm segment registry for this namespace.
+func (i *IPCNamespace) ShmRegistry() *shm.Registry {
+ return i.shms
+}
+
// IPCNamespace returns the task's IPC namespace.
func (t *Task) IPCNamespace() *IPCNamespace {
t.mu.Lock()
diff --git a/pkg/sentry/kernel/shm/BUILD b/pkg/sentry/kernel/shm/BUILD
new file mode 100644
index 000000000..182cc1c76
--- /dev/null
+++ b/pkg/sentry/kernel/shm/BUILD
@@ -0,0 +1,40 @@
+package(licenses = ["notice"]) # Apache 2.0
+
+load("@io_bazel_rules_go//go:def.bzl", "go_library")
+load("//tools/go_stateify:defs.bzl", "go_stateify")
+
+go_stateify(
+ name = "shm_state",
+ srcs = [
+ "shm.go",
+ ],
+ out = "shm_autogen_state.go",
+ package = "shm",
+)
+
+go_library(
+ name = "shm",
+ srcs = [
+ "device.go",
+ "shm.go",
+ "shm_autogen_state.go",
+ ],
+ importpath = "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/shm",
+ visibility = ["//pkg/sentry:internal"],
+ deps = [
+ "//pkg/abi/linux",
+ "//pkg/log",
+ "//pkg/refs",
+ "//pkg/sentry/context",
+ "//pkg/sentry/device",
+ "//pkg/sentry/fs",
+ "//pkg/sentry/kernel/auth",
+ "//pkg/sentry/kernel/time",
+ "//pkg/sentry/memmap",
+ "//pkg/sentry/platform",
+ "//pkg/sentry/usage",
+ "//pkg/sentry/usermem",
+ "//pkg/state",
+ "//pkg/syserror",
+ ],
+)
diff --git a/pkg/sentry/kernel/shm/device.go b/pkg/sentry/kernel/shm/device.go
new file mode 100644
index 000000000..b0dacdbe0
--- /dev/null
+++ b/pkg/sentry/kernel/shm/device.go
@@ -0,0 +1,20 @@
+// Copyright 2018 Google Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package shm
+
+import "gvisor.googlesource.com/gvisor/pkg/sentry/device"
+
+// shmDevice is the kernel shm device.
+var shmDevice = device.NewAnonDevice()
diff --git a/pkg/sentry/kernel/shm/shm.go b/pkg/sentry/kernel/shm/shm.go
new file mode 100644
index 000000000..7217e8103
--- /dev/null
+++ b/pkg/sentry/kernel/shm/shm.go
@@ -0,0 +1,630 @@
+// Copyright 2018 Google Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+// Package shm implements sysv shared memory segments.
+//
+// Known missing features:
+//
+// - SHM_LOCK/SHM_UNLOCK are no-ops. The sentry currently doesn't implement
+// memory locking in general.
+//
+// - SHM_HUGETLB and related flags for shmget(2) are ignored. There's no easy
+// way to implement hugetlb support on a per-map basis, and it has no impact
+// on correctness.
+//
+// - SHM_NORESERVE for shmget(2) is ignored, the sentry doesn't implement swap
+// so it's meaningless to reserve space for swap.
+//
+// - No per-process segment size enforcement. This feature probably isn't used
+// much anyways, since Linux sets the per-process limits to the system-wide
+// limits by default.
+//
+// Lock ordering: mm.mappingMu -> shm registry lock -> shm lock
+package shm
+
+import (
+ "fmt"
+ "math"
+ "sync"
+
+ "gvisor.googlesource.com/gvisor/pkg/abi/linux"
+ "gvisor.googlesource.com/gvisor/pkg/log"
+ "gvisor.googlesource.com/gvisor/pkg/refs"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/context"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/fs"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/auth"
+ ktime "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/time"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/memmap"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/platform"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/usage"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/usermem"
+ "gvisor.googlesource.com/gvisor/pkg/syserror"
+)
+
+// Various limits for shared memory segments.
+const (
+ // shmsTotalMaxPages is the system-wide limit on all shared memory segments, measured
+ // in number of pages.
+ shmsTotalMaxPages = math.MaxInt64 // SHMALL
+
+ // shmMaxSize is the maximum size of a single segment, in bytes.
+ shmMaxSize = math.MaxInt64 // SHMMAX
+
+ // shmMinSize is the minimum specifiable size of a segment, effectively
+ // yielding a size rounded up to the next page size. Measured in bytes.
+ shmMinSize = 1 // SHMMIN
+
+ // shmsTotalMax is the maximum number of segments on the system.
+ shmsTotalMax = 4096 // SHMMNI
+)
+
+// 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.
+type Registry struct {
+ // userNS owns the IPC namespace this registry belong to. Immutable.
+ userNS *auth.UserNamespace
+
+ mu sync.Mutex `state:"nosave"`
+
+ // shms maps segment ids to segments. Protected by mu.
+ shms map[int32]*Shm
+
+ // Sum of the sizes of all existing segments rounded up to page size, in
+ // units of page size. Protected by mu.
+ totalPages uint64
+
+ // lastIDUsed is protected by mu.
+ lastIDUsed int32
+}
+
+// NewRegistry creates a new shm registry.
+func NewRegistry(userNS *auth.UserNamespace) *Registry {
+ return &Registry{
+ userNS: userNS,
+ shms: make(map[int32]*Shm),
+ }
+}
+
+// FindByID looks up a segment given an ID.
+func (r *Registry) FindByID(id int32) *Shm {
+ r.mu.Lock()
+ defer r.mu.Unlock()
+ return r.shms[id]
+}
+
+// Precondition: Caller must hold r.mu.
+func (r *Registry) findByKey(key int32) *Shm {
+ for _, v := range r.shms {
+ if v.key == key {
+ return v
+ }
+ }
+ return nil
+}
+
+// FindOrCreate looks up or creates a segment in the registry. It's functionally
+// analogous to open(2).
+func (r *Registry) FindOrCreate(ctx context.Context, pid, key int32, size uint64, mode linux.FileMode, private, create, exclusive bool) (*Shm, error) {
+ if create && (size < shmMinSize || size > shmMaxSize) {
+ // "A new segment was to be created and size is less than SHMMIN or
+ // greater than SHMMAX." - man shmget(2)
+ return nil, syserror.EINVAL
+ }
+
+ r.mu.Lock()
+ defer r.mu.Unlock()
+
+ if len(r.shms) >= shmsTotalMax {
+ // "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.findByKey(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, syserror.EACCES
+ }
+
+ 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, syserror.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
+ }
+
+ 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
+ if val, ok := usermem.Addr(size).RoundUp(); ok {
+ sizeAligned = uint64(val)
+ } else {
+ return nil, syserror.EINVAL
+ }
+
+ if numPages := sizeAligned / usermem.PageSize; r.totalPages+numPages > shmsTotalMaxPages {
+ // "... allocating a segment of the requested size would cause the
+ // system to exceed the system-wide limit on shared memory (SHMALL)."
+ // - man shmget(2)
+ return nil, syserror.ENOSPC
+ }
+
+ // Need to create a new segment.
+ creator := fs.FileOwnerFromContext(ctx)
+ perms := fs.FilePermsFromMode(mode)
+ return r.newShm(ctx, pid, key, creator, perms, size)
+}
+
+// newShm creates a new segment in the registry.
+func (r *Registry) newShm(ctx context.Context, pid, key int32, creator fs.FileOwner, perms fs.FilePermissions, size uint64) (*Shm, error) {
+ p := platform.FromContext(ctx)
+ if p == nil {
+ panic(fmt.Sprintf("context.Context %T lacks non-nil value for key %T", ctx, platform.CtxPlatform))
+ }
+
+ effectiveSize := uint64(usermem.Addr(size).MustRoundUp())
+ fr, err := p.Memory().Allocate(effectiveSize, usage.Anonymous)
+ if err != nil {
+ return nil, err
+ }
+
+ shm := &Shm{
+ p: p,
+ registry: r,
+ creator: creator,
+ size: size,
+ effectiveSize: effectiveSize,
+ fr: fr,
+ key: key,
+ perms: perms,
+ owner: creator,
+ creatorPID: pid,
+ changeTime: ktime.NowFromContext(ctx),
+ }
+
+ // 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
+ r.shms[id] = shm
+ shm.ID = id
+
+ r.totalPages += effectiveSize / usermem.PageSize
+
+ return shm, nil
+ }
+ }
+
+ log.Warningf("Shm ids exhuasted, they may be leaking")
+ return nil, syserror.ENOSPC
+}
+
+// IPCInfo reports global parameters for sysv shared memory segments on this
+// system. See shmctl(IPC_INFO).
+func (r *Registry) IPCInfo() *linux.ShmParams {
+ return &linux.ShmParams{
+ ShmMax: shmMaxSize,
+ ShmMin: shmMinSize,
+ ShmMni: shmsTotalMax,
+ ShmSeg: shmsTotalMax, // Linux also sets this to SHMMNI.
+ ShmAll: shmsTotalMaxPages,
+ }
+}
+
+// ShmInfo reports linux-specific global parameters for sysv shared memory
+// segments on this system. See shmctl(SHM_INFO).
+func (r *Registry) ShmInfo() *linux.ShmInfo {
+ r.mu.Lock()
+ defer r.mu.Unlock()
+
+ return &linux.ShmInfo{
+ UsedIDs: int32(r.lastIDUsed),
+ ShmTot: r.totalPages,
+ ShmRss: r.totalPages, // We could probably get a better estimate from memory accounting.
+ ShmSwp: 0, // No reclaim at the moment.
+ }
+}
+
+// remove unregisters a segment from this registry, preventing it from being
+// discovered in the future. Caller is responsible for ensuring s is destroyed.
+//
+// Precondition: To preserve lock ordering, caller must not hold s.mu.
+func (r *Registry) remove(s *Shm) {
+ r.mu.Lock()
+ defer r.mu.Unlock()
+ delete(r.shms, s.ID)
+ r.totalPages -= s.effectiveSize / usermem.PageSize
+}
+
+// Shm represents a single shared memory segment.
+//
+// Shm segment are backed directly by an allocation from platform
+// memory. Segments are always mapped as a whole, greatly simplifying how
+// mappings are tracked. However note that mremap and munmap calls may cause the
+// vma for a segment to become fragmented; which requires special care when
+// unmapping a segment. See mm/shm.go.
+//
+// Segments persist until they are explicitly marked for destruction via
+// shmctl(SHM_RMID).
+//
+// Shm implements memmap.Mappable and memmap.MappingIdentity.
+type Shm struct {
+ // AtomicRefCount tracks the number of references to this segment from
+ // maps. A segment always holds a reference to itself, until it's marked for
+ // destruction.
+ refs.AtomicRefCount
+
+ p platform.Platform
+
+ // registry points to the shm registry containing this segment. Immutable.
+ registry *Registry
+
+ // ID is the kernel identifier for this segment. Immutable.
+ ID int32
+
+ // 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
+
+ // effectiveSize of the segment, rounding up to the next page
+ // boundary. Immutable.
+ //
+ // Invariant: effectiveSize must be a multiple of usermem.PageSize.
+ effectiveSize uint64
+
+ // fr is the offset into platform.Memory() that backs this contents of this
+ // segment. Immutable.
+ fr platform.FileRange
+
+ // key is the public identifier for this segment.
+ key int32
+
+ // mu protects all fields below.
+ mu sync.Mutex `state:"nosave"`
+
+ // perms is the access permissions for the segment.
+ perms fs.FilePermissions
+
+ // owner of this segment.
+ owner fs.FileOwner
+ // attachTime is updated on every successful shmat.
+ attachTime ktime.Time
+ // detachTime is updated on every successful shmdt.
+ detachTime ktime.Time
+ // changeTime is updated on every successful changes to the segment via
+ // shmctl(IPC_SET).
+ changeTime ktime.Time
+
+ // creatorPID is the PID of the process that created the segment.
+ creatorPID int32
+ // lastAttachDetachPID is the pid of the process that issued the last shmat
+ // or shmdt syscall.
+ lastAttachDetachPID int32
+
+ // pendingDestruction indicates the segment was marked as destroyed through
+ // shmctl(IPC_RMID). When marked as destroyed, the segment will not be found
+ // in the registry and can no longer be attached. When the last user
+ // detaches from the segment, it is destroyed. Protected by mu.
+ pendingDestruction bool
+}
+
+// MappedName implements memmap.MappingIdentity.MappedName.
+func (s *Shm) MappedName(ctx context.Context) string {
+ return fmt.Sprintf("SYSV%08d", s.key)
+}
+
+// DeviceID implements memmap.MappingIdentity.DeviceID.
+func (s *Shm) DeviceID() uint64 {
+ return shmDevice.DeviceID()
+}
+
+// InodeID implements memmap.MappingIdentity.InodeID.
+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)
+}
+
+// DecRef overrides refs.RefCount.DecRef with a destructor.
+func (s *Shm) DecRef() {
+ s.DecRefWithDestructor(s.destroy)
+}
+
+// Msync implements memmap.MappingIdentity.Msync. Msync is a no-op for shm
+// segments.
+func (s *Shm) Msync(context.Context, memmap.MappableRange) error {
+ return nil
+}
+
+// AddMapping implements memmap.Mappable.AddMapping.
+func (s *Shm) AddMapping(ctx context.Context, ms memmap.MappingSpace, ar usermem.AddrRange, offset uint64) error {
+ s.mu.Lock()
+ defer s.mu.Unlock()
+ s.attachTime = ktime.NowFromContext(ctx)
+ if pid, ok := context.ThreadGroupIDFromContext(ctx); ok {
+ s.lastAttachDetachPID = pid
+ } else {
+ // AddMapping is called during a syscall, so ctx should always be a task
+ // context.
+ log.Warningf("Adding mapping to shm %+v but couldn't get the current pid; not updating the last attach pid", s)
+ }
+ return nil
+}
+
+// RemoveMapping implements memmap.Mappable.RemoveMapping.
+func (s *Shm) RemoveMapping(ctx context.Context, ms memmap.MappingSpace, ar usermem.AddrRange, offset uint64) {
+ s.mu.Lock()
+ defer s.mu.Unlock()
+ // TODO: RemoveMapping may be called during task exit, when ctx
+ // is context.Background. Gracefully handle missing clocks. Failing to
+ // update the detach time in these cases is ok, since no one can observe the
+ // omission.
+ if clock := ktime.RealtimeClockFromContext(ctx); clock != nil {
+ s.detachTime = clock.Now()
+ }
+
+ // If called from a non-task context we also won't have a threadgroup
+ // id. Silently skip updating the lastAttachDetachPid in that case.
+ if pid, ok := context.ThreadGroupIDFromContext(ctx); ok {
+ s.lastAttachDetachPID = pid
+ } else {
+ log.Debugf("Couldn't obtain pid when removing mapping to shm %+v, not updating the last detach pid.", s)
+ }
+}
+
+// CopyMapping implements memmap.Mappable.CopyMapping.
+func (s *Shm) CopyMapping(ctx context.Context, ms memmap.MappingSpace, srcAR, dstAR usermem.AddrRange, offset uint64) error {
+ return nil
+}
+
+// Translate implements memmap.Mappable.Translate.
+func (s *Shm) Translate(ctx context.Context, required, optional memmap.MappableRange, at usermem.AccessType) ([]memmap.Translation, error) {
+ var err error
+ if required.End > s.fr.Length() {
+ err = &memmap.BusError{syserror.EFAULT}
+ }
+ if source := optional.Intersect(memmap.MappableRange{0, s.fr.Length()}); source.Length() != 0 {
+ return []memmap.Translation{
+ {
+ Source: source,
+ File: s.p.Memory(),
+ Offset: s.fr.Start + source.Start,
+ },
+ }, err
+ }
+ return nil, err
+}
+
+// InvalidateUnsavable implements memmap.Mappable.InvalidateUnsavable.
+func (s *Shm) InvalidateUnsavable(ctx context.Context) error {
+ return nil
+}
+
+// AttachOpts describes various flags passed to shmat(2).
+type AttachOpts struct {
+ Execute bool
+ Readonly bool
+ Remap bool
+}
+
+// ConfigureAttach creates an mmap configuration for the segment with the
+// requested attach options.
+//
+// ConfigureAttach returns with a ref on s on success. The caller should drop
+// this once the map is installed. This reference prevents s from being
+// destroyed before the returned configuration is used.
+func (s *Shm) ConfigureAttach(ctx context.Context, addr usermem.Addr, opts AttachOpts) (memmap.MMapOpts, error) {
+ s.mu.Lock()
+ defer s.mu.Unlock()
+ if s.pendingDestruction && s.ReadRefs() == 0 {
+ return memmap.MMapOpts{}, syserror.EIDRM
+ }
+
+ if !s.checkPermissions(ctx, fs.PermMask{
+ Read: true,
+ Write: !opts.Readonly,
+ Execute: opts.Execute,
+ }) {
+ // "The calling process does not have the required permissions for the
+ // requested attach type, and does not have the CAP_IPC_OWNER capability
+ // in the user namespace that governs its IPC namespace." - man shmat(2)
+ return memmap.MMapOpts{}, syserror.EACCES
+ }
+ s.IncRef()
+ return memmap.MMapOpts{
+ Length: s.size,
+ Offset: 0,
+ Addr: addr,
+ Fixed: opts.Remap,
+ Perms: usermem.AccessType{
+ Read: true,
+ Write: !opts.Readonly,
+ Execute: opts.Execute,
+ },
+ MaxPerms: usermem.AnyAccess,
+ Mappable: s,
+ MappingIdentity: s,
+ }, nil
+}
+
+// EffectiveSize returns the size of the underlying shared memory segment. This
+// may be larger than the requested size at creation, due to rounding to page
+// boundaries.
+func (s *Shm) EffectiveSize() uint64 {
+ return s.effectiveSize
+}
+
+// IPCStat returns information about a shm. See shmctl(IPC_STAT).
+func (s *Shm) IPCStat(ctx context.Context) (*linux.ShmidDS, error) {
+ s.mu.Lock()
+ defer s.mu.Unlock()
+
+ // "The caller must have read permission on the shared memory segment."
+ // - man shmctl(2)
+ if !s.checkPermissions(ctx, 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
+ // namespace." - man shmctl(2)
+ return nil, syserror.EACCES
+ }
+
+ var mode uint16
+ if s.pendingDestruction {
+ mode |= linux.SHM_DEST
+ }
+ creds := auth.CredentialsFromContext(ctx)
+
+ nattach := uint64(s.ReadRefs())
+ // Don't report the self-reference we keep prior to being marked for
+ // destruction. However, also don't report a count of -1 for segments marked
+ // as destroyed, with no mappings.
+ if !s.pendingDestruction {
+ nattach--
+ }
+
+ 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()),
+ Seq: 0, // IPC sequences not supported.
+ },
+ ShmSegsz: s.size,
+ ShmAtime: s.attachTime.TimeT(),
+ ShmDtime: s.detachTime.TimeT(),
+ ShmCtime: s.changeTime.TimeT(),
+ ShmCpid: s.creatorPID,
+ ShmLpid: s.lastAttachDetachPID,
+ ShmNattach: nattach,
+ }
+
+ return ds, nil
+}
+
+// Set modifies attributes for a segment. See shmctl(IPC_SET).
+func (s *Shm) Set(ctx context.Context, ds *linux.ShmidDS) error {
+ s.mu.Lock()
+ defer s.mu.Unlock()
+
+ if !s.checkOwnership(ctx) {
+ return syserror.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() {
+ return syserror.EINVAL
+ }
+
+ // 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.owner.UID = uid
+ s.owner.GID = gid
+
+ s.changeTime = ktime.NowFromContext(ctx)
+ return nil
+}
+
+func (s *Shm) destroy() {
+ s.registry.remove(s)
+ s.p.Memory().DecRef(s.fr)
+}
+
+// MarkDestroyed marks a shm for destruction. The shm is actually destroyed once
+// it has no references. See shmctl(IPC_RMID).
+func (s *Shm) MarkDestroyed() {
+ s.mu.Lock()
+ defer s.mu.Unlock()
+ // Prevent the segment from being found in the registry.
+ s.key = linux.IPC_PRIVATE
+ s.pendingDestruction = true
+ s.DecRef()
+}
+
+// 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 490f795c2..7763050a5 100644
--- a/pkg/sentry/kernel/task.go
+++ b/pkg/sentry/kernel/task.go
@@ -21,6 +21,7 @@ import (
"gvisor.googlesource.com/gvisor/pkg/abi/linux"
"gvisor.googlesource.com/gvisor/pkg/bpf"
"gvisor.googlesource.com/gvisor/pkg/sentry/arch"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/context"
"gvisor.googlesource.com/gvisor/pkg/sentry/fs"
"gvisor.googlesource.com/gvisor/pkg/sentry/inet"
"gvisor.googlesource.com/gvisor/pkg/sentry/kernel/auth"
@@ -559,6 +560,8 @@ func (t *Task) Value(key interface{}) interface{} {
return t
case auth.CtxCredentials:
return t.creds
+ case context.CtxThreadGroupID:
+ return int32(t.ThreadGroup().ID())
case fs.CtxRoot:
return t.FSContext().RootDirectory()
case inet.CtxStack:
diff --git a/pkg/sentry/kernel/task_clone.go b/pkg/sentry/kernel/task_clone.go
index 3a74abdfb..0c2427952 100644
--- a/pkg/sentry/kernel/task_clone.go
+++ b/pkg/sentry/kernel/task_clone.go
@@ -197,7 +197,7 @@ func (t *Task) Clone(opts *CloneOptions) (ThreadID, *SyscallControl, error) {
if opts.NewIPCNamespace {
// Note that "If CLONE_NEWIPC is set, then create the process in a new IPC
// namespace"
- ipcns = NewIPCNamespace()
+ ipcns = NewIPCNamespace(userns)
}
tc, err := t.tc.Fork(t, !opts.NewAddressSpace)
@@ -449,7 +449,7 @@ func (t *Task) Unshare(opts *SharingOptions) error {
}
// Note that "If CLONE_NEWIPC is set, then create the process in a new IPC
// namespace"
- t.ipcns = NewIPCNamespace()
+ t.ipcns = NewIPCNamespace(t.creds.UserNamespace)
}
if opts.NewFiles {
oldFDMap := t.tr.FDMap
diff --git a/pkg/sentry/mm/BUILD b/pkg/sentry/mm/BUILD
index 39bde2be3..258389bb2 100644
--- a/pkg/sentry/mm/BUILD
+++ b/pkg/sentry/mm/BUILD
@@ -107,6 +107,7 @@ go_library(
"pma_set.go",
"proc_pid_maps.go",
"save_restore.go",
+ "shm.go",
"special_mappable.go",
"syscalls.go",
"vma.go",
@@ -123,6 +124,7 @@ go_library(
"//pkg/sentry/context",
"//pkg/sentry/fs",
"//pkg/sentry/fs/proc/seqfile",
+ "//pkg/sentry/kernel/shm",
"//pkg/sentry/limits",
"//pkg/sentry/memmap",
"//pkg/sentry/platform",
diff --git a/pkg/sentry/mm/shm.go b/pkg/sentry/mm/shm.go
new file mode 100644
index 000000000..bab137a5a
--- /dev/null
+++ b/pkg/sentry/mm/shm.go
@@ -0,0 +1,66 @@
+// Copyright 2018 Google Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package mm
+
+import (
+ "gvisor.googlesource.com/gvisor/pkg/sentry/context"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/shm"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/usermem"
+ "gvisor.googlesource.com/gvisor/pkg/syserror"
+)
+
+// DetachShm unmaps a sysv shared memory segment.
+func (mm *MemoryManager) DetachShm(ctx context.Context, addr usermem.Addr) error {
+ if addr != addr.RoundDown() {
+ // "... shmaddr is not aligned on a page boundary." - man shmdt(2)
+ return syserror.EINVAL
+ }
+
+ var detached *shm.Shm
+ mm.mappingMu.Lock()
+ defer mm.mappingMu.Unlock()
+
+ // Find and remove the first vma containing an address >= addr that maps a
+ // segment originally attached at addr.
+ vseg := mm.vmas.LowerBoundSegment(addr)
+ for vseg.Ok() {
+ vma := vseg.ValuePtr()
+ if shm, ok := vma.mappable.(*shm.Shm); ok && vseg.Start() >= addr && uint64(vseg.Start()-addr) == vma.off {
+ detached = shm
+ vseg = mm.unmapLocked(ctx, vseg.Range()).NextSegment()
+ break
+ } else {
+ vseg = vseg.NextSegment()
+ }
+ }
+
+ if detached == nil {
+ // There is no shared memory segment attached at addr.
+ return syserror.EINVAL
+ }
+
+ // Remove all vmas that could have been created by the same attach.
+ end := addr + usermem.Addr(detached.EffectiveSize())
+ for vseg.Ok() && vseg.End() <= end {
+ vma := vseg.ValuePtr()
+ if vma.mappable == detached && uint64(vseg.Start()-addr) == vma.off {
+ vseg = mm.unmapLocked(ctx, vseg.Range()).NextSegment()
+ } else {
+ vseg = vseg.NextSegment()
+ }
+ }
+
+ return nil
+}
diff --git a/pkg/sentry/syscalls/linux/BUILD b/pkg/sentry/syscalls/linux/BUILD
index bc67ebf30..f9e0a4be3 100644
--- a/pkg/sentry/syscalls/linux/BUILD
+++ b/pkg/sentry/syscalls/linux/BUILD
@@ -44,6 +44,7 @@ go_library(
"sys_rusage.go",
"sys_sched.go",
"sys_sem.go",
+ "sys_shm.go",
"sys_signal.go",
"sys_socket.go",
"sys_stat.go",
@@ -84,6 +85,7 @@ go_library(
"//pkg/sentry/kernel/pipe",
"//pkg/sentry/kernel/sched",
"//pkg/sentry/kernel/semaphore",
+ "//pkg/sentry/kernel/shm",
"//pkg/sentry/kernel/time",
"//pkg/sentry/limits",
"//pkg/sentry/memmap",
diff --git a/pkg/sentry/syscalls/linux/linux64.go b/pkg/sentry/syscalls/linux/linux64.go
index 44db2d582..237c61007 100644
--- a/pkg/sentry/syscalls/linux/linux64.go
+++ b/pkg/sentry/syscalls/linux/linux64.go
@@ -75,9 +75,9 @@ var AMD64 = &kernel.SyscallTable{
26: Msync,
27: Mincore,
28: Madvise,
- // 29: Shmget, TODO
- // 30: Shmat, TODO
- // 31: Shmctl, TODO
+ 29: Shmget,
+ 30: Shmat,
+ 31: Shmctl,
32: Dup,
33: Dup2,
34: Pause,
@@ -113,7 +113,7 @@ var AMD64 = &kernel.SyscallTable{
64: Semget,
65: Semop,
66: Semctl,
- // 67: Shmdt, TODO
+ 67: Shmdt,
// 68: Msgget, TODO
// 69: Msgsnd, TODO
// 70: Msgrcv, TODO
diff --git a/pkg/sentry/syscalls/linux/sys_shm.go b/pkg/sentry/syscalls/linux/sys_shm.go
new file mode 100644
index 000000000..48ff1d5f0
--- /dev/null
+++ b/pkg/sentry/syscalls/linux/sys_shm.go
@@ -0,0 +1,155 @@
+// Copyright 2018 Google Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+// http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package linux
+
+import (
+ "gvisor.googlesource.com/gvisor/pkg/abi/linux"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/arch"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/kernel"
+ "gvisor.googlesource.com/gvisor/pkg/sentry/kernel/shm"
+ "gvisor.googlesource.com/gvisor/pkg/syserror"
+)
+
+// Shmget implements shmget(2).
+func Shmget(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel.SyscallControl, error) {
+ key := args[0].Int()
+ size := uint64(args[1].SizeT())
+ flag := args[2].Int()
+
+ private := key == linux.IPC_PRIVATE
+ create := flag&linux.IPC_CREAT == linux.IPC_CREAT
+ exclusive := flag&linux.IPC_EXCL == linux.IPC_EXCL
+ mode := linux.FileMode(flag & 0777)
+
+ pid := int32(t.ThreadGroup().ID())
+ r := t.IPCNamespace().ShmRegistry()
+ segment, err := r.FindOrCreate(t, pid, key, size, mode, private, create, exclusive)
+ if err != nil {
+ return 0, nil, err
+ }
+ return uintptr(segment.ID), nil, nil
+}
+
+// findSegment retrives a shm segment by the given id.
+func findSegment(t *kernel.Task, id int32) (*shm.Shm, error) {
+ r := t.IPCNamespace().ShmRegistry()
+ segment := r.FindByID(id)
+ if segment == nil {
+ // No segment with provided id.
+ return nil, syserror.EINVAL
+ }
+ return segment, nil
+}
+
+// Shmat implements shmat(2).
+func Shmat(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel.SyscallControl, error) {
+ id := args[0].Int()
+ addr := args[1].Pointer()
+ flag := args[2].Int()
+
+ segment, err := findSegment(t, id)
+ if err != nil {
+ return 0, nil, syserror.EINVAL
+ }
+
+ opts, err := segment.ConfigureAttach(t, addr, shm.AttachOpts{
+ Execute: flag&linux.SHM_EXEC == linux.SHM_EXEC,
+ Readonly: flag&linux.SHM_RDONLY == linux.SHM_RDONLY,
+ Remap: flag&linux.SHM_REMAP == linux.SHM_REMAP,
+ })
+ if err != nil {
+ return 0, nil, err
+ }
+ defer segment.DecRef()
+ addr, err = t.MemoryManager().MMap(t, opts)
+ return uintptr(addr), nil, err
+}
+
+// Shmdt implements shmdt(2).
+func Shmdt(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel.SyscallControl, error) {
+ addr := args[0].Pointer()
+ err := t.MemoryManager().DetachShm(t, addr)
+ return 0, nil, err
+}
+
+// Shmctl implements shmctl(2).
+func Shmctl(t *kernel.Task, args arch.SyscallArguments) (uintptr, *kernel.SyscallControl, error) {
+ id := args[0].Int()
+ cmd := args[1].Int()
+ buf := args[2].Pointer()
+
+ r := t.IPCNamespace().ShmRegistry()
+
+ switch cmd {
+ case linux.SHM_STAT:
+ // Technically, we should be treating id as "an index into the kernel's
+ // internal array that maintains information about all shared memory
+ // segments on the system". Since we don't track segments in an array,
+ // we'll just pretend the shmid is the index and do the same thing as
+ // IPC_STAT. Linux also uses the index as the shmid.
+ fallthrough
+ case linux.IPC_STAT:
+ segment, err := findSegment(t, id)
+ if err != nil {
+ return 0, nil, syserror.EINVAL
+ }
+
+ stat, err := segment.IPCStat(t)
+ if err == nil {
+ _, err = t.CopyOut(buf, stat)
+ }
+ return 0, nil, err
+
+ case linux.IPC_INFO:
+ params := r.IPCInfo()
+ _, err := t.CopyOut(buf, params)
+ return 0, nil, err
+
+ case linux.SHM_INFO:
+ info := r.ShmInfo()
+ _, err := t.CopyOut(buf, info)
+ return 0, nil, err
+ }
+
+ // Remaining commands refer to a specific segment.
+ segment, err := findSegment(t, id)
+ if err != nil {
+ return 0, nil, syserror.EINVAL
+ }
+
+ switch cmd {
+ case linux.IPC_SET:
+ var ds linux.ShmidDS
+ _, err = t.CopyIn(buf, &ds)
+ if err != nil {
+ return 0, nil, err
+ }
+ err = segment.Set(t, &ds)
+ return 0, nil, err
+
+ case linux.IPC_RMID:
+ segment.MarkDestroyed()
+ return 0, nil, nil
+
+ case linux.SHM_LOCK, linux.SHM_UNLOCK:
+ // We currently do not support memmory locking anywhere.
+ // mlock(2)/munlock(2) are currently stubbed out as no-ops so do the
+ // same here.
+ return 0, nil, nil
+
+ default:
+ return 0, nil, syserror.EINVAL
+ }
+}
diff --git a/runsc/boot/loader.go b/runsc/boot/loader.go
index 0ff54d349..566f2eb46 100644
--- a/runsc/boot/loader.go
+++ b/runsc/boot/loader.go
@@ -146,7 +146,7 @@ func New(spec *specs.Spec, conf *Config, controllerFD int, ioFDs []int, console
// not configurable from runtime spec.
utsns := kernel.NewUTSNamespace(spec.Hostname, "", creds.UserNamespace)
- ipcns := kernel.NewIPCNamespace()
+ ipcns := kernel.NewIPCNamespace(creds.UserNamespace)
if err := enableStrace(conf); err != nil {
return nil, fmt.Errorf("failed to enable strace: %v", err)