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

     1  // Copyright 2013 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  	"bytes"
     9  	"context"
    10  	crand "crypto/rand"
    11  	"fmt"
    12  	"math"
    13  	"math/rand"
    14  	"path/filepath"
    15  	"reflect"
    16  	"regexp"
    17  	"runtime"
    18  	"sort"
    19  	"strconv"
    20  	"strings"
    21  	"sync/atomic"
    22  	"testing"
    23  	"time"
    24  
    25  	"github.com/cockroachdb/datadriven"
    26  	"github.com/cockroachdb/errors"
    27  	"github.com/cockroachdb/errors/oserror"
    28  	"github.com/cockroachdb/pebble/internal/base"
    29  	"github.com/cockroachdb/pebble/internal/keyspan"
    30  	"github.com/cockroachdb/pebble/internal/manifest"
    31  	"github.com/cockroachdb/pebble/objstorage"
    32  	"github.com/cockroachdb/pebble/objstorage/objstorageprovider"
    33  	"github.com/cockroachdb/pebble/objstorage/remote"
    34  	"github.com/cockroachdb/pebble/sstable"
    35  	"github.com/cockroachdb/pebble/vfs"
    36  	"github.com/cockroachdb/pebble/vfs/errorfs"
    37  	"github.com/stretchr/testify/require"
    38  )
    39  
    40  func newVersion(opts *Options, files [numLevels][]*fileMetadata) *version {
    41  	return manifest.NewVersion(
    42  		opts.Comparer.Compare,
    43  		opts.Comparer.FormatKey,
    44  		opts.FlushSplitBytes,
    45  		files)
    46  }
    47  
    48  type compactionPickerForTesting struct {
    49  	score         float64
    50  	level         int
    51  	baseLevel     int
    52  	opts          *Options
    53  	vers          *manifest.Version
    54  	maxLevelBytes [7]int64
    55  }
    56  
    57  var _ compactionPicker = &compactionPickerForTesting{}
    58  
    59  func (p *compactionPickerForTesting) getScores([]compactionInfo) [numLevels]float64 {
    60  	return [numLevels]float64{}
    61  }
    62  
    63  func (p *compactionPickerForTesting) getBaseLevel() int {
    64  	return p.baseLevel
    65  }
    66  
    67  func (p *compactionPickerForTesting) estimatedCompactionDebt(l0ExtraSize uint64) uint64 {
    68  	return 0
    69  }
    70  
    71  func (p *compactionPickerForTesting) forceBaseLevel1() {}
    72  
    73  func (p *compactionPickerForTesting) pickAuto(env compactionEnv) (pc *pickedCompaction) {
    74  	if p.score < 1 {
    75  		return nil
    76  	}
    77  	outputLevel := p.level + 1
    78  	if p.level == 0 {
    79  		outputLevel = p.baseLevel
    80  	}
    81  	iter := p.vers.Levels[p.level].Iter()
    82  	iter.First()
    83  	cInfo := candidateLevelInfo{
    84  		level:       p.level,
    85  		outputLevel: outputLevel,
    86  		file:        iter.Take(),
    87  	}
    88  	if cInfo.level == 0 {
    89  		return pickL0(env, p.opts, p.vers, p.baseLevel)
    90  	}
    91  	return pickAutoLPositive(env, p.opts, p.vers, cInfo, p.baseLevel, p.maxLevelBytes)
    92  }
    93  
    94  func (p *compactionPickerForTesting) pickElisionOnlyCompaction(
    95  	env compactionEnv,
    96  ) (pc *pickedCompaction) {
    97  	return nil
    98  }
    99  
   100  func (p *compactionPickerForTesting) pickRewriteCompaction(
   101  	env compactionEnv,
   102  ) (pc *pickedCompaction) {
   103  	return nil
   104  }
   105  
   106  func (p *compactionPickerForTesting) pickReadTriggeredCompaction(
   107  	env compactionEnv,
   108  ) (pc *pickedCompaction) {
   109  	return nil
   110  }
   111  
   112  func TestPickCompaction(t *testing.T) {
   113  	fileNums := func(files manifest.LevelSlice) string {
   114  		var ss []string
   115  		files.Each(func(meta *fileMetadata) {
   116  			ss = append(ss, strconv.Itoa(int(meta.FileNum)))
   117  		})
   118  		sort.Strings(ss)
   119  		return strings.Join(ss, ",")
   120  	}
   121  
   122  	opts := (*Options)(nil).EnsureDefaults()
   123  	newFileMeta := func(fileNum FileNum, size uint64, smallest, largest base.InternalKey) *fileMetadata {
   124  		m := (&fileMetadata{
   125  			FileNum: fileNum,
   126  			Size:    size,
   127  		}).ExtendPointKeyBounds(opts.Comparer.Compare, smallest, largest)
   128  		m.InitPhysicalBacking()
   129  		return m
   130  	}
   131  
   132  	testCases := []struct {
   133  		desc      string
   134  		version   *version
   135  		picker    compactionPickerForTesting
   136  		want      string
   137  		wantMulti bool
   138  	}{
   139  		{
   140  			desc: "no compaction",
   141  			version: newVersion(opts, [numLevels][]*fileMetadata{
   142  				0: {
   143  					newFileMeta(
   144  						100,
   145  						1,
   146  						base.ParseInternalKey("i.SET.101"),
   147  						base.ParseInternalKey("j.SET.102"),
   148  					),
   149  				},
   150  			}),
   151  			want: "",
   152  		},
   153  
   154  		{
   155  			desc: "1 L0 file",
   156  			version: newVersion(opts, [numLevels][]*fileMetadata{
   157  				0: {
   158  					newFileMeta(
   159  						100,
   160  						1,
   161  						base.ParseInternalKey("i.SET.101"),
   162  						base.ParseInternalKey("j.SET.102"),
   163  					),
   164  				},
   165  			}),
   166  			picker: compactionPickerForTesting{
   167  				score:     99,
   168  				level:     0,
   169  				baseLevel: 1,
   170  			},
   171  			want: "100  ",
   172  		},
   173  
   174  		{
   175  			desc: "2 L0 files (0 overlaps)",
   176  			version: newVersion(opts, [numLevels][]*fileMetadata{
   177  				0: {
   178  					newFileMeta(
   179  						100,
   180  						1,
   181  						base.ParseInternalKey("i.SET.101"),
   182  						base.ParseInternalKey("j.SET.102"),
   183  					),
   184  					newFileMeta(
   185  						110,
   186  						1,
   187  						base.ParseInternalKey("k.SET.111"),
   188  						base.ParseInternalKey("l.SET.112"),
   189  					),
   190  				},
   191  			}),
   192  			picker: compactionPickerForTesting{
   193  				score:     99,
   194  				level:     0,
   195  				baseLevel: 1,
   196  			},
   197  			want: "100,110  ",
   198  		},
   199  
   200  		{
   201  			desc: "2 L0 files, with ikey overlap",
   202  			version: newVersion(opts, [numLevels][]*fileMetadata{
   203  				0: {
   204  					newFileMeta(
   205  						100,
   206  						1,
   207  						base.ParseInternalKey("i.SET.101"),
   208  						base.ParseInternalKey("p.SET.102"),
   209  					),
   210  					newFileMeta(
   211  						110,
   212  						1,
   213  						base.ParseInternalKey("j.SET.111"),
   214  						base.ParseInternalKey("q.SET.112"),
   215  					),
   216  				},
   217  			}),
   218  			picker: compactionPickerForTesting{
   219  				score:     99,
   220  				level:     0,
   221  				baseLevel: 1,
   222  			},
   223  			want: "100,110  ",
   224  		},
   225  
   226  		{
   227  			desc: "2 L0 files, with ukey overlap",
   228  			version: newVersion(opts, [numLevels][]*fileMetadata{
   229  				0: {
   230  					newFileMeta(
   231  						100,
   232  						1,
   233  						base.ParseInternalKey("i.SET.101"),
   234  						base.ParseInternalKey("i.SET.102"),
   235  					),
   236  					newFileMeta(
   237  						110,
   238  						1,
   239  						base.ParseInternalKey("i.SET.111"),
   240  						base.ParseInternalKey("i.SET.112"),
   241  					),
   242  				},
   243  			}),
   244  			picker: compactionPickerForTesting{
   245  				score:     99,
   246  				level:     0,
   247  				baseLevel: 1,
   248  			},
   249  			want: "100,110  ",
   250  		},
   251  
   252  		{
   253  			desc: "1 L0 file, 2 L1 files (0 overlaps)",
   254  			version: newVersion(opts, [numLevels][]*fileMetadata{
   255  				0: {
   256  					newFileMeta(
   257  						100,
   258  						1,
   259  						base.ParseInternalKey("i.SET.101"),
   260  						base.ParseInternalKey("i.SET.102"),
   261  					),
   262  				},
   263  				1: {
   264  					newFileMeta(
   265  						200,
   266  						1,
   267  						base.ParseInternalKey("a.SET.201"),
   268  						base.ParseInternalKey("b.SET.202"),
   269  					),
   270  					newFileMeta(
   271  						210,
   272  						1,
   273  						base.ParseInternalKey("y.SET.211"),
   274  						base.ParseInternalKey("z.SET.212"),
   275  					),
   276  				},
   277  			}),
   278  			picker: compactionPickerForTesting{
   279  				score:     99,
   280  				level:     0,
   281  				baseLevel: 1,
   282  			},
   283  			want: "100  ",
   284  		},
   285  
   286  		{
   287  			desc: "1 L0 file, 2 L1 files (1 overlap), 4 L2 files (3 overlaps)",
   288  			version: newVersion(opts, [numLevels][]*fileMetadata{
   289  				0: {
   290  					newFileMeta(
   291  						100,
   292  						1,
   293  						base.ParseInternalKey("i.SET.101"),
   294  						base.ParseInternalKey("t.SET.102"),
   295  					),
   296  				},
   297  				1: {
   298  					newFileMeta(
   299  						200,
   300  						1,
   301  						base.ParseInternalKey("a.SET.201"),
   302  						base.ParseInternalKey("e.SET.202"),
   303  					),
   304  					newFileMeta(
   305  						210,
   306  						1,
   307  						base.ParseInternalKey("f.SET.211"),
   308  						base.ParseInternalKey("j.SET.212"),
   309  					),
   310  				},
   311  				2: {
   312  					newFileMeta(
   313  						300,
   314  						1,
   315  						base.ParseInternalKey("a.SET.301"),
   316  						base.ParseInternalKey("b.SET.302"),
   317  					),
   318  					newFileMeta(
   319  						310,
   320  						1,
   321  						base.ParseInternalKey("c.SET.311"),
   322  						base.ParseInternalKey("g.SET.312"),
   323  					),
   324  					newFileMeta(
   325  						320,
   326  						1,
   327  						base.ParseInternalKey("h.SET.321"),
   328  						base.ParseInternalKey("m.SET.322"),
   329  					),
   330  					newFileMeta(
   331  						330,
   332  						1,
   333  						base.ParseInternalKey("n.SET.331"),
   334  						base.ParseInternalKey("z.SET.332"),
   335  					),
   336  				},
   337  			}),
   338  			picker: compactionPickerForTesting{
   339  				score:     99,
   340  				level:     0,
   341  				baseLevel: 1,
   342  			},
   343  			want: "100 210 310,320,330",
   344  		},
   345  
   346  		{
   347  			desc: "4 L1 files, 2 L2 files, can grow",
   348  			version: newVersion(opts, [numLevels][]*fileMetadata{
   349  				1: {
   350  					newFileMeta(
   351  						200,
   352  						1,
   353  						base.ParseInternalKey("i1.SET.201"),
   354  						base.ParseInternalKey("i2.SET.202"),
   355  					),
   356  					newFileMeta(
   357  						210,
   358  						1,
   359  						base.ParseInternalKey("j1.SET.211"),
   360  						base.ParseInternalKey("j2.SET.212"),
   361  					),
   362  					newFileMeta(
   363  						220,
   364  						1,
   365  						base.ParseInternalKey("k1.SET.221"),
   366  						base.ParseInternalKey("k2.SET.222"),
   367  					),
   368  					newFileMeta(
   369  						230,
   370  						1,
   371  						base.ParseInternalKey("l1.SET.231"),
   372  						base.ParseInternalKey("l2.SET.232"),
   373  					),
   374  				},
   375  				2: {
   376  					newFileMeta(
   377  						300,
   378  						1,
   379  						base.ParseInternalKey("a0.SET.301"),
   380  						base.ParseInternalKey("l0.SET.302"),
   381  					),
   382  					newFileMeta(
   383  						310,
   384  						1,
   385  						base.ParseInternalKey("l2.SET.311"),
   386  						base.ParseInternalKey("z2.SET.312"),
   387  					),
   388  				},
   389  			}),
   390  			picker: compactionPickerForTesting{
   391  				score:     99,
   392  				level:     1,
   393  				baseLevel: 1,
   394  			},
   395  			want:      "200,210,220 300  ",
   396  			wantMulti: true,
   397  		},
   398  
   399  		{
   400  			desc: "4 L1 files, 2 L2 files, can't grow (range)",
   401  			version: newVersion(opts, [numLevels][]*fileMetadata{
   402  				1: {
   403  					newFileMeta(
   404  						200,
   405  						1,
   406  						base.ParseInternalKey("i1.SET.201"),
   407  						base.ParseInternalKey("i2.SET.202"),
   408  					),
   409  					newFileMeta(
   410  						210,
   411  						1,
   412  						base.ParseInternalKey("j1.SET.211"),
   413  						base.ParseInternalKey("j2.SET.212"),
   414  					),
   415  					newFileMeta(
   416  						220,
   417  						1,
   418  						base.ParseInternalKey("k1.SET.221"),
   419  						base.ParseInternalKey("k2.SET.222"),
   420  					),
   421  					newFileMeta(
   422  						230,
   423  						1,
   424  						base.ParseInternalKey("l1.SET.231"),
   425  						base.ParseInternalKey("l2.SET.232"),
   426  					),
   427  				},
   428  				2: {
   429  					newFileMeta(
   430  						300,
   431  						1,
   432  						base.ParseInternalKey("a0.SET.301"),
   433  						base.ParseInternalKey("j0.SET.302"),
   434  					),
   435  					newFileMeta(
   436  						310,
   437  						1,
   438  						base.ParseInternalKey("j2.SET.311"),
   439  						base.ParseInternalKey("z2.SET.312"),
   440  					),
   441  				},
   442  			}),
   443  			picker: compactionPickerForTesting{
   444  				score:     99,
   445  				level:     1,
   446  				baseLevel: 1,
   447  			},
   448  			want:      "200 300  ",
   449  			wantMulti: true,
   450  		},
   451  
   452  		{
   453  			desc: "4 L1 files, 2 L2 files, can't grow (size)",
   454  			version: newVersion(opts, [numLevels][]*fileMetadata{
   455  				1: {
   456  					newFileMeta(
   457  						200,
   458  						expandedCompactionByteSizeLimit(opts, 1, math.MaxUint64)-1,
   459  						base.ParseInternalKey("i1.SET.201"),
   460  						base.ParseInternalKey("i2.SET.202"),
   461  					),
   462  					newFileMeta(
   463  						210,
   464  						expandedCompactionByteSizeLimit(opts, 1, math.MaxUint64)-1,
   465  						base.ParseInternalKey("j1.SET.211"),
   466  						base.ParseInternalKey("j2.SET.212"),
   467  					),
   468  					newFileMeta(
   469  						220,
   470  						expandedCompactionByteSizeLimit(opts, 1, math.MaxUint64)-1,
   471  						base.ParseInternalKey("k1.SET.221"),
   472  						base.ParseInternalKey("k2.SET.222"),
   473  					),
   474  					newFileMeta(
   475  						230,
   476  						expandedCompactionByteSizeLimit(opts, 1, math.MaxUint64)-1,
   477  						base.ParseInternalKey("l1.SET.231"),
   478  						base.ParseInternalKey("l2.SET.232"),
   479  					),
   480  				},
   481  				2: {
   482  					newFileMeta(
   483  						300,
   484  						expandedCompactionByteSizeLimit(opts, 2, math.MaxUint64)-1,
   485  						base.ParseInternalKey("a0.SET.301"),
   486  						base.ParseInternalKey("l0.SET.302"),
   487  					),
   488  					newFileMeta(
   489  						310,
   490  						expandedCompactionByteSizeLimit(opts, 2, math.MaxUint64)-1,
   491  						base.ParseInternalKey("l2.SET.311"),
   492  						base.ParseInternalKey("z2.SET.312"),
   493  					),
   494  				},
   495  			}),
   496  			picker: compactionPickerForTesting{
   497  				score:     99,
   498  				level:     1,
   499  				baseLevel: 1,
   500  			},
   501  			want: "200 300 ",
   502  		},
   503  	}
   504  
   505  	for _, tc := range testCases {
   506  		vs := &versionSet{
   507  			opts:    opts,
   508  			cmp:     DefaultComparer.Compare,
   509  			cmpName: DefaultComparer.Name,
   510  		}
   511  		vs.versions.Init(nil)
   512  		vs.append(tc.version)
   513  		tc.picker.opts = opts
   514  		tc.picker.vers = tc.version
   515  		vs.picker = &tc.picker
   516  		pc, got := vs.picker.pickAuto(compactionEnv{diskAvailBytes: math.MaxUint64}), ""
   517  		if pc != nil {
   518  			c := newCompaction(pc, opts, time.Now(), nil /* provider */)
   519  
   520  			gotStart := fileNums(c.startLevel.files)
   521  			gotML := ""
   522  			observedMulti := len(c.extraLevels) > 0
   523  			if observedMulti {
   524  				gotML = " " + fileNums(c.extraLevels[0].files)
   525  			}
   526  			gotOutput := " " + fileNums(c.outputLevel.files)
   527  			gotGrandparents := " " + fileNums(c.grandparents)
   528  			got = gotStart + gotML + gotOutput + gotGrandparents
   529  			if tc.wantMulti != observedMulti {
   530  				t.Fatalf("Expected Multi %t; Observed Multi %t, for %s", tc.wantMulti, observedMulti, got)
   531  			}
   532  
   533  		}
   534  		if got != tc.want {
   535  			t.Fatalf("%s:\ngot  %q\nwant %q", tc.desc, got, tc.want)
   536  		}
   537  	}
   538  }
   539  
   540  func TestElideTombstone(t *testing.T) {
   541  	opts := (&Options{}).EnsureDefaults().WithFSDefaults()
   542  
   543  	newFileMeta := func(smallest, largest base.InternalKey) *fileMetadata {
   544  		m := (&fileMetadata{}).ExtendPointKeyBounds(opts.Comparer.Compare, smallest, largest)
   545  		m.InitPhysicalBacking()
   546  		return m
   547  	}
   548  
   549  	type want struct {
   550  		key      string
   551  		expected bool
   552  	}
   553  
   554  	testCases := []struct {
   555  		desc    string
   556  		level   int
   557  		version *version
   558  		wants   []want
   559  	}{
   560  		{
   561  			desc:    "empty",
   562  			level:   1,
   563  			version: newVersion(opts, [numLevels][]*fileMetadata{}),
   564  			wants: []want{
   565  				{"x", true},
   566  			},
   567  		},
   568  		{
   569  			desc:  "non-empty",
   570  			level: 1,
   571  			version: newVersion(opts, [numLevels][]*fileMetadata{
   572  				1: {
   573  					newFileMeta(
   574  						base.ParseInternalKey("c.SET.801"),
   575  						base.ParseInternalKey("g.SET.800"),
   576  					),
   577  					newFileMeta(
   578  						base.ParseInternalKey("x.SET.701"),
   579  						base.ParseInternalKey("y.SET.700"),
   580  					),
   581  				},
   582  				2: {
   583  					newFileMeta(
   584  						base.ParseInternalKey("d.SET.601"),
   585  						base.ParseInternalKey("h.SET.600"),
   586  					),
   587  					newFileMeta(
   588  						base.ParseInternalKey("r.SET.501"),
   589  						base.ParseInternalKey("t.SET.500"),
   590  					),
   591  				},
   592  				3: {
   593  					newFileMeta(
   594  						base.ParseInternalKey("f.SET.401"),
   595  						base.ParseInternalKey("g.SET.400"),
   596  					),
   597  					newFileMeta(
   598  						base.ParseInternalKey("w.SET.301"),
   599  						base.ParseInternalKey("x.SET.300"),
   600  					),
   601  				},
   602  				4: {
   603  					newFileMeta(
   604  						base.ParseInternalKey("f.SET.201"),
   605  						base.ParseInternalKey("m.SET.200"),
   606  					),
   607  					newFileMeta(
   608  						base.ParseInternalKey("t.SET.101"),
   609  						base.ParseInternalKey("t.SET.100"),
   610  					),
   611  				},
   612  			}),
   613  			wants: []want{
   614  				{"b", true},
   615  				{"c", true},
   616  				{"d", true},
   617  				{"e", true},
   618  				{"f", false},
   619  				{"g", false},
   620  				{"h", false},
   621  				{"l", false},
   622  				{"m", false},
   623  				{"n", true},
   624  				{"q", true},
   625  				{"r", true},
   626  				{"s", true},
   627  				{"t", false},
   628  				{"u", true},
   629  				{"v", true},
   630  				{"w", false},
   631  				{"x", false},
   632  				{"y", true},
   633  				{"z", true},
   634  			},
   635  		},
   636  		{
   637  			desc:  "repeated ukey",
   638  			level: 1,
   639  			version: newVersion(opts, [numLevels][]*fileMetadata{
   640  				6: {
   641  					newFileMeta(
   642  						base.ParseInternalKey("i.SET.401"),
   643  						base.ParseInternalKey("i.SET.400"),
   644  					),
   645  					newFileMeta(
   646  						base.ParseInternalKey("i.SET.301"),
   647  						base.ParseInternalKey("k.SET.300"),
   648  					),
   649  					newFileMeta(
   650  						base.ParseInternalKey("k.SET.201"),
   651  						base.ParseInternalKey("m.SET.200"),
   652  					),
   653  					newFileMeta(
   654  						base.ParseInternalKey("m.SET.101"),
   655  						base.ParseInternalKey("m.SET.100"),
   656  					),
   657  				},
   658  			}),
   659  			wants: []want{
   660  				{"h", true},
   661  				{"i", false},
   662  				{"j", false},
   663  				{"k", false},
   664  				{"l", false},
   665  				{"m", false},
   666  				{"n", true},
   667  			},
   668  		},
   669  	}
   670  
   671  	for _, tc := range testCases {
   672  		c := compaction{
   673  			cmp:      DefaultComparer.Compare,
   674  			comparer: DefaultComparer,
   675  			version:  tc.version,
   676  			inputs:   []compactionLevel{{level: tc.level}, {level: tc.level + 1}},
   677  			smallest: base.ParseInternalKey("a.SET.0"),
   678  			largest:  base.ParseInternalKey("z.SET.0"),
   679  		}
   680  		c.startLevel, c.outputLevel = &c.inputs[0], &c.inputs[1]
   681  		c.setupInuseKeyRanges()
   682  		for _, w := range tc.wants {
   683  			if got := c.elideTombstone([]byte(w.key)); got != w.expected {
   684  				t.Errorf("%s: ukey=%q: got %v, want %v", tc.desc, w.key, got, w.expected)
   685  			}
   686  		}
   687  	}
   688  }
   689  
   690  func TestElideRangeTombstone(t *testing.T) {
   691  	opts := (*Options)(nil).EnsureDefaults()
   692  
   693  	newFileMeta := func(smallest, largest base.InternalKey) *fileMetadata {
   694  		m := (&fileMetadata{}).ExtendPointKeyBounds(
   695  			opts.Comparer.Compare, smallest, largest,
   696  		)
   697  		m.InitPhysicalBacking()
   698  		return m
   699  	}
   700  
   701  	type want struct {
   702  		key      string
   703  		endKey   string
   704  		expected bool
   705  	}
   706  
   707  	testCases := []struct {
   708  		desc     string
   709  		level    int
   710  		version  *version
   711  		wants    []want
   712  		flushing flushableList
   713  	}{
   714  		{
   715  			desc:    "empty",
   716  			level:   1,
   717  			version: newVersion(opts, [numLevels][]*fileMetadata{}),
   718  			wants: []want{
   719  				{"x", "y", true},
   720  			},
   721  		},
   722  		{
   723  			desc:  "non-empty",
   724  			level: 1,
   725  			version: newVersion(opts, [numLevels][]*fileMetadata{
   726  				1: {
   727  					newFileMeta(
   728  						base.ParseInternalKey("c.SET.801"),
   729  						base.ParseInternalKey("g.SET.800"),
   730  					),
   731  					newFileMeta(
   732  						base.ParseInternalKey("x.SET.701"),
   733  						base.ParseInternalKey("y.SET.700"),
   734  					),
   735  				},
   736  				2: {
   737  					newFileMeta(
   738  						base.ParseInternalKey("d.SET.601"),
   739  						base.ParseInternalKey("h.SET.600"),
   740  					),
   741  					newFileMeta(
   742  						base.ParseInternalKey("r.SET.501"),
   743  						base.ParseInternalKey("t.SET.500"),
   744  					),
   745  				},
   746  				3: {
   747  					newFileMeta(
   748  						base.ParseInternalKey("f.SET.401"),
   749  						base.ParseInternalKey("g.SET.400"),
   750  					),
   751  					newFileMeta(
   752  						base.ParseInternalKey("w.SET.301"),
   753  						base.ParseInternalKey("x.SET.300"),
   754  					),
   755  				},
   756  				4: {
   757  					newFileMeta(
   758  						base.ParseInternalKey("f.SET.201"),
   759  						base.ParseInternalKey("m.SET.200"),
   760  					),
   761  					newFileMeta(
   762  						base.ParseInternalKey("t.SET.101"),
   763  						base.ParseInternalKey("t.SET.100"),
   764  					),
   765  				},
   766  			}),
   767  			wants: []want{
   768  				{"b", "c", true},
   769  				{"c", "d", true},
   770  				{"d", "e", true},
   771  				{"e", "f", false},
   772  				{"f", "g", false},
   773  				{"g", "h", false},
   774  				{"h", "i", false},
   775  				{"l", "m", false},
   776  				{"m", "n", false},
   777  				{"n", "o", true},
   778  				{"q", "r", true},
   779  				{"r", "s", true},
   780  				{"s", "t", false},
   781  				{"t", "u", false},
   782  				{"u", "v", true},
   783  				{"v", "w", false},
   784  				{"w", "x", false},
   785  				{"x", "y", false},
   786  				{"y", "z", true},
   787  			},
   788  		},
   789  		{
   790  			desc:  "flushing",
   791  			level: -1,
   792  			version: newVersion(opts, [numLevels][]*fileMetadata{
   793  				0: {
   794  					newFileMeta(
   795  						base.ParseInternalKey("h.SET.901"),
   796  						base.ParseInternalKey("j.SET.900"),
   797  					),
   798  				},
   799  				1: {
   800  					newFileMeta(
   801  						base.ParseInternalKey("c.SET.801"),
   802  						base.ParseInternalKey("g.SET.800"),
   803  					),
   804  					newFileMeta(
   805  						base.ParseInternalKey("x.SET.701"),
   806  						base.ParseInternalKey("y.SET.700"),
   807  					),
   808  				},
   809  			}),
   810  			wants: []want{
   811  				{"m", "n", false},
   812  			},
   813  			// Pretend one memtable is being flushed
   814  			flushing: flushableList{nil},
   815  		},
   816  	}
   817  
   818  	for _, tc := range testCases {
   819  		c := compaction{
   820  			cmp:      DefaultComparer.Compare,
   821  			comparer: DefaultComparer,
   822  			version:  tc.version,
   823  			inputs:   []compactionLevel{{level: tc.level}, {level: tc.level + 1}},
   824  			smallest: base.ParseInternalKey("a.SET.0"),
   825  			largest:  base.ParseInternalKey("z.SET.0"),
   826  			flushing: tc.flushing,
   827  		}
   828  		c.startLevel, c.outputLevel = &c.inputs[0], &c.inputs[1]
   829  		c.setupInuseKeyRanges()
   830  		for _, w := range tc.wants {
   831  			if got := c.elideRangeTombstone([]byte(w.key), []byte(w.endKey)); got != w.expected {
   832  				t.Errorf("%s: keys=%q-%q: got %v, want %v", tc.desc, w.key, w.endKey, got, w.expected)
   833  			}
   834  		}
   835  	}
   836  }
   837  
   838  func TestCompactionTransform(t *testing.T) {
   839  	datadriven.RunTest(t, "testdata/compaction_transform", func(t *testing.T, td *datadriven.TestData) string {
   840  		switch td.Cmd {
   841  		case "transform":
   842  			var snapshots []uint64
   843  			var keyRanges []manifest.UserKeyRange
   844  			disableElision := td.HasArg("disable-elision")
   845  			td.MaybeScanArgs(t, "snapshots", &snapshots)
   846  			if arg, ok := td.Arg("in-use-key-ranges"); ok {
   847  				for _, keyRange := range arg.Vals {
   848  					parts := strings.SplitN(keyRange, "-", 2)
   849  					start := []byte(strings.TrimSpace(parts[0]))
   850  					end := []byte(strings.TrimSpace(parts[1]))
   851  					keyRanges = append(keyRanges, manifest.UserKeyRange{
   852  						Start: start,
   853  						End:   end,
   854  					})
   855  				}
   856  			}
   857  			span := keyspan.ParseSpan(td.Input)
   858  			for i := range span.Keys {
   859  				if i > 0 {
   860  					if span.Keys[i-1].Trailer < span.Keys[i].Trailer {
   861  						return "span keys not sorted"
   862  					}
   863  				}
   864  			}
   865  			var outSpan keyspan.Span
   866  			c := compaction{
   867  				cmp:                base.DefaultComparer.Compare,
   868  				comparer:           base.DefaultComparer,
   869  				disableSpanElision: disableElision,
   870  				inuseKeyRanges:     keyRanges,
   871  			}
   872  			transformer := rangeKeyCompactionTransform(base.DefaultComparer.Equal, snapshots, c.elideRangeTombstone)
   873  			if err := transformer.Transform(base.DefaultComparer.Compare, span, &outSpan); err != nil {
   874  				return fmt.Sprintf("error: %s", err)
   875  			}
   876  			return outSpan.String()
   877  		default:
   878  			return fmt.Sprintf("unknown command: %s", td.Cmd)
   879  		}
   880  	})
   881  }
   882  
   883  type cpuPermissionGranter struct {
   884  	// requestCount is used to confirm that every GetPermission function call
   885  	// has a corresponding CPUWorkDone function call.
   886  	requestCount int
   887  	used         bool
   888  	permit       bool
   889  }
   890  
   891  type cpuWorkHandle struct {
   892  	permit bool
   893  }
   894  
   895  func (c cpuWorkHandle) Permitted() bool {
   896  	return c.permit
   897  }
   898  
   899  func (t *cpuPermissionGranter) GetPermission(dur time.Duration) CPUWorkHandle {
   900  	t.requestCount++
   901  	t.used = true
   902  	return cpuWorkHandle{t.permit}
   903  }
   904  
   905  func (t *cpuPermissionGranter) CPUWorkDone(_ CPUWorkHandle) {
   906  	t.requestCount--
   907  }
   908  
   909  // Simple test to check if compactions are using the granter, and if exactly
   910  // the acquired handles are returned.
   911  func TestCompactionCPUGranter(t *testing.T) {
   912  	mem := vfs.NewMem()
   913  	opts := (&Options{FS: mem}).WithFSDefaults()
   914  	g := &cpuPermissionGranter{permit: true}
   915  	opts.Experimental.CPUWorkPermissionGranter = g
   916  	d, err := Open("", opts)
   917  	if err != nil {
   918  		t.Fatalf("Open: %v", err)
   919  	}
   920  	defer d.Close()
   921  
   922  	d.Set([]byte{'a'}, []byte{'a'}, nil)
   923  	err = d.Compact([]byte{'a'}, []byte{'b'}, true)
   924  	if err != nil {
   925  		t.Fatalf("Compact: %v", err)
   926  	}
   927  	require.True(t, g.used)
   928  	require.Equal(t, g.requestCount, 0)
   929  }
   930  
   931  // Tests that there's no errors or panics when the default CPU granter is used.
   932  func TestCompactionCPUGranterDefault(t *testing.T) {
   933  	mem := vfs.NewMem()
   934  	opts := (&Options{FS: mem}).WithFSDefaults()
   935  	d, err := Open("", opts)
   936  	if err != nil {
   937  		t.Fatalf("Open: %v", err)
   938  	}
   939  	defer d.Close()
   940  
   941  	d.Set([]byte{'a'}, []byte{'a'}, nil)
   942  	err = d.Compact([]byte{'a'}, []byte{'b'}, true)
   943  	if err != nil {
   944  		t.Fatalf("Compact: %v", err)
   945  	}
   946  }
   947  
   948  func TestCompaction(t *testing.T) {
   949  	const memTableSize = 10000
   950  	// Tuned so that 2 values can reside in the memtable before a flush, but a
   951  	// 3rd value will cause a flush. Needs to account for the max skiplist node
   952  	// size.
   953  	const valueSize = 3500
   954  
   955  	mem := vfs.NewMem()
   956  	opts := &Options{
   957  		FS:                    mem,
   958  		MemTableSize:          memTableSize,
   959  		DebugCheck:            DebugCheckLevels,
   960  		L0CompactionThreshold: 8,
   961  	}
   962  	opts.testingRandomized(t).WithFSDefaults()
   963  	d, err := Open("", opts)
   964  	if err != nil {
   965  		t.Fatalf("Open: %v", err)
   966  	}
   967  
   968  	get1 := func(iter internalIterator) (ret string) {
   969  		b := &bytes.Buffer{}
   970  		for key, _ := iter.First(); key != nil; key, _ = iter.Next() {
   971  			b.Write(key.UserKey)
   972  		}
   973  		if err := iter.Close(); err != nil {
   974  			t.Fatalf("iterator Close: %v", err)
   975  		}
   976  		return b.String()
   977  	}
   978  	getAll := func() (gotMem, gotDisk string, err error) {
   979  		d.mu.Lock()
   980  		defer d.mu.Unlock()
   981  
   982  		if d.mu.mem.mutable != nil {
   983  			gotMem = get1(d.mu.mem.mutable.newIter(nil))
   984  		}
   985  		ss := []string(nil)
   986  		v := d.mu.versions.currentVersion()
   987  		provider, err := objstorageprovider.Open(objstorageprovider.DefaultSettings(mem, "" /* dirName */))
   988  		if err != nil {
   989  			t.Fatalf("%v", err)
   990  		}
   991  		defer provider.Close()
   992  		for _, levelMetadata := range v.Levels {
   993  			iter := levelMetadata.Iter()
   994  			for meta := iter.First(); meta != nil; meta = iter.Next() {
   995  				if meta.Virtual {
   996  					continue
   997  				}
   998  				f, err := provider.OpenForReading(context.Background(), base.FileTypeTable, meta.FileBacking.DiskFileNum, objstorage.OpenOptions{})
   999  				if err != nil {
  1000  					return "", "", errors.WithStack(err)
  1001  				}
  1002  				r, err := sstable.NewReader(f, sstable.ReaderOptions{})
  1003  				if err != nil {
  1004  					return "", "", errors.WithStack(err)
  1005  				}
  1006  				defer r.Close()
  1007  				iter, err := r.NewIter(nil /* lower */, nil /* upper */)
  1008  				if err != nil {
  1009  					return "", "", errors.WithStack(err)
  1010  				}
  1011  				ss = append(ss, get1(iter)+".")
  1012  			}
  1013  		}
  1014  		sort.Strings(ss)
  1015  		return gotMem, strings.Join(ss, ""), nil
  1016  	}
  1017  
  1018  	value := bytes.Repeat([]byte("x"), valueSize)
  1019  	testCases := []struct {
  1020  		key, wantMem, wantDisk string
  1021  	}{
  1022  		{"+A", "A", ""},
  1023  		{"+a", "Aa", ""},
  1024  		{"+B", "B", "Aa."},
  1025  		{"+b", "Bb", "Aa."},
  1026  		// The next level-0 table overwrites the B key.
  1027  		{"+C", "C", "Aa.Bb."},
  1028  		{"+B", "BC", "Aa.Bb."},
  1029  		// The next level-0 table deletes the a key.
  1030  		{"+D", "D", "Aa.BC.Bb."},
  1031  		{"-a", "Da", "Aa.BC.Bb."},
  1032  		{"+d", "Dad", "Aa.BC.Bb."},
  1033  		{"+E", "E", "Aa.BC.Bb.Dad."},
  1034  		{"+e", "Ee", "Aa.BC.Bb.Dad."},
  1035  		// The next addition creates the fourth level-0 table, and l0CompactionTrigger == 8,
  1036  		// but since the sublevel count is doubled when comparing with l0CompactionTrigger,
  1037  		// the addition of the 4th sublevel triggers a non-trivial compaction into one level-1 table.
  1038  		// Note that the keys in this one larger table are interleaved from the four smaller ones.
  1039  		{"+F", "F", "ABCDEbde."},
  1040  	}
  1041  	for _, tc := range testCases {
  1042  		if key := tc.key[1:]; tc.key[0] == '+' {
  1043  			if err := d.Set([]byte(key), value, nil); err != nil {
  1044  				t.Errorf("%q: Set: %v", key, err)
  1045  				break
  1046  			}
  1047  		} else {
  1048  			if err := d.Delete([]byte(key), nil); err != nil {
  1049  				t.Errorf("%q: Delete: %v", key, err)
  1050  				break
  1051  			}
  1052  		}
  1053  
  1054  		// try backs off to allow any writes to the memfs to complete.
  1055  		err := try(100*time.Microsecond, 20*time.Second, func() error {
  1056  			gotMem, gotDisk, err := getAll()
  1057  			if err != nil {
  1058  				return err
  1059  			}
  1060  			if testing.Verbose() {
  1061  				fmt.Printf("mem=%s (%s) disk=%s (%s)\n", gotMem, tc.wantMem, gotDisk, tc.wantDisk)
  1062  			}
  1063  
  1064  			if gotMem != tc.wantMem {
  1065  				return errors.Errorf("mem: got %q, want %q", gotMem, tc.wantMem)
  1066  			}
  1067  			if gotDisk != tc.wantDisk {
  1068  				return errors.Errorf("ldb: got %q, want %q", gotDisk, tc.wantDisk)
  1069  			}
  1070  			return nil
  1071  		})
  1072  		if err != nil {
  1073  			t.Errorf("%q: %v", tc.key, err)
  1074  		}
  1075  	}
  1076  	if err := d.Close(); err != nil {
  1077  		t.Fatalf("db Close: %v", err)
  1078  	}
  1079  }
  1080  
  1081  func TestValidateVersionEdit(t *testing.T) {
  1082  	const badKey = "malformed-key"
  1083  
  1084  	errValidationFailed := errors.New("validation failed")
  1085  	validateFn := func(key []byte) error {
  1086  		if string(key) == badKey {
  1087  			return errValidationFailed
  1088  		}
  1089  		return nil
  1090  	}
  1091  
  1092  	cmp := DefaultComparer.Compare
  1093  	newFileMeta := func(smallest, largest base.InternalKey) *fileMetadata {
  1094  		m := (&fileMetadata{}).ExtendPointKeyBounds(cmp, smallest, largest)
  1095  		m.InitPhysicalBacking()
  1096  		return m
  1097  	}
  1098  
  1099  	testCases := []struct {
  1100  		desc    string
  1101  		ve      *versionEdit
  1102  		vFunc   func([]byte) error
  1103  		wantErr error
  1104  	}{
  1105  		{
  1106  			desc: "single new file; start key",
  1107  			ve: &versionEdit{
  1108  				NewFiles: []manifest.NewFileEntry{
  1109  					{
  1110  						Meta: newFileMeta(
  1111  							manifest.InternalKey{UserKey: []byte(badKey)},
  1112  							manifest.InternalKey{UserKey: []byte("z")},
  1113  						),
  1114  					},
  1115  				},
  1116  			},
  1117  			vFunc:   validateFn,
  1118  			wantErr: errValidationFailed,
  1119  		},
  1120  		{
  1121  			desc: "single new file; end key",
  1122  			ve: &versionEdit{
  1123  				NewFiles: []manifest.NewFileEntry{
  1124  					{
  1125  						Meta: newFileMeta(
  1126  							manifest.InternalKey{UserKey: []byte("a")},
  1127  							manifest.InternalKey{UserKey: []byte(badKey)},
  1128  						),
  1129  					},
  1130  				},
  1131  			},
  1132  			vFunc:   validateFn,
  1133  			wantErr: errValidationFailed,
  1134  		},
  1135  		{
  1136  			desc: "multiple new files",
  1137  			ve: &versionEdit{
  1138  				NewFiles: []manifest.NewFileEntry{
  1139  					{
  1140  						Meta: newFileMeta(
  1141  							manifest.InternalKey{UserKey: []byte("a")},
  1142  							manifest.InternalKey{UserKey: []byte("c")},
  1143  						),
  1144  					},
  1145  					{
  1146  						Meta: newFileMeta(
  1147  							manifest.InternalKey{UserKey: []byte(badKey)},
  1148  							manifest.InternalKey{UserKey: []byte("z")},
  1149  						),
  1150  					},
  1151  				},
  1152  			},
  1153  			vFunc:   validateFn,
  1154  			wantErr: errValidationFailed,
  1155  		},
  1156  		{
  1157  			desc: "single deleted file; start key",
  1158  			ve: &versionEdit{
  1159  				DeletedFiles: map[manifest.DeletedFileEntry]*manifest.FileMetadata{
  1160  					deletedFileEntry{Level: 0, FileNum: 0}: newFileMeta(
  1161  						manifest.InternalKey{UserKey: []byte(badKey)},
  1162  						manifest.InternalKey{UserKey: []byte("z")},
  1163  					),
  1164  				},
  1165  			},
  1166  			vFunc:   validateFn,
  1167  			wantErr: errValidationFailed,
  1168  		},
  1169  		{
  1170  			desc: "single deleted file; end key",
  1171  			ve: &versionEdit{
  1172  				DeletedFiles: map[manifest.DeletedFileEntry]*manifest.FileMetadata{
  1173  					deletedFileEntry{Level: 0, FileNum: 0}: newFileMeta(
  1174  						manifest.InternalKey{UserKey: []byte("a")},
  1175  						manifest.InternalKey{UserKey: []byte(badKey)},
  1176  					),
  1177  				},
  1178  			},
  1179  			vFunc:   validateFn,
  1180  			wantErr: errValidationFailed,
  1181  		},
  1182  		{
  1183  			desc: "multiple deleted files",
  1184  			ve: &versionEdit{
  1185  				DeletedFiles: map[manifest.DeletedFileEntry]*manifest.FileMetadata{
  1186  					deletedFileEntry{Level: 0, FileNum: 0}: newFileMeta(
  1187  						manifest.InternalKey{UserKey: []byte("a")},
  1188  						manifest.InternalKey{UserKey: []byte("c")},
  1189  					),
  1190  					deletedFileEntry{Level: 0, FileNum: 1}: newFileMeta(
  1191  						manifest.InternalKey{UserKey: []byte(badKey)},
  1192  						manifest.InternalKey{UserKey: []byte("z")},
  1193  					),
  1194  				},
  1195  			},
  1196  			vFunc:   validateFn,
  1197  			wantErr: errValidationFailed,
  1198  		},
  1199  		{
  1200  			desc: "no errors",
  1201  			ve: &versionEdit{
  1202  				NewFiles: []manifest.NewFileEntry{
  1203  					{
  1204  						Level: 0,
  1205  						Meta: newFileMeta(
  1206  							manifest.InternalKey{UserKey: []byte("b")},
  1207  							manifest.InternalKey{UserKey: []byte("c")},
  1208  						),
  1209  					},
  1210  					{
  1211  						Level: 0,
  1212  						Meta: newFileMeta(
  1213  							manifest.InternalKey{UserKey: []byte("d")},
  1214  							manifest.InternalKey{UserKey: []byte("g")},
  1215  						),
  1216  					},
  1217  				},
  1218  				DeletedFiles: map[manifest.DeletedFileEntry]*manifest.FileMetadata{
  1219  					deletedFileEntry{Level: 6, FileNum: 0}: newFileMeta(
  1220  						manifest.InternalKey{UserKey: []byte("a")},
  1221  						manifest.InternalKey{UserKey: []byte("d")},
  1222  					),
  1223  					deletedFileEntry{Level: 6, FileNum: 1}: newFileMeta(
  1224  						manifest.InternalKey{UserKey: []byte("x")},
  1225  						manifest.InternalKey{UserKey: []byte("z")},
  1226  					),
  1227  				},
  1228  			},
  1229  			vFunc: validateFn,
  1230  		},
  1231  	}
  1232  
  1233  	for _, tc := range testCases {
  1234  		t.Run(tc.desc, func(t *testing.T) {
  1235  			err := validateVersionEdit(tc.ve, tc.vFunc, base.DefaultFormatter)
  1236  			if tc.wantErr != nil {
  1237  				if !errors.Is(err, tc.wantErr) {
  1238  					t.Fatalf("got: %s; want: %s", err, tc.wantErr)
  1239  				}
  1240  				return
  1241  			}
  1242  			if err != nil {
  1243  				t.Fatalf("got %s; wanted no error", err)
  1244  			}
  1245  		})
  1246  	}
  1247  }
  1248  
  1249  func TestManualCompaction(t *testing.T) {
  1250  	var mem vfs.FS
  1251  	var d *DB
  1252  	defer func() {
  1253  		if d != nil {
  1254  			require.NoError(t, closeAllSnapshots(d))
  1255  			require.NoError(t, d.Close())
  1256  		}
  1257  	}()
  1258  
  1259  	seed := time.Now().UnixNano()
  1260  	rng := rand.New(rand.NewSource(seed))
  1261  	t.Logf("seed: %d", seed)
  1262  
  1263  	randVersion := func(min, max FormatMajorVersion) FormatMajorVersion {
  1264  		return FormatMajorVersion(int(min) + rng.Intn(int(max)-int(min)+1))
  1265  	}
  1266  
  1267  	var compactionLog bytes.Buffer
  1268  	compactionLogEventListener := &EventListener{
  1269  		CompactionEnd: func(info CompactionInfo) {
  1270  			// Ensure determinism.
  1271  			info.JobID = 1
  1272  			info.Duration = time.Second
  1273  			info.TotalDuration = time.Second
  1274  			fmt.Fprintln(&compactionLog, info.String())
  1275  		},
  1276  	}
  1277  	reset := func(minVersion, maxVersion FormatMajorVersion) {
  1278  		compactionLog.Reset()
  1279  		if d != nil {
  1280  			require.NoError(t, closeAllSnapshots(d))
  1281  			require.NoError(t, d.Close())
  1282  		}
  1283  		mem = vfs.NewMem()
  1284  		require.NoError(t, mem.MkdirAll("ext", 0755))
  1285  
  1286  		opts := (&Options{
  1287  			FS:                          mem,
  1288  			DebugCheck:                  DebugCheckLevels,
  1289  			DisableAutomaticCompactions: true,
  1290  			EventListener:               compactionLogEventListener,
  1291  			FormatMajorVersion:          randVersion(minVersion, maxVersion),
  1292  		}).WithFSDefaults()
  1293  
  1294  		var err error
  1295  		d, err = Open("", opts)
  1296  		require.NoError(t, err)
  1297  	}
  1298  
  1299  	// d.mu must be held when calling.
  1300  	createOngoingCompaction := func(start, end []byte, startLevel, outputLevel int) (ongoingCompaction *compaction) {
  1301  		ongoingCompaction = &compaction{
  1302  			inputs:   []compactionLevel{{level: startLevel}, {level: outputLevel}},
  1303  			smallest: InternalKey{UserKey: start},
  1304  			largest:  InternalKey{UserKey: end},
  1305  		}
  1306  		ongoingCompaction.startLevel = &ongoingCompaction.inputs[0]
  1307  		ongoingCompaction.outputLevel = &ongoingCompaction.inputs[1]
  1308  		// Mark files as compacting.
  1309  		curr := d.mu.versions.currentVersion()
  1310  		ongoingCompaction.startLevel.files = curr.Overlaps(startLevel, d.cmp, start, end, false)
  1311  		ongoingCompaction.outputLevel.files = curr.Overlaps(outputLevel, d.cmp, start, end, false)
  1312  		for _, cl := range ongoingCompaction.inputs {
  1313  			iter := cl.files.Iter()
  1314  			for f := iter.First(); f != nil; f = iter.Next() {
  1315  				f.CompactionState = manifest.CompactionStateCompacting
  1316  			}
  1317  		}
  1318  		d.mu.compact.inProgress[ongoingCompaction] = struct{}{}
  1319  		d.mu.compact.compactingCount++
  1320  		return
  1321  	}
  1322  
  1323  	// d.mu must be held when calling.
  1324  	deleteOngoingCompaction := func(ongoingCompaction *compaction) {
  1325  		for _, cl := range ongoingCompaction.inputs {
  1326  			iter := cl.files.Iter()
  1327  			for f := iter.First(); f != nil; f = iter.Next() {
  1328  				f.CompactionState = manifest.CompactionStateNotCompacting
  1329  			}
  1330  		}
  1331  		delete(d.mu.compact.inProgress, ongoingCompaction)
  1332  		d.mu.compact.compactingCount--
  1333  	}
  1334  
  1335  	runTest := func(t *testing.T, testData string, minVersion, maxVersion FormatMajorVersion, verbose bool) {
  1336  		reset(minVersion, maxVersion)
  1337  		var ongoingCompaction *compaction
  1338  		datadriven.RunTest(t, testData, func(t *testing.T, td *datadriven.TestData) string {
  1339  			switch td.Cmd {
  1340  			case "reset":
  1341  				reset(minVersion, maxVersion)
  1342  				return ""
  1343  
  1344  			case "batch":
  1345  				b := d.NewIndexedBatch()
  1346  				if err := runBatchDefineCmd(td, b); err != nil {
  1347  					return err.Error()
  1348  				}
  1349  				require.NoError(t, b.Commit(nil))
  1350  				return ""
  1351  
  1352  			case "build":
  1353  				if err := runBuildCmd(td, d, mem); err != nil {
  1354  					return err.Error()
  1355  				}
  1356  				return ""
  1357  
  1358  			case "compact":
  1359  				if err := runCompactCmd(td, d); err != nil {
  1360  					return err.Error()
  1361  				}
  1362  				d.mu.Lock()
  1363  				s := d.mu.versions.currentVersion().String()
  1364  				if verbose {
  1365  					s = d.mu.versions.currentVersion().DebugString(base.DefaultFormatter)
  1366  				}
  1367  				d.mu.Unlock()
  1368  				if td.HasArg("hide-file-num") {
  1369  					re := regexp.MustCompile(`([0-9]*):\[`)
  1370  					s = re.ReplaceAllString(s, "[")
  1371  				}
  1372  				return s
  1373  
  1374  			case "define":
  1375  				if d != nil {
  1376  					if err := closeAllSnapshots(d); err != nil {
  1377  						return err.Error()
  1378  					}
  1379  					if err := d.Close(); err != nil {
  1380  						return err.Error()
  1381  					}
  1382  				}
  1383  
  1384  				mem = vfs.NewMem()
  1385  				opts := (&Options{
  1386  					FS:                          mem,
  1387  					DebugCheck:                  DebugCheckLevels,
  1388  					EventListener:               compactionLogEventListener,
  1389  					FormatMajorVersion:          randVersion(minVersion, maxVersion),
  1390  					DisableAutomaticCompactions: true,
  1391  				}).WithFSDefaults()
  1392  
  1393  				var err error
  1394  				if d, err = runDBDefineCmd(td, opts); err != nil {
  1395  					return err.Error()
  1396  				}
  1397  
  1398  				s := d.mu.versions.currentVersion().String()
  1399  				if verbose {
  1400  					s = d.mu.versions.currentVersion().DebugString(base.DefaultFormatter)
  1401  				}
  1402  				return s
  1403  
  1404  			case "file-sizes":
  1405  				return runTableFileSizesCmd(td, d)
  1406  
  1407  			case "flush":
  1408  				if err := d.Flush(); err != nil {
  1409  					return err.Error()
  1410  				}
  1411  				d.mu.Lock()
  1412  				s := d.mu.versions.currentVersion().String()
  1413  				if verbose {
  1414  					s = d.mu.versions.currentVersion().DebugString(base.DefaultFormatter)
  1415  				}
  1416  				d.mu.Unlock()
  1417  				return s
  1418  
  1419  			case "ingest":
  1420  				if err := runIngestCmd(td, d, mem); err != nil {
  1421  					return err.Error()
  1422  				}
  1423  				d.mu.Lock()
  1424  				s := d.mu.versions.currentVersion().String()
  1425  				if verbose {
  1426  					s = d.mu.versions.currentVersion().DebugString(base.DefaultFormatter)
  1427  				}
  1428  				d.mu.Unlock()
  1429  				return s
  1430  
  1431  			case "iter":
  1432  				// TODO(peter): runDBDefineCmd doesn't properly update the visible
  1433  				// sequence number. So we have to use a snapshot with a very large
  1434  				// sequence number, otherwise the DB appears empty.
  1435  				snap := Snapshot{
  1436  					db:     d,
  1437  					seqNum: InternalKeySeqNumMax,
  1438  				}
  1439  				iter, _ := snap.NewIter(nil)
  1440  				return runIterCmd(td, iter, true)
  1441  
  1442  			case "lsm":
  1443  				return runLSMCmd(td, d)
  1444  
  1445  			case "populate":
  1446  				b := d.NewBatch()
  1447  				runPopulateCmd(t, td, b)
  1448  				count := b.Count()
  1449  				require.NoError(t, b.Commit(nil))
  1450  				return fmt.Sprintf("wrote %d keys\n", count)
  1451  
  1452  			case "async-compact":
  1453  				var s string
  1454  				ch := make(chan error, 1)
  1455  				go func() {
  1456  					if err := runCompactCmd(td, d); err != nil {
  1457  						ch <- err
  1458  						close(ch)
  1459  						return
  1460  					}
  1461  					d.mu.Lock()
  1462  					s = d.mu.versions.currentVersion().String()
  1463  					d.mu.Unlock()
  1464  					close(ch)
  1465  				}()
  1466  
  1467  				manualDone := func() bool {
  1468  					select {
  1469  					case <-ch:
  1470  						return true
  1471  					default:
  1472  						return false
  1473  					}
  1474  				}
  1475  
  1476  				err := try(100*time.Microsecond, 20*time.Second, func() error {
  1477  					if manualDone() {
  1478  						return nil
  1479  					}
  1480  
  1481  					d.mu.Lock()
  1482  					defer d.mu.Unlock()
  1483  					if len(d.mu.compact.manual) == 0 {
  1484  						return errors.New("no manual compaction queued")
  1485  					}
  1486  					manual := d.mu.compact.manual[0]
  1487  					if manual.retries == 0 {
  1488  						return errors.New("manual compaction has not been retried")
  1489  					}
  1490  					return nil
  1491  				})
  1492  				if err != nil {
  1493  					return err.Error()
  1494  				}
  1495  
  1496  				if manualDone() {
  1497  					return "manual compaction did not block for ongoing\n" + s
  1498  				}
  1499  
  1500  				d.mu.Lock()
  1501  				deleteOngoingCompaction(ongoingCompaction)
  1502  				ongoingCompaction = nil
  1503  				d.maybeScheduleCompaction()
  1504  				d.mu.Unlock()
  1505  				if err := <-ch; err != nil {
  1506  					return err.Error()
  1507  				}
  1508  				return "manual compaction blocked until ongoing finished\n" + s
  1509  
  1510  			case "add-ongoing-compaction":
  1511  				var startLevel int
  1512  				var outputLevel int
  1513  				var start string
  1514  				var end string
  1515  				td.ScanArgs(t, "startLevel", &startLevel)
  1516  				td.ScanArgs(t, "outputLevel", &outputLevel)
  1517  				td.ScanArgs(t, "start", &start)
  1518  				td.ScanArgs(t, "end", &end)
  1519  				d.mu.Lock()
  1520  				ongoingCompaction = createOngoingCompaction([]byte(start), []byte(end), startLevel, outputLevel)
  1521  				d.mu.Unlock()
  1522  				return ""
  1523  
  1524  			case "remove-ongoing-compaction":
  1525  				d.mu.Lock()
  1526  				deleteOngoingCompaction(ongoingCompaction)
  1527  				ongoingCompaction = nil
  1528  				d.mu.Unlock()
  1529  				return ""
  1530  
  1531  			case "set-concurrent-compactions":
  1532  				var concurrentCompactions int
  1533  				td.ScanArgs(t, "num", &concurrentCompactions)
  1534  				d.opts.MaxConcurrentCompactions = func() int {
  1535  					return concurrentCompactions
  1536  				}
  1537  				return ""
  1538  
  1539  			case "sstable-properties":
  1540  				return runSSTablePropertiesCmd(t, td, d)
  1541  
  1542  			case "wait-pending-table-stats":
  1543  				return runTableStatsCmd(td, d)
  1544  
  1545  			case "close-snapshots":
  1546  				d.mu.Lock()
  1547  				// Re-enable automatic compactions if they were disabled so that
  1548  				// closing snapshots can trigger elision-only compactions if
  1549  				// necessary.
  1550  				d.opts.DisableAutomaticCompactions = false
  1551  
  1552  				var ss []*Snapshot
  1553  				l := &d.mu.snapshots
  1554  				for i := l.root.next; i != &l.root; i = i.next {
  1555  					ss = append(ss, i)
  1556  				}
  1557  				d.mu.Unlock()
  1558  				for i := range ss {
  1559  					if err := ss[i].Close(); err != nil {
  1560  						return err.Error()
  1561  					}
  1562  				}
  1563  				return ""
  1564  
  1565  			case "compaction-log":
  1566  				defer compactionLog.Reset()
  1567  				return compactionLog.String()
  1568  
  1569  			default:
  1570  				return fmt.Sprintf("unknown command: %s", td.Cmd)
  1571  			}
  1572  		})
  1573  	}
  1574  
  1575  	testCases := []struct {
  1576  		testData   string
  1577  		minVersion FormatMajorVersion
  1578  		maxVersion FormatMajorVersion // inclusive
  1579  		verbose    bool
  1580  	}{
  1581  		{
  1582  			testData:   "testdata/manual_compaction",
  1583  			minVersion: FormatMostCompatible,
  1584  			maxVersion: FormatSetWithDelete - 1,
  1585  		},
  1586  		{
  1587  			testData:   "testdata/manual_compaction_set_with_del",
  1588  			minVersion: FormatBlockPropertyCollector,
  1589  			// This test exercises split user keys.
  1590  			maxVersion: FormatSplitUserKeysMarkedCompacted - 1,
  1591  		},
  1592  		{
  1593  			testData:   "testdata/singledel_manual_compaction",
  1594  			minVersion: FormatMostCompatible,
  1595  			maxVersion: FormatSetWithDelete - 1,
  1596  		},
  1597  		{
  1598  			testData:   "testdata/singledel_manual_compaction_set_with_del",
  1599  			minVersion: FormatSetWithDelete,
  1600  			maxVersion: internalFormatNewest,
  1601  		},
  1602  		{
  1603  			testData:   "testdata/manual_compaction_range_keys",
  1604  			minVersion: FormatRangeKeys,
  1605  			maxVersion: internalFormatNewest,
  1606  			verbose:    true,
  1607  		},
  1608  		{
  1609  			testData:   "testdata/manual_compaction_file_boundaries",
  1610  			minVersion: FormatBlockPropertyCollector,
  1611  			// This test exercises split user keys.
  1612  			maxVersion: FormatSplitUserKeysMarkedCompacted - 1,
  1613  		},
  1614  		{
  1615  			testData:   "testdata/manual_compaction_file_boundaries_delsized",
  1616  			minVersion: FormatDeleteSizedAndObsolete,
  1617  			maxVersion: internalFormatNewest,
  1618  		},
  1619  		{
  1620  			testData:   "testdata/manual_compaction_set_with_del_sstable_Pebblev4",
  1621  			minVersion: FormatDeleteSizedAndObsolete,
  1622  			maxVersion: internalFormatNewest,
  1623  		},
  1624  		{
  1625  			testData:   "testdata/manual_compaction_multilevel",
  1626  			minVersion: FormatMostCompatible,
  1627  			maxVersion: internalFormatNewest,
  1628  		},
  1629  	}
  1630  
  1631  	for _, tc := range testCases {
  1632  		t.Run(tc.testData, func(t *testing.T) {
  1633  			runTest(t, tc.testData, tc.minVersion, tc.maxVersion, tc.verbose)
  1634  		})
  1635  	}
  1636  }
  1637  
  1638  func TestCompactionFindGrandparentLimit(t *testing.T) {
  1639  	cmp := DefaultComparer.Compare
  1640  	var grandparents []*fileMetadata
  1641  
  1642  	var fileNum base.FileNum
  1643  	parseMeta := func(s string) *fileMetadata {
  1644  		parts := strings.Split(s, "-")
  1645  		if len(parts) != 2 {
  1646  			t.Fatalf("malformed table spec: %s", s)
  1647  		}
  1648  		fileNum++
  1649  		m := (&fileMetadata{
  1650  			FileNum: fileNum,
  1651  		}).ExtendPointKeyBounds(
  1652  			cmp,
  1653  			InternalKey{UserKey: []byte(parts[0])},
  1654  			InternalKey{UserKey: []byte(parts[1])},
  1655  		)
  1656  		m.InitPhysicalBacking()
  1657  		return m
  1658  	}
  1659  
  1660  	datadriven.RunTest(t, "testdata/compaction_find_grandparent_limit",
  1661  		func(t *testing.T, d *datadriven.TestData) string {
  1662  			switch d.Cmd {
  1663  			case "define":
  1664  				grandparents = nil
  1665  				if len(d.Input) == 0 {
  1666  					return ""
  1667  				}
  1668  				for _, data := range strings.Split(d.Input, "\n") {
  1669  					parts := strings.Fields(data)
  1670  					if len(parts) != 2 {
  1671  						return fmt.Sprintf("malformed test:\n%s", d.Input)
  1672  					}
  1673  
  1674  					meta := parseMeta(parts[0])
  1675  					var err error
  1676  					meta.Size, err = strconv.ParseUint(parts[1], 10, 64)
  1677  					if err != nil {
  1678  						return err.Error()
  1679  					}
  1680  					grandparents = append(grandparents, meta)
  1681  				}
  1682  				return ""
  1683  
  1684  			case "compact":
  1685  				c := &compaction{
  1686  					cmp:          cmp,
  1687  					equal:        DefaultComparer.Equal,
  1688  					comparer:     DefaultComparer,
  1689  					grandparents: manifest.NewLevelSliceKeySorted(cmp, grandparents),
  1690  				}
  1691  				if len(d.CmdArgs) != 1 {
  1692  					return fmt.Sprintf("%s expects 1 argument", d.Cmd)
  1693  				}
  1694  				if len(d.CmdArgs[0].Vals) != 1 {
  1695  					return fmt.Sprintf("%s expects 1 value", d.CmdArgs[0].Key)
  1696  				}
  1697  				var err error
  1698  				c.maxOverlapBytes, err = strconv.ParseUint(d.CmdArgs[0].Vals[0], 10, 64)
  1699  				if err != nil {
  1700  					return err.Error()
  1701  				}
  1702  
  1703  				var buf bytes.Buffer
  1704  				var smallest, largest string
  1705  				var grandparentLimit []byte
  1706  				for i, key := range strings.Fields(d.Input) {
  1707  					if i == 0 {
  1708  						smallest = key
  1709  						grandparentLimit = c.findGrandparentLimit([]byte(key))
  1710  					}
  1711  					if grandparentLimit != nil && c.cmp(grandparentLimit, []byte(key)) < 0 {
  1712  						fmt.Fprintf(&buf, "%s-%s\n", smallest, largest)
  1713  						smallest = key
  1714  						grandparentLimit = c.findGrandparentLimit([]byte(key))
  1715  					}
  1716  					largest = key
  1717  				}
  1718  				fmt.Fprintf(&buf, "%s-%s\n", smallest, largest)
  1719  				return buf.String()
  1720  
  1721  			default:
  1722  				return fmt.Sprintf("unknown command: %s", d.Cmd)
  1723  			}
  1724  		})
  1725  }
  1726  
  1727  func TestCompactionFindL0Limit(t *testing.T) {
  1728  	cmp := DefaultComparer.Compare
  1729  
  1730  	fileNumCounter := 1
  1731  	parseMeta := func(s string) (*fileMetadata, error) {
  1732  		fields := strings.Fields(s)
  1733  		parts := strings.Split(fields[0], "-")
  1734  		if len(parts) != 2 {
  1735  			return nil, errors.Errorf("malformed table spec: %s", s)
  1736  		}
  1737  		m := (&fileMetadata{
  1738  			FileNum: base.FileNum(fileNumCounter),
  1739  		}).ExtendPointKeyBounds(
  1740  			cmp,
  1741  			base.ParseInternalKey(strings.TrimSpace(parts[0])),
  1742  			base.ParseInternalKey(strings.TrimSpace(parts[1])),
  1743  		)
  1744  		fileNumCounter++
  1745  		m.SmallestSeqNum = m.Smallest.SeqNum()
  1746  		m.LargestSeqNum = m.Largest.SeqNum()
  1747  
  1748  		for _, field := range fields[1:] {
  1749  			parts := strings.Split(field, "=")
  1750  			switch parts[0] {
  1751  			case "size":
  1752  				size, err := strconv.ParseUint(parts[1], 10, 64)
  1753  				if err != nil {
  1754  					t.Fatal(err)
  1755  				}
  1756  				m.Size = size
  1757  			}
  1758  		}
  1759  		m.InitPhysicalBacking()
  1760  		return m, nil
  1761  	}
  1762  
  1763  	var vers *version
  1764  	flushSplitBytes := int64(0)
  1765  
  1766  	datadriven.RunTest(t, "testdata/compaction_find_l0_limit",
  1767  		func(t *testing.T, d *datadriven.TestData) string {
  1768  			switch d.Cmd {
  1769  			case "define":
  1770  				fileMetas := [manifest.NumLevels][]*fileMetadata{}
  1771  				baseLevel := manifest.NumLevels - 1
  1772  				level := 0
  1773  				d.MaybeScanArgs(t, "flush_split_bytes", &flushSplitBytes)
  1774  
  1775  				var err error
  1776  				for _, data := range strings.Split(d.Input, "\n") {
  1777  					data = strings.TrimSpace(data)
  1778  					switch data {
  1779  					case "L0", "L1", "L2", "L3", "L4", "L5", "L6":
  1780  						level, err = strconv.Atoi(data[1:])
  1781  						if err != nil {
  1782  							return err.Error()
  1783  						}
  1784  					default:
  1785  						meta, err := parseMeta(data)
  1786  						if err != nil {
  1787  							return err.Error()
  1788  						}
  1789  						if level != 0 && level < baseLevel {
  1790  							baseLevel = level
  1791  						}
  1792  						fileMetas[level] = append(fileMetas[level], meta)
  1793  					}
  1794  				}
  1795  
  1796  				vers = manifest.NewVersion(DefaultComparer.Compare, base.DefaultFormatter, flushSplitBytes, fileMetas)
  1797  				flushSplitKeys := vers.L0Sublevels.FlushSplitKeys()
  1798  
  1799  				var buf strings.Builder
  1800  				buf.WriteString(vers.String())
  1801  				buf.WriteString("flush split keys:\n")
  1802  				for _, key := range flushSplitKeys {
  1803  					fmt.Fprintf(&buf, "\t%s\n", base.DefaultFormatter(key))
  1804  				}
  1805  
  1806  				return buf.String()
  1807  
  1808  			case "flush":
  1809  				c := &compaction{
  1810  					cmp:      cmp,
  1811  					equal:    DefaultComparer.Equal,
  1812  					comparer: DefaultComparer,
  1813  					version:  vers,
  1814  					l0Limits: vers.L0Sublevels.FlushSplitKeys(),
  1815  					inputs:   []compactionLevel{{level: -1}, {level: 0}},
  1816  				}
  1817  				c.startLevel, c.outputLevel = &c.inputs[0], &c.inputs[1]
  1818  
  1819  				var buf bytes.Buffer
  1820  				var smallest, largest string
  1821  				var l0Limit []byte
  1822  				for i, key := range strings.Fields(d.Input) {
  1823  					if i == 0 {
  1824  						smallest = key
  1825  						l0Limit = c.findL0Limit([]byte(key))
  1826  					}
  1827  					if l0Limit != nil && c.cmp(l0Limit, []byte(key)) < 0 {
  1828  						fmt.Fprintf(&buf, "%s-%s\n", smallest, largest)
  1829  						smallest = key
  1830  						l0Limit = c.findL0Limit([]byte(key))
  1831  					}
  1832  					largest = key
  1833  				}
  1834  				fmt.Fprintf(&buf, "%s-%s\n", smallest, largest)
  1835  				return buf.String()
  1836  
  1837  			default:
  1838  				return fmt.Sprintf("unknown command: %s", d.Cmd)
  1839  			}
  1840  		})
  1841  }
  1842  
  1843  func TestCompactionOutputLevel(t *testing.T) {
  1844  	opts := (*Options)(nil).EnsureDefaults()
  1845  	version := &version{}
  1846  
  1847  	datadriven.RunTest(t, "testdata/compaction_output_level",
  1848  		func(t *testing.T, d *datadriven.TestData) (res string) {
  1849  			defer func() {
  1850  				if r := recover(); r != nil {
  1851  					res = fmt.Sprintln(r)
  1852  				}
  1853  			}()
  1854  
  1855  			switch d.Cmd {
  1856  			case "compact":
  1857  				var start, base int
  1858  				d.ScanArgs(t, "start", &start)
  1859  				d.ScanArgs(t, "base", &base)
  1860  				pc := newPickedCompaction(opts, version, start, defaultOutputLevel(start, base), base)
  1861  				c := newCompaction(pc, opts, time.Now(), nil /* provider */)
  1862  				return fmt.Sprintf("output=%d\nmax-output-file-size=%d\n",
  1863  					c.outputLevel.level, c.maxOutputFileSize)
  1864  
  1865  			default:
  1866  				return fmt.Sprintf("unknown command: %s", d.Cmd)
  1867  			}
  1868  		})
  1869  }
  1870  
  1871  func TestCompactionAtomicUnitBounds(t *testing.T) {
  1872  	cmp := DefaultComparer.Compare
  1873  	var files manifest.LevelSlice
  1874  
  1875  	parseMeta := func(s string) *fileMetadata {
  1876  		parts := strings.Split(s, "-")
  1877  		if len(parts) != 2 {
  1878  			t.Fatalf("malformed table spec: %s", s)
  1879  		}
  1880  		m := (&fileMetadata{}).ExtendPointKeyBounds(
  1881  			cmp,
  1882  			base.ParseInternalKey(parts[0]),
  1883  			base.ParseInternalKey(parts[1]),
  1884  		)
  1885  		m.InitPhysicalBacking()
  1886  		return m
  1887  	}
  1888  
  1889  	datadriven.RunTest(t, "testdata/compaction_atomic_unit_bounds",
  1890  		func(t *testing.T, d *datadriven.TestData) string {
  1891  			switch d.Cmd {
  1892  			case "define":
  1893  				files = manifest.LevelSlice{}
  1894  				if len(d.Input) == 0 {
  1895  					return ""
  1896  				}
  1897  				var ff []*fileMetadata
  1898  				for _, data := range strings.Split(d.Input, "\n") {
  1899  					meta := parseMeta(data)
  1900  					meta.FileNum = FileNum(len(ff))
  1901  					ff = append(ff, meta)
  1902  				}
  1903  				files = manifest.NewLevelSliceKeySorted(cmp, ff)
  1904  				return ""
  1905  
  1906  			case "atomic-unit-bounds":
  1907  				c := &compaction{
  1908  					cmp:      cmp,
  1909  					equal:    DefaultComparer.Equal,
  1910  					comparer: DefaultComparer,
  1911  					inputs:   []compactionLevel{{files: files}, {}},
  1912  				}
  1913  				c.startLevel, c.outputLevel = &c.inputs[0], &c.inputs[1]
  1914  				if len(d.CmdArgs) != 1 {
  1915  					return fmt.Sprintf("%s expects 1 argument", d.Cmd)
  1916  				}
  1917  				index, err := strconv.ParseInt(d.CmdArgs[0].String(), 10, 64)
  1918  				if err != nil {
  1919  					return err.Error()
  1920  				}
  1921  				iter := files.Iter()
  1922  				// Advance iter to `index`.
  1923  				_ = iter.First()
  1924  				for i := int64(0); i < index; i++ {
  1925  					_ = iter.Next()
  1926  				}
  1927  				atomicUnit, _ := expandToAtomicUnit(c.cmp, iter.Take().Slice(), true /* disableIsCompacting */)
  1928  				lower, upper := manifest.KeyRange(c.cmp, atomicUnit.Iter())
  1929  				return fmt.Sprintf("%s-%s\n", lower.UserKey, upper.UserKey)
  1930  
  1931  			default:
  1932  				return fmt.Sprintf("unknown command: %s", d.Cmd)
  1933  			}
  1934  		})
  1935  }
  1936  
  1937  func TestCompactionDeleteOnlyHints(t *testing.T) {
  1938  	parseUint64 := func(s string) uint64 {
  1939  		v, err := strconv.ParseUint(s, 10, 64)
  1940  		require.NoError(t, err)
  1941  		return v
  1942  	}
  1943  	var d *DB
  1944  	defer func() {
  1945  		if d != nil {
  1946  			require.NoError(t, closeAllSnapshots(d))
  1947  			require.NoError(t, d.Close())
  1948  		}
  1949  	}()
  1950  
  1951  	var compactInfo *CompactionInfo // protected by d.mu
  1952  	reset := func() (*Options, error) {
  1953  		if d != nil {
  1954  			compactInfo = nil
  1955  			if err := closeAllSnapshots(d); err != nil {
  1956  				return nil, err
  1957  			}
  1958  			if err := d.Close(); err != nil {
  1959  				return nil, err
  1960  			}
  1961  		}
  1962  		opts := (&Options{
  1963  			FS:         vfs.NewMem(),
  1964  			DebugCheck: DebugCheckLevels,
  1965  			EventListener: &EventListener{
  1966  				CompactionEnd: func(info CompactionInfo) {
  1967  					if compactInfo != nil {
  1968  						return
  1969  					}
  1970  					compactInfo = &info
  1971  				},
  1972  			},
  1973  			FormatMajorVersion: internalFormatNewest,
  1974  		}).WithFSDefaults()
  1975  
  1976  		// Collection of table stats can trigger compactions. As we want full
  1977  		// control over when compactions are run, disable stats by default.
  1978  		opts.private.disableTableStats = true
  1979  
  1980  		return opts, nil
  1981  	}
  1982  
  1983  	compactionString := func() string {
  1984  		for d.mu.compact.compactingCount > 0 {
  1985  			d.mu.compact.cond.Wait()
  1986  		}
  1987  
  1988  		s := "(none)"
  1989  		if compactInfo != nil {
  1990  			// Fix the job ID and durations for determinism.
  1991  			compactInfo.JobID = 100
  1992  			compactInfo.Duration = time.Second
  1993  			compactInfo.TotalDuration = 2 * time.Second
  1994  			s = compactInfo.String()
  1995  			compactInfo = nil
  1996  		}
  1997  		return s
  1998  	}
  1999  
  2000  	var err error
  2001  	var opts *Options
  2002  	datadriven.RunTest(t, "testdata/compaction_delete_only_hints",
  2003  		func(t *testing.T, td *datadriven.TestData) string {
  2004  			switch td.Cmd {
  2005  			case "define":
  2006  				opts, err = reset()
  2007  				if err != nil {
  2008  					return err.Error()
  2009  				}
  2010  				d, err = runDBDefineCmd(td, opts)
  2011  				if err != nil {
  2012  					return err.Error()
  2013  				}
  2014  				d.mu.Lock()
  2015  				s := d.mu.versions.currentVersion().String()
  2016  				d.mu.Unlock()
  2017  				return s
  2018  
  2019  			case "force-set-hints":
  2020  				d.mu.Lock()
  2021  				defer d.mu.Unlock()
  2022  				d.mu.compact.deletionHints = d.mu.compact.deletionHints[:0]
  2023  				var buf bytes.Buffer
  2024  				for _, data := range strings.Split(td.Input, "\n") {
  2025  					parts := strings.FieldsFunc(strings.TrimSpace(data),
  2026  						func(r rune) bool { return r == '-' || r == ' ' || r == '.' })
  2027  
  2028  					start, end := []byte(parts[2]), []byte(parts[3])
  2029  
  2030  					var tombstoneFile *fileMetadata
  2031  					tombstoneLevel := int(parseUint64(parts[0][1:]))
  2032  
  2033  					// Set file number to the value provided in the input.
  2034  					tombstoneFile = &fileMetadata{
  2035  						FileNum: base.FileNum(parseUint64(parts[1])),
  2036  					}
  2037  
  2038  					var hintType deleteCompactionHintType
  2039  					switch typ := parts[7]; typ {
  2040  					case "point_key_only":
  2041  						hintType = deleteCompactionHintTypePointKeyOnly
  2042  					case "range_key_only":
  2043  						hintType = deleteCompactionHintTypeRangeKeyOnly
  2044  					case "point_and_range_key":
  2045  						hintType = deleteCompactionHintTypePointAndRangeKey
  2046  					default:
  2047  						return fmt.Sprintf("unknown hint type: %s", typ)
  2048  					}
  2049  
  2050  					h := deleteCompactionHint{
  2051  						hintType:                hintType,
  2052  						start:                   start,
  2053  						end:                     end,
  2054  						fileSmallestSeqNum:      parseUint64(parts[4]),
  2055  						tombstoneLevel:          tombstoneLevel,
  2056  						tombstoneFile:           tombstoneFile,
  2057  						tombstoneSmallestSeqNum: parseUint64(parts[5]),
  2058  						tombstoneLargestSeqNum:  parseUint64(parts[6]),
  2059  					}
  2060  					d.mu.compact.deletionHints = append(d.mu.compact.deletionHints, h)
  2061  					fmt.Fprintln(&buf, h.String())
  2062  				}
  2063  				return buf.String()
  2064  
  2065  			case "get-hints":
  2066  				d.mu.Lock()
  2067  				defer d.mu.Unlock()
  2068  
  2069  				// Force collection of table stats. This requires re-enabling the
  2070  				// collection flag. We also do not want compactions to run as part of
  2071  				// the stats collection job, so we disable it temporarily.
  2072  				d.opts.private.disableTableStats = false
  2073  				d.opts.DisableAutomaticCompactions = true
  2074  				defer func() {
  2075  					d.opts.private.disableTableStats = true
  2076  					d.opts.DisableAutomaticCompactions = false
  2077  				}()
  2078  
  2079  				// NB: collectTableStats attempts to acquire the lock. Temporarily
  2080  				// unlock here to avoid a deadlock.
  2081  				d.mu.Unlock()
  2082  				didRun := d.collectTableStats()
  2083  				d.mu.Lock()
  2084  
  2085  				if !didRun {
  2086  					// If a job was already running, wait for the results.
  2087  					d.waitTableStats()
  2088  				}
  2089  
  2090  				hints := d.mu.compact.deletionHints
  2091  				if len(hints) == 0 {
  2092  					return "(none)"
  2093  				}
  2094  				var buf bytes.Buffer
  2095  				for _, h := range hints {
  2096  					buf.WriteString(h.String() + "\n")
  2097  				}
  2098  				return buf.String()
  2099  
  2100  			case "maybe-compact":
  2101  				d.mu.Lock()
  2102  				d.maybeScheduleCompaction()
  2103  
  2104  				var buf bytes.Buffer
  2105  				fmt.Fprintf(&buf, "Deletion hints:\n")
  2106  				for _, h := range d.mu.compact.deletionHints {
  2107  					fmt.Fprintf(&buf, "  %s\n", h.String())
  2108  				}
  2109  				if len(d.mu.compact.deletionHints) == 0 {
  2110  					fmt.Fprintf(&buf, "  (none)\n")
  2111  				}
  2112  				fmt.Fprintf(&buf, "Compactions:\n")
  2113  				fmt.Fprintf(&buf, "  %s", compactionString())
  2114  				d.mu.Unlock()
  2115  				return buf.String()
  2116  
  2117  			case "compact":
  2118  				if err := runCompactCmd(td, d); err != nil {
  2119  					return err.Error()
  2120  				}
  2121  				d.mu.Lock()
  2122  				compactInfo = nil
  2123  				s := d.mu.versions.currentVersion().String()
  2124  				d.mu.Unlock()
  2125  				return s
  2126  
  2127  			case "close-snapshot":
  2128  				seqNum, err := strconv.ParseUint(strings.TrimSpace(td.Input), 0, 64)
  2129  				if err != nil {
  2130  					return err.Error()
  2131  				}
  2132  				d.mu.Lock()
  2133  				var s *Snapshot
  2134  				l := &d.mu.snapshots
  2135  				for i := l.root.next; i != &l.root; i = i.next {
  2136  					if i.seqNum == seqNum {
  2137  						s = i
  2138  					}
  2139  				}
  2140  				d.mu.Unlock()
  2141  				if s == nil {
  2142  					return "(not found)"
  2143  				} else if err := s.Close(); err != nil {
  2144  					return err.Error()
  2145  				}
  2146  
  2147  				d.mu.Lock()
  2148  				// Closing the snapshot may have triggered a compaction.
  2149  				str := compactionString()
  2150  				d.mu.Unlock()
  2151  				return str
  2152  
  2153  			case "iter":
  2154  				snap := Snapshot{
  2155  					db:     d,
  2156  					seqNum: InternalKeySeqNumMax,
  2157  				}
  2158  				iter, _ := snap.NewIter(nil)
  2159  				return runIterCmd(td, iter, true)
  2160  
  2161  			case "reset":
  2162  				opts, err = reset()
  2163  				if err != nil {
  2164  					return err.Error()
  2165  				}
  2166  				d, err = Open("", opts)
  2167  				if err != nil {
  2168  					return err.Error()
  2169  				}
  2170  				return ""
  2171  
  2172  			case "ingest":
  2173  				if err = runBuildCmd(td, d, d.opts.FS); err != nil {
  2174  					return err.Error()
  2175  				}
  2176  				if err = runIngestCmd(td, d, d.opts.FS); err != nil {
  2177  					return err.Error()
  2178  				}
  2179  				return "OK"
  2180  
  2181  			case "describe-lsm":
  2182  				d.mu.Lock()
  2183  				s := d.mu.versions.currentVersion().String()
  2184  				d.mu.Unlock()
  2185  				return s
  2186  
  2187  			default:
  2188  				return fmt.Sprintf("unknown command: %s", td.Cmd)
  2189  			}
  2190  		})
  2191  }
  2192  
  2193  func TestCompactionTombstones(t *testing.T) {
  2194  	var d *DB
  2195  	defer func() {
  2196  		if d != nil {
  2197  			require.NoError(t, closeAllSnapshots(d))
  2198  			require.NoError(t, d.Close())
  2199  		}
  2200  	}()
  2201  
  2202  	var compactInfo *CompactionInfo // protected by d.mu
  2203  
  2204  	compactionString := func() string {
  2205  		for d.mu.compact.compactingCount > 0 {
  2206  			d.mu.compact.cond.Wait()
  2207  		}
  2208  
  2209  		s := "(none)"
  2210  		if compactInfo != nil {
  2211  			// Fix the job ID and durations for determinism.
  2212  			compactInfo.JobID = 100
  2213  			compactInfo.Duration = time.Second
  2214  			compactInfo.TotalDuration = 2 * time.Second
  2215  			s = compactInfo.String()
  2216  			compactInfo = nil
  2217  		}
  2218  		return s
  2219  	}
  2220  
  2221  	datadriven.RunTest(t, "testdata/compaction_tombstones",
  2222  		func(t *testing.T, td *datadriven.TestData) string {
  2223  			switch td.Cmd {
  2224  			case "define":
  2225  				if d != nil {
  2226  					compactInfo = nil
  2227  					require.NoError(t, closeAllSnapshots(d))
  2228  					if err := d.Close(); err != nil {
  2229  						return err.Error()
  2230  					}
  2231  				}
  2232  				opts := (&Options{
  2233  					FS:         vfs.NewMem(),
  2234  					DebugCheck: DebugCheckLevels,
  2235  					EventListener: &EventListener{
  2236  						CompactionEnd: func(info CompactionInfo) {
  2237  							compactInfo = &info
  2238  						},
  2239  					},
  2240  					FormatMajorVersion: internalFormatNewest,
  2241  				}).WithFSDefaults()
  2242  				var err error
  2243  				d, err = runDBDefineCmd(td, opts)
  2244  				if err != nil {
  2245  					return err.Error()
  2246  				}
  2247  				d.mu.Lock()
  2248  				s := d.mu.versions.currentVersion().String()
  2249  				d.mu.Unlock()
  2250  				return s
  2251  
  2252  			case "maybe-compact":
  2253  				d.mu.Lock()
  2254  				d.opts.DisableAutomaticCompactions = false
  2255  				d.maybeScheduleCompaction()
  2256  				s := compactionString()
  2257  				d.mu.Unlock()
  2258  				return s
  2259  
  2260  			case "wait-pending-table-stats":
  2261  				return runTableStatsCmd(td, d)
  2262  
  2263  			case "close-snapshot":
  2264  				seqNum, err := strconv.ParseUint(strings.TrimSpace(td.Input), 0, 64)
  2265  				if err != nil {
  2266  					return err.Error()
  2267  				}
  2268  				d.mu.Lock()
  2269  				var s *Snapshot
  2270  				l := &d.mu.snapshots
  2271  				for i := l.root.next; i != &l.root; i = i.next {
  2272  					if i.seqNum == seqNum {
  2273  						s = i
  2274  					}
  2275  				}
  2276  				d.mu.Unlock()
  2277  				if s == nil {
  2278  					return "(not found)"
  2279  				} else if err := s.Close(); err != nil {
  2280  					return err.Error()
  2281  				}
  2282  
  2283  				d.mu.Lock()
  2284  				// Closing the snapshot may have triggered a compaction.
  2285  				str := compactionString()
  2286  				d.mu.Unlock()
  2287  				return str
  2288  
  2289  			case "close":
  2290  				if err := d.Close(); err != nil {
  2291  					return err.Error()
  2292  				}
  2293  				d = nil
  2294  				return ""
  2295  
  2296  			case "version":
  2297  				d.mu.Lock()
  2298  				s := d.mu.versions.currentVersion().String()
  2299  				d.mu.Unlock()
  2300  				return s
  2301  
  2302  			default:
  2303  				return fmt.Sprintf("unknown command: %s", td.Cmd)
  2304  			}
  2305  		})
  2306  }
  2307  
  2308  func closeAllSnapshots(d *DB) error {
  2309  	d.mu.Lock()
  2310  	var ss []*Snapshot
  2311  	l := &d.mu.snapshots
  2312  	for i := l.root.next; i != &l.root; i = i.next {
  2313  		ss = append(ss, i)
  2314  	}
  2315  	d.mu.Unlock()
  2316  	for i := range ss {
  2317  		if err := ss[i].Close(); err != nil {
  2318  			return err
  2319  		}
  2320  	}
  2321  	return nil
  2322  }
  2323  
  2324  func TestCompactionReadTriggeredQueue(t *testing.T) {
  2325  
  2326  	// Convert a read compaction to a string which this test
  2327  	// understands.
  2328  	showRC := func(rc *readCompaction) string {
  2329  		return fmt.Sprintf(
  2330  			"L%d: %s-%s %d\n", rc.level, string(rc.start), string(rc.end), rc.fileNum,
  2331  		)
  2332  	}
  2333  
  2334  	var queue *readCompactionQueue
  2335  
  2336  	datadriven.RunTest(t, "testdata/read_compaction_queue",
  2337  		func(t *testing.T, td *datadriven.TestData) string {
  2338  			switch td.Cmd {
  2339  			case "create":
  2340  				queue = &readCompactionQueue{}
  2341  				return "(success)"
  2342  			case "add-compaction":
  2343  				for _, line := range strings.Split(td.Input, "\n") {
  2344  					if line == "" {
  2345  						continue
  2346  					}
  2347  					parts := strings.Split(line, " ")
  2348  
  2349  					if len(parts) != 3 {
  2350  						return "error: malformed data for add-compaction. usage: <level>: <start>-<end> <filenum>"
  2351  					}
  2352  					if l, err := strconv.Atoi(parts[0][1:2]); err == nil {
  2353  						keys := strings.Split(parts[1], "-")
  2354  						fileNum, _ := strconv.Atoi(parts[2])
  2355  						rc := readCompaction{
  2356  							level:   l,
  2357  							start:   []byte(keys[0]),
  2358  							end:     []byte(keys[1]),
  2359  							fileNum: base.FileNum(fileNum),
  2360  						}
  2361  						queue.add(&rc, DefaultComparer.Compare)
  2362  					} else {
  2363  						return err.Error()
  2364  					}
  2365  				}
  2366  				return ""
  2367  			case "remove-compaction":
  2368  				rc := queue.remove()
  2369  				if rc == nil {
  2370  					return "(nil)"
  2371  				}
  2372  				return showRC(rc)
  2373  			case "print-size":
  2374  				// Print the size of the queue.
  2375  				return fmt.Sprintf("%d", queue.size)
  2376  			case "print-queue":
  2377  				// Print each element of the queue on a separate line.
  2378  				var sb strings.Builder
  2379  				if queue.size == 0 {
  2380  					sb.WriteString("(empty)")
  2381  				}
  2382  
  2383  				for i := 0; i < queue.size; i++ {
  2384  					rc := queue.at(i)
  2385  					sb.WriteString(showRC(rc))
  2386  				}
  2387  				return sb.String()
  2388  			default:
  2389  				return fmt.Sprintf("unknown command: %s", td.Cmd)
  2390  			}
  2391  		},
  2392  	)
  2393  }
  2394  
  2395  func (qu *readCompactionQueue) at(i int) *readCompaction {
  2396  	if i >= qu.size {
  2397  		return nil
  2398  	}
  2399  
  2400  	return qu.queue[i]
  2401  }
  2402  
  2403  func TestCompactionReadTriggered(t *testing.T) {
  2404  	var d *DB
  2405  	defer func() {
  2406  		if d != nil {
  2407  			require.NoError(t, d.Close())
  2408  		}
  2409  	}()
  2410  
  2411  	var compactInfo *CompactionInfo // protected by d.mu
  2412  
  2413  	compactionString := func() string {
  2414  		for d.mu.compact.compactingCount > 0 {
  2415  			d.mu.compact.cond.Wait()
  2416  		}
  2417  
  2418  		s := "(none)"
  2419  		if compactInfo != nil {
  2420  			// Fix the job ID and durations for determinism.
  2421  			compactInfo.JobID = 100
  2422  			compactInfo.Duration = time.Second
  2423  			compactInfo.TotalDuration = 2 * time.Second
  2424  			s = compactInfo.String()
  2425  			compactInfo = nil
  2426  		}
  2427  		return s
  2428  	}
  2429  
  2430  	datadriven.RunTest(t, "testdata/compaction_read_triggered",
  2431  		func(t *testing.T, td *datadriven.TestData) string {
  2432  			switch td.Cmd {
  2433  			case "define":
  2434  				if d != nil {
  2435  					compactInfo = nil
  2436  					if err := d.Close(); err != nil {
  2437  						return err.Error()
  2438  					}
  2439  				}
  2440  				opts := (&Options{
  2441  					FS:         vfs.NewMem(),
  2442  					DebugCheck: DebugCheckLevels,
  2443  					EventListener: &EventListener{
  2444  						CompactionEnd: func(info CompactionInfo) {
  2445  							compactInfo = &info
  2446  						},
  2447  					},
  2448  				}).WithFSDefaults()
  2449  				var err error
  2450  				d, err = runDBDefineCmd(td, opts)
  2451  				if err != nil {
  2452  					return err.Error()
  2453  				}
  2454  				d.mu.Lock()
  2455  				s := d.mu.versions.currentVersion().String()
  2456  				d.mu.Unlock()
  2457  				return s
  2458  
  2459  			case "add-read-compaction":
  2460  				d.mu.Lock()
  2461  				td.MaybeScanArgs(t, "flushing", &d.mu.compact.flushing)
  2462  				for _, line := range strings.Split(td.Input, "\n") {
  2463  					if line == "" {
  2464  						continue
  2465  					}
  2466  					parts := strings.Split(line, " ")
  2467  					if len(parts) != 3 {
  2468  						return "error: malformed data for add-read-compaction. usage: <level>: <start>-<end> <filenum>"
  2469  					}
  2470  					if l, err := strconv.Atoi(parts[0][:1]); err == nil {
  2471  						keys := strings.Split(parts[1], "-")
  2472  						fileNum, _ := strconv.Atoi(parts[2])
  2473  						rc := readCompaction{
  2474  							level:   l,
  2475  							start:   []byte(keys[0]),
  2476  							end:     []byte(keys[1]),
  2477  							fileNum: base.FileNum(fileNum),
  2478  						}
  2479  						d.mu.compact.readCompactions.add(&rc, DefaultComparer.Compare)
  2480  					} else {
  2481  						return err.Error()
  2482  					}
  2483  				}
  2484  				d.mu.Unlock()
  2485  				return ""
  2486  
  2487  			case "show-read-compactions":
  2488  				d.mu.Lock()
  2489  				var sb strings.Builder
  2490  				if d.mu.compact.readCompactions.size == 0 {
  2491  					sb.WriteString("(none)")
  2492  				}
  2493  				for i := 0; i < d.mu.compact.readCompactions.size; i++ {
  2494  					rc := d.mu.compact.readCompactions.at(i)
  2495  					sb.WriteString(fmt.Sprintf("(level: %d, start: %s, end: %s)\n", rc.level, string(rc.start), string(rc.end)))
  2496  				}
  2497  				d.mu.Unlock()
  2498  				return sb.String()
  2499  
  2500  			case "maybe-compact":
  2501  				d.mu.Lock()
  2502  				d.opts.DisableAutomaticCompactions = false
  2503  				d.maybeScheduleCompaction()
  2504  				s := compactionString()
  2505  				d.mu.Unlock()
  2506  				return s
  2507  
  2508  			case "version":
  2509  				d.mu.Lock()
  2510  				s := d.mu.versions.currentVersion().String()
  2511  				d.mu.Unlock()
  2512  				return s
  2513  
  2514  			default:
  2515  				return fmt.Sprintf("unknown command: %s", td.Cmd)
  2516  			}
  2517  		})
  2518  }
  2519  
  2520  func TestCompactionInuseKeyRanges(t *testing.T) {
  2521  	cmp := DefaultComparer.Compare
  2522  	parseMeta := func(s string) *fileMetadata {
  2523  		parts := strings.Split(s, "-")
  2524  		if len(parts) != 2 {
  2525  			t.Fatalf("malformed table spec: %s", s)
  2526  		}
  2527  		m := (&fileMetadata{}).ExtendRangeKeyBounds(
  2528  			cmp,
  2529  			base.ParseInternalKey(strings.TrimSpace(parts[0])),
  2530  			base.ParseInternalKey(strings.TrimSpace(parts[1])),
  2531  		)
  2532  		m.SmallestSeqNum = m.Smallest.SeqNum()
  2533  		m.LargestSeqNum = m.Largest.SeqNum()
  2534  		m.InitPhysicalBacking()
  2535  		return m
  2536  	}
  2537  
  2538  	opts := (*Options)(nil).EnsureDefaults()
  2539  
  2540  	var c *compaction
  2541  	datadriven.RunTest(t, "testdata/compaction_inuse_key_ranges", func(t *testing.T, td *datadriven.TestData) string {
  2542  		switch td.Cmd {
  2543  		case "define":
  2544  			c = &compaction{
  2545  				cmp:       DefaultComparer.Compare,
  2546  				equal:     DefaultComparer.Equal,
  2547  				comparer:  DefaultComparer,
  2548  				formatKey: DefaultComparer.FormatKey,
  2549  				inputs:    []compactionLevel{{}, {}},
  2550  			}
  2551  			c.startLevel, c.outputLevel = &c.inputs[0], &c.inputs[1]
  2552  			var files [numLevels][]*fileMetadata
  2553  			var currentLevel int
  2554  			fileNum := FileNum(1)
  2555  
  2556  			for _, data := range strings.Split(td.Input, "\n") {
  2557  				switch data {
  2558  				case "L0", "L1", "L2", "L3", "L4", "L5", "L6":
  2559  					level, err := strconv.Atoi(data[1:])
  2560  					if err != nil {
  2561  						return err.Error()
  2562  					}
  2563  					currentLevel = level
  2564  
  2565  				default:
  2566  					meta := parseMeta(data)
  2567  					meta.FileNum = fileNum
  2568  					fileNum++
  2569  					files[currentLevel] = append(files[currentLevel], meta)
  2570  				}
  2571  			}
  2572  			c.version = newVersion(opts, files)
  2573  			return c.version.String()
  2574  
  2575  		case "inuse-key-ranges":
  2576  			var buf bytes.Buffer
  2577  			for _, line := range strings.Split(td.Input, "\n") {
  2578  				parts := strings.Fields(line)
  2579  				if len(parts) != 3 {
  2580  					fmt.Fprintf(&buf, "expected <level> <smallest> <largest>: %q\n", line)
  2581  					continue
  2582  				}
  2583  				level, err := strconv.Atoi(parts[0])
  2584  				if err != nil {
  2585  					fmt.Fprintf(&buf, "expected <level> <smallest> <largest>: %q: %v\n", line, err)
  2586  					continue
  2587  				}
  2588  				c.outputLevel.level = level
  2589  				c.smallest.UserKey = []byte(parts[1])
  2590  				c.largest.UserKey = []byte(parts[2])
  2591  
  2592  				c.inuseKeyRanges = nil
  2593  				c.setupInuseKeyRanges()
  2594  				if len(c.inuseKeyRanges) == 0 {
  2595  					fmt.Fprintf(&buf, ".\n")
  2596  				} else {
  2597  					for i, r := range c.inuseKeyRanges {
  2598  						if i > 0 {
  2599  							fmt.Fprintf(&buf, " ")
  2600  						}
  2601  						fmt.Fprintf(&buf, "%s-%s", r.Start, r.End)
  2602  					}
  2603  					fmt.Fprintf(&buf, "\n")
  2604  				}
  2605  			}
  2606  			return buf.String()
  2607  
  2608  		default:
  2609  			return fmt.Sprintf("unknown command: %s", td.Cmd)
  2610  		}
  2611  	})
  2612  }
  2613  
  2614  func TestCompactionInuseKeyRangesRandomized(t *testing.T) {
  2615  	var (
  2616  		fileNum     = FileNum(0)
  2617  		opts        = (*Options)(nil).EnsureDefaults()
  2618  		seed        = int64(time.Now().UnixNano())
  2619  		rng         = rand.New(rand.NewSource(seed))
  2620  		endKeyspace = 26 * 26
  2621  	)
  2622  	t.Logf("Using rng seed %d.", seed)
  2623  
  2624  	for iter := 0; iter < 100; iter++ {
  2625  		makeUserKey := func(i int) []byte {
  2626  			if i >= endKeyspace {
  2627  				i = endKeyspace - 1
  2628  			}
  2629  			return []byte{byte(i/26 + 'a'), byte(i%26 + 'a')}
  2630  		}
  2631  		makeIK := func(level, i int) InternalKey {
  2632  			return base.MakeInternalKey(
  2633  				makeUserKey(i),
  2634  				uint64(numLevels-level),
  2635  				base.InternalKeyKindSet,
  2636  			)
  2637  		}
  2638  		makeFile := func(level, start, end int) *fileMetadata {
  2639  			fileNum++
  2640  			m := (&fileMetadata{
  2641  				FileNum: fileNum,
  2642  			}).ExtendPointKeyBounds(
  2643  				opts.Comparer.Compare,
  2644  				makeIK(level, start),
  2645  				makeIK(level, end),
  2646  			)
  2647  			m.SmallestSeqNum = m.Smallest.SeqNum()
  2648  			m.LargestSeqNum = m.Largest.SeqNum()
  2649  			m.InitPhysicalBacking()
  2650  			return m
  2651  		}
  2652  		overlaps := func(startA, endA, startB, endB []byte) bool {
  2653  			disjoint := opts.Comparer.Compare(endB, startA) < 0 || opts.Comparer.Compare(endA, startB) < 0
  2654  			return !disjoint
  2655  		}
  2656  		var files [numLevels][]*fileMetadata
  2657  		for l := 0; l < numLevels; l++ {
  2658  			for i := 0; i < rand.Intn(10); i++ {
  2659  				s := rng.Intn(endKeyspace)
  2660  				maxWidth := rng.Intn(endKeyspace-s) + 1
  2661  				e := rng.Intn(maxWidth) + s
  2662  				sKey, eKey := makeUserKey(s), makeUserKey(e)
  2663  				// Discard the key range if it overlaps any existing files
  2664  				// within this level.
  2665  				var o bool
  2666  				for _, f := range files[l] {
  2667  					o = o || overlaps(sKey, eKey, f.Smallest.UserKey, f.Largest.UserKey)
  2668  				}
  2669  				if o {
  2670  					continue
  2671  				}
  2672  				files[l] = append(files[l], makeFile(l, s, e))
  2673  			}
  2674  			sort.Slice(files[l], func(i, j int) bool {
  2675  				return opts.Comparer.Compare(files[l][i].Smallest.UserKey, files[l][j].Smallest.UserKey) < 0
  2676  			})
  2677  		}
  2678  		v := newVersion(opts, files)
  2679  		t.Log(v.DebugString(opts.Comparer.FormatKey))
  2680  		for i := 0; i < 1000; i++ {
  2681  			l := rng.Intn(numLevels)
  2682  			s := rng.Intn(endKeyspace)
  2683  			maxWidth := rng.Intn(endKeyspace-s) + 1
  2684  			e := rng.Intn(maxWidth) + s
  2685  			sKey, eKey := makeUserKey(s), makeUserKey(e)
  2686  			keyRanges := calculateInuseKeyRanges(v, opts.Comparer.Compare, l, numLevels-1, sKey, eKey)
  2687  
  2688  			for level := l; level < numLevels; level++ {
  2689  				for _, f := range files[level] {
  2690  					if !overlaps(sKey, eKey, f.Smallest.UserKey, f.Largest.UserKey) {
  2691  						// This file doesn't overlap the queried range. Skip it.
  2692  						continue
  2693  					}
  2694  					// This file does overlap the queried range. The key range
  2695  					// [MAX(f.Smallest, sKey), MIN(f.Largest, eKey)] must be fully
  2696  					// contained by a key range in keyRanges.
  2697  					checkStart, checkEnd := f.Smallest.UserKey, f.Largest.UserKey
  2698  					if opts.Comparer.Compare(checkStart, sKey) < 0 {
  2699  						checkStart = sKey
  2700  					}
  2701  					if opts.Comparer.Compare(checkEnd, eKey) > 0 {
  2702  						checkEnd = eKey
  2703  					}
  2704  					var contained bool
  2705  					for _, kr := range keyRanges {
  2706  						contained = contained ||
  2707  							(opts.Comparer.Compare(checkStart, kr.Start) >= 0 &&
  2708  								opts.Comparer.Compare(checkEnd, kr.End) <= 0)
  2709  					}
  2710  					if !contained {
  2711  						t.Errorf("Seed %d, iter %d: File %s overlaps %q-%q, but is not fully contained in any of the key ranges.",
  2712  							seed, iter, f, sKey, eKey)
  2713  					}
  2714  				}
  2715  			}
  2716  		}
  2717  	}
  2718  }
  2719  
  2720  func TestCompactionAllowZeroSeqNum(t *testing.T) {
  2721  	var d *DB
  2722  	defer func() {
  2723  		if d != nil {
  2724  			require.NoError(t, closeAllSnapshots(d))
  2725  			require.NoError(t, d.Close())
  2726  		}
  2727  	}()
  2728  
  2729  	metaRE := regexp.MustCompile(`^L([0-9]+):([^-]+)-(.+)$`)
  2730  	var fileNum base.FileNum
  2731  	parseMeta := func(s string) (level int, meta *fileMetadata) {
  2732  		match := metaRE.FindStringSubmatch(s)
  2733  		if match == nil {
  2734  			t.Fatalf("malformed table spec: %s", s)
  2735  		}
  2736  		level, err := strconv.Atoi(match[1])
  2737  		if err != nil {
  2738  			t.Fatalf("malformed table spec: %s: %s", s, err)
  2739  		}
  2740  		fileNum++
  2741  		meta = (&fileMetadata{
  2742  			FileNum: fileNum,
  2743  		}).ExtendPointKeyBounds(
  2744  			d.cmp,
  2745  			InternalKey{UserKey: []byte(match[2])},
  2746  			InternalKey{UserKey: []byte(match[3])},
  2747  		)
  2748  		meta.InitPhysicalBacking()
  2749  		return level, meta
  2750  	}
  2751  
  2752  	datadriven.RunTest(t, "testdata/compaction_allow_zero_seqnum",
  2753  		func(t *testing.T, td *datadriven.TestData) string {
  2754  			switch td.Cmd {
  2755  			case "define":
  2756  				if d != nil {
  2757  					require.NoError(t, closeAllSnapshots(d))
  2758  					if err := d.Close(); err != nil {
  2759  						return err.Error()
  2760  					}
  2761  				}
  2762  
  2763  				var err error
  2764  				if d, err = runDBDefineCmd(td, nil /* options */); err != nil {
  2765  					return err.Error()
  2766  				}
  2767  
  2768  				d.mu.Lock()
  2769  				s := d.mu.versions.currentVersion().String()
  2770  				d.mu.Unlock()
  2771  				return s
  2772  
  2773  			case "allow-zero-seqnum":
  2774  				d.mu.Lock()
  2775  				c := &compaction{
  2776  					cmp:      d.cmp,
  2777  					comparer: d.opts.Comparer,
  2778  					version:  d.mu.versions.currentVersion(),
  2779  					inputs:   []compactionLevel{{}, {}},
  2780  				}
  2781  				c.startLevel, c.outputLevel = &c.inputs[0], &c.inputs[1]
  2782  				d.mu.Unlock()
  2783  
  2784  				var buf bytes.Buffer
  2785  				for _, line := range strings.Split(td.Input, "\n") {
  2786  					parts := strings.Fields(line)
  2787  					if len(parts) == 0 {
  2788  						continue
  2789  					}
  2790  					c.flushing = nil
  2791  					c.startLevel.level = -1
  2792  
  2793  					var startFiles, outputFiles []*fileMetadata
  2794  
  2795  					switch {
  2796  					case len(parts) == 1 && parts[0] == "flush":
  2797  						c.outputLevel.level = 0
  2798  						d.mu.Lock()
  2799  						c.flushing = d.mu.mem.queue
  2800  						d.mu.Unlock()
  2801  
  2802  					default:
  2803  						for _, p := range parts {
  2804  							level, meta := parseMeta(p)
  2805  							if c.startLevel.level == -1 {
  2806  								c.startLevel.level = level
  2807  							}
  2808  
  2809  							switch level {
  2810  							case c.startLevel.level:
  2811  								startFiles = append(startFiles, meta)
  2812  							case c.startLevel.level + 1:
  2813  								outputFiles = append(outputFiles, meta)
  2814  							default:
  2815  								return fmt.Sprintf("invalid level %d: expected %d or %d",
  2816  									level, c.startLevel.level, c.startLevel.level+1)
  2817  							}
  2818  						}
  2819  						c.outputLevel.level = c.startLevel.level + 1
  2820  						c.startLevel.files = manifest.NewLevelSliceSpecificOrder(startFiles)
  2821  						c.outputLevel.files = manifest.NewLevelSliceKeySorted(c.cmp, outputFiles)
  2822  					}
  2823  
  2824  					c.smallest, c.largest = manifest.KeyRange(c.cmp,
  2825  						c.startLevel.files.Iter(),
  2826  						c.outputLevel.files.Iter())
  2827  
  2828  					c.inuseKeyRanges = nil
  2829  					c.setupInuseKeyRanges()
  2830  					fmt.Fprintf(&buf, "%t\n", c.allowZeroSeqNum())
  2831  				}
  2832  				return buf.String()
  2833  
  2834  			default:
  2835  				return fmt.Sprintf("unknown command: %s", td.Cmd)
  2836  			}
  2837  		})
  2838  }
  2839  
  2840  func TestCompactionErrorOnUserKeyOverlap(t *testing.T) {
  2841  	cmp := DefaultComparer.Compare
  2842  	parseMeta := func(s string) *fileMetadata {
  2843  		parts := strings.Split(s, "-")
  2844  		if len(parts) != 2 {
  2845  			t.Fatalf("malformed table spec: %s", s)
  2846  		}
  2847  		m := (&fileMetadata{}).ExtendPointKeyBounds(
  2848  			cmp,
  2849  			base.ParseInternalKey(strings.TrimSpace(parts[0])),
  2850  			base.ParseInternalKey(strings.TrimSpace(parts[1])),
  2851  		)
  2852  		m.SmallestSeqNum = m.Smallest.SeqNum()
  2853  		m.LargestSeqNum = m.Largest.SeqNum()
  2854  		m.InitPhysicalBacking()
  2855  		return m
  2856  	}
  2857  
  2858  	datadriven.RunTest(t, "testdata/compaction_error_on_user_key_overlap",
  2859  		func(t *testing.T, d *datadriven.TestData) string {
  2860  			switch d.Cmd {
  2861  			case "error-on-user-key-overlap":
  2862  				c := &compaction{
  2863  					cmp:       DefaultComparer.Compare,
  2864  					comparer:  DefaultComparer,
  2865  					formatKey: DefaultComparer.FormatKey,
  2866  				}
  2867  				var files []manifest.NewFileEntry
  2868  				fileNum := FileNum(1)
  2869  
  2870  				for _, data := range strings.Split(d.Input, "\n") {
  2871  					meta := parseMeta(data)
  2872  					meta.FileNum = fileNum
  2873  					fileNum++
  2874  					files = append(files, manifest.NewFileEntry{Level: 1, Meta: meta})
  2875  				}
  2876  
  2877  				result := "OK"
  2878  				ve := &versionEdit{
  2879  					NewFiles: files,
  2880  				}
  2881  				if err := c.errorOnUserKeyOverlap(ve); err != nil {
  2882  					result = fmt.Sprint(err)
  2883  				}
  2884  				return result
  2885  
  2886  			default:
  2887  				return fmt.Sprintf("unknown command: %s", d.Cmd)
  2888  			}
  2889  		})
  2890  }
  2891  
  2892  // TestCompactionErrorCleanup tests an error encountered during a compaction
  2893  // after some output tables have been created. It ensures that the pending
  2894  // output tables are removed from the filesystem.
  2895  func TestCompactionErrorCleanup(t *testing.T) {
  2896  	// protected by d.mu
  2897  	var (
  2898  		initialSetupDone bool
  2899  		tablesCreated    []FileNum
  2900  	)
  2901  
  2902  	mem := vfs.NewMem()
  2903  	ii := errorfs.OnIndex(math.MaxInt32) // start disabled
  2904  	opts := (&Options{
  2905  		FS:     errorfs.Wrap(mem, ii),
  2906  		Levels: make([]LevelOptions, numLevels),
  2907  		EventListener: &EventListener{
  2908  			TableCreated: func(info TableCreateInfo) {
  2909  				t.Log(info)
  2910  
  2911  				// If the initial setup is over, record tables created and
  2912  				// inject an error immediately after the second table is
  2913  				// created.
  2914  				if initialSetupDone {
  2915  					tablesCreated = append(tablesCreated, info.FileNum)
  2916  					if len(tablesCreated) >= 2 {
  2917  						ii.SetIndex(0)
  2918  					}
  2919  				}
  2920  			},
  2921  		},
  2922  	}).WithFSDefaults()
  2923  	for i := range opts.Levels {
  2924  		opts.Levels[i].TargetFileSize = 1
  2925  	}
  2926  	opts.testingRandomized(t)
  2927  	d, err := Open("", opts)
  2928  	require.NoError(t, err)
  2929  
  2930  	ingest := func(keys ...string) {
  2931  		t.Helper()
  2932  		f, err := mem.Create("ext")
  2933  		require.NoError(t, err)
  2934  
  2935  		w := sstable.NewWriter(objstorageprovider.NewFileWritable(f), sstable.WriterOptions{
  2936  			TableFormat: d.FormatMajorVersion().MaxTableFormat(),
  2937  		})
  2938  		for _, k := range keys {
  2939  			require.NoError(t, w.Set([]byte(k), nil))
  2940  		}
  2941  		require.NoError(t, w.Close())
  2942  		require.NoError(t, d.Ingest([]string{"ext"}))
  2943  	}
  2944  	ingest("a", "c")
  2945  	ingest("b")
  2946  
  2947  	// Trigger a manual compaction, which will encounter an injected error
  2948  	// after the second table is created.
  2949  	d.mu.Lock()
  2950  	initialSetupDone = true
  2951  	d.mu.Unlock()
  2952  	err = d.Compact([]byte("a"), []byte("d"), false)
  2953  	require.Error(t, err, "injected error")
  2954  
  2955  	d.mu.Lock()
  2956  	if len(tablesCreated) < 2 {
  2957  		t.Fatalf("expected 2 output tables created by compaction: found %d", len(tablesCreated))
  2958  	}
  2959  	d.mu.Unlock()
  2960  
  2961  	require.NoError(t, d.Close())
  2962  	for _, fileNum := range tablesCreated {
  2963  		filename := fmt.Sprintf("%s.sst", fileNum)
  2964  		if _, err = mem.Stat(filename); err == nil || !oserror.IsNotExist(err) {
  2965  			t.Errorf("expected %q to not exist: %s", filename, err)
  2966  		}
  2967  	}
  2968  }
  2969  
  2970  func TestCompactionCheckOrdering(t *testing.T) {
  2971  	cmp := DefaultComparer.Compare
  2972  	parseMeta := func(s string) *fileMetadata {
  2973  		parts := strings.Split(s, "-")
  2974  		if len(parts) != 2 {
  2975  			t.Fatalf("malformed table spec: %s", s)
  2976  		}
  2977  		m := (&fileMetadata{}).ExtendPointKeyBounds(
  2978  			cmp,
  2979  			base.ParseInternalKey(strings.TrimSpace(parts[0])),
  2980  			base.ParseInternalKey(strings.TrimSpace(parts[1])),
  2981  		)
  2982  		m.SmallestSeqNum = m.Smallest.SeqNum()
  2983  		m.LargestSeqNum = m.Largest.SeqNum()
  2984  		m.InitPhysicalBacking()
  2985  		return m
  2986  	}
  2987  
  2988  	datadriven.RunTest(t, "testdata/compaction_check_ordering",
  2989  		func(t *testing.T, d *datadriven.TestData) string {
  2990  			switch d.Cmd {
  2991  			case "check-ordering":
  2992  				c := &compaction{
  2993  					cmp:       DefaultComparer.Compare,
  2994  					comparer:  DefaultComparer,
  2995  					formatKey: DefaultComparer.FormatKey,
  2996  					logger:    panicLogger{},
  2997  					inputs:    []compactionLevel{{level: -1}, {level: -1}},
  2998  				}
  2999  				c.startLevel, c.outputLevel = &c.inputs[0], &c.inputs[1]
  3000  				var startFiles, outputFiles []*fileMetadata
  3001  				var sublevels []manifest.LevelSlice
  3002  				var files *[]*fileMetadata
  3003  				var sublevel []*fileMetadata
  3004  				var sublevelNum int
  3005  				var parsingSublevel bool
  3006  				fileNum := FileNum(1)
  3007  
  3008  				switchSublevel := func() {
  3009  					if sublevel != nil {
  3010  						sublevels = append(
  3011  							sublevels, manifest.NewLevelSliceSpecificOrder(sublevel),
  3012  						)
  3013  						sublevel = nil
  3014  					}
  3015  					parsingSublevel = false
  3016  				}
  3017  
  3018  				for _, data := range strings.Split(d.Input, "\n") {
  3019  					if data[0] == 'L' && len(data) == 4 {
  3020  						// Format L0.{sublevel}.
  3021  						switchSublevel()
  3022  						level, err := strconv.Atoi(data[1:2])
  3023  						if err != nil {
  3024  							return err.Error()
  3025  						}
  3026  						sublevelNum, err = strconv.Atoi(data[3:])
  3027  						if err != nil {
  3028  							return err.Error()
  3029  						}
  3030  						if c.startLevel.level == -1 {
  3031  							c.startLevel.level = level
  3032  							files = &startFiles
  3033  						}
  3034  						parsingSublevel = true
  3035  					} else if data[0] == 'L' {
  3036  						switchSublevel()
  3037  						level, err := strconv.Atoi(data[1:])
  3038  						if err != nil {
  3039  							return err.Error()
  3040  						}
  3041  						if c.startLevel.level == -1 {
  3042  							c.startLevel.level = level
  3043  							files = &startFiles
  3044  						} else if c.outputLevel.level == -1 {
  3045  							if c.startLevel.level >= level {
  3046  								return fmt.Sprintf("startLevel=%d >= outputLevel=%d\n", c.startLevel.level, level)
  3047  							}
  3048  							c.outputLevel.level = level
  3049  							files = &outputFiles
  3050  						} else {
  3051  							return "outputLevel already set\n"
  3052  						}
  3053  					} else {
  3054  						meta := parseMeta(data)
  3055  						meta.FileNum = fileNum
  3056  						fileNum++
  3057  						*files = append(*files, meta)
  3058  						if parsingSublevel {
  3059  							meta.SubLevel = sublevelNum
  3060  							sublevel = append(sublevel, meta)
  3061  						}
  3062  					}
  3063  				}
  3064  
  3065  				switchSublevel()
  3066  				c.startLevel.files = manifest.NewLevelSliceSpecificOrder(startFiles)
  3067  				c.outputLevel.files = manifest.NewLevelSliceSpecificOrder(outputFiles)
  3068  				if c.outputLevel.level == -1 {
  3069  					c.outputLevel.level = 0
  3070  				}
  3071  				if c.startLevel.level == 0 {
  3072  					// We don't change the input files for the compaction beyond this point.
  3073  					c.startLevel.l0SublevelInfo = generateSublevelInfo(c.cmp, c.startLevel.files)
  3074  				}
  3075  
  3076  				newIters := func(
  3077  					_ context.Context, _ *manifest.FileMetadata, _ *IterOptions, _ internalIterOpts,
  3078  				) (internalIterator, keyspan.FragmentIterator, error) {
  3079  					return &errorIter{}, nil, nil
  3080  				}
  3081  				result := "OK"
  3082  				_, err := c.newInputIter(newIters, nil, nil)
  3083  				if err != nil {
  3084  					result = fmt.Sprint(err)
  3085  				}
  3086  				return result
  3087  
  3088  			default:
  3089  				return fmt.Sprintf("unknown command: %s", d.Cmd)
  3090  			}
  3091  		})
  3092  }
  3093  
  3094  type mockSplitter struct {
  3095  	shouldSplitVal maybeSplit
  3096  }
  3097  
  3098  func (m *mockSplitter) shouldSplitBefore(key *InternalKey, tw *sstable.Writer) maybeSplit {
  3099  	return m.shouldSplitVal
  3100  }
  3101  
  3102  func (m *mockSplitter) onNewOutput(key []byte) []byte {
  3103  	return nil
  3104  }
  3105  
  3106  func TestCompactionOutputSplitters(t *testing.T) {
  3107  	var main, child0, child1 compactionOutputSplitter
  3108  	var prevUserKey []byte
  3109  	pickSplitter := func(input string) *compactionOutputSplitter {
  3110  		switch input {
  3111  		case "main":
  3112  			return &main
  3113  		case "child0":
  3114  			return &child0
  3115  		case "child1":
  3116  			return &child1
  3117  		default:
  3118  			t.Fatalf("invalid splitter slot: %s", input)
  3119  			return nil
  3120  		}
  3121  	}
  3122  
  3123  	datadriven.RunTest(t, "testdata/compaction_output_splitters",
  3124  		func(t *testing.T, d *datadriven.TestData) string {
  3125  			switch d.Cmd {
  3126  			case "reset":
  3127  				main = nil
  3128  				child0 = nil
  3129  				child1 = nil
  3130  			case "init":
  3131  				if len(d.CmdArgs) < 2 {
  3132  					return "expected at least 2 args"
  3133  				}
  3134  				splitterToInit := pickSplitter(d.CmdArgs[0].Key)
  3135  				switch d.CmdArgs[1].Key {
  3136  				case "array":
  3137  					*splitterToInit = &splitterGroup{
  3138  						cmp:       base.DefaultComparer.Compare,
  3139  						splitters: []compactionOutputSplitter{child0, child1},
  3140  					}
  3141  				case "mock":
  3142  					*splitterToInit = &mockSplitter{}
  3143  				case "userkey":
  3144  					*splitterToInit = &userKeyChangeSplitter{
  3145  						cmp: base.DefaultComparer.Compare,
  3146  						unsafePrevUserKey: func() []byte {
  3147  							return prevUserKey
  3148  						},
  3149  						splitter: child0,
  3150  					}
  3151  				}
  3152  				(*splitterToInit).onNewOutput(nil)
  3153  			case "set-should-split":
  3154  				if len(d.CmdArgs) < 2 {
  3155  					return "expected at least 2 args"
  3156  				}
  3157  				splitterToSet := (*pickSplitter(d.CmdArgs[0].Key)).(*mockSplitter)
  3158  				var val maybeSplit
  3159  				switch d.CmdArgs[1].Key {
  3160  				case "split-now":
  3161  					val = splitNow
  3162  				case "no-split":
  3163  					val = noSplit
  3164  				default:
  3165  					t.Fatalf("unexpected value for should-split: %s", d.CmdArgs[1].Key)
  3166  				}
  3167  				splitterToSet.shouldSplitVal = val
  3168  			case "should-split-before":
  3169  				if len(d.CmdArgs) < 1 {
  3170  					return "expected at least 1 arg"
  3171  				}
  3172  				key := base.ParseInternalKey(d.CmdArgs[0].Key)
  3173  				shouldSplit := main.shouldSplitBefore(&key, nil)
  3174  				if shouldSplit == splitNow {
  3175  					main.onNewOutput(key.UserKey)
  3176  					prevUserKey = nil
  3177  				} else {
  3178  					prevUserKey = key.UserKey
  3179  				}
  3180  				return shouldSplit.String()
  3181  			default:
  3182  				return fmt.Sprintf("unknown command: %s", d.Cmd)
  3183  			}
  3184  			return "ok"
  3185  		})
  3186  }
  3187  
  3188  func TestCompactFlushQueuedMemTableAndFlushMetrics(t *testing.T) {
  3189  	if runtime.GOOS == "windows" {
  3190  		t.Skip("test is flaky on windows")
  3191  	}
  3192  
  3193  	// Verify that manual compaction forces a flush of a queued memtable.
  3194  
  3195  	mem := vfs.NewMem()
  3196  	d, err := Open("", testingRandomized(t, &Options{
  3197  		FS: mem,
  3198  	}).WithFSDefaults())
  3199  	require.NoError(t, err)
  3200  
  3201  	// Add the key "a" to the memtable, then fill up the memtable with the key
  3202  	// prefix "b". The compaction will only overlap with the queued memtable,
  3203  	// not the mutable memtable.
  3204  	// NB: The initial memtable size is 256KB, which is filled up with random
  3205  	// values which typically don't compress well. The test also appends the
  3206  	// random value to the "b" key to limit overwriting of the same key, which
  3207  	// would get collapsed at flush time since there are no open snapshots.
  3208  	value := make([]byte, 50)
  3209  	_, err = crand.Read(value)
  3210  	require.NoError(t, err)
  3211  	require.NoError(t, d.Set([]byte("a"), value, nil))
  3212  	for {
  3213  		_, err = crand.Read(value)
  3214  		require.NoError(t, err)
  3215  		require.NoError(t, d.Set(append([]byte("b"), value...), value, nil))
  3216  		d.mu.Lock()
  3217  		done := len(d.mu.mem.queue) == 2
  3218  		d.mu.Unlock()
  3219  		if done {
  3220  			break
  3221  		}
  3222  	}
  3223  
  3224  	require.NoError(t, d.Compact([]byte("a"), []byte("a\x00"), false))
  3225  	d.mu.Lock()
  3226  	require.Equal(t, 1, len(d.mu.mem.queue))
  3227  	d.mu.Unlock()
  3228  	// Flush metrics are updated after and non-atomically with the memtable
  3229  	// being removed from the queue.
  3230  	func() {
  3231  		begin := time.Now()
  3232  		for {
  3233  			metrics := d.Metrics()
  3234  			require.NotNil(t, metrics)
  3235  			if int64(50<<10) < metrics.Flush.WriteThroughput.Bytes {
  3236  				// The writes (during which the flush is idle) and the flush work
  3237  				// should not be so fast as to be unrealistic. If these turn out to be
  3238  				// flaky we could instead inject a clock.
  3239  				tinyInterval := int64(50 * time.Microsecond)
  3240  				require.Less(t, tinyInterval, int64(metrics.Flush.WriteThroughput.WorkDuration))
  3241  				require.Less(t, tinyInterval, int64(metrics.Flush.WriteThroughput.IdleDuration))
  3242  				return
  3243  			}
  3244  			if time.Since(begin) > 2*time.Second {
  3245  				t.Fatal()
  3246  			}
  3247  			time.Sleep(time.Millisecond)
  3248  		}
  3249  	}()
  3250  	require.NoError(t, d.Close())
  3251  }
  3252  
  3253  func TestCompactFlushQueuedLargeBatch(t *testing.T) {
  3254  	// Verify that compaction forces a flush of a queued large batch.
  3255  
  3256  	mem := vfs.NewMem()
  3257  	d, err := Open("", testingRandomized(t, &Options{
  3258  		FS: mem,
  3259  	}).WithFSDefaults())
  3260  	require.NoError(t, err)
  3261  
  3262  	// The default large batch threshold is slightly less than 1/2 of the
  3263  	// memtable size which makes triggering a problem with flushing queued large
  3264  	// batches irritating. Manually adjust the threshold to 1/8 of the memtable
  3265  	// size in order to more easily create a situation where a large batch is
  3266  	// queued but not automatically flushed.
  3267  	d.mu.Lock()
  3268  	d.largeBatchThreshold = d.opts.MemTableSize / 8
  3269  	require.Equal(t, 1, len(d.mu.mem.queue))
  3270  	d.mu.Unlock()
  3271  
  3272  	// Set a record with a large value. This will be transformed into a large
  3273  	// batch and placed in the flushable queue.
  3274  	require.NoError(t, d.Set([]byte("a"), bytes.Repeat([]byte("v"), int(d.largeBatchThreshold)), nil))
  3275  	d.mu.Lock()
  3276  	require.Greater(t, len(d.mu.mem.queue), 1)
  3277  	d.mu.Unlock()
  3278  
  3279  	require.NoError(t, d.Compact([]byte("a"), []byte("a\x00"), false))
  3280  	d.mu.Lock()
  3281  	require.Equal(t, 1, len(d.mu.mem.queue))
  3282  	d.mu.Unlock()
  3283  
  3284  	require.NoError(t, d.Close())
  3285  }
  3286  
  3287  func TestFlushError(t *testing.T) {
  3288  	// Error the first five times we try to write a sstable.
  3289  	var errorOps atomic.Int32
  3290  	errorOps.Store(3)
  3291  	fs := errorfs.Wrap(vfs.NewMem(), errorfs.InjectorFunc(func(op errorfs.Op, path string) error {
  3292  		if op == errorfs.OpCreate && filepath.Ext(path) == ".sst" && errorOps.Add(-1) >= 0 {
  3293  			return errorfs.ErrInjected
  3294  		}
  3295  		return nil
  3296  	}))
  3297  	d, err := Open("", testingRandomized(t, &Options{
  3298  		FS: fs,
  3299  		EventListener: &EventListener{
  3300  			BackgroundError: func(err error) {
  3301  				t.Log(err)
  3302  			},
  3303  		},
  3304  	}).WithFSDefaults())
  3305  	require.NoError(t, err)
  3306  	require.NoError(t, d.Set([]byte("a"), []byte("foo"), NoSync))
  3307  	require.NoError(t, d.Flush())
  3308  	require.NoError(t, d.Close())
  3309  }
  3310  
  3311  func TestAdjustGrandparentOverlapBytesForFlush(t *testing.T) {
  3312  	// 500MB in Lbase
  3313  	var lbaseFiles []*manifest.FileMetadata
  3314  	const lbaseSize = 5 << 20
  3315  	for i := 0; i < 100; i++ {
  3316  		m := &manifest.FileMetadata{Size: lbaseSize, FileNum: FileNum(i)}
  3317  		m.InitPhysicalBacking()
  3318  		lbaseFiles =
  3319  			append(lbaseFiles, m)
  3320  	}
  3321  	const maxOutputFileSize = 2 << 20
  3322  	// 20MB max overlap, so flush split into 25 files.
  3323  	const maxOverlapBytes = 20 << 20
  3324  	ls := manifest.NewLevelSliceSpecificOrder(lbaseFiles)
  3325  	testCases := []struct {
  3326  		flushingBytes        uint64
  3327  		adjustedOverlapBytes uint64
  3328  	}{
  3329  		// Flushes large enough that 25 files is acceptable.
  3330  		{flushingBytes: 128 << 20, adjustedOverlapBytes: 20971520},
  3331  		{flushingBytes: 64 << 20, adjustedOverlapBytes: 20971520},
  3332  		// Small increase in adjustedOverlapBytes.
  3333  		{flushingBytes: 32 << 20, adjustedOverlapBytes: 32768000},
  3334  		// Large increase in adjusterOverlapBytes, to limit to 4 files.
  3335  		{flushingBytes: 1 << 20, adjustedOverlapBytes: 131072000},
  3336  	}
  3337  	for _, tc := range testCases {
  3338  		t.Run("", func(t *testing.T) {
  3339  			c := compaction{
  3340  				grandparents:      ls,
  3341  				maxOverlapBytes:   maxOverlapBytes,
  3342  				maxOutputFileSize: maxOutputFileSize,
  3343  			}
  3344  			adjustGrandparentOverlapBytesForFlush(&c, tc.flushingBytes)
  3345  			require.Equal(t, tc.adjustedOverlapBytes, c.maxOverlapBytes)
  3346  		})
  3347  	}
  3348  }
  3349  
  3350  func TestCompactionInvalidBounds(t *testing.T) {
  3351  	db, err := Open("", testingRandomized(t, &Options{
  3352  		FS: vfs.NewMem(),
  3353  	}).WithFSDefaults())
  3354  	require.NoError(t, err)
  3355  	defer db.Close()
  3356  	require.NoError(t, db.Compact([]byte("a"), []byte("b"), false))
  3357  	require.Error(t, db.Compact([]byte("a"), []byte("a"), false))
  3358  	require.Error(t, db.Compact([]byte("b"), []byte("a"), false))
  3359  }
  3360  
  3361  func Test_calculateInuseKeyRanges(t *testing.T) {
  3362  	opts := (*Options)(nil).EnsureDefaults()
  3363  	cmp := base.DefaultComparer.Compare
  3364  	newFileMeta := func(fileNum FileNum, size uint64, smallest, largest base.InternalKey) *fileMetadata {
  3365  		m := (&fileMetadata{
  3366  			FileNum: fileNum,
  3367  			Size:    size,
  3368  		}).ExtendPointKeyBounds(opts.Comparer.Compare, smallest, largest)
  3369  		m.InitPhysicalBacking()
  3370  		return m
  3371  	}
  3372  	tests := []struct {
  3373  		name     string
  3374  		v        *version
  3375  		level    int
  3376  		depth    int
  3377  		smallest []byte
  3378  		largest  []byte
  3379  		want     []manifest.UserKeyRange
  3380  	}{
  3381  		{
  3382  			name: "No files in next level",
  3383  			v: newVersion(opts, [numLevels][]*fileMetadata{
  3384  				1: {
  3385  					newFileMeta(
  3386  						1,
  3387  						1,
  3388  						base.ParseInternalKey("a.SET.2"),
  3389  						base.ParseInternalKey("c.SET.2"),
  3390  					),
  3391  					newFileMeta(
  3392  						2,
  3393  						1,
  3394  						base.ParseInternalKey("d.SET.2"),
  3395  						base.ParseInternalKey("e.SET.2"),
  3396  					),
  3397  				},
  3398  			}),
  3399  			level:    1,
  3400  			depth:    2,
  3401  			smallest: []byte("a"),
  3402  			largest:  []byte("e"),
  3403  			want: []manifest.UserKeyRange{
  3404  				{
  3405  					Start: []byte("a"),
  3406  					End:   []byte("c"),
  3407  				},
  3408  				{
  3409  					Start: []byte("d"),
  3410  					End:   []byte("e"),
  3411  				},
  3412  			},
  3413  		},
  3414  		{
  3415  			name: "No overlapping key ranges",
  3416  			v: newVersion(opts, [numLevels][]*fileMetadata{
  3417  				1: {
  3418  					newFileMeta(
  3419  						1,
  3420  						1,
  3421  						base.ParseInternalKey("a.SET.1"),
  3422  						base.ParseInternalKey("c.SET.1"),
  3423  					),
  3424  					newFileMeta(
  3425  						2,
  3426  						1,
  3427  						base.ParseInternalKey("l.SET.1"),
  3428  						base.ParseInternalKey("p.SET.1"),
  3429  					),
  3430  				},
  3431  				2: {
  3432  					newFileMeta(
  3433  						3,
  3434  						1,
  3435  						base.ParseInternalKey("d.SET.1"),
  3436  						base.ParseInternalKey("i.SET.1"),
  3437  					),
  3438  					newFileMeta(
  3439  						4,
  3440  						1,
  3441  						base.ParseInternalKey("s.SET.1"),
  3442  						base.ParseInternalKey("w.SET.1"),
  3443  					),
  3444  				},
  3445  			}),
  3446  			level:    1,
  3447  			depth:    2,
  3448  			smallest: []byte("a"),
  3449  			largest:  []byte("z"),
  3450  			want: []manifest.UserKeyRange{
  3451  				{
  3452  					Start: []byte("a"),
  3453  					End:   []byte("c"),
  3454  				},
  3455  				{
  3456  					Start: []byte("d"),
  3457  					End:   []byte("i"),
  3458  				},
  3459  				{
  3460  					Start: []byte("l"),
  3461  					End:   []byte("p"),
  3462  				},
  3463  				{
  3464  					Start: []byte("s"),
  3465  					End:   []byte("w"),
  3466  				},
  3467  			},
  3468  		},
  3469  		{
  3470  			name: "First few non-overlapping, followed by overlapping",
  3471  			v: newVersion(opts, [numLevels][]*fileMetadata{
  3472  				1: {
  3473  					newFileMeta(
  3474  						1,
  3475  						1,
  3476  						base.ParseInternalKey("a.SET.1"),
  3477  						base.ParseInternalKey("c.SET.1"),
  3478  					),
  3479  					newFileMeta(
  3480  						2,
  3481  						1,
  3482  						base.ParseInternalKey("d.SET.1"),
  3483  						base.ParseInternalKey("e.SET.1"),
  3484  					),
  3485  					newFileMeta(
  3486  						3,
  3487  						1,
  3488  						base.ParseInternalKey("n.SET.1"),
  3489  						base.ParseInternalKey("o.SET.1"),
  3490  					),
  3491  					newFileMeta(
  3492  						4,
  3493  						1,
  3494  						base.ParseInternalKey("p.SET.1"),
  3495  						base.ParseInternalKey("q.SET.1"),
  3496  					),
  3497  				},
  3498  				2: {
  3499  					newFileMeta(
  3500  						5,
  3501  						1,
  3502  						base.ParseInternalKey("m.SET.1"),
  3503  						base.ParseInternalKey("q.SET.1"),
  3504  					),
  3505  					newFileMeta(
  3506  						6,
  3507  						1,
  3508  						base.ParseInternalKey("s.SET.1"),
  3509  						base.ParseInternalKey("w.SET.1"),
  3510  					),
  3511  				},
  3512  			}),
  3513  			level:    1,
  3514  			depth:    2,
  3515  			smallest: []byte("a"),
  3516  			largest:  []byte("z"),
  3517  			want: []manifest.UserKeyRange{
  3518  				{
  3519  					Start: []byte("a"),
  3520  					End:   []byte("c"),
  3521  				},
  3522  				{
  3523  					Start: []byte("d"),
  3524  					End:   []byte("e"),
  3525  				},
  3526  				{
  3527  					Start: []byte("m"),
  3528  					End:   []byte("q"),
  3529  				},
  3530  				{
  3531  					Start: []byte("s"),
  3532  					End:   []byte("w"),
  3533  				},
  3534  			},
  3535  		},
  3536  		{
  3537  			name: "All overlapping",
  3538  			v: newVersion(opts, [numLevels][]*fileMetadata{
  3539  				1: {
  3540  					newFileMeta(
  3541  						1,
  3542  						1,
  3543  						base.ParseInternalKey("d.SET.1"),
  3544  						base.ParseInternalKey("e.SET.1"),
  3545  					),
  3546  					newFileMeta(
  3547  						2,
  3548  						1,
  3549  						base.ParseInternalKey("n.SET.1"),
  3550  						base.ParseInternalKey("o.SET.1"),
  3551  					),
  3552  					newFileMeta(
  3553  						3,
  3554  						1,
  3555  						base.ParseInternalKey("p.SET.1"),
  3556  						base.ParseInternalKey("q.SET.1"),
  3557  					),
  3558  				},
  3559  				2: {
  3560  					newFileMeta(
  3561  						4,
  3562  						1,
  3563  						base.ParseInternalKey("a.SET.1"),
  3564  						base.ParseInternalKey("c.SET.1"),
  3565  					),
  3566  					newFileMeta(
  3567  						5,
  3568  						1,
  3569  						base.ParseInternalKey("d.SET.1"),
  3570  						base.ParseInternalKey("w.SET.1"),
  3571  					),
  3572  				},
  3573  			}),
  3574  			level:    1,
  3575  			depth:    2,
  3576  			smallest: []byte("a"),
  3577  			largest:  []byte("z"),
  3578  			want: []manifest.UserKeyRange{
  3579  				{
  3580  					Start: []byte("a"),
  3581  					End:   []byte("c"),
  3582  				},
  3583  				{
  3584  					Start: []byte("d"),
  3585  					End:   []byte("w"),
  3586  				},
  3587  			},
  3588  		},
  3589  	}
  3590  	for _, tt := range tests {
  3591  		t.Run(tt.name, func(t *testing.T) {
  3592  			if got := calculateInuseKeyRanges(tt.v, cmp, tt.level, tt.depth, tt.smallest, tt.largest); !reflect.DeepEqual(got, tt.want) {
  3593  				t.Errorf("calculateInuseKeyRanges() = %v, want %v", got, tt.want)
  3594  			}
  3595  		})
  3596  	}
  3597  }
  3598  
  3599  func TestMarkedForCompaction(t *testing.T) {
  3600  	var mem vfs.FS = vfs.NewMem()
  3601  	var d *DB
  3602  	defer func() {
  3603  		if d != nil {
  3604  			require.NoError(t, d.Close())
  3605  		}
  3606  	}()
  3607  
  3608  	var buf bytes.Buffer
  3609  	opts := (&Options{
  3610  		FS:                          mem,
  3611  		DebugCheck:                  DebugCheckLevels,
  3612  		DisableAutomaticCompactions: true,
  3613  		FormatMajorVersion:          internalFormatNewest,
  3614  		EventListener: &EventListener{
  3615  			CompactionEnd: func(info CompactionInfo) {
  3616  				// Fix the job ID and durations for determinism.
  3617  				info.JobID = 100
  3618  				info.Duration = time.Second
  3619  				info.TotalDuration = 2 * time.Second
  3620  				fmt.Fprintln(&buf, info)
  3621  			},
  3622  		},
  3623  	}).WithFSDefaults()
  3624  
  3625  	reset := func() {
  3626  		if d != nil {
  3627  			require.NoError(t, d.Close())
  3628  		}
  3629  		mem = vfs.NewMem()
  3630  		require.NoError(t, mem.MkdirAll("ext", 0755))
  3631  
  3632  		var err error
  3633  		d, err = Open("", opts)
  3634  		require.NoError(t, err)
  3635  	}
  3636  	datadriven.RunTest(t, "testdata/marked_for_compaction", func(t *testing.T, td *datadriven.TestData) string {
  3637  		switch td.Cmd {
  3638  		case "reset":
  3639  			reset()
  3640  			return ""
  3641  
  3642  		case "define":
  3643  			if d != nil {
  3644  				if err := d.Close(); err != nil {
  3645  					return err.Error()
  3646  				}
  3647  			}
  3648  			var err error
  3649  			if d, err = runDBDefineCmd(td, opts); err != nil {
  3650  				return err.Error()
  3651  			}
  3652  			d.mu.Lock()
  3653  			defer d.mu.Unlock()
  3654  			t := time.Now()
  3655  			d.timeNow = func() time.Time {
  3656  				t = t.Add(time.Second)
  3657  				return t
  3658  			}
  3659  			s := d.mu.versions.currentVersion().DebugString(base.DefaultFormatter)
  3660  			return s
  3661  
  3662  		case "mark-for-compaction":
  3663  			d.mu.Lock()
  3664  			defer d.mu.Unlock()
  3665  			vers := d.mu.versions.currentVersion()
  3666  			var fileNum uint64
  3667  			td.ScanArgs(t, "file", &fileNum)
  3668  			for l, lm := range vers.Levels {
  3669  				iter := lm.Iter()
  3670  				for f := iter.First(); f != nil; f = iter.Next() {
  3671  					if f.FileNum != base.FileNum(fileNum) {
  3672  						continue
  3673  					}
  3674  					f.MarkedForCompaction = true
  3675  					vers.Stats.MarkedForCompaction++
  3676  					vers.Levels[l].InvalidateAnnotation(markedForCompactionAnnotator{})
  3677  					return fmt.Sprintf("marked L%d.%s", l, f.FileNum)
  3678  				}
  3679  			}
  3680  			return "not-found"
  3681  
  3682  		case "maybe-compact":
  3683  			d.mu.Lock()
  3684  			defer d.mu.Unlock()
  3685  			d.opts.DisableAutomaticCompactions = false
  3686  			d.maybeScheduleCompaction()
  3687  			for d.mu.compact.compactingCount > 0 {
  3688  				d.mu.compact.cond.Wait()
  3689  			}
  3690  
  3691  			fmt.Fprintln(&buf, d.mu.versions.currentVersion().DebugString(base.DefaultFormatter))
  3692  			s := strings.TrimSpace(buf.String())
  3693  			buf.Reset()
  3694  			opts.DisableAutomaticCompactions = true
  3695  			return s
  3696  
  3697  		default:
  3698  			return fmt.Sprintf("unknown command: %s", td.Cmd)
  3699  		}
  3700  	})
  3701  }
  3702  
  3703  // createManifestErrorInjector injects errors (when enabled) into vfs.FS calls
  3704  // to create MANIFEST files.
  3705  type createManifestErrorInjector struct {
  3706  	enabled atomic.Bool
  3707  }
  3708  
  3709  // enable enables error injection for the vfs.FS.
  3710  func (i *createManifestErrorInjector) enable() {
  3711  	i.enabled.Store(true)
  3712  }
  3713  
  3714  // MaybeError implements errorfs.Injector.
  3715  func (i *createManifestErrorInjector) MaybeError(op errorfs.Op, path string) error {
  3716  	if !i.enabled.Load() {
  3717  		return nil
  3718  	}
  3719  	// This necessitates having a MaxManifestSize of 1, to reliably induce
  3720  	// logAndApply errors.
  3721  	if strings.Contains(path, "MANIFEST") && op == errorfs.OpCreate {
  3722  		return errorfs.ErrInjected
  3723  	}
  3724  	return nil
  3725  }
  3726  
  3727  var _ errorfs.Injector = &createManifestErrorInjector{}
  3728  
  3729  // TestCompaction_LogAndApplyFails exercises a flush or ingest encountering an
  3730  // unrecoverable error during logAndApply.
  3731  //
  3732  // Regression test for #1669.
  3733  func TestCompaction_LogAndApplyFails(t *testing.T) {
  3734  	// flushKeys writes the given keys to the DB, flushing the resulting memtable.
  3735  	var key = []byte("foo")
  3736  	flushErrC := make(chan error)
  3737  	flushKeys := func(db *DB) error {
  3738  		b := db.NewBatch()
  3739  		err := b.Set(key, nil, nil)
  3740  		require.NoError(t, err)
  3741  		err = b.Commit(nil)
  3742  		require.NoError(t, err)
  3743  		// An error from a failing flush is returned asynchronously.
  3744  		go func() { _ = db.Flush() }()
  3745  		return <-flushErrC
  3746  	}
  3747  
  3748  	// ingestKeys adds the given keys to the DB via an ingestion.
  3749  	ingestKeys := func(db *DB) error {
  3750  		// Create an SST for ingestion.
  3751  		const fName = "ext"
  3752  		f, err := db.opts.FS.Create(fName)
  3753  		require.NoError(t, err)
  3754  		w := sstable.NewWriter(objstorageprovider.NewFileWritable(f), sstable.WriterOptions{})
  3755  		require.NoError(t, w.Set(key, nil))
  3756  		require.NoError(t, w.Close())
  3757  		// Ingest the SST.
  3758  		return db.Ingest([]string{fName})
  3759  	}
  3760  
  3761  	testCases := []struct {
  3762  		name              string
  3763  		addFn             func(db *DB) error
  3764  		backgroundErrorFn func(*DB, error)
  3765  	}{
  3766  		{
  3767  			name:  "flush",
  3768  			addFn: flushKeys,
  3769  			backgroundErrorFn: func(db *DB, err error) {
  3770  				require.True(t, errors.Is(err, errorfs.ErrInjected))
  3771  				flushErrC <- err
  3772  				// A flush will attempt to retry in the background. For the purposes of
  3773  				// testing this particular scenario, where we would have crashed anyway,
  3774  				// drop the memtable on the floor to short circuit the retry loop.
  3775  				// NB: we hold db.mu here.
  3776  				var cur *flushableEntry
  3777  				cur, db.mu.mem.queue = db.mu.mem.queue[0], db.mu.mem.queue[1:]
  3778  				cur.readerUnrefLocked(true)
  3779  			},
  3780  		},
  3781  		{
  3782  			name:  "ingest",
  3783  			addFn: ingestKeys,
  3784  		},
  3785  	}
  3786  
  3787  	runTest := func(t *testing.T, addFn func(db *DB) error, bgFn func(*DB, error)) {
  3788  		var db *DB
  3789  		inj := &createManifestErrorInjector{}
  3790  		logger := &fatalCapturingLogger{t: t}
  3791  		opts := (&Options{
  3792  			FS: errorfs.Wrap(vfs.NewMem(), inj),
  3793  			// Rotate the manifest after each write. This is required to trigger a
  3794  			// file creation, into which errors can be injected.
  3795  			MaxManifestFileSize: 1,
  3796  			Logger:              logger,
  3797  			EventListener: &EventListener{
  3798  				BackgroundError: func(err error) {
  3799  					if bgFn != nil {
  3800  						bgFn(db, err)
  3801  					}
  3802  				},
  3803  			},
  3804  			DisableAutomaticCompactions: true,
  3805  		}).WithFSDefaults()
  3806  
  3807  		db, err := Open("", opts)
  3808  		require.NoError(t, err)
  3809  		defer func() { _ = db.Close() }()
  3810  
  3811  		inj.enable()
  3812  		err = addFn(db)
  3813  		require.True(t, errors.Is(err, errorfs.ErrInjected))
  3814  
  3815  		// Under normal circumstances, such an error in logAndApply would panic and
  3816  		// cause the DB to terminate here. Assert that we captured the fatal error.
  3817  		require.True(t, errors.Is(logger.err, errorfs.ErrInjected))
  3818  	}
  3819  	for _, tc := range testCases {
  3820  		t.Run(tc.name, func(t *testing.T) {
  3821  			runTest(t, tc.addFn, tc.backgroundErrorFn)
  3822  		})
  3823  	}
  3824  }
  3825  
  3826  // TestSharedObjectDeletePacing tests that we don't throttle shared object
  3827  // deletes (see the TargetBytesDeletionRate option).
  3828  func TestSharedObjectDeletePacing(t *testing.T) {
  3829  	var opts Options
  3830  	opts.FS = vfs.NewMem()
  3831  	opts.Experimental.RemoteStorage = remote.MakeSimpleFactory(map[remote.Locator]remote.Storage{
  3832  		"": remote.NewInMem(),
  3833  	})
  3834  	opts.Experimental.CreateOnShared = remote.CreateOnSharedAll
  3835  	opts.TargetByteDeletionRate = 1
  3836  
  3837  	d, err := Open("", &opts)
  3838  	require.NoError(t, err)
  3839  	require.NoError(t, d.SetCreatorID(1))
  3840  
  3841  	randVal := func() []byte {
  3842  		res := make([]byte, 1024)
  3843  		_, err := crand.Read(res)
  3844  		require.NoError(t, err)
  3845  		return res
  3846  	}
  3847  
  3848  	// We must set up things so that we will have more live bytes than obsolete
  3849  	// bytes, otherwise delete pacing will be disabled anyway.
  3850  	key := func(i int) string {
  3851  		return fmt.Sprintf("k%02d", i)
  3852  	}
  3853  	const numKeys = 20
  3854  	for i := 1; i <= numKeys; i++ {
  3855  		require.NoError(t, d.Set([]byte(key(i)), randVal(), nil))
  3856  		require.NoError(t, d.Compact([]byte(key(i)), []byte(key(i)+"1"), false))
  3857  	}
  3858  
  3859  	done := make(chan struct{})
  3860  	go func() {
  3861  		err = d.DeleteRange([]byte(key(5)), []byte(key(9)), nil)
  3862  		if err == nil {
  3863  			err = d.Compact([]byte(key(5)), []byte(key(9)), false)
  3864  		}
  3865  		// Wait for objects to be deleted.
  3866  		for {
  3867  			time.Sleep(10 * time.Millisecond)
  3868  			if len(d.objProvider.List()) < numKeys-2 {
  3869  				break
  3870  			}
  3871  		}
  3872  		close(done)
  3873  	}()
  3874  
  3875  	select {
  3876  	case <-time.After(60 * time.Second):
  3877  		// Don't close the DB in this case (the goroutine above might panic).
  3878  		t.Fatalf("compaction timed out, possibly due to incorrect deletion pacing")
  3879  	case <-done:
  3880  	}
  3881  	require.NoError(t, err)
  3882  	d.Close()
  3883  }
  3884  
  3885  type WriteErrorInjector struct {
  3886  	enabled atomic.Bool
  3887  }
  3888  
  3889  // enable enables error injection for the vfs.FS.
  3890  func (i *WriteErrorInjector) enable() {
  3891  	i.enabled.Store(true)
  3892  }
  3893  
  3894  // disable disabled error injection for the vfs.FS.
  3895  func (i *WriteErrorInjector) disable() {
  3896  	i.enabled.Store(false)
  3897  }
  3898  
  3899  // MaybeError implements errorfs.Injector.
  3900  func (i *WriteErrorInjector) MaybeError(op errorfs.Op, path string) error {
  3901  	if !i.enabled.Load() {
  3902  		return nil
  3903  	}
  3904  	// Fail any future write.
  3905  	if op == errorfs.OpFileWrite {
  3906  		return errorfs.ErrInjected
  3907  	}
  3908  	return nil
  3909  }
  3910  
  3911  var _ errorfs.Injector = &WriteErrorInjector{}
  3912  
  3913  // Cumulative compaction stats shouldn't be updated on compaction error.
  3914  func TestCompactionErrorStats(t *testing.T) {
  3915  	// protected by d.mu
  3916  	var (
  3917  		useInjector   bool
  3918  		tablesCreated []FileNum
  3919  	)
  3920  
  3921  	mem := vfs.NewMem()
  3922  	injector := &WriteErrorInjector{}
  3923  	opts := (&Options{
  3924  		FS:     errorfs.Wrap(mem, injector),
  3925  		Levels: make([]LevelOptions, numLevels),
  3926  		EventListener: &EventListener{
  3927  			TableCreated: func(info TableCreateInfo) {
  3928  				t.Log(info)
  3929  
  3930  				if useInjector {
  3931  					// We'll write 3 tables during compaction, and we only need
  3932  					// the writes to error on the third file write, so only enable
  3933  					// the injector after the first two files have been written to.
  3934  					tablesCreated = append(tablesCreated, info.FileNum)
  3935  					if len(tablesCreated) >= 2 {
  3936  						injector.enable()
  3937  					}
  3938  				}
  3939  			},
  3940  		},
  3941  	}).WithFSDefaults()
  3942  	for i := range opts.Levels {
  3943  		opts.Levels[i].TargetFileSize = 1
  3944  	}
  3945  	opts.testingRandomized(t)
  3946  	d, err := Open("", opts)
  3947  	require.NoError(t, err)
  3948  
  3949  	ingest := func(keys ...string) {
  3950  		t.Helper()
  3951  		f, err := mem.Create("ext")
  3952  		require.NoError(t, err)
  3953  
  3954  		w := sstable.NewWriter(objstorageprovider.NewFileWritable(f), sstable.WriterOptions{
  3955  			TableFormat: d.FormatMajorVersion().MaxTableFormat(),
  3956  		})
  3957  		for _, k := range keys {
  3958  			require.NoError(t, w.Set([]byte(k), nil))
  3959  		}
  3960  		require.NoError(t, w.Close())
  3961  		require.NoError(t, d.Ingest([]string{"ext"}))
  3962  	}
  3963  	ingest("a", "c")
  3964  	// Snapshot will preserve the older "a" key during compaction.
  3965  	snap := d.NewSnapshot()
  3966  	ingest("a", "b")
  3967  
  3968  	// Trigger a manual compaction, which will encounter an injected error
  3969  	// after the second table is created.
  3970  	d.mu.Lock()
  3971  	useInjector = true
  3972  	d.mu.Unlock()
  3973  
  3974  	err = d.Compact([]byte("a"), []byte("d"), false)
  3975  	require.Error(t, err, "injected error")
  3976  
  3977  	// Due to the error, stats shouldn't have been updated.
  3978  	d.mu.Lock()
  3979  	require.Equal(t, 0, int(d.mu.snapshots.cumulativePinnedCount))
  3980  	require.Equal(t, 0, int(d.mu.snapshots.cumulativePinnedSize))
  3981  	useInjector = false
  3982  	d.mu.Unlock()
  3983  
  3984  	injector.disable()
  3985  
  3986  	// The following compaction won't error, but snapshot is open, so snapshot
  3987  	// pinned stats should update.
  3988  	require.NoError(t, d.Compact([]byte("a"), []byte("d"), false))
  3989  	require.NoError(t, snap.Close())
  3990  
  3991  	d.mu.Lock()
  3992  	require.Equal(t, 1, int(d.mu.snapshots.cumulativePinnedCount))
  3993  	require.Equal(t, 9, int(d.mu.snapshots.cumulativePinnedSize))
  3994  	d.mu.Unlock()
  3995  	require.NoError(t, d.Close())
  3996  }