Skip to content

Commit

Permalink
manual: add cgo memory accounting
Browse files Browse the repository at this point in the history
This commit adds a breakdown of cgo memory usage in Pebble. This will
show up as part of the periodic LSM metrics dump in the logs.
  • Loading branch information
RaduBerinde committed Sep 6, 2024
1 parent c8c3806 commit fc6eab0
Show file tree
Hide file tree
Showing 17 changed files with 197 additions and 60 deletions.
4 changes: 3 additions & 1 deletion db.go
Original file line number Diff line number Diff line change
Expand Up @@ -2041,6 +2041,8 @@ func (d *DB) Metrics() *Metrics {

metrics.Uptime = d.timeNow().Sub(d.openedAt)

metrics.manualMemory = manual.GetMetrics()

return metrics
}

Expand Down Expand Up @@ -2368,7 +2370,7 @@ func (d *DB) newMemTable(
memtblOpts.releaseAccountingReservation = mem.releaseAccountingReservation
} else {
mem = new(memTable)
memtblOpts.arenaBuf = manual.New(int(size))
memtblOpts.arenaBuf = manual.New(manual.MemTable, int(size))
memtblOpts.releaseAccountingReservation = d.opts.Cache.Reserve(int(size))
d.memTableCount.Add(1)
d.memTableReserved.Add(int64(size))
Expand Down
4 changes: 2 additions & 2 deletions internal/cache/block_map.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,14 +27,14 @@ type blockMapAllocator struct{}

func (blockMapAllocator) Alloc(n int) []swiss.Group[key, *entry] {
size := uintptr(n) * unsafe.Sizeof(swiss.Group[key, *entry]{})
buf := manual.New(int(size))
buf := manual.New(manual.BlockCacheMap, int(size))
return unsafe.Slice((*swiss.Group[key, *entry])(unsafe.Pointer(unsafe.SliceData(buf))), n)
}

func (blockMapAllocator) Free(v []swiss.Group[key, *entry]) {
size := uintptr(len(v)) * unsafe.Sizeof(swiss.Group[key, *entry]{})
buf := unsafe.Slice((*byte)(unsafe.Pointer(unsafe.SliceData(v))), size)
manual.Free(buf)
manual.Free(manual.BlockCacheMap, buf)
}

var blockMapOptions = []swiss.Option[key, *entry]{
Expand Down
4 changes: 2 additions & 2 deletions internal/cache/entry_normal.go
Original file line number Diff line number Diff line change
Expand Up @@ -80,7 +80,7 @@ func (c *entryAllocCache) alloc() *entry {
if entriesGoAllocated {
return &entry{}
}
b := manual.New(entrySize)
b := manual.New(manual.BlockCacheEntry, entrySize)
return (*entry)(unsafe.Pointer(&b[0]))
}
e := c.entries[n-1]
Expand All @@ -91,7 +91,7 @@ func (c *entryAllocCache) alloc() *entry {
func (c *entryAllocCache) dealloc(e *entry) {
if !entriesGoAllocated {
buf := (*[manual.MaxArrayLen]byte)(unsafe.Pointer(e))[:entrySize:entrySize]
manual.Free(buf)
manual.Free(manual.BlockCacheEntry, buf)
}
}

Expand Down
4 changes: 2 additions & 2 deletions internal/cache/value_cgo.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@ func newValue(n int) *Value {
// When we're not performing leak detection, the lifetime of the returned
// Value is exactly the lifetime of the backing buffer and we can manually
// allocate both.
b := manual.New(ValueMetadataSize + n)
b := manual.New(manual.BlockCacheData, ValueMetadataSize+n)
v := (*Value)(unsafe.Pointer(&b[0]))
v.buf = b[ValueMetadataSize:]
v.ref.init(1)
Expand All @@ -35,5 +35,5 @@ func (v *Value) free() {
n := ValueMetadataSize + cap(v.buf)
buf := (*[manual.MaxArrayLen]byte)(unsafe.Pointer(v))[:n:n]
v.buf = nil
manual.Free(buf)
manual.Free(manual.BlockCacheData, buf)
}
4 changes: 2 additions & 2 deletions internal/cache/value_invariants.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@ func newValue(n int) *Value {
if n == 0 {
return nil
}
b := manual.New(n)
b := manual.New(manual.BlockCacheData, n)
v := &Value{buf: b}
v.ref.init(1)
// Note: this is a no-op if invariants and tracing are disabled or race is
Expand All @@ -47,7 +47,7 @@ func (v *Value) free() {
for i := range v.buf {
v.buf[i] = 0xff
}
manual.Free(v.buf)
manual.Free(manual.BlockCacheData, v.buf)
// Setting Value.buf to nil is needed for correctness of the leak checking
// that is performed when the "invariants" or "tracing" build tags are
// enabled.
Expand Down
95 changes: 49 additions & 46 deletions internal/manual/manual.go
Original file line number Diff line number Diff line change
@@ -1,60 +1,63 @@
// Copyright 2020 The LevelDB-Go and Pebble Authors. All rights reserved. Use
// Copyright 2024 The LevelDB-Go and Pebble Authors. All rights reserved. Use
// of this source code is governed by a BSD-style license that can be found in
// the LICENSE file.

package manual

// #include <stdlib.h>
import "C"
import "unsafe"
import (
"fmt"
"sync/atomic"

// The go:linkname directives provides backdoor access to private functions in
// the runtime. Below we're accessing the throw function.
"github.com/cockroachdb/pebble/internal/invariants"
)

//go:linkname throw runtime.throw
func throw(s string)
// Purpose identifies the use-case for an allocation.
type Purpose uint8

// TODO(peter): Rather than relying an C malloc/free, we could fork the Go
// runtime page allocator and allocate large chunks of memory using mmap or
// similar.
const (
_ Purpose = iota

// New allocates a slice of size n. The returned slice is from manually managed
// memory and MUST be released by calling Free. Failure to do so will result in
// a memory leak.
func New(n int) []byte {
if n == 0 {
return make([]byte, 0)
}
// We need to be conscious of the Cgo pointer passing rules:
//
// https://golang.org/cmd/cgo/#hdr-Passing_pointers
//
// ...
// Note: the current implementation has a bug. While Go code is permitted
// to write nil or a C pointer (but not a Go pointer) to C memory, the
// current implementation may sometimes cause a runtime error if the
// contents of the C memory appear to be a Go pointer. Therefore, avoid
// passing uninitialized C memory to Go code if the Go code is going to
// store pointer values in it. Zero out the memory in C before passing it
// to Go.
ptr := C.calloc(C.size_t(n), 1)
if ptr == nil {
// NB: throw is like panic, except it guarantees the process will be
// terminated. The call below is exactly what the Go runtime invokes when
// it cannot allocate memory.
throw("out of memory")
BlockCacheMap
BlockCacheEntry
BlockCacheData
MemTable

NumPurposes
)

// Metrics contains memory statistics by purpose.
type Metrics [NumPurposes]struct {
// InUseBytes is the total number of bytes currently allocated. This is just
// the sum of the lengths of the allocations and does not include any overhead
// or fragmentation.
InUseBytes uint64
}

var counters [NumPurposes]struct {
InUseBytes atomic.Int64
// Pad to separate counters into cache lines. This reduces the overhead when
// multiple purposes are used frequently. We assume 64 byte cache line size
// which is the case for ARM64 servers and AMD64.
_ [7]uint64
}

func recordAlloc(purpose Purpose, n int) {
counters[purpose].InUseBytes.Add(int64(n))
}

func recordFree(purpose Purpose, n int) {
newVal := counters[purpose].InUseBytes.Add(-int64(n))
if invariants.Enabled && newVal < 0 {
panic(fmt.Sprintf("negative counter value %d", newVal))
}
// Interpret the C pointer as a pointer to a Go array, then slice.
return (*[MaxArrayLen]byte)(unsafe.Pointer(ptr))[:n:n]
}

// Free frees the specified slice.
func Free(b []byte) {
if cap(b) != 0 {
if len(b) == 0 {
b = b[:cap(b)]
}
ptr := unsafe.Pointer(&b[0])
C.free(ptr)
// GetMetrics returns manual memory usage statistics.
func GetMetrics() Metrics {
var res Metrics
for i := range res {
// We load the freed count first to avoid a negative value, since we don't load both counters atomically.
res[i].InUseBytes = uint64(counters[i].InUseBytes.Load())
}
return res
}
63 changes: 63 additions & 0 deletions internal/manual/manual_cgo.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,63 @@
// Copyright 2020 The LevelDB-Go and Pebble Authors. All rights reserved. Use
// of this source code is governed by a BSD-style license that can be found in
// the LICENSE file.

package manual

// #include <stdlib.h>
import "C"
import "unsafe"

// The go:linkname directives provides backdoor access to private functions in
// the runtime. Below we're accessing the throw function.

//go:linkname throw runtime.throw
func throw(s string)

// TODO(peter): Rather than relying an C malloc/free, we could fork the Go
// runtime page allocator and allocate large chunks of memory using mmap or
// similar.

// New allocates a slice of size n. The returned slice is from manually managed
// memory and MUST be released by calling Free. Failure to do so will result in
// a memory leak.
func New(purpose Purpose, n int) []byte {
if n == 0 {
return make([]byte, 0)
}
recordAlloc(purpose, n)
// We need to be conscious of the Cgo pointer passing rules:
//
// https://golang.org/cmd/cgo/#hdr-Passing_pointers
//
// ...
// Note: the current implementation has a bug. While Go code is permitted
// to write nil or a C pointer (but not a Go pointer) to C memory, the
// current implementation may sometimes cause a runtime error if the
// contents of the C memory appear to be a Go pointer. Therefore, avoid
// passing uninitialized C memory to Go code if the Go code is going to
// store pointer values in it. Zero out the memory in C before passing it
// to Go.
ptr := C.calloc(C.size_t(n), 1)
if ptr == nil {
// NB: throw is like panic, except it guarantees the process will be
// terminated. The call below is exactly what the Go runtime invokes when
// it cannot allocate memory.
throw("out of memory")
}
// Interpret the C pointer as a pointer to a Go array, then slice.
return (*[MaxArrayLen]byte)(unsafe.Pointer(ptr))[:n:n]
}

// Free frees the specified slice. It has to be exactly the slice that was
// returned by New.
func Free(purpose Purpose, b []byte) {
if cap(b) != 0 {
recordFree(purpose, cap(b))
if len(b) == 0 {
b = b[:cap(b)]
}
ptr := unsafe.Pointer(&b[0])
C.free(ptr)
}
}
9 changes: 6 additions & 3 deletions internal/manual/manual_nocgo.go
Original file line number Diff line number Diff line change
Expand Up @@ -11,10 +11,13 @@ package manual
// compilation).

// New allocates a slice of size n.
func New(n int) []byte {
func New(purpose Purpose, n int) []byte {
recordAlloc(purpose, n)
return make([]byte, n)
}

// Free frees the specified slice.
func Free(b []byte) {
// Free frees the specified slice. It has to be exactly the slice that was
// returned by New.
func Free(purpose Purpose, b []byte) {
recordFree(purpose, cap(b))
}
2 changes: 1 addition & 1 deletion mem_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -92,7 +92,7 @@ type memTable struct {
func (m *memTable) free() {
if m != nil {
m.releaseAccountingReservation()
manual.Free(m.arenaBuf)
manual.Free(manual.MemTable, m.arenaBuf)
m.arenaBuf = nil
}
}
Expand Down
22 changes: 22 additions & 0 deletions metrics.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@ import (
"github.com/cockroachdb/pebble/internal/base"
"github.com/cockroachdb/pebble/internal/cache"
"github.com/cockroachdb/pebble/internal/humanize"
"github.com/cockroachdb/pebble/internal/manual"
"github.com/cockroachdb/pebble/objstorage/objstorageprovider/sharedcache"
"github.com/cockroachdb/pebble/record"
"github.com/cockroachdb/pebble/sstable"
Expand Down Expand Up @@ -332,6 +333,8 @@ type Metrics struct {
optionsFileSize uint64
manifestFileSize uint64
}

manualMemory manual.Metrics
}

var (
Expand Down Expand Up @@ -655,6 +658,22 @@ func (m *Metrics) SafeFormat(w redact.SafePrinter, _ rune) {
redact.Safe(m.Flush.AsIngestCount),
humanize.Bytes.Uint64(m.Flush.AsIngestBytes),
redact.Safe(m.Flush.AsIngestTableCount))

var inUseTotal uint64
for i := range m.manualMemory {
inUseTotal += m.manualMemory[i].InUseBytes
}
inUse := func(purpose manual.Purpose) uint64 {
return m.manualMemory[purpose].InUseBytes
}
w.Printf("Cgo memory usage: %s block cache: %s (data: %s, maps: %s, entries: %s) memtables: %s\n",
humanize.Bytes.Uint64(inUseTotal),
humanize.Bytes.Uint64(inUse(manual.BlockCacheData)+inUse(manual.BlockCacheMap)+inUse(manual.BlockCacheEntry)),
humanize.Bytes.Uint64(inUse(manual.BlockCacheData)),
humanize.Bytes.Uint64(inUse(manual.BlockCacheMap)),
humanize.Bytes.Uint64(inUse(manual.BlockCacheEntry)),
humanize.Bytes.Uint64(inUse(manual.MemTable)),
)
}

func hitRate(hits, misses int64) float64 {
Expand All @@ -678,5 +697,8 @@ func (m *Metrics) StringForTests() string {
const tableCacheSizeAdjustment = 212
mCopy.TableCache.Size += mCopy.TableCache.Count * tableCacheSizeAdjustment
}
// Don't show cgo memory statistics as they can vary based on architecture,
// invariants tag, etc.
mCopy.manualMemory = manual.Metrics{}
return redact.StringWithoutMarkers(&mCopy)
}
7 changes: 7 additions & 0 deletions metrics_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import (
"github.com/cockroachdb/datadriven"
"github.com/cockroachdb/pebble/internal/cache"
"github.com/cockroachdb/pebble/internal/humanize"
"github.com/cockroachdb/pebble/internal/manual"
"github.com/cockroachdb/pebble/internal/testkeys"
"github.com/cockroachdb/pebble/objstorage/remote"
"github.com/cockroachdb/pebble/sstable"
Expand Down Expand Up @@ -100,6 +101,9 @@ func exampleMetrics() Metrics {
l.MultiLevel.BytesIn = base + 4
l.MultiLevel.BytesRead = base + 4
}
for i := range m.manualMemory {
m.manualMemory[i].InUseBytes = uint64((i + 1) * 1024)
}
return m
}

Expand Down Expand Up @@ -329,6 +333,9 @@ func TestMetrics(t *testing.T) {
d.mu.Unlock()

m := d.Metrics()
// Don't show memory usage as that can depend on architecture, invariants
// tag, etc.
m.manualMemory = manual.Metrics{}
// Some subset of cases show non-determinism in cache hits/misses.
if td.HasArg("zero-cache-hits-misses") {
// Avoid non-determinism.
Expand Down
2 changes: 1 addition & 1 deletion open.go
Original file line number Diff line number Diff line change
Expand Up @@ -246,7 +246,7 @@ func Open(dirname string, opts *Options) (db *DB, err error) {
for _, mem := range d.mu.mem.queue {
switch t := mem.flushable.(type) {
case *memTable:
manual.Free(t.arenaBuf)
manual.Free(manual.MemTable, t.arenaBuf)
t.arenaBuf = nil
}
}
Expand Down
2 changes: 2 additions & 0 deletions testdata/event_listener
Original file line number Diff line number Diff line change
Expand Up @@ -230,6 +230,7 @@ Snapshots: 0 earliest seq num: 0
Table iters: 0
Filter utility: 0.0%
Ingestions: 1 as flushable: 0 (0B in 0 tables)
Cgo memory usage: 0B block cache: 0B (data: 0B, maps: 0B, entries: 0B) memtables: 0B

# Set up a scenario where the table to be ingested overlaps with the memtable.
# The table is ingested as a flushable. The flush metrics refect the flushed
Expand Down Expand Up @@ -331,6 +332,7 @@ Snapshots: 0 earliest seq num: 0
Table iters: 0
Filter utility: 0.0%
Ingestions: 2 as flushable: 1 (1.2KB in 2 tables)
Cgo memory usage: 0B block cache: 0B (data: 0B, maps: 0B, entries: 0B) memtables: 0B

sstables
----
Expand Down
1 change: 1 addition & 0 deletions testdata/ingest
Original file line number Diff line number Diff line change
Expand Up @@ -60,6 +60,7 @@ Snapshots: 0 earliest seq num: 0
Table iters: 0
Filter utility: 0.0%
Ingestions: 1 as flushable: 0 (0B in 0 tables)
Cgo memory usage: 0B block cache: 0B (data: 0B, maps: 0B, entries: 0B) memtables: 0B


iter
Expand Down
Loading

0 comments on commit fc6eab0

Please sign in to comment.