diff options
author | Googler <noreply@google.com> | 2018-04-27 10:37:02 -0700 |
---|---|---|
committer | Adin Scannell <ascannell@google.com> | 2018-04-28 01:44:26 -0400 |
commit | d02b74a5dcfed4bfc8f2f8e545bca4d2afabb296 (patch) | |
tree | 54f95eef73aee6bacbfc736fffc631be2605ed53 /pkg/sentry/platform/ptrace | |
parent | f70210e742919f40aa2f0934a22f1c9ba6dada62 (diff) |
Check in gVisor.
PiperOrigin-RevId: 194583126
Change-Id: Ica1d8821a90f74e7e745962d71801c598c652463
Diffstat (limited to 'pkg/sentry/platform/ptrace')
-rw-r--r-- | pkg/sentry/platform/ptrace/BUILD | 31 | ||||
-rw-r--r-- | pkg/sentry/platform/ptrace/ptrace.go | 242 | ||||
-rw-r--r-- | pkg/sentry/platform/ptrace/ptrace_unsafe.go | 166 | ||||
-rw-r--r-- | pkg/sentry/platform/ptrace/stub_amd64.s | 114 | ||||
-rw-r--r-- | pkg/sentry/platform/ptrace/stub_unsafe.go | 98 | ||||
-rw-r--r-- | pkg/sentry/platform/ptrace/subprocess.go | 559 | ||||
-rw-r--r-- | pkg/sentry/platform/ptrace/subprocess_amd64.go | 104 | ||||
-rw-r--r-- | pkg/sentry/platform/ptrace/subprocess_linux.go | 146 | ||||
-rw-r--r-- | pkg/sentry/platform/ptrace/subprocess_linux_amd64_unsafe.go | 109 | ||||
-rw-r--r-- | pkg/sentry/platform/ptrace/subprocess_unsafe.go | 28 |
10 files changed, 1597 insertions, 0 deletions
diff --git a/pkg/sentry/platform/ptrace/BUILD b/pkg/sentry/platform/ptrace/BUILD new file mode 100644 index 000000000..16b0b3c69 --- /dev/null +++ b/pkg/sentry/platform/ptrace/BUILD @@ -0,0 +1,31 @@ +package(licenses = ["notice"]) # Apache 2.0 + +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "ptrace", + srcs = [ + "ptrace.go", + "ptrace_unsafe.go", + "stub_amd64.s", + "stub_unsafe.go", + "subprocess.go", + "subprocess_amd64.go", + "subprocess_linux.go", + "subprocess_linux_amd64_unsafe.go", + "subprocess_unsafe.go", + ], + importpath = "gvisor.googlesource.com/gvisor/pkg/sentry/platform/ptrace", + visibility = ["//:sandbox"], + deps = [ + "//pkg/abi/linux", + "//pkg/sentry/arch", + "//pkg/sentry/platform", + "//pkg/sentry/platform/filemem", + "//pkg/sentry/platform/interrupt", + "//pkg/sentry/platform/procid", + "//pkg/sentry/platform/safecopy", + "//pkg/sentry/usermem", + "@org_golang_x_sys//unix:go_default_library", + ], +) diff --git a/pkg/sentry/platform/ptrace/ptrace.go b/pkg/sentry/platform/ptrace/ptrace.go new file mode 100644 index 000000000..05f8b1d05 --- /dev/null +++ b/pkg/sentry/platform/ptrace/ptrace.go @@ -0,0 +1,242 @@ +// 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 ptrace provides a ptrace-based implementation of the platform +// interface. This is useful for development and testing purposes primarily, +// and runs on stock kernels without special permissions. +// +// In a nutshell, it works as follows: +// +// The creation of a new address space creates a new child processes with a +// single thread which is traced by a single goroutine. +// +// A context is just a collection of temporary variables. Calling Switch on a +// context does the following: +// +// Locks the runtime thread. +// +// Looks up a traced subprocess thread for the current runtime thread. If +// none exists, the dedicated goroutine is asked to create a new stopped +// thread in the subprocess. This stopped subprocess thread is then traced +// by the current thread and this information is stored for subsequent +// switches. +// +// The context is then bound with information about the subprocess thread +// so that the context may be appropriately interrupted via a signal. +// +// The requested operation is performed in the traced subprocess thread +// (e.g. set registers, execute, return). +// +// FIXME: This package is currently sloppy with cleanup. +// +// Lock order: +// +// subprocess.mu +// context.mu +package ptrace + +import ( + "sync" + + "gvisor.googlesource.com/gvisor/pkg/abi/linux" + "gvisor.googlesource.com/gvisor/pkg/sentry/arch" + "gvisor.googlesource.com/gvisor/pkg/sentry/platform" + "gvisor.googlesource.com/gvisor/pkg/sentry/platform/filemem" + "gvisor.googlesource.com/gvisor/pkg/sentry/platform/interrupt" + "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" +) + +var ( + // stubStart is the link address for our stub, and determines the + // maximum user address. This is valid only after a call to stubInit. + // + // We attempt to link the stub here, and adjust downward as needed. + stubStart uintptr = 0x7fffffff0000 + + // stubEnd is the first byte past the end of the stub, as with + // stubStart this is valid only after a call to stubInit. + stubEnd uintptr + + // stubInitialized controls one-time stub initialization. + stubInitialized sync.Once +) + +type context struct { + // signalInfo is the signal info, if and when a signal is received. + signalInfo arch.SignalInfo + + // interrupt is the interrupt context. + interrupt interrupt.Forwarder + + // mu protects the following fields. + mu sync.Mutex + + // If lastFaultSP is non-nil, the last context switch was due to a fault + // received while executing lastFaultSP. Only context.Switch may set + // lastFaultSP to a non-nil value. + lastFaultSP *subprocess + + // lastFaultAddr is the last faulting address; this is only meaningful if + // lastFaultSP is non-nil. + lastFaultAddr usermem.Addr + + // lastFaultIP is the address of the last faulting instruction; + // this is also only meaningful if lastFaultSP is non-nil. + lastFaultIP usermem.Addr +} + +// Switch runs the provided context in the given address space. +func (c *context) Switch(as platform.AddressSpace, ac arch.Context, cpu int32) (*arch.SignalInfo, usermem.AccessType, error) { + s := as.(*subprocess) + isSyscall := s.switchToApp(c, ac) + + var faultSP *subprocess + var faultAddr usermem.Addr + var faultIP usermem.Addr + if !isSyscall && linux.Signal(c.signalInfo.Signo) == linux.SIGSEGV { + faultSP = s + faultAddr = usermem.Addr(c.signalInfo.Addr()) + faultIP = usermem.Addr(ac.IP()) + } + + // Update the context to reflect the outcome of this context switch. + c.mu.Lock() + lastFaultSP := c.lastFaultSP + lastFaultAddr := c.lastFaultAddr + lastFaultIP := c.lastFaultIP + // At this point, c may not yet be in s.contexts, so c.lastFaultSP won't be + // updated by s.Unmap(). This is fine; we only need to synchronize with + // calls to s.Unmap() that occur after the handling of this fault. + c.lastFaultSP = faultSP + c.lastFaultAddr = faultAddr + c.lastFaultIP = faultIP + c.mu.Unlock() + + // Update subprocesses to reflect the outcome of this context switch. + if lastFaultSP != faultSP { + if lastFaultSP != nil { + lastFaultSP.mu.Lock() + delete(lastFaultSP.contexts, c) + lastFaultSP.mu.Unlock() + } + if faultSP != nil { + faultSP.mu.Lock() + faultSP.contexts[c] = struct{}{} + faultSP.mu.Unlock() + } + } + + if isSyscall { + return nil, usermem.NoAccess, nil + } + if faultSP == nil { + // Non-fault signal. + return &c.signalInfo, usermem.NoAccess, platform.ErrContextSignal + } + + // Got a page fault. Ideally, we'd get real fault type here, but ptrace + // doesn't expose this information. Instead, we use a simple heuristic: + // + // It was an instruction fault iff the faulting addr == instruction + // pointer. + // + // It was a write fault if the fault is immediately repeated. + at := usermem.Read + if faultAddr == faultIP { + at.Execute = true + } + if lastFaultSP == faultSP && + lastFaultAddr == faultAddr && + lastFaultIP == faultIP { + at.Write = true + } + return &c.signalInfo, at, platform.ErrContextSignal +} + +// Interrupt interrupts the running guest application associated with this context. +func (c *context) Interrupt() { + c.interrupt.NotifyInterrupt() +} + +// PTrace represents a collection of ptrace subprocesses. +type PTrace struct { + platform.MMapMinAddr + platform.NoCPUPreemptionDetection + *filemem.FileMem +} + +// New returns a new ptrace-based implementation of the platform interface. +func New() (*PTrace, error) { + stubInitialized.Do(func() { + // Initialize the stub. + stubInit() + + // Create the master process for the global pool. This must be + // done before initializing any other processes. + master, err := newSubprocess(createStub) + if err != nil { + // Should never happen. + panic("unable to initialize ptrace master: " + err.Error()) + } + + // Set the master on the globalPool. + globalPool.master = master + }) + + fm, err := filemem.New("ptrace-memory") + if err != nil { + return nil, err + } + + return &PTrace{FileMem: fm}, nil +} + +// SupportsAddressSpaceIO implements platform.Platform.SupportsAddressSpaceIO. +func (*PTrace) SupportsAddressSpaceIO() bool { + return false +} + +// CooperativelySchedulesAddressSpace implements platform.Platform.CooperativelySchedulesAddressSpace. +func (*PTrace) CooperativelySchedulesAddressSpace() bool { + return false +} + +// MapUnit implements platform.Platform.MapUnit. +func (*PTrace) MapUnit() uint64 { + // The host kernel manages page tables and arbitrary-sized mappings + // have effectively the same cost. + return 0 +} + +// MaxUserAddress returns the first address that may not be used by user +// applications. +func (*PTrace) MaxUserAddress() usermem.Addr { + return usermem.Addr(stubStart) +} + +// NewAddressSpace returns a new subprocess. +func (p *PTrace) NewAddressSpace(_ interface{}) (platform.AddressSpace, <-chan struct{}, error) { + as, err := newSubprocess(globalPool.master.createStub) + return as, nil, err +} + +// NewContext returns an interruptible context. +func (*PTrace) NewContext() platform.Context { + return &context{} +} + +// Memory returns the platform memory used to do allocations. +func (p *PTrace) Memory() platform.Memory { + return p.FileMem +} diff --git a/pkg/sentry/platform/ptrace/ptrace_unsafe.go b/pkg/sentry/platform/ptrace/ptrace_unsafe.go new file mode 100644 index 000000000..b55b2795a --- /dev/null +++ b/pkg/sentry/platform/ptrace/ptrace_unsafe.go @@ -0,0 +1,166 @@ +// 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 ptrace + +import ( + "syscall" + "unsafe" + + "gvisor.googlesource.com/gvisor/pkg/sentry/arch" + "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" +) + +// GETREGSET/SETREGSET register set types. +// +// See include/uapi/linux/elf.h. +const ( + // _NT_PRFPREG is for x86 floating-point state without using xsave. + _NT_PRFPREG = 0x2 + + // _NT_X86_XSTATE is for x86 extended state using xsave. + _NT_X86_XSTATE = 0x202 +) + +// fpRegSet returns the GETREGSET/SETREGSET register set type to be used. +func fpRegSet(useXsave bool) uintptr { + if useXsave { + return _NT_X86_XSTATE + } + return _NT_PRFPREG +} + +// getRegs sets the regular register set. +func (t *thread) getRegs(regs *syscall.PtraceRegs) error { + _, _, errno := syscall.RawSyscall6( + syscall.SYS_PTRACE, + syscall.PTRACE_GETREGS, + uintptr(t.tid), + 0, + uintptr(unsafe.Pointer(regs)), + 0, 0) + if errno != 0 { + return errno + } + return nil +} + +// setRegs sets the regular register set. +func (t *thread) setRegs(regs *syscall.PtraceRegs) error { + _, _, errno := syscall.RawSyscall6( + syscall.SYS_PTRACE, + syscall.PTRACE_SETREGS, + uintptr(t.tid), + 0, + uintptr(unsafe.Pointer(regs)), + 0, 0) + if errno != 0 { + return errno + } + return nil +} + +// getFPRegs gets the floating-point data via the GETREGSET ptrace syscall. +func (t *thread) getFPRegs(fpState *arch.FloatingPointData, fpLen uint64, useXsave bool) error { + iovec := syscall.Iovec{ + Base: (*byte)(fpState), + Len: fpLen, + } + _, _, errno := syscall.RawSyscall6( + syscall.SYS_PTRACE, + syscall.PTRACE_GETREGSET, + uintptr(t.tid), + fpRegSet(useXsave), + uintptr(unsafe.Pointer(&iovec)), + 0, 0) + if errno != 0 { + return errno + } + return nil +} + +// setFPRegs sets the floating-point data via the SETREGSET ptrace syscall. +func (t *thread) setFPRegs(fpState *arch.FloatingPointData, fpLen uint64, useXsave bool) error { + iovec := syscall.Iovec{ + Base: (*byte)(fpState), + Len: fpLen, + } + _, _, errno := syscall.RawSyscall6( + syscall.SYS_PTRACE, + syscall.PTRACE_SETREGSET, + uintptr(t.tid), + fpRegSet(useXsave), + uintptr(unsafe.Pointer(&iovec)), + 0, 0) + if errno != 0 { + return errno + } + return nil +} + +// getSignalInfo retrieves information about the signal that caused the stop. +func (t *thread) getSignalInfo(si *arch.SignalInfo) error { + _, _, errno := syscall.RawSyscall6( + syscall.SYS_PTRACE, + syscall.PTRACE_GETSIGINFO, + uintptr(t.tid), + 0, + uintptr(unsafe.Pointer(si)), + 0, 0) + if errno != 0 { + return errno + } + return nil +} + +// clone creates a new thread from this one. +// +// The returned thread will be stopped and available for any system thread to +// call attach on it. +// +// Precondition: the OS thread must be locked and own t. +func (t *thread) clone(initRegs *syscall.PtraceRegs) (*thread, error) { + r, ok := usermem.Addr(initRegs.Rsp).RoundUp() + if !ok { + return nil, syscall.EINVAL + } + rval, err := t.syscallIgnoreInterrupt( + initRegs, + syscall.SYS_CLONE, + arch.SyscallArgument{Value: uintptr( + syscall.CLONE_FILES | + syscall.CLONE_FS | + syscall.CLONE_SIGHAND | + syscall.CLONE_THREAD | + syscall.CLONE_PTRACE | + syscall.CLONE_VM)}, + // The stack pointer is just made up, but we have it be + // something sensible so the kernel doesn't think we're + // up to no good. Which we are. + arch.SyscallArgument{Value: uintptr(r)}, + arch.SyscallArgument{}, + arch.SyscallArgument{}, + // We use these registers initially, but really they + // could be anything. We're going to stop immediately. + arch.SyscallArgument{Value: uintptr(unsafe.Pointer(initRegs))}) + if err != nil { + return nil, err + } + + return &thread{ + tgid: t.tgid, + tid: int32(rval), + cpu: ^uint32(0), + }, nil +} diff --git a/pkg/sentry/platform/ptrace/stub_amd64.s b/pkg/sentry/platform/ptrace/stub_amd64.s new file mode 100644 index 000000000..9bf87b6f6 --- /dev/null +++ b/pkg/sentry/platform/ptrace/stub_amd64.s @@ -0,0 +1,114 @@ +// 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. + +#include "funcdata.h" +#include "textflag.h" + +#define SYS_GETPID 39 +#define SYS_EXIT 60 +#define SYS_KILL 62 +#define SYS_GETPPID 110 +#define SYS_PRCTL 157 + +#define SIGKILL 9 +#define SIGSTOP 19 + +#define PR_SET_PDEATHSIG 1 + +// stub bootstraps the child and sends itself SIGSTOP to wait for attach. +// +// R15 contains the expected PPID. R15 is used instead of a more typical DI +// since syscalls will clobber DI and createStub wants to pass a new PPID to +// grandchildren. +// +// This should not be used outside the context of a new ptrace child (as the +// function is otherwise a bunch of nonsense). +TEXT ·stub(SB),NOSPLIT,$0 +begin: + // N.B. This loop only executes in the context of a single-threaded + // fork child. + + MOVQ $SYS_PRCTL, AX + MOVQ $PR_SET_PDEATHSIG, DI + MOVQ $SIGKILL, SI + SYSCALL + + CMPQ AX, $0 + JNE error + + // If the parent already died before we called PR_SET_DEATHSIG then + // we'll have an unexpected PPID. + MOVQ $SYS_GETPPID, AX + SYSCALL + + CMPQ AX, $0 + JL error + + CMPQ AX, R15 + JNE parent_dead + + MOVQ $SYS_GETPID, AX + SYSCALL + + CMPQ AX, $0 + JL error + + // SIGSTOP to wait for attach. + // + // The SYSCALL instruction will be used for future syscall injection by + // thread.syscall. + MOVQ AX, DI + MOVQ $SYS_KILL, AX + MOVQ $SIGSTOP, SI + SYSCALL + + // The tracer may "detach" and/or allow code execution here in three cases: + // + // 1. New (traced) stub threads are explicitly detached by the + // goroutine in newSubprocess. However, they are detached while in + // group-stop, so they do not execute code here. + // + // 2. If a tracer thread exits, it implicitly detaches from the stub, + // potentially allowing code execution here. However, the Go runtime + // never exits individual threads, so this case never occurs. + // + // 3. subprocess.createStub clones a new stub process that is untraced, + // thus executing this code. We setup the PDEATHSIG before SIGSTOPing + // ourselves for attach by the tracer. + // + // R15 has been updated with the expected PPID. + JMP begin + +error: + // Exit with -errno. + MOVQ AX, DI + NEGQ DI + MOVQ $SYS_EXIT, AX + SYSCALL + HLT + +parent_dead: + MOVQ $SYS_EXIT, AX + MOVQ $1, DI + SYSCALL + HLT + +// stubCall calls the stub function at the given address with the given PPID. +// +// This is a distinct function because stub, above, may be mapped at any +// arbitrary location, and stub has a specific binary API (see above). +TEXT ·stubCall(SB),NOSPLIT,$0-16 + MOVQ addr+0(FP), AX + MOVQ pid+8(FP), R15 + JMP AX diff --git a/pkg/sentry/platform/ptrace/stub_unsafe.go b/pkg/sentry/platform/ptrace/stub_unsafe.go new file mode 100644 index 000000000..c868a2d68 --- /dev/null +++ b/pkg/sentry/platform/ptrace/stub_unsafe.go @@ -0,0 +1,98 @@ +// 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 ptrace + +import ( + "reflect" + "syscall" + "unsafe" + + "gvisor.googlesource.com/gvisor/pkg/sentry/platform/safecopy" + "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" +) + +// stub is defined in arch-specific assembly. +func stub() + +// stubCall calls the stub at the given address with the given pid. +func stubCall(addr, pid uintptr) + +// unsafeSlice returns a slice for the given address and length. +func unsafeSlice(addr uintptr, length int) (slice []byte) { + sh := (*reflect.SliceHeader)(unsafe.Pointer(&slice)) + sh.Data = addr + sh.Len = length + sh.Cap = length + return +} + +// stubInit initializes the stub. +func stubInit() { + // Grab the existing stub. + stubBegin := reflect.ValueOf(stub).Pointer() + stubLen := int(safecopy.FindEndAddress(stubBegin) - stubBegin) + stubSlice := unsafeSlice(stubBegin, stubLen) + mapLen := uintptr(stubLen) + if offset := mapLen % usermem.PageSize; offset != 0 { + mapLen += usermem.PageSize - offset + } + + for stubStart > 0 { + // Map the target address for the stub. + // + // We don't use FIXED here because we don't want to unmap + // something that may have been there already. We just walk + // down the address space until we find a place where the stub + // can be placed. + addr, _, errno := syscall.RawSyscall6( + syscall.SYS_MMAP, + stubStart, + mapLen, + syscall.PROT_WRITE|syscall.PROT_READ, + syscall.MAP_PRIVATE|syscall.MAP_ANONYMOUS, + 0 /* fd */, 0 /* offset */) + if addr != stubStart || errno != 0 { + if addr != 0 { + // Unmap the region we've mapped accidentally. + syscall.RawSyscall(syscall.SYS_MUNMAP, addr, mapLen, 0) + } + + // Attempt to begin at a lower address. + stubStart -= uintptr(usermem.PageSize) + continue + } + + // Copy the stub to the address. + targetSlice := unsafeSlice(addr, stubLen) + copy(targetSlice, stubSlice) + + // Make the stub executable. + if _, _, errno := syscall.RawSyscall( + syscall.SYS_MPROTECT, + stubStart, + mapLen, + syscall.PROT_EXEC|syscall.PROT_READ); errno != 0 { + panic("mprotect failed: " + errno.Error()) + } + + // Set the end. + stubEnd = stubStart + mapLen + return + } + + // This will happen only if we exhaust the entire address + // space, and it will take a long, long time. + panic("failed to map stub") +} diff --git a/pkg/sentry/platform/ptrace/subprocess.go b/pkg/sentry/platform/ptrace/subprocess.go new file mode 100644 index 000000000..0d6a38f15 --- /dev/null +++ b/pkg/sentry/platform/ptrace/subprocess.go @@ -0,0 +1,559 @@ +// 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 ptrace + +import ( + "fmt" + "os" + "runtime" + "sync" + "syscall" + + "gvisor.googlesource.com/gvisor/pkg/sentry/arch" + "gvisor.googlesource.com/gvisor/pkg/sentry/platform" + "gvisor.googlesource.com/gvisor/pkg/sentry/platform/procid" + "gvisor.googlesource.com/gvisor/pkg/sentry/usermem" +) + +// globalPool exists to solve two distinct problems: +// +// 1) Subprocesses can't always be killed properly (see Release). +// +// 2) Any seccomp filters that have been installed will apply to subprocesses +// created here. Therefore we use the intermediary (master), which is created +// on initialization of the platform. +var globalPool struct { + mu sync.Mutex + master *subprocess + available []*subprocess +} + +// thread is a traced thread; it is a thread identifier. +// +// This is a convenience type for defining ptrace operations. +type thread struct { + tgid int32 + tid int32 + cpu uint32 +} + +// threadPool is a collection of threads. +type threadPool struct { + // mu protects below. + mu sync.Mutex + + // threads is the collection of threads. + // + // This map is indexed by system TID (the calling thread); which will + // be the tracer for the given *thread, and therefore capable of using + // relevant ptrace calls. + threads map[int32]*thread +} + +// lookupOrCreate looks up a given thread or creates one. +// +// newThread will generally be subprocess.newThread. +// +// Precondition: the runtime OS thread must be locked. +func (tp *threadPool) lookupOrCreate(currentTID int32, newThread func() *thread) *thread { + tp.mu.Lock() + t, ok := tp.threads[currentTID] + if !ok { + // Before creating a new thread, see if we can find a thread + // whose system tid has disappeared. + // + // TODO: Other parts of this package depend on + // threads never exiting. + for origTID, t := range tp.threads { + // Signal zero is an easy existence check. + if err := syscall.Tgkill(syscall.Getpid(), int(origTID), 0); err != nil { + // This thread has been abandoned; reuse it. + delete(tp.threads, origTID) + tp.threads[currentTID] = t + tp.mu.Unlock() + return t + } + } + + // Create a new thread. + t = newThread() + tp.threads[currentTID] = t + } + tp.mu.Unlock() + return t +} + +// subprocess is a collection of threads being traced. +type subprocess struct { + platform.NoAddressSpaceIO + + // initRegs are the initial registers for the first thread. + // + // These are used for the register set for system calls. + initRegs syscall.PtraceRegs + + // requests is used to signal creation of new threads. + requests chan chan *thread + + // sysemuThreads are reserved for emulation. + sysemuThreads threadPool + + // syscallThreads are reserved for syscalls (except clone, which is + // handled in the dedicated goroutine corresponding to requests above). + syscallThreads threadPool + + // mu protects the following fields. + mu sync.Mutex + + // contexts is the set of contexts for which it's possible that + // context.lastFaultSP == this subprocess. + contexts map[*context]struct{} +} + +// newSubprocess returns a useable subprocess. +// +// This will either be a newly created subprocess, or one from the global pool. +// The create function will be called in the latter case, which is guaranteed +// to happen with the runtime thread locked. +func newSubprocess(create func() (*thread, error)) (*subprocess, error) { + // See Release. + globalPool.mu.Lock() + if len(globalPool.available) > 0 { + sp := globalPool.available[len(globalPool.available)-1] + globalPool.available = globalPool.available[:len(globalPool.available)-1] + globalPool.mu.Unlock() + return sp, nil + } + globalPool.mu.Unlock() + + // The following goroutine is responsible for creating the first traced + // thread, and responding to requests to make additional threads in the + // traced process. The process will be killed and reaped when the + // request channel is closed, which happens in Release below. + var initRegs syscall.PtraceRegs + errChan := make(chan error) + requests := make(chan chan *thread) + go func() { // S/R-SAFE: Platform-related. + runtime.LockOSThread() + defer runtime.UnlockOSThread() + + // Initialize the first thread. + firstThread, err := create() + if err != nil { + errChan <- err + return + } + + // Grab registers. + // + // Note that we adjust the current register RIP value to be + // just before the current system call executed. This depends + // on the definition of the stub itself. + if err := firstThread.getRegs(&initRegs); err != nil { + panic(fmt.Sprintf("ptrace get regs failed: %v", err)) + } + initRegs.Rip -= initRegsRipAdjustment + + // Ready to handle requests. + errChan <- nil + + // Wait for requests to create threads. + for r := range requests { + t, err := firstThread.clone(&initRegs) + if err != nil { + // Should not happen: not recoverable. + panic(fmt.Sprintf("error initializing first thread: %v", err)) + } + + // Since the new thread was created with + // clone(CLONE_PTRACE), it will begin execution with + // SIGSTOP pending and with this thread as its tracer. + // (Hopefully nobody tgkilled it with a signal < + // SIGSTOP before the SIGSTOP was delivered, in which + // case that signal would be delivered before SIGSTOP.) + if sig := t.wait(); sig != syscall.SIGSTOP { + panic(fmt.Sprintf("error waiting for new clone: expected SIGSTOP, got %v", sig)) + } + + // Detach the thread without suppressing the SIGSTOP, + // causing it to enter group-stop. + if _, _, errno := syscall.RawSyscall6(syscall.SYS_PTRACE, syscall.PTRACE_DETACH, uintptr(t.tid), 0, uintptr(syscall.SIGSTOP), 0, 0); errno != 0 { + panic(fmt.Sprintf("can't detach new clone: %v", errno)) + } + + // Return the thread. + r <- t + } + + // Requests should never be closed. + panic("unreachable") + }() + + // Wait until error or readiness. + if err := <-errChan; err != nil { + return nil, err + } + + // Ready. + sp := &subprocess{ + initRegs: initRegs, + requests: requests, + sysemuThreads: threadPool{ + threads: make(map[int32]*thread), + }, + syscallThreads: threadPool{ + threads: make(map[int32]*thread), + }, + contexts: make(map[*context]struct{}), + } + + sp.unmap() + return sp, nil +} + +// unmap unmaps non-stub regions of the process. +// +// This will panic on failure (which should never happen). +func (s *subprocess) unmap() { + s.Unmap(0, uint64(stubStart)) + if maximumUserAddress != stubEnd { + s.Unmap(usermem.Addr(stubEnd), uint64(maximumUserAddress-stubEnd)) + } +} + +// Release kills the subprocess. +// +// Just kidding! We can't safely co-ordinate the detaching of all the +// tracees (since the tracers are random runtime threads, and the process +// won't exit until tracers have been notifier). +// +// Therefore we simply unmap everything in the subprocess and return it to the +// globalPool. This has the added benefit of reducing creation time for new +// subprocesses. +func (s *subprocess) Release() error { + go func() { // S/R-SAFE: Platform. + s.unmap() + globalPool.mu.Lock() + globalPool.available = append(globalPool.available, s) + globalPool.mu.Unlock() + }() + return nil +} + +// newThread creates a new traced thread. +// +// Precondition: the OS thread must be locked. +func (s *subprocess) newThread() *thread { + // Ask the first thread to create a new one. + r := make(chan *thread) + s.requests <- r + t := <-r + + // Attach the subprocess to this one. + t.attach() + + // Return the new thread, which is now bound. + return t +} + +// attach attachs to the thread. +func (t *thread) attach() { + if _, _, errno := syscall.RawSyscall(syscall.SYS_PTRACE, syscall.PTRACE_ATTACH, uintptr(t.tid), 0); errno != 0 { + panic(fmt.Sprintf("unable to attach: %v", errno)) + } + + // PTRACE_ATTACH sends SIGSTOP, and wakes the tracee if it was already + // stopped from the SIGSTOP queued by CLONE_PTRACE (see inner loop of + // newSubprocess), so we always expect to see signal-delivery-stop with + // SIGSTOP. + if sig := t.wait(); sig != syscall.SIGSTOP { + panic(fmt.Sprintf("wait failed: expected SIGSTOP, got %v", sig)) + } + + // Initialize options. + t.init() +} + +// wait waits for a stop event. +func (t *thread) wait() syscall.Signal { + var status syscall.WaitStatus + + for { + r, err := syscall.Wait4(int(t.tid), &status, syscall.WALL|syscall.WUNTRACED, nil) + if err == syscall.EINTR || err == syscall.EAGAIN { + // Wait was interrupted; wait again. + continue + } else if err != nil { + panic(fmt.Sprintf("ptrace wait failed: %v", err)) + } + if int(r) != int(t.tid) { + panic(fmt.Sprintf("ptrace wait returned %v, expected %v", r, t.tid)) + } + if !status.Stopped() { + panic(fmt.Sprintf("ptrace status unexpected: got %v, wanted stopped", status)) + } + if status.StopSignal() == 0 { + continue // Spurious stop. + } + return status.StopSignal() + } +} + +// init initializes trace options. +func (t *thread) init() { + // Set our TRACESYSGOOD option to differeniate real SIGTRAP. + _, _, errno := syscall.RawSyscall6( + syscall.SYS_PTRACE, + syscall.PTRACE_SETOPTIONS, + uintptr(t.tid), + 0, + syscall.PTRACE_O_TRACESYSGOOD, + 0, 0) + if errno != 0 { + panic(fmt.Sprintf("ptrace set options failed: %v", errno)) + } +} + +// syscall executes a system call cycle in the traced context. +// +// This is _not_ for use by application system calls, rather it is for use when +// a system call must be injected into the remote context (e.g. mmap, munmap). +// Note that clones are handled separately. +func (t *thread) syscall(regs *syscall.PtraceRegs) (uintptr, error) { + // Set registers. + if err := t.setRegs(regs); err != nil { + panic(fmt.Sprintf("ptrace set regs failed: %v", err)) + } + + for { + // Execute the syscall instruction. + if _, _, errno := syscall.RawSyscall(syscall.SYS_PTRACE, syscall.PTRACE_SYSCALL, uintptr(t.tid), 0); errno != 0 { + panic(fmt.Sprintf("ptrace syscall-enter failed: %v", errno)) + } + + sig := t.wait() + if sig == (0x80 | syscall.SIGTRAP) { + // Reached syscall-enter-stop. + break + } else { + // Some other signal caused a thread stop; ignore. + continue + } + } + + // Complete the actual system call. + if _, _, errno := syscall.RawSyscall(syscall.SYS_PTRACE, syscall.PTRACE_SYSCALL, uintptr(t.tid), 0); errno != 0 { + panic(fmt.Sprintf("ptrace syscall-enter failed: %v", errno)) + } + + // Wait for syscall-exit-stop. "[Signal-delivery-stop] never happens + // between syscall-enter-stop and syscall-exit-stop; it happens *after* + // syscall-exit-stop.)" - ptrace(2), "Syscall-stops" + if sig := t.wait(); sig != (0x80 | syscall.SIGTRAP) { + panic(fmt.Sprintf("wait failed: expected SIGTRAP, got %v [%d]", sig, sig)) + } + + // Grab registers. + if err := t.getRegs(regs); err != nil { + panic(fmt.Sprintf("ptrace get regs failed: %v", err)) + } + + return syscallReturnValue(regs) +} + +// syscallIgnoreInterrupt ignores interrupts on the system call thread and +// restarts the syscall if the kernel indicates that should happen. +func (t *thread) syscallIgnoreInterrupt( + initRegs *syscall.PtraceRegs, + sysno uintptr, + args ...arch.SyscallArgument) (uintptr, error) { + for { + regs := createSyscallRegs(initRegs, sysno, args...) + rval, err := t.syscall(®s) + switch err { + case ERESTARTSYS: + continue + case ERESTARTNOINTR: + continue + case ERESTARTNOHAND: + continue + default: + return rval, err + } + } +} + +// NotifyInterrupt implements interrupt.Receiver.NotifyInterrupt. +func (t *thread) NotifyInterrupt() { + syscall.Tgkill(int(t.tgid), int(t.tid), syscall.Signal(platform.SignalInterrupt)) +} + +// switchToApp is called from the main SwitchToApp entrypoint. +// +// This function returns true on a system call, false on a signal. +func (s *subprocess) switchToApp(c *context, ac arch.Context) bool { + regs := &ac.StateData().Regs + s.resetSysemuRegs(regs) + + // Extract floating point state. + fpState := ac.FloatingPointData() + fpLen, _ := ac.FeatureSet().ExtendedStateSize() + useXsave := ac.FeatureSet().UseXsave() + + // Lock the thread for ptrace operations. + runtime.LockOSThread() + defer runtime.UnlockOSThread() + + // Grab our thread from the pool. + currentTID := int32(procid.Current()) + t := s.sysemuThreads.lookupOrCreate(currentTID, s.newThread) + + // Check for interrupts, and ensure that future interrupts will signal t. + if !c.interrupt.Enable(t) { + // Pending interrupt; simulate. + c.signalInfo = arch.SignalInfo{Signo: int32(platform.SignalInterrupt)} + return false + } + defer c.interrupt.Disable() + + // Ensure that the CPU set is bound appropriately; this makes the + // emulation below several times faster, presumably by avoiding + // interprocessor wakeups and by simplifying the schedule. + t.bind() + + // Set registers. + if err := t.setRegs(regs); err != nil { + panic(fmt.Sprintf("ptrace set regs failed: %v", err)) + } + if err := t.setFPRegs(fpState, uint64(fpLen), useXsave); err != nil { + panic(fmt.Sprintf("ptrace set fpregs failed: %v", err)) + } + + for { + // Start running until the next system call. + if isSingleStepping(regs) { + if _, _, errno := syscall.RawSyscall( + syscall.SYS_PTRACE, + syscall.PTRACE_SYSEMU_SINGLESTEP, + uintptr(t.tid), 0); errno != 0 { + panic(fmt.Sprintf("ptrace sysemu failed: %v", errno)) + } + } else { + if _, _, errno := syscall.RawSyscall( + syscall.SYS_PTRACE, + syscall.PTRACE_SYSEMU, + uintptr(t.tid), 0); errno != 0 { + panic(fmt.Sprintf("ptrace sysemu failed: %v", errno)) + } + } + + // Wait for the syscall-enter stop. + sig := t.wait() + + // Refresh all registers. + if err := t.getRegs(regs); err != nil { + panic(fmt.Sprintf("ptrace get regs failed: %v", err)) + } + if err := t.getFPRegs(fpState, uint64(fpLen), useXsave); err != nil { + panic(fmt.Sprintf("ptrace get fpregs failed: %v", err)) + } + + // Is it a system call? + if sig == (0x80 | syscall.SIGTRAP) { + // Ensure registers are sane. + updateSyscallRegs(regs) + return true + } + + if sig == syscall.SIGSTOP { + // SIGSTOP was delivered to another thread in the same thread + // group, which initiated another group stop. Just ignore it. + continue + } + + // Grab signal information. + if err := t.getSignalInfo(&c.signalInfo); err != nil { + // Should never happen. + panic(fmt.Sprintf("ptrace get signal info failed: %v", err)) + } + + // We have a signal. We verify however, that the signal was + // either delivered from the kernel or from this process. We + // don't respect other signals. + if c.signalInfo.Code > 0 { + return false // kernel. + } else if c.signalInfo.Code <= 0 && c.signalInfo.Pid() == int32(os.Getpid()) { + return false // this process. + } + } +} + +// syscall executes the given system call without handling interruptions. +func (s *subprocess) syscall(sysno uintptr, args ...arch.SyscallArgument) (uintptr, error) { + // Grab a thread. + runtime.LockOSThread() + defer runtime.UnlockOSThread() + currentTID := int32(procid.Current()) + t := s.syscallThreads.lookupOrCreate(currentTID, s.newThread) + + return t.syscallIgnoreInterrupt(&s.initRegs, sysno, args...) +} + +// MapFile implements platform.AddressSpace.MapFile. +func (s *subprocess) MapFile(addr usermem.Addr, fd int, fr platform.FileRange, at usermem.AccessType, precommit bool) error { + var flags int + if precommit { + flags |= syscall.MAP_POPULATE + } + _, err := s.syscall( + syscall.SYS_MMAP, + arch.SyscallArgument{Value: uintptr(addr)}, + arch.SyscallArgument{Value: uintptr(fr.Length())}, + arch.SyscallArgument{Value: uintptr(at.Prot())}, + arch.SyscallArgument{Value: uintptr(flags | syscall.MAP_SHARED | syscall.MAP_FIXED)}, + arch.SyscallArgument{Value: uintptr(fd)}, + arch.SyscallArgument{Value: uintptr(fr.Start)}) + return err +} + +// Unmap implements platform.AddressSpace.Unmap. +func (s *subprocess) Unmap(addr usermem.Addr, length uint64) { + ar, ok := addr.ToRange(length) + if !ok { + panic(fmt.Sprintf("addr %#x + length %#x overflows", addr, length)) + } + s.mu.Lock() + for c := range s.contexts { + c.mu.Lock() + if c.lastFaultSP == s && ar.Contains(c.lastFaultAddr) { + // Forget the last fault so that if c faults again, the fault isn't + // incorrectly reported as a write fault. If this is being called + // due to munmap() of the corresponding vma, handling of the second + // fault will fail anyway. + c.lastFaultSP = nil + delete(s.contexts, c) + } + c.mu.Unlock() + } + s.mu.Unlock() + _, err := s.syscall( + syscall.SYS_MUNMAP, + arch.SyscallArgument{Value: uintptr(addr)}, + arch.SyscallArgument{Value: uintptr(length)}) + if err != nil { + // We never expect this to happen. + panic(fmt.Sprintf("munmap(%x, %x)) failed: %v", addr, length, err)) + } +} diff --git a/pkg/sentry/platform/ptrace/subprocess_amd64.go b/pkg/sentry/platform/ptrace/subprocess_amd64.go new file mode 100644 index 000000000..8211215df --- /dev/null +++ b/pkg/sentry/platform/ptrace/subprocess_amd64.go @@ -0,0 +1,104 @@ +// 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. + +// +build amd64 + +package ptrace + +import ( + "syscall" + + "gvisor.googlesource.com/gvisor/pkg/sentry/arch" +) + +const ( + // maximumUserAddress is the largest possible user address. + maximumUserAddress = 0x7ffffffff000 + + // initRegsRipAdjustment is the size of the syscall instruction. + initRegsRipAdjustment = 2 +) + +// Linux kernel errnos which "should never be seen by user programs", but will +// be revealed to ptrace syscall exit tracing. +// +// These constants are used in subprocess.go. +const ( + ERESTARTSYS = syscall.Errno(512) + ERESTARTNOINTR = syscall.Errno(513) + ERESTARTNOHAND = syscall.Errno(514) +) + +// resetSysemuRegs sets up emulation registers. +// +// This should be called prior to calling sysemu. +func (s *subprocess) resetSysemuRegs(regs *syscall.PtraceRegs) { + regs.Cs = s.initRegs.Cs + regs.Ss = s.initRegs.Ss + regs.Ds = s.initRegs.Ds + regs.Es = s.initRegs.Es + regs.Fs = s.initRegs.Fs + regs.Gs = s.initRegs.Gs +} + +// createSyscallRegs sets up syscall registers. +// +// This should be called to generate registers for a system call. +func createSyscallRegs(initRegs *syscall.PtraceRegs, sysno uintptr, args ...arch.SyscallArgument) syscall.PtraceRegs { + // Copy initial registers (RIP, segments, etc.). + regs := *initRegs + + // Set our syscall number. + regs.Rax = uint64(sysno) + if len(args) >= 1 { + regs.Rdi = args[0].Uint64() + } + if len(args) >= 2 { + regs.Rsi = args[1].Uint64() + } + if len(args) >= 3 { + regs.Rdx = args[2].Uint64() + } + if len(args) >= 4 { + regs.R10 = args[3].Uint64() + } + if len(args) >= 5 { + regs.R8 = args[4].Uint64() + } + if len(args) >= 6 { + regs.R9 = args[5].Uint64() + } + + return regs +} + +// isSingleStepping determines if the registers indicate single-stepping. +func isSingleStepping(regs *syscall.PtraceRegs) bool { + return (regs.Eflags & arch.X86TrapFlag) != 0 +} + +// updateSyscallRegs updates registers after finishing sysemu. +func updateSyscallRegs(regs *syscall.PtraceRegs) { + // Ptrace puts -ENOSYS in rax on syscall-enter-stop. + regs.Rax = regs.Orig_rax +} + +// syscallReturnValue extracts a sensible return from registers. +func syscallReturnValue(regs *syscall.PtraceRegs) (uintptr, error) { + rval := int64(regs.Rax) + if rval < 0 { + return 0, syscall.Errno(-rval) + } + return uintptr(rval), nil +} diff --git a/pkg/sentry/platform/ptrace/subprocess_linux.go b/pkg/sentry/platform/ptrace/subprocess_linux.go new file mode 100644 index 000000000..227dd4882 --- /dev/null +++ b/pkg/sentry/platform/ptrace/subprocess_linux.go @@ -0,0 +1,146 @@ +// 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. + +// +build linux + +package ptrace + +import ( + "fmt" + "syscall" + + "gvisor.googlesource.com/gvisor/pkg/sentry/arch" + "gvisor.googlesource.com/gvisor/pkg/sentry/platform/procid" +) + +// createStub creates a fresh stub processes. +// +// Precondition: the runtime OS thread must be locked. +func createStub() (*thread, error) { + // Declare all variables up front in order to ensure that there's no + // need for allocations between beforeFork & afterFork. + var ( + pid uintptr + ppid uintptr + errno syscall.Errno + ) + + // Remember the current ppid for the pdeathsig race. + ppid, _, _ = syscall.RawSyscall(syscall.SYS_GETPID, 0, 0, 0) + + // Among other things, beforeFork masks all signals. + beforeFork() + pid, _, errno = syscall.RawSyscall6(syscall.SYS_CLONE, uintptr(syscall.SIGCHLD)|syscall.CLONE_FILES, 0, 0, 0, 0, 0) + if errno != 0 { + afterFork() + return nil, errno + } + + // Is this the parent? + if pid != 0 { + // Among other things, restore signal mask. + afterFork() + + // Initialize the first thread. + t := &thread{ + tgid: int32(pid), + tid: int32(pid), + cpu: ^uint32(0), + } + if sig := t.wait(); sig != syscall.SIGSTOP { + return nil, fmt.Errorf("wait failed: expected SIGSTOP, got %v", sig) + } + t.attach() + + return t, nil + } + + // afterForkInChild resets all signals to their default dispositions + // and restores the signal mask to its pre-fork state. + afterForkInChild() + + // Explicitly unmask all signals to ensure that the tracer can see + // them. + errno = unmaskAllSignals() + if errno != 0 { + syscall.RawSyscall(syscall.SYS_EXIT, uintptr(errno), 0, 0) + } + + // Call the stub; should not return. + stubCall(stubStart, ppid) + panic("unreachable") +} + +// createStub creates a stub processes as a child of an existing subprocesses. +// +// Precondition: the runtime OS thread must be locked. +func (s *subprocess) createStub() (*thread, error) { + // There's no need to lock the runtime thread here, as this can only be + // called from a context that is already locked. + currentTID := int32(procid.Current()) + t := s.syscallThreads.lookupOrCreate(currentTID, s.newThread) + + // Pass the expected PPID to the child via R15. + regs := s.initRegs + regs.R15 = uint64(t.tgid) + + // Call fork in a subprocess. + // + // The new child must set up PDEATHSIG to ensure it dies if this + // process dies. Since this process could die at any time, this cannot + // be done via instrumentation from here. + // + // Instead, we create the child untraced, which will do the PDEATHSIG + // setup and then SIGSTOP itself for our attach below. + pid, err := t.syscallIgnoreInterrupt( + ®s, + syscall.SYS_CLONE, + arch.SyscallArgument{Value: uintptr(syscall.SIGCHLD | syscall.CLONE_FILES)}, + arch.SyscallArgument{Value: 0}, + arch.SyscallArgument{Value: 0}, + arch.SyscallArgument{Value: 0}, + arch.SyscallArgument{Value: 0}, + arch.SyscallArgument{Value: 0}) + if err != nil { + return nil, err + } + + // Wait for child to enter group-stop, so we don't stop its + // bootstrapping work with t.attach below. + // + // We unfortunately don't have a handy part of memory to write the wait + // status. If the wait succeeds, we'll assume that it was the SIGSTOP. + // If the child actually exited, the attach below will fail. + _, err = t.syscallIgnoreInterrupt( + &s.initRegs, + syscall.SYS_WAIT4, + arch.SyscallArgument{Value: uintptr(pid)}, + arch.SyscallArgument{Value: 0}, + arch.SyscallArgument{Value: syscall.WUNTRACED}, + arch.SyscallArgument{Value: 0}, + arch.SyscallArgument{Value: 0}, + arch.SyscallArgument{Value: 0}) + if err != nil { + return nil, err + } + + childT := &thread{ + tgid: int32(pid), + tid: int32(pid), + cpu: ^uint32(0), + } + childT.attach() + + return childT, nil +} diff --git a/pkg/sentry/platform/ptrace/subprocess_linux_amd64_unsafe.go b/pkg/sentry/platform/ptrace/subprocess_linux_amd64_unsafe.go new file mode 100644 index 000000000..697431472 --- /dev/null +++ b/pkg/sentry/platform/ptrace/subprocess_linux_amd64_unsafe.go @@ -0,0 +1,109 @@ +// 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. + +// +build amd64 linux + +package ptrace + +import ( + "sync" + "sync/atomic" + "syscall" + "unsafe" + + "golang.org/x/sys/unix" + "gvisor.googlesource.com/gvisor/pkg/abi/linux" +) + +// maskPool contains reusable CPU masks for setting affinity. Unfortunately, +// runtime.NumCPU doesn't actually record the number of CPUs on the system, it +// just records the number of CPUs available in the scheduler affinity set at +// startup. This may a) change over time and b) gives a number far lower than +// the maximum indexable CPU. To prevent lots of allocation in the hot path, we +// use a pool to store large masks that we can reuse during bind. +var maskPool = sync.Pool{ + New: func() interface{} { + const maxCPUs = 1024 // Not a hard limit; see below. + return make([]uintptr, maxCPUs/64) + }, +} + +// unmaskAllSignals unmasks all signals on the current thread. +// +//go:nosplit +func unmaskAllSignals() syscall.Errno { + var set linux.SignalSet + _, _, errno := syscall.RawSyscall6(syscall.SYS_RT_SIGPROCMASK, linux.SIG_SETMASK, uintptr(unsafe.Pointer(&set)), 0, linux.SignalSetSize, 0, 0) + return errno +} + +// getCPU gets the current CPU. +// +// Precondition: the current runtime thread should be locked. +func getCPU() (uint32, error) { + var cpu uintptr + if _, _, errno := syscall.RawSyscall( + unix.SYS_GETCPU, + uintptr(unsafe.Pointer(&cpu)), + 0, 0); errno != 0 { + return 0, errno + } + return uint32(cpu), nil +} + +// setCPU sets the CPU affinity. +func (t *thread) setCPU(cpu uint32) error { + mask := maskPool.Get().([]uintptr) + n := int(cpu / 64) + v := uintptr(1 << uintptr(cpu%64)) + if n >= len(mask) { + // See maskPool note above. We've actually exceeded the number + // of available cores. Grow the mask and return it. + mask = make([]uintptr, n+1) + } + mask[n] |= v + if _, _, errno := syscall.RawSyscall( + unix.SYS_SCHED_SETAFFINITY, + uintptr(t.tid), + uintptr(len(mask)*8), + uintptr(unsafe.Pointer(&mask[0]))); errno != 0 { + return errno + } + mask[n] &^= v + maskPool.Put(mask) + return nil +} + +// bind attempts to ensure that the thread is on the same CPU as the current +// thread. This provides no guarantees as it is fundamentally a racy operation: +// CPU sets may change and we may be rescheduled in the middle of this +// operation. As a result, no failures are reported. +// +// Precondition: the current runtime thread should be locked. +func (t *thread) bind() { + currentCPU, err := getCPU() + if err != nil { + return + } + if oldCPU := atomic.SwapUint32(&t.cpu, currentCPU); oldCPU != currentCPU { + // Set the affinity on the thread and save the CPU for next + // round; we don't expect CPUs to bounce around too frequently. + // + // (It's worth noting that we could move CPUs between this point + // and when the tracee finishes executing. But that would be + // roughly the status quo anyways -- we're just maximizing our + // chances of colocation, not guaranteeing it.) + t.setCPU(currentCPU) + } +} diff --git a/pkg/sentry/platform/ptrace/subprocess_unsafe.go b/pkg/sentry/platform/ptrace/subprocess_unsafe.go new file mode 100644 index 000000000..fe41641d3 --- /dev/null +++ b/pkg/sentry/platform/ptrace/subprocess_unsafe.go @@ -0,0 +1,28 @@ +// 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 ptrace + +import ( + _ "unsafe" // required for go:linkname. +) + +//go:linkname beforeFork syscall.runtime_BeforeFork +func beforeFork() + +//go:linkname afterFork syscall.runtime_AfterFork +func afterFork() + +//go:linkname afterForkInChild syscall.runtime_AfterForkInChild +func afterForkInChild() |