summaryrefslogtreecommitdiffhomepage
path: root/pkg/sentry
diff options
context:
space:
mode:
Diffstat (limited to 'pkg/sentry')
-rw-r--r--pkg/sentry/kernel/BUILD1
-rw-r--r--pkg/sentry/kernel/task.go21
-rw-r--r--pkg/sentry/kernel/task_clone.go36
-rw-r--r--pkg/sentry/seccheck/BUILD54
-rw-r--r--pkg/sentry/seccheck/clone.go53
-rw-r--r--pkg/sentry/seccheck/seccheck.go136
-rw-r--r--pkg/sentry/seccheck/seccheck_test.go157
-rw-r--r--pkg/sentry/seccheck/task.go39
8 files changed, 496 insertions, 1 deletions
diff --git a/pkg/sentry/kernel/BUILD b/pkg/sentry/kernel/BUILD
index 816e60329..c0f13bf52 100644
--- a/pkg/sentry/kernel/BUILD
+++ b/pkg/sentry/kernel/BUILD
@@ -268,6 +268,7 @@ go_library(
"//pkg/sentry/mm",
"//pkg/sentry/pgalloc",
"//pkg/sentry/platform",
+ "//pkg/sentry/seccheck",
"//pkg/sentry/socket/netlink/port",
"//pkg/sentry/socket/unix/transport",
"//pkg/sentry/time",
diff --git a/pkg/sentry/kernel/task.go b/pkg/sentry/kernel/task.go
index 59eeb253d..9a95bf44c 100644
--- a/pkg/sentry/kernel/task.go
+++ b/pkg/sentry/kernel/task.go
@@ -30,6 +30,7 @@ import (
"gvisor.dev/gvisor/pkg/sentry/kernel/sched"
ktime "gvisor.dev/gvisor/pkg/sentry/kernel/time"
"gvisor.dev/gvisor/pkg/sentry/platform"
+ "gvisor.dev/gvisor/pkg/sentry/seccheck"
"gvisor.dev/gvisor/pkg/sentry/usage"
"gvisor.dev/gvisor/pkg/sentry/vfs"
"gvisor.dev/gvisor/pkg/sync"
@@ -874,3 +875,23 @@ func (t *Task) ResetKcov() {
t.kcov = nil
}
}
+
+// Preconditions: The TaskSet mutex must be locked.
+func (t *Task) loadSeccheckInfoLocked(req seccheck.TaskFieldSet, mask *seccheck.TaskFieldSet, info *seccheck.TaskInfo) {
+ if req.Contains(seccheck.TaskFieldThreadID) {
+ info.ThreadID = int32(t.k.tasks.Root.tids[t])
+ mask.Add(seccheck.TaskFieldThreadID)
+ }
+ if req.Contains(seccheck.TaskFieldThreadStartTime) {
+ info.ThreadStartTime = t.startTime
+ mask.Add(seccheck.TaskFieldThreadStartTime)
+ }
+ if req.Contains(seccheck.TaskFieldThreadGroupID) {
+ info.ThreadGroupID = int32(t.k.tasks.Root.tgids[t.tg])
+ mask.Add(seccheck.TaskFieldThreadGroupID)
+ }
+ if req.Contains(seccheck.TaskFieldThreadGroupStartTime) {
+ info.ThreadGroupStartTime = t.tg.leader.startTime
+ mask.Add(seccheck.TaskFieldThreadGroupStartTime)
+ }
+}
diff --git a/pkg/sentry/kernel/task_clone.go b/pkg/sentry/kernel/task_clone.go
index da4b77ca2..26a981f36 100644
--- a/pkg/sentry/kernel/task_clone.go
+++ b/pkg/sentry/kernel/task_clone.go
@@ -23,6 +23,7 @@ import (
"gvisor.dev/gvisor/pkg/errors/linuxerr"
"gvisor.dev/gvisor/pkg/hostarch"
"gvisor.dev/gvisor/pkg/sentry/inet"
+ "gvisor.dev/gvisor/pkg/sentry/seccheck"
"gvisor.dev/gvisor/pkg/usermem"
)
@@ -235,7 +236,23 @@ func (t *Task) Clone(args *linux.CloneArgs) (ThreadID, *SyscallControl, error) {
// nt that it must receive before its task goroutine starts running.
tid := nt.k.tasks.Root.IDOfTask(nt)
defer nt.Start(tid)
- t.traceCloneEvent(tid)
+
+ if seccheck.Global.Enabled(seccheck.PointClone) {
+ mask, info := getCloneSeccheckInfo(t, nt, args)
+ if err := seccheck.Global.Clone(t, mask, &info); err != nil {
+ // nt has been visible to the rest of the system since NewTask, so
+ // it may be blocking execve or a group stop, have been notified
+ // for group signal delivery, had children reparented to it, etc.
+ // Thus we can't just drop it on the floor. Instead, instruct the
+ // task goroutine to exit immediately, as quietly as possible.
+ nt.exitTracerNotified = true
+ nt.exitTracerAcked = true
+ nt.exitParentNotified = true
+ nt.exitParentAcked = true
+ nt.runState = (*runExitMain)(nil)
+ return 0, nil, err
+ }
+ }
// "If fork/clone and execve are allowed by @prog, any child processes will
// be constrained to the same filters and system call ABI as the parent." -
@@ -260,6 +277,7 @@ func (t *Task) Clone(args *linux.CloneArgs) (ThreadID, *SyscallControl, error) {
ntid.CopyOut(t, hostarch.Addr(args.ParentTID))
}
+ t.traceCloneEvent(tid)
kind := ptraceCloneKindClone
if args.Flags&linux.CLONE_VFORK != 0 {
kind = ptraceCloneKindVfork
@@ -279,6 +297,22 @@ func (t *Task) Clone(args *linux.CloneArgs) (ThreadID, *SyscallControl, error) {
return ntid, nil, nil
}
+func getCloneSeccheckInfo(t, nt *Task, args *linux.CloneArgs) (seccheck.CloneFieldSet, seccheck.CloneInfo) {
+ req := seccheck.Global.CloneReq()
+ info := seccheck.CloneInfo{
+ Credentials: t.Credentials(),
+ Args: *args,
+ }
+ var mask seccheck.CloneFieldSet
+ mask.Add(seccheck.CloneFieldCredentials)
+ mask.Add(seccheck.CloneFieldArgs)
+ t.k.tasks.mu.RLock()
+ defer t.k.tasks.mu.RUnlock()
+ t.loadSeccheckInfoLocked(req.Invoker, &mask.Invoker, &info.Invoker)
+ nt.loadSeccheckInfoLocked(req.Created, &mask.Created, &info.Created)
+ return mask, info
+}
+
// maybeBeginVforkStop checks if a previously-started vfork child is still
// running and has not yet released its MM, such that its parent t should enter
// a vforkStop.
diff --git a/pkg/sentry/seccheck/BUILD b/pkg/sentry/seccheck/BUILD
new file mode 100644
index 000000000..943fa180d
--- /dev/null
+++ b/pkg/sentry/seccheck/BUILD
@@ -0,0 +1,54 @@
+load("//tools:defs.bzl", "go_library", "go_test")
+load("//tools/go_fieldenum:defs.bzl", "go_fieldenum")
+load("//tools/go_generics:defs.bzl", "go_template_instance")
+
+licenses(["notice"])
+
+go_fieldenum(
+ name = "seccheck_fieldenum",
+ srcs = [
+ "clone.go",
+ "task.go",
+ ],
+ out = "seccheck_fieldenum.go",
+ package = "seccheck",
+)
+
+go_template_instance(
+ name = "seqatomic_checkerslice",
+ out = "seqatomic_checkerslice_unsafe.go",
+ package = "seccheck",
+ suffix = "CheckerSlice",
+ template = "//pkg/sync/seqatomic:generic_seqatomic",
+ types = {
+ "Value": "[]Checker",
+ },
+)
+
+go_library(
+ name = "seccheck",
+ srcs = [
+ "clone.go",
+ "seccheck.go",
+ "seccheck_fieldenum.go",
+ "seqatomic_checkerslice_unsafe.go",
+ "task.go",
+ ],
+ visibility = ["//:sandbox"],
+ deps = [
+ "//pkg/abi/linux",
+ "//pkg/context",
+ "//pkg/gohacks",
+ "//pkg/sentry/kernel/auth",
+ "//pkg/sentry/kernel/time",
+ "//pkg/sync",
+ ],
+)
+
+go_test(
+ name = "seccheck_test",
+ size = "small",
+ srcs = ["seccheck_test.go"],
+ library = ":seccheck",
+ deps = ["//pkg/context"],
+)
diff --git a/pkg/sentry/seccheck/clone.go b/pkg/sentry/seccheck/clone.go
new file mode 100644
index 000000000..7546fa021
--- /dev/null
+++ b/pkg/sentry/seccheck/clone.go
@@ -0,0 +1,53 @@
+// 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 seccheck
+
+import (
+ "gvisor.dev/gvisor/pkg/abi/linux"
+ "gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/sentry/kernel/auth"
+)
+
+// CloneInfo contains information used by the Clone checkpoint.
+//
+// +fieldenum Clone
+type CloneInfo struct {
+ // Invoker identifies the invoking thread.
+ Invoker TaskInfo
+
+ // Credentials are the invoking thread's credentials.
+ Credentials *auth.Credentials
+
+ // Args contains the arguments to kernel.Task.Clone().
+ Args linux.CloneArgs
+
+ // Created identifies the created thread.
+ Created TaskInfo
+}
+
+// CloneReq returns fields required by the Clone checkpoint.
+func (s *state) CloneReq() CloneFieldSet {
+ return s.cloneReq.Load()
+}
+
+// Clone is called at the Clone checkpoint.
+func (s *state) Clone(ctx context.Context, mask CloneFieldSet, info *CloneInfo) error {
+ for _, c := range s.getCheckers() {
+ if err := c.Clone(ctx, mask, *info); err != nil {
+ return err
+ }
+ }
+ return nil
+}
diff --git a/pkg/sentry/seccheck/seccheck.go b/pkg/sentry/seccheck/seccheck.go
new file mode 100644
index 000000000..b6c9d44ce
--- /dev/null
+++ b/pkg/sentry/seccheck/seccheck.go
@@ -0,0 +1,136 @@
+// 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 seccheck defines a structure for dynamically-configured security
+// checks in the sentry.
+package seccheck
+
+import (
+ "sync/atomic"
+
+ "gvisor.dev/gvisor/pkg/context"
+ "gvisor.dev/gvisor/pkg/sync"
+)
+
+// A Point represents a checkpoint, a point at which a security check occurs.
+type Point uint
+
+// PointX represents the checkpoint X.
+const (
+ PointClone Point = iota
+ // Add new Points above this line.
+ pointLength
+
+ numPointBitmaskUint32s = (int(pointLength)-1)/32 + 1
+)
+
+// A Checker performs security checks at checkpoints.
+//
+// Each Checker method X is called at checkpoint X; if the method may return a
+// non-nil error and does so, it causes the checked operation to fail
+// immediately (without calling subsequent Checkers) and return the error. The
+// info argument contains information relevant to the check. The mask argument
+// indicates what fields in info are valid; the mask should usually be a
+// superset of fields requested by the Checker's corresponding CheckerReq, but
+// may be missing requested fields in some cases (e.g. if the Checker is
+// registered concurrently with invocations of checkpoints).
+type Checker interface {
+ Clone(ctx context.Context, mask CloneFieldSet, info CloneInfo) error
+}
+
+// CheckerDefaults may be embedded by implementations of Checker to obtain
+// no-op implementations of Checker methods that may be explicitly overridden.
+type CheckerDefaults struct{}
+
+// Clone implements Checker.Clone.
+func (CheckerDefaults) Clone(ctx context.Context, mask CloneFieldSet, info CloneInfo) error {
+ return nil
+}
+
+// CheckerReq indicates what checkpoints a corresponding Checker runs at, and
+// what information it requires at those checkpoints.
+type CheckerReq struct {
+ // Points are the set of checkpoints for which the corresponding Checker
+ // must be called. Note that methods not specified in Points may still be
+ // called; implementations of Checker may embed CheckerDefaults to obtain
+ // no-op implementations of Checker methods.
+ Points []Point
+
+ // All of the following fields indicate what fields in the corresponding
+ // XInfo struct will be requested at the corresponding checkpoint.
+ Clone CloneFields
+}
+
+// Global is the method receiver of all seccheck functions.
+var Global state
+
+// state is the type of global, and is separated out for testing.
+type state struct {
+ // registrationMu serializes all changes to the set of registered Checkers
+ // for all checkpoints.
+ registrationMu sync.Mutex
+
+ // enabledPoints is a bitmask of checkpoints for which at least one Checker
+ // is registered.
+ //
+ // enabledPoints is accessed using atomic memory operations. Mutation of
+ // enabledPoints is serialized by registrationMu.
+ enabledPoints [numPointBitmaskUint32s]uint32
+
+ // registrationSeq supports store-free atomic reads of registeredCheckers.
+ registrationSeq sync.SeqCount
+
+ // checkers is the set of all registered Checkers in order of execution.
+ //
+ // checkers is accessed using instantiations of SeqAtomic functions.
+ // Mutation of checkers is serialized by registrationMu.
+ checkers []Checker
+
+ // All of the following xReq variables indicate what fields in the
+ // corresponding XInfo struct have been requested by any registered
+ // checker, are accessed using atomic memory operations, and are mutated
+ // with registrationMu locked.
+ cloneReq CloneFieldSet
+}
+
+// AppendChecker registers the given Checker to execute at checkpoints. The
+// Checker will execute after all previously-registered Checkers, and only if
+// those Checkers return a nil error.
+func (s *state) AppendChecker(c Checker, req *CheckerReq) {
+ s.registrationMu.Lock()
+ defer s.registrationMu.Unlock()
+ s.cloneReq.AddFieldsLoadable(req.Clone)
+ s.appendCheckerLocked(c)
+ for _, p := range req.Points {
+ word, bit := p/32, p%32
+ atomic.StoreUint32(&s.enabledPoints[word], s.enabledPoints[word]|(uint32(1)<<bit))
+ }
+}
+
+// Enabled returns true if any Checker is registered for the given checkpoint.
+func (s *state) Enabled(p Point) bool {
+ word, bit := p/32, p%32
+ return atomic.LoadUint32(&s.enabledPoints[word])&(uint32(1)<<bit) != 0
+}
+
+func (s *state) getCheckers() []Checker {
+ return SeqAtomicLoadCheckerSlice(&s.registrationSeq, &s.checkers)
+}
+
+// Preconditions: s.registrationMu must be locked.
+func (s *state) appendCheckerLocked(c Checker) {
+ s.registrationSeq.BeginWrite()
+ s.checkers = append(s.checkers, c)
+ s.registrationSeq.EndWrite()
+}
diff --git a/pkg/sentry/seccheck/seccheck_test.go b/pkg/sentry/seccheck/seccheck_test.go
new file mode 100644
index 000000000..687810d18
--- /dev/null
+++ b/pkg/sentry/seccheck/seccheck_test.go
@@ -0,0 +1,157 @@
+// 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 seccheck
+
+import (
+ "errors"
+ "testing"
+
+ "gvisor.dev/gvisor/pkg/context"
+)
+
+type testChecker struct {
+ CheckerDefaults
+
+ onClone func(ctx context.Context, mask CloneFieldSet, info CloneInfo) error
+}
+
+// Clone implements Checker.Clone.
+func (c *testChecker) Clone(ctx context.Context, mask CloneFieldSet, info CloneInfo) error {
+ if c.onClone == nil {
+ return nil
+ }
+ return c.onClone(ctx, mask, info)
+}
+
+func TestNoChecker(t *testing.T) {
+ var s state
+ if s.Enabled(PointClone) {
+ t.Errorf("Enabled(PointClone): got true, wanted false")
+ }
+}
+
+func TestCheckerNotRegisteredForPoint(t *testing.T) {
+ var s state
+ s.AppendChecker(&testChecker{}, &CheckerReq{})
+ if s.Enabled(PointClone) {
+ t.Errorf("Enabled(PointClone): got true, wanted false")
+ }
+}
+
+func TestCheckerRegistered(t *testing.T) {
+ var s state
+ checkerCalled := false
+ s.AppendChecker(&testChecker{onClone: func(ctx context.Context, mask CloneFieldSet, info CloneInfo) error {
+ checkerCalled = true
+ return nil
+ }}, &CheckerReq{
+ Points: []Point{PointClone},
+ Clone: CloneFields{
+ Credentials: true,
+ },
+ })
+
+ if !s.Enabled(PointClone) {
+ t.Errorf("Enabled(PointClone): got false, wanted true")
+ }
+ if !s.CloneReq().Contains(CloneFieldCredentials) {
+ t.Errorf("CloneReq().Contains(CloneFieldCredentials): got false, wanted true")
+ }
+ if err := s.Clone(context.Background(), CloneFieldSet{}, &CloneInfo{}); err != nil {
+ t.Errorf("Clone(): got %v, wanted nil", err)
+ }
+ if !checkerCalled {
+ t.Errorf("Clone() did not call Checker.Clone()")
+ }
+}
+
+func TestMultipleCheckersRegistered(t *testing.T) {
+ var s state
+ checkersCalled := [2]bool{}
+ s.AppendChecker(&testChecker{onClone: func(ctx context.Context, mask CloneFieldSet, info CloneInfo) error {
+ checkersCalled[0] = true
+ return nil
+ }}, &CheckerReq{
+ Points: []Point{PointClone},
+ Clone: CloneFields{
+ Args: true,
+ },
+ })
+ s.AppendChecker(&testChecker{onClone: func(ctx context.Context, mask CloneFieldSet, info CloneInfo) error {
+ checkersCalled[1] = true
+ return nil
+ }}, &CheckerReq{
+ Points: []Point{PointClone},
+ Clone: CloneFields{
+ Created: TaskFields{
+ ThreadID: true,
+ },
+ },
+ })
+
+ if !s.Enabled(PointClone) {
+ t.Errorf("Enabled(PointClone): got false, wanted true")
+ }
+ // CloneReq() should return the union of requested fields from all calls to
+ // AppendChecker.
+ req := s.CloneReq()
+ if !req.Contains(CloneFieldArgs) {
+ t.Errorf("req.Contains(CloneFieldArgs): got false, wanted true")
+ }
+ if !req.Created.Contains(TaskFieldThreadID) {
+ t.Errorf("req.Created.Contains(TaskFieldThreadID): got false, wanted true")
+ }
+ if err := s.Clone(context.Background(), CloneFieldSet{}, &CloneInfo{}); err != nil {
+ t.Errorf("Clone(): got %v, wanted nil", err)
+ }
+ for i := range checkersCalled {
+ if !checkersCalled[i] {
+ t.Errorf("Clone() did not call Checker.Clone() index %d", i)
+ }
+ }
+}
+
+func TestCheckpointReturnsFirstCheckerError(t *testing.T) {
+ errFirstChecker := errors.New("first Checker error")
+ errSecondChecker := errors.New("second Checker error")
+
+ var s state
+ checkersCalled := [2]bool{}
+ s.AppendChecker(&testChecker{onClone: func(ctx context.Context, mask CloneFieldSet, info CloneInfo) error {
+ checkersCalled[0] = true
+ return errFirstChecker
+ }}, &CheckerReq{
+ Points: []Point{PointClone},
+ })
+ s.AppendChecker(&testChecker{onClone: func(ctx context.Context, mask CloneFieldSet, info CloneInfo) error {
+ checkersCalled[1] = true
+ return errSecondChecker
+ }}, &CheckerReq{
+ Points: []Point{PointClone},
+ })
+
+ if !s.Enabled(PointClone) {
+ t.Errorf("Enabled(PointClone): got false, wanted true")
+ }
+ if err := s.Clone(context.Background(), CloneFieldSet{}, &CloneInfo{}); err != errFirstChecker {
+ t.Errorf("Clone(): got %v, wanted %v", err, errFirstChecker)
+ }
+ if !checkersCalled[0] {
+ t.Errorf("Clone() did not call first Checker")
+ }
+ if checkersCalled[1] {
+ t.Errorf("Clone() called second Checker")
+ }
+}
diff --git a/pkg/sentry/seccheck/task.go b/pkg/sentry/seccheck/task.go
new file mode 100644
index 000000000..1dee33203
--- /dev/null
+++ b/pkg/sentry/seccheck/task.go
@@ -0,0 +1,39 @@
+// 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 seccheck
+
+import (
+ ktime "gvisor.dev/gvisor/pkg/sentry/kernel/time"
+)
+
+// TaskInfo contains information unambiguously identifying a single thread
+// and/or its containing process.
+//
+// +fieldenum Task
+type TaskInfo struct {
+ // ThreadID is the thread's ID in the root PID namespace.
+ ThreadID int32
+
+ // ThreadStartTime is the thread's CLOCK_REALTIME start time.
+ ThreadStartTime ktime.Time
+
+ // ThreadGroupID is the thread's group leader's ID in the root PID
+ // namespace.
+ ThreadGroupID int32
+
+ // ThreadGroupStartTime is the thread's group leader's CLOCK_REALTIME start
+ // time.
+ ThreadGroupStartTime ktime.Time
+}