diff options
author | Dean Deng <deandeng@google.com> | 2020-08-25 16:26:53 -0700 |
---|---|---|
committer | gVisor bot <gvisor-bot@google.com> | 2020-08-25 16:28:45 -0700 |
commit | cb573c8e0bf042ea111dc83c72f25d135623032f (patch) | |
tree | d0d1a4f4febe8ec3ca2c707a2a9a25d95cbc13b3 | |
parent | 70a7a3ac704a47ec50525d06438ba4983da3af8b (diff) |
Expose basic coverage information to userspace through kcov interface.
In Linux, a kernel configuration is set that compiles the kernel with a
custom function that is called at the beginning of every basic block, which
updates the memory-mapped coverage information. The Go coverage tool does not
allow us to inject arbitrary instructions into basic blocks, but it does
provide data that we can convert to a kcov-like format and transfer them to
userspace through a memory mapping.
Note that this is not a strict implementation of kcov, which is especially
tricky to do because we do not have the same coverage tools available in Go
that that are available for the actual Linux kernel. In Linux, a kernel
configuration is set that compiles the kernel with a custom function that is
called at the beginning of every basic block to write program counters to the
kcov memory mapping. In Go, however, coverage tools only give us a count of
basic blocks as they are executed. Every time we return to userspace, we
collect the coverage information and write out PCs for each block that was
executed, providing userspace with the illusion that the kcov data is always
up to date. For convenience, we also generate a unique synthetic PC for each
block instead of using actual PCs. Finally, we do not provide thread-specific
coverage data (each kcov instance only contains PCs executed by the thread
owning it); instead, we will supply data for any file specified by --
instrumentation_filter.
Also, fix issue in nogo that was causing pkg/coverage:coverage_nogo
compilation to fail.
PiperOrigin-RevId: 328426526
-rw-r--r-- | pkg/abi/linux/ioctl.go | 21 | ||||
-rw-r--r-- | pkg/coverage/BUILD | 14 | ||||
-rw-r--r-- | pkg/coverage/coverage.go | 175 | ||||
-rw-r--r-- | pkg/sentry/fsimpl/sys/BUILD | 5 | ||||
-rw-r--r-- | pkg/sentry/fsimpl/sys/kcov.go | 116 | ||||
-rw-r--r-- | pkg/sentry/fsimpl/sys/sys.go | 18 | ||||
-rw-r--r-- | pkg/sentry/kernel/BUILD | 3 | ||||
-rw-r--r-- | pkg/sentry/kernel/kcov.go | 321 | ||||
-rw-r--r-- | pkg/sentry/kernel/kcov_unsafe.go | 28 | ||||
-rw-r--r-- | pkg/sentry/kernel/kernel.go | 2 | ||||
-rw-r--r-- | pkg/sentry/kernel/task.go | 18 | ||||
-rw-r--r-- | pkg/sentry/kernel/task_exit.go | 2 | ||||
-rw-r--r-- | runsc/config/config.go | 6 | ||||
-rw-r--r-- | test/syscalls/BUILD | 4 | ||||
-rw-r--r-- | test/syscalls/linux/BUILD | 14 | ||||
-rw-r--r-- | test/syscalls/linux/kcov.cc | 70 |
16 files changed, 813 insertions, 4 deletions
diff --git a/pkg/abi/linux/ioctl.go b/pkg/abi/linux/ioctl.go index 2c5e56ae5..d6dbedc3e 100644 --- a/pkg/abi/linux/ioctl.go +++ b/pkg/abi/linux/ioctl.go @@ -117,3 +117,24 @@ const ( func IOC(dir, typ, nr, size uint32) uint32 { return uint32(dir)<<_IOC_DIRSHIFT | typ<<_IOC_TYPESHIFT | nr<<_IOC_NRSHIFT | size<<_IOC_SIZESHIFT } + +// Kcov ioctls from kernel/kcov.h. +var ( + KCOV_INIT_TRACE = IOC(_IOC_READ, 'c', 1, 8) + KCOV_ENABLE = IOC(_IOC_NONE, 'c', 100, 0) + KCOV_DISABLE = IOC(_IOC_NONE, 'c', 101, 0) +) + +// Kcov trace types from kernel/kcov.h. +const ( + KCOV_TRACE_PC = 0 + KCOV_TRACE_CMP = 1 +) + +// Kcov state constants from kernel/kcov.h. +const ( + KCOV_MODE_DISABLED = 0 + KCOV_MODE_INIT = 1 + KCOV_MODE_TRACE_PC = 2 + KCOV_MODE_TRACE_CMP = 3 +) diff --git a/pkg/coverage/BUILD b/pkg/coverage/BUILD new file mode 100644 index 000000000..a198e8028 --- /dev/null +++ b/pkg/coverage/BUILD @@ -0,0 +1,14 @@ +load("//tools:defs.bzl", "go_library") + +package(licenses = ["notice"]) + +go_library( + name = "coverage", + srcs = ["coverage.go"], + visibility = ["//:sandbox"], + deps = [ + "//pkg/sync", + "//pkg/usermem", + "@io_bazel_rules_go//go/tools/coverdata", + ], +) diff --git a/pkg/coverage/coverage.go b/pkg/coverage/coverage.go new file mode 100644 index 000000000..6831adcce --- /dev/null +++ b/pkg/coverage/coverage.go @@ -0,0 +1,175 @@ +// Copyright 2020 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 coverage provides an interface through which Go coverage data can +// be collected, converted to kcov format, and exposed to userspace. +// +// Coverage can be enabled by calling bazel {build,test} with +// --collect_coverage_data and --instrumentation_filter with the desired +// coverage surface. This causes bazel to use the Go cover tool manually to +// generate instrumented files. It injects a hook that registers all coverage +// data with the coverdata package. +package coverage + +import ( + "fmt" + "io" + "sort" + "sync/atomic" + + "gvisor.dev/gvisor/pkg/sync" + "gvisor.dev/gvisor/pkg/usermem" + + "github.com/bazelbuild/rules_go/go/tools/coverdata" +) + +// KcovAvailable returns whether the kcov coverage interface is available. It is +// available as long as coverage is enabled for some files. +func KcovAvailable() bool { + return len(coverdata.Cover.Blocks) > 0 +} + +// coverageMu must be held while accessing coverdata.Cover. This prevents +// concurrent reads/writes from multiple threads collecting coverage data. +var coverageMu sync.RWMutex + +// once ensures that globalData is only initialized once. +var once sync.Once + +var globalData struct { + // files is the set of covered files sorted by filename. It is calculated at + // startup. + files []string + + // syntheticPCs are a set of PCs calculated at startup, where the PC + // at syntheticPCs[i][j] corresponds to file i, block j. + syntheticPCs [][]uint64 +} + +// ClearCoverageData clears existing coverage data. +func ClearCoverageData() { + coverageMu.Lock() + defer coverageMu.Unlock() + for _, counters := range coverdata.Cover.Counters { + for index := 0; index < len(counters); index++ { + atomic.StoreUint32(&counters[index], 0) + } + } +} + +var coveragePool = sync.Pool{ + New: func() interface{} { + return make([]byte, 0) + }, +} + +// ConsumeCoverageData builds and writes the collection of covered PCs. It +// returns the number of bytes written. +// +// In Linux, a kernel configuration is set that compiles the kernel with a +// custom function that is called at the beginning of every basic block, which +// updates the memory-mapped coverage information. The Go coverage tool does not +// allow us to inject arbitrary instructions into basic blocks, but it does +// provide data that we can convert to a kcov-like format and transfer them to +// userspace through a memory mapping. +// +// Note that this is not a strict implementation of kcov, which is especially +// tricky to do because we do not have the same coverage tools available in Go +// that that are available for the actual Linux kernel. In Linux, a kernel +// configuration is set that compiles the kernel with a custom function that is +// called at the beginning of every basic block to write program counters to the +// kcov memory mapping. In Go, however, coverage tools only give us a count of +// basic blocks as they are executed. Every time we return to userspace, we +// collect the coverage information and write out PCs for each block that was +// executed, providing userspace with the illusion that the kcov data is always +// up to date. For convenience, we also generate a unique synthetic PC for each +// block instead of using actual PCs. Finally, we do not provide thread-specific +// coverage data (each kcov instance only contains PCs executed by the thread +// owning it); instead, we will supply data for any file specified by -- +// instrumentation_filter. +// +// Note that we "consume", i.e. clear, coverdata when this function is run, to +// ensure that each event is only reported once. +// +// TODO(b/160639712): evaluate whether it is ok to reset the global coverage +// data every time this function is run. We could technically have each thread +// store a local snapshot against which we compare the most recent coverdata so +// that separate threads do not affect each other's view of the data. +func ConsumeCoverageData(w io.Writer) int { + once.Do(initCoverageData) + + coverageMu.Lock() + defer coverageMu.Unlock() + + total := 0 + var pcBuffer [8]byte + for fileIndex, file := range globalData.files { + counters := coverdata.Cover.Counters[file] + for index := 0; index < len(counters); index++ { + val := atomic.SwapUint32(&counters[index], 0) + if val != 0 { + // Calculate the synthetic PC. + pc := globalData.syntheticPCs[fileIndex][index] + + usermem.ByteOrder.PutUint64(pcBuffer[:], pc) + n, err := w.Write(pcBuffer[:]) + if err != nil { + if err == io.EOF { + // Simply stop writing if we encounter EOF; it's ok if we attempted to + // write more than we can hold. + return total + n + } + panic(fmt.Sprintf("Internal error writing PCs to kcov area: %v", err)) + } + total += n + } + } + } + + if total == 0 { + // An empty profile indicates that coverage is not enabled, in which case + // there shouldn't be any task work registered. + panic("kcov task work is registered, but no coverage data was found") + } + return total +} + +// initCoverageData initializes globalData. It should only be called once, +// before any kcov data is written. +func initCoverageData() { + // First, order all files. Then calculate synthetic PCs for every block + // (using the well-defined ordering for files as well). + for file := range coverdata.Cover.Blocks { + globalData.files = append(globalData.files, file) + } + sort.Strings(globalData.files) + + // nextSyntheticPC is the first PC that we generate for a block. + // + // This uses a standard-looking kernel range for simplicity. + // + // FIXME(b/160639712): This is only necessary because syzkaller requires + // addresses in the kernel range. If we can remove this constraint, then we + // should be able to use the actual addresses. + var nextSyntheticPC uint64 = 0xffffffff80000000 + for _, file := range globalData.files { + blocks := coverdata.Cover.Blocks[file] + thisFile := make([]uint64, 0, len(blocks)) + for range blocks { + thisFile = append(thisFile, nextSyntheticPC) + nextSyntheticPC++ // Advance. + } + globalData.syntheticPCs = append(globalData.syntheticPCs, thisFile) + } +} diff --git a/pkg/sentry/fsimpl/sys/BUILD b/pkg/sentry/fsimpl/sys/BUILD index 1b548ccd4..f9b232da6 100644 --- a/pkg/sentry/fsimpl/sys/BUILD +++ b/pkg/sentry/fsimpl/sys/BUILD @@ -5,17 +5,22 @@ licenses(["notice"]) go_library( name = "sys", srcs = [ + "kcov.go", "sys.go", ], visibility = ["//pkg/sentry:internal"], deps = [ "//pkg/abi/linux", "//pkg/context", + "//pkg/coverage", + "//pkg/sentry/arch", "//pkg/sentry/fsimpl/kernfs", "//pkg/sentry/kernel", "//pkg/sentry/kernel/auth", + "//pkg/sentry/memmap", "//pkg/sentry/vfs", "//pkg/syserror", + "//pkg/usermem", ], ) diff --git a/pkg/sentry/fsimpl/sys/kcov.go b/pkg/sentry/fsimpl/sys/kcov.go new file mode 100644 index 000000000..92710d877 --- /dev/null +++ b/pkg/sentry/fsimpl/sys/kcov.go @@ -0,0 +1,116 @@ +// Copyright 2020 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 sys + +import ( + "gvisor.dev/gvisor/pkg/abi/linux" + "gvisor.dev/gvisor/pkg/context" + "gvisor.dev/gvisor/pkg/sentry/arch" + "gvisor.dev/gvisor/pkg/sentry/fsimpl/kernfs" + "gvisor.dev/gvisor/pkg/sentry/kernel" + "gvisor.dev/gvisor/pkg/sentry/kernel/auth" + "gvisor.dev/gvisor/pkg/sentry/memmap" + "gvisor.dev/gvisor/pkg/sentry/vfs" + "gvisor.dev/gvisor/pkg/syserror" + "gvisor.dev/gvisor/pkg/usermem" +) + +func (fs *filesystem) newKcovFile(ctx context.Context, creds *auth.Credentials) *kernfs.Dentry { + k := &kcovInode{} + k.InodeAttrs.Init(creds, 0, 0, fs.NextIno(), linux.S_IFREG|0600) + d := &kernfs.Dentry{} + d.Init(k) + return d +} + +// kcovInode implements kernfs.Inode. +type kcovInode struct { + kernfs.InodeAttrs + kernfs.InodeNoopRefCount + kernfs.InodeNotSymlink + kernfs.InodeNotDirectory +} + +func (i *kcovInode) Open(ctx context.Context, rp *vfs.ResolvingPath, vfsd *vfs.Dentry, opts vfs.OpenOptions) (*vfs.FileDescription, error) { + k := kernel.KernelFromContext(ctx) + if k == nil { + panic("KernelFromContext returned nil") + } + fd := &kcovFD{ + inode: i, + kcov: k.NewKcov(), + } + + if err := fd.vfsfd.Init(fd, opts.Flags, rp.Mount(), vfsd, &vfs.FileDescriptionOptions{ + DenyPRead: true, + DenyPWrite: true, + }); err != nil { + return nil, err + } + return &fd.vfsfd, nil +} + +type kcovFD struct { + vfs.FileDescriptionDefaultImpl + vfs.NoLockFD + + vfsfd vfs.FileDescription + inode *kcovInode + kcov *kernel.Kcov +} + +// Ioctl implements vfs.FileDescriptionImpl.Ioctl. +func (fd *kcovFD) Ioctl(ctx context.Context, uio usermem.IO, args arch.SyscallArguments) (uintptr, error) { + cmd := uint32(args[1].Int()) + arg := args[2].Uint64() + switch uint32(cmd) { + case linux.KCOV_INIT_TRACE: + return 0, fd.kcov.InitTrace(arg) + case linux.KCOV_ENABLE: + return 0, fd.kcov.EnableTrace(ctx, uint8(arg)) + case linux.KCOV_DISABLE: + if arg != 0 { + // This arg is unused; it should be 0. + return 0, syserror.EINVAL + } + return 0, fd.kcov.DisableTrace(ctx) + default: + return 0, syserror.ENOTTY + } +} + +// ConfigureMmap implements vfs.FileDescriptionImpl.ConfigureMmap. +func (fd *kcovFD) ConfigureMMap(ctx context.Context, opts *memmap.MMapOpts) error { + return fd.kcov.ConfigureMMap(ctx, opts) +} + +// Release implements vfs.FileDescriptionImpl.Release. +func (fd *kcovFD) Release(ctx context.Context) { + // kcov instances have reference counts in Linux, but this seems sufficient + // for our purposes. + fd.kcov.Reset() +} + +// SetStat implements vfs.FileDescriptionImpl.SetStat. +func (fd *kcovFD) SetStat(ctx context.Context, opts vfs.SetStatOptions) error { + creds := auth.CredentialsFromContext(ctx) + fs := fd.vfsfd.VirtualDentry().Mount().Filesystem() + return fd.inode.SetStat(ctx, fs, creds, opts) +} + +// Stat implements vfs.FileDescriptionImpl.Stat. +func (fd *kcovFD) Stat(ctx context.Context, opts vfs.StatOptions) (linux.Statx, error) { + return fd.inode.Stat(ctx, fd.vfsfd.Mount().Filesystem(), opts) +} diff --git a/pkg/sentry/fsimpl/sys/sys.go b/pkg/sentry/fsimpl/sys/sys.go index 393feb802..1f042d9f7 100644 --- a/pkg/sentry/fsimpl/sys/sys.go +++ b/pkg/sentry/fsimpl/sys/sys.go @@ -21,6 +21,7 @@ import ( "gvisor.dev/gvisor/pkg/abi/linux" "gvisor.dev/gvisor/pkg/context" + "gvisor.dev/gvisor/pkg/coverage" "gvisor.dev/gvisor/pkg/sentry/fsimpl/kernfs" "gvisor.dev/gvisor/pkg/sentry/kernel" "gvisor.dev/gvisor/pkg/sentry/kernel/auth" @@ -73,7 +74,7 @@ func (fsType FilesystemType) GetFilesystem(ctx context.Context, vfsObj *vfs.Virt }), "firmware": fs.newDir(creds, defaultSysDirMode, nil), "fs": fs.newDir(creds, defaultSysDirMode, nil), - "kernel": fs.newDir(creds, defaultSysDirMode, nil), + "kernel": kernelDir(ctx, fs, creds), "module": fs.newDir(creds, defaultSysDirMode, nil), "power": fs.newDir(creds, defaultSysDirMode, nil), }) @@ -94,6 +95,21 @@ func cpuDir(ctx context.Context, fs *filesystem, creds *auth.Credentials) *kernf return fs.newDir(creds, defaultSysDirMode, children) } +func kernelDir(ctx context.Context, fs *filesystem, creds *auth.Credentials) *kernfs.Dentry { + // If kcov is available, set up /sys/kernel/debug/kcov. Technically, debugfs + // should be mounted at debug/, but for our purposes, it is sufficient to + // keep it in sys. + var children map[string]*kernfs.Dentry + if coverage.KcovAvailable() { + children = map[string]*kernfs.Dentry{ + "debug": fs.newDir(creds, linux.FileMode(0700), map[string]*kernfs.Dentry{ + "kcov": fs.newKcovFile(ctx, creds), + }), + } + } + return fs.newDir(creds, defaultSysDirMode, children) +} + // Release implements vfs.FilesystemImpl.Release. func (fs *filesystem) Release(ctx context.Context) { fs.Filesystem.VFSFilesystem().VirtualFilesystem().PutAnonBlockDevMinor(fs.devMinor) diff --git a/pkg/sentry/kernel/BUILD b/pkg/sentry/kernel/BUILD index 5416a310d..d1ecceba3 100644 --- a/pkg/sentry/kernel/BUILD +++ b/pkg/sentry/kernel/BUILD @@ -91,6 +91,8 @@ go_library( "fd_table_unsafe.go", "fs_context.go", "ipc_namespace.go", + "kcov.go", + "kcov_unsafe.go", "kernel.go", "kernel_opts.go", "kernel_state.go", @@ -157,6 +159,7 @@ go_library( "//pkg/bits", "//pkg/bpf", "//pkg/context", + "//pkg/coverage", "//pkg/cpuid", "//pkg/eventchannel", "//pkg/fspath", diff --git a/pkg/sentry/kernel/kcov.go b/pkg/sentry/kernel/kcov.go new file mode 100644 index 000000000..aad63aa99 --- /dev/null +++ b/pkg/sentry/kernel/kcov.go @@ -0,0 +1,321 @@ +// Copyright 2020 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 kernel + +import ( + "fmt" + "io" + "sync" + + "gvisor.dev/gvisor/pkg/abi/linux" + "gvisor.dev/gvisor/pkg/context" + "gvisor.dev/gvisor/pkg/coverage" + "gvisor.dev/gvisor/pkg/safemem" + "gvisor.dev/gvisor/pkg/sentry/memmap" + "gvisor.dev/gvisor/pkg/sentry/mm" + "gvisor.dev/gvisor/pkg/sentry/pgalloc" + "gvisor.dev/gvisor/pkg/sentry/usage" + "gvisor.dev/gvisor/pkg/syserror" + "gvisor.dev/gvisor/pkg/usermem" +) + +// kcovAreaSizeMax is the maximum number of uint64 entries allowed in the kcov +// area. On Linux, the maximum is INT_MAX / 8. +const kcovAreaSizeMax = 10 * 1024 * 1024 + +// Kcov provides kernel coverage data to userspace through a memory-mapped +// region, as kcov does in Linux. +// +// To give the illusion that the data is always up to date, we update the shared +// memory every time before we return to userspace. +type Kcov struct { + // mfp provides application memory. It is immutable after creation. + mfp pgalloc.MemoryFileProvider + + // mu protects all of the fields below. + mu sync.RWMutex + + // mode is the current kcov mode. + mode uint8 + + // size is the size of the mapping through which the kernel conveys coverage + // information to userspace. + size uint64 + + // owningTask is the task that currently owns coverage data on the system. The + // interface for kcov essentially requires that coverage is only going to a + // single task. Note that kcov should only generate coverage data for the + // owning task, but we currently generate global coverage. + owningTask *Task + + // count is a locally cached version of the first uint64 in the kcov data, + // which is the number of subsequent entries representing PCs. + // + // It is used with kcovInode.countBlock(), to copy in/out the first element of + // the actual data in an efficient manner, avoid boilerplate, and prevent + // accidental garbage escapes by the temporary counts. + count uint64 + + mappable *mm.SpecialMappable +} + +// NewKcov creates and returns a Kcov instance. +func (k *Kernel) NewKcov() *Kcov { + return &Kcov{ + mfp: k, + } +} + +var coveragePool = sync.Pool{ + New: func() interface{} { + return make([]byte, 0) + }, +} + +// TaskWork implements TaskWorker.TaskWork. +func (kcov *Kcov) TaskWork(t *Task) { + kcov.mu.Lock() + defer kcov.mu.Unlock() + + rw := &kcovReadWriter{ + mf: kcov.mfp.MemoryFile(), + fr: kcov.mappable.FileRange(), + } + + // Read in the PC count. + if _, err := safemem.ReadFullToBlocks(rw, kcov.countBlock()); err != nil { + panic(fmt.Sprintf("Internal error reading count from kcov area: %v", err)) + } + + rw.off = 8 * (1 + kcov.count) + n := coverage.ConsumeCoverageData(&kcovIOWriter{rw}) + + // Update the pc count, based on the number of entries written. Note that if + // we reached the end of the kcov area, we may not have written everything in + // output. + kcov.count += uint64(n / 8) + rw.off = 0 + if _, err := safemem.WriteFullFromBlocks(rw, kcov.countBlock()); err != nil { + panic(fmt.Sprintf("Internal error writing count to kcov area: %v", err)) + } + + // Re-register for future work. + t.RegisterWork(kcov) +} + +// InitTrace performs the KCOV_INIT_TRACE ioctl. +func (kcov *Kcov) InitTrace(size uint64) error { + kcov.mu.Lock() + defer kcov.mu.Unlock() + + if kcov.mode != linux.KCOV_MODE_DISABLED { + return syserror.EBUSY + } + + // To simplify all the logic around mapping, we require that the length of the + // shared region is a multiple of the system page size. + if (8*size)&(usermem.PageSize-1) != 0 { + return syserror.EINVAL + } + + // We need space for at least two uint64s to hold current position and a + // single PC. + if size < 2 || size > kcovAreaSizeMax { + return syserror.EINVAL + } + + kcov.size = size + kcov.mode = linux.KCOV_MODE_INIT + return nil +} + +// EnableTrace performs the KCOV_ENABLE_TRACE ioctl. +func (kcov *Kcov) EnableTrace(ctx context.Context, traceMode uint8) error { + t := TaskFromContext(ctx) + if t == nil { + panic("kcovInode.EnableTrace() cannot be used outside of a task goroutine") + } + + kcov.mu.Lock() + defer kcov.mu.Unlock() + + // KCOV_ENABLE must be preceded by KCOV_INIT_TRACE and an mmap call. + if kcov.mode != linux.KCOV_MODE_INIT || kcov.mappable == nil { + return syserror.EINVAL + } + + switch traceMode { + case linux.KCOV_TRACE_PC: + kcov.mode = traceMode + case linux.KCOV_TRACE_CMP: + // We do not support KCOV_MODE_TRACE_CMP. + return syserror.ENOTSUP + default: + return syserror.EINVAL + } + + if kcov.owningTask != nil && kcov.owningTask != t { + return syserror.EBUSY + } + + kcov.owningTask = t + t.RegisterWork(kcov) + + // Clear existing coverage data; the task expects to read only coverage data + // from the time it is activated. + coverage.ClearCoverageData() + return nil +} + +// DisableTrace performs the KCOV_DISABLE_TRACE ioctl. +func (kcov *Kcov) DisableTrace(ctx context.Context) error { + kcov.mu.Lock() + defer kcov.mu.Unlock() + + t := TaskFromContext(ctx) + if t == nil { + panic("kcovInode.EnableTrace() cannot be used outside of a task goroutine") + } + + if t != kcov.owningTask { + return syserror.EINVAL + } + kcov.owningTask = nil + kcov.mode = linux.KCOV_MODE_INIT + kcov.resetLocked() + return nil +} + +// Reset is called when the owning task exits. +func (kcov *Kcov) Reset() { + kcov.mu.Lock() + kcov.resetLocked() + kcov.mu.Unlock() +} + +// The kcov instance is reset when the owning task exits or when tracing is +// disabled. +func (kcov *Kcov) resetLocked() { + kcov.owningTask = nil + if kcov.mappable != nil { + kcov.mappable = nil + } +} + +// ConfigureMMap is called by the vfs.FileDescription for this kcov instance to +// implement vfs.FileDescription.ConfigureMMap. +func (kcov *Kcov) ConfigureMMap(ctx context.Context, opts *memmap.MMapOpts) error { + kcov.mu.Lock() + defer kcov.mu.Unlock() + + if kcov.mode != linux.KCOV_MODE_INIT { + return syserror.EINVAL + } + + if kcov.mappable == nil { + // Set up the kcov area. + fr, err := kcov.mfp.MemoryFile().Allocate(kcov.size*8, usage.Anonymous) + if err != nil { + return err + } + + // Get the thread id for the mmap name. + t := TaskFromContext(ctx) + if t == nil { + panic("ThreadFromContext returned nil") + } + // For convenience, a special mappable is used here. Note that these mappings + // will look different under /proc/[pid]/maps than they do on Linux. + kcov.mappable = mm.NewSpecialMappable(fmt.Sprintf("[kcov:%d]", t.ThreadID()), kcov.mfp, fr) + } + opts.Mappable = kcov.mappable + opts.MappingIdentity = kcov.mappable + return nil +} + +// kcovReadWriter implements safemem.Reader and safemem.Writer. +type kcovReadWriter struct { + off uint64 + mf *pgalloc.MemoryFile + fr memmap.FileRange +} + +// ReadToBlocks implements safemem.Reader.ReadToBlocks. +func (rw *kcovReadWriter) ReadToBlocks(dsts safemem.BlockSeq) (uint64, error) { + if dsts.IsEmpty() { + return 0, nil + } + + // Limit the read to the kcov range and check for overflow. + if rw.fr.Length() <= rw.off { + return 0, io.EOF + } + start := rw.fr.Start + rw.off + end := rw.fr.Start + rw.fr.Length() + if rend := start + dsts.NumBytes(); rend < end { + end = rend + } + + // Get internal mappings. + bs, err := rw.mf.MapInternal(memmap.FileRange{start, end}, usermem.Read) + if err != nil { + return 0, err + } + + // Copy from internal mappings. + n, err := safemem.CopySeq(dsts, bs) + rw.off += n + return n, err +} + +// WriteFromBlocks implements safemem.Writer.WriteFromBlocks. +func (rw *kcovReadWriter) WriteFromBlocks(srcs safemem.BlockSeq) (uint64, error) { + if srcs.IsEmpty() { + return 0, nil + } + + // Limit the write to the kcov area and check for overflow. + if rw.fr.Length() <= rw.off { + return 0, io.EOF + } + start := rw.fr.Start + rw.off + end := rw.fr.Start + rw.fr.Length() + if wend := start + srcs.NumBytes(); wend < end { + end = wend + } + + // Get internal mapping. + bs, err := rw.mf.MapInternal(memmap.FileRange{start, end}, usermem.Write) + if err != nil { + return 0, err + } + + // Copy to internal mapping. + n, err := safemem.CopySeq(bs, srcs) + rw.off += n + return n, err +} + +// kcovIOWriter implements io.Writer as a basic wrapper over kcovReadWriter. +type kcovIOWriter struct { + rw *kcovReadWriter +} + +// Write implements io.Writer.Write. +func (w *kcovIOWriter) Write(p []byte) (int, error) { + bs := safemem.BlockSeqOf(safemem.BlockFromSafeSlice(p)) + n, err := safemem.WriteFullFromBlocks(w.rw, bs) + return int(n), err +} diff --git a/pkg/sentry/kernel/kcov_unsafe.go b/pkg/sentry/kernel/kcov_unsafe.go new file mode 100644 index 000000000..6f64022eb --- /dev/null +++ b/pkg/sentry/kernel/kcov_unsafe.go @@ -0,0 +1,28 @@ +// Copyright 2020 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 kernel + +import ( + "unsafe" + + "gvisor.dev/gvisor/pkg/safemem" +) + +// countBlock provides a safemem.BlockSeq for k.count. +// +// Like k.count, the block returned is protected by k.mu. +func (k *Kcov) countBlock() safemem.BlockSeq { + return safemem.BlockSeqOf(safemem.BlockFromSafePointer(unsafe.Pointer(&k.count), int(unsafe.Sizeof(k.count)))) +} diff --git a/pkg/sentry/kernel/kernel.go b/pkg/sentry/kernel/kernel.go index 2e0175e36..402aa1718 100644 --- a/pkg/sentry/kernel/kernel.go +++ b/pkg/sentry/kernel/kernel.go @@ -248,7 +248,7 @@ type Kernel struct { // SpecialOpts contains special kernel options. SpecialOpts - // VFS keeps the filesystem state used across the kernel. + // vfs keeps the filesystem state used across the kernel. vfs vfs.VirtualFilesystem // hostMount is the Mount used for file descriptors that were imported diff --git a/pkg/sentry/kernel/task.go b/pkg/sentry/kernel/task.go index 5aee699e7..a436610c9 100644 --- a/pkg/sentry/kernel/task.go +++ b/pkg/sentry/kernel/task.go @@ -574,6 +574,11 @@ type Task struct { // // startTime is protected by mu. startTime ktime.Time + + // kcov is the kcov instance providing code coverage owned by this task. + // + // kcov is exclusive to the task goroutine. + kcov *Kcov } func (t *Task) savePtraceTracer() *Task { @@ -903,3 +908,16 @@ func (t *Task) UID() uint32 { func (t *Task) GID() uint32 { return uint32(t.Credentials().EffectiveKGID) } + +// SetKcov sets the kcov instance associated with t. +func (t *Task) SetKcov(k *Kcov) { + t.kcov = k +} + +// ResetKcov clears the kcov instance associated with t. +func (t *Task) ResetKcov() { + if t.kcov != nil { + t.kcov.Reset() + t.kcov = nil + } +} diff --git a/pkg/sentry/kernel/task_exit.go b/pkg/sentry/kernel/task_exit.go index c165d6cb1..b76f7f503 100644 --- a/pkg/sentry/kernel/task_exit.go +++ b/pkg/sentry/kernel/task_exit.go @@ -239,6 +239,8 @@ func (*runExitMain) execute(t *Task) taskRunState { t.traceExitEvent() lastExiter := t.exitThreadGroup() + t.ResetKcov() + // If the task has a cleartid, and the thread group wasn't killed by a // signal, handle that before releasing the MM. if t.cleartid != 0 { diff --git a/runsc/config/config.go b/runsc/config/config.go index ca85cef51..8cf0378d5 100644 --- a/runsc/config/config.go +++ b/runsc/config/config.go @@ -300,10 +300,10 @@ type Config struct { // E.g. 0.2 CPU quota will result in 1, and 1.9 in 2. CPUNumFromQuota bool - // Enables VFS2 (not plumbled through yet). + // Enables VFS2 (not plumbed through yet). VFS2 bool - // Enables FUSE usage (not plumbled through yet). + // Enables FUSE usage (not plumbed through yet). FUSE bool // TestOnlyAllowRunAsCurrentUserWithoutChroot should only be used in @@ -353,6 +353,8 @@ func (c *Config) ToFlags() []string { "--tx-checksum-offload=" + strconv.FormatBool(c.TXChecksumOffload), "--overlayfs-stale-read=" + strconv.FormatBool(c.OverlayfsStaleRead), "--qdisc=" + c.QDisc.String(), + "--vfs2=" + strconv.FormatBool(c.VFS2), + "--fuse=" + strconv.FormatBool(c.FUSE), } if c.CPUNumFromQuota { f = append(f, "--cpu-num-from-quota") diff --git a/test/syscalls/BUILD b/test/syscalls/BUILD index eea1401ac..65e8299c3 100644 --- a/test/syscalls/BUILD +++ b/test/syscalls/BUILD @@ -263,6 +263,10 @@ syscall_test( ) syscall_test( + test = "//test/syscalls/linux:kcov_test", +) + +syscall_test( test = "//test/syscalls/linux:kill_test", ) diff --git a/test/syscalls/linux/BUILD b/test/syscalls/linux/BUILD index ed0b6ecf4..5a323d331 100644 --- a/test/syscalls/linux/BUILD +++ b/test/syscalls/linux/BUILD @@ -1068,6 +1068,20 @@ cc_binary( ) cc_binary( + name = "kcov_test", + testonly = 1, + srcs = ["kcov.cc"], + linkstatic = 1, + deps = [ + "//test/util:capability_util", + "//test/util:file_descriptor", + gtest, + "//test/util:test_main", + "//test/util:test_util", + ], +) + +cc_binary( name = "kill_test", testonly = 1, srcs = ["kill.cc"], diff --git a/test/syscalls/linux/kcov.cc b/test/syscalls/linux/kcov.cc new file mode 100644 index 000000000..f3c30444e --- /dev/null +++ b/test/syscalls/linux/kcov.cc @@ -0,0 +1,70 @@ +// Copyright 2018 The gVisor Authors. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include <sys/errno.h> +#include <sys/ioctl.h> +#include <sys/mman.h> + +#include "gtest/gtest.h" +#include "test/util/capability_util.h" +#include "test/util/file_descriptor.h" +#include "test/util/test_util.h" + +namespace gvisor { +namespace testing { + +namespace { + +// For this test to work properly, it must be run with coverage enabled. On +// native Linux, this involves compiling the kernel with kcov enabled. For +// gVisor, we need to enable the Go coverage tool, e.g. +// bazel test --collect_coverage_data --instrumentation_filter=//pkg/... <test>. +TEST(KcovTest, Kcov) { + SKIP_IF(!ASSERT_NO_ERRNO_AND_VALUE(HaveCapability((CAP_DAC_OVERRIDE)))); + + constexpr int kSize = 4096; + constexpr int KCOV_INIT_TRACE = 0x80086301; + constexpr int KCOV_ENABLE = 0x6364; + + int fd; + ASSERT_THAT(fd = open("/sys/kernel/debug/kcov", O_RDWR), + AnyOf(SyscallSucceeds(), SyscallFailsWithErrno(ENOENT))); + + // Kcov not enabled. + SKIP_IF(errno == ENOENT); + + ASSERT_THAT(ioctl(fd, KCOV_INIT_TRACE, kSize), SyscallSucceeds()); + uint64_t* area = (uint64_t*)mmap(nullptr, kSize * sizeof(uint64_t), + PROT_READ | PROT_WRITE, MAP_SHARED, fd, 0); + + ASSERT_TRUE(area != MAP_FAILED); + ASSERT_THAT(ioctl(fd, KCOV_ENABLE, 0), SyscallSucceeds()); + + for (int i = 0; i < 10; i++) { + // Make some syscalls to generate coverage data. + ASSERT_THAT(ioctl(fd, KCOV_ENABLE, 0), SyscallFailsWithErrno(EINVAL)); + } + + uint64_t num_pcs = *(uint64_t*)(area); + EXPECT_GT(num_pcs, 0); + for (uint64_t i = 1; i <= num_pcs; i++) { + // Verify that PCs are in the standard kernel range. + EXPECT_GT(area[i], 0xffffffff7fffffffL); + } +} + +} // namespace + +} // namespace testing +} // namespace gvisor |