Skip to content

Commit

Permalink
sstable: reduce block cache memory fragmentation
Browse files Browse the repository at this point in the history
Currently, the sstable writer contains heuristics to flush sstable
blocks once the size reaches a specified threshold. In CRDB this is
defined as 32KiB. However, when these blocks are loaded into memory
additional metadata is allocated sometimes exceeding the 32KiB threshold.
Since CRDB uses jemalloc, these allocations use a 40KiB size class which
leads to significant internal fragmentation. In addition, since the
system is unaware of these size classes we cannot design heuristics that
prioritize reducing memory fragmentation. Reducing internal
fragmentation can help reduce CRDB's memory footprint. This commit
decrements the target block size to prevent internal fragmentation for
small key-value pairs and adds support for optionally specifying size
classes to enable a new set of heuristics that will reduce internal
fragmentation for workloads with larger key-value pairs.

Fixes: #999.
  • Loading branch information
CheranMahalingam committed Apr 25, 2024
1 parent c34894c commit 6c053d7
Show file tree
Hide file tree
Showing 9 changed files with 338 additions and 98 deletions.
7 changes: 4 additions & 3 deletions internal/base/options.go
Original file line number Diff line number Diff line change
Expand Up @@ -6,9 +6,10 @@ package base

// SSTable block defaults.
const (
DefaultBlockRestartInterval = 16
DefaultBlockSize = 4096
DefaultBlockSizeThreshold = 90
DefaultBlockRestartInterval = 16
DefaultBlockSize = 4096
DefaultBlockSizeThreshold = 90
SizeClassAwareBlockSizeThreshold = 60
)

// FilterType is the level at which to apply a filter: block or table.
Expand Down
6 changes: 6 additions & 0 deletions internal/cache/value.go
Original file line number Diff line number Diff line change
Expand Up @@ -4,6 +4,12 @@

package cache

import "unsafe"

// ValueMetadataSize denotes the number of bytes of metadata allocated for a
// cache entry.
const ValueMetadataSize = int(unsafe.Sizeof(Value{}))

