Line data Source code
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 : "fmt"
11 : "math"
12 : "runtime/pprof"
13 : "slices"
14 : "sort"
15 : "sync/atomic"
16 : "time"
17 :
18 : "github.com/cockroachdb/crlib/crtime"
19 : "github.com/cockroachdb/errors"
20 : "github.com/cockroachdb/pebble/internal/base"
21 : "github.com/cockroachdb/pebble/internal/compact"
22 : "github.com/cockroachdb/pebble/internal/keyspan"
23 : "github.com/cockroachdb/pebble/internal/keyspan/keyspanimpl"
24 : "github.com/cockroachdb/pebble/internal/manifest"
25 : "github.com/cockroachdb/pebble/internal/sstableinternal"
26 : "github.com/cockroachdb/pebble/objstorage"
27 : "github.com/cockroachdb/pebble/objstorage/objstorageprovider/objiotracing"
28 : "github.com/cockroachdb/pebble/objstorage/remote"
29 : "github.com/cockroachdb/pebble/sstable"
30 : "github.com/cockroachdb/pebble/vfs"
31 : )
32 :
33 : var errEmptyTable = errors.New("pebble: empty table")
34 :
35 : // ErrCancelledCompaction is returned if a compaction is cancelled by a
36 : // concurrent excise or ingest-split operation.
37 : var ErrCancelledCompaction = errors.New("pebble: compaction cancelled by a concurrent operation, will retry compaction")
38 :
39 : var compactLabels = pprof.Labels("pebble", "compact")
40 : var flushLabels = pprof.Labels("pebble", "flush")
41 : var gcLabels = pprof.Labels("pebble", "gc")
42 :
43 : // expandedCompactionByteSizeLimit is the maximum number of bytes in all
44 : // compacted files. We avoid expanding the lower level file set of a compaction
45 : // if it would make the total compaction cover more than this many bytes.
46 1 : func expandedCompactionByteSizeLimit(opts *Options, level int, availBytes uint64) uint64 {
47 1 : v := uint64(25 * opts.Level(level).TargetFileSize)
48 1 :
49 1 : // Never expand a compaction beyond half the available capacity, divided
50 1 : // by the maximum number of concurrent compactions. Each of the concurrent
51 1 : // compactions may expand up to this limit, so this attempts to limit
52 1 : // compactions to half of available disk space. Note that this will not
53 1 : // prevent compaction picking from pursuing compactions that are larger
54 1 : // than this threshold before expansion.
55 1 : diskMax := (availBytes / 2) / uint64(opts.MaxConcurrentCompactions())
56 1 : if v > diskMax {
57 1 : v = diskMax
58 1 : }
59 1 : return v
60 : }
61 :
62 : // maxGrandparentOverlapBytes is the maximum bytes of overlap with level+1
63 : // before we stop building a single file in a level-1 to level compaction.
64 1 : func maxGrandparentOverlapBytes(opts *Options, level int) uint64 {
65 1 : return uint64(10 * opts.Level(level).TargetFileSize)
66 1 : }
67 :
68 : // maxReadCompactionBytes is used to prevent read compactions which
69 : // are too wide.
70 1 : func maxReadCompactionBytes(opts *Options, level int) uint64 {
71 1 : return uint64(10 * opts.Level(level).TargetFileSize)
72 1 : }
73 :
74 : // noCloseIter wraps around a FragmentIterator, intercepting and eliding
75 : // calls to Close. It is used during compaction to ensure that rangeDelIters
76 : // are not closed prematurely.
77 : type noCloseIter struct {
78 : keyspan.FragmentIterator
79 : }
80 :
81 1 : func (i *noCloseIter) Close() {}
82 :
83 : type compactionLevel struct {
84 : level int
85 : files manifest.LevelSlice
86 : // l0SublevelInfo contains information about L0 sublevels being compacted.
87 : // It's only set for the start level of a compaction starting out of L0 and
88 : // is nil for all other compactions.
89 : l0SublevelInfo []sublevelInfo
90 : }
91 :
92 1 : func (cl compactionLevel) Clone() compactionLevel {
93 1 : newCL := compactionLevel{
94 1 : level: cl.level,
95 1 : files: cl.files,
96 1 : }
97 1 : return newCL
98 1 : }
99 1 : func (cl compactionLevel) String() string {
100 1 : return fmt.Sprintf(`Level %d, Files %s`, cl.level, cl.files)
101 1 : }
102 :
103 : // compactionWritable is a objstorage.Writable wrapper that, on every write,
104 : // updates a metric in `versions` on bytes written by in-progress compactions so
105 : // far. It also increments a per-compaction `written` int.
106 : type compactionWritable struct {
107 : objstorage.Writable
108 :
109 : versions *versionSet
110 : written *int64
111 : }
112 :
113 : // Write is part of the objstorage.Writable interface.
114 1 : func (c *compactionWritable) Write(p []byte) error {
115 1 : if err := c.Writable.Write(p); err != nil {
116 0 : return err
117 0 : }
118 :
119 1 : *c.written += int64(len(p))
120 1 : c.versions.incrementCompactionBytes(int64(len(p)))
121 1 : return nil
122 : }
123 :
124 : type compactionKind int
125 :
126 : const (
127 : compactionKindDefault compactionKind = iota
128 : compactionKindFlush
129 : // compactionKindMove denotes a move compaction where the input file is
130 : // retained and linked in a new level without being obsoleted.
131 : compactionKindMove
132 : // compactionKindCopy denotes a copy compaction where the input file is
133 : // copied byte-by-byte into a new file with a new FileNum in the output level.
134 : compactionKindCopy
135 : // compactionKindDeleteOnly denotes a compaction that only deletes input
136 : // files. It can occur when wide range tombstones completely contain sstables.
137 : compactionKindDeleteOnly
138 : compactionKindElisionOnly
139 : compactionKindRead
140 : compactionKindTombstoneDensity
141 : compactionKindRewrite
142 : compactionKindIngestedFlushable
143 : )
144 :
145 1 : func (k compactionKind) String() string {
146 1 : switch k {
147 1 : case compactionKindDefault:
148 1 : return "default"
149 0 : case compactionKindFlush:
150 0 : return "flush"
151 1 : case compactionKindMove:
152 1 : return "move"
153 1 : case compactionKindDeleteOnly:
154 1 : return "delete-only"
155 1 : case compactionKindElisionOnly:
156 1 : return "elision-only"
157 1 : case compactionKindRead:
158 1 : return "read"
159 0 : case compactionKindTombstoneDensity:
160 0 : return "tombstone-density"
161 1 : case compactionKindRewrite:
162 1 : return "rewrite"
163 0 : case compactionKindIngestedFlushable:
164 0 : return "ingested-flushable"
165 1 : case compactionKindCopy:
166 1 : return "copy"
167 : }
168 0 : return "?"
169 : }
170 :
171 : // compaction is a table compaction from one level to the next, starting from a
172 : // given version.
173 : type compaction struct {
174 : // cancel is a bool that can be used by other goroutines to signal a compaction
175 : // to cancel, such as if a conflicting excise operation raced it to manifest
176 : // application. Only holders of the manifest lock will write to this atomic.
177 : cancel atomic.Bool
178 :
179 : kind compactionKind
180 : // isDownload is true if this compaction was started as part of a Download
181 : // operation. In this case kind is compactionKindCopy or
182 : // compactionKindRewrite.
183 : isDownload bool
184 :
185 : cmp Compare
186 : equal Equal
187 : comparer *base.Comparer
188 : formatKey base.FormatKey
189 : logger Logger
190 : version *version
191 : stats base.InternalIteratorStats
192 : beganAt time.Time
193 : // versionEditApplied is set to true when a compaction has completed and the
194 : // resulting version has been installed (if successful), but the compaction
195 : // goroutine is still cleaning up (eg, deleting obsolete files).
196 : versionEditApplied bool
197 : bufferPool sstable.BufferPool
198 :
199 : // startLevel is the level that is being compacted. Inputs from startLevel
200 : // and outputLevel will be merged to produce a set of outputLevel files.
201 : startLevel *compactionLevel
202 :
203 : // outputLevel is the level that files are being produced in. outputLevel is
204 : // equal to startLevel+1 except when:
205 : // - if startLevel is 0, the output level equals compactionPicker.baseLevel().
206 : // - in multilevel compaction, the output level is the lowest level involved in
207 : // the compaction
208 : // A compaction's outputLevel is nil for delete-only compactions.
209 : outputLevel *compactionLevel
210 :
211 : // extraLevels point to additional levels in between the input and output
212 : // levels that get compacted in multilevel compactions
213 : extraLevels []*compactionLevel
214 :
215 : inputs []compactionLevel
216 :
217 : // maxOutputFileSize is the maximum size of an individual table created
218 : // during compaction.
219 : maxOutputFileSize uint64
220 : // maxOverlapBytes is the maximum number of bytes of overlap allowed for a
221 : // single output table with the tables in the grandparent level.
222 : maxOverlapBytes uint64
223 :
224 : // flushing contains the flushables (aka memtables) that are being flushed.
225 : flushing flushableList
226 : // bytesWritten contains the number of bytes that have been written to outputs.
227 : bytesWritten int64
228 :
229 : // The boundaries of the input data.
230 : smallest InternalKey
231 : largest InternalKey
232 :
233 : // A list of fragment iterators to close when the compaction finishes. Used by
234 : // input iteration to keep rangeDelIters open for the lifetime of the
235 : // compaction, and only close them when the compaction finishes.
236 : closers []*noCloseIter
237 :
238 : // grandparents are the tables in level+2 that overlap with the files being
239 : // compacted. Used to determine output table boundaries. Do not assume that the actual files
240 : // in the grandparent when this compaction finishes will be the same.
241 : grandparents manifest.LevelSlice
242 :
243 : // Boundaries at which flushes to L0 should be split. Determined by
244 : // L0Sublevels. If nil, flushes aren't split.
245 : l0Limits [][]byte
246 :
247 : delElision compact.TombstoneElision
248 : rangeKeyElision compact.TombstoneElision
249 :
250 : // allowedZeroSeqNum is true if seqnums can be zeroed if there are no
251 : // snapshots requiring them to be kept. This determination is made by
252 : // looking for an sstable which overlaps the bounds of the compaction at a
253 : // lower level in the LSM during runCompaction.
254 : allowedZeroSeqNum bool
255 :
256 : // deletionHints are set if this is a compactionKindDeleteOnly. Used to figure
257 : // out whether an input must be deleted in its entirety, or excised into
258 : // virtual sstables.
259 : deletionHints []deleteCompactionHint
260 :
261 : // exciseEnabled is set to true if this is a compactionKindDeleteOnly and
262 : // this compaction is allowed to excise files.
263 : exciseEnabled bool
264 :
265 : metrics map[int]*LevelMetrics
266 :
267 : pickerMetrics compactionPickerMetrics
268 : }
269 :
270 : // inputLargestSeqNumAbsolute returns the maximum LargestSeqNumAbsolute of any
271 : // input sstables.
272 1 : func (c *compaction) inputLargestSeqNumAbsolute() base.SeqNum {
273 1 : var seqNum base.SeqNum
274 1 : for _, cl := range c.inputs {
275 1 : cl.files.Each(func(m *manifest.FileMetadata) {
276 1 : seqNum = max(seqNum, m.LargestSeqNumAbsolute)
277 1 : })
278 : }
279 1 : return seqNum
280 : }
281 :
282 1 : func (c *compaction) makeInfo(jobID JobID) CompactionInfo {
283 1 : info := CompactionInfo{
284 1 : JobID: int(jobID),
285 1 : Reason: c.kind.String(),
286 1 : Input: make([]LevelInfo, 0, len(c.inputs)),
287 1 : Annotations: []string{},
288 1 : }
289 1 : if c.isDownload {
290 1 : info.Reason = "download," + info.Reason
291 1 : }
292 1 : for _, cl := range c.inputs {
293 1 : inputInfo := LevelInfo{Level: cl.level, Tables: nil}
294 1 : iter := cl.files.Iter()
295 1 : for m := iter.First(); m != nil; m = iter.Next() {
296 1 : inputInfo.Tables = append(inputInfo.Tables, m.TableInfo())
297 1 : }
298 1 : info.Input = append(info.Input, inputInfo)
299 : }
300 1 : if c.outputLevel != nil {
301 1 : info.Output.Level = c.outputLevel.level
302 1 :
303 1 : // If there are no inputs from the output level (eg, a move
304 1 : // compaction), add an empty LevelInfo to info.Input.
305 1 : if len(c.inputs) > 0 && c.inputs[len(c.inputs)-1].level != c.outputLevel.level {
306 0 : info.Input = append(info.Input, LevelInfo{Level: c.outputLevel.level})
307 0 : }
308 1 : } else {
309 1 : // For a delete-only compaction, set the output level to L6. The
310 1 : // output level is not meaningful here, but complicating the
311 1 : // info.Output interface with a pointer doesn't seem worth the
312 1 : // semantic distinction.
313 1 : info.Output.Level = numLevels - 1
314 1 : }
315 :
316 1 : for i, score := range c.pickerMetrics.scores {
317 1 : info.Input[i].Score = score
318 1 : }
319 1 : info.SingleLevelOverlappingRatio = c.pickerMetrics.singleLevelOverlappingRatio
320 1 : info.MultiLevelOverlappingRatio = c.pickerMetrics.multiLevelOverlappingRatio
321 1 : if len(info.Input) > 2 {
322 1 : info.Annotations = append(info.Annotations, "multilevel")
323 1 : }
324 1 : return info
325 : }
326 :
327 1 : func (c *compaction) userKeyBounds() base.UserKeyBounds {
328 1 : return base.UserKeyBoundsFromInternal(c.smallest, c.largest)
329 1 : }
330 :
331 : func newCompaction(
332 : pc *pickedCompaction, opts *Options, beganAt time.Time, provider objstorage.Provider,
333 1 : ) *compaction {
334 1 : c := &compaction{
335 1 : kind: compactionKindDefault,
336 1 : cmp: pc.cmp,
337 1 : equal: opts.Comparer.Equal,
338 1 : comparer: opts.Comparer,
339 1 : formatKey: opts.Comparer.FormatKey,
340 1 : inputs: pc.inputs,
341 1 : smallest: pc.smallest,
342 1 : largest: pc.largest,
343 1 : logger: opts.Logger,
344 1 : version: pc.version,
345 1 : beganAt: beganAt,
346 1 : maxOutputFileSize: pc.maxOutputFileSize,
347 1 : maxOverlapBytes: pc.maxOverlapBytes,
348 1 : pickerMetrics: pc.pickerMetrics,
349 1 : }
350 1 : c.startLevel = &c.inputs[0]
351 1 : if pc.startLevel.l0SublevelInfo != nil {
352 1 : c.startLevel.l0SublevelInfo = pc.startLevel.l0SublevelInfo
353 1 : }
354 1 : c.outputLevel = &c.inputs[1]
355 1 :
356 1 : if len(pc.extraLevels) > 0 {
357 1 : c.extraLevels = pc.extraLevels
358 1 : c.outputLevel = &c.inputs[len(c.inputs)-1]
359 1 : }
360 : // Compute the set of outputLevel+1 files that overlap this compaction (these
361 : // are the grandparent sstables).
362 1 : if c.outputLevel.level+1 < numLevels {
363 1 : c.grandparents = c.version.Overlaps(c.outputLevel.level+1, c.userKeyBounds())
364 1 : }
365 1 : c.delElision, c.rangeKeyElision = compact.SetupTombstoneElision(
366 1 : c.cmp, c.version, c.outputLevel.level, base.UserKeyBoundsFromInternal(c.smallest, c.largest),
367 1 : )
368 1 : c.kind = pc.kind
369 1 :
370 1 : if c.kind == compactionKindDefault && c.outputLevel.files.Empty() && !c.hasExtraLevelData() &&
371 1 : c.startLevel.files.Len() == 1 && c.grandparents.SizeSum() <= c.maxOverlapBytes {
372 1 : // This compaction can be converted into a move or copy from one level
373 1 : // to the next. We avoid such a move if there is lots of overlapping
374 1 : // grandparent data. Otherwise, the move could create a parent file
375 1 : // that will require a very expensive merge later on.
376 1 : iter := c.startLevel.files.Iter()
377 1 : meta := iter.First()
378 1 : isRemote := false
379 1 : // We should always be passed a provider, except in some unit tests.
380 1 : if provider != nil {
381 1 : isRemote = !objstorage.IsLocalTable(provider, meta.FileBacking.DiskFileNum)
382 1 : }
383 : // Avoid a trivial move or copy if all of these are true, as rewriting a
384 : // new file is better:
385 : //
386 : // 1) The source file is a virtual sstable
387 : // 2) The existing file `meta` is on non-remote storage
388 : // 3) The output level prefers shared storage
389 1 : mustCopy := !isRemote && remote.ShouldCreateShared(opts.Experimental.CreateOnShared, c.outputLevel.level)
390 1 : if mustCopy {
391 1 : // If the source is virtual, it's best to just rewrite the file as all
392 1 : // conditions in the above comment are met.
393 1 : if !meta.Virtual {
394 1 : c.kind = compactionKindCopy
395 1 : }
396 1 : } else {
397 1 : c.kind = compactionKindMove
398 1 : }
399 : }
400 1 : return c
401 : }
402 :
403 : func newDeleteOnlyCompaction(
404 : opts *Options,
405 : cur *version,
406 : inputs []compactionLevel,
407 : beganAt time.Time,
408 : hints []deleteCompactionHint,
409 : exciseEnabled bool,
410 1 : ) *compaction {
411 1 : c := &compaction{
412 1 : kind: compactionKindDeleteOnly,
413 1 : cmp: opts.Comparer.Compare,
414 1 : equal: opts.Comparer.Equal,
415 1 : comparer: opts.Comparer,
416 1 : formatKey: opts.Comparer.FormatKey,
417 1 : logger: opts.Logger,
418 1 : version: cur,
419 1 : beganAt: beganAt,
420 1 : inputs: inputs,
421 1 : deletionHints: hints,
422 1 : exciseEnabled: exciseEnabled,
423 1 : }
424 1 :
425 1 : // Set c.smallest, c.largest.
426 1 : files := make([]manifest.LevelIterator, 0, len(inputs))
427 1 : for _, in := range inputs {
428 1 : files = append(files, in.files.Iter())
429 1 : }
430 1 : c.smallest, c.largest = manifest.KeyRange(opts.Comparer.Compare, files...)
431 1 : return c
432 : }
433 :
434 1 : func adjustGrandparentOverlapBytesForFlush(c *compaction, flushingBytes uint64) {
435 1 : // Heuristic to place a lower bound on compaction output file size
436 1 : // caused by Lbase. Prior to this heuristic we have observed an L0 in
437 1 : // production with 310K files of which 290K files were < 10KB in size.
438 1 : // Our hypothesis is that it was caused by L1 having 2600 files and
439 1 : // ~10GB, such that each flush got split into many tiny files due to
440 1 : // overlapping with most of the files in Lbase.
441 1 : //
442 1 : // The computation below is general in that it accounts
443 1 : // for flushing different volumes of data (e.g. we may be flushing
444 1 : // many memtables). For illustration, we consider the typical
445 1 : // example of flushing a 64MB memtable. So 12.8MB output,
446 1 : // based on the compression guess below. If the compressed bytes
447 1 : // guess is an over-estimate we will end up with smaller files,
448 1 : // and if an under-estimate we will end up with larger files.
449 1 : // With a 2MB target file size, 7 files. We are willing to accept
450 1 : // 4x the number of files, if it results in better write amplification
451 1 : // when later compacting to Lbase, i.e., ~450KB files (target file
452 1 : // size / 4).
453 1 : //
454 1 : // Note that this is a pessimistic heuristic in that
455 1 : // fileCountUpperBoundDueToGrandparents could be far from the actual
456 1 : // number of files produced due to the grandparent limits. For
457 1 : // example, in the extreme, consider a flush that overlaps with 1000
458 1 : // files in Lbase f0...f999, and the initially calculated value of
459 1 : // maxOverlapBytes will cause splits at f10, f20,..., f990, which
460 1 : // means an upper bound file count of 100 files. Say the input bytes
461 1 : // in the flush are such that acceptableFileCount=10. We will fatten
462 1 : // up maxOverlapBytes by 10x to ensure that the upper bound file count
463 1 : // drops to 10. However, it is possible that in practice, even without
464 1 : // this change, we would have produced no more than 10 files, and that
465 1 : // this change makes the files unnecessarily wide. Say the input bytes
466 1 : // are distributed such that 10% are in f0...f9, 10% in f10...f19, ...
467 1 : // 10% in f80...f89 and 10% in f990...f999. The original value of
468 1 : // maxOverlapBytes would have actually produced only 10 sstables. But
469 1 : // by increasing maxOverlapBytes by 10x, we may produce 1 sstable that
470 1 : // spans f0...f89, i.e., a much wider sstable than necessary.
471 1 : //
472 1 : // We could produce a tighter estimate of
473 1 : // fileCountUpperBoundDueToGrandparents if we had knowledge of the key
474 1 : // distribution of the flush. The 4x multiplier mentioned earlier is
475 1 : // a way to try to compensate for this pessimism.
476 1 : //
477 1 : // TODO(sumeer): we don't have compression info for the data being
478 1 : // flushed, but it is likely that existing files that overlap with
479 1 : // this flush in Lbase are representative wrt compression ratio. We
480 1 : // could store the uncompressed size in FileMetadata and estimate
481 1 : // the compression ratio.
482 1 : const approxCompressionRatio = 0.2
483 1 : approxOutputBytes := approxCompressionRatio * float64(flushingBytes)
484 1 : approxNumFilesBasedOnTargetSize :=
485 1 : int(math.Ceil(approxOutputBytes / float64(c.maxOutputFileSize)))
486 1 : acceptableFileCount := float64(4 * approxNumFilesBasedOnTargetSize)
487 1 : // The byte calculation is linear in numGrandparentFiles, but we will
488 1 : // incur this linear cost in compact.Runner.TableSplitLimit() too, so we are
489 1 : // also willing to pay it now. We could approximate this cheaply by using the
490 1 : // mean file size of Lbase.
491 1 : grandparentFileBytes := c.grandparents.SizeSum()
492 1 : fileCountUpperBoundDueToGrandparents :=
493 1 : float64(grandparentFileBytes) / float64(c.maxOverlapBytes)
494 1 : if fileCountUpperBoundDueToGrandparents > acceptableFileCount {
495 1 : c.maxOverlapBytes = uint64(
496 1 : float64(c.maxOverlapBytes) *
497 1 : (fileCountUpperBoundDueToGrandparents / acceptableFileCount))
498 1 : }
499 : }
500 :
501 : func newFlush(
502 : opts *Options, cur *version, baseLevel int, flushing flushableList, beganAt time.Time,
503 1 : ) (*compaction, error) {
504 1 : c := &compaction{
505 1 : kind: compactionKindFlush,
506 1 : cmp: opts.Comparer.Compare,
507 1 : equal: opts.Comparer.Equal,
508 1 : comparer: opts.Comparer,
509 1 : formatKey: opts.Comparer.FormatKey,
510 1 : logger: opts.Logger,
511 1 : version: cur,
512 1 : beganAt: beganAt,
513 1 : inputs: []compactionLevel{{level: -1}, {level: 0}},
514 1 : maxOutputFileSize: math.MaxUint64,
515 1 : maxOverlapBytes: math.MaxUint64,
516 1 : flushing: flushing,
517 1 : }
518 1 : c.startLevel = &c.inputs[0]
519 1 : c.outputLevel = &c.inputs[1]
520 1 :
521 1 : if len(flushing) > 0 {
522 1 : if _, ok := flushing[0].flushable.(*ingestedFlushable); ok {
523 1 : if len(flushing) != 1 {
524 0 : panic("pebble: ingestedFlushable must be flushed one at a time.")
525 : }
526 1 : c.kind = compactionKindIngestedFlushable
527 1 : return c, nil
528 : }
529 : }
530 :
531 : // Make sure there's no ingestedFlushable after the first flushable in the
532 : // list.
533 1 : for _, f := range flushing {
534 1 : if _, ok := f.flushable.(*ingestedFlushable); ok {
535 0 : panic("pebble: flushing shouldn't contain ingestedFlushable flushable")
536 : }
537 : }
538 :
539 1 : if cur.L0Sublevels != nil {
540 1 : c.l0Limits = cur.L0Sublevels.FlushSplitKeys()
541 1 : }
542 :
543 1 : smallestSet, largestSet := false, false
544 1 : updatePointBounds := func(iter internalIterator) {
545 1 : if kv := iter.First(); kv != nil {
546 1 : if !smallestSet ||
547 1 : base.InternalCompare(c.cmp, c.smallest, kv.K) > 0 {
548 1 : smallestSet = true
549 1 : c.smallest = kv.K.Clone()
550 1 : }
551 : }
552 1 : if kv := iter.Last(); kv != nil {
553 1 : if !largestSet ||
554 1 : base.InternalCompare(c.cmp, c.largest, kv.K) < 0 {
555 1 : largestSet = true
556 1 : c.largest = kv.K.Clone()
557 1 : }
558 : }
559 : }
560 :
561 1 : updateRangeBounds := func(iter keyspan.FragmentIterator) error {
562 1 : // File bounds require s != nil && !s.Empty(). We only need to check for
563 1 : // s != nil here, as the memtable's FragmentIterator would never surface
564 1 : // empty spans.
565 1 : if s, err := iter.First(); err != nil {
566 0 : return err
567 1 : } else if s != nil {
568 1 : if key := s.SmallestKey(); !smallestSet ||
569 1 : base.InternalCompare(c.cmp, c.smallest, key) > 0 {
570 1 : smallestSet = true
571 1 : c.smallest = key.Clone()
572 1 : }
573 : }
574 1 : if s, err := iter.Last(); err != nil {
575 0 : return err
576 1 : } else if s != nil {
577 1 : if key := s.LargestKey(); !largestSet ||
578 1 : base.InternalCompare(c.cmp, c.largest, key) < 0 {
579 1 : largestSet = true
580 1 : c.largest = key.Clone()
581 1 : }
582 : }
583 1 : return nil
584 : }
585 :
586 1 : var flushingBytes uint64
587 1 : for i := range flushing {
588 1 : f := flushing[i]
589 1 : updatePointBounds(f.newIter(nil))
590 1 : if rangeDelIter := f.newRangeDelIter(nil); rangeDelIter != nil {
591 1 : if err := updateRangeBounds(rangeDelIter); err != nil {
592 0 : return nil, err
593 0 : }
594 : }
595 1 : if rangeKeyIter := f.newRangeKeyIter(nil); rangeKeyIter != nil {
596 1 : if err := updateRangeBounds(rangeKeyIter); err != nil {
597 0 : return nil, err
598 0 : }
599 : }
600 1 : flushingBytes += f.inuseBytes()
601 : }
602 :
603 1 : if opts.FlushSplitBytes > 0 {
604 1 : c.maxOutputFileSize = uint64(opts.Level(0).TargetFileSize)
605 1 : c.maxOverlapBytes = maxGrandparentOverlapBytes(opts, 0)
606 1 : c.grandparents = c.version.Overlaps(baseLevel, c.userKeyBounds())
607 1 : adjustGrandparentOverlapBytesForFlush(c, flushingBytes)
608 1 : }
609 :
610 : // We don't elide tombstones for flushes.
611 1 : c.delElision, c.rangeKeyElision = compact.NoTombstoneElision(), compact.NoTombstoneElision()
612 1 : return c, nil
613 : }
614 :
615 1 : func (c *compaction) hasExtraLevelData() bool {
616 1 : if len(c.extraLevels) == 0 {
617 1 : // not a multi level compaction
618 1 : return false
619 1 : } else if c.extraLevels[0].files.Empty() {
620 1 : // a multi level compaction without data in the intermediate input level;
621 1 : // e.g. for a multi level compaction with levels 4,5, and 6, this could
622 1 : // occur if there is no files to compact in 5, or in 5 and 6 (i.e. a move).
623 1 : return false
624 1 : }
625 1 : return true
626 : }
627 :
628 : // errorOnUserKeyOverlap returns an error if the last two written sstables in
629 : // this compaction have revisions of the same user key present in both sstables,
630 : // when it shouldn't (eg. when splitting flushes).
631 1 : func (c *compaction) errorOnUserKeyOverlap(ve *versionEdit) error {
632 1 : if n := len(ve.NewFiles); n > 1 {
633 1 : meta := ve.NewFiles[n-1].Meta
634 1 : prevMeta := ve.NewFiles[n-2].Meta
635 1 : if !prevMeta.Largest.IsExclusiveSentinel() &&
636 1 : c.cmp(prevMeta.Largest.UserKey, meta.Smallest.UserKey) >= 0 {
637 1 : return errors.Errorf("pebble: compaction split user key across two sstables: %s in %s and %s",
638 1 : prevMeta.Largest.Pretty(c.formatKey),
639 1 : prevMeta.FileNum,
640 1 : meta.FileNum)
641 1 : }
642 : }
643 1 : return nil
644 : }
645 :
646 : // allowZeroSeqNum returns true if seqnum's can be zeroed if there are no
647 : // snapshots requiring them to be kept. It performs this determination by
648 : // looking at the TombstoneElision values which are set up based on sstables
649 : // which overlap the bounds of the compaction at a lower level in the LSM.
650 1 : func (c *compaction) allowZeroSeqNum() bool {
651 1 : // TODO(peter): we disable zeroing of seqnums during flushing to match
652 1 : // RocksDB behavior and to avoid generating overlapping sstables during
653 1 : // DB.replayWAL. When replaying WAL files at startup, we flush after each
654 1 : // WAL is replayed building up a single version edit that is
655 1 : // applied. Because we don't apply the version edit after each flush, this
656 1 : // code doesn't know that L0 contains files and zeroing of seqnums should
657 1 : // be disabled. That is fixable, but it seems safer to just match the
658 1 : // RocksDB behavior for now.
659 1 : return len(c.flushing) == 0 && c.delElision.ElidesEverything() && c.rangeKeyElision.ElidesEverything()
660 1 : }
661 :
662 : // newInputIters returns an iterator over all the input tables in a compaction.
663 : func (c *compaction) newInputIters(
664 : newIters tableNewIters, newRangeKeyIter keyspanimpl.TableNewSpanIter,
665 : ) (
666 : pointIter internalIterator,
667 : rangeDelIter, rangeKeyIter keyspan.FragmentIterator,
668 : retErr error,
669 1 : ) {
670 1 : // Validate the ordering of compaction input files for defense in depth.
671 1 : if len(c.flushing) == 0 {
672 1 : if c.startLevel.level >= 0 {
673 1 : err := manifest.CheckOrdering(c.cmp, c.formatKey,
674 1 : manifest.Level(c.startLevel.level), c.startLevel.files.Iter())
675 1 : if err != nil {
676 1 : return nil, nil, nil, err
677 1 : }
678 : }
679 1 : err := manifest.CheckOrdering(c.cmp, c.formatKey,
680 1 : manifest.Level(c.outputLevel.level), c.outputLevel.files.Iter())
681 1 : if err != nil {
682 1 : return nil, nil, nil, err
683 1 : }
684 1 : if c.startLevel.level == 0 {
685 1 : if c.startLevel.l0SublevelInfo == nil {
686 0 : panic("l0SublevelInfo not created for compaction out of L0")
687 : }
688 1 : for _, info := range c.startLevel.l0SublevelInfo {
689 1 : err := manifest.CheckOrdering(c.cmp, c.formatKey,
690 1 : info.sublevel, info.Iter())
691 1 : if err != nil {
692 1 : return nil, nil, nil, err
693 1 : }
694 : }
695 : }
696 1 : if len(c.extraLevels) > 0 {
697 1 : if len(c.extraLevels) > 1 {
698 0 : panic("n>2 multi level compaction not implemented yet")
699 : }
700 1 : interLevel := c.extraLevels[0]
701 1 : err := manifest.CheckOrdering(c.cmp, c.formatKey,
702 1 : manifest.Level(interLevel.level), interLevel.files.Iter())
703 1 : if err != nil {
704 0 : return nil, nil, nil, err
705 0 : }
706 : }
707 : }
708 :
709 : // There are three classes of keys that a compaction needs to process: point
710 : // keys, range deletion tombstones and range keys. Collect all iterators for
711 : // all these classes of keys from all the levels. We'll aggregate them
712 : // together farther below.
713 : //
714 : // numInputLevels is an approximation of the number of iterator levels. Due
715 : // to idiosyncrasies in iterator construction, we may (rarely) exceed this
716 : // initial capacity.
717 1 : numInputLevels := max(len(c.flushing), len(c.inputs))
718 1 : iters := make([]internalIterator, 0, numInputLevels)
719 1 : rangeDelIters := make([]keyspan.FragmentIterator, 0, numInputLevels)
720 1 : rangeKeyIters := make([]keyspan.FragmentIterator, 0, numInputLevels)
721 1 :
722 1 : // If construction of the iterator inputs fails, ensure that we close all
723 1 : // the consitutent iterators.
724 1 : defer func() {
725 1 : if retErr != nil {
726 0 : for _, iter := range iters {
727 0 : if iter != nil {
728 0 : iter.Close()
729 0 : }
730 : }
731 0 : for _, rangeDelIter := range rangeDelIters {
732 0 : rangeDelIter.Close()
733 0 : }
734 : }
735 : }()
736 1 : iterOpts := IterOptions{
737 1 : Category: categoryCompaction,
738 1 : logger: c.logger,
739 1 : }
740 1 :
741 1 : // Populate iters, rangeDelIters and rangeKeyIters with the appropriate
742 1 : // constituent iterators. This depends on whether this is a flush or a
743 1 : // compaction.
744 1 : if len(c.flushing) != 0 {
745 1 : // If flushing, we need to build the input iterators over the memtables
746 1 : // stored in c.flushing.
747 1 : for i := range c.flushing {
748 1 : f := c.flushing[i]
749 1 : iters = append(iters, f.newFlushIter(nil))
750 1 : rangeDelIter := f.newRangeDelIter(nil)
751 1 : if rangeDelIter != nil {
752 1 : rangeDelIters = append(rangeDelIters, rangeDelIter)
753 1 : }
754 1 : if rangeKeyIter := f.newRangeKeyIter(nil); rangeKeyIter != nil {
755 1 : rangeKeyIters = append(rangeKeyIters, rangeKeyIter)
756 1 : }
757 : }
758 1 : } else {
759 1 : addItersForLevel := func(level *compactionLevel, l manifest.Layer) error {
760 1 : // Add a *levelIter for point iterators. Because we don't call
761 1 : // initRangeDel, the levelIter will close and forget the range
762 1 : // deletion iterator when it steps on to a new file. Surfacing range
763 1 : // deletions to compactions are handled below.
764 1 : iters = append(iters, newLevelIter(context.Background(),
765 1 : iterOpts, c.comparer, newIters, level.files.Iter(), l, internalIterOpts{
766 1 : compaction: true,
767 1 : bufferPool: &c.bufferPool,
768 1 : }))
769 1 : // TODO(jackson): Use keyspanimpl.LevelIter to avoid loading all the range
770 1 : // deletions into memory upfront. (See #2015, which reverted this.) There
771 1 : // will be no user keys that are split between sstables within a level in
772 1 : // Cockroach 23.1, which unblocks this optimization.
773 1 :
774 1 : // Add the range deletion iterator for each file as an independent level
775 1 : // in mergingIter, as opposed to making a levelIter out of those. This
776 1 : // is safer as levelIter expects all keys coming from underlying
777 1 : // iterators to be in order. Due to compaction / tombstone writing
778 1 : // logic in finishOutput(), it is possible for range tombstones to not
779 1 : // be strictly ordered across all files in one level.
780 1 : //
781 1 : // Consider this example from the metamorphic tests (also repeated in
782 1 : // finishOutput()), consisting of three L3 files with their bounds
783 1 : // specified in square brackets next to the file name:
784 1 : //
785 1 : // ./000240.sst [tmgc#391,MERGE-tmgc#391,MERGE]
786 1 : // tmgc#391,MERGE [786e627a]
787 1 : // tmgc-udkatvs#331,RANGEDEL
788 1 : //
789 1 : // ./000241.sst [tmgc#384,MERGE-tmgc#384,MERGE]
790 1 : // tmgc#384,MERGE [666c7070]
791 1 : // tmgc-tvsalezade#383,RANGEDEL
792 1 : // tmgc-tvsalezade#331,RANGEDEL
793 1 : //
794 1 : // ./000242.sst [tmgc#383,RANGEDEL-tvsalezade#72057594037927935,RANGEDEL]
795 1 : // tmgc-tvsalezade#383,RANGEDEL
796 1 : // tmgc#375,SET [72646c78766965616c72776865676e79]
797 1 : // tmgc-tvsalezade#356,RANGEDEL
798 1 : //
799 1 : // Here, the range tombstone in 000240.sst falls "after" one in
800 1 : // 000241.sst, despite 000240.sst being ordered "before" 000241.sst for
801 1 : // levelIter's purposes. While each file is still consistent before its
802 1 : // bounds, it's safer to have all rangedel iterators be visible to
803 1 : // mergingIter.
804 1 : iter := level.files.Iter()
805 1 : for f := iter.First(); f != nil; f = iter.Next() {
806 1 : rangeDelIter, err := c.newRangeDelIter(newIters, iter.Take(), iterOpts, l)
807 1 : if err != nil {
808 0 : // The error will already be annotated with the BackingFileNum, so
809 0 : // we annotate it with the FileNum.
810 0 : return errors.Wrapf(err, "pebble: could not open table %s", errors.Safe(f.FileNum))
811 0 : }
812 1 : if rangeDelIter == nil {
813 1 : continue
814 : }
815 1 : rangeDelIters = append(rangeDelIters, rangeDelIter)
816 1 : c.closers = append(c.closers, rangeDelIter)
817 : }
818 :
819 : // Check if this level has any range keys.
820 1 : hasRangeKeys := false
821 1 : for f := iter.First(); f != nil; f = iter.Next() {
822 1 : if f.HasRangeKeys {
823 1 : hasRangeKeys = true
824 1 : break
825 : }
826 : }
827 1 : if hasRangeKeys {
828 1 : newRangeKeyIterWrapper := func(ctx context.Context, file *manifest.FileMetadata, iterOptions keyspan.SpanIterOptions) (keyspan.FragmentIterator, error) {
829 1 : rangeKeyIter, err := newRangeKeyIter(ctx, file, iterOptions)
830 1 : if err != nil {
831 0 : return nil, err
832 1 : } else if rangeKeyIter == nil {
833 0 : return emptyKeyspanIter, nil
834 0 : }
835 : // Ensure that the range key iter is not closed until the compaction is
836 : // finished. This is necessary because range key processing
837 : // requires the range keys to be held in memory for up to the
838 : // lifetime of the compaction.
839 1 : noCloseIter := &noCloseIter{rangeKeyIter}
840 1 : c.closers = append(c.closers, noCloseIter)
841 1 :
842 1 : // We do not need to truncate range keys to sstable boundaries, or
843 1 : // only read within the file's atomic compaction units, unlike with
844 1 : // range tombstones. This is because range keys were added after we
845 1 : // stopped splitting user keys across sstables, so all the range keys
846 1 : // in this sstable must wholly lie within the file's bounds.
847 1 : return noCloseIter, err
848 : }
849 1 : li := keyspanimpl.NewLevelIter(
850 1 : context.Background(), keyspan.SpanIterOptions{}, c.cmp,
851 1 : newRangeKeyIterWrapper, level.files.Iter(), l, manifest.KeyTypeRange,
852 1 : )
853 1 : rangeKeyIters = append(rangeKeyIters, li)
854 : }
855 1 : return nil
856 : }
857 :
858 1 : for i := range c.inputs {
859 1 : // If the level is annotated with l0SublevelInfo, expand it into one
860 1 : // level per sublevel.
861 1 : // TODO(jackson): Perform this expansion even earlier when we pick the
862 1 : // compaction?
863 1 : if len(c.inputs[i].l0SublevelInfo) > 0 {
864 1 : for _, info := range c.startLevel.l0SublevelInfo {
865 1 : sublevelCompactionLevel := &compactionLevel{0, info.LevelSlice, nil}
866 1 : if err := addItersForLevel(sublevelCompactionLevel, info.sublevel); err != nil {
867 0 : return nil, nil, nil, err
868 0 : }
869 : }
870 1 : continue
871 : }
872 1 : if err := addItersForLevel(&c.inputs[i], manifest.Level(c.inputs[i].level)); err != nil {
873 0 : return nil, nil, nil, err
874 0 : }
875 : }
876 : }
877 :
878 : // If there's only one constituent point iterator, we can avoid the overhead
879 : // of a *mergingIter. This is possible, for example, when performing a flush
880 : // of a single memtable. Otherwise, combine all the iterators into a merging
881 : // iter.
882 1 : pointIter = iters[0]
883 1 : if len(iters) > 1 {
884 1 : pointIter = newMergingIter(c.logger, &c.stats, c.cmp, nil, iters...)
885 1 : }
886 :
887 : // In normal operation, levelIter iterates over the point operations in a
888 : // level, and initializes a rangeDelIter pointer for the range deletions in
889 : // each table. During compaction, we want to iterate over the merged view of
890 : // point operations and range deletions. In order to do this we create one
891 : // levelIter per level to iterate over the point operations, and collect up
892 : // all the range deletion files.
893 : //
894 : // The range deletion levels are combined with a keyspanimpl.MergingIter. The
895 : // resulting merged rangedel iterator is then included using an
896 : // InterleavingIter.
897 : // TODO(jackson): Consider using a defragmenting iterator to stitch together
898 : // logical range deletions that were fragmented due to previous file
899 : // boundaries.
900 1 : if len(rangeDelIters) > 0 {
901 1 : mi := &keyspanimpl.MergingIter{}
902 1 : mi.Init(c.comparer, keyspan.NoopTransform, new(keyspanimpl.MergingBuffers), rangeDelIters...)
903 1 : rangeDelIter = mi
904 1 : }
905 :
906 : // If there are range key iterators, we need to combine them using
907 : // keyspanimpl.MergingIter, and then interleave them among the points.
908 1 : if len(rangeKeyIters) > 0 {
909 1 : mi := &keyspanimpl.MergingIter{}
910 1 : mi.Init(c.comparer, keyspan.NoopTransform, new(keyspanimpl.MergingBuffers), rangeKeyIters...)
911 1 : // TODO(radu): why do we have a defragmenter here but not above?
912 1 : di := &keyspan.DefragmentingIter{}
913 1 : di.Init(c.comparer, mi, keyspan.DefragmentInternal, keyspan.StaticDefragmentReducer, new(keyspan.DefragmentingBuffers))
914 1 : rangeKeyIter = di
915 1 : }
916 1 : return pointIter, rangeDelIter, rangeKeyIter, nil
917 : }
918 :
919 : func (c *compaction) newRangeDelIter(
920 : newIters tableNewIters, f manifest.LevelFile, opts IterOptions, l manifest.Layer,
921 1 : ) (*noCloseIter, error) {
922 1 : opts.layer = l
923 1 : iterSet, err := newIters(context.Background(), f.FileMetadata, &opts,
924 1 : internalIterOpts{
925 1 : compaction: true,
926 1 : bufferPool: &c.bufferPool,
927 1 : }, iterRangeDeletions)
928 1 : if err != nil {
929 0 : return nil, err
930 1 : } else if iterSet.rangeDeletion == nil {
931 1 : // The file doesn't contain any range deletions.
932 1 : return nil, nil
933 1 : }
934 : // Ensure that rangeDelIter is not closed until the compaction is
935 : // finished. This is necessary because range tombstone processing
936 : // requires the range tombstones to be held in memory for up to the
937 : // lifetime of the compaction.
938 1 : return &noCloseIter{iterSet.rangeDeletion}, nil
939 : }
940 :
941 1 : func (c *compaction) String() string {
942 1 : if len(c.flushing) != 0 {
943 0 : return "flush\n"
944 0 : }
945 :
946 1 : var buf bytes.Buffer
947 1 : for level := c.startLevel.level; level <= c.outputLevel.level; level++ {
948 1 : i := level - c.startLevel.level
949 1 : fmt.Fprintf(&buf, "%d:", level)
950 1 : iter := c.inputs[i].files.Iter()
951 1 : for f := iter.First(); f != nil; f = iter.Next() {
952 1 : fmt.Fprintf(&buf, " %s:%s-%s", f.FileNum, f.Smallest, f.Largest)
953 1 : }
954 1 : fmt.Fprintf(&buf, "\n")
955 : }
956 1 : return buf.String()
957 : }
958 :
959 : type manualCompaction struct {
960 : // Count of the retries either due to too many concurrent compactions, or a
961 : // concurrent compaction to overlapping levels.
962 : retries int
963 : level int
964 : outputLevel int
965 : done chan error
966 : start []byte
967 : end []byte
968 : split bool
969 : }
970 :
971 : type readCompaction struct {
972 : level int
973 : // [start, end] key ranges are used for de-duping.
974 : start []byte
975 : end []byte
976 :
977 : // The file associated with the compaction.
978 : // If the file no longer belongs in the same
979 : // level, then we skip the compaction.
980 : fileNum base.FileNum
981 : }
982 :
983 1 : func (d *DB) addInProgressCompaction(c *compaction) {
984 1 : d.mu.compact.inProgress[c] = struct{}{}
985 1 : var isBase, isIntraL0 bool
986 1 : for _, cl := range c.inputs {
987 1 : iter := cl.files.Iter()
988 1 : for f := iter.First(); f != nil; f = iter.Next() {
989 1 : if f.IsCompacting() {
990 0 : d.opts.Logger.Fatalf("L%d->L%d: %s already being compacted", c.startLevel.level, c.outputLevel.level, f.FileNum)
991 0 : }
992 1 : f.SetCompactionState(manifest.CompactionStateCompacting)
993 1 : if c.startLevel != nil && c.outputLevel != nil && c.startLevel.level == 0 {
994 1 : if c.outputLevel.level == 0 {
995 1 : f.IsIntraL0Compacting = true
996 1 : isIntraL0 = true
997 1 : } else {
998 1 : isBase = true
999 1 : }
1000 : }
1001 : }
1002 : }
1003 :
1004 1 : if (isIntraL0 || isBase) && c.version.L0Sublevels != nil {
1005 1 : l0Inputs := []manifest.LevelSlice{c.startLevel.files}
1006 1 : if isIntraL0 {
1007 1 : l0Inputs = append(l0Inputs, c.outputLevel.files)
1008 1 : }
1009 1 : if err := c.version.L0Sublevels.UpdateStateForStartedCompaction(l0Inputs, isBase); err != nil {
1010 0 : d.opts.Logger.Fatalf("could not update state for compaction: %s", err)
1011 0 : }
1012 : }
1013 : }
1014 :
1015 : // Removes compaction markers from files in a compaction. The rollback parameter
1016 : // indicates whether the compaction state should be rolled back to its original
1017 : // state in the case of an unsuccessful compaction.
1018 : //
1019 : // DB.mu must be held when calling this method, however this method can drop and
1020 : // re-acquire that mutex. All writes to the manifest for this compaction should
1021 : // have completed by this point.
1022 1 : func (d *DB) clearCompactingState(c *compaction, rollback bool) {
1023 1 : c.versionEditApplied = true
1024 1 : for _, cl := range c.inputs {
1025 1 : iter := cl.files.Iter()
1026 1 : for f := iter.First(); f != nil; f = iter.Next() {
1027 1 : if !f.IsCompacting() {
1028 0 : d.opts.Logger.Fatalf("L%d->L%d: %s not being compacted", c.startLevel.level, c.outputLevel.level, f.FileNum)
1029 0 : }
1030 1 : if !rollback {
1031 1 : // On success all compactions other than move and delete-only compactions
1032 1 : // transition the file into the Compacted state. Move-compacted files
1033 1 : // become eligible for compaction again and transition back to NotCompacting.
1034 1 : // Delete-only compactions could, on rare occasion, leave files untouched
1035 1 : // (eg. if files have a loose bound), so we revert them all to NotCompacting
1036 1 : // just in case they need to be compacted again.
1037 1 : if c.kind != compactionKindMove && c.kind != compactionKindDeleteOnly {
1038 1 : f.SetCompactionState(manifest.CompactionStateCompacted)
1039 1 : } else {
1040 1 : f.SetCompactionState(manifest.CompactionStateNotCompacting)
1041 1 : }
1042 1 : } else {
1043 1 : // Else, on rollback, all input files unconditionally transition back to
1044 1 : // NotCompacting.
1045 1 : f.SetCompactionState(manifest.CompactionStateNotCompacting)
1046 1 : }
1047 1 : f.IsIntraL0Compacting = false
1048 : }
1049 : }
1050 1 : l0InProgress := inProgressL0Compactions(d.getInProgressCompactionInfoLocked(c))
1051 1 : func() {
1052 1 : // InitCompactingFileInfo requires that no other manifest writes be
1053 1 : // happening in parallel with it, i.e. we're not in the midst of installing
1054 1 : // another version. Otherwise, it's possible that we've created another
1055 1 : // L0Sublevels instance, but not added it to the versions list, causing
1056 1 : // all the indices in FileMetadata to be inaccurate. To ensure this,
1057 1 : // grab the manifest lock.
1058 1 : d.mu.versions.logLock()
1059 1 : defer d.mu.versions.logUnlock()
1060 1 : d.mu.versions.currentVersion().L0Sublevels.InitCompactingFileInfo(l0InProgress)
1061 1 : }()
1062 : }
1063 :
1064 1 : func (d *DB) calculateDiskAvailableBytes() uint64 {
1065 1 : if space, err := d.opts.FS.GetDiskUsage(d.dirname); err == nil {
1066 1 : d.diskAvailBytes.Store(space.AvailBytes)
1067 1 : return space.AvailBytes
1068 1 : } else if !errors.Is(err, vfs.ErrUnsupported) {
1069 1 : d.opts.EventListener.BackgroundError(err)
1070 1 : }
1071 1 : return d.diskAvailBytes.Load()
1072 : }
1073 :
1074 : // maybeScheduleFlush schedules a flush if necessary.
1075 : //
1076 : // d.mu must be held when calling this.
1077 1 : func (d *DB) maybeScheduleFlush() {
1078 1 : if d.mu.compact.flushing || d.closed.Load() != nil || d.opts.ReadOnly {
1079 1 : return
1080 1 : }
1081 1 : if len(d.mu.mem.queue) <= 1 {
1082 1 : return
1083 1 : }
1084 :
1085 1 : if !d.passedFlushThreshold() {
1086 1 : return
1087 1 : }
1088 :
1089 1 : d.mu.compact.flushing = true
1090 1 : go d.flush()
1091 : }
1092 :
1093 1 : func (d *DB) passedFlushThreshold() bool {
1094 1 : var n int
1095 1 : var size uint64
1096 1 : for ; n < len(d.mu.mem.queue)-1; n++ {
1097 1 : if !d.mu.mem.queue[n].readyForFlush() {
1098 1 : break
1099 : }
1100 1 : if d.mu.mem.queue[n].flushForced {
1101 1 : // A flush was forced. Pretend the memtable size is the configured
1102 1 : // size. See minFlushSize below.
1103 1 : size += d.opts.MemTableSize
1104 1 : } else {
1105 1 : size += d.mu.mem.queue[n].totalBytes()
1106 1 : }
1107 : }
1108 1 : if n == 0 {
1109 1 : // None of the immutable memtables are ready for flushing.
1110 1 : return false
1111 1 : }
1112 :
1113 : // Only flush once the sum of the queued memtable sizes exceeds half the
1114 : // configured memtable size. This prevents flushing of memtables at startup
1115 : // while we're undergoing the ramp period on the memtable size. See
1116 : // DB.newMemTable().
1117 1 : minFlushSize := d.opts.MemTableSize / 2
1118 1 : return size >= minFlushSize
1119 : }
1120 :
1121 1 : func (d *DB) maybeScheduleDelayedFlush(tbl *memTable, dur time.Duration) {
1122 1 : var mem *flushableEntry
1123 1 : for _, m := range d.mu.mem.queue {
1124 1 : if m.flushable == tbl {
1125 1 : mem = m
1126 1 : break
1127 : }
1128 : }
1129 1 : if mem == nil || mem.flushForced {
1130 1 : return
1131 1 : }
1132 1 : deadline := d.timeNow().Add(dur)
1133 1 : if !mem.delayedFlushForcedAt.IsZero() && deadline.After(mem.delayedFlushForcedAt) {
1134 1 : // Already scheduled to flush sooner than within `dur`.
1135 1 : return
1136 1 : }
1137 1 : mem.delayedFlushForcedAt = deadline
1138 1 : go func() {
1139 1 : timer := time.NewTimer(dur)
1140 1 : defer timer.Stop()
1141 1 :
1142 1 : select {
1143 1 : case <-d.closedCh:
1144 1 : return
1145 1 : case <-mem.flushed:
1146 1 : return
1147 1 : case <-timer.C:
1148 1 : d.commit.mu.Lock()
1149 1 : defer d.commit.mu.Unlock()
1150 1 : d.mu.Lock()
1151 1 : defer d.mu.Unlock()
1152 1 :
1153 1 : // NB: The timer may fire concurrently with a call to Close. If a
1154 1 : // Close call beat us to acquiring d.mu, d.closed holds ErrClosed,
1155 1 : // and it's too late to flush anything. Otherwise, the Close call
1156 1 : // will block on locking d.mu until we've finished scheduling the
1157 1 : // flush and set `d.mu.compact.flushing` to true. Close will wait
1158 1 : // for the current flush to complete.
1159 1 : if d.closed.Load() != nil {
1160 1 : return
1161 1 : }
1162 :
1163 1 : if d.mu.mem.mutable == tbl {
1164 1 : d.makeRoomForWrite(nil)
1165 1 : } else {
1166 1 : mem.flushForced = true
1167 1 : }
1168 1 : d.maybeScheduleFlush()
1169 : }
1170 : }()
1171 : }
1172 :
1173 1 : func (d *DB) flush() {
1174 1 : pprof.Do(context.Background(), flushLabels, func(context.Context) {
1175 1 : flushingWorkStart := crtime.NowMono()
1176 1 : d.mu.Lock()
1177 1 : defer d.mu.Unlock()
1178 1 : idleDuration := flushingWorkStart.Sub(d.mu.compact.noOngoingFlushStartTime)
1179 1 : var bytesFlushed uint64
1180 1 : var err error
1181 1 : if bytesFlushed, err = d.flush1(); err != nil {
1182 1 : // TODO(peter): count consecutive flush errors and backoff.
1183 1 : d.opts.EventListener.BackgroundError(err)
1184 1 : }
1185 1 : d.mu.compact.flushing = false
1186 1 : d.mu.compact.noOngoingFlushStartTime = crtime.NowMono()
1187 1 : workDuration := d.mu.compact.noOngoingFlushStartTime.Sub(flushingWorkStart)
1188 1 : d.mu.compact.flushWriteThroughput.Bytes += int64(bytesFlushed)
1189 1 : d.mu.compact.flushWriteThroughput.WorkDuration += workDuration
1190 1 : d.mu.compact.flushWriteThroughput.IdleDuration += idleDuration
1191 1 : // More flush work may have arrived while we were flushing, so schedule
1192 1 : // another flush if needed.
1193 1 : d.maybeScheduleFlush()
1194 1 : // The flush may have produced too many files in a level, so schedule a
1195 1 : // compaction if needed.
1196 1 : d.maybeScheduleCompaction()
1197 1 : d.mu.compact.cond.Broadcast()
1198 : })
1199 : }
1200 :
1201 : // runIngestFlush is used to generate a flush version edit for sstables which
1202 : // were ingested as flushables. Both DB.mu and the manifest lock must be held
1203 : // while runIngestFlush is called.
1204 1 : func (d *DB) runIngestFlush(c *compaction) (*manifest.VersionEdit, error) {
1205 1 : if len(c.flushing) != 1 {
1206 0 : panic("pebble: ingestedFlushable must be flushed one at a time.")
1207 : }
1208 :
1209 : // Construct the VersionEdit, levelMetrics etc.
1210 1 : c.metrics = make(map[int]*LevelMetrics, numLevels)
1211 1 : // Finding the target level for ingestion must use the latest version
1212 1 : // after the logLock has been acquired.
1213 1 : c.version = d.mu.versions.currentVersion()
1214 1 :
1215 1 : baseLevel := d.mu.versions.picker.getBaseLevel()
1216 1 : ve := &versionEdit{}
1217 1 : var ingestSplitFiles []ingestSplitFile
1218 1 : ingestFlushable := c.flushing[0].flushable.(*ingestedFlushable)
1219 1 :
1220 1 : updateLevelMetricsOnExcise := func(m *fileMetadata, level int, added []newFileEntry) {
1221 1 : levelMetrics := c.metrics[level]
1222 1 : if levelMetrics == nil {
1223 1 : levelMetrics = &LevelMetrics{}
1224 1 : c.metrics[level] = levelMetrics
1225 1 : }
1226 1 : levelMetrics.NumFiles--
1227 1 : levelMetrics.Size -= int64(m.Size)
1228 1 : for i := range added {
1229 1 : levelMetrics.NumFiles++
1230 1 : levelMetrics.Size += int64(added[i].Meta.Size)
1231 1 : }
1232 : }
1233 :
1234 1 : suggestSplit := d.opts.Experimental.IngestSplit != nil && d.opts.Experimental.IngestSplit() &&
1235 1 : d.FormatMajorVersion() >= FormatVirtualSSTables
1236 1 :
1237 1 : if suggestSplit || ingestFlushable.exciseSpan.Valid() {
1238 1 : // We could add deleted files to ve.
1239 1 : ve.DeletedFiles = make(map[manifest.DeletedFileEntry]*manifest.FileMetadata)
1240 1 : }
1241 :
1242 1 : ctx := context.Background()
1243 1 : overlapChecker := &overlapChecker{
1244 1 : comparer: d.opts.Comparer,
1245 1 : newIters: d.newIters,
1246 1 : opts: IterOptions{
1247 1 : logger: d.opts.Logger,
1248 1 : Category: categoryIngest,
1249 1 : },
1250 1 : v: c.version,
1251 1 : }
1252 1 : replacedFiles := make(map[base.FileNum][]newFileEntry)
1253 1 : for _, file := range ingestFlushable.files {
1254 1 : var fileToSplit *fileMetadata
1255 1 : var level int
1256 1 :
1257 1 : // This file fits perfectly within the excise span, so we can slot it at L6.
1258 1 : if ingestFlushable.exciseSpan.Valid() &&
1259 1 : ingestFlushable.exciseSpan.Contains(d.cmp, file.FileMetadata.Smallest) &&
1260 1 : ingestFlushable.exciseSpan.Contains(d.cmp, file.FileMetadata.Largest) {
1261 1 : level = 6
1262 1 : } else {
1263 1 : // TODO(radu): this can perform I/O; we should not do this while holding DB.mu.
1264 1 : lsmOverlap, err := overlapChecker.DetermineLSMOverlap(ctx, file.UserKeyBounds())
1265 1 : if err != nil {
1266 0 : return nil, err
1267 0 : }
1268 1 : level, fileToSplit, err = ingestTargetLevel(
1269 1 : ctx, d.cmp, lsmOverlap, baseLevel, d.mu.compact.inProgress, file.FileMetadata, suggestSplit,
1270 1 : )
1271 1 : if err != nil {
1272 0 : return nil, err
1273 0 : }
1274 : }
1275 :
1276 : // Add the current flushableIngest file to the version.
1277 1 : ve.NewFiles = append(ve.NewFiles, newFileEntry{Level: level, Meta: file.FileMetadata})
1278 1 : if fileToSplit != nil {
1279 0 : ingestSplitFiles = append(ingestSplitFiles, ingestSplitFile{
1280 0 : ingestFile: file.FileMetadata,
1281 0 : splitFile: fileToSplit,
1282 0 : level: level,
1283 0 : })
1284 0 : }
1285 1 : levelMetrics := c.metrics[level]
1286 1 : if levelMetrics == nil {
1287 1 : levelMetrics = &LevelMetrics{}
1288 1 : c.metrics[level] = levelMetrics
1289 1 : }
1290 1 : levelMetrics.BytesIngested += file.Size
1291 1 : levelMetrics.TablesIngested++
1292 : }
1293 1 : if ingestFlushable.exciseSpan.Valid() {
1294 1 : // Iterate through all levels and find files that intersect with exciseSpan.
1295 1 : for l := range c.version.Levels {
1296 1 : overlaps := c.version.Overlaps(l, base.UserKeyBoundsEndExclusive(ingestFlushable.exciseSpan.Start, ingestFlushable.exciseSpan.End))
1297 1 : iter := overlaps.Iter()
1298 1 :
1299 1 : for m := iter.First(); m != nil; m = iter.Next() {
1300 1 : newFiles, err := d.excise(context.TODO(), ingestFlushable.exciseSpan.UserKeyBounds(), m, ve, l)
1301 1 : if err != nil {
1302 0 : return nil, err
1303 0 : }
1304 :
1305 1 : if _, ok := ve.DeletedFiles[deletedFileEntry{
1306 1 : Level: l,
1307 1 : FileNum: m.FileNum,
1308 1 : }]; !ok {
1309 1 : // We did not excise this file.
1310 1 : continue
1311 : }
1312 1 : replacedFiles[m.FileNum] = newFiles
1313 1 : updateLevelMetricsOnExcise(m, l, newFiles)
1314 : }
1315 : }
1316 : }
1317 :
1318 1 : if len(ingestSplitFiles) > 0 {
1319 0 : if err := d.ingestSplit(context.TODO(), ve, updateLevelMetricsOnExcise, ingestSplitFiles, replacedFiles); err != nil {
1320 0 : return nil, err
1321 0 : }
1322 : }
1323 :
1324 1 : return ve, nil
1325 : }
1326 :
1327 : // flush runs a compaction that copies the immutable memtables from memory to
1328 : // disk.
1329 : //
1330 : // d.mu must be held when calling this, but the mutex may be dropped and
1331 : // re-acquired during the course of this method.
1332 1 : func (d *DB) flush1() (bytesFlushed uint64, err error) {
1333 1 : // NB: The flushable queue can contain flushables of type ingestedFlushable.
1334 1 : // The sstables in ingestedFlushable.files must be placed into the appropriate
1335 1 : // level in the lsm. Let's say the flushable queue contains a prefix of
1336 1 : // regular immutable memtables, then an ingestedFlushable, and then the
1337 1 : // mutable memtable. When the flush of the ingestedFlushable is performed,
1338 1 : // it needs an updated view of the lsm. That is, the prefix of immutable
1339 1 : // memtables must have already been flushed. Similarly, if there are two
1340 1 : // contiguous ingestedFlushables in the queue, then the first flushable must
1341 1 : // be flushed, so that the second flushable can see an updated view of the
1342 1 : // lsm.
1343 1 : //
1344 1 : // Given the above, we restrict flushes to either some prefix of regular
1345 1 : // memtables, or a single flushable of type ingestedFlushable. The DB.flush
1346 1 : // function will call DB.maybeScheduleFlush again, so a new flush to finish
1347 1 : // the remaining flush work should be scheduled right away.
1348 1 : //
1349 1 : // NB: Large batches placed in the flushable queue share the WAL with the
1350 1 : // previous memtable in the queue. We must ensure the property that both the
1351 1 : // large batch and the memtable with which it shares a WAL are flushed
1352 1 : // together. The property ensures that the minimum unflushed log number
1353 1 : // isn't incremented incorrectly. Since a flushableBatch.readyToFlush always
1354 1 : // returns true, and since the large batch will always be placed right after
1355 1 : // the memtable with which it shares a WAL, the property is naturally
1356 1 : // ensured. The large batch will always be placed after the memtable with
1357 1 : // which it shares a WAL because we ensure it in DB.commitWrite by holding
1358 1 : // the commitPipeline.mu and then holding DB.mu. As an extra defensive
1359 1 : // measure, if we try to flush the memtable without also flushing the
1360 1 : // flushable batch in the same flush, since the memtable and flushableBatch
1361 1 : // have the same logNum, the logNum invariant check below will trigger.
1362 1 : var n, inputs int
1363 1 : var inputBytes uint64
1364 1 : var ingest bool
1365 1 : for ; n < len(d.mu.mem.queue)-1; n++ {
1366 1 : if f, ok := d.mu.mem.queue[n].flushable.(*ingestedFlushable); ok {
1367 1 : if n == 0 {
1368 1 : // The first flushable is of type ingestedFlushable. Since these
1369 1 : // must be flushed individually, we perform a flush for just
1370 1 : // this.
1371 1 : if !f.readyForFlush() {
1372 0 : // This check is almost unnecessary, but we guard against it
1373 0 : // just in case this invariant changes in the future.
1374 0 : panic("pebble: ingestedFlushable should always be ready to flush.")
1375 : }
1376 : // By setting n = 1, we ensure that the first flushable(n == 0)
1377 : // is scheduled for a flush. The number of tables added is equal to the
1378 : // number of files in the ingest operation.
1379 1 : n = 1
1380 1 : inputs = len(f.files)
1381 1 : ingest = true
1382 1 : break
1383 1 : } else {
1384 1 : // There was some prefix of flushables which weren't of type
1385 1 : // ingestedFlushable. So, perform a flush for those.
1386 1 : break
1387 : }
1388 : }
1389 1 : if !d.mu.mem.queue[n].readyForFlush() {
1390 0 : break
1391 : }
1392 1 : inputBytes += d.mu.mem.queue[n].inuseBytes()
1393 : }
1394 1 : if n == 0 {
1395 0 : // None of the immutable memtables are ready for flushing.
1396 0 : return 0, nil
1397 0 : }
1398 1 : if !ingest {
1399 1 : // Flushes of memtables add the prefix of n memtables from the flushable
1400 1 : // queue.
1401 1 : inputs = n
1402 1 : }
1403 :
1404 : // Require that every memtable being flushed has a log number less than the
1405 : // new minimum unflushed log number.
1406 1 : minUnflushedLogNum := d.mu.mem.queue[n].logNum
1407 1 : if !d.opts.DisableWAL {
1408 1 : for i := 0; i < n; i++ {
1409 1 : if logNum := d.mu.mem.queue[i].logNum; logNum >= minUnflushedLogNum {
1410 0 : panic(errors.AssertionFailedf("logNum invariant violated: flushing %d items; %d:type=%T,logNum=%d; %d:type=%T,logNum=%d",
1411 0 : n,
1412 0 : i, d.mu.mem.queue[i].flushable, logNum,
1413 0 : n, d.mu.mem.queue[n].flushable, minUnflushedLogNum))
1414 : }
1415 : }
1416 : }
1417 :
1418 1 : c, err := newFlush(d.opts, d.mu.versions.currentVersion(),
1419 1 : d.mu.versions.picker.getBaseLevel(), d.mu.mem.queue[:n], d.timeNow())
1420 1 : if err != nil {
1421 0 : return 0, err
1422 0 : }
1423 1 : d.addInProgressCompaction(c)
1424 1 :
1425 1 : jobID := d.newJobIDLocked()
1426 1 : d.opts.EventListener.FlushBegin(FlushInfo{
1427 1 : JobID: int(jobID),
1428 1 : Input: inputs,
1429 1 : InputBytes: inputBytes,
1430 1 : Ingest: ingest,
1431 1 : })
1432 1 : startTime := d.timeNow()
1433 1 :
1434 1 : var ve *manifest.VersionEdit
1435 1 : var stats compact.Stats
1436 1 : // To determine the target level of the files in the ingestedFlushable, we
1437 1 : // need to acquire the logLock, and not release it for that duration. Since,
1438 1 : // we need to acquire the logLock below to perform the logAndApply step
1439 1 : // anyway, we create the VersionEdit for ingestedFlushable outside of
1440 1 : // runCompaction. For all other flush cases, we construct the VersionEdit
1441 1 : // inside runCompaction.
1442 1 : if c.kind != compactionKindIngestedFlushable {
1443 1 : ve, stats, err = d.runCompaction(jobID, c)
1444 1 : }
1445 :
1446 : // Acquire logLock. This will be released either on an error, by way of
1447 : // logUnlock, or through a call to logAndApply if there is no error.
1448 1 : d.mu.versions.logLock()
1449 1 :
1450 1 : if c.kind == compactionKindIngestedFlushable {
1451 1 : ve, err = d.runIngestFlush(c)
1452 1 : }
1453 :
1454 1 : info := FlushInfo{
1455 1 : JobID: int(jobID),
1456 1 : Input: inputs,
1457 1 : InputBytes: inputBytes,
1458 1 : Duration: d.timeNow().Sub(startTime),
1459 1 : Done: true,
1460 1 : Ingest: ingest,
1461 1 : Err: err,
1462 1 : }
1463 1 : if err == nil {
1464 1 : validateVersionEdit(ve, d.opts.Experimental.KeyValidationFunc, d.opts.Comparer.FormatKey, d.opts.Logger)
1465 1 : for i := range ve.NewFiles {
1466 1 : e := &ve.NewFiles[i]
1467 1 : info.Output = append(info.Output, e.Meta.TableInfo())
1468 1 : // Ingested tables are not necessarily flushed to L0. Record the level of
1469 1 : // each ingested file explicitly.
1470 1 : if ingest {
1471 1 : info.IngestLevels = append(info.IngestLevels, e.Level)
1472 1 : }
1473 : }
1474 1 : if len(ve.NewFiles) == 0 {
1475 1 : info.Err = errEmptyTable
1476 1 : }
1477 :
1478 : // The flush succeeded or it produced an empty sstable. In either case we
1479 : // want to bump the minimum unflushed log number to the log number of the
1480 : // oldest unflushed memtable.
1481 1 : ve.MinUnflushedLogNum = minUnflushedLogNum
1482 1 : if c.kind != compactionKindIngestedFlushable {
1483 1 : metrics := c.metrics[0]
1484 1 : if d.opts.DisableWAL {
1485 1 : // If the WAL is disabled, every flushable has a zero [logSize],
1486 1 : // resulting in zero bytes in. Instead, use the number of bytes we
1487 1 : // flushed as the BytesIn. This ensures we get a reasonable w-amp
1488 1 : // calculation even when the WAL is disabled.
1489 1 : metrics.BytesIn = metrics.BytesFlushed
1490 1 : } else {
1491 1 : for i := 0; i < n; i++ {
1492 1 : metrics.BytesIn += d.mu.mem.queue[i].logSize
1493 1 : }
1494 : }
1495 1 : } else {
1496 1 : // c.kind == compactionKindIngestedFlushable && we could have deleted files due
1497 1 : // to ingest-time splits or excises.
1498 1 : ingestFlushable := c.flushing[0].flushable.(*ingestedFlushable)
1499 1 : for c2 := range d.mu.compact.inProgress {
1500 1 : // Check if this compaction overlaps with the excise span. Note that just
1501 1 : // checking if the inputs individually overlap with the excise span
1502 1 : // isn't sufficient; for instance, a compaction could have [a,b] and [e,f]
1503 1 : // as inputs and write it all out as [a,b,e,f] in one sstable. If we're
1504 1 : // doing a [c,d) excise at the same time as this compaction, we will have
1505 1 : // to error out the whole compaction as we can't guarantee it hasn't/won't
1506 1 : // write a file overlapping with the excise span.
1507 1 : if ingestFlushable.exciseSpan.OverlapsInternalKeyRange(d.cmp, c2.smallest, c2.largest) {
1508 1 : c2.cancel.Store(true)
1509 1 : continue
1510 : }
1511 : }
1512 :
1513 1 : if len(ve.DeletedFiles) > 0 {
1514 1 : // Iterate through all other compactions, and check if their inputs have
1515 1 : // been replaced due to an ingest-time split or excise. In that case,
1516 1 : // cancel the compaction.
1517 1 : for c2 := range d.mu.compact.inProgress {
1518 1 : for i := range c2.inputs {
1519 1 : iter := c2.inputs[i].files.Iter()
1520 1 : for f := iter.First(); f != nil; f = iter.Next() {
1521 1 : if _, ok := ve.DeletedFiles[deletedFileEntry{FileNum: f.FileNum, Level: c2.inputs[i].level}]; ok {
1522 1 : c2.cancel.Store(true)
1523 1 : break
1524 : }
1525 : }
1526 : }
1527 : }
1528 : }
1529 : }
1530 1 : err = d.mu.versions.logAndApply(jobID, ve, c.metrics, false, /* forceRotation */
1531 1 : func() []compactionInfo { return d.getInProgressCompactionInfoLocked(c) })
1532 1 : if err != nil {
1533 1 : info.Err = err
1534 1 : }
1535 1 : } else {
1536 1 : // We won't be performing the logAndApply step because of the error,
1537 1 : // so logUnlock.
1538 1 : d.mu.versions.logUnlock()
1539 1 : }
1540 :
1541 : // If err != nil, then the flush will be retried, and we will recalculate
1542 : // these metrics.
1543 1 : if err == nil {
1544 1 : d.mu.snapshots.cumulativePinnedCount += stats.CumulativePinnedKeys
1545 1 : d.mu.snapshots.cumulativePinnedSize += stats.CumulativePinnedSize
1546 1 : d.mu.versions.metrics.Keys.MissizedTombstonesCount += stats.CountMissizedDels
1547 1 : }
1548 :
1549 1 : d.clearCompactingState(c, err != nil)
1550 1 : delete(d.mu.compact.inProgress, c)
1551 1 : d.mu.versions.incrementCompactions(c.kind, c.extraLevels, c.pickerMetrics)
1552 1 :
1553 1 : var flushed flushableList
1554 1 : if err == nil {
1555 1 : flushed = d.mu.mem.queue[:n]
1556 1 : d.mu.mem.queue = d.mu.mem.queue[n:]
1557 1 : d.updateReadStateLocked(d.opts.DebugCheck)
1558 1 : d.updateTableStatsLocked(ve.NewFiles)
1559 1 : if ingest {
1560 1 : d.mu.versions.metrics.Flush.AsIngestCount++
1561 1 : for _, l := range c.metrics {
1562 1 : d.mu.versions.metrics.Flush.AsIngestBytes += l.BytesIngested
1563 1 : d.mu.versions.metrics.Flush.AsIngestTableCount += l.TablesIngested
1564 1 : }
1565 : }
1566 1 : d.maybeTransitionSnapshotsToFileOnlyLocked()
1567 :
1568 : }
1569 : // Signal FlushEnd after installing the new readState. This helps for unit
1570 : // tests that use the callback to trigger a read using an iterator with
1571 : // IterOptions.OnlyReadGuaranteedDurable.
1572 1 : info.TotalDuration = d.timeNow().Sub(startTime)
1573 1 : d.opts.EventListener.FlushEnd(info)
1574 1 :
1575 1 : // The order of these operations matters here for ease of testing.
1576 1 : // Removing the reader reference first allows tests to be guaranteed that
1577 1 : // the memtable reservation has been released by the time a synchronous
1578 1 : // flush returns. readerUnrefLocked may also produce obsolete files so the
1579 1 : // call to deleteObsoleteFiles must happen after it.
1580 1 : for i := range flushed {
1581 1 : flushed[i].readerUnrefLocked(true)
1582 1 : }
1583 :
1584 1 : d.deleteObsoleteFiles(jobID)
1585 1 :
1586 1 : // Mark all the memtables we flushed as flushed.
1587 1 : for i := range flushed {
1588 1 : close(flushed[i].flushed)
1589 1 : }
1590 :
1591 1 : return inputBytes, err
1592 : }
1593 :
1594 : // maybeTransitionSnapshotsToFileOnlyLocked transitions any "eventually
1595 : // file-only" snapshots to be file-only if all their visible state has been
1596 : // flushed to sstables.
1597 : //
1598 : // REQUIRES: d.mu.
1599 1 : func (d *DB) maybeTransitionSnapshotsToFileOnlyLocked() {
1600 1 : earliestUnflushedSeqNum := d.getEarliestUnflushedSeqNumLocked()
1601 1 : currentVersion := d.mu.versions.currentVersion()
1602 1 : for s := d.mu.snapshots.root.next; s != &d.mu.snapshots.root; {
1603 1 : if s.efos == nil {
1604 1 : s = s.next
1605 1 : continue
1606 : }
1607 1 : overlapsFlushable := false
1608 1 : if base.Visible(earliestUnflushedSeqNum, s.efos.seqNum, base.SeqNumMax) {
1609 1 : // There are some unflushed keys that are still visible to the EFOS.
1610 1 : // Check if any memtables older than the EFOS contain keys within a
1611 1 : // protected range of the EFOS. If no, we can transition.
1612 1 : protectedRanges := make([]bounded, len(s.efos.protectedRanges))
1613 1 : for i := range s.efos.protectedRanges {
1614 1 : protectedRanges[i] = s.efos.protectedRanges[i]
1615 1 : }
1616 1 : for i := range d.mu.mem.queue {
1617 1 : if !base.Visible(d.mu.mem.queue[i].logSeqNum, s.efos.seqNum, base.SeqNumMax) {
1618 1 : // All keys in this memtable are newer than the EFOS. Skip this
1619 1 : // memtable.
1620 1 : continue
1621 : }
1622 : // NB: computePossibleOverlaps could have false positives, such as if
1623 : // the flushable is a flushable ingest and not a memtable. In that
1624 : // case we don't open the sstables to check; we just pessimistically
1625 : // assume an overlap.
1626 1 : d.mu.mem.queue[i].computePossibleOverlaps(func(b bounded) shouldContinue {
1627 1 : overlapsFlushable = true
1628 1 : return stopIteration
1629 1 : }, protectedRanges...)
1630 1 : if overlapsFlushable {
1631 1 : break
1632 : }
1633 : }
1634 : }
1635 1 : if overlapsFlushable {
1636 1 : s = s.next
1637 1 : continue
1638 : }
1639 1 : currentVersion.Ref()
1640 1 :
1641 1 : // NB: s.efos.transitionToFileOnlySnapshot could close s, in which
1642 1 : // case s.next would be nil. Save it before calling it.
1643 1 : next := s.next
1644 1 : _ = s.efos.transitionToFileOnlySnapshot(currentVersion)
1645 1 : s = next
1646 : }
1647 : }
1648 :
1649 : // maybeScheduleCompactionAsync should be used when
1650 : // we want to possibly schedule a compaction, but don't
1651 : // want to eat the cost of running maybeScheduleCompaction.
1652 : // This method should be launched in a separate goroutine.
1653 : // d.mu must not be held when this is called.
1654 0 : func (d *DB) maybeScheduleCompactionAsync() {
1655 0 : defer d.compactionSchedulers.Done()
1656 0 :
1657 0 : d.mu.Lock()
1658 0 : d.maybeScheduleCompaction()
1659 0 : d.mu.Unlock()
1660 0 : }
1661 :
1662 : // maybeScheduleCompaction schedules a compaction if necessary.
1663 : //
1664 : // d.mu must be held when calling this.
1665 1 : func (d *DB) maybeScheduleCompaction() {
1666 1 : d.maybeScheduleCompactionPicker(pickAuto)
1667 1 : }
1668 :
1669 1 : func pickAuto(picker compactionPicker, env compactionEnv) *pickedCompaction {
1670 1 : return picker.pickAuto(env)
1671 1 : }
1672 :
1673 1 : func pickElisionOnly(picker compactionPicker, env compactionEnv) *pickedCompaction {
1674 1 : return picker.pickElisionOnlyCompaction(env)
1675 1 : }
1676 :
1677 : // tryScheduleDownloadCompaction tries to start a download compaction.
1678 : //
1679 : // Returns true if we started a download compaction (or completed it
1680 : // immediately because it is a no-op or we hit an error).
1681 : //
1682 : // Requires d.mu to be held. Updates d.mu.compact.downloads.
1683 1 : func (d *DB) tryScheduleDownloadCompaction(env compactionEnv, maxConcurrentDownloads int) bool {
1684 1 : vers := d.mu.versions.currentVersion()
1685 1 : for i := 0; i < len(d.mu.compact.downloads); {
1686 1 : download := d.mu.compact.downloads[i]
1687 1 : switch d.tryLaunchDownloadCompaction(download, vers, env, maxConcurrentDownloads) {
1688 1 : case launchedCompaction:
1689 1 : return true
1690 0 : case didNotLaunchCompaction:
1691 0 : // See if we can launch a compaction for another download task.
1692 0 : i++
1693 1 : case downloadTaskCompleted:
1694 1 : // Task is completed and must be removed.
1695 1 : d.mu.compact.downloads = slices.Delete(d.mu.compact.downloads, i, i+1)
1696 : }
1697 : }
1698 1 : return false
1699 : }
1700 :
1701 : // maybeScheduleCompactionPicker schedules a compaction if necessary,
1702 : // calling `pickFunc` to pick automatic compactions.
1703 : //
1704 : // Requires d.mu to be held.
1705 : func (d *DB) maybeScheduleCompactionPicker(
1706 : pickFunc func(compactionPicker, compactionEnv) *pickedCompaction,
1707 1 : ) {
1708 1 : if d.closed.Load() != nil || d.opts.ReadOnly {
1709 1 : return
1710 1 : }
1711 1 : maxCompactions := d.opts.MaxConcurrentCompactions()
1712 1 : maxDownloads := d.opts.MaxConcurrentDownloads()
1713 1 :
1714 1 : if d.mu.compact.compactingCount >= maxCompactions &&
1715 1 : (len(d.mu.compact.downloads) == 0 || d.mu.compact.downloadingCount >= maxDownloads) {
1716 1 : if len(d.mu.compact.manual) > 0 {
1717 1 : // Inability to run head blocks later manual compactions.
1718 1 : d.mu.compact.manual[0].retries++
1719 1 : }
1720 1 : return
1721 : }
1722 :
1723 : // Compaction picking needs a coherent view of a Version. In particular, we
1724 : // need to exclude concurrent ingestions from making a decision on which level
1725 : // to ingest into that conflicts with our compaction
1726 : // decision. versionSet.logLock provides the necessary mutual exclusion.
1727 1 : d.mu.versions.logLock()
1728 1 : defer d.mu.versions.logUnlock()
1729 1 :
1730 1 : // Check for the closed flag again, in case the DB was closed while we were
1731 1 : // waiting for logLock().
1732 1 : if d.closed.Load() != nil {
1733 1 : return
1734 1 : }
1735 :
1736 1 : env := compactionEnv{
1737 1 : diskAvailBytes: d.diskAvailBytes.Load(),
1738 1 : earliestSnapshotSeqNum: d.mu.snapshots.earliest(),
1739 1 : earliestUnflushedSeqNum: d.getEarliestUnflushedSeqNumLocked(),
1740 1 : }
1741 1 :
1742 1 : if d.mu.compact.compactingCount < maxCompactions {
1743 1 : // Check for delete-only compactions first, because they're expected to be
1744 1 : // cheap and reduce future compaction work.
1745 1 : if !d.opts.private.disableDeleteOnlyCompactions &&
1746 1 : !d.opts.DisableAutomaticCompactions &&
1747 1 : len(d.mu.compact.deletionHints) > 0 {
1748 1 : d.tryScheduleDeleteOnlyCompaction()
1749 1 : }
1750 :
1751 1 : for len(d.mu.compact.manual) > 0 && d.mu.compact.compactingCount < maxCompactions {
1752 1 : if manual := d.mu.compact.manual[0]; !d.tryScheduleManualCompaction(env, manual) {
1753 1 : // Inability to run head blocks later manual compactions.
1754 1 : manual.retries++
1755 1 : break
1756 : }
1757 1 : d.mu.compact.manual = d.mu.compact.manual[1:]
1758 : }
1759 :
1760 1 : for !d.opts.DisableAutomaticCompactions && d.mu.compact.compactingCount < maxCompactions &&
1761 1 : d.tryScheduleAutoCompaction(env, pickFunc) {
1762 1 : }
1763 : }
1764 :
1765 1 : for len(d.mu.compact.downloads) > 0 && d.mu.compact.downloadingCount < maxDownloads &&
1766 1 : d.tryScheduleDownloadCompaction(env, maxDownloads) {
1767 1 : }
1768 : }
1769 :
1770 : // tryScheduleDeleteOnlyCompaction tries to kick off a delete-only compaction
1771 : // for all files that can be deleted as suggested by deletionHints.
1772 : //
1773 : // Requires d.mu to be held. Updates d.mu.compact.deletionHints.
1774 1 : func (d *DB) tryScheduleDeleteOnlyCompaction() {
1775 1 : v := d.mu.versions.currentVersion()
1776 1 : snapshots := d.mu.snapshots.toSlice()
1777 1 : // We need to save the value of exciseEnabled in the compaction itself, as
1778 1 : // it can change dynamically between now and when the compaction runs.
1779 1 : exciseEnabled := d.FormatMajorVersion() >= FormatVirtualSSTables &&
1780 1 : d.opts.Experimental.EnableDeleteOnlyCompactionExcises != nil && d.opts.Experimental.EnableDeleteOnlyCompactionExcises()
1781 1 : inputs, resolvedHints, unresolvedHints := checkDeleteCompactionHints(d.cmp, v, d.mu.compact.deletionHints, snapshots, exciseEnabled)
1782 1 : d.mu.compact.deletionHints = unresolvedHints
1783 1 :
1784 1 : if len(inputs) > 0 {
1785 1 : c := newDeleteOnlyCompaction(d.opts, v, inputs, d.timeNow(), resolvedHints, exciseEnabled)
1786 1 : d.mu.compact.compactingCount++
1787 1 : d.addInProgressCompaction(c)
1788 1 : go d.compact(c, nil)
1789 1 : }
1790 : }
1791 :
1792 : // tryScheduleManualCompaction tries to kick off the given manual compaction.
1793 : //
1794 : // Returns false if we are not able to run this compaction at this time.
1795 : //
1796 : // Requires d.mu to be held.
1797 1 : func (d *DB) tryScheduleManualCompaction(env compactionEnv, manual *manualCompaction) bool {
1798 1 : v := d.mu.versions.currentVersion()
1799 1 : env.inProgressCompactions = d.getInProgressCompactionInfoLocked(nil)
1800 1 : pc, retryLater := pickManualCompaction(v, d.opts, env, d.mu.versions.picker.getBaseLevel(), manual)
1801 1 : if pc == nil {
1802 1 : if !retryLater {
1803 1 : // Manual compaction is a no-op. Signal completion and exit.
1804 1 : manual.done <- nil
1805 1 : return true
1806 1 : }
1807 : // We are not able to run this manual compaction at this time.
1808 1 : return false
1809 : }
1810 :
1811 1 : c := newCompaction(pc, d.opts, d.timeNow(), d.ObjProvider())
1812 1 : d.mu.compact.compactingCount++
1813 1 : d.addInProgressCompaction(c)
1814 1 : go d.compact(c, manual.done)
1815 1 : return true
1816 : }
1817 :
1818 : // tryScheduleAutoCompaction tries to kick off an automatic compaction.
1819 : //
1820 : // Returns false if no automatic compactions are necessary or able to run at
1821 : // this time.
1822 : //
1823 : // Requires d.mu to be held.
1824 : func (d *DB) tryScheduleAutoCompaction(
1825 : env compactionEnv, pickFunc func(compactionPicker, compactionEnv) *pickedCompaction,
1826 1 : ) bool {
1827 1 : env.inProgressCompactions = d.getInProgressCompactionInfoLocked(nil)
1828 1 : env.readCompactionEnv = readCompactionEnv{
1829 1 : readCompactions: &d.mu.compact.readCompactions,
1830 1 : flushing: d.mu.compact.flushing || d.passedFlushThreshold(),
1831 1 : rescheduleReadCompaction: &d.mu.compact.rescheduleReadCompaction,
1832 1 : }
1833 1 : pc := pickFunc(d.mu.versions.picker, env)
1834 1 : if pc == nil {
1835 1 : return false
1836 1 : }
1837 1 : c := newCompaction(pc, d.opts, d.timeNow(), d.ObjProvider())
1838 1 : d.mu.compact.compactingCount++
1839 1 : d.addInProgressCompaction(c)
1840 1 : go d.compact(c, nil)
1841 1 : return true
1842 : }
1843 :
1844 : // deleteCompactionHintType indicates whether the deleteCompactionHint was
1845 : // generated from a span containing a range del (point key only), a range key
1846 : // delete (range key only), or both a point and range key.
1847 : type deleteCompactionHintType uint8
1848 :
1849 : const (
1850 : // NOTE: While these are primarily used as enumeration types, they are also
1851 : // used for some bitwise operations. Care should be taken when updating.
1852 : deleteCompactionHintTypeUnknown deleteCompactionHintType = iota
1853 : deleteCompactionHintTypePointKeyOnly
1854 : deleteCompactionHintTypeRangeKeyOnly
1855 : deleteCompactionHintTypePointAndRangeKey
1856 : )
1857 :
1858 : // String implements fmt.Stringer.
1859 1 : func (h deleteCompactionHintType) String() string {
1860 1 : switch h {
1861 0 : case deleteCompactionHintTypeUnknown:
1862 0 : return "unknown"
1863 1 : case deleteCompactionHintTypePointKeyOnly:
1864 1 : return "point-key-only"
1865 1 : case deleteCompactionHintTypeRangeKeyOnly:
1866 1 : return "range-key-only"
1867 1 : case deleteCompactionHintTypePointAndRangeKey:
1868 1 : return "point-and-range-key"
1869 0 : default:
1870 0 : panic(fmt.Sprintf("unknown hint type: %d", h))
1871 : }
1872 : }
1873 :
1874 : // compactionHintFromKeys returns a deleteCompactionHintType given a slice of
1875 : // keyspan.Keys.
1876 1 : func compactionHintFromKeys(keys []keyspan.Key) deleteCompactionHintType {
1877 1 : var hintType deleteCompactionHintType
1878 1 : for _, k := range keys {
1879 1 : switch k.Kind() {
1880 1 : case base.InternalKeyKindRangeDelete:
1881 1 : hintType |= deleteCompactionHintTypePointKeyOnly
1882 1 : case base.InternalKeyKindRangeKeyDelete:
1883 1 : hintType |= deleteCompactionHintTypeRangeKeyOnly
1884 0 : default:
1885 0 : panic(fmt.Sprintf("unsupported key kind: %s", k.Kind()))
1886 : }
1887 : }
1888 1 : return hintType
1889 : }
1890 :
1891 : // A deleteCompactionHint records a user key and sequence number span that has been
1892 : // deleted by a range tombstone. A hint is recorded if at least one sstable
1893 : // falls completely within both the user key and sequence number spans.
1894 : // Once the tombstones and the observed completely-contained sstables fall
1895 : // into the same snapshot stripe, a delete-only compaction may delete any
1896 : // sstables within the range.
1897 : type deleteCompactionHint struct {
1898 : // The type of key span that generated this hint (point key, range key, or
1899 : // both).
1900 : hintType deleteCompactionHintType
1901 : // start and end are user keys specifying a key range [start, end) of
1902 : // deleted keys.
1903 : start []byte
1904 : end []byte
1905 : // The level of the file containing the range tombstone(s) when the hint
1906 : // was created. Only lower levels need to be searched for files that may
1907 : // be deleted.
1908 : tombstoneLevel int
1909 : // The file containing the range tombstone(s) that created the hint.
1910 : tombstoneFile *fileMetadata
1911 : // The smallest and largest sequence numbers of the abutting tombstones
1912 : // merged to form this hint. All of a tables' keys must be less than the
1913 : // tombstone smallest sequence number to be deleted. All of a tables'
1914 : // sequence numbers must fall into the same snapshot stripe as the
1915 : // tombstone largest sequence number to be deleted.
1916 : tombstoneLargestSeqNum base.SeqNum
1917 : tombstoneSmallestSeqNum base.SeqNum
1918 : // The smallest sequence number of a sstable that was found to be covered
1919 : // by this hint. The hint cannot be resolved until this sequence number is
1920 : // in the same snapshot stripe as the largest tombstone sequence number.
1921 : // This is set when a hint is created, so the LSM may look different and
1922 : // notably no longer contain the sstable that contained the key at this
1923 : // sequence number.
1924 : fileSmallestSeqNum base.SeqNum
1925 : }
1926 :
1927 : type deletionHintOverlap int8
1928 :
1929 : const (
1930 : // hintDoesNotApply indicates that the hint does not apply to the file.
1931 : hintDoesNotApply deletionHintOverlap = iota
1932 : // hintExcisesFile indicates that the hint excises a portion of the file,
1933 : // and the format major version of the DB supports excises.
1934 : hintExcisesFile
1935 : // hintDeletesFile indicates that the hint deletes the entirety of the file.
1936 : hintDeletesFile
1937 : )
1938 :
1939 1 : func (h deleteCompactionHint) String() string {
1940 1 : return fmt.Sprintf(
1941 1 : "L%d.%s %s-%s seqnums(tombstone=%d-%d, file-smallest=%d, type=%s)",
1942 1 : h.tombstoneLevel, h.tombstoneFile.FileNum, h.start, h.end,
1943 1 : h.tombstoneSmallestSeqNum, h.tombstoneLargestSeqNum, h.fileSmallestSeqNum,
1944 1 : h.hintType,
1945 1 : )
1946 1 : }
1947 :
1948 : func (h *deleteCompactionHint) canDeleteOrExcise(
1949 : cmp Compare, m *fileMetadata, snapshots compact.Snapshots, exciseEnabled bool,
1950 1 : ) deletionHintOverlap {
1951 1 : // The file can only be deleted if all of its keys are older than the
1952 1 : // earliest tombstone aggregated into the hint. Note that we use
1953 1 : // m.LargestSeqNumAbsolute, not m.LargestSeqNum. Consider a compaction that
1954 1 : // zeroes sequence numbers. A compaction may zero the sequence number of a
1955 1 : // key with a sequence number > h.tombstoneSmallestSeqNum and set it to
1956 1 : // zero. If we looked at m.LargestSeqNum, the resulting output file would
1957 1 : // appear to not contain any keys more recent than the oldest tombstone. To
1958 1 : // avoid this error, the largest pre-zeroing sequence number is maintained
1959 1 : // in LargestSeqNumAbsolute and used here to make the determination whether
1960 1 : // the file's keys are older than all of the hint's tombstones.
1961 1 : if m.LargestSeqNumAbsolute >= h.tombstoneSmallestSeqNum || m.SmallestSeqNum < h.fileSmallestSeqNum {
1962 1 : return hintDoesNotApply
1963 1 : }
1964 :
1965 : // The file's oldest key must be in the same snapshot stripe as the
1966 : // newest tombstone. NB: We already checked the hint's sequence numbers,
1967 : // but this file's oldest sequence number might be lower than the hint's
1968 : // smallest sequence number despite the file falling within the key range
1969 : // if this file was constructed after the hint by a compaction.
1970 1 : if snapshots.Index(h.tombstoneLargestSeqNum) != snapshots.Index(m.SmallestSeqNum) {
1971 0 : return hintDoesNotApply
1972 0 : }
1973 :
1974 1 : switch h.hintType {
1975 1 : case deleteCompactionHintTypePointKeyOnly:
1976 1 : // A hint generated by a range del span cannot delete tables that contain
1977 1 : // range keys.
1978 1 : if m.HasRangeKeys {
1979 1 : return hintDoesNotApply
1980 1 : }
1981 1 : case deleteCompactionHintTypeRangeKeyOnly:
1982 1 : // A hint generated by a range key del span cannot delete tables that
1983 1 : // contain point keys.
1984 1 : if m.HasPointKeys {
1985 1 : return hintDoesNotApply
1986 1 : }
1987 1 : case deleteCompactionHintTypePointAndRangeKey:
1988 : // A hint from a span that contains both range dels *and* range keys can
1989 : // only be deleted if both bounds fall within the hint. The next check takes
1990 : // care of this.
1991 0 : default:
1992 0 : panic(fmt.Sprintf("pebble: unknown delete compaction hint type: %d", h.hintType))
1993 : }
1994 1 : if cmp(h.start, m.Smallest.UserKey) <= 0 &&
1995 1 : base.UserKeyExclusive(h.end).CompareUpperBounds(cmp, m.UserKeyBounds().End) >= 0 {
1996 1 : return hintDeletesFile
1997 1 : }
1998 1 : if !exciseEnabled {
1999 1 : // The file's keys must be completely contained within the hint range; excises
2000 1 : // aren't allowed.
2001 1 : return hintDoesNotApply
2002 1 : }
2003 : // Check for any overlap. In cases of partial overlap, we can excise the part of the file
2004 : // that overlaps with the deletion hint.
2005 1 : if cmp(h.end, m.Smallest.UserKey) > 0 &&
2006 1 : (m.UserKeyBounds().End.CompareUpperBounds(cmp, base.UserKeyInclusive(h.start)) >= 0) {
2007 1 : return hintExcisesFile
2008 1 : }
2009 0 : return hintDoesNotApply
2010 : }
2011 :
2012 : // checkDeleteCompactionHints checks the passed-in deleteCompactionHints for those that
2013 : // can be resolved and those that cannot. A hint is considered resolved when its largest
2014 : // tombstone sequence number and the smallest sequence number of covered files fall in
2015 : // the same snapshot stripe. No more than maxHintsPerDeleteOnlyCompaction will be resolved
2016 : // per method call. Resolved and unresolved hints are returned in separate return values.
2017 : // The files that the resolved hints apply to, are returned as compactionLevels.
2018 : func checkDeleteCompactionHints(
2019 : cmp Compare,
2020 : v *version,
2021 : hints []deleteCompactionHint,
2022 : snapshots compact.Snapshots,
2023 : exciseEnabled bool,
2024 1 : ) (levels []compactionLevel, resolved, unresolved []deleteCompactionHint) {
2025 1 : var files map[*fileMetadata]bool
2026 1 : var byLevel [numLevels][]*fileMetadata
2027 1 :
2028 1 : // Delete-only compactions can be quadratic (O(mn)) in terms of runtime
2029 1 : // where m = number of files in the delete-only compaction and n = number
2030 1 : // of resolved hints. To prevent these from growing unbounded, we cap
2031 1 : // the number of hints we resolve for one delete-only compaction. This
2032 1 : // cap only applies if exciseEnabled == true.
2033 1 : const maxHintsPerDeleteOnlyCompaction = 10
2034 1 :
2035 1 : unresolvedHints := hints[:0]
2036 1 : // Lazily populate resolvedHints, similar to files above.
2037 1 : resolvedHints := make([]deleteCompactionHint, 0)
2038 1 : for _, h := range hints {
2039 1 : // Check each compaction hint to see if it's resolvable. Resolvable
2040 1 : // hints are removed and trigger a delete-only compaction if any files
2041 1 : // in the current LSM still meet their criteria. Unresolvable hints
2042 1 : // are saved and don't trigger a delete-only compaction.
2043 1 : //
2044 1 : // When a compaction hint is created, the sequence numbers of the
2045 1 : // range tombstones and the covered file with the oldest key are
2046 1 : // recorded. The largest tombstone sequence number and the smallest
2047 1 : // file sequence number must be in the same snapshot stripe for the
2048 1 : // hint to be resolved. The below graphic models a compaction hint
2049 1 : // covering the keyspace [b, r). The hint completely contains two
2050 1 : // files, 000002 and 000003. The file 000003 contains the lowest
2051 1 : // covered sequence number at #90. The tombstone b.RANGEDEL.230:h has
2052 1 : // the highest tombstone sequence number incorporated into the hint.
2053 1 : // The hint may be resolved only once the snapshots at #100, #180 and
2054 1 : // #210 are all closed. File 000001 is not included within the hint
2055 1 : // because it extends beyond the range tombstones in user key space.
2056 1 : //
2057 1 : // 250
2058 1 : //
2059 1 : // |-b...230:h-|
2060 1 : // _____________________________________________________ snapshot #210
2061 1 : // 200 |--h.RANGEDEL.200:r--|
2062 1 : //
2063 1 : // _____________________________________________________ snapshot #180
2064 1 : //
2065 1 : // 150 +--------+
2066 1 : // +---------+ | 000003 |
2067 1 : // | 000002 | | |
2068 1 : // +_________+ | |
2069 1 : // 100_____________________|________|___________________ snapshot #100
2070 1 : // +--------+
2071 1 : // _____________________________________________________ snapshot #70
2072 1 : // +---------------+
2073 1 : // 50 | 000001 |
2074 1 : // | |
2075 1 : // +---------------+
2076 1 : // ______________________________________________________________
2077 1 : // a b c d e f g h i j k l m n o p q r s t u v w x y z
2078 1 :
2079 1 : if snapshots.Index(h.tombstoneLargestSeqNum) != snapshots.Index(h.fileSmallestSeqNum) ||
2080 1 : (len(resolvedHints) >= maxHintsPerDeleteOnlyCompaction && exciseEnabled) {
2081 1 : // Cannot resolve yet.
2082 1 : unresolvedHints = append(unresolvedHints, h)
2083 1 : continue
2084 : }
2085 :
2086 : // The hint h will be resolved and dropped, if it either affects no files at all
2087 : // or if the number of files it creates (eg. through excision) is less than or
2088 : // equal to the number of files it deletes. First, determine how many files are
2089 : // affected by this hint.
2090 1 : filesDeletedByCurrentHint := 0
2091 1 : var filesDeletedByLevel [7][]*fileMetadata
2092 1 : for l := h.tombstoneLevel + 1; l < numLevels; l++ {
2093 1 : overlaps := v.Overlaps(l, base.UserKeyBoundsEndExclusive(h.start, h.end))
2094 1 : iter := overlaps.Iter()
2095 1 :
2096 1 : for m := iter.First(); m != nil; m = iter.Next() {
2097 1 : doesHintApply := h.canDeleteOrExcise(cmp, m, snapshots, exciseEnabled)
2098 1 : if m.IsCompacting() || doesHintApply == hintDoesNotApply || files[m] {
2099 1 : continue
2100 : }
2101 1 : switch doesHintApply {
2102 1 : case hintDeletesFile:
2103 1 : filesDeletedByCurrentHint++
2104 1 : case hintExcisesFile:
2105 1 : // Account for the original file being deleted.
2106 1 : filesDeletedByCurrentHint++
2107 1 : // An excise could produce up to 2 new files. If the hint
2108 1 : // leaves a fragment of the file on the left, decrement
2109 1 : // the counter once. If the hint leaves a fragment of the
2110 1 : // file on the right, decrement the counter once.
2111 1 : if cmp(h.start, m.Smallest.UserKey) > 0 {
2112 1 : filesDeletedByCurrentHint--
2113 1 : }
2114 1 : if m.UserKeyBounds().End.IsUpperBoundFor(cmp, h.end) {
2115 1 : filesDeletedByCurrentHint--
2116 1 : }
2117 : }
2118 1 : filesDeletedByLevel[l] = append(filesDeletedByLevel[l], m)
2119 : }
2120 : }
2121 1 : if filesDeletedByCurrentHint < 0 {
2122 1 : // This hint does not delete a sufficient number of files to warrant
2123 1 : // a delete-only compaction at this stage. Drop it (ie. don't add it
2124 1 : // to either resolved or unresolved hints) so it doesn't stick around
2125 1 : // forever.
2126 1 : continue
2127 : }
2128 : // This hint will be resolved and dropped.
2129 1 : for l := h.tombstoneLevel + 1; l < numLevels; l++ {
2130 1 : byLevel[l] = append(byLevel[l], filesDeletedByLevel[l]...)
2131 1 : for _, m := range filesDeletedByLevel[l] {
2132 1 : if files == nil {
2133 1 : // Construct files lazily, assuming most calls will not
2134 1 : // produce delete-only compactions.
2135 1 : files = make(map[*fileMetadata]bool)
2136 1 : }
2137 1 : files[m] = true
2138 : }
2139 : }
2140 1 : resolvedHints = append(resolvedHints, h)
2141 : }
2142 :
2143 1 : var compactLevels []compactionLevel
2144 1 : for l, files := range byLevel {
2145 1 : if len(files) == 0 {
2146 1 : continue
2147 : }
2148 1 : compactLevels = append(compactLevels, compactionLevel{
2149 1 : level: l,
2150 1 : files: manifest.NewLevelSliceKeySorted(cmp, files),
2151 1 : })
2152 : }
2153 1 : return compactLevels, resolvedHints, unresolvedHints
2154 : }
2155 :
2156 : // compact runs one compaction and maybe schedules another call to compact.
2157 1 : func (d *DB) compact(c *compaction, errChannel chan error) {
2158 1 : pprof.Do(context.Background(), compactLabels, func(context.Context) {
2159 1 : d.mu.Lock()
2160 1 : defer d.mu.Unlock()
2161 1 : if err := d.compact1(c, errChannel); err != nil {
2162 1 : // TODO(peter): count consecutive compaction errors and backoff.
2163 1 : d.opts.EventListener.BackgroundError(err)
2164 1 : }
2165 1 : if c.isDownload {
2166 1 : d.mu.compact.downloadingCount--
2167 1 : } else {
2168 1 : d.mu.compact.compactingCount--
2169 1 : }
2170 1 : delete(d.mu.compact.inProgress, c)
2171 1 : // Add this compaction's duration to the cumulative duration. NB: This
2172 1 : // must be atomic with the above removal of c from
2173 1 : // d.mu.compact.InProgress to ensure Metrics.Compact.Duration does not
2174 1 : // miss or double count a completing compaction's duration.
2175 1 : d.mu.compact.duration += d.timeNow().Sub(c.beganAt)
2176 1 :
2177 1 : // The previous compaction may have produced too many files in a
2178 1 : // level, so reschedule another compaction if needed.
2179 1 : d.maybeScheduleCompaction()
2180 1 : d.mu.compact.cond.Broadcast()
2181 : })
2182 : }
2183 :
2184 : // cleanupVersionEdit cleans up any on-disk artifacts that were created
2185 : // for the application of a versionEdit that is no longer going to be applied.
2186 : //
2187 : // d.mu must be held when calling this method.
2188 1 : func (d *DB) cleanupVersionEdit(ve *versionEdit) {
2189 1 : obsoleteFiles := make([]*fileBacking, 0, len(ve.NewFiles))
2190 1 : deletedFiles := make(map[base.FileNum]struct{})
2191 1 : for key := range ve.DeletedFiles {
2192 1 : deletedFiles[key.FileNum] = struct{}{}
2193 1 : }
2194 1 : for i := range ve.NewFiles {
2195 1 : if ve.NewFiles[i].Meta.Virtual {
2196 0 : // We handle backing files separately.
2197 0 : continue
2198 : }
2199 1 : if _, ok := deletedFiles[ve.NewFiles[i].Meta.FileNum]; ok {
2200 0 : // This file is being moved in this ve to a different level.
2201 0 : // Don't mark it as obsolete.
2202 0 : continue
2203 : }
2204 1 : obsoleteFiles = append(obsoleteFiles, ve.NewFiles[i].Meta.PhysicalMeta().FileBacking)
2205 : }
2206 1 : for i := range ve.CreatedBackingTables {
2207 0 : if ve.CreatedBackingTables[i].IsUnused() {
2208 0 : obsoleteFiles = append(obsoleteFiles, ve.CreatedBackingTables[i])
2209 0 : }
2210 : }
2211 1 : for i := range obsoleteFiles {
2212 1 : // Add this file to zombie tables as well, as the versionSet
2213 1 : // asserts on whether every obsolete file was at one point
2214 1 : // marked zombie.
2215 1 : d.mu.versions.zombieTables[obsoleteFiles[i].DiskFileNum] = tableInfo{
2216 1 : fileInfo: fileInfo{
2217 1 : FileNum: obsoleteFiles[i].DiskFileNum,
2218 1 : FileSize: obsoleteFiles[i].Size,
2219 1 : },
2220 1 : // TODO(bilal): This is harmless if it's wrong, as it only causes
2221 1 : // incorrect accounting for the size of it in metrics. Currently
2222 1 : // all compactions only write to local files anyway except with
2223 1 : // disaggregated storage; if this becomes the norm, we should do
2224 1 : // an objprovider lookup here.
2225 1 : isLocal: true,
2226 1 : }
2227 1 : }
2228 1 : d.mu.versions.addObsoleteLocked(obsoleteFiles)
2229 : }
2230 :
2231 : // compact1 runs one compaction.
2232 : //
2233 : // d.mu must be held when calling this, but the mutex may be dropped and
2234 : // re-acquired during the course of this method.
2235 1 : func (d *DB) compact1(c *compaction, errChannel chan error) (err error) {
2236 1 : if errChannel != nil {
2237 1 : defer func() {
2238 1 : errChannel <- err
2239 1 : }()
2240 : }
2241 :
2242 1 : jobID := d.newJobIDLocked()
2243 1 : info := c.makeInfo(jobID)
2244 1 : d.opts.EventListener.CompactionBegin(info)
2245 1 : startTime := d.timeNow()
2246 1 :
2247 1 : ve, stats, err := d.runCompaction(jobID, c)
2248 1 :
2249 1 : info.Duration = d.timeNow().Sub(startTime)
2250 1 : if err == nil {
2251 1 : validateVersionEdit(ve, d.opts.Experimental.KeyValidationFunc, d.opts.Comparer.FormatKey, d.opts.Logger)
2252 1 : err = func() error {
2253 1 : var err error
2254 1 : d.mu.versions.logLock()
2255 1 : // Check if this compaction had a conflicting operation (eg. a d.excise())
2256 1 : // that necessitates it restarting from scratch. Note that since we hold
2257 1 : // the manifest lock, we don't expect this bool to change its value
2258 1 : // as only the holder of the manifest lock will ever write to it.
2259 1 : if c.cancel.Load() {
2260 1 : d.mu.versions.metrics.Compact.CancelledCount++
2261 1 : d.mu.versions.metrics.Compact.CancelledBytes += c.bytesWritten
2262 1 :
2263 1 : err = firstError(err, ErrCancelledCompaction)
2264 1 : // This is the first time we've seen a cancellation during the
2265 1 : // life of this compaction (or the original condition on err == nil
2266 1 : // would not have been true). We should delete any tables already
2267 1 : // created, as d.runCompaction did not do that.
2268 1 : d.cleanupVersionEdit(ve)
2269 1 : // logAndApply calls logUnlock. If we didn't call it, we need to call
2270 1 : // logUnlock ourselves.
2271 1 : d.mu.versions.logUnlock()
2272 1 : return err
2273 1 : }
2274 1 : return d.mu.versions.logAndApply(jobID, ve, c.metrics, false /* forceRotation */, func() []compactionInfo {
2275 1 : return d.getInProgressCompactionInfoLocked(c)
2276 1 : })
2277 : }()
2278 : }
2279 :
2280 1 : info.Done = true
2281 1 : info.Err = err
2282 1 : if err == nil {
2283 1 : for i := range ve.NewFiles {
2284 1 : e := &ve.NewFiles[i]
2285 1 : info.Output.Tables = append(info.Output.Tables, e.Meta.TableInfo())
2286 1 : }
2287 1 : d.mu.snapshots.cumulativePinnedCount += stats.CumulativePinnedKeys
2288 1 : d.mu.snapshots.cumulativePinnedSize += stats.CumulativePinnedSize
2289 1 : d.mu.versions.metrics.Keys.MissizedTombstonesCount += stats.CountMissizedDels
2290 : }
2291 :
2292 : // NB: clearing compacting state must occur before updating the read state;
2293 : // L0Sublevels initialization depends on it.
2294 1 : d.clearCompactingState(c, err != nil)
2295 1 : if err != nil && errors.Is(err, ErrCancelledCompaction) {
2296 1 : d.mu.versions.metrics.Compact.CancelledCount++
2297 1 : d.mu.versions.metrics.Compact.CancelledBytes += c.bytesWritten
2298 1 : }
2299 1 : d.mu.versions.incrementCompactions(c.kind, c.extraLevels, c.pickerMetrics)
2300 1 : d.mu.versions.incrementCompactionBytes(-c.bytesWritten)
2301 1 :
2302 1 : info.TotalDuration = d.timeNow().Sub(c.beganAt)
2303 1 : d.opts.EventListener.CompactionEnd(info)
2304 1 :
2305 1 : // Update the read state before deleting obsolete files because the
2306 1 : // read-state update will cause the previous version to be unref'd and if
2307 1 : // there are no references obsolete tables will be added to the obsolete
2308 1 : // table list.
2309 1 : if err == nil {
2310 1 : d.updateReadStateLocked(d.opts.DebugCheck)
2311 1 : d.updateTableStatsLocked(ve.NewFiles)
2312 1 : }
2313 1 : d.deleteObsoleteFiles(jobID)
2314 1 :
2315 1 : return err
2316 : }
2317 :
2318 : // runCopyCompaction runs a copy compaction where a new FileNum is created that
2319 : // is a byte-for-byte copy of the input file or span thereof in some cases. This
2320 : // is used in lieu of a move compaction when a file is being moved across the
2321 : // local/remote storage boundary. It could also be used in lieu of a rewrite
2322 : // compaction as part of a Download() call, which allows copying only a span of
2323 : // the external file, provided the file does not contain range keys or value
2324 : // blocks (see sstable.CopySpan).
2325 : //
2326 : // d.mu must be held when calling this method. The mutex will be released when
2327 : // doing IO.
2328 : func (d *DB) runCopyCompaction(
2329 : jobID JobID, c *compaction,
2330 1 : ) (ve *versionEdit, stats compact.Stats, _ error) {
2331 1 : iter := c.startLevel.files.Iter()
2332 1 : inputMeta := iter.First()
2333 1 : if iter.Next() != nil {
2334 0 : return nil, compact.Stats{}, base.AssertionFailedf("got more than one file for a move compaction")
2335 0 : }
2336 1 : if c.cancel.Load() {
2337 0 : return nil, compact.Stats{}, ErrCancelledCompaction
2338 0 : }
2339 1 : ve = &versionEdit{
2340 1 : DeletedFiles: map[deletedFileEntry]*fileMetadata{
2341 1 : {Level: c.startLevel.level, FileNum: inputMeta.FileNum}: inputMeta,
2342 1 : },
2343 1 : }
2344 1 :
2345 1 : objMeta, err := d.objProvider.Lookup(fileTypeTable, inputMeta.FileBacking.DiskFileNum)
2346 1 : if err != nil {
2347 0 : return nil, compact.Stats{}, err
2348 0 : }
2349 1 : if !objMeta.IsExternal() {
2350 1 : if objMeta.IsRemote() || !remote.ShouldCreateShared(d.opts.Experimental.CreateOnShared, c.outputLevel.level) {
2351 0 : panic("pebble: scheduled a copy compaction that is not actually moving files to shared storage")
2352 : }
2353 : // Note that based on logic in the compaction picker, we're guaranteed
2354 : // inputMeta.Virtual is false.
2355 1 : if inputMeta.Virtual {
2356 0 : panic(errors.AssertionFailedf("cannot do a copy compaction of a virtual sstable across local/remote storage"))
2357 : }
2358 : }
2359 :
2360 : // We are in the relatively more complex case where we need to copy this
2361 : // file to remote storage. Drop the db mutex while we do the copy
2362 : //
2363 : // To ease up cleanup of the local file and tracking of refs, we create
2364 : // a new FileNum. This has the potential of making the block cache less
2365 : // effective, however.
2366 1 : newMeta := &fileMetadata{
2367 1 : Size: inputMeta.Size,
2368 1 : CreationTime: inputMeta.CreationTime,
2369 1 : SmallestSeqNum: inputMeta.SmallestSeqNum,
2370 1 : LargestSeqNum: inputMeta.LargestSeqNum,
2371 1 : LargestSeqNumAbsolute: inputMeta.LargestSeqNumAbsolute,
2372 1 : Stats: inputMeta.Stats,
2373 1 : Virtual: inputMeta.Virtual,
2374 1 : SyntheticPrefixAndSuffix: inputMeta.SyntheticPrefixAndSuffix,
2375 1 : }
2376 1 : if inputMeta.HasPointKeys {
2377 1 : newMeta.ExtendPointKeyBounds(c.cmp, inputMeta.SmallestPointKey, inputMeta.LargestPointKey)
2378 1 : }
2379 1 : if inputMeta.HasRangeKeys {
2380 1 : newMeta.ExtendRangeKeyBounds(c.cmp, inputMeta.SmallestRangeKey, inputMeta.LargestRangeKey)
2381 1 : }
2382 1 : newMeta.FileNum = d.mu.versions.getNextFileNum()
2383 1 : if objMeta.IsExternal() {
2384 1 : // external -> local/shared copy. File must be virtual.
2385 1 : // We will update this size later after we produce the new backing file.
2386 1 : newMeta.InitProviderBacking(base.DiskFileNum(newMeta.FileNum), inputMeta.FileBacking.Size)
2387 1 : } else {
2388 1 : // local -> shared copy. New file is guaranteed to not be virtual.
2389 1 : newMeta.InitPhysicalBacking()
2390 1 : }
2391 :
2392 : // Before dropping the db mutex, grab a ref to the current version. This
2393 : // prevents any concurrent excises from deleting files that this compaction
2394 : // needs to read/maintain a reference to.
2395 1 : vers := d.mu.versions.currentVersion()
2396 1 : vers.Ref()
2397 1 : defer vers.UnrefLocked()
2398 1 :
2399 1 : // NB: The order here is reversed, lock after unlock. This is similar to
2400 1 : // runCompaction.
2401 1 : d.mu.Unlock()
2402 1 : defer d.mu.Lock()
2403 1 :
2404 1 : deleteOnExit := false
2405 1 : defer func() {
2406 1 : if deleteOnExit {
2407 1 : _ = d.objProvider.Remove(fileTypeTable, newMeta.FileBacking.DiskFileNum)
2408 1 : }
2409 : }()
2410 :
2411 : // If the src obj is external, we're doing an external to local/shared copy.
2412 1 : if objMeta.IsExternal() {
2413 1 : ctx := context.TODO()
2414 1 : src, err := d.objProvider.OpenForReading(
2415 1 : ctx, fileTypeTable, inputMeta.FileBacking.DiskFileNum, objstorage.OpenOptions{},
2416 1 : )
2417 1 : if err != nil {
2418 0 : return nil, compact.Stats{}, err
2419 0 : }
2420 1 : defer func() {
2421 1 : if src != nil {
2422 0 : src.Close()
2423 0 : }
2424 : }()
2425 :
2426 1 : w, _, err := d.objProvider.Create(
2427 1 : ctx, fileTypeTable, newMeta.FileBacking.DiskFileNum,
2428 1 : objstorage.CreateOptions{
2429 1 : PreferSharedStorage: remote.ShouldCreateShared(d.opts.Experimental.CreateOnShared, c.outputLevel.level),
2430 1 : },
2431 1 : )
2432 1 : if err != nil {
2433 0 : return nil, compact.Stats{}, err
2434 0 : }
2435 1 : deleteOnExit = true
2436 1 :
2437 1 : start, end := newMeta.Smallest, newMeta.Largest
2438 1 : if newMeta.SyntheticPrefixAndSuffix.HasPrefix() {
2439 1 : syntheticPrefix := newMeta.SyntheticPrefixAndSuffix.Prefix()
2440 1 : start.UserKey = syntheticPrefix.Invert(start.UserKey)
2441 1 : end.UserKey = syntheticPrefix.Invert(end.UserKey)
2442 1 : }
2443 1 : if newMeta.SyntheticPrefixAndSuffix.HasSuffix() {
2444 0 : // Extend the bounds as necessary so that the keys don't include suffixes.
2445 0 : start.UserKey = start.UserKey[:c.comparer.Split(start.UserKey)]
2446 0 : if n := c.comparer.Split(end.UserKey); n < len(end.UserKey) {
2447 0 : end = base.MakeRangeDeleteSentinelKey(c.comparer.ImmediateSuccessor(nil, end.UserKey[:n]))
2448 0 : }
2449 : }
2450 :
2451 : // NB: external files are always virtual.
2452 1 : var wrote uint64
2453 1 : err = d.tableCache.withVirtualReader(inputMeta.VirtualMeta(), func(r sstable.VirtualReader) error {
2454 1 : var err error
2455 1 : wrote, err = sstable.CopySpan(ctx,
2456 1 : src, r.UnsafeReader(), d.opts.MakeReaderOptions(),
2457 1 : w, d.opts.MakeWriterOptions(c.outputLevel.level, d.TableFormat()),
2458 1 : start, end,
2459 1 : )
2460 1 : return err
2461 1 : })
2462 :
2463 1 : src = nil // We passed src to CopySpan; it's responsible for closing it.
2464 1 : if err != nil {
2465 1 : if errors.Is(err, sstable.ErrEmptySpan) {
2466 1 : // The virtual table was empty. Just remove the backing file.
2467 1 : // Note that deleteOnExit is true so we will delete the created object.
2468 1 : c.metrics = map[int]*LevelMetrics{
2469 1 : c.outputLevel.level: {
2470 1 : BytesIn: inputMeta.Size,
2471 1 : },
2472 1 : }
2473 1 : return ve, compact.Stats{}, nil
2474 1 : }
2475 0 : return nil, compact.Stats{}, err
2476 : }
2477 1 : newMeta.FileBacking.Size = wrote
2478 1 : newMeta.Size = wrote
2479 1 : } else {
2480 1 : _, err := d.objProvider.LinkOrCopyFromLocal(context.TODO(), d.opts.FS,
2481 1 : d.objProvider.Path(objMeta), fileTypeTable, newMeta.FileBacking.DiskFileNum,
2482 1 : objstorage.CreateOptions{PreferSharedStorage: true})
2483 1 : if err != nil {
2484 0 : return nil, compact.Stats{}, err
2485 0 : }
2486 1 : deleteOnExit = true
2487 : }
2488 1 : ve.NewFiles = []newFileEntry{{
2489 1 : Level: c.outputLevel.level,
2490 1 : Meta: newMeta,
2491 1 : }}
2492 1 : if newMeta.Virtual {
2493 1 : ve.CreatedBackingTables = []*fileBacking{newMeta.FileBacking}
2494 1 : }
2495 1 : c.metrics = map[int]*LevelMetrics{
2496 1 : c.outputLevel.level: {
2497 1 : BytesIn: inputMeta.Size,
2498 1 : BytesCompacted: newMeta.Size,
2499 1 : TablesCompacted: 1,
2500 1 : },
2501 1 : }
2502 1 :
2503 1 : if err := d.objProvider.Sync(); err != nil {
2504 0 : return nil, compact.Stats{}, err
2505 0 : }
2506 1 : deleteOnExit = false
2507 1 : return ve, compact.Stats{}, nil
2508 : }
2509 :
2510 : // applyHintOnFile applies a deleteCompactionHint to a file, and updates the
2511 : // versionEdit accordingly. It returns a list of new files that were created
2512 : // if the hint was applied partially to a file (eg. through an excise as opposed
2513 : // to an outright deletion). levelMetrics is kept up-to-date with the number
2514 : // of tables deleted or excised.
2515 : func (d *DB) applyHintOnFile(
2516 : h deleteCompactionHint,
2517 : f *fileMetadata,
2518 : level int,
2519 : levelMetrics *LevelMetrics,
2520 : ve *versionEdit,
2521 : hintOverlap deletionHintOverlap,
2522 1 : ) (newFiles []manifest.NewFileEntry, err error) {
2523 1 : if hintOverlap == hintDoesNotApply {
2524 0 : return nil, nil
2525 0 : }
2526 :
2527 : // The hint overlaps with at least part of the file.
2528 1 : if hintOverlap == hintDeletesFile {
2529 1 : // The hint deletes the entirety of this file.
2530 1 : ve.DeletedFiles[deletedFileEntry{
2531 1 : Level: level,
2532 1 : FileNum: f.FileNum,
2533 1 : }] = f
2534 1 : levelMetrics.TablesDeleted++
2535 1 : return nil, nil
2536 1 : }
2537 : // The hint overlaps with only a part of the file, not the entirety of it. We need
2538 : // to use d.excise. (hintOverlap == hintExcisesFile)
2539 1 : if d.FormatMajorVersion() < FormatVirtualSSTables {
2540 0 : panic("pebble: delete-only compaction hint excising a file is not supported in this version")
2541 : }
2542 :
2543 1 : levelMetrics.TablesExcised++
2544 1 : newFiles, err = d.excise(context.TODO(), base.UserKeyBoundsEndExclusive(h.start, h.end), f, ve, level)
2545 1 : if err != nil {
2546 0 : return nil, errors.Wrap(err, "error when running excise for delete-only compaction")
2547 0 : }
2548 1 : if _, ok := ve.DeletedFiles[deletedFileEntry{
2549 1 : Level: level,
2550 1 : FileNum: f.FileNum,
2551 1 : }]; !ok {
2552 0 : panic("pebble: delete-only compaction hint overlapping a file did not excise that file")
2553 : }
2554 1 : return newFiles, nil
2555 : }
2556 :
2557 : func (d *DB) runDeleteOnlyCompactionForLevel(
2558 : cl compactionLevel,
2559 : levelMetrics *LevelMetrics,
2560 : ve *versionEdit,
2561 : snapshots compact.Snapshots,
2562 : fragments []deleteCompactionHintFragment,
2563 : exciseEnabled bool,
2564 1 : ) error {
2565 1 : curFragment := 0
2566 1 : iter := cl.files.Iter()
2567 1 : if cl.level == 0 {
2568 0 : panic("cannot run delete-only compaction for L0")
2569 : }
2570 :
2571 : // Outer loop loops on files. Middle loop loops on fragments. Inner loop
2572 : // loops on raw fragments of hints. Number of fragments are bounded by
2573 : // the number of hints this compaction was created with, which is capped
2574 : // in the compaction picker to avoid very CPU-hot loops here.
2575 1 : for f := iter.First(); f != nil; f = iter.Next() {
2576 1 : // curFile usually matches f, except if f got excised in which case
2577 1 : // it maps to a virtual file that replaces f, or nil if f got removed
2578 1 : // in its entirety.
2579 1 : curFile := f
2580 1 : for curFragment < len(fragments) && d.cmp(fragments[curFragment].start, f.Smallest.UserKey) <= 0 {
2581 1 : curFragment++
2582 1 : }
2583 1 : if curFragment > 0 {
2584 1 : curFragment--
2585 1 : }
2586 :
2587 1 : for ; curFragment < len(fragments); curFragment++ {
2588 1 : if f.UserKeyBounds().End.CompareUpperBounds(d.cmp, base.UserKeyInclusive(fragments[curFragment].start)) < 0 {
2589 1 : break
2590 : }
2591 : // Process all overlapping hints with this file. Note that applying
2592 : // a hint twice is idempotent; curFile should have already been excised
2593 : // the first time, resulting in no change the second time.
2594 1 : for _, h := range fragments[curFragment].hints {
2595 1 : if h.tombstoneLevel >= cl.level {
2596 1 : // We cannot excise out the deletion tombstone itself, or anything
2597 1 : // above it.
2598 1 : continue
2599 : }
2600 1 : hintOverlap := h.canDeleteOrExcise(d.cmp, curFile, snapshots, exciseEnabled)
2601 1 : if hintOverlap == hintDoesNotApply {
2602 1 : continue
2603 : }
2604 1 : newFiles, err := d.applyHintOnFile(h, curFile, cl.level, levelMetrics, ve, hintOverlap)
2605 1 : if err != nil {
2606 0 : return err
2607 0 : }
2608 1 : if _, ok := ve.DeletedFiles[manifest.DeletedFileEntry{Level: cl.level, FileNum: curFile.FileNum}]; ok {
2609 1 : curFile = nil
2610 1 : }
2611 1 : if len(newFiles) > 0 {
2612 1 : curFile = newFiles[len(newFiles)-1].Meta
2613 1 : } else if curFile == nil {
2614 1 : // Nothing remains of the file.
2615 1 : break
2616 : }
2617 : }
2618 1 : if curFile == nil {
2619 1 : // Nothing remains of the file.
2620 1 : break
2621 : }
2622 : }
2623 1 : if _, ok := ve.DeletedFiles[deletedFileEntry{
2624 1 : Level: cl.level,
2625 1 : FileNum: f.FileNum,
2626 1 : }]; !ok {
2627 0 : panic("pebble: delete-only compaction scheduled with hints that did not delete or excise a file")
2628 : }
2629 : }
2630 1 : return nil
2631 : }
2632 :
2633 : // deleteCompactionHintFragment represents a fragment of the key space and
2634 : // contains a set of deleteCompactionHints that apply to that fragment; a
2635 : // fragment starts at the start field and ends where the next fragment starts.
2636 : type deleteCompactionHintFragment struct {
2637 : start []byte
2638 : hints []deleteCompactionHint
2639 : }
2640 :
2641 : // Delete compaction hints can overlap with each other, and multiple fragments
2642 : // can apply to a single file. This function takes a list of hints and fragments
2643 : // them, to make it easier to apply them to non-overlapping files occupying a level;
2644 : // that way, files and hint fragments can be iterated on in lockstep, while efficiently
2645 : // being able to apply all hints overlapping with a given file.
2646 : func fragmentDeleteCompactionHints(
2647 : cmp Compare, hints []deleteCompactionHint,
2648 1 : ) []deleteCompactionHintFragment {
2649 1 : fragments := make([]deleteCompactionHintFragment, 0, len(hints)*2)
2650 1 : for i := range hints {
2651 1 : fragments = append(fragments, deleteCompactionHintFragment{start: hints[i].start},
2652 1 : deleteCompactionHintFragment{start: hints[i].end})
2653 1 : }
2654 1 : slices.SortFunc(fragments, func(i, j deleteCompactionHintFragment) int {
2655 1 : return cmp(i.start, j.start)
2656 1 : })
2657 1 : fragments = slices.CompactFunc(fragments, func(i, j deleteCompactionHintFragment) bool {
2658 1 : return bytes.Equal(i.start, j.start)
2659 1 : })
2660 1 : for _, h := range hints {
2661 1 : startIdx := sort.Search(len(fragments), func(i int) bool {
2662 1 : return cmp(fragments[i].start, h.start) >= 0
2663 1 : })
2664 1 : endIdx := sort.Search(len(fragments), func(i int) bool {
2665 1 : return cmp(fragments[i].start, h.end) >= 0
2666 1 : })
2667 1 : for i := startIdx; i < endIdx; i++ {
2668 1 : fragments[i].hints = append(fragments[i].hints, h)
2669 1 : }
2670 : }
2671 1 : return fragments
2672 : }
2673 :
2674 : // Runs a delete-only compaction.
2675 : //
2676 : // d.mu must *not* be held when calling this.
2677 : func (d *DB) runDeleteOnlyCompaction(
2678 : jobID JobID, c *compaction, snapshots compact.Snapshots,
2679 1 : ) (ve *versionEdit, stats compact.Stats, retErr error) {
2680 1 : c.metrics = make(map[int]*LevelMetrics, len(c.inputs))
2681 1 : fragments := fragmentDeleteCompactionHints(d.cmp, c.deletionHints)
2682 1 : ve = &versionEdit{
2683 1 : DeletedFiles: map[deletedFileEntry]*fileMetadata{},
2684 1 : }
2685 1 : for _, cl := range c.inputs {
2686 1 : levelMetrics := &LevelMetrics{}
2687 1 : if err := d.runDeleteOnlyCompactionForLevel(cl, levelMetrics, ve, snapshots, fragments, c.exciseEnabled); err != nil {
2688 0 : return nil, stats, err
2689 0 : }
2690 1 : c.metrics[cl.level] = levelMetrics
2691 : }
2692 : // Remove any files that were added and deleted in the same versionEdit.
2693 1 : ve.NewFiles = slices.DeleteFunc(ve.NewFiles, func(e manifest.NewFileEntry) bool {
2694 1 : deletedFileEntry := manifest.DeletedFileEntry{Level: e.Level, FileNum: e.Meta.FileNum}
2695 1 : if _, deleted := ve.DeletedFiles[deletedFileEntry]; deleted {
2696 1 : delete(ve.DeletedFiles, deletedFileEntry)
2697 1 : return true
2698 1 : }
2699 1 : return false
2700 : })
2701 : // Remove any entries from CreatedBackingTables that are not used in any
2702 : // NewFiles.
2703 1 : usedBackingFiles := make(map[base.DiskFileNum]struct{})
2704 1 : for _, e := range ve.NewFiles {
2705 1 : if e.Meta.Virtual {
2706 1 : usedBackingFiles[e.Meta.FileBacking.DiskFileNum] = struct{}{}
2707 1 : }
2708 : }
2709 1 : ve.CreatedBackingTables = slices.DeleteFunc(ve.CreatedBackingTables, func(b *fileBacking) bool {
2710 1 : _, used := usedBackingFiles[b.DiskFileNum]
2711 1 : return !used
2712 1 : })
2713 : // Refresh the disk available statistic whenever a compaction/flush
2714 : // completes, before re-acquiring the mutex.
2715 1 : d.calculateDiskAvailableBytes()
2716 1 : return ve, stats, nil
2717 : }
2718 :
2719 : func (d *DB) runMoveCompaction(
2720 : jobID JobID, c *compaction,
2721 1 : ) (ve *versionEdit, stats compact.Stats, _ error) {
2722 1 : iter := c.startLevel.files.Iter()
2723 1 : meta := iter.First()
2724 1 : if iter.Next() != nil {
2725 0 : return nil, stats, base.AssertionFailedf("got more than one file for a move compaction")
2726 0 : }
2727 1 : if c.cancel.Load() {
2728 0 : return ve, stats, ErrCancelledCompaction
2729 0 : }
2730 1 : c.metrics = map[int]*LevelMetrics{
2731 1 : c.outputLevel.level: {
2732 1 : BytesMoved: meta.Size,
2733 1 : TablesMoved: 1,
2734 1 : },
2735 1 : }
2736 1 : ve = &versionEdit{
2737 1 : DeletedFiles: map[deletedFileEntry]*fileMetadata{
2738 1 : {Level: c.startLevel.level, FileNum: meta.FileNum}: meta,
2739 1 : },
2740 1 : NewFiles: []newFileEntry{
2741 1 : {Level: c.outputLevel.level, Meta: meta},
2742 1 : },
2743 1 : }
2744 1 :
2745 1 : return ve, stats, nil
2746 : }
2747 :
2748 : // runCompaction runs a compaction that produces new on-disk tables from
2749 : // memtables or old on-disk tables.
2750 : //
2751 : // runCompaction cannot be used for compactionKindIngestedFlushable.
2752 : //
2753 : // d.mu must be held when calling this, but the mutex may be dropped and
2754 : // re-acquired during the course of this method.
2755 : func (d *DB) runCompaction(
2756 : jobID JobID, c *compaction,
2757 1 : ) (ve *versionEdit, stats compact.Stats, retErr error) {
2758 1 : if c.cancel.Load() {
2759 1 : return ve, stats, ErrCancelledCompaction
2760 1 : }
2761 1 : switch c.kind {
2762 1 : case compactionKindDeleteOnly:
2763 1 : // Before dropping the db mutex, grab a ref to the current version. This
2764 1 : // prevents any concurrent excises from deleting files that this compaction
2765 1 : // needs to read/maintain a reference to.
2766 1 : //
2767 1 : // Note that delete-only compactions can call excise(), which needs to be able
2768 1 : // to read these files.
2769 1 : vers := d.mu.versions.currentVersion()
2770 1 : vers.Ref()
2771 1 : defer vers.UnrefLocked()
2772 1 : // Release the d.mu lock while doing I/O.
2773 1 : // Note the unusual order: Unlock and then Lock.
2774 1 : snapshots := d.mu.snapshots.toSlice()
2775 1 : d.mu.Unlock()
2776 1 : defer d.mu.Lock()
2777 1 : return d.runDeleteOnlyCompaction(jobID, c, snapshots)
2778 1 : case compactionKindMove:
2779 1 : return d.runMoveCompaction(jobID, c)
2780 1 : case compactionKindCopy:
2781 1 : return d.runCopyCompaction(jobID, c)
2782 0 : case compactionKindIngestedFlushable:
2783 0 : panic("pebble: runCompaction cannot handle compactionKindIngestedFlushable.")
2784 : }
2785 :
2786 1 : snapshots := d.mu.snapshots.toSlice()
2787 1 :
2788 1 : if c.flushing == nil {
2789 1 : // Before dropping the db mutex, grab a ref to the current version. This
2790 1 : // prevents any concurrent excises from deleting files that this compaction
2791 1 : // needs to read/maintain a reference to.
2792 1 : //
2793 1 : // Note that unlike user iterators, compactionIter does not maintain a ref
2794 1 : // of the version or read state.
2795 1 : vers := d.mu.versions.currentVersion()
2796 1 : vers.Ref()
2797 1 : defer vers.UnrefLocked()
2798 1 : }
2799 :
2800 : // The table is typically written at the maximum allowable format implied by
2801 : // the current format major version of the DB, but Options may define
2802 : // additional constraints.
2803 1 : tableFormat := d.TableFormat()
2804 1 :
2805 1 : // Release the d.mu lock while doing I/O.
2806 1 : // Note the unusual order: Unlock and then Lock.
2807 1 : d.mu.Unlock()
2808 1 : defer d.mu.Lock()
2809 1 :
2810 1 : result := d.compactAndWrite(jobID, c, snapshots, tableFormat)
2811 1 : if result.Err == nil {
2812 1 : ve, result.Err = c.makeVersionEdit(result)
2813 1 : }
2814 1 : if result.Err != nil {
2815 1 : // Delete any created tables.
2816 1 : obsoleteFiles := make([]*fileBacking, 0, len(result.Tables))
2817 1 : d.mu.Lock()
2818 1 : for i := range result.Tables {
2819 1 : backing := &fileBacking{
2820 1 : DiskFileNum: result.Tables[i].ObjMeta.DiskFileNum,
2821 1 : Size: result.Tables[i].WriterMeta.Size,
2822 1 : }
2823 1 : obsoleteFiles = append(obsoleteFiles, backing)
2824 1 : // Add this file to zombie tables as well, as the versionSet
2825 1 : // asserts on whether every obsolete file was at one point
2826 1 : // marked zombie.
2827 1 : d.mu.versions.zombieTables[backing.DiskFileNum] = tableInfo{
2828 1 : fileInfo: fileInfo{
2829 1 : FileNum: backing.DiskFileNum,
2830 1 : FileSize: backing.Size,
2831 1 : },
2832 1 : isLocal: true,
2833 1 : }
2834 1 : }
2835 1 : d.mu.versions.addObsoleteLocked(obsoleteFiles)
2836 1 : d.mu.Unlock()
2837 : }
2838 : // Refresh the disk available statistic whenever a compaction/flush
2839 : // completes, before re-acquiring the mutex.
2840 1 : d.calculateDiskAvailableBytes()
2841 1 : return ve, result.Stats, result.Err
2842 : }
2843 :
2844 : // compactAndWrite runs the data part of a compaction, where we set up a
2845 : // compaction iterator and use it to write output tables.
2846 : func (d *DB) compactAndWrite(
2847 : jobID JobID, c *compaction, snapshots compact.Snapshots, tableFormat sstable.TableFormat,
2848 1 : ) (result compact.Result) {
2849 1 : // Compactions use a pool of buffers to read blocks, avoiding polluting the
2850 1 : // block cache with blocks that will not be read again. We initialize the
2851 1 : // buffer pool with a size 12. This initial size does not need to be
2852 1 : // accurate, because the pool will grow to accommodate the maximum number of
2853 1 : // blocks allocated at a given time over the course of the compaction. But
2854 1 : // choosing a size larger than that working set avoids any additional
2855 1 : // allocations to grow the size of the pool over the course of iteration.
2856 1 : //
2857 1 : // Justification for initial size 12: In a two-level compaction, at any
2858 1 : // given moment we'll have 2 index blocks in-use and 2 data blocks in-use.
2859 1 : // Additionally, when decoding a compressed block, we'll temporarily
2860 1 : // allocate 1 additional block to hold the compressed buffer. In the worst
2861 1 : // case that all input sstables have two-level index blocks (+2), value
2862 1 : // blocks (+2), range deletion blocks (+n) and range key blocks (+n), we'll
2863 1 : // additionally require 2n+4 blocks where n is the number of input sstables.
2864 1 : // Range deletion and range key blocks are relatively rare, and the cost of
2865 1 : // an additional allocation or two over the course of the compaction is
2866 1 : // considered to be okay. A larger initial size would cause the pool to hold
2867 1 : // on to more memory, even when it's not in-use because the pool will
2868 1 : // recycle buffers up to the current capacity of the pool. The memory use of
2869 1 : // a 12-buffer pool is expected to be within reason, even if all the buffers
2870 1 : // grow to the typical size of an index block (256 KiB) which would
2871 1 : // translate to 3 MiB per compaction.
2872 1 : c.bufferPool.Init(12)
2873 1 : defer c.bufferPool.Release()
2874 1 :
2875 1 : pointIter, rangeDelIter, rangeKeyIter, err := c.newInputIters(d.newIters, d.tableNewRangeKeyIter)
2876 1 : defer func() {
2877 1 : for _, closer := range c.closers {
2878 1 : closer.FragmentIterator.Close()
2879 1 : }
2880 : }()
2881 1 : if err != nil {
2882 0 : return compact.Result{Err: err}
2883 0 : }
2884 1 : c.allowedZeroSeqNum = c.allowZeroSeqNum()
2885 1 : cfg := compact.IterConfig{
2886 1 : Comparer: c.comparer,
2887 1 : Merge: d.merge,
2888 1 : TombstoneElision: c.delElision,
2889 1 : RangeKeyElision: c.rangeKeyElision,
2890 1 : Snapshots: snapshots,
2891 1 : AllowZeroSeqNum: c.allowedZeroSeqNum,
2892 1 : IneffectualSingleDeleteCallback: d.opts.Experimental.IneffectualSingleDeleteCallback,
2893 1 : SingleDeleteInvariantViolationCallback: d.opts.Experimental.SingleDeleteInvariantViolationCallback,
2894 1 : }
2895 1 : iter := compact.NewIter(cfg, pointIter, rangeDelIter, rangeKeyIter)
2896 1 :
2897 1 : runnerCfg := compact.RunnerConfig{
2898 1 : CompactionBounds: base.UserKeyBoundsFromInternal(c.smallest, c.largest),
2899 1 : L0SplitKeys: c.l0Limits,
2900 1 : Grandparents: c.grandparents,
2901 1 : MaxGrandparentOverlapBytes: c.maxOverlapBytes,
2902 1 : TargetOutputFileSize: c.maxOutputFileSize,
2903 1 : }
2904 1 : runner := compact.NewRunner(runnerCfg, iter)
2905 1 : for runner.MoreDataToWrite() {
2906 1 : if c.cancel.Load() {
2907 1 : return runner.Finish().WithError(ErrCancelledCompaction)
2908 1 : }
2909 : // Create a new table.
2910 1 : writerOpts := d.opts.MakeWriterOptions(c.outputLevel.level, tableFormat)
2911 1 : objMeta, tw, cpuWorkHandle, err := d.newCompactionOutput(jobID, c, writerOpts)
2912 1 : if err != nil {
2913 1 : return runner.Finish().WithError(err)
2914 1 : }
2915 1 : runner.WriteTable(objMeta, tw)
2916 1 : d.opts.Experimental.CPUWorkPermissionGranter.CPUWorkDone(cpuWorkHandle)
2917 : }
2918 1 : result = runner.Finish()
2919 1 : if result.Err == nil {
2920 1 : result.Err = d.objProvider.Sync()
2921 1 : }
2922 1 : return result
2923 : }
2924 :
2925 : // makeVersionEdit creates the version edit for a compaction, based on the
2926 : // tables in compact.Result.
2927 1 : func (c *compaction) makeVersionEdit(result compact.Result) (*versionEdit, error) {
2928 1 : ve := &versionEdit{
2929 1 : DeletedFiles: map[deletedFileEntry]*fileMetadata{},
2930 1 : }
2931 1 : for _, cl := range c.inputs {
2932 1 : iter := cl.files.Iter()
2933 1 : for f := iter.First(); f != nil; f = iter.Next() {
2934 1 : ve.DeletedFiles[deletedFileEntry{
2935 1 : Level: cl.level,
2936 1 : FileNum: f.FileNum,
2937 1 : }] = f
2938 1 : }
2939 : }
2940 :
2941 1 : startLevelBytes := c.startLevel.files.SizeSum()
2942 1 : outputMetrics := &LevelMetrics{
2943 1 : BytesIn: startLevelBytes,
2944 1 : BytesRead: c.outputLevel.files.SizeSum(),
2945 1 : }
2946 1 : if len(c.extraLevels) > 0 {
2947 1 : outputMetrics.BytesIn += c.extraLevels[0].files.SizeSum()
2948 1 : }
2949 1 : outputMetrics.BytesRead += outputMetrics.BytesIn
2950 1 :
2951 1 : c.metrics = map[int]*LevelMetrics{
2952 1 : c.outputLevel.level: outputMetrics,
2953 1 : }
2954 1 : if len(c.flushing) == 0 && c.metrics[c.startLevel.level] == nil {
2955 1 : c.metrics[c.startLevel.level] = &LevelMetrics{}
2956 1 : }
2957 1 : if len(c.extraLevels) > 0 {
2958 1 : c.metrics[c.extraLevels[0].level] = &LevelMetrics{}
2959 1 : outputMetrics.MultiLevel.BytesInTop = startLevelBytes
2960 1 : outputMetrics.MultiLevel.BytesIn = outputMetrics.BytesIn
2961 1 : outputMetrics.MultiLevel.BytesRead = outputMetrics.BytesRead
2962 1 : }
2963 :
2964 1 : inputLargestSeqNumAbsolute := c.inputLargestSeqNumAbsolute()
2965 1 : ve.NewFiles = make([]newFileEntry, len(result.Tables))
2966 1 : for i := range result.Tables {
2967 1 : t := &result.Tables[i]
2968 1 :
2969 1 : fileMeta := &fileMetadata{
2970 1 : FileNum: base.PhysicalTableFileNum(t.ObjMeta.DiskFileNum),
2971 1 : CreationTime: t.CreationTime.Unix(),
2972 1 : Size: t.WriterMeta.Size,
2973 1 : SmallestSeqNum: t.WriterMeta.SmallestSeqNum,
2974 1 : LargestSeqNum: t.WriterMeta.LargestSeqNum,
2975 1 : }
2976 1 : if c.flushing == nil {
2977 1 : // Set the file's LargestSeqNumAbsolute to be the maximum value of any
2978 1 : // of the compaction's input sstables.
2979 1 : // TODO(jackson): This could be narrowed to be the maximum of input
2980 1 : // sstables that overlap the output sstable's key range.
2981 1 : fileMeta.LargestSeqNumAbsolute = inputLargestSeqNumAbsolute
2982 1 : } else {
2983 1 : fileMeta.LargestSeqNumAbsolute = t.WriterMeta.LargestSeqNum
2984 1 : }
2985 1 : fileMeta.InitPhysicalBacking()
2986 1 :
2987 1 : // If the file didn't contain any range deletions, we can fill its
2988 1 : // table stats now, avoiding unnecessarily loading the table later.
2989 1 : maybeSetStatsFromProperties(
2990 1 : fileMeta.PhysicalMeta(), &t.WriterMeta.Properties,
2991 1 : )
2992 1 :
2993 1 : if t.WriterMeta.HasPointKeys {
2994 1 : fileMeta.ExtendPointKeyBounds(c.cmp, t.WriterMeta.SmallestPoint, t.WriterMeta.LargestPoint)
2995 1 : }
2996 1 : if t.WriterMeta.HasRangeDelKeys {
2997 1 : fileMeta.ExtendPointKeyBounds(c.cmp, t.WriterMeta.SmallestRangeDel, t.WriterMeta.LargestRangeDel)
2998 1 : }
2999 1 : if t.WriterMeta.HasRangeKeys {
3000 1 : fileMeta.ExtendRangeKeyBounds(c.cmp, t.WriterMeta.SmallestRangeKey, t.WriterMeta.LargestRangeKey)
3001 1 : }
3002 :
3003 1 : ve.NewFiles[i] = newFileEntry{
3004 1 : Level: c.outputLevel.level,
3005 1 : Meta: fileMeta,
3006 1 : }
3007 1 :
3008 1 : // Update metrics.
3009 1 : if c.flushing == nil {
3010 1 : outputMetrics.TablesCompacted++
3011 1 : outputMetrics.BytesCompacted += fileMeta.Size
3012 1 : } else {
3013 1 : outputMetrics.TablesFlushed++
3014 1 : outputMetrics.BytesFlushed += fileMeta.Size
3015 1 : }
3016 1 : outputMetrics.Size += int64(fileMeta.Size)
3017 1 : outputMetrics.NumFiles++
3018 1 : outputMetrics.Additional.BytesWrittenDataBlocks += t.WriterMeta.Properties.DataSize
3019 1 : outputMetrics.Additional.BytesWrittenValueBlocks += t.WriterMeta.Properties.ValueBlocksSize
3020 : }
3021 :
3022 : // Sanity check that the tables are ordered and don't overlap.
3023 1 : for i := 1; i < len(ve.NewFiles); i++ {
3024 1 : if ve.NewFiles[i-1].Meta.UserKeyBounds().End.IsUpperBoundFor(c.cmp, ve.NewFiles[i].Meta.Smallest.UserKey) {
3025 0 : return nil, base.AssertionFailedf("pebble: compaction output tables overlap: %s and %s",
3026 0 : ve.NewFiles[i-1].Meta.DebugString(c.formatKey, true),
3027 0 : ve.NewFiles[i].Meta.DebugString(c.formatKey, true),
3028 0 : )
3029 0 : }
3030 : }
3031 :
3032 1 : return ve, nil
3033 : }
3034 :
3035 : // newCompactionOutput creates an object for a new table produced by a
3036 : // compaction or flush.
3037 : func (d *DB) newCompactionOutput(
3038 : jobID JobID, c *compaction, writerOpts sstable.WriterOptions,
3039 1 : ) (objstorage.ObjectMetadata, sstable.RawWriter, CPUWorkHandle, error) {
3040 1 : diskFileNum := d.mu.versions.getNextDiskFileNum()
3041 1 :
3042 1 : var writeCategory vfs.DiskWriteCategory
3043 1 : if d.opts.EnableSQLRowSpillMetrics {
3044 0 : // In the scenario that the Pebble engine is used for SQL row spills the
3045 0 : // data written to the memtable will correspond to spills to disk and
3046 0 : // should be categorized as such.
3047 0 : writeCategory = "sql-row-spill"
3048 1 : } else if c.kind == compactionKindFlush {
3049 1 : writeCategory = "pebble-memtable-flush"
3050 1 : } else {
3051 1 : writeCategory = "pebble-compaction"
3052 1 : }
3053 :
3054 1 : var reason string
3055 1 : if c.kind == compactionKindFlush {
3056 1 : reason = "flushing"
3057 1 : } else {
3058 1 : reason = "compacting"
3059 1 : }
3060 :
3061 1 : ctx := context.TODO()
3062 1 : if objiotracing.Enabled {
3063 0 : ctx = objiotracing.WithLevel(ctx, c.outputLevel.level)
3064 0 : if c.kind == compactionKindFlush {
3065 0 : ctx = objiotracing.WithReason(ctx, objiotracing.ForFlush)
3066 0 : } else {
3067 0 : ctx = objiotracing.WithReason(ctx, objiotracing.ForCompaction)
3068 0 : }
3069 : }
3070 :
3071 : // Prefer shared storage if present.
3072 1 : createOpts := objstorage.CreateOptions{
3073 1 : PreferSharedStorage: remote.ShouldCreateShared(d.opts.Experimental.CreateOnShared, c.outputLevel.level),
3074 1 : WriteCategory: writeCategory,
3075 1 : }
3076 1 : writable, objMeta, err := d.objProvider.Create(ctx, fileTypeTable, diskFileNum, createOpts)
3077 1 : if err != nil {
3078 1 : return objstorage.ObjectMetadata{}, nil, nil, err
3079 1 : }
3080 :
3081 1 : if c.kind != compactionKindFlush {
3082 1 : writable = &compactionWritable{
3083 1 : Writable: writable,
3084 1 : versions: d.mu.versions,
3085 1 : written: &c.bytesWritten,
3086 1 : }
3087 1 : }
3088 1 : d.opts.EventListener.TableCreated(TableCreateInfo{
3089 1 : JobID: int(jobID),
3090 1 : Reason: reason,
3091 1 : Path: d.objProvider.Path(objMeta),
3092 1 : FileNum: diskFileNum,
3093 1 : })
3094 1 :
3095 1 : writerOpts.SetInternal(sstableinternal.WriterOptions{
3096 1 : CacheOpts: sstableinternal.CacheOptions{
3097 1 : Cache: d.opts.Cache,
3098 1 : CacheID: d.cacheID,
3099 1 : FileNum: diskFileNum,
3100 1 : },
3101 1 : })
3102 1 :
3103 1 : const MaxFileWriteAdditionalCPUTime = time.Millisecond * 100
3104 1 : cpuWorkHandle := d.opts.Experimental.CPUWorkPermissionGranter.GetPermission(
3105 1 : MaxFileWriteAdditionalCPUTime,
3106 1 : )
3107 1 : writerOpts.Parallelism =
3108 1 : d.opts.Experimental.MaxWriterConcurrency > 0 &&
3109 1 : (cpuWorkHandle.Permitted() || d.opts.Experimental.ForceWriterParallelism)
3110 1 :
3111 1 : // TODO(jackson): Make the compaction body generic over the RawWriter type,
3112 1 : // so that we don't need to pay the cost of dynamic dispatch?
3113 1 : tw := sstable.NewRawWriter(writable, writerOpts)
3114 1 : return objMeta, tw, cpuWorkHandle, nil
3115 : }
3116 :
3117 : // validateVersionEdit validates that start and end keys across new and deleted
3118 : // files in a versionEdit pass the given validation function.
3119 : func validateVersionEdit(
3120 : ve *versionEdit, validateFn func([]byte) error, format base.FormatKey, logger Logger,
3121 1 : ) {
3122 1 : validateKey := func(f *manifest.FileMetadata, key []byte) {
3123 1 : if err := validateFn(key); err != nil {
3124 1 : logger.Fatalf("pebble: version edit validation failed (key=%s file=%s): %v", format(key), f, err)
3125 1 : }
3126 : }
3127 :
3128 : // Validate both new and deleted files.
3129 1 : for _, f := range ve.NewFiles {
3130 1 : validateKey(f.Meta, f.Meta.Smallest.UserKey)
3131 1 : validateKey(f.Meta, f.Meta.Largest.UserKey)
3132 1 : }
3133 1 : for _, m := range ve.DeletedFiles {
3134 1 : validateKey(m, m.Smallest.UserKey)
3135 1 : validateKey(m, m.Largest.UserKey)
3136 1 : }
3137 : }
|