summaryrefslogtreecommitdiffhomepage
path: root/pkg
diff options
context:
space:
mode:
Diffstat (limited to 'pkg')
-rw-r--r--pkg/refs/refcounter.go10
-rw-r--r--pkg/refsvfs2/refs_map.go93
-rw-r--r--pkg/sentry/fsimpl/devpts/root_inode_refs.go56
-rw-r--r--pkg/sentry/fsimpl/fuse/inode_refs.go56
-rw-r--r--pkg/sentry/fsimpl/gofer/directory.go4
-rw-r--r--pkg/sentry/fsimpl/gofer/filesystem.go8
-rw-r--r--pkg/sentry/fsimpl/gofer/gofer.go58
-rw-r--r--pkg/sentry/fsimpl/gofer/save_restore.go4
-rw-r--r--pkg/sentry/fsimpl/host/connected_endpoint_refs.go56
-rw-r--r--pkg/sentry/fsimpl/host/inode_refs.go56
-rw-r--r--pkg/sentry/fsimpl/kernfs/static_directory_refs.go56
-rw-r--r--pkg/sentry/fsimpl/kernfs/synthetic_directory_refs.go56
-rw-r--r--pkg/sentry/fsimpl/overlay/overlay.go53
-rw-r--r--pkg/sentry/fsimpl/overlay/save_restore.go4
-rw-r--r--pkg/sentry/fsimpl/proc/fd_dir_inode_refs.go56
-rw-r--r--pkg/sentry/fsimpl/proc/fd_info_dir_inode_refs.go56
-rw-r--r--pkg/sentry/fsimpl/proc/subtasks_inode_refs.go56
-rw-r--r--pkg/sentry/fsimpl/proc/task_inode_refs.go56
-rw-r--r--pkg/sentry/fsimpl/proc/tasks_inode_refs.go56
-rw-r--r--pkg/sentry/fsimpl/sys/dir_refs.go56
-rw-r--r--pkg/sentry/fsimpl/tmpfs/inode_refs.go56
-rw-r--r--pkg/sentry/kernel/fd_table_refs.go56
-rw-r--r--pkg/sentry/kernel/fs_context_refs.go56
-rw-r--r--pkg/sentry/kernel/ipc_namespace_refs.go56
-rw-r--r--pkg/sentry/kernel/process_group_refs.go56
-rw-r--r--pkg/sentry/kernel/session_refs.go56
-rw-r--r--pkg/sentry/kernel/shm/shm_refs.go56
-rw-r--r--pkg/sentry/mm/aio_mappable_refs.go56
-rw-r--r--pkg/sentry/mm/special_mappable_refs.go56
-rw-r--r--pkg/sentry/socket/unix/socket_refs.go56
-rw-r--r--pkg/sentry/socket/unix/socket_vfs2_refs.go56
-rw-r--r--pkg/sentry/socket/unix/transport/queue_refs.go56
-rw-r--r--pkg/sentry/vfs/file_description_refs.go56
-rw-r--r--pkg/sentry/vfs/filesystem_refs.go56
-rw-r--r--pkg/sentry/vfs/mount.go31
-rw-r--r--pkg/sentry/vfs/mount_namespace_refs.go56
-rw-r--r--pkg/sentry/vfs/save_restore.go4
-rw-r--r--pkg/tcpip/link/tun/tun_endpoint_refs.go56
38 files changed, 1179 insertions, 658 deletions
diff --git a/pkg/refs/refcounter.go b/pkg/refs/refcounter.go
index 699ea8ac3..6992e1de8 100644
--- a/pkg/refs/refcounter.go
+++ b/pkg/refs/refcounter.go
@@ -319,7 +319,8 @@ func makeStackKey(pcs []uintptr) stackKey {
return key
}
-func recordStack() []uintptr {
+// RecordStack constructs and returns the PCs on the current stack.
+func RecordStack() []uintptr {
pcs := make([]uintptr, maxStackFrames)
n := runtime.Callers(1, pcs)
if n == 0 {
@@ -342,7 +343,8 @@ func recordStack() []uintptr {
return v
}
-func formatStack(pcs []uintptr) string {
+// FormatStack converts the given stack into a readable format.
+func FormatStack(pcs []uintptr) string {
frames := runtime.CallersFrames(pcs)
var trace bytes.Buffer
for {
@@ -367,7 +369,7 @@ func (r *AtomicRefCount) finalize() {
if n := r.ReadRefs(); n != 0 {
msg := fmt.Sprintf("%sAtomicRefCount %p owned by %q garbage collected with ref count of %d (want 0)", note, r, r.name, n)
if len(r.stack) != 0 {
- msg += ":\nCaller:\n" + formatStack(r.stack)
+ msg += ":\nCaller:\n" + FormatStack(r.stack)
} else {
msg += " (enable trace logging to debug)"
}
@@ -392,7 +394,7 @@ func (r *AtomicRefCount) EnableLeakCheck(name string) {
case NoLeakChecking:
return
case LeaksLogTraces:
- r.stack = recordStack()
+ r.stack = RecordStack()
}
r.name = name
runtime.SetFinalizer(r, (*AtomicRefCount).finalize)
diff --git a/pkg/refsvfs2/refs_map.go b/pkg/refsvfs2/refs_map.go
index be75b0cc2..57938d2f0 100644
--- a/pkg/refsvfs2/refs_map.go
+++ b/pkg/refsvfs2/refs_map.go
@@ -37,61 +37,98 @@ var (
// CheckedObject represents a reference-counted object with an informative
// leak detection message.
type CheckedObject interface {
+ // RefType is the type of the reference-counted object.
+ RefType() string
+
// LeakMessage supplies a warning to be printed upon leak detection.
LeakMessage() string
+
+ // LogRefs indicates whether reference-related events should be logged.
+ LogRefs() bool
}
func init() {
liveObjects = make(map[CheckedObject]struct{})
}
-// LeakCheckEnabled returns whether leak checking is enabled. The following
+// leakCheckEnabled returns whether leak checking is enabled. The following
// functions should only be called if it returns true.
-func LeakCheckEnabled() bool {
+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
+func Register(obj CheckedObject) {
+ if leakCheckEnabled() {
+ for _, str := range ignored {
+ if strings.Contains(obj.RefType(), 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()
+ logEvent(obj, "registered")
}
- 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
+func Unregister(obj CheckedObject) {
+ if leakCheckEnabled() {
+ liveObjectsMu.Lock()
+ defer liveObjectsMu.Unlock()
+ if _, ok := liveObjects[obj]; !ok {
+ for _, str := range ignored {
+ if strings.Contains(obj.RefType(), str) {
+ return
+ }
}
+ panic(fmt.Sprintf("Expected to find entry in leak checking map for reference %p", obj))
}
- panic(fmt.Sprintf("Expected to find entry in leak checking map for reference %p", obj))
+ delete(liveObjects, obj)
+ logEvent(obj, "unregistered")
+ }
+}
+
+// LogIncRef logs a reference increment.
+func LogIncRef(obj CheckedObject, refs int64) {
+ logEvent(obj, fmt.Sprintf("IncRef to %d", refs))
+}
+
+// LogTryIncRef logs a successful TryIncRef call.
+func LogTryIncRef(obj CheckedObject, refs int64) {
+ logEvent(obj, fmt.Sprintf("TryIncRef to %d", refs))
+}
+
+// LogDecRef logs a reference decrement.
+func LogDecRef(obj CheckedObject, refs int64) {
+ logEvent(obj, fmt.Sprintf("DecRef to %d", refs))
+}
+
+// logEvent logs a message for the given reference-counted object.
+func logEvent(obj CheckedObject, msg string) {
+ if obj.LogRefs() {
+ log.Infof("[%s %p] %s:", obj.RefType(), obj, msg)
+ log.Infof(refs_vfs1.FormatStack(refs_vfs1.RecordStack()))
}
- 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())
+ if leakCheckEnabled() {
+ 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())
+ }
}
}
}
diff --git a/pkg/sentry/fsimpl/devpts/root_inode_refs.go b/pkg/sentry/fsimpl/devpts/root_inode_refs.go
index 577b92f36..cbafd113a 100644
--- a/pkg/sentry/fsimpl/devpts/root_inode_refs.go
+++ b/pkg/sentry/fsimpl/devpts/root_inode_refs.go
@@ -7,9 +7,15 @@ import (
"gvisor.dev/gvisor/pkg/refsvfs2"
)
-// ownerType is used to customize logging. Note that we use a pointer to T so
-// that we do not copy the entire object when passed as a format parameter.
-var rootInodeownerType *rootInode
+// enableLogging indicates whether reference-related events should be logged (with
+// stack traces). This is false by default and should only be set to true for
+// debugging purposes, as it can generate an extremely large amount of output
+// and drastically degrade performance.
+const rootInodeenableLogging = false
+
+// obj is used to customize logging. Note that we use a pointer to T so that
+// we do not copy the entire object when passed as a format parameter.
+var rootInodeobj *rootInode
// Refs implements refs.RefCounter. It keeps a reference count using atomic
// operations and calls the destructor when the count reaches zero.
@@ -29,16 +35,24 @@ type rootInodeRefs struct {
refCount int64
}
-// EnableLeakCheck enables reference leak checking on r.
-func (r *rootInodeRefs) EnableLeakCheck() {
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Register(r, fmt.Sprintf("%T", rootInodeownerType))
- }
+// RefType implements refsvfs2.CheckedObject.RefType.
+func (r *rootInodeRefs) RefType() string {
+ return fmt.Sprintf("%T", rootInodeobj)[1:]
}
// LeakMessage implements refsvfs2.CheckedObject.LeakMessage.
func (r *rootInodeRefs) LeakMessage() string {
- return fmt.Sprintf("%T %p: reference count of %d instead of 0", rootInodeownerType, r, r.ReadRefs())
+ return fmt.Sprintf("[%s %p] reference count of %d instead of 0", r.RefType(), r, r.ReadRefs())
+}
+
+// LogRefs implements refsvfs2.CheckedObject.LogRefs.
+func (r *rootInodeRefs) LogRefs() bool {
+ return rootInodeenableLogging
+}
+
+// EnableLeakCheck enables reference leak checking on r.
+func (r *rootInodeRefs) EnableLeakCheck() {
+ refsvfs2.Register(r)
}
// ReadRefs returns the current number of references. The returned count is
@@ -52,8 +66,10 @@ func (r *rootInodeRefs) ReadRefs() int64 {
//
//go:nosplit
func (r *rootInodeRefs) IncRef() {
- if v := atomic.AddInt64(&r.refCount, 1); v <= 0 {
- panic(fmt.Sprintf("Incrementing non-positive count %p on %T", r, rootInodeownerType))
+ v := atomic.AddInt64(&r.refCount, 1)
+ refsvfs2.LogIncRef(r, v+1)
+ if v <= 0 {
+ panic(fmt.Sprintf("Incrementing non-positive count %p on %s", r, r.RefType()))
}
}
@@ -66,14 +82,14 @@ func (r *rootInodeRefs) IncRef() {
//go:nosplit
func (r *rootInodeRefs) TryIncRef() bool {
const speculativeRef = 1 << 32
- v := atomic.AddInt64(&r.refCount, speculativeRef)
- if int32(v) < 0 {
+ if v := atomic.AddInt64(&r.refCount, speculativeRef); int32(v) < 0 {
atomic.AddInt64(&r.refCount, -speculativeRef)
return false
}
- atomic.AddInt64(&r.refCount, -speculativeRef+1)
+ v := atomic.AddInt64(&r.refCount, -speculativeRef+1)
+ refsvfs2.LogTryIncRef(r, v+1)
return true
}
@@ -90,14 +106,14 @@ func (r *rootInodeRefs) TryIncRef() bool {
//
//go:nosplit
func (r *rootInodeRefs) DecRef(destroy func()) {
- switch v := atomic.AddInt64(&r.refCount, -1); {
+ v := atomic.AddInt64(&r.refCount, -1)
+ refsvfs2.LogDecRef(r, v+1)
+ switch {
case v < -1:
- panic(fmt.Sprintf("Decrementing non-positive ref count %p, owned by %T", r, rootInodeownerType))
+ panic(fmt.Sprintf("Decrementing non-positive ref count %p, owned by %s", r, r.RefType()))
case v == -1:
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Unregister(r, fmt.Sprintf("%T", rootInodeownerType))
- }
+ refsvfs2.Unregister(r)
if destroy != nil {
destroy()
@@ -106,7 +122,7 @@ func (r *rootInodeRefs) DecRef(destroy func()) {
}
func (r *rootInodeRefs) afterLoad() {
- if refsvfs2.LeakCheckEnabled() && r.ReadRefs() > 0 {
+ if r.ReadRefs() > 0 {
r.EnableLeakCheck()
}
}
diff --git a/pkg/sentry/fsimpl/fuse/inode_refs.go b/pkg/sentry/fsimpl/fuse/inode_refs.go
index 970f90a91..31cb3791f 100644
--- a/pkg/sentry/fsimpl/fuse/inode_refs.go
+++ b/pkg/sentry/fsimpl/fuse/inode_refs.go
@@ -7,9 +7,15 @@ import (
"gvisor.dev/gvisor/pkg/refsvfs2"
)
-// ownerType is used to customize logging. Note that we use a pointer to T so
-// that we do not copy the entire object when passed as a format parameter.
-var inodeownerType *inode
+// enableLogging indicates whether reference-related events should be logged (with
+// stack traces). This is false by default and should only be set to true for
+// debugging purposes, as it can generate an extremely large amount of output
+// and drastically degrade performance.
+const inodeenableLogging = false
+
+// obj is used to customize logging. Note that we use a pointer to T so that
+// we do not copy the entire object when passed as a format parameter.
+var inodeobj *inode
// Refs implements refs.RefCounter. It keeps a reference count using atomic
// operations and calls the destructor when the count reaches zero.
@@ -29,16 +35,24 @@ type inodeRefs struct {
refCount int64
}
-// EnableLeakCheck enables reference leak checking on r.
-func (r *inodeRefs) EnableLeakCheck() {
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Register(r, fmt.Sprintf("%T", inodeownerType))
- }
+// RefType implements refsvfs2.CheckedObject.RefType.
+func (r *inodeRefs) RefType() string {
+ return fmt.Sprintf("%T", inodeobj)[1:]
}
// LeakMessage implements refsvfs2.CheckedObject.LeakMessage.
func (r *inodeRefs) LeakMessage() string {
- return fmt.Sprintf("%T %p: reference count of %d instead of 0", inodeownerType, r, r.ReadRefs())
+ return fmt.Sprintf("[%s %p] reference count of %d instead of 0", r.RefType(), r, r.ReadRefs())
+}
+
+// LogRefs implements refsvfs2.CheckedObject.LogRefs.
+func (r *inodeRefs) LogRefs() bool {
+ return inodeenableLogging
+}
+
+// EnableLeakCheck enables reference leak checking on r.
+func (r *inodeRefs) EnableLeakCheck() {
+ refsvfs2.Register(r)
}
// ReadRefs returns the current number of references. The returned count is
@@ -52,8 +66,10 @@ func (r *inodeRefs) ReadRefs() int64 {
//
//go:nosplit
func (r *inodeRefs) IncRef() {
- if v := atomic.AddInt64(&r.refCount, 1); v <= 0 {
- panic(fmt.Sprintf("Incrementing non-positive count %p on %T", r, inodeownerType))
+ v := atomic.AddInt64(&r.refCount, 1)
+ refsvfs2.LogIncRef(r, v+1)
+ if v <= 0 {
+ panic(fmt.Sprintf("Incrementing non-positive count %p on %s", r, r.RefType()))
}
}
@@ -66,14 +82,14 @@ func (r *inodeRefs) IncRef() {
//go:nosplit
func (r *inodeRefs) TryIncRef() bool {
const speculativeRef = 1 << 32
- v := atomic.AddInt64(&r.refCount, speculativeRef)
- if int32(v) < 0 {
+ if v := atomic.AddInt64(&r.refCount, speculativeRef); int32(v) < 0 {
atomic.AddInt64(&r.refCount, -speculativeRef)
return false
}
- atomic.AddInt64(&r.refCount, -speculativeRef+1)
+ v := atomic.AddInt64(&r.refCount, -speculativeRef+1)
+ refsvfs2.LogTryIncRef(r, v+1)
return true
}
@@ -90,14 +106,14 @@ func (r *inodeRefs) TryIncRef() bool {
//
//go:nosplit
func (r *inodeRefs) DecRef(destroy func()) {
- switch v := atomic.AddInt64(&r.refCount, -1); {
+ v := atomic.AddInt64(&r.refCount, -1)
+ refsvfs2.LogDecRef(r, v+1)
+ switch {
case v < -1:
- panic(fmt.Sprintf("Decrementing non-positive ref count %p, owned by %T", r, inodeownerType))
+ panic(fmt.Sprintf("Decrementing non-positive ref count %p, owned by %s", r, r.RefType()))
case v == -1:
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Unregister(r, fmt.Sprintf("%T", inodeownerType))
- }
+ refsvfs2.Unregister(r)
if destroy != nil {
destroy()
@@ -106,7 +122,7 @@ func (r *inodeRefs) DecRef(destroy func()) {
}
func (r *inodeRefs) afterLoad() {
- if refsvfs2.LeakCheckEnabled() && r.ReadRefs() > 0 {
+ if r.ReadRefs() > 0 {
r.EnableLeakCheck()
}
}
diff --git a/pkg/sentry/fsimpl/gofer/directory.go b/pkg/sentry/fsimpl/gofer/directory.go
index e993c8e36..ce1b2a390 100644
--- a/pkg/sentry/fsimpl/gofer/directory.go
+++ b/pkg/sentry/fsimpl/gofer/directory.go
@@ -101,9 +101,7 @@ func (d *dentry) createSyntheticChildLocked(opts *createSyntheticOpts) {
hostFD: -1,
nlink: uint32(2),
}
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Register(child, "gofer.dentry")
- }
+ refsvfs2.Register(child)
switch opts.mode.FileType() {
case linux.S_IFDIR:
// Nothing else needs to be done.
diff --git a/pkg/sentry/fsimpl/gofer/filesystem.go b/pkg/sentry/fsimpl/gofer/filesystem.go
index baecb88c4..57a2ca43c 100644
--- a/pkg/sentry/fsimpl/gofer/filesystem.go
+++ b/pkg/sentry/fsimpl/gofer/filesystem.go
@@ -262,7 +262,7 @@ func (fs *filesystem) revalidateChildLocked(ctx context.Context, vfsObj *vfs.Vir
// treat their invalidation as deletion.
child.setDeleted()
parent.syntheticChildren--
- child.decRefLocked()
+ child.decRefNoCaching()
parent.dirents = nil
}
*ds = appendDentry(*ds, child)
@@ -631,7 +631,7 @@ func (fs *filesystem) unlinkAt(ctx context.Context, rp *vfs.ResolvingPath, dir b
child.setDeleted()
if child.isSynthetic() {
parent.syntheticChildren--
- child.decRefLocked()
+ child.decRefNoCaching()
}
ds = appendDentry(ds, child)
}
@@ -1361,7 +1361,7 @@ func (fs *filesystem) RenameAt(ctx context.Context, rp *vfs.ResolvingPath, oldPa
replaced.setDeleted()
if replaced.isSynthetic() {
newParent.syntheticChildren--
- replaced.decRefLocked()
+ replaced.decRefNoCaching()
}
ds = appendDentry(ds, replaced)
}
@@ -1370,7 +1370,7 @@ func (fs *filesystem) RenameAt(ctx context.Context, rp *vfs.ResolvingPath, oldPa
// with reference counts and queue oldParent for checkCachingLocked if the
// parent isn't actually changing.
if oldParent != newParent {
- oldParent.decRefLocked()
+ oldParent.decRefNoCaching()
ds = appendDentry(ds, oldParent)
newParent.IncRef()
if renamed.isSynthetic() {
diff --git a/pkg/sentry/fsimpl/gofer/gofer.go b/pkg/sentry/fsimpl/gofer/gofer.go
index 80668ebc1..6f82ce61b 100644
--- a/pkg/sentry/fsimpl/gofer/gofer.go
+++ b/pkg/sentry/fsimpl/gofer/gofer.go
@@ -590,7 +590,7 @@ func (fs *filesystem) Release(ctx context.Context) {
// Precondition: d.fs.renameMu is locked.
func (d *dentry) releaseSyntheticRecursiveLocked(ctx context.Context) {
if d.isSynthetic() {
- d.decRefLocked()
+ d.decRefNoCaching()
d.checkCachingLocked(ctx)
}
if d.isDir() {
@@ -860,10 +860,7 @@ func (fs *filesystem) newDentry(ctx context.Context, file p9file, qid p9.QID, ma
d.nlink = uint32(attr.NLink)
}
d.vfsd.Init(d)
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Register(d, "gofer.dentry")
- }
-
+ refsvfs2.Register(d)
fs.syncMu.Lock()
fs.syncableDentries[d] = struct{}{}
fs.syncMu.Unlock()
@@ -1222,17 +1219,19 @@ func dentryGIDFromP9GID(gid p9.GID) uint32 {
func (d *dentry) IncRef() {
// d.refs may be 0 if d.fs.renameMu is locked, which serializes against
// d.checkCachingLocked().
- atomic.AddInt64(&d.refs, 1)
+ r := atomic.AddInt64(&d.refs, 1)
+ refsvfs2.LogIncRef(d, r)
}
// TryIncRef implements vfs.DentryImpl.TryIncRef.
func (d *dentry) TryIncRef() bool {
for {
- refs := atomic.LoadInt64(&d.refs)
- if refs <= 0 {
+ r := atomic.LoadInt64(&d.refs)
+ if r <= 0 {
return false
}
- if atomic.CompareAndSwapInt64(&d.refs, refs, refs+1) {
+ if atomic.CompareAndSwapInt64(&d.refs, r, r+1) {
+ refsvfs2.LogTryIncRef(d, r+1)
return true
}
}
@@ -1240,22 +1239,28 @@ func (d *dentry) TryIncRef() bool {
// DecRef implements vfs.DentryImpl.DecRef.
func (d *dentry) DecRef(ctx context.Context) {
- if refs := atomic.AddInt64(&d.refs, -1); refs == 0 {
+ if d.decRefNoCaching() == 0 {
d.fs.renameMu.Lock()
d.checkCachingLocked(ctx)
d.fs.renameMu.Unlock()
- } else if refs < 0 {
- panic("gofer.dentry.DecRef() called without holding a reference")
}
}
-// decRefLocked decrements d's reference count without calling
+// decRefNoCaching decrements d's reference count without calling
// d.checkCachingLocked, even if d's reference count reaches 0; callers are
// responsible for ensuring that d.checkCachingLocked will be called later.
-func (d *dentry) decRefLocked() {
- if refs := atomic.AddInt64(&d.refs, -1); refs < 0 {
- panic("gofer.dentry.decRefLocked() called without holding a reference")
+func (d *dentry) decRefNoCaching() int64 {
+ r := atomic.AddInt64(&d.refs, -1)
+ refsvfs2.LogDecRef(d, r)
+ if r < 0 {
+ panic("gofer.dentry.decRefNoCaching() called without holding a reference")
}
+ return r
+}
+
+// RefType implements refsvfs2.CheckedObject.Type.
+func (d *dentry) RefType() string {
+ return "gofer.dentry"
}
// LeakMessage implements refsvfs2.CheckedObject.LeakMessage.
@@ -1263,6 +1268,14 @@ func (d *dentry) LeakMessage() string {
return fmt.Sprintf("[gofer.dentry %p] reference count of %d instead of -1", d, atomic.LoadInt64(&d.refs))
}
+// LogRefs implements refsvfs2.CheckedObject.LogRefs.
+//
+// This should only be set to true for debugging purposes, as it can generate an
+// extremely large amount of output and drastically degrade performance.
+func (d *dentry) LogRefs() bool {
+ return false
+}
+
// InotifyWithParent implements vfs.DentryImpl.InotifyWithParent.
func (d *dentry) InotifyWithParent(ctx context.Context, events, cookie uint32, et vfs.EventType) {
if d.isDir() {
@@ -1486,17 +1499,10 @@ func (d *dentry) destroyLocked(ctx context.Context) {
// Drop the reference held by d on its parent without recursively locking
// d.fs.renameMu.
- if d.parent != nil {
- if refs := atomic.AddInt64(&d.parent.refs, -1); refs == 0 {
- d.parent.checkCachingLocked(ctx)
- } else if refs < 0 {
- panic("gofer.dentry.DecRef() called without holding a reference")
- }
- }
-
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Unregister(d, "gofer.dentry")
+ if d.parent != nil && d.parent.decRefNoCaching() == 0 {
+ d.parent.checkCachingLocked(ctx)
}
+ refsvfs2.Unregister(d)
}
func (d *dentry) isDeleted() bool {
diff --git a/pkg/sentry/fsimpl/gofer/save_restore.go b/pkg/sentry/fsimpl/gofer/save_restore.go
index 2ea224c43..17849dcc0 100644
--- a/pkg/sentry/fsimpl/gofer/save_restore.go
+++ b/pkg/sentry/fsimpl/gofer/save_restore.go
@@ -140,8 +140,8 @@ func (d *dentry) beforeSave() {
// afterLoad is invoked by stateify.
func (d *dentry) afterLoad() {
d.hostFD = -1
- if refsvfs2.LeakCheckEnabled() && atomic.LoadInt64(&d.refs) != -1 {
- refsvfs2.Register(d, "gofer.dentry")
+ if atomic.LoadInt64(&d.refs) != -1 {
+ refsvfs2.Register(d)
}
}
diff --git a/pkg/sentry/fsimpl/host/connected_endpoint_refs.go b/pkg/sentry/fsimpl/host/connected_endpoint_refs.go
index 6d2f22c8d..faa9d2fd2 100644
--- a/pkg/sentry/fsimpl/host/connected_endpoint_refs.go
+++ b/pkg/sentry/fsimpl/host/connected_endpoint_refs.go
@@ -7,9 +7,15 @@ import (
"gvisor.dev/gvisor/pkg/refsvfs2"
)
-// ownerType is used to customize logging. Note that we use a pointer to T so
-// that we do not copy the entire object when passed as a format parameter.
-var ConnectedEndpointownerType *ConnectedEndpoint
+// enableLogging indicates whether reference-related events should be logged (with
+// stack traces). This is false by default and should only be set to true for
+// debugging purposes, as it can generate an extremely large amount of output
+// and drastically degrade performance.
+const ConnectedEndpointenableLogging = false
+
+// obj is used to customize logging. Note that we use a pointer to T so that
+// we do not copy the entire object when passed as a format parameter.
+var ConnectedEndpointobj *ConnectedEndpoint
// Refs implements refs.RefCounter. It keeps a reference count using atomic
// operations and calls the destructor when the count reaches zero.
@@ -29,16 +35,24 @@ type ConnectedEndpointRefs struct {
refCount int64
}
-// EnableLeakCheck enables reference leak checking on r.
-func (r *ConnectedEndpointRefs) EnableLeakCheck() {
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Register(r, fmt.Sprintf("%T", ConnectedEndpointownerType))
- }
+// RefType implements refsvfs2.CheckedObject.RefType.
+func (r *ConnectedEndpointRefs) RefType() string {
+ return fmt.Sprintf("%T", ConnectedEndpointobj)[1:]
}
// LeakMessage implements refsvfs2.CheckedObject.LeakMessage.
func (r *ConnectedEndpointRefs) LeakMessage() string {
- return fmt.Sprintf("%T %p: reference count of %d instead of 0", ConnectedEndpointownerType, r, r.ReadRefs())
+ return fmt.Sprintf("[%s %p] reference count of %d instead of 0", r.RefType(), r, r.ReadRefs())
+}
+
+// LogRefs implements refsvfs2.CheckedObject.LogRefs.
+func (r *ConnectedEndpointRefs) LogRefs() bool {
+ return ConnectedEndpointenableLogging
+}
+
+// EnableLeakCheck enables reference leak checking on r.
+func (r *ConnectedEndpointRefs) EnableLeakCheck() {
+ refsvfs2.Register(r)
}
// ReadRefs returns the current number of references. The returned count is
@@ -52,8 +66,10 @@ func (r *ConnectedEndpointRefs) ReadRefs() int64 {
//
//go:nosplit
func (r *ConnectedEndpointRefs) IncRef() {
- if v := atomic.AddInt64(&r.refCount, 1); v <= 0 {
- panic(fmt.Sprintf("Incrementing non-positive count %p on %T", r, ConnectedEndpointownerType))
+ v := atomic.AddInt64(&r.refCount, 1)
+ refsvfs2.LogIncRef(r, v+1)
+ if v <= 0 {
+ panic(fmt.Sprintf("Incrementing non-positive count %p on %s", r, r.RefType()))
}
}
@@ -66,14 +82,14 @@ func (r *ConnectedEndpointRefs) IncRef() {
//go:nosplit
func (r *ConnectedEndpointRefs) TryIncRef() bool {
const speculativeRef = 1 << 32
- v := atomic.AddInt64(&r.refCount, speculativeRef)
- if int32(v) < 0 {
+ if v := atomic.AddInt64(&r.refCount, speculativeRef); int32(v) < 0 {
atomic.AddInt64(&r.refCount, -speculativeRef)
return false
}
- atomic.AddInt64(&r.refCount, -speculativeRef+1)
+ v := atomic.AddInt64(&r.refCount, -speculativeRef+1)
+ refsvfs2.LogTryIncRef(r, v+1)
return true
}
@@ -90,14 +106,14 @@ func (r *ConnectedEndpointRefs) TryIncRef() bool {
//
//go:nosplit
func (r *ConnectedEndpointRefs) DecRef(destroy func()) {
- switch v := atomic.AddInt64(&r.refCount, -1); {
+ v := atomic.AddInt64(&r.refCount, -1)
+ refsvfs2.LogDecRef(r, v+1)
+ switch {
case v < -1:
- panic(fmt.Sprintf("Decrementing non-positive ref count %p, owned by %T", r, ConnectedEndpointownerType))
+ panic(fmt.Sprintf("Decrementing non-positive ref count %p, owned by %s", r, r.RefType()))
case v == -1:
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Unregister(r, fmt.Sprintf("%T", ConnectedEndpointownerType))
- }
+ refsvfs2.Unregister(r)
if destroy != nil {
destroy()
@@ -106,7 +122,7 @@ func (r *ConnectedEndpointRefs) DecRef(destroy func()) {
}
func (r *ConnectedEndpointRefs) afterLoad() {
- if refsvfs2.LeakCheckEnabled() && r.ReadRefs() > 0 {
+ if r.ReadRefs() > 0 {
r.EnableLeakCheck()
}
}
diff --git a/pkg/sentry/fsimpl/host/inode_refs.go b/pkg/sentry/fsimpl/host/inode_refs.go
index 3504cc603..ef2f56522 100644
--- a/pkg/sentry/fsimpl/host/inode_refs.go
+++ b/pkg/sentry/fsimpl/host/inode_refs.go
@@ -7,9 +7,15 @@ import (
"gvisor.dev/gvisor/pkg/refsvfs2"
)
-// ownerType is used to customize logging. Note that we use a pointer to T so
-// that we do not copy the entire object when passed as a format parameter.
-var inodeownerType *inode
+// enableLogging indicates whether reference-related events should be logged (with
+// stack traces). This is false by default and should only be set to true for
+// debugging purposes, as it can generate an extremely large amount of output
+// and drastically degrade performance.
+const inodeenableLogging = false
+
+// obj is used to customize logging. Note that we use a pointer to T so that
+// we do not copy the entire object when passed as a format parameter.
+var inodeobj *inode
// Refs implements refs.RefCounter. It keeps a reference count using atomic
// operations and calls the destructor when the count reaches zero.
@@ -29,16 +35,24 @@ type inodeRefs struct {
refCount int64
}
-// EnableLeakCheck enables reference leak checking on r.
-func (r *inodeRefs) EnableLeakCheck() {
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Register(r, fmt.Sprintf("%T", inodeownerType))
- }
+// RefType implements refsvfs2.CheckedObject.RefType.
+func (r *inodeRefs) RefType() string {
+ return fmt.Sprintf("%T", inodeobj)[1:]
}
// LeakMessage implements refsvfs2.CheckedObject.LeakMessage.
func (r *inodeRefs) LeakMessage() string {
- return fmt.Sprintf("%T %p: reference count of %d instead of 0", inodeownerType, r, r.ReadRefs())
+ return fmt.Sprintf("[%s %p] reference count of %d instead of 0", r.RefType(), r, r.ReadRefs())
+}
+
+// LogRefs implements refsvfs2.CheckedObject.LogRefs.
+func (r *inodeRefs) LogRefs() bool {
+ return inodeenableLogging
+}
+
+// EnableLeakCheck enables reference leak checking on r.
+func (r *inodeRefs) EnableLeakCheck() {
+ refsvfs2.Register(r)
}
// ReadRefs returns the current number of references. The returned count is
@@ -52,8 +66,10 @@ func (r *inodeRefs) ReadRefs() int64 {
//
//go:nosplit
func (r *inodeRefs) IncRef() {
- if v := atomic.AddInt64(&r.refCount, 1); v <= 0 {
- panic(fmt.Sprintf("Incrementing non-positive count %p on %T", r, inodeownerType))
+ v := atomic.AddInt64(&r.refCount, 1)
+ refsvfs2.LogIncRef(r, v+1)
+ if v <= 0 {
+ panic(fmt.Sprintf("Incrementing non-positive count %p on %s", r, r.RefType()))
}
}
@@ -66,14 +82,14 @@ func (r *inodeRefs) IncRef() {
//go:nosplit
func (r *inodeRefs) TryIncRef() bool {
const speculativeRef = 1 << 32
- v := atomic.AddInt64(&r.refCount, speculativeRef)
- if int32(v) < 0 {
+ if v := atomic.AddInt64(&r.refCount, speculativeRef); int32(v) < 0 {
atomic.AddInt64(&r.refCount, -speculativeRef)
return false
}
- atomic.AddInt64(&r.refCount, -speculativeRef+1)
+ v := atomic.AddInt64(&r.refCount, -speculativeRef+1)
+ refsvfs2.LogTryIncRef(r, v+1)
return true
}
@@ -90,14 +106,14 @@ func (r *inodeRefs) TryIncRef() bool {
//
//go:nosplit
func (r *inodeRefs) DecRef(destroy func()) {
- switch v := atomic.AddInt64(&r.refCount, -1); {
+ v := atomic.AddInt64(&r.refCount, -1)
+ refsvfs2.LogDecRef(r, v+1)
+ switch {
case v < -1:
- panic(fmt.Sprintf("Decrementing non-positive ref count %p, owned by %T", r, inodeownerType))
+ panic(fmt.Sprintf("Decrementing non-positive ref count %p, owned by %s", r, r.RefType()))
case v == -1:
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Unregister(r, fmt.Sprintf("%T", inodeownerType))
- }
+ refsvfs2.Unregister(r)
if destroy != nil {
destroy()
@@ -106,7 +122,7 @@ func (r *inodeRefs) DecRef(destroy func()) {
}
func (r *inodeRefs) afterLoad() {
- if refsvfs2.LeakCheckEnabled() && r.ReadRefs() > 0 {
+ if r.ReadRefs() > 0 {
r.EnableLeakCheck()
}
}
diff --git a/pkg/sentry/fsimpl/kernfs/static_directory_refs.go b/pkg/sentry/fsimpl/kernfs/static_directory_refs.go
index 9472a96b9..d86cf76fe 100644
--- a/pkg/sentry/fsimpl/kernfs/static_directory_refs.go
+++ b/pkg/sentry/fsimpl/kernfs/static_directory_refs.go
@@ -7,9 +7,15 @@ import (
"gvisor.dev/gvisor/pkg/refsvfs2"
)
-// ownerType is used to customize logging. Note that we use a pointer to T so
-// that we do not copy the entire object when passed as a format parameter.
-var StaticDirectoryownerType *StaticDirectory
+// enableLogging indicates whether reference-related events should be logged (with
+// stack traces). This is false by default and should only be set to true for
+// debugging purposes, as it can generate an extremely large amount of output
+// and drastically degrade performance.
+const StaticDirectoryenableLogging = false
+
+// obj is used to customize logging. Note that we use a pointer to T so that
+// we do not copy the entire object when passed as a format parameter.
+var StaticDirectoryobj *StaticDirectory
// Refs implements refs.RefCounter. It keeps a reference count using atomic
// operations and calls the destructor when the count reaches zero.
@@ -29,16 +35,24 @@ type StaticDirectoryRefs struct {
refCount int64
}
-// EnableLeakCheck enables reference leak checking on r.
-func (r *StaticDirectoryRefs) EnableLeakCheck() {
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Register(r, fmt.Sprintf("%T", StaticDirectoryownerType))
- }
+// RefType implements refsvfs2.CheckedObject.RefType.
+func (r *StaticDirectoryRefs) RefType() string {
+ return fmt.Sprintf("%T", StaticDirectoryobj)[1:]
}
// LeakMessage implements refsvfs2.CheckedObject.LeakMessage.
func (r *StaticDirectoryRefs) LeakMessage() string {
- return fmt.Sprintf("%T %p: reference count of %d instead of 0", StaticDirectoryownerType, r, r.ReadRefs())
+ return fmt.Sprintf("[%s %p] reference count of %d instead of 0", r.RefType(), r, r.ReadRefs())
+}
+
+// LogRefs implements refsvfs2.CheckedObject.LogRefs.
+func (r *StaticDirectoryRefs) LogRefs() bool {
+ return StaticDirectoryenableLogging
+}
+
+// EnableLeakCheck enables reference leak checking on r.
+func (r *StaticDirectoryRefs) EnableLeakCheck() {
+ refsvfs2.Register(r)
}
// ReadRefs returns the current number of references. The returned count is
@@ -52,8 +66,10 @@ func (r *StaticDirectoryRefs) ReadRefs() int64 {
//
//go:nosplit
func (r *StaticDirectoryRefs) IncRef() {
- if v := atomic.AddInt64(&r.refCount, 1); v <= 0 {
- panic(fmt.Sprintf("Incrementing non-positive count %p on %T", r, StaticDirectoryownerType))
+ v := atomic.AddInt64(&r.refCount, 1)
+ refsvfs2.LogIncRef(r, v+1)
+ if v <= 0 {
+ panic(fmt.Sprintf("Incrementing non-positive count %p on %s", r, r.RefType()))
}
}
@@ -66,14 +82,14 @@ func (r *StaticDirectoryRefs) IncRef() {
//go:nosplit
func (r *StaticDirectoryRefs) TryIncRef() bool {
const speculativeRef = 1 << 32
- v := atomic.AddInt64(&r.refCount, speculativeRef)
- if int32(v) < 0 {
+ if v := atomic.AddInt64(&r.refCount, speculativeRef); int32(v) < 0 {
atomic.AddInt64(&r.refCount, -speculativeRef)
return false
}
- atomic.AddInt64(&r.refCount, -speculativeRef+1)
+ v := atomic.AddInt64(&r.refCount, -speculativeRef+1)
+ refsvfs2.LogTryIncRef(r, v+1)
return true
}
@@ -90,14 +106,14 @@ func (r *StaticDirectoryRefs) TryIncRef() bool {
//
//go:nosplit
func (r *StaticDirectoryRefs) DecRef(destroy func()) {
- switch v := atomic.AddInt64(&r.refCount, -1); {
+ v := atomic.AddInt64(&r.refCount, -1)
+ refsvfs2.LogDecRef(r, v+1)
+ switch {
case v < -1:
- panic(fmt.Sprintf("Decrementing non-positive ref count %p, owned by %T", r, StaticDirectoryownerType))
+ panic(fmt.Sprintf("Decrementing non-positive ref count %p, owned by %s", r, r.RefType()))
case v == -1:
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Unregister(r, fmt.Sprintf("%T", StaticDirectoryownerType))
- }
+ refsvfs2.Unregister(r)
if destroy != nil {
destroy()
@@ -106,7 +122,7 @@ func (r *StaticDirectoryRefs) DecRef(destroy func()) {
}
func (r *StaticDirectoryRefs) afterLoad() {
- if refsvfs2.LeakCheckEnabled() && r.ReadRefs() > 0 {
+ if r.ReadRefs() > 0 {
r.EnableLeakCheck()
}
}
diff --git a/pkg/sentry/fsimpl/kernfs/synthetic_directory_refs.go b/pkg/sentry/fsimpl/kernfs/synthetic_directory_refs.go
index 7c4fde369..446837e30 100644
--- a/pkg/sentry/fsimpl/kernfs/synthetic_directory_refs.go
+++ b/pkg/sentry/fsimpl/kernfs/synthetic_directory_refs.go
@@ -7,9 +7,15 @@ import (
"gvisor.dev/gvisor/pkg/refsvfs2"
)
-// ownerType is used to customize logging. Note that we use a pointer to T so
-// that we do not copy the entire object when passed as a format parameter.
-var syntheticDirectoryownerType *syntheticDirectory
+// enableLogging indicates whether reference-related events should be logged (with
+// stack traces). This is false by default and should only be set to true for
+// debugging purposes, as it can generate an extremely large amount of output
+// and drastically degrade performance.
+const syntheticDirectoryenableLogging = false
+
+// obj is used to customize logging. Note that we use a pointer to T so that
+// we do not copy the entire object when passed as a format parameter.
+var syntheticDirectoryobj *syntheticDirectory
// Refs implements refs.RefCounter. It keeps a reference count using atomic
// operations and calls the destructor when the count reaches zero.
@@ -29,16 +35,24 @@ type syntheticDirectoryRefs struct {
refCount int64
}
-// EnableLeakCheck enables reference leak checking on r.
-func (r *syntheticDirectoryRefs) EnableLeakCheck() {
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Register(r, fmt.Sprintf("%T", syntheticDirectoryownerType))
- }
+// RefType implements refsvfs2.CheckedObject.RefType.
+func (r *syntheticDirectoryRefs) RefType() string {
+ return fmt.Sprintf("%T", syntheticDirectoryobj)[1:]
}
// LeakMessage implements refsvfs2.CheckedObject.LeakMessage.
func (r *syntheticDirectoryRefs) LeakMessage() string {
- return fmt.Sprintf("%T %p: reference count of %d instead of 0", syntheticDirectoryownerType, r, r.ReadRefs())
+ return fmt.Sprintf("[%s %p] reference count of %d instead of 0", r.RefType(), r, r.ReadRefs())
+}
+
+// LogRefs implements refsvfs2.CheckedObject.LogRefs.
+func (r *syntheticDirectoryRefs) LogRefs() bool {
+ return syntheticDirectoryenableLogging
+}
+
+// EnableLeakCheck enables reference leak checking on r.
+func (r *syntheticDirectoryRefs) EnableLeakCheck() {
+ refsvfs2.Register(r)
}
// ReadRefs returns the current number of references. The returned count is
@@ -52,8 +66,10 @@ func (r *syntheticDirectoryRefs) ReadRefs() int64 {
//
//go:nosplit
func (r *syntheticDirectoryRefs) IncRef() {
- if v := atomic.AddInt64(&r.refCount, 1); v <= 0 {
- panic(fmt.Sprintf("Incrementing non-positive count %p on %T", r, syntheticDirectoryownerType))
+ v := atomic.AddInt64(&r.refCount, 1)
+ refsvfs2.LogIncRef(r, v+1)
+ if v <= 0 {
+ panic(fmt.Sprintf("Incrementing non-positive count %p on %s", r, r.RefType()))
}
}
@@ -66,14 +82,14 @@ func (r *syntheticDirectoryRefs) IncRef() {
//go:nosplit
func (r *syntheticDirectoryRefs) TryIncRef() bool {
const speculativeRef = 1 << 32
- v := atomic.AddInt64(&r.refCount, speculativeRef)
- if int32(v) < 0 {
+ if v := atomic.AddInt64(&r.refCount, speculativeRef); int32(v) < 0 {
atomic.AddInt64(&r.refCount, -speculativeRef)
return false
}
- atomic.AddInt64(&r.refCount, -speculativeRef+1)
+ v := atomic.AddInt64(&r.refCount, -speculativeRef+1)
+ refsvfs2.LogTryIncRef(r, v+1)
return true
}
@@ -90,14 +106,14 @@ func (r *syntheticDirectoryRefs) TryIncRef() bool {
//
//go:nosplit
func (r *syntheticDirectoryRefs) DecRef(destroy func()) {
- switch v := atomic.AddInt64(&r.refCount, -1); {
+ v := atomic.AddInt64(&r.refCount, -1)
+ refsvfs2.LogDecRef(r, v+1)
+ switch {
case v < -1:
- panic(fmt.Sprintf("Decrementing non-positive ref count %p, owned by %T", r, syntheticDirectoryownerType))
+ panic(fmt.Sprintf("Decrementing non-positive ref count %p, owned by %s", r, r.RefType()))
case v == -1:
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Unregister(r, fmt.Sprintf("%T", syntheticDirectoryownerType))
- }
+ refsvfs2.Unregister(r)
if destroy != nil {
destroy()
@@ -106,7 +122,7 @@ func (r *syntheticDirectoryRefs) DecRef(destroy func()) {
}
func (r *syntheticDirectoryRefs) afterLoad() {
- if refsvfs2.LeakCheckEnabled() && r.ReadRefs() > 0 {
+ if r.ReadRefs() > 0 {
r.EnableLeakCheck()
}
}
diff --git a/pkg/sentry/fsimpl/overlay/overlay.go b/pkg/sentry/fsimpl/overlay/overlay.go
index c812f0a70..f6c58f2e7 100644
--- a/pkg/sentry/fsimpl/overlay/overlay.go
+++ b/pkg/sentry/fsimpl/overlay/overlay.go
@@ -505,9 +505,7 @@ func (fs *filesystem) newDentry() *dentry {
}
d.lowerVDs = d.inlineLowerVDs[:0]
d.vfsd.Init(d)
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Register(d, "overlay.dentry")
- }
+ refsvfs2.Register(d)
return d
}
@@ -515,17 +513,19 @@ func (fs *filesystem) newDentry() *dentry {
func (d *dentry) IncRef() {
// d.refs may be 0 if d.fs.renameMu is locked, which serializes against
// d.checkDropLocked().
- atomic.AddInt64(&d.refs, 1)
+ r := atomic.AddInt64(&d.refs, 1)
+ refsvfs2.LogIncRef(d, r)
}
// TryIncRef implements vfs.DentryImpl.TryIncRef.
func (d *dentry) TryIncRef() bool {
for {
- refs := atomic.LoadInt64(&d.refs)
- if refs <= 0 {
+ r := atomic.LoadInt64(&d.refs)
+ if r <= 0 {
return false
}
- if atomic.CompareAndSwapInt64(&d.refs, refs, refs+1) {
+ if atomic.CompareAndSwapInt64(&d.refs, r, r+1) {
+ refsvfs2.LogTryIncRef(d, r+1)
return true
}
}
@@ -533,15 +533,27 @@ func (d *dentry) TryIncRef() bool {
// DecRef implements vfs.DentryImpl.DecRef.
func (d *dentry) DecRef(ctx context.Context) {
- if refs := atomic.AddInt64(&d.refs, -1); refs == 0 {
+ r := atomic.AddInt64(&d.refs, -1)
+ refsvfs2.LogDecRef(d, r)
+ if r == 0 {
d.fs.renameMu.Lock()
d.checkDropLocked(ctx)
d.fs.renameMu.Unlock()
- } else if refs < 0 {
+ } else if r < 0 {
panic("overlay.dentry.DecRef() called without holding a reference")
}
}
+func (d *dentry) decRefLocked(ctx context.Context) {
+ r := atomic.AddInt64(&d.refs, -1)
+ refsvfs2.LogDecRef(d, r)
+ if r == 0 {
+ d.checkDropLocked(ctx)
+ } else if r < 0 {
+ panic("overlay.dentry.decRefLocked() called without holding a reference")
+ }
+}
+
// checkDropLocked should be called after d's reference count becomes 0 or it
// becomes deleted.
//
@@ -601,15 +613,14 @@ func (d *dentry) destroyLocked(ctx context.Context) {
d.parent.dirMu.Unlock()
// Drop the reference held by d on its parent without recursively
// locking d.fs.renameMu.
- if refs := atomic.AddInt64(&d.parent.refs, -1); refs == 0 {
- d.parent.checkDropLocked(ctx)
- } else if refs < 0 {
- panic("overlay.dentry.DecRef() called without holding a reference")
- }
- }
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Unregister(d, "overlay.dentry")
+ d.parent.decRefLocked(ctx)
}
+ refsvfs2.Unregister(d)
+}
+
+// RefType implements refsvfs2.CheckedObject.Type.
+func (d *dentry) RefType() string {
+ return "overlay.dentry"
}
// LeakMessage implements refsvfs2.CheckedObject.LeakMessage.
@@ -617,6 +628,14 @@ func (d *dentry) LeakMessage() string {
return fmt.Sprintf("[overlay.dentry %p] reference count of %d instead of -1", d, atomic.LoadInt64(&d.refs))
}
+// LogRefs implements refsvfs2.CheckedObject.LogRefs.
+//
+// This should only be set to true for debugging purposes, as it can generate an
+// extremely large amount of output and drastically degrade performance.
+func (d *dentry) LogRefs() bool {
+ return false
+}
+
// InotifyWithParent implements vfs.DentryImpl.InotifyWithParent.
func (d *dentry) InotifyWithParent(ctx context.Context, events uint32, cookie uint32, et vfs.EventType) {
if d.isDir() {
diff --git a/pkg/sentry/fsimpl/overlay/save_restore.go b/pkg/sentry/fsimpl/overlay/save_restore.go
index 054e17b17..54809f16c 100644
--- a/pkg/sentry/fsimpl/overlay/save_restore.go
+++ b/pkg/sentry/fsimpl/overlay/save_restore.go
@@ -21,7 +21,7 @@ import (
)
func (d *dentry) afterLoad() {
- if refsvfs2.LeakCheckEnabled() && atomic.LoadInt64(&d.refs) != -1 {
- refsvfs2.Register(d, "overlay.dentry")
+ if atomic.LoadInt64(&d.refs) != -1 {
+ refsvfs2.Register(d)
}
}
diff --git a/pkg/sentry/fsimpl/proc/fd_dir_inode_refs.go b/pkg/sentry/fsimpl/proc/fd_dir_inode_refs.go
index 2f0dd126e..895298eb9 100644
--- a/pkg/sentry/fsimpl/proc/fd_dir_inode_refs.go
+++ b/pkg/sentry/fsimpl/proc/fd_dir_inode_refs.go
@@ -7,9 +7,15 @@ import (
"gvisor.dev/gvisor/pkg/refsvfs2"
)
-// ownerType is used to customize logging. Note that we use a pointer to T so
-// that we do not copy the entire object when passed as a format parameter.
-var fdDirInodeownerType *fdDirInode
+// enableLogging indicates whether reference-related events should be logged (with
+// stack traces). This is false by default and should only be set to true for
+// debugging purposes, as it can generate an extremely large amount of output
+// and drastically degrade performance.
+const fdDirInodeenableLogging = false
+
+// obj is used to customize logging. Note that we use a pointer to T so that
+// we do not copy the entire object when passed as a format parameter.
+var fdDirInodeobj *fdDirInode
// Refs implements refs.RefCounter. It keeps a reference count using atomic
// operations and calls the destructor when the count reaches zero.
@@ -29,16 +35,24 @@ type fdDirInodeRefs struct {
refCount int64
}
-// EnableLeakCheck enables reference leak checking on r.
-func (r *fdDirInodeRefs) EnableLeakCheck() {
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Register(r, fmt.Sprintf("%T", fdDirInodeownerType))
- }
+// RefType implements refsvfs2.CheckedObject.RefType.
+func (r *fdDirInodeRefs) RefType() string {
+ return fmt.Sprintf("%T", fdDirInodeobj)[1:]
}
// LeakMessage implements refsvfs2.CheckedObject.LeakMessage.
func (r *fdDirInodeRefs) LeakMessage() string {
- return fmt.Sprintf("%T %p: reference count of %d instead of 0", fdDirInodeownerType, r, r.ReadRefs())
+ return fmt.Sprintf("[%s %p] reference count of %d instead of 0", r.RefType(), r, r.ReadRefs())
+}
+
+// LogRefs implements refsvfs2.CheckedObject.LogRefs.
+func (r *fdDirInodeRefs) LogRefs() bool {
+ return fdDirInodeenableLogging
+}
+
+// EnableLeakCheck enables reference leak checking on r.
+func (r *fdDirInodeRefs) EnableLeakCheck() {
+ refsvfs2.Register(r)
}
// ReadRefs returns the current number of references. The returned count is
@@ -52,8 +66,10 @@ func (r *fdDirInodeRefs) ReadRefs() int64 {
//
//go:nosplit
func (r *fdDirInodeRefs) IncRef() {
- if v := atomic.AddInt64(&r.refCount, 1); v <= 0 {
- panic(fmt.Sprintf("Incrementing non-positive count %p on %T", r, fdDirInodeownerType))
+ v := atomic.AddInt64(&r.refCount, 1)
+ refsvfs2.LogIncRef(r, v+1)
+ if v <= 0 {
+ panic(fmt.Sprintf("Incrementing non-positive count %p on %s", r, r.RefType()))
}
}
@@ -66,14 +82,14 @@ func (r *fdDirInodeRefs) IncRef() {
//go:nosplit
func (r *fdDirInodeRefs) TryIncRef() bool {
const speculativeRef = 1 << 32
- v := atomic.AddInt64(&r.refCount, speculativeRef)
- if int32(v) < 0 {
+ if v := atomic.AddInt64(&r.refCount, speculativeRef); int32(v) < 0 {
atomic.AddInt64(&r.refCount, -speculativeRef)
return false
}
- atomic.AddInt64(&r.refCount, -speculativeRef+1)
+ v := atomic.AddInt64(&r.refCount, -speculativeRef+1)
+ refsvfs2.LogTryIncRef(r, v+1)
return true
}
@@ -90,14 +106,14 @@ func (r *fdDirInodeRefs) TryIncRef() bool {
//
//go:nosplit
func (r *fdDirInodeRefs) DecRef(destroy func()) {
- switch v := atomic.AddInt64(&r.refCount, -1); {
+ v := atomic.AddInt64(&r.refCount, -1)
+ refsvfs2.LogDecRef(r, v+1)
+ switch {
case v < -1:
- panic(fmt.Sprintf("Decrementing non-positive ref count %p, owned by %T", r, fdDirInodeownerType))
+ panic(fmt.Sprintf("Decrementing non-positive ref count %p, owned by %s", r, r.RefType()))
case v == -1:
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Unregister(r, fmt.Sprintf("%T", fdDirInodeownerType))
- }
+ refsvfs2.Unregister(r)
if destroy != nil {
destroy()
@@ -106,7 +122,7 @@ func (r *fdDirInodeRefs) DecRef(destroy func()) {
}
func (r *fdDirInodeRefs) afterLoad() {
- if refsvfs2.LeakCheckEnabled() && r.ReadRefs() > 0 {
+ if r.ReadRefs() > 0 {
r.EnableLeakCheck()
}
}
diff --git a/pkg/sentry/fsimpl/proc/fd_info_dir_inode_refs.go b/pkg/sentry/fsimpl/proc/fd_info_dir_inode_refs.go
index 2065c97b4..8e3458485 100644
--- a/pkg/sentry/fsimpl/proc/fd_info_dir_inode_refs.go
+++ b/pkg/sentry/fsimpl/proc/fd_info_dir_inode_refs.go
@@ -7,9 +7,15 @@ import (
"gvisor.dev/gvisor/pkg/refsvfs2"
)
-// ownerType is used to customize logging. Note that we use a pointer to T so
-// that we do not copy the entire object when passed as a format parameter.
-var fdInfoDirInodeownerType *fdInfoDirInode
+// enableLogging indicates whether reference-related events should be logged (with
+// stack traces). This is false by default and should only be set to true for
+// debugging purposes, as it can generate an extremely large amount of output
+// and drastically degrade performance.
+const fdInfoDirInodeenableLogging = false
+
+// obj is used to customize logging. Note that we use a pointer to T so that
+// we do not copy the entire object when passed as a format parameter.
+var fdInfoDirInodeobj *fdInfoDirInode
// Refs implements refs.RefCounter. It keeps a reference count using atomic
// operations and calls the destructor when the count reaches zero.
@@ -29,16 +35,24 @@ type fdInfoDirInodeRefs struct {
refCount int64
}
-// EnableLeakCheck enables reference leak checking on r.
-func (r *fdInfoDirInodeRefs) EnableLeakCheck() {
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Register(r, fmt.Sprintf("%T", fdInfoDirInodeownerType))
- }
+// RefType implements refsvfs2.CheckedObject.RefType.
+func (r *fdInfoDirInodeRefs) RefType() string {
+ return fmt.Sprintf("%T", fdInfoDirInodeobj)[1:]
}
// LeakMessage implements refsvfs2.CheckedObject.LeakMessage.
func (r *fdInfoDirInodeRefs) LeakMessage() string {
- return fmt.Sprintf("%T %p: reference count of %d instead of 0", fdInfoDirInodeownerType, r, r.ReadRefs())
+ return fmt.Sprintf("[%s %p] reference count of %d instead of 0", r.RefType(), r, r.ReadRefs())
+}
+
+// LogRefs implements refsvfs2.CheckedObject.LogRefs.
+func (r *fdInfoDirInodeRefs) LogRefs() bool {
+ return fdInfoDirInodeenableLogging
+}
+
+// EnableLeakCheck enables reference leak checking on r.
+func (r *fdInfoDirInodeRefs) EnableLeakCheck() {
+ refsvfs2.Register(r)
}
// ReadRefs returns the current number of references. The returned count is
@@ -52,8 +66,10 @@ func (r *fdInfoDirInodeRefs) ReadRefs() int64 {
//
//go:nosplit
func (r *fdInfoDirInodeRefs) IncRef() {
- if v := atomic.AddInt64(&r.refCount, 1); v <= 0 {
- panic(fmt.Sprintf("Incrementing non-positive count %p on %T", r, fdInfoDirInodeownerType))
+ v := atomic.AddInt64(&r.refCount, 1)
+ refsvfs2.LogIncRef(r, v+1)
+ if v <= 0 {
+ panic(fmt.Sprintf("Incrementing non-positive count %p on %s", r, r.RefType()))
}
}
@@ -66,14 +82,14 @@ func (r *fdInfoDirInodeRefs) IncRef() {
//go:nosplit
func (r *fdInfoDirInodeRefs) TryIncRef() bool {
const speculativeRef = 1 << 32
- v := atomic.AddInt64(&r.refCount, speculativeRef)
- if int32(v) < 0 {
+ if v := atomic.AddInt64(&r.refCount, speculativeRef); int32(v) < 0 {
atomic.AddInt64(&r.refCount, -speculativeRef)
return false
}
- atomic.AddInt64(&r.refCount, -speculativeRef+1)
+ v := atomic.AddInt64(&r.refCount, -speculativeRef+1)
+ refsvfs2.LogTryIncRef(r, v+1)
return true
}
@@ -90,14 +106,14 @@ func (r *fdInfoDirInodeRefs) TryIncRef() bool {
//
//go:nosplit
func (r *fdInfoDirInodeRefs) DecRef(destroy func()) {
- switch v := atomic.AddInt64(&r.refCount, -1); {
+ v := atomic.AddInt64(&r.refCount, -1)
+ refsvfs2.LogDecRef(r, v+1)
+ switch {
case v < -1:
- panic(fmt.Sprintf("Decrementing non-positive ref count %p, owned by %T", r, fdInfoDirInodeownerType))
+ panic(fmt.Sprintf("Decrementing non-positive ref count %p, owned by %s", r, r.RefType()))
case v == -1:
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Unregister(r, fmt.Sprintf("%T", fdInfoDirInodeownerType))
- }
+ refsvfs2.Unregister(r)
if destroy != nil {
destroy()
@@ -106,7 +122,7 @@ func (r *fdInfoDirInodeRefs) DecRef(destroy func()) {
}
func (r *fdInfoDirInodeRefs) afterLoad() {
- if refsvfs2.LeakCheckEnabled() && r.ReadRefs() > 0 {
+ if r.ReadRefs() > 0 {
r.EnableLeakCheck()
}
}
diff --git a/pkg/sentry/fsimpl/proc/subtasks_inode_refs.go b/pkg/sentry/fsimpl/proc/subtasks_inode_refs.go
index c4c0baf31..c609e618a 100644
--- a/pkg/sentry/fsimpl/proc/subtasks_inode_refs.go
+++ b/pkg/sentry/fsimpl/proc/subtasks_inode_refs.go
@@ -7,9 +7,15 @@ import (
"gvisor.dev/gvisor/pkg/refsvfs2"
)
-// ownerType is used to customize logging. Note that we use a pointer to T so
-// that we do not copy the entire object when passed as a format parameter.
-var subtasksInodeownerType *subtasksInode
+// enableLogging indicates whether reference-related events should be logged (with
+// stack traces). This is false by default and should only be set to true for
+// debugging purposes, as it can generate an extremely large amount of output
+// and drastically degrade performance.
+const subtasksInodeenableLogging = false
+
+// obj is used to customize logging. Note that we use a pointer to T so that
+// we do not copy the entire object when passed as a format parameter.
+var subtasksInodeobj *subtasksInode
// Refs implements refs.RefCounter. It keeps a reference count using atomic
// operations and calls the destructor when the count reaches zero.
@@ -29,16 +35,24 @@ type subtasksInodeRefs struct {
refCount int64
}
-// EnableLeakCheck enables reference leak checking on r.
-func (r *subtasksInodeRefs) EnableLeakCheck() {
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Register(r, fmt.Sprintf("%T", subtasksInodeownerType))
- }
+// RefType implements refsvfs2.CheckedObject.RefType.
+func (r *subtasksInodeRefs) RefType() string {
+ return fmt.Sprintf("%T", subtasksInodeobj)[1:]
}
// LeakMessage implements refsvfs2.CheckedObject.LeakMessage.
func (r *subtasksInodeRefs) LeakMessage() string {
- return fmt.Sprintf("%T %p: reference count of %d instead of 0", subtasksInodeownerType, r, r.ReadRefs())
+ return fmt.Sprintf("[%s %p] reference count of %d instead of 0", r.RefType(), r, r.ReadRefs())
+}
+
+// LogRefs implements refsvfs2.CheckedObject.LogRefs.
+func (r *subtasksInodeRefs) LogRefs() bool {
+ return subtasksInodeenableLogging
+}
+
+// EnableLeakCheck enables reference leak checking on r.
+func (r *subtasksInodeRefs) EnableLeakCheck() {
+ refsvfs2.Register(r)
}
// ReadRefs returns the current number of references. The returned count is
@@ -52,8 +66,10 @@ func (r *subtasksInodeRefs) ReadRefs() int64 {
//
//go:nosplit
func (r *subtasksInodeRefs) IncRef() {
- if v := atomic.AddInt64(&r.refCount, 1); v <= 0 {
- panic(fmt.Sprintf("Incrementing non-positive count %p on %T", r, subtasksInodeownerType))
+ v := atomic.AddInt64(&r.refCount, 1)
+ refsvfs2.LogIncRef(r, v+1)
+ if v <= 0 {
+ panic(fmt.Sprintf("Incrementing non-positive count %p on %s", r, r.RefType()))
}
}
@@ -66,14 +82,14 @@ func (r *subtasksInodeRefs) IncRef() {
//go:nosplit
func (r *subtasksInodeRefs) TryIncRef() bool {
const speculativeRef = 1 << 32
- v := atomic.AddInt64(&r.refCount, speculativeRef)
- if int32(v) < 0 {
+ if v := atomic.AddInt64(&r.refCount, speculativeRef); int32(v) < 0 {
atomic.AddInt64(&r.refCount, -speculativeRef)
return false
}
- atomic.AddInt64(&r.refCount, -speculativeRef+1)
+ v := atomic.AddInt64(&r.refCount, -speculativeRef+1)
+ refsvfs2.LogTryIncRef(r, v+1)
return true
}
@@ -90,14 +106,14 @@ func (r *subtasksInodeRefs) TryIncRef() bool {
//
//go:nosplit
func (r *subtasksInodeRefs) DecRef(destroy func()) {
- switch v := atomic.AddInt64(&r.refCount, -1); {
+ v := atomic.AddInt64(&r.refCount, -1)
+ refsvfs2.LogDecRef(r, v+1)
+ switch {
case v < -1:
- panic(fmt.Sprintf("Decrementing non-positive ref count %p, owned by %T", r, subtasksInodeownerType))
+ panic(fmt.Sprintf("Decrementing non-positive ref count %p, owned by %s", r, r.RefType()))
case v == -1:
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Unregister(r, fmt.Sprintf("%T", subtasksInodeownerType))
- }
+ refsvfs2.Unregister(r)
if destroy != nil {
destroy()
@@ -106,7 +122,7 @@ func (r *subtasksInodeRefs) DecRef(destroy func()) {
}
func (r *subtasksInodeRefs) afterLoad() {
- if refsvfs2.LeakCheckEnabled() && r.ReadRefs() > 0 {
+ if r.ReadRefs() > 0 {
r.EnableLeakCheck()
}
}
diff --git a/pkg/sentry/fsimpl/proc/task_inode_refs.go b/pkg/sentry/fsimpl/proc/task_inode_refs.go
index 67638f6ae..2ee58bd62 100644
--- a/pkg/sentry/fsimpl/proc/task_inode_refs.go
+++ b/pkg/sentry/fsimpl/proc/task_inode_refs.go
@@ -7,9 +7,15 @@ import (
"gvisor.dev/gvisor/pkg/refsvfs2"
)
-// ownerType is used to customize logging. Note that we use a pointer to T so
-// that we do not copy the entire object when passed as a format parameter.
-var taskInodeownerType *taskInode
+// enableLogging indicates whether reference-related events should be logged (with
+// stack traces). This is false by default and should only be set to true for
+// debugging purposes, as it can generate an extremely large amount of output
+// and drastically degrade performance.
+const taskInodeenableLogging = false
+
+// obj is used to customize logging. Note that we use a pointer to T so that
+// we do not copy the entire object when passed as a format parameter.
+var taskInodeobj *taskInode
// Refs implements refs.RefCounter. It keeps a reference count using atomic
// operations and calls the destructor when the count reaches zero.
@@ -29,16 +35,24 @@ type taskInodeRefs struct {
refCount int64
}
-// EnableLeakCheck enables reference leak checking on r.
-func (r *taskInodeRefs) EnableLeakCheck() {
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Register(r, fmt.Sprintf("%T", taskInodeownerType))
- }
+// RefType implements refsvfs2.CheckedObject.RefType.
+func (r *taskInodeRefs) RefType() string {
+ return fmt.Sprintf("%T", taskInodeobj)[1:]
}
// LeakMessage implements refsvfs2.CheckedObject.LeakMessage.
func (r *taskInodeRefs) LeakMessage() string {
- return fmt.Sprintf("%T %p: reference count of %d instead of 0", taskInodeownerType, r, r.ReadRefs())
+ return fmt.Sprintf("[%s %p] reference count of %d instead of 0", r.RefType(), r, r.ReadRefs())
+}
+
+// LogRefs implements refsvfs2.CheckedObject.LogRefs.
+func (r *taskInodeRefs) LogRefs() bool {
+ return taskInodeenableLogging
+}
+
+// EnableLeakCheck enables reference leak checking on r.
+func (r *taskInodeRefs) EnableLeakCheck() {
+ refsvfs2.Register(r)
}
// ReadRefs returns the current number of references. The returned count is
@@ -52,8 +66,10 @@ func (r *taskInodeRefs) ReadRefs() int64 {
//
//go:nosplit
func (r *taskInodeRefs) IncRef() {
- if v := atomic.AddInt64(&r.refCount, 1); v <= 0 {
- panic(fmt.Sprintf("Incrementing non-positive count %p on %T", r, taskInodeownerType))
+ v := atomic.AddInt64(&r.refCount, 1)
+ refsvfs2.LogIncRef(r, v+1)
+ if v <= 0 {
+ panic(fmt.Sprintf("Incrementing non-positive count %p on %s", r, r.RefType()))
}
}
@@ -66,14 +82,14 @@ func (r *taskInodeRefs) IncRef() {
//go:nosplit
func (r *taskInodeRefs) TryIncRef() bool {
const speculativeRef = 1 << 32
- v := atomic.AddInt64(&r.refCount, speculativeRef)
- if int32(v) < 0 {
+ if v := atomic.AddInt64(&r.refCount, speculativeRef); int32(v) < 0 {
atomic.AddInt64(&r.refCount, -speculativeRef)
return false
}
- atomic.AddInt64(&r.refCount, -speculativeRef+1)
+ v := atomic.AddInt64(&r.refCount, -speculativeRef+1)
+ refsvfs2.LogTryIncRef(r, v+1)
return true
}
@@ -90,14 +106,14 @@ func (r *taskInodeRefs) TryIncRef() bool {
//
//go:nosplit
func (r *taskInodeRefs) DecRef(destroy func()) {
- switch v := atomic.AddInt64(&r.refCount, -1); {
+ v := atomic.AddInt64(&r.refCount, -1)
+ refsvfs2.LogDecRef(r, v+1)
+ switch {
case v < -1:
- panic(fmt.Sprintf("Decrementing non-positive ref count %p, owned by %T", r, taskInodeownerType))
+ panic(fmt.Sprintf("Decrementing non-positive ref count %p, owned by %s", r, r.RefType()))
case v == -1:
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Unregister(r, fmt.Sprintf("%T", taskInodeownerType))
- }
+ refsvfs2.Unregister(r)
if destroy != nil {
destroy()
@@ -106,7 +122,7 @@ func (r *taskInodeRefs) DecRef(destroy func()) {
}
func (r *taskInodeRefs) afterLoad() {
- if refsvfs2.LeakCheckEnabled() && r.ReadRefs() > 0 {
+ if r.ReadRefs() > 0 {
r.EnableLeakCheck()
}
}
diff --git a/pkg/sentry/fsimpl/proc/tasks_inode_refs.go b/pkg/sentry/fsimpl/proc/tasks_inode_refs.go
index b882335d7..71dcaed9e 100644
--- a/pkg/sentry/fsimpl/proc/tasks_inode_refs.go
+++ b/pkg/sentry/fsimpl/proc/tasks_inode_refs.go
@@ -7,9 +7,15 @@ import (
"gvisor.dev/gvisor/pkg/refsvfs2"
)
-// ownerType is used to customize logging. Note that we use a pointer to T so
-// that we do not copy the entire object when passed as a format parameter.
-var tasksInodeownerType *tasksInode
+// enableLogging indicates whether reference-related events should be logged (with
+// stack traces). This is false by default and should only be set to true for
+// debugging purposes, as it can generate an extremely large amount of output
+// and drastically degrade performance.
+const tasksInodeenableLogging = false
+
+// obj is used to customize logging. Note that we use a pointer to T so that
+// we do not copy the entire object when passed as a format parameter.
+var tasksInodeobj *tasksInode
// Refs implements refs.RefCounter. It keeps a reference count using atomic
// operations and calls the destructor when the count reaches zero.
@@ -29,16 +35,24 @@ type tasksInodeRefs struct {
refCount int64
}
-// EnableLeakCheck enables reference leak checking on r.
-func (r *tasksInodeRefs) EnableLeakCheck() {
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Register(r, fmt.Sprintf("%T", tasksInodeownerType))
- }
+// RefType implements refsvfs2.CheckedObject.RefType.
+func (r *tasksInodeRefs) RefType() string {
+ return fmt.Sprintf("%T", tasksInodeobj)[1:]
}
// LeakMessage implements refsvfs2.CheckedObject.LeakMessage.
func (r *tasksInodeRefs) LeakMessage() string {
- return fmt.Sprintf("%T %p: reference count of %d instead of 0", tasksInodeownerType, r, r.ReadRefs())
+ return fmt.Sprintf("[%s %p] reference count of %d instead of 0", r.RefType(), r, r.ReadRefs())
+}
+
+// LogRefs implements refsvfs2.CheckedObject.LogRefs.
+func (r *tasksInodeRefs) LogRefs() bool {
+ return tasksInodeenableLogging
+}
+
+// EnableLeakCheck enables reference leak checking on r.
+func (r *tasksInodeRefs) EnableLeakCheck() {
+ refsvfs2.Register(r)
}
// ReadRefs returns the current number of references. The returned count is
@@ -52,8 +66,10 @@ func (r *tasksInodeRefs) ReadRefs() int64 {
//
//go:nosplit
func (r *tasksInodeRefs) IncRef() {
- if v := atomic.AddInt64(&r.refCount, 1); v <= 0 {
- panic(fmt.Sprintf("Incrementing non-positive count %p on %T", r, tasksInodeownerType))
+ v := atomic.AddInt64(&r.refCount, 1)
+ refsvfs2.LogIncRef(r, v+1)
+ if v <= 0 {
+ panic(fmt.Sprintf("Incrementing non-positive count %p on %s", r, r.RefType()))
}
}
@@ -66,14 +82,14 @@ func (r *tasksInodeRefs) IncRef() {
//go:nosplit
func (r *tasksInodeRefs) TryIncRef() bool {
const speculativeRef = 1 << 32
- v := atomic.AddInt64(&r.refCount, speculativeRef)
- if int32(v) < 0 {
+ if v := atomic.AddInt64(&r.refCount, speculativeRef); int32(v) < 0 {
atomic.AddInt64(&r.refCount, -speculativeRef)
return false
}
- atomic.AddInt64(&r.refCount, -speculativeRef+1)
+ v := atomic.AddInt64(&r.refCount, -speculativeRef+1)
+ refsvfs2.LogTryIncRef(r, v+1)
return true
}
@@ -90,14 +106,14 @@ func (r *tasksInodeRefs) TryIncRef() bool {
//
//go:nosplit
func (r *tasksInodeRefs) DecRef(destroy func()) {
- switch v := atomic.AddInt64(&r.refCount, -1); {
+ v := atomic.AddInt64(&r.refCount, -1)
+ refsvfs2.LogDecRef(r, v+1)
+ switch {
case v < -1:
- panic(fmt.Sprintf("Decrementing non-positive ref count %p, owned by %T", r, tasksInodeownerType))
+ panic(fmt.Sprintf("Decrementing non-positive ref count %p, owned by %s", r, r.RefType()))
case v == -1:
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Unregister(r, fmt.Sprintf("%T", tasksInodeownerType))
- }
+ refsvfs2.Unregister(r)
if destroy != nil {
destroy()
@@ -106,7 +122,7 @@ func (r *tasksInodeRefs) DecRef(destroy func()) {
}
func (r *tasksInodeRefs) afterLoad() {
- if refsvfs2.LeakCheckEnabled() && r.ReadRefs() > 0 {
+ if r.ReadRefs() > 0 {
r.EnableLeakCheck()
}
}
diff --git a/pkg/sentry/fsimpl/sys/dir_refs.go b/pkg/sentry/fsimpl/sys/dir_refs.go
index 371ad3a8c..176a4bf98 100644
--- a/pkg/sentry/fsimpl/sys/dir_refs.go
+++ b/pkg/sentry/fsimpl/sys/dir_refs.go
@@ -7,9 +7,15 @@ import (
"gvisor.dev/gvisor/pkg/refsvfs2"
)
-// ownerType is used to customize logging. Note that we use a pointer to T so
-// that we do not copy the entire object when passed as a format parameter.
-var dirownerType *dir
+// enableLogging indicates whether reference-related events should be logged (with
+// stack traces). This is false by default and should only be set to true for
+// debugging purposes, as it can generate an extremely large amount of output
+// and drastically degrade performance.
+const direnableLogging = false
+
+// obj is used to customize logging. Note that we use a pointer to T so that
+// we do not copy the entire object when passed as a format parameter.
+var dirobj *dir
// Refs implements refs.RefCounter. It keeps a reference count using atomic
// operations and calls the destructor when the count reaches zero.
@@ -29,16 +35,24 @@ type dirRefs struct {
refCount int64
}
-// EnableLeakCheck enables reference leak checking on r.
-func (r *dirRefs) EnableLeakCheck() {
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Register(r, fmt.Sprintf("%T", dirownerType))
- }
+// RefType implements refsvfs2.CheckedObject.RefType.
+func (r *dirRefs) RefType() string {
+ return fmt.Sprintf("%T", dirobj)[1:]
}
// LeakMessage implements refsvfs2.CheckedObject.LeakMessage.
func (r *dirRefs) LeakMessage() string {
- return fmt.Sprintf("%T %p: reference count of %d instead of 0", dirownerType, r, r.ReadRefs())
+ return fmt.Sprintf("[%s %p] reference count of %d instead of 0", r.RefType(), r, r.ReadRefs())
+}
+
+// LogRefs implements refsvfs2.CheckedObject.LogRefs.
+func (r *dirRefs) LogRefs() bool {
+ return direnableLogging
+}
+
+// EnableLeakCheck enables reference leak checking on r.
+func (r *dirRefs) EnableLeakCheck() {
+ refsvfs2.Register(r)
}
// ReadRefs returns the current number of references. The returned count is
@@ -52,8 +66,10 @@ func (r *dirRefs) ReadRefs() int64 {
//
//go:nosplit
func (r *dirRefs) IncRef() {
- if v := atomic.AddInt64(&r.refCount, 1); v <= 0 {
- panic(fmt.Sprintf("Incrementing non-positive count %p on %T", r, dirownerType))
+ v := atomic.AddInt64(&r.refCount, 1)
+ refsvfs2.LogIncRef(r, v+1)
+ if v <= 0 {
+ panic(fmt.Sprintf("Incrementing non-positive count %p on %s", r, r.RefType()))
}
}
@@ -66,14 +82,14 @@ func (r *dirRefs) IncRef() {
//go:nosplit
func (r *dirRefs) TryIncRef() bool {
const speculativeRef = 1 << 32
- v := atomic.AddInt64(&r.refCount, speculativeRef)
- if int32(v) < 0 {
+ if v := atomic.AddInt64(&r.refCount, speculativeRef); int32(v) < 0 {
atomic.AddInt64(&r.refCount, -speculativeRef)
return false
}
- atomic.AddInt64(&r.refCount, -speculativeRef+1)
+ v := atomic.AddInt64(&r.refCount, -speculativeRef+1)
+ refsvfs2.LogTryIncRef(r, v+1)
return true
}
@@ -90,14 +106,14 @@ func (r *dirRefs) TryIncRef() bool {
//
//go:nosplit
func (r *dirRefs) DecRef(destroy func()) {
- switch v := atomic.AddInt64(&r.refCount, -1); {
+ v := atomic.AddInt64(&r.refCount, -1)
+ refsvfs2.LogDecRef(r, v+1)
+ switch {
case v < -1:
- panic(fmt.Sprintf("Decrementing non-positive ref count %p, owned by %T", r, dirownerType))
+ panic(fmt.Sprintf("Decrementing non-positive ref count %p, owned by %s", r, r.RefType()))
case v == -1:
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Unregister(r, fmt.Sprintf("%T", dirownerType))
- }
+ refsvfs2.Unregister(r)
if destroy != nil {
destroy()
@@ -106,7 +122,7 @@ func (r *dirRefs) DecRef(destroy func()) {
}
func (r *dirRefs) afterLoad() {
- if refsvfs2.LeakCheckEnabled() && r.ReadRefs() > 0 {
+ if r.ReadRefs() > 0 {
r.EnableLeakCheck()
}
}
diff --git a/pkg/sentry/fsimpl/tmpfs/inode_refs.go b/pkg/sentry/fsimpl/tmpfs/inode_refs.go
index 55b1d39fe..eff30ca09 100644
--- a/pkg/sentry/fsimpl/tmpfs/inode_refs.go
+++ b/pkg/sentry/fsimpl/tmpfs/inode_refs.go
@@ -7,9 +7,15 @@ import (
"gvisor.dev/gvisor/pkg/refsvfs2"
)
-// ownerType is used to customize logging. Note that we use a pointer to T so
-// that we do not copy the entire object when passed as a format parameter.
-var inodeownerType *inode
+// enableLogging indicates whether reference-related events should be logged (with
+// stack traces). This is false by default and should only be set to true for
+// debugging purposes, as it can generate an extremely large amount of output
+// and drastically degrade performance.
+const inodeenableLogging = false
+
+// obj is used to customize logging. Note that we use a pointer to T so that
+// we do not copy the entire object when passed as a format parameter.
+var inodeobj *inode
// Refs implements refs.RefCounter. It keeps a reference count using atomic
// operations and calls the destructor when the count reaches zero.
@@ -29,16 +35,24 @@ type inodeRefs struct {
refCount int64
}
-// EnableLeakCheck enables reference leak checking on r.
-func (r *inodeRefs) EnableLeakCheck() {
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Register(r, fmt.Sprintf("%T", inodeownerType))
- }
+// RefType implements refsvfs2.CheckedObject.RefType.
+func (r *inodeRefs) RefType() string {
+ return fmt.Sprintf("%T", inodeobj)[1:]
}
// LeakMessage implements refsvfs2.CheckedObject.LeakMessage.
func (r *inodeRefs) LeakMessage() string {
- return fmt.Sprintf("%T %p: reference count of %d instead of 0", inodeownerType, r, r.ReadRefs())
+ return fmt.Sprintf("[%s %p] reference count of %d instead of 0", r.RefType(), r, r.ReadRefs())
+}
+
+// LogRefs implements refsvfs2.CheckedObject.LogRefs.
+func (r *inodeRefs) LogRefs() bool {
+ return inodeenableLogging
+}
+
+// EnableLeakCheck enables reference leak checking on r.
+func (r *inodeRefs) EnableLeakCheck() {
+ refsvfs2.Register(r)
}
// ReadRefs returns the current number of references. The returned count is
@@ -52,8 +66,10 @@ func (r *inodeRefs) ReadRefs() int64 {
//
//go:nosplit
func (r *inodeRefs) IncRef() {
- if v := atomic.AddInt64(&r.refCount, 1); v <= 0 {
- panic(fmt.Sprintf("Incrementing non-positive count %p on %T", r, inodeownerType))
+ v := atomic.AddInt64(&r.refCount, 1)
+ refsvfs2.LogIncRef(r, v+1)
+ if v <= 0 {
+ panic(fmt.Sprintf("Incrementing non-positive count %p on %s", r, r.RefType()))
}
}
@@ -66,14 +82,14 @@ func (r *inodeRefs) IncRef() {
//go:nosplit
func (r *inodeRefs) TryIncRef() bool {
const speculativeRef = 1 << 32
- v := atomic.AddInt64(&r.refCount, speculativeRef)
- if int32(v) < 0 {
+ if v := atomic.AddInt64(&r.refCount, speculativeRef); int32(v) < 0 {
atomic.AddInt64(&r.refCount, -speculativeRef)
return false
}
- atomic.AddInt64(&r.refCount, -speculativeRef+1)
+ v := atomic.AddInt64(&r.refCount, -speculativeRef+1)
+ refsvfs2.LogTryIncRef(r, v+1)
return true
}
@@ -90,14 +106,14 @@ func (r *inodeRefs) TryIncRef() bool {
//
//go:nosplit
func (r *inodeRefs) DecRef(destroy func()) {
- switch v := atomic.AddInt64(&r.refCount, -1); {
+ v := atomic.AddInt64(&r.refCount, -1)
+ refsvfs2.LogDecRef(r, v+1)
+ switch {
case v < -1:
- panic(fmt.Sprintf("Decrementing non-positive ref count %p, owned by %T", r, inodeownerType))
+ panic(fmt.Sprintf("Decrementing non-positive ref count %p, owned by %s", r, r.RefType()))
case v == -1:
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Unregister(r, fmt.Sprintf("%T", inodeownerType))
- }
+ refsvfs2.Unregister(r)
if destroy != nil {
destroy()
@@ -106,7 +122,7 @@ func (r *inodeRefs) DecRef(destroy func()) {
}
func (r *inodeRefs) afterLoad() {
- if refsvfs2.LeakCheckEnabled() && r.ReadRefs() > 0 {
+ if r.ReadRefs() > 0 {
r.EnableLeakCheck()
}
}
diff --git a/pkg/sentry/kernel/fd_table_refs.go b/pkg/sentry/kernel/fd_table_refs.go
index cbf2e85ed..992606f36 100644
--- a/pkg/sentry/kernel/fd_table_refs.go
+++ b/pkg/sentry/kernel/fd_table_refs.go
@@ -7,9 +7,15 @@ import (
"gvisor.dev/gvisor/pkg/refsvfs2"
)
-// ownerType is used to customize logging. Note that we use a pointer to T so
-// that we do not copy the entire object when passed as a format parameter.
-var FDTableownerType *FDTable
+// enableLogging indicates whether reference-related events should be logged (with
+// stack traces). This is false by default and should only be set to true for
+// debugging purposes, as it can generate an extremely large amount of output
+// and drastically degrade performance.
+const FDTableenableLogging = false
+
+// obj is used to customize logging. Note that we use a pointer to T so that
+// we do not copy the entire object when passed as a format parameter.
+var FDTableobj *FDTable
// Refs implements refs.RefCounter. It keeps a reference count using atomic
// operations and calls the destructor when the count reaches zero.
@@ -29,16 +35,24 @@ type FDTableRefs struct {
refCount int64
}
-// EnableLeakCheck enables reference leak checking on r.
-func (r *FDTableRefs) EnableLeakCheck() {
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Register(r, fmt.Sprintf("%T", FDTableownerType))
- }
+// RefType implements refsvfs2.CheckedObject.RefType.
+func (r *FDTableRefs) RefType() string {
+ return fmt.Sprintf("%T", FDTableobj)[1:]
}
// LeakMessage implements refsvfs2.CheckedObject.LeakMessage.
func (r *FDTableRefs) LeakMessage() string {
- return fmt.Sprintf("%T %p: reference count of %d instead of 0", FDTableownerType, r, r.ReadRefs())
+ return fmt.Sprintf("[%s %p] reference count of %d instead of 0", r.RefType(), r, r.ReadRefs())
+}
+
+// LogRefs implements refsvfs2.CheckedObject.LogRefs.
+func (r *FDTableRefs) LogRefs() bool {
+ return FDTableenableLogging
+}
+
+// EnableLeakCheck enables reference leak checking on r.
+func (r *FDTableRefs) EnableLeakCheck() {
+ refsvfs2.Register(r)
}
// ReadRefs returns the current number of references. The returned count is
@@ -52,8 +66,10 @@ func (r *FDTableRefs) ReadRefs() int64 {
//
//go:nosplit
func (r *FDTableRefs) IncRef() {
- if v := atomic.AddInt64(&r.refCount, 1); v <= 0 {
- panic(fmt.Sprintf("Incrementing non-positive count %p on %T", r, FDTableownerType))
+ v := atomic.AddInt64(&r.refCount, 1)
+ refsvfs2.LogIncRef(r, v+1)
+ if v <= 0 {
+ panic(fmt.Sprintf("Incrementing non-positive count %p on %s", r, r.RefType()))
}
}
@@ -66,14 +82,14 @@ func (r *FDTableRefs) IncRef() {
//go:nosplit
func (r *FDTableRefs) TryIncRef() bool {
const speculativeRef = 1 << 32
- v := atomic.AddInt64(&r.refCount, speculativeRef)
- if int32(v) < 0 {
+ if v := atomic.AddInt64(&r.refCount, speculativeRef); int32(v) < 0 {
atomic.AddInt64(&r.refCount, -speculativeRef)
return false
}
- atomic.AddInt64(&r.refCount, -speculativeRef+1)
+ v := atomic.AddInt64(&r.refCount, -speculativeRef+1)
+ refsvfs2.LogTryIncRef(r, v+1)
return true
}
@@ -90,14 +106,14 @@ func (r *FDTableRefs) TryIncRef() bool {
//
//go:nosplit
func (r *FDTableRefs) DecRef(destroy func()) {
- switch v := atomic.AddInt64(&r.refCount, -1); {
+ v := atomic.AddInt64(&r.refCount, -1)
+ refsvfs2.LogDecRef(r, v+1)
+ switch {
case v < -1:
- panic(fmt.Sprintf("Decrementing non-positive ref count %p, owned by %T", r, FDTableownerType))
+ panic(fmt.Sprintf("Decrementing non-positive ref count %p, owned by %s", r, r.RefType()))
case v == -1:
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Unregister(r, fmt.Sprintf("%T", FDTableownerType))
- }
+ refsvfs2.Unregister(r)
if destroy != nil {
destroy()
@@ -106,7 +122,7 @@ func (r *FDTableRefs) DecRef(destroy func()) {
}
func (r *FDTableRefs) afterLoad() {
- if refsvfs2.LeakCheckEnabled() && r.ReadRefs() > 0 {
+ if r.ReadRefs() > 0 {
r.EnableLeakCheck()
}
}
diff --git a/pkg/sentry/kernel/fs_context_refs.go b/pkg/sentry/kernel/fs_context_refs.go
index 025f11faa..ff812ab16 100644
--- a/pkg/sentry/kernel/fs_context_refs.go
+++ b/pkg/sentry/kernel/fs_context_refs.go
@@ -7,9 +7,15 @@ import (
"gvisor.dev/gvisor/pkg/refsvfs2"
)
-// ownerType is used to customize logging. Note that we use a pointer to T so
-// that we do not copy the entire object when passed as a format parameter.
-var FSContextownerType *FSContext
+// enableLogging indicates whether reference-related events should be logged (with
+// stack traces). This is false by default and should only be set to true for
+// debugging purposes, as it can generate an extremely large amount of output
+// and drastically degrade performance.
+const FSContextenableLogging = false
+
+// obj is used to customize logging. Note that we use a pointer to T so that
+// we do not copy the entire object when passed as a format parameter.
+var FSContextobj *FSContext
// Refs implements refs.RefCounter. It keeps a reference count using atomic
// operations and calls the destructor when the count reaches zero.
@@ -29,16 +35,24 @@ type FSContextRefs struct {
refCount int64
}
-// EnableLeakCheck enables reference leak checking on r.
-func (r *FSContextRefs) EnableLeakCheck() {
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Register(r, fmt.Sprintf("%T", FSContextownerType))
- }
+// RefType implements refsvfs2.CheckedObject.RefType.
+func (r *FSContextRefs) RefType() string {
+ return fmt.Sprintf("%T", FSContextobj)[1:]
}
// LeakMessage implements refsvfs2.CheckedObject.LeakMessage.
func (r *FSContextRefs) LeakMessage() string {
- return fmt.Sprintf("%T %p: reference count of %d instead of 0", FSContextownerType, r, r.ReadRefs())
+ return fmt.Sprintf("[%s %p] reference count of %d instead of 0", r.RefType(), r, r.ReadRefs())
+}
+
+// LogRefs implements refsvfs2.CheckedObject.LogRefs.
+func (r *FSContextRefs) LogRefs() bool {
+ return FSContextenableLogging
+}
+
+// EnableLeakCheck enables reference leak checking on r.
+func (r *FSContextRefs) EnableLeakCheck() {
+ refsvfs2.Register(r)
}
// ReadRefs returns the current number of references. The returned count is
@@ -52,8 +66,10 @@ func (r *FSContextRefs) ReadRefs() int64 {
//
//go:nosplit
func (r *FSContextRefs) IncRef() {
- if v := atomic.AddInt64(&r.refCount, 1); v <= 0 {
- panic(fmt.Sprintf("Incrementing non-positive count %p on %T", r, FSContextownerType))
+ v := atomic.AddInt64(&r.refCount, 1)
+ refsvfs2.LogIncRef(r, v+1)
+ if v <= 0 {
+ panic(fmt.Sprintf("Incrementing non-positive count %p on %s", r, r.RefType()))
}
}
@@ -66,14 +82,14 @@ func (r *FSContextRefs) IncRef() {
//go:nosplit
func (r *FSContextRefs) TryIncRef() bool {
const speculativeRef = 1 << 32
- v := atomic.AddInt64(&r.refCount, speculativeRef)
- if int32(v) < 0 {
+ if v := atomic.AddInt64(&r.refCount, speculativeRef); int32(v) < 0 {
atomic.AddInt64(&r.refCount, -speculativeRef)
return false
}
- atomic.AddInt64(&r.refCount, -speculativeRef+1)
+ v := atomic.AddInt64(&r.refCount, -speculativeRef+1)
+ refsvfs2.LogTryIncRef(r, v+1)
return true
}
@@ -90,14 +106,14 @@ func (r *FSContextRefs) TryIncRef() bool {
//
//go:nosplit
func (r *FSContextRefs) DecRef(destroy func()) {
- switch v := atomic.AddInt64(&r.refCount, -1); {
+ v := atomic.AddInt64(&r.refCount, -1)
+ refsvfs2.LogDecRef(r, v+1)
+ switch {
case v < -1:
- panic(fmt.Sprintf("Decrementing non-positive ref count %p, owned by %T", r, FSContextownerType))
+ panic(fmt.Sprintf("Decrementing non-positive ref count %p, owned by %s", r, r.RefType()))
case v == -1:
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Unregister(r, fmt.Sprintf("%T", FSContextownerType))
- }
+ refsvfs2.Unregister(r)
if destroy != nil {
destroy()
@@ -106,7 +122,7 @@ func (r *FSContextRefs) DecRef(destroy func()) {
}
func (r *FSContextRefs) afterLoad() {
- if refsvfs2.LeakCheckEnabled() && r.ReadRefs() > 0 {
+ if r.ReadRefs() > 0 {
r.EnableLeakCheck()
}
}
diff --git a/pkg/sentry/kernel/ipc_namespace_refs.go b/pkg/sentry/kernel/ipc_namespace_refs.go
index aec0f7a41..5b37e617a 100644
--- a/pkg/sentry/kernel/ipc_namespace_refs.go
+++ b/pkg/sentry/kernel/ipc_namespace_refs.go
@@ -7,9 +7,15 @@ import (
"gvisor.dev/gvisor/pkg/refsvfs2"
)
-// ownerType is used to customize logging. Note that we use a pointer to T so
-// that we do not copy the entire object when passed as a format parameter.
-var IPCNamespaceownerType *IPCNamespace
+// enableLogging indicates whether reference-related events should be logged (with
+// stack traces). This is false by default and should only be set to true for
+// debugging purposes, as it can generate an extremely large amount of output
+// and drastically degrade performance.
+const IPCNamespaceenableLogging = false
+
+// obj is used to customize logging. Note that we use a pointer to T so that
+// we do not copy the entire object when passed as a format parameter.
+var IPCNamespaceobj *IPCNamespace
// Refs implements refs.RefCounter. It keeps a reference count using atomic
// operations and calls the destructor when the count reaches zero.
@@ -29,16 +35,24 @@ type IPCNamespaceRefs struct {
refCount int64
}
-// EnableLeakCheck enables reference leak checking on r.
-func (r *IPCNamespaceRefs) EnableLeakCheck() {
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Register(r, fmt.Sprintf("%T", IPCNamespaceownerType))
- }
+// RefType implements refsvfs2.CheckedObject.RefType.
+func (r *IPCNamespaceRefs) RefType() string {
+ return fmt.Sprintf("%T", IPCNamespaceobj)[1:]
}
// LeakMessage implements refsvfs2.CheckedObject.LeakMessage.
func (r *IPCNamespaceRefs) LeakMessage() string {
- return fmt.Sprintf("%T %p: reference count of %d instead of 0", IPCNamespaceownerType, r, r.ReadRefs())
+ return fmt.Sprintf("[%s %p] reference count of %d instead of 0", r.RefType(), r, r.ReadRefs())
+}
+
+// LogRefs implements refsvfs2.CheckedObject.LogRefs.
+func (r *IPCNamespaceRefs) LogRefs() bool {
+ return IPCNamespaceenableLogging
+}
+
+// EnableLeakCheck enables reference leak checking on r.
+func (r *IPCNamespaceRefs) EnableLeakCheck() {
+ refsvfs2.Register(r)
}
// ReadRefs returns the current number of references. The returned count is
@@ -52,8 +66,10 @@ func (r *IPCNamespaceRefs) ReadRefs() int64 {
//
//go:nosplit
func (r *IPCNamespaceRefs) IncRef() {
- if v := atomic.AddInt64(&r.refCount, 1); v <= 0 {
- panic(fmt.Sprintf("Incrementing non-positive count %p on %T", r, IPCNamespaceownerType))
+ v := atomic.AddInt64(&r.refCount, 1)
+ refsvfs2.LogIncRef(r, v+1)
+ if v <= 0 {
+ panic(fmt.Sprintf("Incrementing non-positive count %p on %s", r, r.RefType()))
}
}
@@ -66,14 +82,14 @@ func (r *IPCNamespaceRefs) IncRef() {
//go:nosplit
func (r *IPCNamespaceRefs) TryIncRef() bool {
const speculativeRef = 1 << 32
- v := atomic.AddInt64(&r.refCount, speculativeRef)
- if int32(v) < 0 {
+ if v := atomic.AddInt64(&r.refCount, speculativeRef); int32(v) < 0 {
atomic.AddInt64(&r.refCount, -speculativeRef)
return false
}
- atomic.AddInt64(&r.refCount, -speculativeRef+1)
+ v := atomic.AddInt64(&r.refCount, -speculativeRef+1)
+ refsvfs2.LogTryIncRef(r, v+1)
return true
}
@@ -90,14 +106,14 @@ func (r *IPCNamespaceRefs) TryIncRef() bool {
//
//go:nosplit
func (r *IPCNamespaceRefs) DecRef(destroy func()) {
- switch v := atomic.AddInt64(&r.refCount, -1); {
+ v := atomic.AddInt64(&r.refCount, -1)
+ refsvfs2.LogDecRef(r, v+1)
+ switch {
case v < -1:
- panic(fmt.Sprintf("Decrementing non-positive ref count %p, owned by %T", r, IPCNamespaceownerType))
+ panic(fmt.Sprintf("Decrementing non-positive ref count %p, owned by %s", r, r.RefType()))
case v == -1:
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Unregister(r, fmt.Sprintf("%T", IPCNamespaceownerType))
- }
+ refsvfs2.Unregister(r)
if destroy != nil {
destroy()
@@ -106,7 +122,7 @@ func (r *IPCNamespaceRefs) DecRef(destroy func()) {
}
func (r *IPCNamespaceRefs) afterLoad() {
- if refsvfs2.LeakCheckEnabled() && r.ReadRefs() > 0 {
+ if r.ReadRefs() > 0 {
r.EnableLeakCheck()
}
}
diff --git a/pkg/sentry/kernel/process_group_refs.go b/pkg/sentry/kernel/process_group_refs.go
index 1f4486817..29bd0b80f 100644
--- a/pkg/sentry/kernel/process_group_refs.go
+++ b/pkg/sentry/kernel/process_group_refs.go
@@ -7,9 +7,15 @@ import (
"gvisor.dev/gvisor/pkg/refsvfs2"
)
-// ownerType is used to customize logging. Note that we use a pointer to T so
-// that we do not copy the entire object when passed as a format parameter.
-var ProcessGroupownerType *ProcessGroup
+// enableLogging indicates whether reference-related events should be logged (with
+// stack traces). This is false by default and should only be set to true for
+// debugging purposes, as it can generate an extremely large amount of output
+// and drastically degrade performance.
+const ProcessGroupenableLogging = false
+
+// obj is used to customize logging. Note that we use a pointer to T so that
+// we do not copy the entire object when passed as a format parameter.
+var ProcessGroupobj *ProcessGroup
// Refs implements refs.RefCounter. It keeps a reference count using atomic
// operations and calls the destructor when the count reaches zero.
@@ -29,16 +35,24 @@ type ProcessGroupRefs struct {
refCount int64
}
-// EnableLeakCheck enables reference leak checking on r.
-func (r *ProcessGroupRefs) EnableLeakCheck() {
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Register(r, fmt.Sprintf("%T", ProcessGroupownerType))
- }
+// RefType implements refsvfs2.CheckedObject.RefType.
+func (r *ProcessGroupRefs) RefType() string {
+ return fmt.Sprintf("%T", ProcessGroupobj)[1:]
}
// LeakMessage implements refsvfs2.CheckedObject.LeakMessage.
func (r *ProcessGroupRefs) LeakMessage() string {
- return fmt.Sprintf("%T %p: reference count of %d instead of 0", ProcessGroupownerType, r, r.ReadRefs())
+ return fmt.Sprintf("[%s %p] reference count of %d instead of 0", r.RefType(), r, r.ReadRefs())
+}
+
+// LogRefs implements refsvfs2.CheckedObject.LogRefs.
+func (r *ProcessGroupRefs) LogRefs() bool {
+ return ProcessGroupenableLogging
+}
+
+// EnableLeakCheck enables reference leak checking on r.
+func (r *ProcessGroupRefs) EnableLeakCheck() {
+ refsvfs2.Register(r)
}
// ReadRefs returns the current number of references. The returned count is
@@ -52,8 +66,10 @@ func (r *ProcessGroupRefs) ReadRefs() int64 {
//
//go:nosplit
func (r *ProcessGroupRefs) IncRef() {
- if v := atomic.AddInt64(&r.refCount, 1); v <= 0 {
- panic(fmt.Sprintf("Incrementing non-positive count %p on %T", r, ProcessGroupownerType))
+ v := atomic.AddInt64(&r.refCount, 1)
+ refsvfs2.LogIncRef(r, v+1)
+ if v <= 0 {
+ panic(fmt.Sprintf("Incrementing non-positive count %p on %s", r, r.RefType()))
}
}
@@ -66,14 +82,14 @@ func (r *ProcessGroupRefs) IncRef() {
//go:nosplit
func (r *ProcessGroupRefs) TryIncRef() bool {
const speculativeRef = 1 << 32
- v := atomic.AddInt64(&r.refCount, speculativeRef)
- if int32(v) < 0 {
+ if v := atomic.AddInt64(&r.refCount, speculativeRef); int32(v) < 0 {
atomic.AddInt64(&r.refCount, -speculativeRef)
return false
}
- atomic.AddInt64(&r.refCount, -speculativeRef+1)
+ v := atomic.AddInt64(&r.refCount, -speculativeRef+1)
+ refsvfs2.LogTryIncRef(r, v+1)
return true
}
@@ -90,14 +106,14 @@ func (r *ProcessGroupRefs) TryIncRef() bool {
//
//go:nosplit
func (r *ProcessGroupRefs) DecRef(destroy func()) {
- switch v := atomic.AddInt64(&r.refCount, -1); {
+ v := atomic.AddInt64(&r.refCount, -1)
+ refsvfs2.LogDecRef(r, v+1)
+ switch {
case v < -1:
- panic(fmt.Sprintf("Decrementing non-positive ref count %p, owned by %T", r, ProcessGroupownerType))
+ panic(fmt.Sprintf("Decrementing non-positive ref count %p, owned by %s", r, r.RefType()))
case v == -1:
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Unregister(r, fmt.Sprintf("%T", ProcessGroupownerType))
- }
+ refsvfs2.Unregister(r)
if destroy != nil {
destroy()
@@ -106,7 +122,7 @@ func (r *ProcessGroupRefs) DecRef(destroy func()) {
}
func (r *ProcessGroupRefs) afterLoad() {
- if refsvfs2.LeakCheckEnabled() && r.ReadRefs() > 0 {
+ if r.ReadRefs() > 0 {
r.EnableLeakCheck()
}
}
diff --git a/pkg/sentry/kernel/session_refs.go b/pkg/sentry/kernel/session_refs.go
index 86df93be1..430cb131c 100644
--- a/pkg/sentry/kernel/session_refs.go
+++ b/pkg/sentry/kernel/session_refs.go
@@ -7,9 +7,15 @@ import (
"gvisor.dev/gvisor/pkg/refsvfs2"
)
-// ownerType is used to customize logging. Note that we use a pointer to T so
-// that we do not copy the entire object when passed as a format parameter.
-var SessionownerType *Session
+// enableLogging indicates whether reference-related events should be logged (with
+// stack traces). This is false by default and should only be set to true for
+// debugging purposes, as it can generate an extremely large amount of output
+// and drastically degrade performance.
+const SessionenableLogging = false
+
+// obj is used to customize logging. Note that we use a pointer to T so that
+// we do not copy the entire object when passed as a format parameter.
+var Sessionobj *Session
// Refs implements refs.RefCounter. It keeps a reference count using atomic
// operations and calls the destructor when the count reaches zero.
@@ -29,16 +35,24 @@ type SessionRefs struct {
refCount int64
}
-// EnableLeakCheck enables reference leak checking on r.
-func (r *SessionRefs) EnableLeakCheck() {
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Register(r, fmt.Sprintf("%T", SessionownerType))
- }
+// RefType implements refsvfs2.CheckedObject.RefType.
+func (r *SessionRefs) RefType() string {
+ return fmt.Sprintf("%T", Sessionobj)[1:]
}
// LeakMessage implements refsvfs2.CheckedObject.LeakMessage.
func (r *SessionRefs) LeakMessage() string {
- return fmt.Sprintf("%T %p: reference count of %d instead of 0", SessionownerType, r, r.ReadRefs())
+ return fmt.Sprintf("[%s %p] reference count of %d instead of 0", r.RefType(), r, r.ReadRefs())
+}
+
+// LogRefs implements refsvfs2.CheckedObject.LogRefs.
+func (r *SessionRefs) LogRefs() bool {
+ return SessionenableLogging
+}
+
+// EnableLeakCheck enables reference leak checking on r.
+func (r *SessionRefs) EnableLeakCheck() {
+ refsvfs2.Register(r)
}
// ReadRefs returns the current number of references. The returned count is
@@ -52,8 +66,10 @@ func (r *SessionRefs) ReadRefs() int64 {
//
//go:nosplit
func (r *SessionRefs) IncRef() {
- if v := atomic.AddInt64(&r.refCount, 1); v <= 0 {
- panic(fmt.Sprintf("Incrementing non-positive count %p on %T", r, SessionownerType))
+ v := atomic.AddInt64(&r.refCount, 1)
+ refsvfs2.LogIncRef(r, v+1)
+ if v <= 0 {
+ panic(fmt.Sprintf("Incrementing non-positive count %p on %s", r, r.RefType()))
}
}
@@ -66,14 +82,14 @@ func (r *SessionRefs) IncRef() {
//go:nosplit
func (r *SessionRefs) TryIncRef() bool {
const speculativeRef = 1 << 32
- v := atomic.AddInt64(&r.refCount, speculativeRef)
- if int32(v) < 0 {
+ if v := atomic.AddInt64(&r.refCount, speculativeRef); int32(v) < 0 {
atomic.AddInt64(&r.refCount, -speculativeRef)
return false
}
- atomic.AddInt64(&r.refCount, -speculativeRef+1)
+ v := atomic.AddInt64(&r.refCount, -speculativeRef+1)
+ refsvfs2.LogTryIncRef(r, v+1)
return true
}
@@ -90,14 +106,14 @@ func (r *SessionRefs) TryIncRef() bool {
//
//go:nosplit
func (r *SessionRefs) DecRef(destroy func()) {
- switch v := atomic.AddInt64(&r.refCount, -1); {
+ v := atomic.AddInt64(&r.refCount, -1)
+ refsvfs2.LogDecRef(r, v+1)
+ switch {
case v < -1:
- panic(fmt.Sprintf("Decrementing non-positive ref count %p, owned by %T", r, SessionownerType))
+ panic(fmt.Sprintf("Decrementing non-positive ref count %p, owned by %s", r, r.RefType()))
case v == -1:
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Unregister(r, fmt.Sprintf("%T", SessionownerType))
- }
+ refsvfs2.Unregister(r)
if destroy != nil {
destroy()
@@ -106,7 +122,7 @@ func (r *SessionRefs) DecRef(destroy func()) {
}
func (r *SessionRefs) afterLoad() {
- if refsvfs2.LeakCheckEnabled() && r.ReadRefs() > 0 {
+ if r.ReadRefs() > 0 {
r.EnableLeakCheck()
}
}
diff --git a/pkg/sentry/kernel/shm/shm_refs.go b/pkg/sentry/kernel/shm/shm_refs.go
index 58b0e80cf..5f148594c 100644
--- a/pkg/sentry/kernel/shm/shm_refs.go
+++ b/pkg/sentry/kernel/shm/shm_refs.go
@@ -7,9 +7,15 @@ import (
"gvisor.dev/gvisor/pkg/refsvfs2"
)
-// ownerType is used to customize logging. Note that we use a pointer to T so
-// that we do not copy the entire object when passed as a format parameter.
-var ShmownerType *Shm
+// enableLogging indicates whether reference-related events should be logged (with
+// stack traces). This is false by default and should only be set to true for
+// debugging purposes, as it can generate an extremely large amount of output
+// and drastically degrade performance.
+const ShmenableLogging = false
+
+// obj is used to customize logging. Note that we use a pointer to T so that
+// we do not copy the entire object when passed as a format parameter.
+var Shmobj *Shm
// Refs implements refs.RefCounter. It keeps a reference count using atomic
// operations and calls the destructor when the count reaches zero.
@@ -29,16 +35,24 @@ type ShmRefs struct {
refCount int64
}
-// EnableLeakCheck enables reference leak checking on r.
-func (r *ShmRefs) EnableLeakCheck() {
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Register(r, fmt.Sprintf("%T", ShmownerType))
- }
+// RefType implements refsvfs2.CheckedObject.RefType.
+func (r *ShmRefs) RefType() string {
+ return fmt.Sprintf("%T", Shmobj)[1:]
}
// LeakMessage implements refsvfs2.CheckedObject.LeakMessage.
func (r *ShmRefs) LeakMessage() string {
- return fmt.Sprintf("%T %p: reference count of %d instead of 0", ShmownerType, r, r.ReadRefs())
+ return fmt.Sprintf("[%s %p] reference count of %d instead of 0", r.RefType(), r, r.ReadRefs())
+}
+
+// LogRefs implements refsvfs2.CheckedObject.LogRefs.
+func (r *ShmRefs) LogRefs() bool {
+ return ShmenableLogging
+}
+
+// EnableLeakCheck enables reference leak checking on r.
+func (r *ShmRefs) EnableLeakCheck() {
+ refsvfs2.Register(r)
}
// ReadRefs returns the current number of references. The returned count is
@@ -52,8 +66,10 @@ func (r *ShmRefs) ReadRefs() int64 {
//
//go:nosplit
func (r *ShmRefs) IncRef() {
- if v := atomic.AddInt64(&r.refCount, 1); v <= 0 {
- panic(fmt.Sprintf("Incrementing non-positive count %p on %T", r, ShmownerType))
+ v := atomic.AddInt64(&r.refCount, 1)
+ refsvfs2.LogIncRef(r, v+1)
+ if v <= 0 {
+ panic(fmt.Sprintf("Incrementing non-positive count %p on %s", r, r.RefType()))
}
}
@@ -66,14 +82,14 @@ func (r *ShmRefs) IncRef() {
//go:nosplit
func (r *ShmRefs) TryIncRef() bool {
const speculativeRef = 1 << 32
- v := atomic.AddInt64(&r.refCount, speculativeRef)
- if int32(v) < 0 {
+ if v := atomic.AddInt64(&r.refCount, speculativeRef); int32(v) < 0 {
atomic.AddInt64(&r.refCount, -speculativeRef)
return false
}
- atomic.AddInt64(&r.refCount, -speculativeRef+1)
+ v := atomic.AddInt64(&r.refCount, -speculativeRef+1)
+ refsvfs2.LogTryIncRef(r, v+1)
return true
}
@@ -90,14 +106,14 @@ func (r *ShmRefs) TryIncRef() bool {
//
//go:nosplit
func (r *ShmRefs) DecRef(destroy func()) {
- switch v := atomic.AddInt64(&r.refCount, -1); {
+ v := atomic.AddInt64(&r.refCount, -1)
+ refsvfs2.LogDecRef(r, v+1)
+ switch {
case v < -1:
- panic(fmt.Sprintf("Decrementing non-positive ref count %p, owned by %T", r, ShmownerType))
+ panic(fmt.Sprintf("Decrementing non-positive ref count %p, owned by %s", r, r.RefType()))
case v == -1:
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Unregister(r, fmt.Sprintf("%T", ShmownerType))
- }
+ refsvfs2.Unregister(r)
if destroy != nil {
destroy()
@@ -106,7 +122,7 @@ func (r *ShmRefs) DecRef(destroy func()) {
}
func (r *ShmRefs) afterLoad() {
- if refsvfs2.LeakCheckEnabled() && r.ReadRefs() > 0 {
+ if r.ReadRefs() > 0 {
r.EnableLeakCheck()
}
}
diff --git a/pkg/sentry/mm/aio_mappable_refs.go b/pkg/sentry/mm/aio_mappable_refs.go
index 6a8c753ed..9d94bf879 100644
--- a/pkg/sentry/mm/aio_mappable_refs.go
+++ b/pkg/sentry/mm/aio_mappable_refs.go
@@ -7,9 +7,15 @@ import (
"gvisor.dev/gvisor/pkg/refsvfs2"
)
-// ownerType is used to customize logging. Note that we use a pointer to T so
-// that we do not copy the entire object when passed as a format parameter.
-var aioMappableownerType *aioMappable
+// enableLogging indicates whether reference-related events should be logged (with
+// stack traces). This is false by default and should only be set to true for
+// debugging purposes, as it can generate an extremely large amount of output
+// and drastically degrade performance.
+const aioMappableenableLogging = false
+
+// obj is used to customize logging. Note that we use a pointer to T so that
+// we do not copy the entire object when passed as a format parameter.
+var aioMappableobj *aioMappable
// Refs implements refs.RefCounter. It keeps a reference count using atomic
// operations and calls the destructor when the count reaches zero.
@@ -29,16 +35,24 @@ type aioMappableRefs struct {
refCount int64
}
-// EnableLeakCheck enables reference leak checking on r.
-func (r *aioMappableRefs) EnableLeakCheck() {
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Register(r, fmt.Sprintf("%T", aioMappableownerType))
- }
+// RefType implements refsvfs2.CheckedObject.RefType.
+func (r *aioMappableRefs) RefType() string {
+ return fmt.Sprintf("%T", aioMappableobj)[1:]
}
// LeakMessage implements refsvfs2.CheckedObject.LeakMessage.
func (r *aioMappableRefs) LeakMessage() string {
- return fmt.Sprintf("%T %p: reference count of %d instead of 0", aioMappableownerType, r, r.ReadRefs())
+ return fmt.Sprintf("[%s %p] reference count of %d instead of 0", r.RefType(), r, r.ReadRefs())
+}
+
+// LogRefs implements refsvfs2.CheckedObject.LogRefs.
+func (r *aioMappableRefs) LogRefs() bool {
+ return aioMappableenableLogging
+}
+
+// EnableLeakCheck enables reference leak checking on r.
+func (r *aioMappableRefs) EnableLeakCheck() {
+ refsvfs2.Register(r)
}
// ReadRefs returns the current number of references. The returned count is
@@ -52,8 +66,10 @@ func (r *aioMappableRefs) ReadRefs() int64 {
//
//go:nosplit
func (r *aioMappableRefs) IncRef() {
- if v := atomic.AddInt64(&r.refCount, 1); v <= 0 {
- panic(fmt.Sprintf("Incrementing non-positive count %p on %T", r, aioMappableownerType))
+ v := atomic.AddInt64(&r.refCount, 1)
+ refsvfs2.LogIncRef(r, v+1)
+ if v <= 0 {
+ panic(fmt.Sprintf("Incrementing non-positive count %p on %s", r, r.RefType()))
}
}
@@ -66,14 +82,14 @@ func (r *aioMappableRefs) IncRef() {
//go:nosplit
func (r *aioMappableRefs) TryIncRef() bool {
const speculativeRef = 1 << 32
- v := atomic.AddInt64(&r.refCount, speculativeRef)
- if int32(v) < 0 {
+ if v := atomic.AddInt64(&r.refCount, speculativeRef); int32(v) < 0 {
atomic.AddInt64(&r.refCount, -speculativeRef)
return false
}
- atomic.AddInt64(&r.refCount, -speculativeRef+1)
+ v := atomic.AddInt64(&r.refCount, -speculativeRef+1)
+ refsvfs2.LogTryIncRef(r, v+1)
return true
}
@@ -90,14 +106,14 @@ func (r *aioMappableRefs) TryIncRef() bool {
//
//go:nosplit
func (r *aioMappableRefs) DecRef(destroy func()) {
- switch v := atomic.AddInt64(&r.refCount, -1); {
+ v := atomic.AddInt64(&r.refCount, -1)
+ refsvfs2.LogDecRef(r, v+1)
+ switch {
case v < -1:
- panic(fmt.Sprintf("Decrementing non-positive ref count %p, owned by %T", r, aioMappableownerType))
+ panic(fmt.Sprintf("Decrementing non-positive ref count %p, owned by %s", r, r.RefType()))
case v == -1:
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Unregister(r, fmt.Sprintf("%T", aioMappableownerType))
- }
+ refsvfs2.Unregister(r)
if destroy != nil {
destroy()
@@ -106,7 +122,7 @@ func (r *aioMappableRefs) DecRef(destroy func()) {
}
func (r *aioMappableRefs) afterLoad() {
- if refsvfs2.LeakCheckEnabled() && r.ReadRefs() > 0 {
+ if r.ReadRefs() > 0 {
r.EnableLeakCheck()
}
}
diff --git a/pkg/sentry/mm/special_mappable_refs.go b/pkg/sentry/mm/special_mappable_refs.go
index aa75939ea..f17d4361f 100644
--- a/pkg/sentry/mm/special_mappable_refs.go
+++ b/pkg/sentry/mm/special_mappable_refs.go
@@ -7,9 +7,15 @@ import (
"gvisor.dev/gvisor/pkg/refsvfs2"
)
-// ownerType is used to customize logging. Note that we use a pointer to T so
-// that we do not copy the entire object when passed as a format parameter.
-var SpecialMappableownerType *SpecialMappable
+// enableLogging indicates whether reference-related events should be logged (with
+// stack traces). This is false by default and should only be set to true for
+// debugging purposes, as it can generate an extremely large amount of output
+// and drastically degrade performance.
+const SpecialMappableenableLogging = false
+
+// obj is used to customize logging. Note that we use a pointer to T so that
+// we do not copy the entire object when passed as a format parameter.
+var SpecialMappableobj *SpecialMappable
// Refs implements refs.RefCounter. It keeps a reference count using atomic
// operations and calls the destructor when the count reaches zero.
@@ -29,16 +35,24 @@ type SpecialMappableRefs struct {
refCount int64
}
-// EnableLeakCheck enables reference leak checking on r.
-func (r *SpecialMappableRefs) EnableLeakCheck() {
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Register(r, fmt.Sprintf("%T", SpecialMappableownerType))
- }
+// RefType implements refsvfs2.CheckedObject.RefType.
+func (r *SpecialMappableRefs) RefType() string {
+ return fmt.Sprintf("%T", SpecialMappableobj)[1:]
}
// LeakMessage implements refsvfs2.CheckedObject.LeakMessage.
func (r *SpecialMappableRefs) LeakMessage() string {
- return fmt.Sprintf("%T %p: reference count of %d instead of 0", SpecialMappableownerType, r, r.ReadRefs())
+ return fmt.Sprintf("[%s %p] reference count of %d instead of 0", r.RefType(), r, r.ReadRefs())
+}
+
+// LogRefs implements refsvfs2.CheckedObject.LogRefs.
+func (r *SpecialMappableRefs) LogRefs() bool {
+ return SpecialMappableenableLogging
+}
+
+// EnableLeakCheck enables reference leak checking on r.
+func (r *SpecialMappableRefs) EnableLeakCheck() {
+ refsvfs2.Register(r)
}
// ReadRefs returns the current number of references. The returned count is
@@ -52,8 +66,10 @@ func (r *SpecialMappableRefs) ReadRefs() int64 {
//
//go:nosplit
func (r *SpecialMappableRefs) IncRef() {
- if v := atomic.AddInt64(&r.refCount, 1); v <= 0 {
- panic(fmt.Sprintf("Incrementing non-positive count %p on %T", r, SpecialMappableownerType))
+ v := atomic.AddInt64(&r.refCount, 1)
+ refsvfs2.LogIncRef(r, v+1)
+ if v <= 0 {
+ panic(fmt.Sprintf("Incrementing non-positive count %p on %s", r, r.RefType()))
}
}
@@ -66,14 +82,14 @@ func (r *SpecialMappableRefs) IncRef() {
//go:nosplit
func (r *SpecialMappableRefs) TryIncRef() bool {
const speculativeRef = 1 << 32
- v := atomic.AddInt64(&r.refCount, speculativeRef)
- if int32(v) < 0 {
+ if v := atomic.AddInt64(&r.refCount, speculativeRef); int32(v) < 0 {
atomic.AddInt64(&r.refCount, -speculativeRef)
return false
}
- atomic.AddInt64(&r.refCount, -speculativeRef+1)
+ v := atomic.AddInt64(&r.refCount, -speculativeRef+1)
+ refsvfs2.LogTryIncRef(r, v+1)
return true
}
@@ -90,14 +106,14 @@ func (r *SpecialMappableRefs) TryIncRef() bool {
//
//go:nosplit
func (r *SpecialMappableRefs) DecRef(destroy func()) {
- switch v := atomic.AddInt64(&r.refCount, -1); {
+ v := atomic.AddInt64(&r.refCount, -1)
+ refsvfs2.LogDecRef(r, v+1)
+ switch {
case v < -1:
- panic(fmt.Sprintf("Decrementing non-positive ref count %p, owned by %T", r, SpecialMappableownerType))
+ panic(fmt.Sprintf("Decrementing non-positive ref count %p, owned by %s", r, r.RefType()))
case v == -1:
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Unregister(r, fmt.Sprintf("%T", SpecialMappableownerType))
- }
+ refsvfs2.Unregister(r)
if destroy != nil {
destroy()
@@ -106,7 +122,7 @@ func (r *SpecialMappableRefs) DecRef(destroy func()) {
}
func (r *SpecialMappableRefs) afterLoad() {
- if refsvfs2.LeakCheckEnabled() && r.ReadRefs() > 0 {
+ if r.ReadRefs() > 0 {
r.EnableLeakCheck()
}
}
diff --git a/pkg/sentry/socket/unix/socket_refs.go b/pkg/sentry/socket/unix/socket_refs.go
index 45b7c77d5..aec464529 100644
--- a/pkg/sentry/socket/unix/socket_refs.go
+++ b/pkg/sentry/socket/unix/socket_refs.go
@@ -7,9 +7,15 @@ import (
"gvisor.dev/gvisor/pkg/refsvfs2"
)
-// ownerType is used to customize logging. Note that we use a pointer to T so
-// that we do not copy the entire object when passed as a format parameter.
-var socketOperationsownerType *SocketOperations
+// enableLogging indicates whether reference-related events should be logged (with
+// stack traces). This is false by default and should only be set to true for
+// debugging purposes, as it can generate an extremely large amount of output
+// and drastically degrade performance.
+const socketOperationsenableLogging = false
+
+// obj is used to customize logging. Note that we use a pointer to T so that
+// we do not copy the entire object when passed as a format parameter.
+var socketOperationsobj *SocketOperations
// Refs implements refs.RefCounter. It keeps a reference count using atomic
// operations and calls the destructor when the count reaches zero.
@@ -29,16 +35,24 @@ type socketOperationsRefs struct {
refCount int64
}
-// EnableLeakCheck enables reference leak checking on r.
-func (r *socketOperationsRefs) EnableLeakCheck() {
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Register(r, fmt.Sprintf("%T", socketOperationsownerType))
- }
+// RefType implements refsvfs2.CheckedObject.RefType.
+func (r *socketOperationsRefs) RefType() string {
+ return fmt.Sprintf("%T", socketOperationsobj)[1:]
}
// LeakMessage implements refsvfs2.CheckedObject.LeakMessage.
func (r *socketOperationsRefs) LeakMessage() string {
- return fmt.Sprintf("%T %p: reference count of %d instead of 0", socketOperationsownerType, r, r.ReadRefs())
+ return fmt.Sprintf("[%s %p] reference count of %d instead of 0", r.RefType(), r, r.ReadRefs())
+}
+
+// LogRefs implements refsvfs2.CheckedObject.LogRefs.
+func (r *socketOperationsRefs) LogRefs() bool {
+ return socketOperationsenableLogging
+}
+
+// EnableLeakCheck enables reference leak checking on r.
+func (r *socketOperationsRefs) EnableLeakCheck() {
+ refsvfs2.Register(r)
}
// ReadRefs returns the current number of references. The returned count is
@@ -52,8 +66,10 @@ func (r *socketOperationsRefs) ReadRefs() int64 {
//
//go:nosplit
func (r *socketOperationsRefs) IncRef() {
- if v := atomic.AddInt64(&r.refCount, 1); v <= 0 {
- panic(fmt.Sprintf("Incrementing non-positive count %p on %T", r, socketOperationsownerType))
+ v := atomic.AddInt64(&r.refCount, 1)
+ refsvfs2.LogIncRef(r, v+1)
+ if v <= 0 {
+ panic(fmt.Sprintf("Incrementing non-positive count %p on %s", r, r.RefType()))
}
}
@@ -66,14 +82,14 @@ func (r *socketOperationsRefs) IncRef() {
//go:nosplit
func (r *socketOperationsRefs) TryIncRef() bool {
const speculativeRef = 1 << 32
- v := atomic.AddInt64(&r.refCount, speculativeRef)
- if int32(v) < 0 {
+ if v := atomic.AddInt64(&r.refCount, speculativeRef); int32(v) < 0 {
atomic.AddInt64(&r.refCount, -speculativeRef)
return false
}
- atomic.AddInt64(&r.refCount, -speculativeRef+1)
+ v := atomic.AddInt64(&r.refCount, -speculativeRef+1)
+ refsvfs2.LogTryIncRef(r, v+1)
return true
}
@@ -90,14 +106,14 @@ func (r *socketOperationsRefs) TryIncRef() bool {
//
//go:nosplit
func (r *socketOperationsRefs) DecRef(destroy func()) {
- switch v := atomic.AddInt64(&r.refCount, -1); {
+ v := atomic.AddInt64(&r.refCount, -1)
+ refsvfs2.LogDecRef(r, v+1)
+ switch {
case v < -1:
- panic(fmt.Sprintf("Decrementing non-positive ref count %p, owned by %T", r, socketOperationsownerType))
+ panic(fmt.Sprintf("Decrementing non-positive ref count %p, owned by %s", r, r.RefType()))
case v == -1:
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Unregister(r, fmt.Sprintf("%T", socketOperationsownerType))
- }
+ refsvfs2.Unregister(r)
if destroy != nil {
destroy()
@@ -106,7 +122,7 @@ func (r *socketOperationsRefs) DecRef(destroy func()) {
}
func (r *socketOperationsRefs) afterLoad() {
- if refsvfs2.LeakCheckEnabled() && r.ReadRefs() > 0 {
+ if r.ReadRefs() > 0 {
r.EnableLeakCheck()
}
}
diff --git a/pkg/sentry/socket/unix/socket_vfs2_refs.go b/pkg/sentry/socket/unix/socket_vfs2_refs.go
index 479dd5ef0..8794375b3 100644
--- a/pkg/sentry/socket/unix/socket_vfs2_refs.go
+++ b/pkg/sentry/socket/unix/socket_vfs2_refs.go
@@ -7,9 +7,15 @@ import (
"gvisor.dev/gvisor/pkg/refsvfs2"
)
-// ownerType is used to customize logging. Note that we use a pointer to T so
-// that we do not copy the entire object when passed as a format parameter.
-var socketVFS2ownerType *SocketVFS2
+// enableLogging indicates whether reference-related events should be logged (with
+// stack traces). This is false by default and should only be set to true for
+// debugging purposes, as it can generate an extremely large amount of output
+// and drastically degrade performance.
+const socketVFS2enableLogging = false
+
+// obj is used to customize logging. Note that we use a pointer to T so that
+// we do not copy the entire object when passed as a format parameter.
+var socketVFS2obj *SocketVFS2
// Refs implements refs.RefCounter. It keeps a reference count using atomic
// operations and calls the destructor when the count reaches zero.
@@ -29,16 +35,24 @@ type socketVFS2Refs struct {
refCount int64
}
-// EnableLeakCheck enables reference leak checking on r.
-func (r *socketVFS2Refs) EnableLeakCheck() {
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Register(r, fmt.Sprintf("%T", socketVFS2ownerType))
- }
+// RefType implements refsvfs2.CheckedObject.RefType.
+func (r *socketVFS2Refs) RefType() string {
+ return fmt.Sprintf("%T", socketVFS2obj)[1:]
}
// LeakMessage implements refsvfs2.CheckedObject.LeakMessage.
func (r *socketVFS2Refs) LeakMessage() string {
- return fmt.Sprintf("%T %p: reference count of %d instead of 0", socketVFS2ownerType, r, r.ReadRefs())
+ return fmt.Sprintf("[%s %p] reference count of %d instead of 0", r.RefType(), r, r.ReadRefs())
+}
+
+// LogRefs implements refsvfs2.CheckedObject.LogRefs.
+func (r *socketVFS2Refs) LogRefs() bool {
+ return socketVFS2enableLogging
+}
+
+// EnableLeakCheck enables reference leak checking on r.
+func (r *socketVFS2Refs) EnableLeakCheck() {
+ refsvfs2.Register(r)
}
// ReadRefs returns the current number of references. The returned count is
@@ -52,8 +66,10 @@ func (r *socketVFS2Refs) ReadRefs() int64 {
//
//go:nosplit
func (r *socketVFS2Refs) IncRef() {
- if v := atomic.AddInt64(&r.refCount, 1); v <= 0 {
- panic(fmt.Sprintf("Incrementing non-positive count %p on %T", r, socketVFS2ownerType))
+ v := atomic.AddInt64(&r.refCount, 1)
+ refsvfs2.LogIncRef(r, v+1)
+ if v <= 0 {
+ panic(fmt.Sprintf("Incrementing non-positive count %p on %s", r, r.RefType()))
}
}
@@ -66,14 +82,14 @@ func (r *socketVFS2Refs) IncRef() {
//go:nosplit
func (r *socketVFS2Refs) TryIncRef() bool {
const speculativeRef = 1 << 32
- v := atomic.AddInt64(&r.refCount, speculativeRef)
- if int32(v) < 0 {
+ if v := atomic.AddInt64(&r.refCount, speculativeRef); int32(v) < 0 {
atomic.AddInt64(&r.refCount, -speculativeRef)
return false
}
- atomic.AddInt64(&r.refCount, -speculativeRef+1)
+ v := atomic.AddInt64(&r.refCount, -speculativeRef+1)
+ refsvfs2.LogTryIncRef(r, v+1)
return true
}
@@ -90,14 +106,14 @@ func (r *socketVFS2Refs) TryIncRef() bool {
//
//go:nosplit
func (r *socketVFS2Refs) DecRef(destroy func()) {
- switch v := atomic.AddInt64(&r.refCount, -1); {
+ v := atomic.AddInt64(&r.refCount, -1)
+ refsvfs2.LogDecRef(r, v+1)
+ switch {
case v < -1:
- panic(fmt.Sprintf("Decrementing non-positive ref count %p, owned by %T", r, socketVFS2ownerType))
+ panic(fmt.Sprintf("Decrementing non-positive ref count %p, owned by %s", r, r.RefType()))
case v == -1:
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Unregister(r, fmt.Sprintf("%T", socketVFS2ownerType))
- }
+ refsvfs2.Unregister(r)
if destroy != nil {
destroy()
@@ -106,7 +122,7 @@ func (r *socketVFS2Refs) DecRef(destroy func()) {
}
func (r *socketVFS2Refs) afterLoad() {
- if refsvfs2.LeakCheckEnabled() && r.ReadRefs() > 0 {
+ if r.ReadRefs() > 0 {
r.EnableLeakCheck()
}
}
diff --git a/pkg/sentry/socket/unix/transport/queue_refs.go b/pkg/sentry/socket/unix/transport/queue_refs.go
index de3bb9270..ec67d4b14 100644
--- a/pkg/sentry/socket/unix/transport/queue_refs.go
+++ b/pkg/sentry/socket/unix/transport/queue_refs.go
@@ -7,9 +7,15 @@ import (
"gvisor.dev/gvisor/pkg/refsvfs2"
)
-// ownerType is used to customize logging. Note that we use a pointer to T so
-// that we do not copy the entire object when passed as a format parameter.
-var queueownerType *queue
+// enableLogging indicates whether reference-related events should be logged (with
+// stack traces). This is false by default and should only be set to true for
+// debugging purposes, as it can generate an extremely large amount of output
+// and drastically degrade performance.
+const queueenableLogging = false
+
+// obj is used to customize logging. Note that we use a pointer to T so that
+// we do not copy the entire object when passed as a format parameter.
+var queueobj *queue
// Refs implements refs.RefCounter. It keeps a reference count using atomic
// operations and calls the destructor when the count reaches zero.
@@ -29,16 +35,24 @@ type queueRefs struct {
refCount int64
}
-// EnableLeakCheck enables reference leak checking on r.
-func (r *queueRefs) EnableLeakCheck() {
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Register(r, fmt.Sprintf("%T", queueownerType))
- }
+// RefType implements refsvfs2.CheckedObject.RefType.
+func (r *queueRefs) RefType() string {
+ return fmt.Sprintf("%T", queueobj)[1:]
}
// LeakMessage implements refsvfs2.CheckedObject.LeakMessage.
func (r *queueRefs) LeakMessage() string {
- return fmt.Sprintf("%T %p: reference count of %d instead of 0", queueownerType, r, r.ReadRefs())
+ return fmt.Sprintf("[%s %p] reference count of %d instead of 0", r.RefType(), r, r.ReadRefs())
+}
+
+// LogRefs implements refsvfs2.CheckedObject.LogRefs.
+func (r *queueRefs) LogRefs() bool {
+ return queueenableLogging
+}
+
+// EnableLeakCheck enables reference leak checking on r.
+func (r *queueRefs) EnableLeakCheck() {
+ refsvfs2.Register(r)
}
// ReadRefs returns the current number of references. The returned count is
@@ -52,8 +66,10 @@ func (r *queueRefs) ReadRefs() int64 {
//
//go:nosplit
func (r *queueRefs) IncRef() {
- if v := atomic.AddInt64(&r.refCount, 1); v <= 0 {
- panic(fmt.Sprintf("Incrementing non-positive count %p on %T", r, queueownerType))
+ v := atomic.AddInt64(&r.refCount, 1)
+ refsvfs2.LogIncRef(r, v+1)
+ if v <= 0 {
+ panic(fmt.Sprintf("Incrementing non-positive count %p on %s", r, r.RefType()))
}
}
@@ -66,14 +82,14 @@ func (r *queueRefs) IncRef() {
//go:nosplit
func (r *queueRefs) TryIncRef() bool {
const speculativeRef = 1 << 32
- v := atomic.AddInt64(&r.refCount, speculativeRef)
- if int32(v) < 0 {
+ if v := atomic.AddInt64(&r.refCount, speculativeRef); int32(v) < 0 {
atomic.AddInt64(&r.refCount, -speculativeRef)
return false
}
- atomic.AddInt64(&r.refCount, -speculativeRef+1)
+ v := atomic.AddInt64(&r.refCount, -speculativeRef+1)
+ refsvfs2.LogTryIncRef(r, v+1)
return true
}
@@ -90,14 +106,14 @@ func (r *queueRefs) TryIncRef() bool {
//
//go:nosplit
func (r *queueRefs) DecRef(destroy func()) {
- switch v := atomic.AddInt64(&r.refCount, -1); {
+ v := atomic.AddInt64(&r.refCount, -1)
+ refsvfs2.LogDecRef(r, v+1)
+ switch {
case v < -1:
- panic(fmt.Sprintf("Decrementing non-positive ref count %p, owned by %T", r, queueownerType))
+ panic(fmt.Sprintf("Decrementing non-positive ref count %p, owned by %s", r, r.RefType()))
case v == -1:
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Unregister(r, fmt.Sprintf("%T", queueownerType))
- }
+ refsvfs2.Unregister(r)
if destroy != nil {
destroy()
@@ -106,7 +122,7 @@ func (r *queueRefs) DecRef(destroy func()) {
}
func (r *queueRefs) afterLoad() {
- if refsvfs2.LeakCheckEnabled() && r.ReadRefs() > 0 {
+ if r.ReadRefs() > 0 {
r.EnableLeakCheck()
}
}
diff --git a/pkg/sentry/vfs/file_description_refs.go b/pkg/sentry/vfs/file_description_refs.go
index 951cfbd4d..5cc5fe104 100644
--- a/pkg/sentry/vfs/file_description_refs.go
+++ b/pkg/sentry/vfs/file_description_refs.go
@@ -7,9 +7,15 @@ import (
"gvisor.dev/gvisor/pkg/refsvfs2"
)
-// ownerType is used to customize logging. Note that we use a pointer to T so
-// that we do not copy the entire object when passed as a format parameter.
-var FileDescriptionownerType *FileDescription
+// enableLogging indicates whether reference-related events should be logged (with
+// stack traces). This is false by default and should only be set to true for
+// debugging purposes, as it can generate an extremely large amount of output
+// and drastically degrade performance.
+const FileDescriptionenableLogging = false
+
+// obj is used to customize logging. Note that we use a pointer to T so that
+// we do not copy the entire object when passed as a format parameter.
+var FileDescriptionobj *FileDescription
// Refs implements refs.RefCounter. It keeps a reference count using atomic
// operations and calls the destructor when the count reaches zero.
@@ -29,16 +35,24 @@ type FileDescriptionRefs struct {
refCount int64
}
-// EnableLeakCheck enables reference leak checking on r.
-func (r *FileDescriptionRefs) EnableLeakCheck() {
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Register(r, fmt.Sprintf("%T", FileDescriptionownerType))
- }
+// RefType implements refsvfs2.CheckedObject.RefType.
+func (r *FileDescriptionRefs) RefType() string {
+ return fmt.Sprintf("%T", FileDescriptionobj)[1:]
}
// LeakMessage implements refsvfs2.CheckedObject.LeakMessage.
func (r *FileDescriptionRefs) LeakMessage() string {
- return fmt.Sprintf("%T %p: reference count of %d instead of 0", FileDescriptionownerType, r, r.ReadRefs())
+ return fmt.Sprintf("[%s %p] reference count of %d instead of 0", r.RefType(), r, r.ReadRefs())
+}
+
+// LogRefs implements refsvfs2.CheckedObject.LogRefs.
+func (r *FileDescriptionRefs) LogRefs() bool {
+ return FileDescriptionenableLogging
+}
+
+// EnableLeakCheck enables reference leak checking on r.
+func (r *FileDescriptionRefs) EnableLeakCheck() {
+ refsvfs2.Register(r)
}
// ReadRefs returns the current number of references. The returned count is
@@ -52,8 +66,10 @@ func (r *FileDescriptionRefs) ReadRefs() int64 {
//
//go:nosplit
func (r *FileDescriptionRefs) IncRef() {
- if v := atomic.AddInt64(&r.refCount, 1); v <= 0 {
- panic(fmt.Sprintf("Incrementing non-positive count %p on %T", r, FileDescriptionownerType))
+ v := atomic.AddInt64(&r.refCount, 1)
+ refsvfs2.LogIncRef(r, v+1)
+ if v <= 0 {
+ panic(fmt.Sprintf("Incrementing non-positive count %p on %s", r, r.RefType()))
}
}
@@ -66,14 +82,14 @@ func (r *FileDescriptionRefs) IncRef() {
//go:nosplit
func (r *FileDescriptionRefs) TryIncRef() bool {
const speculativeRef = 1 << 32
- v := atomic.AddInt64(&r.refCount, speculativeRef)
- if int32(v) < 0 {
+ if v := atomic.AddInt64(&r.refCount, speculativeRef); int32(v) < 0 {
atomic.AddInt64(&r.refCount, -speculativeRef)
return false
}
- atomic.AddInt64(&r.refCount, -speculativeRef+1)
+ v := atomic.AddInt64(&r.refCount, -speculativeRef+1)
+ refsvfs2.LogTryIncRef(r, v+1)
return true
}
@@ -90,14 +106,14 @@ func (r *FileDescriptionRefs) TryIncRef() bool {
//
//go:nosplit
func (r *FileDescriptionRefs) DecRef(destroy func()) {
- switch v := atomic.AddInt64(&r.refCount, -1); {
+ v := atomic.AddInt64(&r.refCount, -1)
+ refsvfs2.LogDecRef(r, v+1)
+ switch {
case v < -1:
- panic(fmt.Sprintf("Decrementing non-positive ref count %p, owned by %T", r, FileDescriptionownerType))
+ panic(fmt.Sprintf("Decrementing non-positive ref count %p, owned by %s", r, r.RefType()))
case v == -1:
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Unregister(r, fmt.Sprintf("%T", FileDescriptionownerType))
- }
+ refsvfs2.Unregister(r)
if destroy != nil {
destroy()
@@ -106,7 +122,7 @@ func (r *FileDescriptionRefs) DecRef(destroy func()) {
}
func (r *FileDescriptionRefs) afterLoad() {
- if refsvfs2.LeakCheckEnabled() && r.ReadRefs() > 0 {
+ if r.ReadRefs() > 0 {
r.EnableLeakCheck()
}
}
diff --git a/pkg/sentry/vfs/filesystem_refs.go b/pkg/sentry/vfs/filesystem_refs.go
index f1abc120d..6b403c04a 100644
--- a/pkg/sentry/vfs/filesystem_refs.go
+++ b/pkg/sentry/vfs/filesystem_refs.go
@@ -7,9 +7,15 @@ import (
"gvisor.dev/gvisor/pkg/refsvfs2"
)
-// ownerType is used to customize logging. Note that we use a pointer to T so
-// that we do not copy the entire object when passed as a format parameter.
-var FilesystemownerType *Filesystem
+// enableLogging indicates whether reference-related events should be logged (with
+// stack traces). This is false by default and should only be set to true for
+// debugging purposes, as it can generate an extremely large amount of output
+// and drastically degrade performance.
+const FilesystemenableLogging = false
+
+// obj is used to customize logging. Note that we use a pointer to T so that
+// we do not copy the entire object when passed as a format parameter.
+var Filesystemobj *Filesystem
// Refs implements refs.RefCounter. It keeps a reference count using atomic
// operations and calls the destructor when the count reaches zero.
@@ -29,16 +35,24 @@ type FilesystemRefs struct {
refCount int64
}
-// EnableLeakCheck enables reference leak checking on r.
-func (r *FilesystemRefs) EnableLeakCheck() {
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Register(r, fmt.Sprintf("%T", FilesystemownerType))
- }
+// RefType implements refsvfs2.CheckedObject.RefType.
+func (r *FilesystemRefs) RefType() string {
+ return fmt.Sprintf("%T", Filesystemobj)[1:]
}
// LeakMessage implements refsvfs2.CheckedObject.LeakMessage.
func (r *FilesystemRefs) LeakMessage() string {
- return fmt.Sprintf("%T %p: reference count of %d instead of 0", FilesystemownerType, r, r.ReadRefs())
+ return fmt.Sprintf("[%s %p] reference count of %d instead of 0", r.RefType(), r, r.ReadRefs())
+}
+
+// LogRefs implements refsvfs2.CheckedObject.LogRefs.
+func (r *FilesystemRefs) LogRefs() bool {
+ return FilesystemenableLogging
+}
+
+// EnableLeakCheck enables reference leak checking on r.
+func (r *FilesystemRefs) EnableLeakCheck() {
+ refsvfs2.Register(r)
}
// ReadRefs returns the current number of references. The returned count is
@@ -52,8 +66,10 @@ func (r *FilesystemRefs) ReadRefs() int64 {
//
//go:nosplit
func (r *FilesystemRefs) IncRef() {
- if v := atomic.AddInt64(&r.refCount, 1); v <= 0 {
- panic(fmt.Sprintf("Incrementing non-positive count %p on %T", r, FilesystemownerType))
+ v := atomic.AddInt64(&r.refCount, 1)
+ refsvfs2.LogIncRef(r, v+1)
+ if v <= 0 {
+ panic(fmt.Sprintf("Incrementing non-positive count %p on %s", r, r.RefType()))
}
}
@@ -66,14 +82,14 @@ func (r *FilesystemRefs) IncRef() {
//go:nosplit
func (r *FilesystemRefs) TryIncRef() bool {
const speculativeRef = 1 << 32
- v := atomic.AddInt64(&r.refCount, speculativeRef)
- if int32(v) < 0 {
+ if v := atomic.AddInt64(&r.refCount, speculativeRef); int32(v) < 0 {
atomic.AddInt64(&r.refCount, -speculativeRef)
return false
}
- atomic.AddInt64(&r.refCount, -speculativeRef+1)
+ v := atomic.AddInt64(&r.refCount, -speculativeRef+1)
+ refsvfs2.LogTryIncRef(r, v+1)
return true
}
@@ -90,14 +106,14 @@ func (r *FilesystemRefs) TryIncRef() bool {
//
//go:nosplit
func (r *FilesystemRefs) DecRef(destroy func()) {
- switch v := atomic.AddInt64(&r.refCount, -1); {
+ v := atomic.AddInt64(&r.refCount, -1)
+ refsvfs2.LogDecRef(r, v+1)
+ switch {
case v < -1:
- panic(fmt.Sprintf("Decrementing non-positive ref count %p, owned by %T", r, FilesystemownerType))
+ panic(fmt.Sprintf("Decrementing non-positive ref count %p, owned by %s", r, r.RefType()))
case v == -1:
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Unregister(r, fmt.Sprintf("%T", FilesystemownerType))
- }
+ refsvfs2.Unregister(r)
if destroy != nil {
destroy()
@@ -106,7 +122,7 @@ func (r *FilesystemRefs) DecRef(destroy func()) {
}
func (r *FilesystemRefs) afterLoad() {
- if refsvfs2.LeakCheckEnabled() && r.ReadRefs() > 0 {
+ if r.ReadRefs() > 0 {
r.EnableLeakCheck()
}
}
diff --git a/pkg/sentry/vfs/mount.go b/pkg/sentry/vfs/mount.go
index d452d2cda..3ea981ad4 100644
--- a/pkg/sentry/vfs/mount.go
+++ b/pkg/sentry/vfs/mount.go
@@ -107,9 +107,7 @@ func newMount(vfs *VirtualFilesystem, fs *Filesystem, root *Dentry, mntns *Mount
if opts.ReadOnly {
mnt.setReadOnlyLocked(true)
}
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Register(mnt, "vfs.Mount")
- }
+ refsvfs2.Register(mnt)
return mnt
}
@@ -474,11 +472,12 @@ func (vfs *VirtualFilesystem) disconnectLocked(mnt *Mount) VirtualDentry {
// tryIncMountedRef does not require that a reference is held on mnt.
func (mnt *Mount) tryIncMountedRef() bool {
for {
- refs := atomic.LoadInt64(&mnt.refs)
- if refs <= 0 { // refs < 0 => MSB set => eagerly unmounted
+ r := atomic.LoadInt64(&mnt.refs)
+ if r <= 0 { // r < 0 => MSB set => eagerly unmounted
return false
}
- if atomic.CompareAndSwapInt64(&mnt.refs, refs, refs+1) {
+ if atomic.CompareAndSwapInt64(&mnt.refs, r, r+1) {
+ refsvfs2.LogTryIncRef(mnt, r+1)
return true
}
}
@@ -488,16 +487,15 @@ func (mnt *Mount) tryIncMountedRef() bool {
func (mnt *Mount) IncRef() {
// In general, negative values for mnt.refs are valid because the MSB is
// the eager-unmount bit.
- atomic.AddInt64(&mnt.refs, 1)
+ r := atomic.AddInt64(&mnt.refs, 1)
+ refsvfs2.LogIncRef(mnt, r)
}
// DecRef decrements mnt's reference count.
func (mnt *Mount) DecRef(ctx context.Context) {
r := atomic.AddInt64(&mnt.refs, -1)
if r&^math.MinInt64 == 0 { // mask out MSB
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Unregister(mnt, "vfs.Mount")
- }
+ refsvfs2.Unregister(mnt)
mnt.destroy(ctx)
}
}
@@ -520,11 +518,24 @@ func (mnt *Mount) destroy(ctx context.Context) {
}
}
+// RefType implements refsvfs2.CheckedObject.Type.
+func (mnt *Mount) RefType() string {
+ return "vfs.Mount"
+}
+
// LeakMessage implements refsvfs2.CheckedObject.LeakMessage.
func (mnt *Mount) LeakMessage() string {
return fmt.Sprintf("[vfs.Mount %p] reference count of %d instead of 0", mnt, atomic.LoadInt64(&mnt.refs))
}
+// LogRefs implements refsvfs2.CheckedObject.LogRefs.
+//
+// This should only be set to true for debugging purposes, as it can generate an
+// extremely large amount of output and drastically degrade performance.
+func (mnt *Mount) LogRefs() bool {
+ return false
+}
+
// DecRef decrements mntns' reference count.
func (mntns *MountNamespace) DecRef(ctx context.Context) {
vfs := mntns.root.fs.VirtualFilesystem()
diff --git a/pkg/sentry/vfs/mount_namespace_refs.go b/pkg/sentry/vfs/mount_namespace_refs.go
index 32e28ebf8..7c39d445f 100644
--- a/pkg/sentry/vfs/mount_namespace_refs.go
+++ b/pkg/sentry/vfs/mount_namespace_refs.go
@@ -7,9 +7,15 @@ import (
"gvisor.dev/gvisor/pkg/refsvfs2"
)
-// ownerType is used to customize logging. Note that we use a pointer to T so
-// that we do not copy the entire object when passed as a format parameter.
-var MountNamespaceownerType *MountNamespace
+// enableLogging indicates whether reference-related events should be logged (with
+// stack traces). This is false by default and should only be set to true for
+// debugging purposes, as it can generate an extremely large amount of output
+// and drastically degrade performance.
+const MountNamespaceenableLogging = false
+
+// obj is used to customize logging. Note that we use a pointer to T so that
+// we do not copy the entire object when passed as a format parameter.
+var MountNamespaceobj *MountNamespace
// Refs implements refs.RefCounter. It keeps a reference count using atomic
// operations and calls the destructor when the count reaches zero.
@@ -29,16 +35,24 @@ type MountNamespaceRefs struct {
refCount int64
}
-// EnableLeakCheck enables reference leak checking on r.
-func (r *MountNamespaceRefs) EnableLeakCheck() {
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Register(r, fmt.Sprintf("%T", MountNamespaceownerType))
- }
+// RefType implements refsvfs2.CheckedObject.RefType.
+func (r *MountNamespaceRefs) RefType() string {
+ return fmt.Sprintf("%T", MountNamespaceobj)[1:]
}
// LeakMessage implements refsvfs2.CheckedObject.LeakMessage.
func (r *MountNamespaceRefs) LeakMessage() string {
- return fmt.Sprintf("%T %p: reference count of %d instead of 0", MountNamespaceownerType, r, r.ReadRefs())
+ return fmt.Sprintf("[%s %p] reference count of %d instead of 0", r.RefType(), r, r.ReadRefs())
+}
+
+// LogRefs implements refsvfs2.CheckedObject.LogRefs.
+func (r *MountNamespaceRefs) LogRefs() bool {
+ return MountNamespaceenableLogging
+}
+
+// EnableLeakCheck enables reference leak checking on r.
+func (r *MountNamespaceRefs) EnableLeakCheck() {
+ refsvfs2.Register(r)
}
// ReadRefs returns the current number of references. The returned count is
@@ -52,8 +66,10 @@ func (r *MountNamespaceRefs) ReadRefs() int64 {
//
//go:nosplit
func (r *MountNamespaceRefs) IncRef() {
- if v := atomic.AddInt64(&r.refCount, 1); v <= 0 {
- panic(fmt.Sprintf("Incrementing non-positive count %p on %T", r, MountNamespaceownerType))
+ v := atomic.AddInt64(&r.refCount, 1)
+ refsvfs2.LogIncRef(r, v+1)
+ if v <= 0 {
+ panic(fmt.Sprintf("Incrementing non-positive count %p on %s", r, r.RefType()))
}
}
@@ -66,14 +82,14 @@ func (r *MountNamespaceRefs) IncRef() {
//go:nosplit
func (r *MountNamespaceRefs) TryIncRef() bool {
const speculativeRef = 1 << 32
- v := atomic.AddInt64(&r.refCount, speculativeRef)
- if int32(v) < 0 {
+ if v := atomic.AddInt64(&r.refCount, speculativeRef); int32(v) < 0 {
atomic.AddInt64(&r.refCount, -speculativeRef)
return false
}
- atomic.AddInt64(&r.refCount, -speculativeRef+1)
+ v := atomic.AddInt64(&r.refCount, -speculativeRef+1)
+ refsvfs2.LogTryIncRef(r, v+1)
return true
}
@@ -90,14 +106,14 @@ func (r *MountNamespaceRefs) TryIncRef() bool {
//
//go:nosplit
func (r *MountNamespaceRefs) DecRef(destroy func()) {
- switch v := atomic.AddInt64(&r.refCount, -1); {
+ v := atomic.AddInt64(&r.refCount, -1)
+ refsvfs2.LogDecRef(r, v+1)
+ switch {
case v < -1:
- panic(fmt.Sprintf("Decrementing non-positive ref count %p, owned by %T", r, MountNamespaceownerType))
+ panic(fmt.Sprintf("Decrementing non-positive ref count %p, owned by %s", r, r.RefType()))
case v == -1:
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Unregister(r, fmt.Sprintf("%T", MountNamespaceownerType))
- }
+ refsvfs2.Unregister(r)
if destroy != nil {
destroy()
@@ -106,7 +122,7 @@ func (r *MountNamespaceRefs) DecRef(destroy func()) {
}
func (r *MountNamespaceRefs) afterLoad() {
- if refsvfs2.LeakCheckEnabled() && r.ReadRefs() > 0 {
+ if r.ReadRefs() > 0 {
r.EnableLeakCheck()
}
}
diff --git a/pkg/sentry/vfs/save_restore.go b/pkg/sentry/vfs/save_restore.go
index 46e50d55d..7723ed643 100644
--- a/pkg/sentry/vfs/save_restore.go
+++ b/pkg/sentry/vfs/save_restore.go
@@ -111,8 +111,8 @@ func (vfs *VirtualFilesystem) loadMounts(mounts []*Mount) {
}
func (mnt *Mount) afterLoad() {
- if refsvfs2.LeakCheckEnabled() && atomic.LoadInt64(&mnt.refs) != 0 {
- refsvfs2.Register(mnt, "vfs.Mount")
+ if atomic.LoadInt64(&mnt.refs) != 0 {
+ refsvfs2.Register(mnt)
}
}
diff --git a/pkg/tcpip/link/tun/tun_endpoint_refs.go b/pkg/tcpip/link/tun/tun_endpoint_refs.go
index 7ca1ace61..850d05c38 100644
--- a/pkg/tcpip/link/tun/tun_endpoint_refs.go
+++ b/pkg/tcpip/link/tun/tun_endpoint_refs.go
@@ -7,9 +7,15 @@ import (
"gvisor.dev/gvisor/pkg/refsvfs2"
)
-// ownerType is used to customize logging. Note that we use a pointer to T so
-// that we do not copy the entire object when passed as a format parameter.
-var tunEndpointownerType *tunEndpoint
+// enableLogging indicates whether reference-related events should be logged (with
+// stack traces). This is false by default and should only be set to true for
+// debugging purposes, as it can generate an extremely large amount of output
+// and drastically degrade performance.
+const tunEndpointenableLogging = false
+
+// obj is used to customize logging. Note that we use a pointer to T so that
+// we do not copy the entire object when passed as a format parameter.
+var tunEndpointobj *tunEndpoint
// Refs implements refs.RefCounter. It keeps a reference count using atomic
// operations and calls the destructor when the count reaches zero.
@@ -29,16 +35,24 @@ type tunEndpointRefs struct {
refCount int64
}
-// EnableLeakCheck enables reference leak checking on r.
-func (r *tunEndpointRefs) EnableLeakCheck() {
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Register(r, fmt.Sprintf("%T", tunEndpointownerType))
- }
+// RefType implements refsvfs2.CheckedObject.RefType.
+func (r *tunEndpointRefs) RefType() string {
+ return fmt.Sprintf("%T", tunEndpointobj)[1:]
}
// LeakMessage implements refsvfs2.CheckedObject.LeakMessage.
func (r *tunEndpointRefs) LeakMessage() string {
- return fmt.Sprintf("%T %p: reference count of %d instead of 0", tunEndpointownerType, r, r.ReadRefs())
+ return fmt.Sprintf("[%s %p] reference count of %d instead of 0", r.RefType(), r, r.ReadRefs())
+}
+
+// LogRefs implements refsvfs2.CheckedObject.LogRefs.
+func (r *tunEndpointRefs) LogRefs() bool {
+ return tunEndpointenableLogging
+}
+
+// EnableLeakCheck enables reference leak checking on r.
+func (r *tunEndpointRefs) EnableLeakCheck() {
+ refsvfs2.Register(r)
}
// ReadRefs returns the current number of references. The returned count is
@@ -52,8 +66,10 @@ func (r *tunEndpointRefs) ReadRefs() int64 {
//
//go:nosplit
func (r *tunEndpointRefs) IncRef() {
- if v := atomic.AddInt64(&r.refCount, 1); v <= 0 {
- panic(fmt.Sprintf("Incrementing non-positive count %p on %T", r, tunEndpointownerType))
+ v := atomic.AddInt64(&r.refCount, 1)
+ refsvfs2.LogIncRef(r, v+1)
+ if v <= 0 {
+ panic(fmt.Sprintf("Incrementing non-positive count %p on %s", r, r.RefType()))
}
}
@@ -66,14 +82,14 @@ func (r *tunEndpointRefs) IncRef() {
//go:nosplit
func (r *tunEndpointRefs) TryIncRef() bool {
const speculativeRef = 1 << 32
- v := atomic.AddInt64(&r.refCount, speculativeRef)
- if int32(v) < 0 {
+ if v := atomic.AddInt64(&r.refCount, speculativeRef); int32(v) < 0 {
atomic.AddInt64(&r.refCount, -speculativeRef)
return false
}
- atomic.AddInt64(&r.refCount, -speculativeRef+1)
+ v := atomic.AddInt64(&r.refCount, -speculativeRef+1)
+ refsvfs2.LogTryIncRef(r, v+1)
return true
}
@@ -90,14 +106,14 @@ func (r *tunEndpointRefs) TryIncRef() bool {
//
//go:nosplit
func (r *tunEndpointRefs) DecRef(destroy func()) {
- switch v := atomic.AddInt64(&r.refCount, -1); {
+ v := atomic.AddInt64(&r.refCount, -1)
+ refsvfs2.LogDecRef(r, v+1)
+ switch {
case v < -1:
- panic(fmt.Sprintf("Decrementing non-positive ref count %p, owned by %T", r, tunEndpointownerType))
+ panic(fmt.Sprintf("Decrementing non-positive ref count %p, owned by %s", r, r.RefType()))
case v == -1:
- if refsvfs2.LeakCheckEnabled() {
- refsvfs2.Unregister(r, fmt.Sprintf("%T", tunEndpointownerType))
- }
+ refsvfs2.Unregister(r)
if destroy != nil {
destroy()
@@ -106,7 +122,7 @@ func (r *tunEndpointRefs) DecRef(destroy func()) {
}
func (r *tunEndpointRefs) afterLoad() {
- if refsvfs2.LeakCheckEnabled() && r.ReadRefs() > 0 {
+ if r.ReadRefs() > 0 {
r.EnableLeakCheck()
}
}