// Value holds a reference counted immutable value.
type Value struct {
buf []byte
Expand Down
39 changes: 39 additions & 0 deletions internal/cache/value_cgo.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,39 @@
// 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.

//go:build ((!invariants && !tracing) || race) && cgo
// +build !invariants,!tracing race
// +build cgo

package cache

import (
"unsafe"

"github.com/cockroachdb/pebble/internal/manual"
)

func newValue(n int) *Value {
if n == 0 {
return nil
}

// 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)
v := (*Value)(unsafe.Pointer(&b[0]))
v.buf = b[ValueMetadataSize:]
v.ref.init(1)
return v
}

func (v *Value) free() {
// When we're not performing leak detection, the Value and buffer were
// allocated contiguously.
n := ValueMetadataSize + cap(v.buf)
buf := (*[manual.MaxArrayLen]byte)(unsafe.Pointer(v))[:n:n]
v.buf = nil
manual.Free(buf)
}
43 changes: 5 additions & 38 deletions internal/cache/value_normal.go
Original file line number Diff line number Diff line change
Expand Up @@ -2,56 +2,23 @@
// of this source code is governed by a BSD-style license that can be found in
// the LICENSE file.

//go:build (!invariants && !tracing) || race
//go:build ((!invariants && !tracing) || race) && !cgo
// +build !invariants,!tracing race
// +build !cgo

package cache

import (
"unsafe"

"github.com/cockroachdb/pebble/internal/manual"
)

const valueSize = int(unsafe.Sizeof(Value{}))

func newValue(n int) *Value {
if n == 0 {
return nil
}

if !cgoEnabled {
// If Cgo is disabled then all memory is allocated from the Go heap and we
// can't play the trick below to combine the Value and buffer allocation.
v := &Value{buf: make([]byte, n)}
v.ref.init(1)
return v
}

// 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.
//
// TODO(peter): It may be better to separate the allocation of the value and
// the buffer in order to reduce internal fragmentation in malloc. If the
// buffer is right at a power of 2, adding valueSize might push the
// allocation over into the next larger size.
b := manual.New(valueSize + n)
v := (*Value)(unsafe.Pointer(&b[0]))
v.buf = b[valueSize:]
// Since Cgo is disabled then all memory is allocated from the Go heap we
// can't play the trick below to combine the Value and buffer allocation.
v := &Value{buf: make([]byte, n)}
v.ref.init(1)
return v
}

func (v *Value) free() {
if !cgoEnabled {
return
}

// When we're not performing leak detection, the Value and buffer were
// allocated contiguously.
n := valueSize + cap(v.buf)
buf := (*[manual.MaxArrayLen]byte)(unsafe.Pointer(v))[:n:n]
v.buf = nil
manual.Free(buf)
}
7 changes: 7 additions & 0 deletions options.go
Original file line number Diff line number Diff line change
Expand Up @@ -1060,6 +1060,12 @@ type Options struct {
// to temporarily persist data spilled to disk for row-oriented SQL query execution.
EnableSQLRowSpillMetrics bool

// AllocatorSizeClasses provides a sorted list containing the supported size
// classes of the underlying memory allocator. This provides hints to the
// sstable block writer's flushing policy to select block sizes that
// preemptively reduce internal fragmentation when loaded into the block cache.
AllocatorSizeClasses []int

// private options are only used by internal tests or are used internally
// for facilitating upgrade paths of unconfigurable functionality.
private struct {
Expand Down Expand Up @@ -1970,6 +1976,7 @@ func (o *Options) MakeWriterOptions(level int, format sstable.TableFormat) sstab
writerOpts.FilterPolicy = levelOpts.FilterPolicy
writerOpts.FilterType = levelOpts.FilterType
writerOpts.IndexBlockSize = levelOpts.IndexBlockSize
writerOpts.AllocatorSizeClasses = o.AllocatorSizeClasses
return writerOpts
}

Expand Down
19 changes: 18 additions & 1 deletion sstable/options.go
Original file line number Diff line number Diff line change
Expand Up @@ -132,9 +132,17 @@ type WriterOptions struct {
// specified percentage of the target block size and adding the next entry
// would cause the block to be larger than the target block size.
//
// The default value is 90
// The default value is 90.
BlockSizeThreshold int

// SizeClassAwareThreshold imposes a minimum block size restriction for blocks
// to be flushed, that is computed as the percentage of the target block size.
// Note that this threshold takes precedence over BlockSizeThreshold when
// valid AllocatorSizeClasses are specified.
//
// The default value is 60.
SizeClassAwareThreshold int

// Cache is used to cache uncompressed blocks from sstables.
//
// The default is a nil cache.
Expand Down Expand Up @@ -230,6 +238,12 @@ type WriterOptions struct {
// 750MB sstables -- see
// https://github.com/cockroachdb/cockroach/issues/117113).
DisableValueBlocks bool

// AllocatorSizeClasses provides a sorted list containing the supported size
// classes of the underlying memory allocator. This provides hints to the
// writer's flushing policy to select block sizes that preemptively reduce
// internal fragmentation when loaded into the block cache.
AllocatorSizeClasses []int
}

func (o WriterOptions) ensureDefaults() WriterOptions {
Expand All @@ -242,6 +256,9 @@ func (o WriterOptions) ensureDefaults() WriterOptions {
if o.BlockSizeThreshold <= 0 {
o.BlockSizeThreshold = base.DefaultBlockSizeThreshold
}
if o.SizeClassAwareThreshold <= 0 {
o.SizeClassAwareThreshold = base.SizeClassAwareBlockSizeThreshold
}
if o.Comparer == nil {
o.Comparer = base.DefaultComparer
}
Expand Down
59 changes: 59 additions & 0 deletions sstable/testdata/flush_heuristics
Original file line number Diff line number Diff line change
@@ -0,0 +1,59 @@
# Block size exceeds target block size.
build key-size=0 val-size=0 block-size=64 target-size=64 threshold=59 size-class-threshold=0
----
true

# Block size does not exceed threshold size.
build key-size=0 val-size=0 block-size=59 target-size=64 threshold=59 size-class-threshold=0
----
false

# New block size exceeds the target size.
build key-size=1 val-size=1 block-size=60 target-size=64 threshold=32 size-class-threshold=0
----
true

# New block size does not exceed the target size.
build key-size=1 val-size=1 block-size=40 target-size=64 threshold=32 size-class-threshold=0
----
false

# New block size does not exceed the target size with hints enabled.
build key-size=1 val-size=1 block-size=8 target-size=64 threshold=0 size-class-threshold=0 hints=8,16,32,64,128
----
false

# Block size does meet size class aware threshold with hints enabled.
build key-size=1 val-size=1 block-size=6 target-size=64 threshold=0 size-class-threshold=38 hints=8,16,32,64,128
----
false

# New block size reduces internal fragmentation.
build key-size=1 val-size=60 block-size=8 target-size=64 threshold=0 size-class-threshold=38 hints=8,16,32,64,128
----
false

# New block size increases internal fragmentation.
build key-size=1 val-size=40 block-size=8 target-size=64 threshold=0 size-class-threshold=38 hints=8,16,32,64,128
----
true

# Block size target exceeded with hints enabled.
build key-size=1 val-size=1 block-size=32 target-size=64 threshold=0 size-class-threshold=38 hints=8,16,32,64,128
----
true

# Block size target exceeded, however, new block would reduce internal fragmentation.
build key-size=1 val-size=1 block-size=38 target-size=64 threshold=0 size-class-threshold=38 hints=8,16,32,64,128
----
false

# Fall back to heuristics with hints disabled when size class is limited.
build key-size=1 val-size=1 block-size=27 target-size=64 threshold=59 size-class-threshold=38 hints=8,16,32
----
false

# Fall back to heuristics with hints disabled when new size class could not be computed.
build key-size=1 val-size=60 block-size=18 target-size=64 threshold=0 size-class-threshold=38 hints=8,16,32,64
----
false
Loading

0 comments on commit 6c053d7

Please sign in to comment.