Rewrite reference leak checker without finalizers.

Our current reference leak checker uses finalizers to verify whether an object
has reached zero references before it is garbage collected. There are multiple
problems with this mechanism, so a rewrite is in order.

With finalizers, there is no way to guarantee that a finalizer will run before
the program exits. When an unreachable object with a finalizer is garbage
collected, its finalizer will be added to a queue and run asynchronously. The
best we can do is run garbage collection upon sandbox exit to make sure that
all finalizers are enqueued.

Furthermore, if there is a chain of finalized objects, e.g. A points to B
points to C, garbage collection needs to run multiple times before all of the
finalizers are enqueued. The first GC run will register the finalizer for A but
not free it. It takes another GC run to free A, at which point B's finalizer
can be registered. As a result, we need to run GC as many times as the length
of the longest such chain to have a somewhat reliable leak checker.

Finally, a cyclical chain of structs pointing to one another will never be
garbage collected if a finalizer is set. This is a well-known issue with Go
finalizers (https://github.com/golang/go/issues/7358). Using leak checking on
filesystem objects that produce cycles will not work and even result in memory
leaks.

The new leak checker stores reference counted objects in a global map when
leak check is enabled and removes them once they are destroyed. At sandbox
exit, any remaining objects in the map are considered as leaked. This provides
a deterministic way of detecting leaks without relying on the complexities of
finalizers and garbage collection.

This approach has several benefits over the former, including:
- Always detects leaks of objects that should be destroyed very close to
  sandbox exit. The old checker very rarely detected these leaks, because it
  relied on garbage collection to be run in a short window of time.
- Panics if we forgot to enable leak check on a ref-counted object (we will try
  to remove it from the map when it is destroyed, but it will never have been
  added).
- Can store extra logging information in the map values without adding to the
  size of the ref count struct itself. With the size of just an int64, the ref
  count object remains compact, meaning frequent operations like IncRef/DecRef
  are more cache-efficient.
- Can aggregate leak results in a single report after the sandbox exits.
  Instead of having warnings littered in the log, which were
  non-deterministically triggered by garbage collection, we can print all
  warning messages at once. Note that this could also be a limitation--the
  sandbox must exit properly for leaks to be detected.

Some basic benchmarking indicates that this change does not significantly
affect performance when leak checking is enabled, which is understandable
since registering/unregistering is only done once for each filesystem object.

Updates #1486.

PiperOrigin-RevId: 338685972
This commit is contained in:
Dean Deng 2020-10-23 09:14:52 -07:00 committed by gVisor bot
parent 293877cf64
commit 9ca66ec598
31 changed files with 214 additions and 78 deletions

View File

@ -19,8 +19,16 @@ go_template(
)
go_library(
name = "refs_vfs2",
srcs = ["refs.go"],
visibility = ["//pkg/sentry:internal"],
deps = ["//pkg/context"],
name = "refsvfs2",
srcs = [
"refs.go",
"refs_map.go",
],
visibility = ["//:sandbox"],
deps = [
"//pkg/context",
"//pkg/log",
"//pkg/refs",
"//pkg/sync",
],
)

View File

@ -12,8 +12,8 @@
// See the License for the specific language governing permissions and
// limitations under the License.
// Package refs_vfs2 defines an interface for a reference-counted object.
package refs_vfs2
// Package refsvfs2 defines an interface for a reference-counted object.
package refsvfs2
import (
"gvisor.dev/gvisor/pkg/context"

97
pkg/refsvfs2/refs_map.go Normal file
View File

@ -0,0 +1,97 @@
// 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 refsvfs2
import (
"fmt"
"strings"
"gvisor.dev/gvisor/pkg/log"
refs_vfs1 "gvisor.dev/gvisor/pkg/refs"
"gvisor.dev/gvisor/pkg/sync"
)
// TODO(gvisor.dev/issue/1193): re-enable once kernfs refs are fixed.
var ignored []string = []string{"kernfs.", "proc.", "sys.", "devpts.", "fuse."}
var (
// liveObjects is a global map of reference-counted objects. Objects are
// inserted when leak check is enabled, and they are removed when they are
// destroyed. It is protected by liveObjectsMu.
liveObjects map[CheckedObject]struct{}
liveObjectsMu sync.Mutex
)
// CheckedObject represents a reference-counted object with an informative
// leak detection message.
type CheckedObject interface {
// LeakMessage supplies a warning to be printed upon leak detection.
LeakMessage() string
}
func init() {
liveObjects = make(map[CheckedObject]struct{})
}
// LeakCheckEnabled returns whether leak checking is enabled. The following
// functions should only be called if it returns true.
func LeakCheckEnabled() bool {
return refs_vfs1.GetLeakMode() != refs_vfs1.NoLeakChecking
}
// Register adds obj to the live object map.
func Register(obj CheckedObject, typ string) {
for _, str := range ignored {
if strings.Contains(typ, str) {
return
}
}
liveObjectsMu.Lock()
if _, ok := liveObjects[obj]; ok {
panic(fmt.Sprintf("Unexpected entry in leak checking map: reference %p already added", obj))
}
liveObjects[obj] = struct{}{}
liveObjectsMu.Unlock()
}
// Unregister removes obj from the live object map.
func Unregister(obj CheckedObject, typ string) {
liveObjectsMu.Lock()
defer liveObjectsMu.Unlock()
if _, ok := liveObjects[obj]; !ok {
for _, str := range ignored {
if strings.Contains(typ, str) {
return
}
}
panic(fmt.Sprintf("Expected to find entry in leak checking map for reference %p", obj))
}
delete(liveObjects, obj)
}
// DoLeakCheck iterates through the live object map and logs a message for each
// object. It is called once no reference-counted objects should be reachable
// anymore, at which point anything left in the map is considered a leak.
func DoLeakCheck() {
liveObjectsMu.Lock()
defer liveObjectsMu.Unlock()
leaked := len(liveObjects)
if leaked > 0 {
log.Warningf("Leak checking detected %d leaked objects:", leaked)
for obj := range liveObjects {
log.Warningf(obj.LeakMessage())
}
}
}

View File

@ -21,11 +21,9 @@ package refs_template
import (
"fmt"
"runtime"
"sync/atomic"
"gvisor.dev/gvisor/pkg/log"
refs_vfs1 "gvisor.dev/gvisor/pkg/refs"
"gvisor.dev/gvisor/pkg/refsvfs2"
)
// T is the type of the reference counted object. It is only used to customize
@ -42,11 +40,6 @@ var ownerType *T
// Note that the number of references is actually refCount + 1 so that a default
// zero-value Refs object contains one reference.
//
// TODO(gvisor.dev/issue/1486): Store stack traces when leak check is enabled in
// a map with 16-bit hashes, and store the hash in the top 16 bits of refCount.
// This will allow us to add stack trace information to the leak messages
// without growing the size of Refs.
//
// +stateify savable
type Refs struct {
// refCount is composed of two fields:
@ -59,24 +52,16 @@ type Refs struct {
refCount int64
}
func (r *Refs) finalize() {
var note string
switch refs_vfs1.GetLeakMode() {
case refs_vfs1.NoLeakChecking:
return
case refs_vfs1.UninitializedLeakChecking:
note = "(Leak checker uninitialized): "
}
if n := r.ReadRefs(); n != 0 {
log.Warningf("%sRefs %p owned by %T garbage collected with ref count of %d (want 0)", note, r, ownerType, n)
// EnableLeakCheck enables reference leak checking on r.
func (r *Refs) EnableLeakCheck() {
if refsvfs2.LeakCheckEnabled() {
refsvfs2.Register(r, fmt.Sprintf("%T", ownerType))
}
}
// EnableLeakCheck checks for reference leaks when Refs gets garbage collected.
func (r *Refs) EnableLeakCheck() {
if refs_vfs1.GetLeakMode() != refs_vfs1.NoLeakChecking {
runtime.SetFinalizer(r, (*Refs).finalize)
}
// LeakMessage implements refsvfs2.CheckedObject.LeakMessage.
func (r *Refs) LeakMessage() string {
return fmt.Sprintf("%T %p: reference count of %d instead of 0", ownerType, r, r.ReadRefs())
}
// ReadRefs returns the current number of references. The returned count is
@ -91,7 +76,7 @@ func (r *Refs) ReadRefs() int64 {
//go:nosplit
func (r *Refs) IncRef() {
if v := atomic.AddInt64(&r.refCount, 1); v <= 0 {
panic(fmt.Sprintf("Incrementing non-positive ref count %p owned by %T", r, ownerType))
panic(fmt.Sprintf("Incrementing non-positive count %p on %T", r, ownerType))
}
}
@ -134,9 +119,18 @@ func (r *Refs) DecRef(destroy func()) {
panic(fmt.Sprintf("Decrementing non-positive ref count %p, owned by %T", r, ownerType))
case v == -1:
if refsvfs2.LeakCheckEnabled() {
refsvfs2.Unregister(r, fmt.Sprintf("%T", ownerType))
}
// Call the destructor.
if destroy != nil {
destroy()
}
}
}
func (r *Refs) afterLoad() {
if refsvfs2.LeakCheckEnabled() && r.ReadRefs() > 0 {
r.EnableLeakCheck()
}
}

View File

@ -8,7 +8,7 @@ go_template_instance(
out = "root_inode_refs.go",
package = "devpts",
prefix = "rootInode",
template = "//pkg/refs_vfs2:refs_template",
template = "//pkg/refsvfs2:refs_template",
types = {
"T": "rootInode",
},
@ -33,6 +33,7 @@ go_library(
"//pkg/marshal",
"//pkg/marshal/primitive",
"//pkg/refs",
"//pkg/refsvfs2",
"//pkg/safemem",
"//pkg/sentry/arch",
"//pkg/sentry/fs",

View File

@ -20,7 +20,7 @@ go_template_instance(
out = "inode_refs.go",
package = "fuse",
prefix = "inode",
template = "//pkg/refs_vfs2:refs_template",
template = "//pkg/refsvfs2:refs_template",
types = {
"T": "inode",
},
@ -49,6 +49,7 @@ go_library(
"//pkg/log",
"//pkg/marshal",
"//pkg/refs",
"//pkg/refsvfs2",
"//pkg/safemem",
"//pkg/sentry/fsimpl/devtmpfs",
"//pkg/sentry/fsimpl/kernfs",

View File

@ -8,7 +8,7 @@ go_template_instance(
out = "inode_refs.go",
package = "host",
prefix = "inode",
template = "//pkg/refs_vfs2:refs_template",
template = "//pkg/refsvfs2:refs_template",
types = {
"T": "inode",
},
@ -19,7 +19,7 @@ go_template_instance(
out = "connected_endpoint_refs.go",
package = "host",
prefix = "ConnectedEndpoint",
template = "//pkg/refs_vfs2:refs_template",
template = "//pkg/refsvfs2:refs_template",
types = {
"T": "ConnectedEndpoint",
},
@ -51,6 +51,7 @@ go_library(
"//pkg/log",
"//pkg/marshal/primitive",
"//pkg/refs",
"//pkg/refsvfs2",
"//pkg/safemem",
"//pkg/sentry/arch",
"//pkg/sentry/fs/fsutil",

View File

@ -31,7 +31,7 @@ go_template_instance(
out = "dentry_refs.go",
package = "kernfs",
prefix = "Dentry",
template = "//pkg/refs_vfs2:refs_template",
template = "//pkg/refsvfs2:refs_template",
types = {
"T": "Dentry",
},
@ -42,7 +42,7 @@ go_template_instance(
out = "static_directory_refs.go",
package = "kernfs",
prefix = "StaticDirectory",
template = "//pkg/refs_vfs2:refs_template",
template = "//pkg/refsvfs2:refs_template",
types = {
"T": "StaticDirectory",
},
@ -53,7 +53,7 @@ go_template_instance(
out = "dir_refs.go",
package = "kernfs_test",
prefix = "dir",
template = "//pkg/refs_vfs2:refs_template",
template = "//pkg/refsvfs2:refs_template",
types = {
"T": "dir",
},
@ -64,7 +64,7 @@ go_template_instance(
out = "readonly_dir_refs.go",
package = "kernfs_test",
prefix = "readonlyDir",
template = "//pkg/refs_vfs2:refs_template",
template = "//pkg/refsvfs2:refs_template",
types = {
"T": "readonlyDir",
},
@ -75,7 +75,7 @@ go_template_instance(
out = "synthetic_directory_refs.go",
package = "kernfs",
prefix = "syntheticDirectory",
template = "//pkg/refs_vfs2:refs_template",
template = "//pkg/refsvfs2:refs_template",
types = {
"T": "syntheticDirectory",
},
@ -104,6 +104,7 @@ go_library(
"//pkg/fspath",
"//pkg/log",
"//pkg/refs",
"//pkg/refsvfs2",
"//pkg/sentry/fs/lock",
"//pkg/sentry/kernel/auth",
"//pkg/sentry/memmap",
@ -129,6 +130,7 @@ go_test(
"//pkg/context",
"//pkg/log",
"//pkg/refs",
"//pkg/refsvfs2",
"//pkg/sentry/contexttest",
"//pkg/sentry/fsimpl/testutil",
"//pkg/sentry/kernel/auth",

View File

@ -8,7 +8,7 @@ go_template_instance(
out = "fd_dir_inode_refs.go",
package = "proc",
prefix = "fdDirInode",
template = "//pkg/refs_vfs2:refs_template",
template = "//pkg/refsvfs2:refs_template",
types = {
"T": "fdDirInode",
},
@ -19,7 +19,7 @@ go_template_instance(
out = "fd_info_dir_inode_refs.go",
package = "proc",
prefix = "fdInfoDirInode",
template = "//pkg/refs_vfs2:refs_template",
template = "//pkg/refsvfs2:refs_template",
types = {
"T": "fdInfoDirInode",
},
@ -30,7 +30,7 @@ go_template_instance(
out = "subtasks_inode_refs.go",
package = "proc",
prefix = "subtasksInode",
template = "//pkg/refs_vfs2:refs_template",
template = "//pkg/refsvfs2:refs_template",
types = {
"T": "subtasksInode",
},
@ -41,7 +41,7 @@ go_template_instance(
out = "task_inode_refs.go",
package = "proc",
prefix = "taskInode",
template = "//pkg/refs_vfs2:refs_template",
template = "//pkg/refsvfs2:refs_template",
types = {
"T": "taskInode",
},
@ -52,7 +52,7 @@ go_template_instance(
out = "tasks_inode_refs.go",
package = "proc",
prefix = "tasksInode",
template = "//pkg/refs_vfs2:refs_template",
template = "//pkg/refsvfs2:refs_template",
types = {
"T": "tasksInode",
},
@ -82,6 +82,7 @@ go_library(
"//pkg/context",
"//pkg/log",
"//pkg/refs",
"//pkg/refsvfs2",
"//pkg/safemem",
"//pkg/sentry/fs/lock",
"//pkg/sentry/fsbridge",

View File

@ -8,7 +8,7 @@ go_template_instance(
out = "dir_refs.go",
package = "sys",
prefix = "dir",
template = "//pkg/refs_vfs2:refs_template",
template = "//pkg/refsvfs2:refs_template",
types = {
"T": "dir",
},
@ -28,6 +28,7 @@ go_library(
"//pkg/coverage",
"//pkg/log",
"//pkg/refs",
"//pkg/refsvfs2",
"//pkg/sentry/arch",
"//pkg/sentry/fsimpl/kernfs",
"//pkg/sentry/kernel",

View File

@ -31,7 +31,7 @@ go_template_instance(
out = "inode_refs.go",
package = "tmpfs",
prefix = "inode",
template = "//pkg/refs_vfs2:refs_template",
template = "//pkg/refsvfs2:refs_template",
types = {
"T": "inode",
},
@ -60,6 +60,7 @@ go_library(
"//pkg/fspath",
"//pkg/log",
"//pkg/refs",
"//pkg/refsvfs2",
"//pkg/safemem",
"//pkg/sentry/arch",
"//pkg/sentry/fs",

View File

@ -79,7 +79,7 @@ go_template_instance(
out = "fd_table_refs.go",
package = "kernel",
prefix = "FDTable",
template = "//pkg/refs_vfs2:refs_template",
template = "//pkg/refsvfs2:refs_template",
types = {
"T": "FDTable",
},
@ -90,7 +90,7 @@ go_template_instance(
out = "fs_context_refs.go",
package = "kernel",
prefix = "FSContext",
template = "//pkg/refs_vfs2:refs_template",
template = "//pkg/refsvfs2:refs_template",
types = {
"T": "FSContext",
},
@ -101,7 +101,7 @@ go_template_instance(
out = "ipc_namespace_refs.go",
package = "kernel",
prefix = "IPCNamespace",
template = "//pkg/refs_vfs2:refs_template",
template = "//pkg/refsvfs2:refs_template",
types = {
"T": "IPCNamespace",
},
@ -112,7 +112,7 @@ go_template_instance(
out = "process_group_refs.go",
package = "kernel",
prefix = "ProcessGroup",
template = "//pkg/refs_vfs2:refs_template",
template = "//pkg/refsvfs2:refs_template",
types = {
"T": "ProcessGroup",
},
@ -123,7 +123,7 @@ go_template_instance(
out = "session_refs.go",
package = "kernel",
prefix = "Session",
template = "//pkg/refs_vfs2:refs_template",
template = "//pkg/refsvfs2:refs_template",
types = {
"T": "Session",
},
@ -229,7 +229,7 @@ go_library(
"//pkg/marshal/primitive",
"//pkg/metric",
"//pkg/refs",
"//pkg/refs_vfs2",
"//pkg/refsvfs2",
"//pkg/safemem",
"//pkg/secio",
"//pkg/sentry/arch",

View File

@ -19,7 +19,7 @@ import (
"syscall"
"gvisor.dev/gvisor/pkg/context"
"gvisor.dev/gvisor/pkg/refs_vfs2"
"gvisor.dev/gvisor/pkg/refsvfs2"
"gvisor.dev/gvisor/pkg/sentry/socket/unix/transport"
"gvisor.dev/gvisor/pkg/sync"
)
@ -27,7 +27,7 @@ import (
// +stateify savable
type abstractEndpoint struct {
ep transport.BoundEndpoint
socket refs_vfs2.RefCounter
socket refsvfs2.RefCounter
name string
ns *AbstractSocketNamespace
}
@ -57,7 +57,7 @@ func NewAbstractSocketNamespace() *AbstractSocketNamespace {
// its backing socket.
type boundEndpoint struct {
transport.BoundEndpoint
socket refs_vfs2.RefCounter
socket refsvfs2.RefCounter
}
// Release implements transport.BoundEndpoint.Release.
@ -89,7 +89,7 @@ func (a *AbstractSocketNamespace) BoundEndpoint(name string) transport.BoundEndp
//
// When the last reference managed by socket is dropped, ep may be removed from the
// namespace.
func (a *AbstractSocketNamespace) Bind(ctx context.Context, name string, ep transport.BoundEndpoint, socket refs_vfs2.RefCounter) error {
func (a *AbstractSocketNamespace) Bind(ctx context.Context, name string, ep transport.BoundEndpoint, socket refsvfs2.RefCounter) error {
a.mu.Lock()
defer a.mu.Unlock()
@ -109,7 +109,7 @@ func (a *AbstractSocketNamespace) Bind(ctx context.Context, name string, ep tran
// Remove removes the specified socket at name from the abstract socket
// namespace, if it has not yet been replaced.
func (a *AbstractSocketNamespace) Remove(name string, socket refs_vfs2.RefCounter) {
func (a *AbstractSocketNamespace) Remove(name string, socket refsvfs2.RefCounter) {
a.mu.Lock()
defer a.mu.Unlock()

View File

@ -110,7 +110,7 @@ func (f *FDTable) saveDescriptorTable() map[int32]descriptor {
func (f *FDTable) loadDescriptorTable(m map[int32]descriptor) {
ctx := context.Background()
f.init() // Initialize table.
f.initNoLeakCheck() // Initialize table.
f.used = 0
for fd, d := range m {
if file, fileVFS2 := f.setAll(ctx, fd, d.file, d.fileVFS2, d.flags); file != nil || fileVFS2 != nil {

View File

@ -31,14 +31,21 @@ type descriptorTable struct {
slice unsafe.Pointer `state:".(map[int32]*descriptor)"`
}
// init initializes the table.
// initNoLeakCheck initializes the table without enabling leak checking.
//
// TODO(gvisor.dev/1486): Enable leak check for FDTable.
func (f *FDTable) init() {
// This is used when loading an FDTable after S/R, during which the ref count
// object itself will enable leak checking if necessary.
func (f *FDTable) initNoLeakCheck() {
var slice []unsafe.Pointer // Empty slice.
atomic.StorePointer(&f.slice, unsafe.Pointer(&slice))
}
// init initializes the table with leak checking.
func (f *FDTable) init() {
f.initNoLeakCheck()
f.EnableLeakCheck()
}
// get gets a file entry.
//
// The boolean indicates whether this was in range.

View File

@ -130,13 +130,15 @@ func (f *FSContext) Fork() *FSContext {
f.root.IncRef()
}
return &FSContext{
ctx := &FSContext{
cwd: f.cwd,
root: f.root,
cwdVFS2: f.cwdVFS2,
rootVFS2: f.rootVFS2,
umask: f.umask,
}
ctx.EnableLeakCheck()
return ctx
}
// WorkingDirectory returns the current working directory.

View File

@ -55,7 +55,7 @@ func (i *IPCNamespace) ShmRegistry() *shm.Registry {
return i.shms
}
// DecRef implements refs_vfs2.RefCounter.DecRef.
// DecRef implements refsvfs2.RefCounter.DecRef.
func (i *IPCNamespace) DecRef(ctx context.Context) {
i.IPCNamespaceRefs.DecRef(func() {
i.shms.Release(ctx)

View File

@ -8,7 +8,7 @@ go_template_instance(
out = "shm_refs.go",
package = "shm",
prefix = "Shm",
template = "//pkg/refs_vfs2:refs_template",
template = "//pkg/refsvfs2:refs_template",
types = {
"T": "Shm",
},
@ -27,7 +27,7 @@ go_library(
"//pkg/context",
"//pkg/log",
"//pkg/refs",
"//pkg/refs_vfs2",
"//pkg/refsvfs2",
"//pkg/sentry/device",
"//pkg/sentry/fs",
"//pkg/sentry/kernel/auth",

View File

@ -78,7 +78,7 @@ go_template_instance(
out = "aio_mappable_refs.go",
package = "mm",
prefix = "aioMappable",
template = "//pkg/refs_vfs2:refs_template",
template = "//pkg/refsvfs2:refs_template",
types = {
"T": "aioMappable",
},
@ -89,7 +89,7 @@ go_template_instance(
out = "special_mappable_refs.go",
package = "mm",
prefix = "SpecialMappable",
template = "//pkg/refs_vfs2:refs_template",
template = "//pkg/refsvfs2:refs_template",
types = {
"T": "SpecialMappable",
},
@ -127,6 +127,7 @@ go_library(
"//pkg/context",
"//pkg/log",
"//pkg/refs",
"//pkg/refsvfs2",
"//pkg/safecopy",
"//pkg/safemem",
"//pkg/sentry/arch",

View File

@ -8,7 +8,7 @@ go_template_instance(
out = "socket_refs.go",
package = "unix",
prefix = "socketOperations",
template = "//pkg/refs_vfs2:refs_template",
template = "//pkg/refsvfs2:refs_template",
types = {
"T": "SocketOperations",
},
@ -19,7 +19,7 @@ go_template_instance(
out = "socket_vfs2_refs.go",
package = "unix",
prefix = "socketVFS2",
template = "//pkg/refs_vfs2:refs_template",
template = "//pkg/refsvfs2:refs_template",
types = {
"T": "SocketVFS2",
},
@ -43,6 +43,7 @@ go_library(
"//pkg/log",
"//pkg/marshal",
"//pkg/refs",
"//pkg/refsvfs2",
"//pkg/safemem",
"//pkg/sentry/arch",
"//pkg/sentry/device",

View File

@ -20,7 +20,7 @@ go_template_instance(
out = "queue_refs.go",
package = "transport",
prefix = "queue",
template = "//pkg/refs_vfs2:refs_template",
template = "//pkg/refsvfs2:refs_template",
types = {
"T": "queue",
},
@ -44,6 +44,7 @@ go_library(
"//pkg/ilist",
"//pkg/log",
"//pkg/refs",
"//pkg/refsvfs2",
"//pkg/sync",
"//pkg/syserr",
"//pkg/tcpip",

View File

@ -81,7 +81,6 @@ func NewWithDirent(ctx context.Context, d *fs.Dirent, ep transport.Endpoint, sty
},
}
s.EnableLeakCheck()
return fs.NewFile(ctx, d, flags, &s)
}

View File

@ -80,6 +80,7 @@ func NewFileDescription(ep transport.Endpoint, stype linux.SockType, flags uint3
stype: stype,
},
}
sock.EnableLeakCheck()
sock.LockFD.Init(locks)
vfsfd := &sock.vfsfd
if err := vfsfd.Init(sock, flags, mnt, d, &vfs.FileDescriptionOptions{

View File

@ -32,7 +32,7 @@ go_template_instance(
out = "file_description_refs.go",
package = "vfs",
prefix = "FileDescription",
template = "//pkg/refs_vfs2:refs_template",
template = "//pkg/refsvfs2:refs_template",
types = {
"T": "FileDescription",
},
@ -43,7 +43,7 @@ go_template_instance(
out = "mount_namespace_refs.go",
package = "vfs",
prefix = "MountNamespace",
template = "//pkg/refs_vfs2:refs_template",
template = "//pkg/refsvfs2:refs_template",
types = {
"T": "MountNamespace",
},
@ -54,7 +54,7 @@ go_template_instance(
out = "filesystem_refs.go",
package = "vfs",
prefix = "Filesystem",
template = "//pkg/refs_vfs2:refs_template",
template = "//pkg/refsvfs2:refs_template",
types = {
"T": "Filesystem",
},
@ -99,6 +99,7 @@ go_library(
"//pkg/gohacks",
"//pkg/log",
"//pkg/refs",
"//pkg/refsvfs2",
"//pkg/safemem",
"//pkg/sentry/arch",
"//pkg/sentry/fs",

View File

@ -8,7 +8,7 @@ go_template_instance(
out = "tun_endpoint_refs.go",
package = "tun",
prefix = "tunEndpoint",
template = "//pkg/refs_vfs2:refs_template",
template = "//pkg/refsvfs2:refs_template",
types = {
"T": "tunEndpoint",
},
@ -28,6 +28,7 @@ go_library(
"//pkg/context",
"//pkg/log",
"//pkg/refs",
"//pkg/refsvfs2",
"//pkg/sync",
"//pkg/syserror",
"//pkg/tcpip",

View File

@ -150,7 +150,6 @@ func attachOrCreateNIC(s *stack.Stack, name, prefix string, linkCaps stack.LinkE
// 2. Creating a new NIC.
id := tcpip.NICID(s.UniqueID())
// TODO(gvisor.dev/1486): enable leak check for tunEndpoint.
endpoint := &tunEndpoint{
Endpoint: channel.New(defaultDevOutQueueLen, defaultDevMtu, ""),
stack: s,
@ -158,6 +157,7 @@ func attachOrCreateNIC(s *stack.Stack, name, prefix string, linkCaps stack.LinkE
name: name,
isTap: prefix == "tap",
}
endpoint.EnableLeakCheck()
endpoint.Endpoint.LinkEPCapabilities = linkCaps
if endpoint.name == "" {
endpoint.name = fmt.Sprintf("%s%d", prefix, id)

View File

@ -38,6 +38,7 @@ go_library(
"//pkg/memutil",
"//pkg/rand",
"//pkg/refs",
"//pkg/refsvfs2",
"//pkg/sentry/arch",
"//pkg/sentry/arch:registers_go_proto",
"//pkg/sentry/control",

View File

@ -35,6 +35,7 @@ import (
"gvisor.dev/gvisor/pkg/memutil"
"gvisor.dev/gvisor/pkg/rand"
"gvisor.dev/gvisor/pkg/refs"
"gvisor.dev/gvisor/pkg/refsvfs2"
"gvisor.dev/gvisor/pkg/sentry/arch"
"gvisor.dev/gvisor/pkg/sentry/control"
"gvisor.dev/gvisor/pkg/sentry/fdimport"
@ -476,6 +477,12 @@ func (l *Loader) Destroy() {
// save/restore.
l.k.Release()
// All sentry-created resources should have been released at this point;
// check for reference leaks.
if refsvfs2.LeakCheckEnabled() {
refsvfs2.DoLeakCheck()
}
// In the success case, stdioFDs and goferFDs will only contain
// released/closed FDs that ownership has been passed over to host FDs and
// gofer sessions. Close them here in case of failure.

View File

@ -53,6 +53,9 @@ var (
runscPath = flag.String("runsc", "", "path to runsc binary")
addUDSTree = flag.Bool("add-uds-tree", false, "expose a tree of UDS utilities for use in tests")
// TODO(gvisor.dev/issue/4572): properly support leak checking for runsc, and
// set to true as the default for the test runner.
leakCheck = flag.Bool("leak-check", false, "check for reference leaks")
)
// runTestCaseNative runs the test case directly on the host machine.
@ -174,6 +177,9 @@ func runRunsc(tc gtest.TestCase, spec *specs.Spec) error {
if *addUDSTree {
args = append(args, "-fsgofer-host-uds")
}
if *leakCheck {
args = append(args, "-ref-leak-mode=log-names")
}
testLogDir := ""
if undeclaredOutputsDir, ok := syscall.Getenv("TEST_UNDECLARED_OUTPUTS_DIR"); ok {

View File

@ -630,7 +630,9 @@ TEST_P(UnixSocketPairCmsgTest, FDPassNotCoalesced) {
TransferTest(pair2->first_fd(), pair2->second_fd());
}
TEST_P(UnixSocketPairCmsgTest, FDPassPeek) {
// TODO(b/171425923): Enable random/cooperative save once fixed.
TEST_P(UnixSocketPairCmsgTest, FDPassPeek_NoRandomSave) {
const DisableSave ds;
auto sockets = ASSERT_NO_ERRNO_AND_VALUE(NewSocketPair());
char sent_data[20];

View File

@ -168,7 +168,6 @@ func init() {
"pkg/pool/pool.go:15",
"pkg/refs/refcounter.go:510",
"pkg/refs/refcounter_test.go:169",
"pkg/refs_vfs2/refs.go:16",
"pkg/safemem/block_unsafe.go:89",
"pkg/seccomp/seccomp.go:82",
"pkg/segment/test/set_functions.go:15",