Skip to content

Commit

Permalink
db: ensure Metrics.WAL.BytesWritten is nondecreasing
Browse files Browse the repository at this point in the history
The Metrics.WAL.BytesWritten metric is intended to be a monotonically
increasing counter of all bytes written to the write-ahead log. Previously, it
was possible for this metric to violate monotonicity immediately after a WAL
rotation. The d.logSize value—which corresponds to the size of the current
WAL—was not reset to zero. It was only reset after the first write to the new
WAL.

Close #3505.
  • Loading branch information
jbowens committed May 13, 2024
1 parent db693e5 commit 3622ade
Show file tree
Hide file tree
Showing 4 changed files with 94 additions and 19 deletions.
10 changes: 10 additions & 0 deletions db.go
Original file line number Diff line number Diff line change
Expand Up @@ -2543,6 +2543,16 @@ func (d *DB) rotateMemtable(newLogNum FileNum, logSeqNum uint64, prev *memTable)
var entry *flushableEntry
d.mu.mem.mutable, entry = d.newMemTable(newLogNum, logSeqNum)
d.mu.mem.queue = append(d.mu.mem.queue, entry)
// d.logSize tracks the log size of the WAL file corresponding to the most
// recent flushable. The log size of the previous mutable memtable no longer
// applies to the current mutable memtable.
//
// It's tempting to perform this update in rotateWAL, but that would not be
// atomic with the enqueue of the new flushable. A call to DB.Metrics()
// could acquire DB.mu after the WAL has been rotated but before the new
// memtable has been appended; this would result in omitting the log size of
// the most recent flushable.
d.logSize.Store(0)
d.updateReadStateLocked(nil)
if prev.writerUnref() {
d.maybeScheduleFlush()
Expand Down
65 changes: 65 additions & 0 deletions metrics_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -7,9 +7,12 @@ package pebble
import (
"bytes"
"fmt"
"math/rand"
"strconv"
"strings"
"sync"
"testing"
"time"

"github.com/cockroachdb/datadriven"
"github.com/cockroachdb/pebble/internal/humanize"
Expand Down Expand Up @@ -334,3 +337,65 @@ func TestMetricsWAmpDisableWAL(t *testing.T) {
require.Greater(t, tot.WriteAmp(), 1.0)
require.NoError(t, d.Close())
}

// TestMetricsWALBytesWrittenMonotonicity tests that the
// Metrics.WAL.BytesWritten metric is always nondecreasing.
// It's a regression test for issue #3505.
func TestMetricsWALBytesWrittenMonotonicity(t *testing.T) {
fs := vfs.NewMem()
d, err := Open("", &Options{
FS: fs,
// Use a tiny memtable size so that we get frequent flushes. While a
// flush is in-progress or completing, the WAL bytes written should
// remain nondecreasing.
MemTableSize: 1 << 20, /* 20 KiB */
})
require.NoError(t, err)

stopCh := make(chan struct{})

ks := testkeys.Alpha(3)
var wg sync.WaitGroup
const concurrentWriters = 5
wg.Add(concurrentWriters)
for w := 0; w < concurrentWriters; w++ {
go func() {
defer wg.Done()
data := make([]byte, 1<<10) // 1 KiB
rng := rand.New(rand.NewSource(time.Now().UnixNano()))
_, err := rng.Read(data)
require.NoError(t, err)

buf := make([]byte, ks.MaxLen())
for i := 0; ; i++ {
select {
case <-stopCh:
return
default:
}
n := testkeys.WriteKey(buf, ks, int64(i)%ks.Count())
require.NoError(t, d.Set(buf[:n], data, NoSync))
}
}()
}

func() {
defer func() { close(stopCh) }()
abort := time.After(time.Second)
var prevWALBytesWritten uint64
for {
select {
case <-abort:
return
default:
}

m := d.Metrics()
if m.WAL.BytesWritten < prevWALBytesWritten {
t.Fatalf("WAL bytes written decreased: %d -> %d", prevWALBytesWritten, m.WAL.BytesWritten)
}
prevWALBytesWritten = m.WAL.BytesWritten
}
}()
wg.Wait()
}
8 changes: 4 additions & 4 deletions testdata/event_listener
Original file line number Diff line number Diff line change
Expand Up @@ -278,9 +278,9 @@ level | tables size val-bl vtables | score | in | tables size | tables siz
4 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0
5 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0
6 | 1 662B 0B 0 | - | 1.3KB | 0 0B | 0 0B | 1 662B | 1.3KB | 1 0.5
total | 3 2.0KB 0B 0 | - | 825B | 1 717B | 0 0B | 4 3.4KB | 1.3KB | 3 4.2
total | 3 2.0KB 0B 0 | - | 798B | 1 717B | 0 0B | 4 3.4KB | 1.3KB | 3 4.3
-------------------------------------------------------------------------------------------------------------------
WAL: 1 files (27B) in: 48B written: 108B (125% overhead)
WAL: 1 files (0B) in: 48B written: 81B (69% overhead)
Flushes: 3
Compactions: 1 estimated debt: 2.0KB in progress: 0 (0B)
default: 1 delete: 0 elision: 0 move: 0 read: 0 rewrite: 0 multi-level: 0
Expand Down Expand Up @@ -377,9 +377,9 @@ level | tables size val-bl vtables | score | in | tables size | tables siz
4 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0
5 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0
6 | 2 1.3KB 0B 0 | - | 1.3KB | 1 717B | 0 0B | 1 662B | 1.3KB | 1 0.5
total | 6 4.0KB 0B 0 | - | 2.2KB | 3 2.1KB | 0 0B | 5 5.5KB | 1.3KB | 5 2.4
total | 6 4.0KB 0B 0 | - | 2.2KB | 3 2.1KB | 0 0B | 5 5.4KB | 1.3KB | 5 2.5
-------------------------------------------------------------------------------------------------------------------
WAL: 1 files (29B) in: 82B written: 137B (67% overhead)
WAL: 1 files (0B) in: 82B written: 108B (32% overhead)
Flushes: 6
Compactions: 1 estimated debt: 4.0KB in progress: 0 (0B)
default: 1 delete: 0 elision: 0 move: 0 read: 0 rewrite: 0 multi-level: 0
Expand Down
30 changes: 15 additions & 15 deletions testdata/metrics
Original file line number Diff line number Diff line change
Expand Up @@ -57,9 +57,9 @@ level | tables size val-bl vtables | score | in | tables size | tables siz
4 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0
5 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0
6 | 0 0B 0B 0 | - | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0
total | 1 661B 0B 0 | - | 56B | 0 0B | 0 0B | 1 717B | 0B | 1 12.8
total | 1 661B 0B 0 | - | 28B | 0 0B | 0 0B | 1 689B | 0B | 1 24.6
-------------------------------------------------------------------------------------------------------------------
WAL: 1 files (28B) in: 17B written: 56B (229% overhead)
WAL: 1 files (0B) in: 17B written: 28B (65% overhead)
Flushes: 1
Compactions: 0 estimated debt: 0B in progress: 0 (0B)
default: 0 delete: 0 elision: 0 move: 0 read: 0 rewrite: 0 multi-level: 0
Expand Down Expand Up @@ -111,9 +111,9 @@ level | tables size val-bl vtables | score | in | tables size | tables siz
4 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0
5 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0
6 | 1 667B 0B 0 | - | 1.3KB | 0 0B | 0 0B | 1 667B | 1.3KB | 1 0.5
total | 1 667B 0B 0 | - | 84B | 0 0B | 0 0B | 3 2.0KB | 1.3KB | 1 24.7
total | 1 667B 0B 0 | - | 56B | 0 0B | 0 0B | 3 2.0KB | 1.3KB | 1 36.5
-------------------------------------------------------------------------------------------------------------------
WAL: 1 files (28B) in: 34B written: 84B (147% overhead)
WAL: 1 files (0B) in: 34B written: 56B (65% overhead)
Flushes: 2
Compactions: 1 estimated debt: 0B in progress: 0 (0B)
default: 1 delete: 0 elision: 0 move: 0 read: 0 rewrite: 0 multi-level: 0
Expand Down Expand Up @@ -150,9 +150,9 @@ level | tables size val-bl vtables | score | in | tables size | tables siz
4 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0
5 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0
6 | 1 667B 0B 0 | - | 1.3KB | 0 0B | 0 0B | 1 667B | 1.3KB | 1 0.5
total | 1 667B 0B 0 | - | 84B | 0 0B | 0 0B | 3 2.0KB | 1.3KB | 1 24.7
total | 1 667B 0B 0 | - | 56B | 0 0B | 0 0B | 3 2.0KB | 1.3KB | 1 36.5
-------------------------------------------------------------------------------------------------------------------
WAL: 1 files (28B) in: 34B written: 84B (147% overhead)
WAL: 1 files (0B) in: 34B written: 56B (65% overhead)
Flushes: 2
Compactions: 1 estimated debt: 0B in progress: 0 (0B)
default: 1 delete: 0 elision: 0 move: 0 read: 0 rewrite: 0 multi-level: 0
Expand Down Expand Up @@ -186,9 +186,9 @@ level | tables size val-bl vtables | score | in | tables size | tables siz
4 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0
5 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0
6 | 1 667B 0B 0 | - | 1.3KB | 0 0B | 0 0B | 1 667B | 1.3KB | 1 0.5
total | 1 667B 0B 0 | - | 84B | 0 0B | 0 0B | 3 2.0KB | 1.3KB | 1 24.7
total | 1 667B 0B 0 | - | 56B | 0 0B | 0 0B | 3 2.0KB | 1.3KB | 1 36.5
-------------------------------------------------------------------------------------------------------------------
WAL: 1 files (28B) in: 34B written: 84B (147% overhead)
WAL: 1 files (0B) in: 34B written: 56B (65% overhead)
Flushes: 2
Compactions: 1 estimated debt: 0B in progress: 0 (0B)
default: 1 delete: 0 elision: 0 move: 0 read: 0 rewrite: 0 multi-level: 0
Expand Down Expand Up @@ -225,9 +225,9 @@ level | tables size val-bl vtables | score | in | tables size | tables siz
4 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0
5 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0
6 | 1 667B 0B 0 | - | 1.3KB | 0 0B | 0 0B | 1 667B | 1.3KB | 1 0.5
total | 1 667B 0B 0 | - | 84B | 0 0B | 0 0B | 3 2.0KB | 1.3KB | 1 24.7
total | 1 667B 0B 0 | - | 56B | 0 0B | 0 0B | 3 2.0KB | 1.3KB | 1 36.5
-------------------------------------------------------------------------------------------------------------------
WAL: 1 files (28B) in: 34B written: 84B (147% overhead)
WAL: 1 files (0B) in: 34B written: 56B (65% overhead)
Flushes: 2
Compactions: 1 estimated debt: 0B in progress: 0 (0B)
default: 1 delete: 0 elision: 0 move: 0 read: 0 rewrite: 0 multi-level: 0
Expand Down Expand Up @@ -290,9 +290,9 @@ level | tables size val-bl vtables | score | in | tables size | tables siz
4 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0
5 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0
6 | 1 667B 0B 0 | - | 1.3KB | 0 0B | 0 0B | 1 667B | 1.3KB | 1 0.5
total | 4 2.9KB 38B 0 | - | 242B | 0 0B | 0 0B | 6 4.4KB | 1.3KB | 2 18.6
total | 4 2.9KB 38B 0 | - | 149B | 0 0B | 0 0B | 6 4.3KB | 1.3KB | 2 29.7
-------------------------------------------------------------------------------------------------------------------
WAL: 1 files (93B) in: 116B written: 242B (109% overhead)
WAL: 1 files (0B) in: 116B written: 149B (28% overhead)
Flushes: 3
Compactions: 1 estimated debt: 2.9KB in progress: 0 (0B)
default: 1 delete: 0 elision: 0 move: 0 read: 0 rewrite: 0 multi-level: 0
Expand Down Expand Up @@ -339,9 +339,9 @@ level | tables size val-bl vtables | score | in | tables size | tables siz
4 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0
5 | 0 0B 0B 0 | 0.00 | 0B | 0 0B | 0 0B | 0 0B | 0B | 0 0.0
6 | 3 2.2KB 41B 0 | - | 3.5KB | 0 0B | 0 0B | 3 2.2KB | 3.5KB | 1 0.6
total | 3 2.2KB 41B 0 | - | 242B | 0 0B | 0 0B | 8 6.0KB | 3.5KB | 1 25.3
total | 3 2.2KB 41B 0 | - | 149B | 0 0B | 0 0B | 8 5.9KB | 3.5KB | 1 40.5
-------------------------------------------------------------------------------------------------------------------
WAL: 1 files (93B) in: 116B written: 242B (109% overhead)
WAL: 1 files (0B) in: 116B written: 149B (28% overhead)
Flushes: 3
Compactions: 2 estimated debt: 0B in progress: 0 (0B)
default: 2 delete: 0 elision: 0 move: 0 read: 0 rewrite: 0 multi-level: 0
Expand Down Expand Up @@ -436,7 +436,7 @@ level | tables size val-bl vtables | score | in | tables size | tables siz
6 | 3 2.2KB 41B 0 | - | 3.5KB | 0 0B | 0 0B | 3 2.2KB | 3.5KB | 1 0.6
total | 7 5.0KB 41B 0 | - | 2.3KB | 3 2.1KB | 0 0B | 9 8.7KB | 3.5KB | 3 3.8
-------------------------------------------------------------------------------------------------------------------
WAL: 1 files (26B) in: 176B written: 213B (21% overhead)
WAL: 1 files (0B) in: 176B written: 187B (6% overhead)
Flushes: 8
Compactions: 2 estimated debt: 5.0KB in progress: 0 (0B)
default: 2 delete: 0 elision: 0 move: 0 read: 0 rewrite: 0 multi-level: 0
Expand Down

0 comments on commit 3622ade

Please sign in to comment.