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