github.com/cockroachdb/pebble@v1.1.1-0.20240513155919-3622ade60459/commit_test.go (about)

     1  // Copyright 2018 The LevelDB-Go and Pebble Authors. All rights reserved. Use
     2  // of this source code is governed by a BSD-style license that can be found in
     3  // the LICENSE file.
     4  
     5  package pebble
     6  
     7  import (
     8  	"encoding/binary"
     9  	"fmt"
    10  	"io"
    11  	"sync"
    12  	"sync/atomic"
    13  	"testing"
    14  	"time"
    15  
    16  	"github.com/cockroachdb/pebble/internal/arenaskl"
    17  	"github.com/cockroachdb/pebble/internal/invariants"
    18  	"github.com/cockroachdb/pebble/record"
    19  	"github.com/cockroachdb/pebble/vfs"
    20  	"github.com/prometheus/client_golang/prometheus"
    21  	"github.com/stretchr/testify/require"
    22  	"golang.org/x/exp/rand"
    23  )
    24  
    25  type testCommitEnv struct {
    26  	logSeqNum     atomic.Uint64
    27  	visibleSeqNum atomic.Uint64
    28  	writeCount    atomic.Uint64
    29  	applyBuf      struct {
    30  		sync.Mutex
    31  		buf []uint64
    32  	}
    33  	queueSemChan chan struct{}
    34  }
    35  
    36  func (e *testCommitEnv) env() commitEnv {
    37  	return commitEnv{
    38  		logSeqNum:     &e.logSeqNum,
    39  		visibleSeqNum: &e.visibleSeqNum,
    40  		apply:         e.apply,
    41  		write:         e.write,
    42  	}
    43  }
    44  
    45  func (e *testCommitEnv) apply(b *Batch, mem *memTable) error {
    46  	e.applyBuf.Lock()
    47  	e.applyBuf.buf = append(e.applyBuf.buf, b.SeqNum())
    48  	e.applyBuf.Unlock()
    49  	return nil
    50  }
    51  
    52  func (e *testCommitEnv) write(b *Batch, wg *sync.WaitGroup, _ *error) (*memTable, error) {
    53  	e.writeCount.Add(1)
    54  	if wg != nil {
    55  		wg.Done()
    56  		<-e.queueSemChan
    57  	}
    58  	return nil, nil
    59  }
    60  
    61  func TestCommitQueue(t *testing.T) {
    62  	var q commitQueue
    63  	var batches [16]Batch
    64  	for i := range batches {
    65  		q.enqueue(&batches[i])
    66  	}
    67  	if b := q.dequeueApplied(); b != nil {
    68  		t.Fatalf("unexpectedly dequeued batch: %p", b)
    69  	}
    70  	batches[1].applied.Store(true)
    71  	if b := q.dequeueApplied(); b != nil {
    72  		t.Fatalf("unexpectedly dequeued batch: %p", b)
    73  	}
    74  	for i := range batches {
    75  		batches[i].applied.Store(true)
    76  		if b := q.dequeueApplied(); b != &batches[i] {
    77  			t.Fatalf("%d: expected batch %p, but found %p", i, &batches[i], b)
    78  		}
    79  	}
    80  	if b := q.dequeueApplied(); b != nil {
    81  		t.Fatalf("unexpectedly dequeued batch: %p", b)
    82  	}
    83  }
    84  
    85  func TestCommitPipeline(t *testing.T) {
    86  	var e testCommitEnv
    87  	p := newCommitPipeline(e.env())
    88  
    89  	n := 10000
    90  	if invariants.RaceEnabled {
    91  		// Under race builds we have to limit the concurrency or we hit the
    92  		// following error:
    93  		//
    94  		//   race: limit on 8128 simultaneously alive goroutines is exceeded, dying
    95  		n = 1000
    96  	}
    97  
    98  	var wg sync.WaitGroup
    99  	wg.Add(n)
   100  	for i := 0; i < n; i++ {
   101  		go func(i int) {
   102  			defer wg.Done()
   103  			var b Batch
   104  			_ = b.Set([]byte(fmt.Sprint(i)), nil, nil)
   105  			_ = p.Commit(&b, false, false)
   106  		}(i)
   107  	}
   108  	wg.Wait()
   109  
   110  	if s := e.writeCount.Load(); uint64(n) != s {
   111  		t.Fatalf("expected %d written batches, but found %d", n, s)
   112  	}
   113  	if n != len(e.applyBuf.buf) {
   114  		t.Fatalf("expected %d written batches, but found %d",
   115  			n, len(e.applyBuf.buf))
   116  	}
   117  	if s := e.logSeqNum.Load(); uint64(n) != s {
   118  		t.Fatalf("expected %d, but found %d", n, s)
   119  	}
   120  	if s := e.visibleSeqNum.Load(); uint64(n) != s {
   121  		t.Fatalf("expected %d, but found %d", n, s)
   122  	}
   123  }
   124  
   125  func TestCommitPipelineSync(t *testing.T) {
   126  	n := 10000
   127  	if invariants.RaceEnabled {
   128  		// Under race builds we have to limit the concurrency or we hit the
   129  		// following error:
   130  		//
   131  		//   race: limit on 8128 simultaneously alive goroutines is exceeded, dying
   132  		n = 1000
   133  	}
   134  
   135  	for _, noSyncWait := range []bool{false, true} {
   136  		t.Run(fmt.Sprintf("no-sync-wait=%t", noSyncWait), func(t *testing.T) {
   137  			var e testCommitEnv
   138  			p := newCommitPipeline(e.env())
   139  			e.queueSemChan = p.logSyncQSem
   140  
   141  			var wg sync.WaitGroup
   142  			wg.Add(n)
   143  			for i := 0; i < n; i++ {
   144  				go func(i int) {
   145  					defer wg.Done()
   146  					var b Batch
   147  					require.NoError(t, b.Set([]byte(fmt.Sprint(i)), nil, nil))
   148  					require.NoError(t, p.Commit(&b, true, noSyncWait))
   149  					if noSyncWait {
   150  						require.NoError(t, b.SyncWait())
   151  					}
   152  				}(i)
   153  			}
   154  			wg.Wait()
   155  			if s := e.writeCount.Load(); uint64(n) != s {
   156  				t.Fatalf("expected %d written batches, but found %d", n, s)
   157  			}
   158  			if n != len(e.applyBuf.buf) {
   159  				t.Fatalf("expected %d written batches, but found %d",
   160  					n, len(e.applyBuf.buf))
   161  			}
   162  			if s := e.logSeqNum.Load(); uint64(n) != s {
   163  				t.Fatalf("expected %d, but found %d", n, s)
   164  			}
   165  			if s := e.visibleSeqNum.Load(); uint64(n) != s {
   166  				t.Fatalf("expected %d, but found %d", n, s)
   167  			}
   168  		})
   169  	}
   170  }
   171  
   172  func TestCommitPipelineAllocateSeqNum(t *testing.T) {
   173  	var e testCommitEnv
   174  	p := newCommitPipeline(e.env())
   175  
   176  	const n = 10
   177  	var wg sync.WaitGroup
   178  	wg.Add(n)
   179  	var prepareCount atomic.Uint64
   180  	var applyCount atomic.Uint64
   181  	for i := 1; i <= n; i++ {
   182  		go func(i int) {
   183  			defer wg.Done()
   184  			p.AllocateSeqNum(i, func(_ uint64) {
   185  				prepareCount.Add(1)
   186  			}, func(seqNum uint64) {
   187  				applyCount.Add(1)
   188  			})
   189  		}(i)
   190  	}
   191  	wg.Wait()
   192  
   193  	if s := prepareCount.Load(); n != s {
   194  		t.Fatalf("expected %d prepares, but found %d", n, s)
   195  	}
   196  	if s := applyCount.Load(); n != s {
   197  		t.Fatalf("expected %d applies, but found %d", n, s)
   198  	}
   199  	// AllocateSeqNum always returns a non-zero sequence number causing the
   200  	// values we see to be offset from 1.
   201  	const total = 1 + 1 + 2 + 3 + 4 + 5 + 6 + 7 + 8 + 9 + 10
   202  	if s := e.logSeqNum.Load(); total != s {
   203  		t.Fatalf("expected %d, but found %d", total, s)
   204  	}
   205  	if s := e.visibleSeqNum.Load(); total != s {
   206  		t.Fatalf("expected %d, but found %d", total, s)
   207  	}
   208  }
   209  
   210  type syncDelayFile struct {
   211  	vfs.File
   212  	done chan struct{}
   213  }
   214  
   215  func (f *syncDelayFile) Sync() error {
   216  	<-f.done
   217  	return nil
   218  }
   219  
   220  func TestCommitPipelineWALClose(t *testing.T) {
   221  	// This test stresses the edge case of N goroutines blocked in the
   222  	// commitPipeline waiting for the log to sync when we concurrently decide to
   223  	// rotate and close the log.
   224  
   225  	mem := vfs.NewMem()
   226  	f, err := mem.Create("test-wal")
   227  	require.NoError(t, err)
   228  
   229  	// syncDelayFile will block on the done channel befor returning from Sync
   230  	// call.
   231  	sf := &syncDelayFile{
   232  		File: f,
   233  		done: make(chan struct{}),
   234  	}
   235  
   236  	// A basic commitEnv which writes to a WAL.
   237  	var wal *record.LogWriter
   238  	var walDone sync.WaitGroup
   239  	testEnv := commitEnv{
   240  		logSeqNum:     new(atomic.Uint64),
   241  		visibleSeqNum: new(atomic.Uint64),
   242  		apply: func(b *Batch, mem *memTable) error {
   243  			// At this point, we've called SyncRecord but the sync is blocked.
   244  			walDone.Done()
   245  			return nil
   246  		},
   247  		write: func(b *Batch, syncWG *sync.WaitGroup, syncErr *error) (*memTable, error) {
   248  			_, err := wal.SyncRecord(b.data, syncWG, syncErr)
   249  			return nil, err
   250  		},
   251  	}
   252  	p := newCommitPipeline(testEnv)
   253  	wal = record.NewLogWriter(sf, 0 /* logNum */, record.LogWriterConfig{
   254  		WALFsyncLatency: prometheus.NewHistogram(prometheus.HistogramOpts{}),
   255  		QueueSemChan:    p.logSyncQSem,
   256  	})
   257  
   258  	// Launch N (commitConcurrency) goroutines which each create a batch and
   259  	// commit it with sync==true. Because of the syncDelayFile, none of these
   260  	// operations can complete until syncDelayFile.done is closed.
   261  	errCh := make(chan error, cap(p.commitQueueSem))
   262  	walDone.Add(cap(errCh))
   263  	for i := 0; i < cap(errCh); i++ {
   264  		go func(i int) {
   265  			b := &Batch{}
   266  			if err := b.LogData([]byte("foo"), nil); err != nil {
   267  				errCh <- err
   268  				return
   269  			}
   270  			errCh <- p.Commit(b, true /* sync */, false)
   271  		}(i)
   272  	}
   273  
   274  	// Wait for all of the WAL writes to queue up. This ensures we don't violate
   275  	// the concurrency requirements of LogWriter, and also ensures all of the WAL
   276  	// writes are queued.
   277  	walDone.Wait()
   278  	close(sf.done)
   279  
   280  	// Close the WAL. A "queue is full" panic means that something is broken.
   281  	require.NoError(t, wal.Close())
   282  	for i := 0; i < cap(errCh); i++ {
   283  		require.NoError(t, <-errCh)
   284  	}
   285  }
   286  
   287  func BenchmarkCommitPipeline(b *testing.B) {
   288  	for _, noSyncWait := range []bool{false, true} {
   289  		for _, parallelism := range []int{1, 2, 4, 8, 16, 32, 64, 128} {
   290  			b.Run(fmt.Sprintf("no-sync-wait=%t/parallel=%d", noSyncWait, parallelism),
   291  				func(b *testing.B) {
   292  					b.SetParallelism(parallelism)
   293  					mem := newMemTable(memTableOptions{})
   294  					var wal *record.LogWriter
   295  					nullCommitEnv := commitEnv{
   296  						logSeqNum:     new(atomic.Uint64),
   297  						visibleSeqNum: new(atomic.Uint64),
   298  						apply: func(b *Batch, mem *memTable) error {
   299  							err := mem.apply(b, b.SeqNum())
   300  							if err != nil {
   301  								return err
   302  							}
   303  							mem.writerUnref()
   304  							return nil
   305  						},
   306  						write: func(b *Batch, syncWG *sync.WaitGroup, syncErr *error) (*memTable, error) {
   307  							for {
   308  								err := mem.prepare(b)
   309  								if err == arenaskl.ErrArenaFull {
   310  									mem = newMemTable(memTableOptions{})
   311  									continue
   312  								}
   313  								if err != nil {
   314  									return nil, err
   315  								}
   316  								break
   317  							}
   318  
   319  							_, err := wal.SyncRecord(b.data, syncWG, syncErr)
   320  							return mem, err
   321  						},
   322  					}
   323  					p := newCommitPipeline(nullCommitEnv)
   324  					wal = record.NewLogWriter(io.Discard, 0, /* logNum */
   325  						record.LogWriterConfig{
   326  							WALFsyncLatency: prometheus.NewHistogram(prometheus.HistogramOpts{}),
   327  							QueueSemChan:    p.logSyncQSem,
   328  						})
   329  					const keySize = 8
   330  					b.SetBytes(2 * keySize)
   331  					b.ResetTimer()
   332  
   333  					b.RunParallel(func(pb *testing.PB) {
   334  						rng := rand.New(rand.NewSource(uint64(time.Now().UnixNano())))
   335  						buf := make([]byte, keySize)
   336  
   337  						for pb.Next() {
   338  							batch := newBatch(nil)
   339  							binary.BigEndian.PutUint64(buf, rng.Uint64())
   340  							batch.Set(buf, buf, nil)
   341  							if err := p.Commit(batch, true /* sync */, noSyncWait); err != nil {
   342  								b.Fatal(err)
   343  							}
   344  							if noSyncWait {
   345  								if err := batch.SyncWait(); err != nil {
   346  									b.Fatal(err)
   347  								}
   348  							}
   349  							batch.release()
   350  						}
   351  					})
   352  				})
   353  		}
   354  	}
   355  }