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 : "cmp"
10 : "context"
11 : "fmt"
12 : "io"
13 : "math"
14 : "runtime/pprof"
15 : "slices"
16 : "sort"
17 : "sync/atomic"
18 : "time"
19 :
20 : "github.com/cockroachdb/errors"
21 : "github.com/cockroachdb/pebble/internal/base"
22 : "github.com/cockroachdb/pebble/internal/invalidating"
23 : "github.com/cockroachdb/pebble/internal/invariants"
24 : "github.com/cockroachdb/pebble/internal/keyspan"
25 : "github.com/cockroachdb/pebble/internal/manifest"
26 : "github.com/cockroachdb/pebble/internal/private"
27 : "github.com/cockroachdb/pebble/internal/rangedel"
28 : "github.com/cockroachdb/pebble/internal/rangekey"
29 : "github.com/cockroachdb/pebble/objstorage"
30 : "github.com/cockroachdb/pebble/objstorage/objstorageprovider/objiotracing"
31 : "github.com/cockroachdb/pebble/objstorage/remote"
32 : "github.com/cockroachdb/pebble/sstable"
33 : "github.com/cockroachdb/pebble/vfs"
34 : )
35 :
36 : var errEmptyTable = errors.New("pebble: empty table")
37 :
38 : // ErrCancelledCompaction is returned if a compaction is cancelled by a
39 : // concurrent excise or ingest-split operation.
40 : var ErrCancelledCompaction = errors.New("pebble: compaction cancelled by a concurrent operation, will retry compaction")
41 :
42 : var compactLabels = pprof.Labels("pebble", "compact")
43 : var flushLabels = pprof.Labels("pebble", "flush")
44 : var gcLabels = pprof.Labels("pebble", "gc")
45 :
46 : // getInternalWriterProperties accesses a private variable (in the
47 : // internal/private package) initialized by the sstable Writer. This indirection
48 : // is necessary to ensure non-Pebble users constructing sstables for ingestion
49 : // are unable to set internal-only properties.
50 : var getInternalWriterProperties = private.SSTableInternalProperties.(func(*sstable.Writer) *sstable.Properties)
51 :
52 : // expandedCompactionByteSizeLimit is the maximum number of bytes in all
53 : // compacted files. We avoid expanding the lower level file set of a compaction
54 : // if it would make the total compaction cover more than this many bytes.
55 2 : func expandedCompactionByteSizeLimit(opts *Options, level int, availBytes uint64) uint64 {
56 2 : v := uint64(25 * opts.Level(level).TargetFileSize)
57 2 :
58 2 : // Never expand a compaction beyond half the available capacity, divided
59 2 : // by the maximum number of concurrent compactions. Each of the concurrent
60 2 : // compactions may expand up to this limit, so this attempts to limit
61 2 : // compactions to half of available disk space. Note that this will not
62 2 : // prevent compaction picking from pursuing compactions that are larger
63 2 : // than this threshold before expansion.
64 2 : diskMax := (availBytes / 2) / uint64(opts.MaxConcurrentCompactions())
65 2 : if v > diskMax {
66 1 : v = diskMax
67 1 : }
68 2 : return v
69 : }
70 :
71 : // maxGrandparentOverlapBytes is the maximum bytes of overlap with level+1
72 : // before we stop building a single file in a level-1 to level compaction.
73 2 : func maxGrandparentOverlapBytes(opts *Options, level int) uint64 {
74 2 : return uint64(10 * opts.Level(level).TargetFileSize)
75 2 : }
76 :
77 : // maxReadCompactionBytes is used to prevent read compactions which
78 : // are too wide.
79 2 : func maxReadCompactionBytes(opts *Options, level int) uint64 {
80 2 : return uint64(10 * opts.Level(level).TargetFileSize)
81 2 : }
82 :
83 : // noCloseIter wraps around a FragmentIterator, intercepting and eliding
84 : // calls to Close. It is used during compaction to ensure that rangeDelIters
85 : // are not closed prematurely.
86 : type noCloseIter struct {
87 : keyspan.FragmentIterator
88 : }
89 :
90 2 : func (i noCloseIter) Close() error {
91 2 : return nil
92 2 : }
93 :
94 : type compactionLevel struct {
95 : level int
96 : files manifest.LevelSlice
97 : // l0SublevelInfo contains information about L0 sublevels being compacted.
98 : // It's only set for the start level of a compaction starting out of L0 and
99 : // is nil for all other compactions.
100 : l0SublevelInfo []sublevelInfo
101 : }
102 :
103 2 : func (cl compactionLevel) Clone() compactionLevel {
104 2 : newCL := compactionLevel{
105 2 : level: cl.level,
106 2 : files: cl.files.Reslice(func(start, end *manifest.LevelIterator) {}),
107 : }
108 2 : return newCL
109 : }
110 1 : func (cl compactionLevel) String() string {
111 1 : return fmt.Sprintf(`Level %d, Files %s`, cl.level, cl.files)
112 1 : }
113 :
114 : // Return output from compactionOutputSplitters. See comment on
115 : // compactionOutputSplitter.shouldSplitBefore() on how this value is used.
116 : type maybeSplit int
117 :
118 : const (
119 : noSplit maybeSplit = iota
120 : splitNow
121 : )
122 :
123 : // String implements the Stringer interface.
124 1 : func (c maybeSplit) String() string {
125 1 : if c == noSplit {
126 1 : return "no-split"
127 1 : }
128 1 : return "split-now"
129 : }
130 :
131 : // compactionOutputSplitter is an interface for encapsulating logic around
132 : // switching the output of a compaction to a new output file. Additional
133 : // constraints around switching compaction outputs that are specific to that
134 : // compaction type (eg. flush splits) are implemented in
135 : // compactionOutputSplitters that compose other child compactionOutputSplitters.
136 : type compactionOutputSplitter interface {
137 : // shouldSplitBefore returns whether we should split outputs before the
138 : // specified "current key". The return value is splitNow or noSplit.
139 : // splitNow means a split is advised before the specified key, and noSplit
140 : // means no split is advised. If shouldSplitBefore(a) advises a split then
141 : // shouldSplitBefore(b) should also advise a split given b >= a, until
142 : // onNewOutput is called.
143 : shouldSplitBefore(key *InternalKey, tw *sstable.Writer) maybeSplit
144 : // onNewOutput updates internal splitter state when the compaction switches
145 : // to a new sstable, and returns the next limit for the new output which
146 : // would get used to truncate range tombstones if the compaction iterator
147 : // runs out of keys. The limit returned MUST be > key according to the
148 : // compaction's comparator. The specified key is the first key in the new
149 : // output, or nil if this sstable will only contain range tombstones already
150 : // in the fragmenter.
151 : onNewOutput(key []byte) []byte
152 : }
153 :
154 : // fileSizeSplitter is a compactionOutputSplitter that enforces target file
155 : // sizes. This splitter splits to a new output file when the estimated file size
156 : // is 0.5x-2x the target file size. If there are overlapping grandparent files,
157 : // this splitter will attempt to split at a grandparent boundary. For example,
158 : // consider the example where a compaction wrote 'd' to the current output file,
159 : // and the next key has a user key 'g':
160 : //
161 : // previous key next key
162 : // | |
163 : // | |
164 : // +---------------|----+ +--|----------+
165 : // grandparents: | 000006 | | | | 000007 |
166 : // +---------------|----+ +--|----------+
167 : // a b d e f g i
168 : //
169 : // Splitting the output file F before 'g' will ensure that the current output
170 : // file F does not overlap the grandparent file 000007. Aligning sstable
171 : // boundaries like this can significantly reduce write amplification, since a
172 : // subsequent compaction of F into the grandparent level will avoid needlessly
173 : // rewriting any keys within 000007 that do not overlap F's bounds. Consider the
174 : // following compaction:
175 : //
176 : // +----------------------+
177 : // input | |
178 : // level +----------------------+
179 : // \/
180 : // +---------------+ +---------------+
181 : // output |XXXXXXX| | | |XXXXXXXX|
182 : // level +---------------+ +---------------+
183 : //
184 : // The input-level file overlaps two files in the output level, but only
185 : // partially. The beginning of the first output-level file and the end of the
186 : // second output-level file will be rewritten verbatim. This write I/O is
187 : // "wasted" in the sense that no merging is being performed.
188 : //
189 : // To prevent the above waste, this splitter attempts to split output files
190 : // before the start key of grandparent files. It still strives to write output
191 : // files of approximately the target file size, by constraining this splitting
192 : // at grandparent points to apply only if the current output's file size is
193 : // about the right order of magnitude.
194 : //
195 : // Note that, unlike most other splitters, this splitter does not guarantee that
196 : // it will advise splits only at user key change boundaries.
197 : type fileSizeSplitter struct {
198 : frontier frontier
199 : targetFileSize uint64
200 : atGrandparentBoundary bool
201 : boundariesObserved uint64
202 : nextGrandparent *fileMetadata
203 : grandparents manifest.LevelIterator
204 : }
205 :
206 : func newFileSizeSplitter(
207 : f *frontiers, targetFileSize uint64, grandparents manifest.LevelIterator,
208 2 : ) *fileSizeSplitter {
209 2 : s := &fileSizeSplitter{targetFileSize: targetFileSize}
210 2 : s.nextGrandparent = grandparents.First()
211 2 : s.grandparents = grandparents
212 2 : if s.nextGrandparent != nil {
213 2 : s.frontier.Init(f, s.nextGrandparent.Smallest.UserKey, s.reached)
214 2 : }
215 2 : return s
216 : }
217 :
218 2 : func (f *fileSizeSplitter) reached(nextKey []byte) []byte {
219 2 : f.atGrandparentBoundary = true
220 2 : f.boundariesObserved++
221 2 : // NB: f.grandparents is a bounded iterator, constrained to the compaction
222 2 : // key range.
223 2 : f.nextGrandparent = f.grandparents.Next()
224 2 : if f.nextGrandparent == nil {
225 2 : return nil
226 2 : }
227 : // TODO(jackson): Should we also split before or immediately after
228 : // grandparents' largest keys? Splitting before the start boundary prevents
229 : // overlap with the grandparent. Also splitting after the end boundary may
230 : // increase the probability of move compactions.
231 2 : return f.nextGrandparent.Smallest.UserKey
232 : }
233 :
234 2 : func (f *fileSizeSplitter) shouldSplitBefore(key *InternalKey, tw *sstable.Writer) maybeSplit {
235 2 : atGrandparentBoundary := f.atGrandparentBoundary
236 2 :
237 2 : // Clear f.atGrandparentBoundary unconditionally.
238 2 : //
239 2 : // This is a bit subtle. Even if do decide to split, it's possible that a
240 2 : // higher-level splitter will ignore our request (eg, because we're between
241 2 : // two internal keys with the same user key). In this case, the next call to
242 2 : // shouldSplitBefore will find atGrandparentBoundary=false. This is
243 2 : // desirable, because in this case we would've already written the earlier
244 2 : // key with the same user key to the output file. The current output file is
245 2 : // already doomed to overlap the grandparent whose bound triggered
246 2 : // atGrandparentBoundary=true. We should continue on, waiting for the next
247 2 : // grandparent boundary.
248 2 : f.atGrandparentBoundary = false
249 2 :
250 2 : // If the key is a range tombstone, the EstimatedSize may not grow right
251 2 : // away when a range tombstone is added to the fragmenter: It's dependent on
252 2 : // whether or not the this new range deletion will start a new fragment.
253 2 : // Range deletions are rare, so we choose to simply not split yet.
254 2 : // TODO(jackson): Reconsider this, and consider range keys too as a part of
255 2 : // #2321.
256 2 : if key.Kind() == InternalKeyKindRangeDelete || tw == nil {
257 2 : return noSplit
258 2 : }
259 :
260 2 : estSize := tw.EstimatedSize()
261 2 : switch {
262 2 : case estSize < f.targetFileSize/2:
263 2 : // The estimated file size is less than half the target file size. Don't
264 2 : // split it, even if currently aligned with a grandparent file because
265 2 : // it's too small.
266 2 : return noSplit
267 2 : case estSize >= 2*f.targetFileSize:
268 2 : // The estimated file size is double the target file size. Split it even
269 2 : // if we were not aligned with a grandparent file boundary to avoid
270 2 : // excessively exceeding the target file size.
271 2 : return splitNow
272 2 : case !atGrandparentBoundary:
273 2 : // Don't split if we're not at a grandparent, except if we've exhausted
274 2 : // all the grandparents overlapping this compaction's key range. Then we
275 2 : // may want to split purely based on file size.
276 2 : if f.nextGrandparent == nil {
277 2 : // There are no more grandparents. Optimize for the target file size
278 2 : // and split as soon as we hit the target file size.
279 2 : if estSize >= f.targetFileSize {
280 2 : return splitNow
281 2 : }
282 : }
283 2 : return noSplit
284 2 : default:
285 2 : // INVARIANT: atGrandparentBoundary
286 2 : // INVARIANT: targetSize/2 < estSize < 2*targetSize
287 2 : //
288 2 : // The estimated file size is close enough to the target file size that
289 2 : // we should consider splitting.
290 2 : //
291 2 : // Determine whether to split now based on how many grandparent
292 2 : // boundaries we have already observed while building this output file.
293 2 : // The intuition here is that if the grandparent level is dense in this
294 2 : // part of the keyspace, we're likely to continue to have more
295 2 : // opportunities to split this file aligned with a grandparent. If this
296 2 : // is the first grandparent boundary observed, we split immediately
297 2 : // (we're already at ≥50% the target file size). Otherwise, each
298 2 : // overlapping grandparent we've observed increases the minimum file
299 2 : // size by 5% of the target file size, up to at most 90% of the target
300 2 : // file size.
301 2 : //
302 2 : // TODO(jackson): The particular thresholds are somewhat unprincipled.
303 2 : // This is the same heuristic as RocksDB implements. Is there are more
304 2 : // principled formulation that can, further reduce w-amp, produce files
305 2 : // closer to the target file size, or is more understandable?
306 2 :
307 2 : // NB: Subtract 1 from `boundariesObserved` to account for the current
308 2 : // boundary we're considering splitting at. `reached` will have
309 2 : // incremented it at the same time it set `atGrandparentBoundary`.
310 2 : minimumPctOfTargetSize := 50 + 5*min(f.boundariesObserved-1, 8)
311 2 : if estSize < (minimumPctOfTargetSize*f.targetFileSize)/100 {
312 2 : return noSplit
313 2 : }
314 2 : return splitNow
315 : }
316 : }
317 :
318 2 : func (f *fileSizeSplitter) onNewOutput(key []byte) []byte {
319 2 : f.boundariesObserved = 0
320 2 : return nil
321 2 : }
322 :
323 2 : func newLimitFuncSplitter(f *frontiers, limitFunc func(userKey []byte) []byte) *limitFuncSplitter {
324 2 : s := &limitFuncSplitter{limitFunc: limitFunc}
325 2 : s.frontier.Init(f, nil, s.reached)
326 2 : return s
327 2 : }
328 :
329 : type limitFuncSplitter struct {
330 : frontier frontier
331 : limitFunc func(userKey []byte) []byte
332 : split maybeSplit
333 : }
334 :
335 2 : func (lf *limitFuncSplitter) shouldSplitBefore(key *InternalKey, tw *sstable.Writer) maybeSplit {
336 2 : return lf.split
337 2 : }
338 :
339 2 : func (lf *limitFuncSplitter) reached(nextKey []byte) []byte {
340 2 : lf.split = splitNow
341 2 : return nil
342 2 : }
343 :
344 2 : func (lf *limitFuncSplitter) onNewOutput(key []byte) []byte {
345 2 : lf.split = noSplit
346 2 : if key != nil {
347 2 : // TODO(jackson): For some users, like L0 flush splits, there's no need
348 2 : // to binary search over all the flush splits every time. The next split
349 2 : // point must be ahead of the previous flush split point.
350 2 : limit := lf.limitFunc(key)
351 2 : lf.frontier.Update(limit)
352 2 : return limit
353 2 : }
354 2 : lf.frontier.Update(nil)
355 2 : return nil
356 : }
357 :
358 : // splitterGroup is a compactionOutputSplitter that splits whenever one of its
359 : // child splitters advises a compaction split.
360 : type splitterGroup struct {
361 : cmp Compare
362 : splitters []compactionOutputSplitter
363 : }
364 :
365 : func (a *splitterGroup) shouldSplitBefore(
366 : key *InternalKey, tw *sstable.Writer,
367 2 : ) (suggestion maybeSplit) {
368 2 : for _, splitter := range a.splitters {
369 2 : if splitter.shouldSplitBefore(key, tw) == splitNow {
370 2 : return splitNow
371 2 : }
372 : }
373 2 : return noSplit
374 : }
375 :
376 2 : func (a *splitterGroup) onNewOutput(key []byte) []byte {
377 2 : var earliestLimit []byte
378 2 : for _, splitter := range a.splitters {
379 2 : limit := splitter.onNewOutput(key)
380 2 : if limit == nil {
381 2 : continue
382 : }
383 2 : if earliestLimit == nil || a.cmp(limit, earliestLimit) < 0 {
384 2 : earliestLimit = limit
385 2 : }
386 : }
387 2 : return earliestLimit
388 : }
389 :
390 : // userKeyChangeSplitter is a compactionOutputSplitter that takes in a child
391 : // splitter, and splits when 1) that child splitter has advised a split, and 2)
392 : // the compaction output is at the boundary between two user keys (also
393 : // the boundary between atomic compaction units). Use this splitter to wrap
394 : // any splitters that don't guarantee user key splits (i.e. splitters that make
395 : // their determination in ways other than comparing the current key against a
396 : // limit key.) If a wrapped splitter advises a split, it must continue
397 : // to advise a split until a new output.
398 : type userKeyChangeSplitter struct {
399 : cmp Compare
400 : splitter compactionOutputSplitter
401 : unsafePrevUserKey func() []byte
402 : }
403 :
404 2 : func (u *userKeyChangeSplitter) shouldSplitBefore(key *InternalKey, tw *sstable.Writer) maybeSplit {
405 2 : // NB: The userKeyChangeSplitter only needs to suffer a key comparison if
406 2 : // the wrapped splitter requests a split.
407 2 : //
408 2 : // We could implement this splitter using frontiers: When the inner splitter
409 2 : // requests a split before key `k`, we'd update a frontier to be
410 2 : // ImmediateSuccessor(k). Then on the next key greater than >k, the
411 2 : // frontier's `reached` func would be called and we'd return splitNow.
412 2 : // This doesn't really save work since duplicate user keys are rare, and it
413 2 : // requires us to materialize the ImmediateSuccessor key. It also prevents
414 2 : // us from splitting on the same key that the inner splitter requested a
415 2 : // split for—instead we need to wait until the next key. The current
416 2 : // implementation uses `unsafePrevUserKey` to gain access to the previous
417 2 : // key which allows it to immediately respect the inner splitter if
418 2 : // possible.
419 2 : if split := u.splitter.shouldSplitBefore(key, tw); split != splitNow {
420 2 : return split
421 2 : }
422 2 : if u.cmp(key.UserKey, u.unsafePrevUserKey()) > 0 {
423 2 : return splitNow
424 2 : }
425 2 : return noSplit
426 : }
427 :
428 2 : func (u *userKeyChangeSplitter) onNewOutput(key []byte) []byte {
429 2 : return u.splitter.onNewOutput(key)
430 2 : }
431 :
432 : // compactionWritable is a objstorage.Writable wrapper that, on every write,
433 : // updates a metric in `versions` on bytes written by in-progress compactions so
434 : // far. It also increments a per-compaction `written` int.
435 : type compactionWritable struct {
436 : objstorage.Writable
437 :
438 : versions *versionSet
439 : written *int64
440 : }
441 :
442 : // Write is part of the objstorage.Writable interface.
443 2 : func (c *compactionWritable) Write(p []byte) error {
444 2 : if err := c.Writable.Write(p); err != nil {
445 0 : return err
446 0 : }
447 :
448 2 : *c.written += int64(len(p))
449 2 : c.versions.incrementCompactionBytes(int64(len(p)))
450 2 : return nil
451 : }
452 :
453 : type compactionKind int
454 :
455 : const (
456 : compactionKindDefault compactionKind = iota
457 : compactionKindFlush
458 : // compactionKindMove denotes a move compaction where the input file is
459 : // retained and linked in a new level without being obsoleted.
460 : compactionKindMove
461 : // compactionKindCopy denotes a copy compaction where the input file is
462 : // copied byte-by-byte into a new file with a new FileNum in the output level.
463 : compactionKindCopy
464 : compactionKindDeleteOnly
465 : compactionKindElisionOnly
466 : compactionKindRead
467 : compactionKindRewrite
468 : compactionKindIngestedFlushable
469 : )
470 :
471 2 : func (k compactionKind) String() string {
472 2 : switch k {
473 2 : case compactionKindDefault:
474 2 : return "default"
475 0 : case compactionKindFlush:
476 0 : return "flush"
477 2 : case compactionKindMove:
478 2 : return "move"
479 2 : case compactionKindDeleteOnly:
480 2 : return "delete-only"
481 2 : case compactionKindElisionOnly:
482 2 : return "elision-only"
483 1 : case compactionKindRead:
484 1 : return "read"
485 1 : case compactionKindRewrite:
486 1 : return "rewrite"
487 0 : case compactionKindIngestedFlushable:
488 0 : return "ingested-flushable"
489 2 : case compactionKindCopy:
490 2 : return "copy"
491 : }
492 0 : return "?"
493 : }
494 :
495 : // rangeKeyCompactionTransform is used to transform range key spans as part of the
496 : // keyspan.MergingIter. As part of this transformation step, we can elide range
497 : // keys in the last snapshot stripe, as well as coalesce range keys within
498 : // snapshot stripes.
499 : func rangeKeyCompactionTransform(
500 : eq base.Equal, snapshots []uint64, elideRangeKey func(start, end []byte) bool,
501 2 : ) keyspan.Transformer {
502 2 : return keyspan.TransformerFunc(func(cmp base.Compare, s keyspan.Span, dst *keyspan.Span) error {
503 2 : elideInLastStripe := func(keys []keyspan.Key) []keyspan.Key {
504 2 : // Unsets and deletes in the last snapshot stripe can be elided.
505 2 : k := 0
506 2 : for j := range keys {
507 2 : if elideRangeKey(s.Start, s.End) &&
508 2 : (keys[j].Kind() == InternalKeyKindRangeKeyUnset || keys[j].Kind() == InternalKeyKindRangeKeyDelete) {
509 2 : continue
510 : }
511 2 : keys[k] = keys[j]
512 2 : k++
513 : }
514 2 : keys = keys[:k]
515 2 : return keys
516 : }
517 : // snapshots are in ascending order, while s.keys are in descending seqnum
518 : // order. Partition s.keys by snapshot stripes, and call rangekey.Coalesce
519 : // on each partition.
520 2 : dst.Start = s.Start
521 2 : dst.End = s.End
522 2 : dst.Keys = dst.Keys[:0]
523 2 : i, j := len(snapshots)-1, 0
524 2 : usedLen := 0
525 2 : for i >= 0 {
526 2 : start := j
527 2 : for j < len(s.Keys) && !base.Visible(s.Keys[j].SeqNum(), snapshots[i], base.InternalKeySeqNumMax) {
528 2 : // Include j in current partition.
529 2 : j++
530 2 : }
531 2 : if j > start {
532 2 : keysDst := dst.Keys[usedLen:cap(dst.Keys)]
533 2 : if err := rangekey.Coalesce(cmp, eq, s.Keys[start:j], &keysDst); err != nil {
534 0 : return err
535 0 : }
536 2 : if j == len(s.Keys) {
537 2 : // This is the last snapshot stripe. Unsets and deletes can be elided.
538 2 : keysDst = elideInLastStripe(keysDst)
539 2 : }
540 2 : usedLen += len(keysDst)
541 2 : dst.Keys = append(dst.Keys, keysDst...)
542 : }
543 2 : i--
544 : }
545 2 : if j < len(s.Keys) {
546 2 : keysDst := dst.Keys[usedLen:cap(dst.Keys)]
547 2 : if err := rangekey.Coalesce(cmp, eq, s.Keys[j:], &keysDst); err != nil {
548 0 : return err
549 0 : }
550 2 : keysDst = elideInLastStripe(keysDst)
551 2 : usedLen += len(keysDst)
552 2 : dst.Keys = append(dst.Keys, keysDst...)
553 : }
554 2 : return nil
555 : })
556 : }
557 :
558 : // compaction is a table compaction from one level to the next, starting from a
559 : // given version.
560 : type compaction struct {
561 : // cancel is a bool that can be used by other goroutines to signal a compaction
562 : // to cancel, such as if a conflicting excise operation raced it to manifest
563 : // application. Only holders of the manifest lock will write to this atomic.
564 : cancel atomic.Bool
565 :
566 : kind compactionKind
567 : cmp Compare
568 : equal Equal
569 : comparer *base.Comparer
570 : formatKey base.FormatKey
571 : logger Logger
572 : version *version
573 : stats base.InternalIteratorStats
574 : beganAt time.Time
575 : // versionEditApplied is set to true when a compaction has completed and the
576 : // resulting version has been installed (if successful), but the compaction
577 : // goroutine is still cleaning up (eg, deleting obsolete files).
578 : versionEditApplied bool
579 : bufferPool sstable.BufferPool
580 :
581 : // startLevel is the level that is being compacted. Inputs from startLevel
582 : // and outputLevel will be merged to produce a set of outputLevel files.
583 : startLevel *compactionLevel
584 :
585 : // outputLevel is the level that files are being produced in. outputLevel is
586 : // equal to startLevel+1 except when:
587 : // - if startLevel is 0, the output level equals compactionPicker.baseLevel().
588 : // - in multilevel compaction, the output level is the lowest level involved in
589 : // the compaction
590 : // A compaction's outputLevel is nil for delete-only compactions.
591 : outputLevel *compactionLevel
592 :
593 : // extraLevels point to additional levels in between the input and output
594 : // levels that get compacted in multilevel compactions
595 : extraLevels []*compactionLevel
596 :
597 : inputs []compactionLevel
598 :
599 : // maxOutputFileSize is the maximum size of an individual table created
600 : // during compaction.
601 : maxOutputFileSize uint64
602 : // maxOverlapBytes is the maximum number of bytes of overlap allowed for a
603 : // single output table with the tables in the grandparent level.
604 : maxOverlapBytes uint64
605 : // disableSpanElision disables elision of range tombstones and range keys. Used
606 : // by tests to allow range tombstones or range keys to be added to tables where
607 : // they would otherwise be elided.
608 : disableSpanElision bool
609 :
610 : // flushing contains the flushables (aka memtables) that are being flushed.
611 : flushing flushableList
612 : // bytesIterated contains the number of bytes that have been flushed/compacted.
613 : bytesIterated uint64
614 : // bytesWritten contains the number of bytes that have been written to outputs.
615 : bytesWritten int64
616 :
617 : // The boundaries of the input data.
618 : smallest InternalKey
619 : largest InternalKey
620 :
621 : // The range deletion tombstone fragmenter. Adds range tombstones as they are
622 : // returned from `compactionIter` and fragments them for output to files.
623 : // Referenced by `compactionIter` which uses it to check whether keys are deleted.
624 : rangeDelFrag keyspan.Fragmenter
625 : // The range key fragmenter. Similar to rangeDelFrag in that it gets range
626 : // keys from the compaction iter and fragments them for output to files.
627 : rangeKeyFrag keyspan.Fragmenter
628 : // The range deletion tombstone iterator, that merges and fragments
629 : // tombstones across levels. This iterator is included within the compaction
630 : // input iterator as a single level.
631 : // TODO(jackson): Remove this when the refactor of FragmentIterator,
632 : // InterleavingIterator, etc is complete.
633 : rangeDelIter keyspan.InternalIteratorShim
634 : // rangeKeyInterleaving is the interleaving iter for range keys.
635 : rangeKeyInterleaving keyspan.InterleavingIter
636 :
637 : // A list of objects to close when the compaction finishes. Used by input
638 : // iteration to keep rangeDelIters open for the lifetime of the compaction,
639 : // and only close them when the compaction finishes.
640 : closers []io.Closer
641 :
642 : // grandparents are the tables in level+2 that overlap with the files being
643 : // compacted. Used to determine output table boundaries. Do not assume that the actual files
644 : // in the grandparent when this compaction finishes will be the same.
645 : grandparents manifest.LevelSlice
646 :
647 : // Boundaries at which flushes to L0 should be split. Determined by
648 : // L0Sublevels. If nil, flushes aren't split.
649 : l0Limits [][]byte
650 :
651 : // List of disjoint inuse key ranges the compaction overlaps with in
652 : // grandparent and lower levels. See setupInuseKeyRanges() for the
653 : // construction. Used by elideTombstone() and elideRangeTombstone() to
654 : // determine if keys affected by a tombstone possibly exist at a lower level.
655 : inuseKeyRanges []manifest.UserKeyRange
656 : // inuseEntireRange is set if the above inuse key ranges wholly contain the
657 : // compaction's key range. This allows compactions in higher levels to often
658 : // elide key comparisons.
659 : inuseEntireRange bool
660 : elideTombstoneIndex int
661 :
662 : // allowedZeroSeqNum is true if seqnums can be zeroed if there are no
663 : // snapshots requiring them to be kept. This determination is made by
664 : // looking for an sstable which overlaps the bounds of the compaction at a
665 : // lower level in the LSM during runCompaction.
666 : allowedZeroSeqNum bool
667 :
668 : metrics map[int]*LevelMetrics
669 :
670 : pickerMetrics compactionPickerMetrics
671 : }
672 :
673 2 : func (c *compaction) makeInfo(jobID int) CompactionInfo {
674 2 : info := CompactionInfo{
675 2 : JobID: jobID,
676 2 : Reason: c.kind.String(),
677 2 : Input: make([]LevelInfo, 0, len(c.inputs)),
678 2 : Annotations: []string{},
679 2 : }
680 2 : for _, cl := range c.inputs {
681 2 : inputInfo := LevelInfo{Level: cl.level, Tables: nil}
682 2 : iter := cl.files.Iter()
683 2 : for m := iter.First(); m != nil; m = iter.Next() {
684 2 : inputInfo.Tables = append(inputInfo.Tables, m.TableInfo())
685 2 : }
686 2 : info.Input = append(info.Input, inputInfo)
687 : }
688 2 : if c.outputLevel != nil {
689 2 : info.Output.Level = c.outputLevel.level
690 2 :
691 2 : // If there are no inputs from the output level (eg, a move
692 2 : // compaction), add an empty LevelInfo to info.Input.
693 2 : if len(c.inputs) > 0 && c.inputs[len(c.inputs)-1].level != c.outputLevel.level {
694 0 : info.Input = append(info.Input, LevelInfo{Level: c.outputLevel.level})
695 0 : }
696 2 : } else {
697 2 : // For a delete-only compaction, set the output level to L6. The
698 2 : // output level is not meaningful here, but complicating the
699 2 : // info.Output interface with a pointer doesn't seem worth the
700 2 : // semantic distinction.
701 2 : info.Output.Level = numLevels - 1
702 2 : }
703 :
704 2 : for i, score := range c.pickerMetrics.scores {
705 2 : info.Input[i].Score = score
706 2 : }
707 2 : info.SingleLevelOverlappingRatio = c.pickerMetrics.singleLevelOverlappingRatio
708 2 : info.MultiLevelOverlappingRatio = c.pickerMetrics.multiLevelOverlappingRatio
709 2 : if len(info.Input) > 2 {
710 2 : info.Annotations = append(info.Annotations, "multilevel")
711 2 : }
712 2 : return info
713 : }
714 :
715 : func newCompaction(
716 : pc *pickedCompaction, opts *Options, beganAt time.Time, provider objstorage.Provider,
717 2 : ) *compaction {
718 2 : c := &compaction{
719 2 : kind: compactionKindDefault,
720 2 : cmp: pc.cmp,
721 2 : equal: opts.equal(),
722 2 : comparer: opts.Comparer,
723 2 : formatKey: opts.Comparer.FormatKey,
724 2 : inputs: pc.inputs,
725 2 : smallest: pc.smallest,
726 2 : largest: pc.largest,
727 2 : logger: opts.Logger,
728 2 : version: pc.version,
729 2 : beganAt: beganAt,
730 2 : maxOutputFileSize: pc.maxOutputFileSize,
731 2 : maxOverlapBytes: pc.maxOverlapBytes,
732 2 : pickerMetrics: pc.pickerMetrics,
733 2 : }
734 2 : c.startLevel = &c.inputs[0]
735 2 : if pc.startLevel.l0SublevelInfo != nil {
736 2 : c.startLevel.l0SublevelInfo = pc.startLevel.l0SublevelInfo
737 2 : }
738 2 : c.outputLevel = &c.inputs[1]
739 2 :
740 2 : if len(pc.extraLevels) > 0 {
741 2 : c.extraLevels = pc.extraLevels
742 2 : c.outputLevel = &c.inputs[len(c.inputs)-1]
743 2 : }
744 : // Compute the set of outputLevel+1 files that overlap this compaction (these
745 : // are the grandparent sstables).
746 2 : if c.outputLevel.level+1 < numLevels {
747 2 : c.grandparents = c.version.Overlaps(c.outputLevel.level+1, c.cmp,
748 2 : c.smallest.UserKey, c.largest.UserKey, c.largest.IsExclusiveSentinel())
749 2 : }
750 2 : c.setupInuseKeyRanges()
751 2 : c.kind = pc.kind
752 2 :
753 2 : if c.kind == compactionKindDefault && c.outputLevel.files.Empty() && !c.hasExtraLevelData() &&
754 2 : c.startLevel.files.Len() == 1 && c.grandparents.SizeSum() <= c.maxOverlapBytes {
755 2 : // This compaction can be converted into a move or copy from one level
756 2 : // to the next. We avoid such a move if there is lots of overlapping
757 2 : // grandparent data. Otherwise, the move could create a parent file
758 2 : // that will require a very expensive merge later on.
759 2 : iter := c.startLevel.files.Iter()
760 2 : meta := iter.First()
761 2 : isRemote := false
762 2 : // We should always be passed a provider, except in some unit tests.
763 2 : if provider != nil {
764 2 : objMeta, err := provider.Lookup(fileTypeTable, meta.FileBacking.DiskFileNum)
765 2 : if err != nil {
766 0 : panic(errors.Wrapf(err, "cannot lookup table %s in provider", meta.FileBacking.DiskFileNum))
767 : }
768 2 : isRemote = objMeta.IsRemote()
769 : }
770 : // Avoid a trivial move or copy if all of these are true, as rewriting a
771 : // new file is better:
772 : //
773 : // 1) The source file is a virtual sstable
774 : // 2) The existing file `meta` is on non-remote storage
775 : // 3) The output level prefers shared storage
776 2 : mustCopy := !isRemote && remote.ShouldCreateShared(opts.Experimental.CreateOnShared, c.outputLevel.level)
777 2 : if mustCopy {
778 2 : // If the source is virtual, it's best to just rewrite the file as all
779 2 : // conditions in the above comment are met.
780 2 : if !meta.Virtual {
781 2 : c.kind = compactionKindCopy
782 2 : }
783 2 : } else {
784 2 : c.kind = compactionKindMove
785 2 : }
786 : }
787 2 : return c
788 : }
789 :
790 : func newDeleteOnlyCompaction(
791 : opts *Options, cur *version, inputs []compactionLevel, beganAt time.Time,
792 2 : ) *compaction {
793 2 : c := &compaction{
794 2 : kind: compactionKindDeleteOnly,
795 2 : cmp: opts.Comparer.Compare,
796 2 : equal: opts.equal(),
797 2 : comparer: opts.Comparer,
798 2 : formatKey: opts.Comparer.FormatKey,
799 2 : logger: opts.Logger,
800 2 : version: cur,
801 2 : beganAt: beganAt,
802 2 : inputs: inputs,
803 2 : }
804 2 :
805 2 : // Set c.smallest, c.largest.
806 2 : files := make([]manifest.LevelIterator, 0, len(inputs))
807 2 : for _, in := range inputs {
808 2 : files = append(files, in.files.Iter())
809 2 : }
810 2 : c.smallest, c.largest = manifest.KeyRange(opts.Comparer.Compare, files...)
811 2 : return c
812 : }
813 :
814 2 : func adjustGrandparentOverlapBytesForFlush(c *compaction, flushingBytes uint64) {
815 2 : // Heuristic to place a lower bound on compaction output file size
816 2 : // caused by Lbase. Prior to this heuristic we have observed an L0 in
817 2 : // production with 310K files of which 290K files were < 10KB in size.
818 2 : // Our hypothesis is that it was caused by L1 having 2600 files and
819 2 : // ~10GB, such that each flush got split into many tiny files due to
820 2 : // overlapping with most of the files in Lbase.
821 2 : //
822 2 : // The computation below is general in that it accounts
823 2 : // for flushing different volumes of data (e.g. we may be flushing
824 2 : // many memtables). For illustration, we consider the typical
825 2 : // example of flushing a 64MB memtable. So 12.8MB output,
826 2 : // based on the compression guess below. If the compressed bytes
827 2 : // guess is an over-estimate we will end up with smaller files,
828 2 : // and if an under-estimate we will end up with larger files.
829 2 : // With a 2MB target file size, 7 files. We are willing to accept
830 2 : // 4x the number of files, if it results in better write amplification
831 2 : // when later compacting to Lbase, i.e., ~450KB files (target file
832 2 : // size / 4).
833 2 : //
834 2 : // Note that this is a pessimistic heuristic in that
835 2 : // fileCountUpperBoundDueToGrandparents could be far from the actual
836 2 : // number of files produced due to the grandparent limits. For
837 2 : // example, in the extreme, consider a flush that overlaps with 1000
838 2 : // files in Lbase f0...f999, and the initially calculated value of
839 2 : // maxOverlapBytes will cause splits at f10, f20,..., f990, which
840 2 : // means an upper bound file count of 100 files. Say the input bytes
841 2 : // in the flush are such that acceptableFileCount=10. We will fatten
842 2 : // up maxOverlapBytes by 10x to ensure that the upper bound file count
843 2 : // drops to 10. However, it is possible that in practice, even without
844 2 : // this change, we would have produced no more than 10 files, and that
845 2 : // this change makes the files unnecessarily wide. Say the input bytes
846 2 : // are distributed such that 10% are in f0...f9, 10% in f10...f19, ...
847 2 : // 10% in f80...f89 and 10% in f990...f999. The original value of
848 2 : // maxOverlapBytes would have actually produced only 10 sstables. But
849 2 : // by increasing maxOverlapBytes by 10x, we may produce 1 sstable that
850 2 : // spans f0...f89, i.e., a much wider sstable than necessary.
851 2 : //
852 2 : // We could produce a tighter estimate of
853 2 : // fileCountUpperBoundDueToGrandparents if we had knowledge of the key
854 2 : // distribution of the flush. The 4x multiplier mentioned earlier is
855 2 : // a way to try to compensate for this pessimism.
856 2 : //
857 2 : // TODO(sumeer): we don't have compression info for the data being
858 2 : // flushed, but it is likely that existing files that overlap with
859 2 : // this flush in Lbase are representative wrt compression ratio. We
860 2 : // could store the uncompressed size in FileMetadata and estimate
861 2 : // the compression ratio.
862 2 : const approxCompressionRatio = 0.2
863 2 : approxOutputBytes := approxCompressionRatio * float64(flushingBytes)
864 2 : approxNumFilesBasedOnTargetSize :=
865 2 : int(math.Ceil(approxOutputBytes / float64(c.maxOutputFileSize)))
866 2 : acceptableFileCount := float64(4 * approxNumFilesBasedOnTargetSize)
867 2 : // The byte calculation is linear in numGrandparentFiles, but we will
868 2 : // incur this linear cost in findGrandparentLimit too, so we are also
869 2 : // willing to pay it now. We could approximate this cheaply by using
870 2 : // the mean file size of Lbase.
871 2 : grandparentFileBytes := c.grandparents.SizeSum()
872 2 : fileCountUpperBoundDueToGrandparents :=
873 2 : float64(grandparentFileBytes) / float64(c.maxOverlapBytes)
874 2 : if fileCountUpperBoundDueToGrandparents > acceptableFileCount {
875 2 : c.maxOverlapBytes = uint64(
876 2 : float64(c.maxOverlapBytes) *
877 2 : (fileCountUpperBoundDueToGrandparents / acceptableFileCount))
878 2 : }
879 : }
880 :
881 : func newFlush(
882 : opts *Options, cur *version, baseLevel int, flushing flushableList, beganAt time.Time,
883 2 : ) *compaction {
884 2 : c := &compaction{
885 2 : kind: compactionKindFlush,
886 2 : cmp: opts.Comparer.Compare,
887 2 : equal: opts.equal(),
888 2 : comparer: opts.Comparer,
889 2 : formatKey: opts.Comparer.FormatKey,
890 2 : logger: opts.Logger,
891 2 : version: cur,
892 2 : beganAt: beganAt,
893 2 : inputs: []compactionLevel{{level: -1}, {level: 0}},
894 2 : maxOutputFileSize: math.MaxUint64,
895 2 : maxOverlapBytes: math.MaxUint64,
896 2 : flushing: flushing,
897 2 : }
898 2 : c.startLevel = &c.inputs[0]
899 2 : c.outputLevel = &c.inputs[1]
900 2 :
901 2 : if len(flushing) > 0 {
902 2 : if _, ok := flushing[0].flushable.(*ingestedFlushable); ok {
903 2 : if len(flushing) != 1 {
904 0 : panic("pebble: ingestedFlushable must be flushed one at a time.")
905 : }
906 2 : c.kind = compactionKindIngestedFlushable
907 2 : return c
908 : }
909 : }
910 :
911 : // Make sure there's no ingestedFlushable after the first flushable in the
912 : // list.
913 2 : for _, f := range flushing {
914 2 : if _, ok := f.flushable.(*ingestedFlushable); ok {
915 0 : panic("pebble: flushing shouldn't contain ingestedFlushable flushable")
916 : }
917 : }
918 :
919 2 : if cur.L0Sublevels != nil {
920 2 : c.l0Limits = cur.L0Sublevels.FlushSplitKeys()
921 2 : }
922 :
923 2 : smallestSet, largestSet := false, false
924 2 : updatePointBounds := func(iter internalIterator) {
925 2 : if key, _ := iter.First(); key != nil {
926 2 : if !smallestSet ||
927 2 : base.InternalCompare(c.cmp, c.smallest, *key) > 0 {
928 2 : smallestSet = true
929 2 : c.smallest = key.Clone()
930 2 : }
931 : }
932 2 : if key, _ := iter.Last(); key != nil {
933 2 : if !largestSet ||
934 2 : base.InternalCompare(c.cmp, c.largest, *key) < 0 {
935 2 : largestSet = true
936 2 : c.largest = key.Clone()
937 2 : }
938 : }
939 : }
940 :
941 2 : updateRangeBounds := func(iter keyspan.FragmentIterator) {
942 2 : // File bounds require s != nil && !s.Empty(). We only need to check for
943 2 : // s != nil here, as the memtable's FragmentIterator would never surface
944 2 : // empty spans.
945 2 : if s := iter.First(); s != nil {
946 2 : if key := s.SmallestKey(); !smallestSet ||
947 2 : base.InternalCompare(c.cmp, c.smallest, key) > 0 {
948 2 : smallestSet = true
949 2 : c.smallest = key.Clone()
950 2 : }
951 : }
952 2 : if s := iter.Last(); s != nil {
953 2 : if key := s.LargestKey(); !largestSet ||
954 2 : base.InternalCompare(c.cmp, c.largest, key) < 0 {
955 2 : largestSet = true
956 2 : c.largest = key.Clone()
957 2 : }
958 : }
959 : }
960 :
961 2 : var flushingBytes uint64
962 2 : for i := range flushing {
963 2 : f := flushing[i]
964 2 : updatePointBounds(f.newIter(nil))
965 2 : if rangeDelIter := f.newRangeDelIter(nil); rangeDelIter != nil {
966 2 : updateRangeBounds(rangeDelIter)
967 2 : }
968 2 : if rangeKeyIter := f.newRangeKeyIter(nil); rangeKeyIter != nil {
969 2 : updateRangeBounds(rangeKeyIter)
970 2 : }
971 2 : flushingBytes += f.inuseBytes()
972 : }
973 :
974 2 : if opts.FlushSplitBytes > 0 {
975 2 : c.maxOutputFileSize = uint64(opts.Level(0).TargetFileSize)
976 2 : c.maxOverlapBytes = maxGrandparentOverlapBytes(opts, 0)
977 2 : c.grandparents = c.version.Overlaps(baseLevel, c.cmp, c.smallest.UserKey,
978 2 : c.largest.UserKey, c.largest.IsExclusiveSentinel())
979 2 : adjustGrandparentOverlapBytesForFlush(c, flushingBytes)
980 2 : }
981 :
982 2 : c.setupInuseKeyRanges()
983 2 : return c
984 : }
985 :
986 2 : func (c *compaction) hasExtraLevelData() bool {
987 2 : if len(c.extraLevels) == 0 {
988 2 : // not a multi level compaction
989 2 : return false
990 2 : } else if c.extraLevels[0].files.Empty() {
991 2 : // a multi level compaction without data in the intermediate input level;
992 2 : // e.g. for a multi level compaction with levels 4,5, and 6, this could
993 2 : // occur if there is no files to compact in 5, or in 5 and 6 (i.e. a move).
994 2 : return false
995 2 : }
996 2 : return true
997 : }
998 :
999 2 : func (c *compaction) setupInuseKeyRanges() {
1000 2 : level := c.outputLevel.level + 1
1001 2 : if c.outputLevel.level == 0 {
1002 2 : level = 0
1003 2 : }
1004 : // calculateInuseKeyRanges will return a series of sorted spans. Overlapping
1005 : // or abutting spans have already been merged.
1006 2 : c.inuseKeyRanges = c.version.CalculateInuseKeyRanges(
1007 2 : c.cmp, level, numLevels-1, c.smallest.UserKey, c.largest.UserKey,
1008 2 : )
1009 2 : // Check if there's a single in-use span that encompasses the entire key
1010 2 : // range of the compaction. This is an optimization to avoid key comparisons
1011 2 : // against inuseKeyRanges during the compaction when every key within the
1012 2 : // compaction overlaps with an in-use span.
1013 2 : if len(c.inuseKeyRanges) > 0 {
1014 2 : c.inuseEntireRange = c.cmp(c.inuseKeyRanges[0].Start, c.smallest.UserKey) <= 0 &&
1015 2 : c.cmp(c.inuseKeyRanges[0].End, c.largest.UserKey) >= 0
1016 2 : }
1017 : }
1018 :
1019 : // findGrandparentLimit takes the start user key for a table and returns the
1020 : // user key to which that table can extend without excessively overlapping
1021 : // the grandparent level. If no limit is needed considering the grandparent
1022 : // files, this function returns nil. This is done in order to prevent a table
1023 : // at level N from overlapping too much data at level N+1. We want to avoid
1024 : // such large overlaps because they translate into large compactions. The
1025 : // current heuristic stops output of a table if the addition of another key
1026 : // would cause the table to overlap more than 10x the target file size at
1027 : // level N. See maxGrandparentOverlapBytes.
1028 2 : func (c *compaction) findGrandparentLimit(start []byte) []byte {
1029 2 : iter := c.grandparents.Iter()
1030 2 : var overlappedBytes uint64
1031 2 : var greater bool
1032 2 : for f := iter.SeekGE(c.cmp, start); f != nil; f = iter.Next() {
1033 2 : overlappedBytes += f.Size
1034 2 : // To ensure forward progress we always return a larger user
1035 2 : // key than where we started. See comments above clients of
1036 2 : // this function for how this is used.
1037 2 : greater = greater || c.cmp(f.Smallest.UserKey, start) > 0
1038 2 : if !greater {
1039 2 : continue
1040 : }
1041 :
1042 : // We return the smallest bound of a sstable rather than the
1043 : // largest because the smallest is always inclusive, and limits
1044 : // are used exlusively when truncating range tombstones. If we
1045 : // truncated an output to the largest key while there's a
1046 : // pending tombstone, the next output file would also overlap
1047 : // the same grandparent f.
1048 2 : if overlappedBytes > c.maxOverlapBytes {
1049 2 : return f.Smallest.UserKey
1050 2 : }
1051 : }
1052 2 : return nil
1053 : }
1054 :
1055 : // findL0Limit takes the start key for a table and returns the user key to which
1056 : // that table can be extended without hitting the next l0Limit. Having flushed
1057 : // sstables "bridging across" an l0Limit could lead to increased L0 -> LBase
1058 : // compaction sizes as well as elevated read amplification.
1059 2 : func (c *compaction) findL0Limit(start []byte) []byte {
1060 2 : if c.startLevel.level > -1 || c.outputLevel.level != 0 || len(c.l0Limits) == 0 {
1061 2 : return nil
1062 2 : }
1063 2 : index := sort.Search(len(c.l0Limits), func(i int) bool {
1064 2 : return c.cmp(c.l0Limits[i], start) > 0
1065 2 : })
1066 2 : if index < len(c.l0Limits) {
1067 2 : return c.l0Limits[index]
1068 2 : }
1069 2 : return nil
1070 : }
1071 :
1072 : // errorOnUserKeyOverlap returns an error if the last two written sstables in
1073 : // this compaction have revisions of the same user key present in both sstables,
1074 : // when it shouldn't (eg. when splitting flushes).
1075 2 : func (c *compaction) errorOnUserKeyOverlap(ve *versionEdit) error {
1076 2 : if n := len(ve.NewFiles); n > 1 {
1077 2 : meta := ve.NewFiles[n-1].Meta
1078 2 : prevMeta := ve.NewFiles[n-2].Meta
1079 2 : if !prevMeta.Largest.IsExclusiveSentinel() &&
1080 2 : c.cmp(prevMeta.Largest.UserKey, meta.Smallest.UserKey) >= 0 {
1081 1 : return errors.Errorf("pebble: compaction split user key across two sstables: %s in %s and %s",
1082 1 : prevMeta.Largest.Pretty(c.formatKey),
1083 1 : prevMeta.FileNum,
1084 1 : meta.FileNum)
1085 1 : }
1086 : }
1087 2 : return nil
1088 : }
1089 :
1090 : // allowZeroSeqNum returns true if seqnum's can be zeroed if there are no
1091 : // snapshots requiring them to be kept. It performs this determination by
1092 : // looking for an sstable which overlaps the bounds of the compaction at a
1093 : // lower level in the LSM.
1094 2 : func (c *compaction) allowZeroSeqNum() bool {
1095 2 : return c.elideRangeTombstone(c.smallest.UserKey, c.largest.UserKey)
1096 2 : }
1097 :
1098 : // elideTombstone returns true if it is ok to elide a tombstone for the
1099 : // specified key. A return value of true guarantees that there are no key/value
1100 : // pairs at c.level+2 or higher that possibly contain the specified user
1101 : // key. The keys in multiple invocations to elideTombstone must be supplied in
1102 : // order.
1103 2 : func (c *compaction) elideTombstone(key []byte) bool {
1104 2 : if c.inuseEntireRange || len(c.flushing) != 0 {
1105 2 : return false
1106 2 : }
1107 :
1108 2 : for ; c.elideTombstoneIndex < len(c.inuseKeyRanges); c.elideTombstoneIndex++ {
1109 2 : r := &c.inuseKeyRanges[c.elideTombstoneIndex]
1110 2 : if c.cmp(key, r.End) <= 0 {
1111 2 : if c.cmp(key, r.Start) >= 0 {
1112 2 : return false
1113 2 : }
1114 2 : break
1115 : }
1116 : }
1117 2 : return true
1118 : }
1119 :
1120 : // elideRangeTombstone returns true if it is ok to elide the specified range
1121 : // tombstone. A return value of true guarantees that there are no key/value
1122 : // pairs at c.outputLevel.level+1 or higher that possibly overlap the specified
1123 : // tombstone.
1124 2 : func (c *compaction) elideRangeTombstone(start, end []byte) bool {
1125 2 : // Disable range tombstone elision if the testing knob for that is enabled,
1126 2 : // or if we are flushing memtables. The latter requirement is due to
1127 2 : // inuseKeyRanges not accounting for key ranges in other memtables that are
1128 2 : // being flushed in the same compaction. It's possible for a range tombstone
1129 2 : // in one memtable to overlap keys in a preceding memtable in c.flushing.
1130 2 : //
1131 2 : // This function is also used in setting allowZeroSeqNum, so disabling
1132 2 : // elision of range tombstones also disables zeroing of SeqNums.
1133 2 : //
1134 2 : // TODO(peter): we disable zeroing of seqnums during flushing to match
1135 2 : // RocksDB behavior and to avoid generating overlapping sstables during
1136 2 : // DB.replayWAL. When replaying WAL files at startup, we flush after each
1137 2 : // WAL is replayed building up a single version edit that is
1138 2 : // applied. Because we don't apply the version edit after each flush, this
1139 2 : // code doesn't know that L0 contains files and zeroing of seqnums should
1140 2 : // be disabled. That is fixable, but it seems safer to just match the
1141 2 : // RocksDB behavior for now.
1142 2 : if c.disableSpanElision || len(c.flushing) != 0 {
1143 2 : return false
1144 2 : }
1145 :
1146 2 : lower := sort.Search(len(c.inuseKeyRanges), func(i int) bool {
1147 2 : return c.cmp(c.inuseKeyRanges[i].End, start) >= 0
1148 2 : })
1149 2 : upper := sort.Search(len(c.inuseKeyRanges), func(i int) bool {
1150 2 : return c.cmp(c.inuseKeyRanges[i].Start, end) > 0
1151 2 : })
1152 2 : return lower >= upper
1153 : }
1154 :
1155 : // elideRangeKey returns true if it is ok to elide the specified range key. A
1156 : // return value of true guarantees that there are no key/value pairs at
1157 : // c.outputLevel.level+1 or higher that possibly overlap the specified range key.
1158 2 : func (c *compaction) elideRangeKey(start, end []byte) bool {
1159 2 : // TODO(bilal): Track inuseKeyRanges separately for the range keyspace as
1160 2 : // opposed to the point keyspace. Once that is done, elideRangeTombstone
1161 2 : // can just check in the point keyspace, and this function can check for
1162 2 : // inuseKeyRanges in the range keyspace.
1163 2 : return c.elideRangeTombstone(start, end)
1164 2 : }
1165 :
1166 : // newInputIter returns an iterator over all the input tables in a compaction.
1167 : func (c *compaction) newInputIter(
1168 : newIters tableNewIters, newRangeKeyIter keyspan.TableNewSpanIter, snapshots []uint64,
1169 2 : ) (_ internalIterator, retErr error) {
1170 2 : // Validate the ordering of compaction input files for defense in depth.
1171 2 : if len(c.flushing) == 0 {
1172 2 : if c.startLevel.level >= 0 {
1173 2 : err := manifest.CheckOrdering(c.cmp, c.formatKey,
1174 2 : manifest.Level(c.startLevel.level), c.startLevel.files.Iter())
1175 2 : if err != nil {
1176 1 : return nil, err
1177 1 : }
1178 : }
1179 2 : err := manifest.CheckOrdering(c.cmp, c.formatKey,
1180 2 : manifest.Level(c.outputLevel.level), c.outputLevel.files.Iter())
1181 2 : if err != nil {
1182 1 : return nil, err
1183 1 : }
1184 2 : if c.startLevel.level == 0 {
1185 2 : if c.startLevel.l0SublevelInfo == nil {
1186 0 : panic("l0SublevelInfo not created for compaction out of L0")
1187 : }
1188 2 : for _, info := range c.startLevel.l0SublevelInfo {
1189 2 : err := manifest.CheckOrdering(c.cmp, c.formatKey,
1190 2 : info.sublevel, info.Iter())
1191 2 : if err != nil {
1192 1 : return nil, err
1193 1 : }
1194 : }
1195 : }
1196 2 : if len(c.extraLevels) > 0 {
1197 2 : if len(c.extraLevels) > 1 {
1198 0 : panic("n>2 multi level compaction not implemented yet")
1199 : }
1200 2 : interLevel := c.extraLevels[0]
1201 2 : err := manifest.CheckOrdering(c.cmp, c.formatKey,
1202 2 : manifest.Level(interLevel.level), interLevel.files.Iter())
1203 2 : if err != nil {
1204 0 : return nil, err
1205 0 : }
1206 : }
1207 : }
1208 :
1209 : // There are three classes of keys that a compaction needs to process: point
1210 : // keys, range deletion tombstones and range keys. Collect all iterators for
1211 : // all these classes of keys from all the levels. We'll aggregate them
1212 : // together farther below.
1213 : //
1214 : // numInputLevels is an approximation of the number of iterator levels. Due
1215 : // to idiosyncrasies in iterator construction, we may (rarely) exceed this
1216 : // initial capacity.
1217 2 : numInputLevels := max(len(c.flushing), len(c.inputs))
1218 2 : iters := make([]internalIterator, 0, numInputLevels)
1219 2 : rangeDelIters := make([]keyspan.FragmentIterator, 0, numInputLevels)
1220 2 : rangeKeyIters := make([]keyspan.FragmentIterator, 0, numInputLevels)
1221 2 :
1222 2 : // If construction of the iterator inputs fails, ensure that we close all
1223 2 : // the consitutent iterators.
1224 2 : defer func() {
1225 2 : if retErr != nil {
1226 0 : for _, iter := range iters {
1227 0 : if iter != nil {
1228 0 : iter.Close()
1229 0 : }
1230 : }
1231 0 : for _, rangeDelIter := range rangeDelIters {
1232 0 : rangeDelIter.Close()
1233 0 : }
1234 : }
1235 : }()
1236 2 : iterOpts := IterOptions{
1237 2 : CategoryAndQoS: sstable.CategoryAndQoS{
1238 2 : Category: "pebble-compaction",
1239 2 : QoSLevel: sstable.NonLatencySensitiveQoSLevel,
1240 2 : },
1241 2 : logger: c.logger,
1242 2 : }
1243 2 :
1244 2 : // Populate iters, rangeDelIters and rangeKeyIters with the appropriate
1245 2 : // constituent iterators. This depends on whether this is a flush or a
1246 2 : // compaction.
1247 2 : if len(c.flushing) != 0 {
1248 2 : // If flushing, we need to build the input iterators over the memtables
1249 2 : // stored in c.flushing.
1250 2 : for i := range c.flushing {
1251 2 : f := c.flushing[i]
1252 2 : iters = append(iters, f.newFlushIter(nil, &c.bytesIterated))
1253 2 : rangeDelIter := f.newRangeDelIter(nil)
1254 2 : if rangeDelIter != nil {
1255 2 : rangeDelIters = append(rangeDelIters, rangeDelIter)
1256 2 : }
1257 2 : if rangeKeyIter := f.newRangeKeyIter(nil); rangeKeyIter != nil {
1258 2 : rangeKeyIters = append(rangeKeyIters, rangeKeyIter)
1259 2 : }
1260 : }
1261 2 : } else {
1262 2 : addItersForLevel := func(level *compactionLevel, l manifest.Level) error {
1263 2 : // Add a *levelIter for point iterators. Because we don't call
1264 2 : // initRangeDel, the levelIter will close and forget the range
1265 2 : // deletion iterator when it steps on to a new file. Surfacing range
1266 2 : // deletions to compactions are handled below.
1267 2 : iters = append(iters, newLevelIter(context.Background(),
1268 2 : iterOpts, c.comparer, newIters, level.files.Iter(), l, internalIterOpts{
1269 2 : bytesIterated: &c.bytesIterated,
1270 2 : bufferPool: &c.bufferPool,
1271 2 : }))
1272 2 : // TODO(jackson): Use keyspan.LevelIter to avoid loading all the range
1273 2 : // deletions into memory upfront. (See #2015, which reverted this.)
1274 2 : // There will be no user keys that are split between sstables
1275 2 : // within a level in Cockroach 23.1, which unblocks this optimization.
1276 2 :
1277 2 : // Add the range deletion iterator for each file as an independent level
1278 2 : // in mergingIter, as opposed to making a levelIter out of those. This
1279 2 : // is safer as levelIter expects all keys coming from underlying
1280 2 : // iterators to be in order. Due to compaction / tombstone writing
1281 2 : // logic in finishOutput(), it is possible for range tombstones to not
1282 2 : // be strictly ordered across all files in one level.
1283 2 : //
1284 2 : // Consider this example from the metamorphic tests (also repeated in
1285 2 : // finishOutput()), consisting of three L3 files with their bounds
1286 2 : // specified in square brackets next to the file name:
1287 2 : //
1288 2 : // ./000240.sst [tmgc#391,MERGE-tmgc#391,MERGE]
1289 2 : // tmgc#391,MERGE [786e627a]
1290 2 : // tmgc-udkatvs#331,RANGEDEL
1291 2 : //
1292 2 : // ./000241.sst [tmgc#384,MERGE-tmgc#384,MERGE]
1293 2 : // tmgc#384,MERGE [666c7070]
1294 2 : // tmgc-tvsalezade#383,RANGEDEL
1295 2 : // tmgc-tvsalezade#331,RANGEDEL
1296 2 : //
1297 2 : // ./000242.sst [tmgc#383,RANGEDEL-tvsalezade#72057594037927935,RANGEDEL]
1298 2 : // tmgc-tvsalezade#383,RANGEDEL
1299 2 : // tmgc#375,SET [72646c78766965616c72776865676e79]
1300 2 : // tmgc-tvsalezade#356,RANGEDEL
1301 2 : //
1302 2 : // Here, the range tombstone in 000240.sst falls "after" one in
1303 2 : // 000241.sst, despite 000240.sst being ordered "before" 000241.sst for
1304 2 : // levelIter's purposes. While each file is still consistent before its
1305 2 : // bounds, it's safer to have all rangedel iterators be visible to
1306 2 : // mergingIter.
1307 2 : iter := level.files.Iter()
1308 2 : for f := iter.First(); f != nil; f = iter.Next() {
1309 2 : rangeDelIter, closer, err := c.newRangeDelIter(
1310 2 : newIters, iter.Take(), iterOpts, l, &c.bytesIterated)
1311 2 : if err != nil {
1312 0 : // The error will already be annotated with the BackingFileNum, so
1313 0 : // we annotate it with the FileNum.
1314 0 : return errors.Wrapf(err, "pebble: could not open table %s", errors.Safe(f.FileNum))
1315 0 : }
1316 2 : if rangeDelIter == nil {
1317 2 : continue
1318 : }
1319 2 : rangeDelIters = append(rangeDelIters, rangeDelIter)
1320 2 : c.closers = append(c.closers, closer)
1321 : }
1322 :
1323 : // Check if this level has any range keys.
1324 2 : hasRangeKeys := false
1325 2 : for f := iter.First(); f != nil; f = iter.Next() {
1326 2 : if f.HasRangeKeys {
1327 2 : hasRangeKeys = true
1328 2 : break
1329 : }
1330 : }
1331 2 : if hasRangeKeys {
1332 2 : li := &keyspan.LevelIter{}
1333 2 : newRangeKeyIterWrapper := func(file *manifest.FileMetadata, iterOptions keyspan.SpanIterOptions) (keyspan.FragmentIterator, error) {
1334 2 : iter, err := newRangeKeyIter(file, iterOptions)
1335 2 : if err != nil {
1336 0 : return nil, err
1337 2 : } else if iter == nil {
1338 0 : return emptyKeyspanIter, nil
1339 0 : }
1340 : // Ensure that the range key iter is not closed until the compaction is
1341 : // finished. This is necessary because range key processing
1342 : // requires the range keys to be held in memory for up to the
1343 : // lifetime of the compaction.
1344 2 : c.closers = append(c.closers, iter)
1345 2 : iter = noCloseIter{iter}
1346 2 :
1347 2 : // We do not need to truncate range keys to sstable boundaries, or
1348 2 : // only read within the file's atomic compaction units, unlike with
1349 2 : // range tombstones. This is because range keys were added after we
1350 2 : // stopped splitting user keys across sstables, so all the range keys
1351 2 : // in this sstable must wholly lie within the file's bounds.
1352 2 : return iter, err
1353 : }
1354 2 : li.Init(keyspan.SpanIterOptions{}, c.cmp, newRangeKeyIterWrapper, level.files.Iter(), l, manifest.KeyTypeRange)
1355 2 : rangeKeyIters = append(rangeKeyIters, li)
1356 : }
1357 2 : return nil
1358 : }
1359 :
1360 2 : for i := range c.inputs {
1361 2 : // If the level is annotated with l0SublevelInfo, expand it into one
1362 2 : // level per sublevel.
1363 2 : // TODO(jackson): Perform this expansion even earlier when we pick the
1364 2 : // compaction?
1365 2 : if len(c.inputs[i].l0SublevelInfo) > 0 {
1366 2 : for _, info := range c.startLevel.l0SublevelInfo {
1367 2 : sublevelCompactionLevel := &compactionLevel{0, info.LevelSlice, nil}
1368 2 : if err := addItersForLevel(sublevelCompactionLevel, info.sublevel); err != nil {
1369 0 : return nil, err
1370 0 : }
1371 : }
1372 2 : continue
1373 : }
1374 2 : if err := addItersForLevel(&c.inputs[i], manifest.Level(c.inputs[i].level)); err != nil {
1375 0 : return nil, err
1376 0 : }
1377 : }
1378 : }
1379 :
1380 : // In normal operation, levelIter iterates over the point operations in a
1381 : // level, and initializes a rangeDelIter pointer for the range deletions in
1382 : // each table. During compaction, we want to iterate over the merged view of
1383 : // point operations and range deletions. In order to do this we create one
1384 : // levelIter per level to iterate over the point operations, and collect up
1385 : // all the range deletion files.
1386 : //
1387 : // The range deletion levels are first combined with a keyspan.MergingIter
1388 : // (currently wrapped by a keyspan.InternalIteratorShim to satisfy the
1389 : // internal iterator interface). The resulting merged rangedel iterator is
1390 : // then included with the point levels in a single mergingIter.
1391 : //
1392 : // Combine all the rangedel iterators using a keyspan.MergingIterator and a
1393 : // InternalIteratorShim so that the range deletions may be interleaved in
1394 : // the compaction input.
1395 : // TODO(jackson): Replace the InternalIteratorShim with an interleaving
1396 : // iterator.
1397 2 : if len(rangeDelIters) > 0 {
1398 2 : c.rangeDelIter.Init(c.cmp, rangeDelIters...)
1399 2 : iters = append(iters, &c.rangeDelIter)
1400 2 : }
1401 :
1402 : // If there's only one constituent point iterator, we can avoid the overhead
1403 : // of a *mergingIter. This is possible, for example, when performing a flush
1404 : // of a single memtable. Otherwise, combine all the iterators into a merging
1405 : // iter.
1406 2 : iter := iters[0]
1407 2 : if len(iters) > 0 {
1408 2 : iter = newMergingIter(c.logger, &c.stats, c.cmp, nil, iters...)
1409 2 : }
1410 : // If there are range key iterators, we need to combine them using
1411 : // keyspan.MergingIter, and then interleave them among the points.
1412 2 : if len(rangeKeyIters) > 0 {
1413 2 : mi := &keyspan.MergingIter{}
1414 2 : mi.Init(c.cmp, rangeKeyCompactionTransform(c.equal, snapshots, c.elideRangeKey), new(keyspan.MergingBuffers), rangeKeyIters...)
1415 2 : di := &keyspan.DefragmentingIter{}
1416 2 : di.Init(c.comparer, mi, keyspan.DefragmentInternal, keyspan.StaticDefragmentReducer, new(keyspan.DefragmentingBuffers))
1417 2 : c.rangeKeyInterleaving.Init(c.comparer, iter, di, keyspan.InterleavingIterOpts{})
1418 2 : iter = &c.rangeKeyInterleaving
1419 2 : }
1420 2 : return iter, nil
1421 : }
1422 :
1423 : func (c *compaction) newRangeDelIter(
1424 : newIters tableNewIters,
1425 : f manifest.LevelFile,
1426 : opts IterOptions,
1427 : l manifest.Level,
1428 : bytesIterated *uint64,
1429 2 : ) (keyspan.FragmentIterator, io.Closer, error) {
1430 2 : opts.level = l
1431 2 : iter, rangeDelIter, err := newIters(context.Background(), f.FileMetadata,
1432 2 : &opts, internalIterOpts{
1433 2 : bytesIterated: &c.bytesIterated,
1434 2 : bufferPool: &c.bufferPool,
1435 2 : })
1436 2 : if err != nil {
1437 0 : return nil, nil, err
1438 0 : }
1439 : // TODO(peter): It is mildly wasteful to open the point iterator only to
1440 : // immediately close it. One way to solve this would be to add new
1441 : // methods to tableCache for creating point and range-deletion iterators
1442 : // independently. We'd only want to use those methods here,
1443 : // though. Doesn't seem worth the hassle in the near term.
1444 2 : if err = iter.Close(); err != nil {
1445 0 : if rangeDelIter != nil {
1446 0 : err = errors.CombineErrors(err, rangeDelIter.Close())
1447 0 : }
1448 0 : return nil, nil, err
1449 : }
1450 2 : if rangeDelIter == nil {
1451 2 : // The file doesn't contain any range deletions.
1452 2 : return nil, nil, nil
1453 2 : }
1454 :
1455 : // Ensure that rangeDelIter is not closed until the compaction is
1456 : // finished. This is necessary because range tombstone processing
1457 : // requires the range tombstones to be held in memory for up to the
1458 : // lifetime of the compaction.
1459 2 : closer := rangeDelIter
1460 2 : rangeDelIter = noCloseIter{rangeDelIter}
1461 2 :
1462 2 : // Truncate the range tombstones returned by the iterator to the
1463 2 : // upper bound of the atomic compaction unit of the file. We want to
1464 2 : // truncate the range tombstone to the bounds of the file, but files
1465 2 : // with split user keys pose an obstacle: The file's largest bound
1466 2 : // is inclusive whereas the range tombstone's end is exclusive.
1467 2 : //
1468 2 : // Consider the example:
1469 2 : //
1470 2 : // 000001:[b-f#200] range del [c,k)
1471 2 : // 000002:[f#190-g#inf] range del [c,k)
1472 2 : // 000003:[g#500-i#3]
1473 2 : //
1474 2 : // Files 000001 and 000002 contain the untruncated range tombstones
1475 2 : // [c,k). While the keyspace covered by 000003 was at one point
1476 2 : // deleted by the tombstone [c,k), the tombstone may have already
1477 2 : // been compacted away and the file does not contain an untruncated
1478 2 : // range tombstone. We want to bound 000001's tombstone to the file
1479 2 : // bounds, but it's not possible to encode a range tombstone with an
1480 2 : // end boundary within a user key (eg, between sequence numbers
1481 2 : // f#200 and f#190). Instead, we expand 000001 to its atomic
1482 2 : // compaction unit (000001 and 000002) and truncate the tombstone to
1483 2 : // g#inf.
1484 2 : //
1485 2 : // NB: We must not use the atomic compaction unit of the entire
1486 2 : // compaction, because the [c,k) tombstone contained in the file
1487 2 : // 000001 ≥ g. If 000001, 000002 and 000003 are all included in the
1488 2 : // same compaction, the compaction's atomic compaction unit includes
1489 2 : // 000003. However 000003's keys must not be covered by 000001's
1490 2 : // untruncated range tombstone.
1491 2 : //
1492 2 : // Note that we need do this truncation at read time in order to
1493 2 : // handle sstables generated by RocksDB and earlier versions of
1494 2 : // Pebble which do not truncate range tombstones to atomic
1495 2 : // compaction unit boundaries at write time.
1496 2 : //
1497 2 : // The current Pebble compaction logic DOES truncate tombstones to
1498 2 : // atomic unit boundaries at compaction time too.
1499 2 : atomicUnit, _ := expandToAtomicUnit(c.cmp, f.Slice(), true /* disableIsCompacting */)
1500 2 : lowerBound, upperBound := manifest.KeyRange(c.cmp, atomicUnit.Iter())
1501 2 : // Range deletion tombstones are often written to sstables
1502 2 : // untruncated on the end key side. However, they are still only
1503 2 : // valid within a given file's bounds. The logic for writing range
1504 2 : // tombstones to an output file sometimes has an incomplete view
1505 2 : // of range tombstones outside the file's internal key bounds. Skip
1506 2 : // any range tombstones completely outside file bounds.
1507 2 : rangeDelIter = keyspan.Truncate(
1508 2 : c.cmp, rangeDelIter, lowerBound.UserKey, upperBound.UserKey,
1509 2 : &f.Smallest, &f.Largest, false, /* panicOnUpperTruncate */
1510 2 : )
1511 2 : return rangeDelIter, closer, nil
1512 : }
1513 :
1514 1 : func (c *compaction) String() string {
1515 1 : if len(c.flushing) != 0 {
1516 0 : return "flush\n"
1517 0 : }
1518 :
1519 1 : var buf bytes.Buffer
1520 1 : for level := c.startLevel.level; level <= c.outputLevel.level; level++ {
1521 1 : i := level - c.startLevel.level
1522 1 : fmt.Fprintf(&buf, "%d:", level)
1523 1 : iter := c.inputs[i].files.Iter()
1524 1 : for f := iter.First(); f != nil; f = iter.Next() {
1525 1 : fmt.Fprintf(&buf, " %s:%s-%s", f.FileNum, f.Smallest, f.Largest)
1526 1 : }
1527 1 : fmt.Fprintf(&buf, "\n")
1528 : }
1529 1 : return buf.String()
1530 : }
1531 :
1532 : type manualCompaction struct {
1533 : // Count of the retries either due to too many concurrent compactions, or a
1534 : // concurrent compaction to overlapping levels.
1535 : retries int
1536 : level int
1537 : outputLevel int
1538 : done chan error
1539 : start []byte
1540 : end []byte
1541 : split bool
1542 : }
1543 :
1544 : type readCompaction struct {
1545 : level int
1546 : // [start, end] key ranges are used for de-duping.
1547 : start []byte
1548 : end []byte
1549 :
1550 : // The file associated with the compaction.
1551 : // If the file no longer belongs in the same
1552 : // level, then we skip the compaction.
1553 : fileNum base.FileNum
1554 : }
1555 :
1556 : type downloadSpan struct {
1557 : start []byte
1558 : end []byte
1559 : // doneChans contains a list of channels passed into compactions as done
1560 : // channels. Each channel has a buffer size of 1 and is only passed into
1561 : // one compaction. This slice can grow over the lifetime of a downloadSpan.
1562 : doneChans []chan error
1563 : // compactionsStarted is the number of compactions started for this
1564 : // downloadSpan. Must be equal to len(doneChans)-1, i.e. there's one spare
1565 : // doneChan created each time a compaction starts up, for the next compaction.
1566 : compactionsStarted int
1567 : }
1568 :
1569 2 : func (d *DB) addInProgressCompaction(c *compaction) {
1570 2 : d.mu.compact.inProgress[c] = struct{}{}
1571 2 : var isBase, isIntraL0 bool
1572 2 : for _, cl := range c.inputs {
1573 2 : iter := cl.files.Iter()
1574 2 : for f := iter.First(); f != nil; f = iter.Next() {
1575 2 : if f.IsCompacting() {
1576 0 : d.opts.Logger.Fatalf("L%d->L%d: %s already being compacted", c.startLevel.level, c.outputLevel.level, f.FileNum)
1577 0 : }
1578 2 : f.SetCompactionState(manifest.CompactionStateCompacting)
1579 2 : if c.startLevel != nil && c.outputLevel != nil && c.startLevel.level == 0 {
1580 2 : if c.outputLevel.level == 0 {
1581 2 : f.IsIntraL0Compacting = true
1582 2 : isIntraL0 = true
1583 2 : } else {
1584 2 : isBase = true
1585 2 : }
1586 : }
1587 : }
1588 : }
1589 :
1590 2 : if (isIntraL0 || isBase) && c.version.L0Sublevels != nil {
1591 2 : l0Inputs := []manifest.LevelSlice{c.startLevel.files}
1592 2 : if isIntraL0 {
1593 2 : l0Inputs = append(l0Inputs, c.outputLevel.files)
1594 2 : }
1595 2 : if err := c.version.L0Sublevels.UpdateStateForStartedCompaction(l0Inputs, isBase); err != nil {
1596 0 : d.opts.Logger.Fatalf("could not update state for compaction: %s", err)
1597 0 : }
1598 : }
1599 : }
1600 :
1601 : // Removes compaction markers from files in a compaction. The rollback parameter
1602 : // indicates whether the compaction state should be rolled back to its original
1603 : // state in the case of an unsuccessful compaction.
1604 : //
1605 : // DB.mu must be held when calling this method, however this method can drop and
1606 : // re-acquire that mutex. All writes to the manifest for this compaction should
1607 : // have completed by this point.
1608 2 : func (d *DB) clearCompactingState(c *compaction, rollback bool) {
1609 2 : c.versionEditApplied = true
1610 2 : for _, cl := range c.inputs {
1611 2 : iter := cl.files.Iter()
1612 2 : for f := iter.First(); f != nil; f = iter.Next() {
1613 2 : if !f.IsCompacting() {
1614 0 : d.opts.Logger.Fatalf("L%d->L%d: %s not being compacted", c.startLevel.level, c.outputLevel.level, f.FileNum)
1615 0 : }
1616 2 : if !rollback {
1617 2 : // On success all compactions other than move-compactions transition the
1618 2 : // file into the Compacted state. Move-compacted files become eligible
1619 2 : // for compaction again and transition back to NotCompacting.
1620 2 : if c.kind != compactionKindMove {
1621 2 : f.SetCompactionState(manifest.CompactionStateCompacted)
1622 2 : } else {
1623 2 : f.SetCompactionState(manifest.CompactionStateNotCompacting)
1624 2 : }
1625 2 : } else {
1626 2 : // Else, on rollback, all input files unconditionally transition back to
1627 2 : // NotCompacting.
1628 2 : f.SetCompactionState(manifest.CompactionStateNotCompacting)
1629 2 : }
1630 2 : f.IsIntraL0Compacting = false
1631 : }
1632 : }
1633 2 : l0InProgress := inProgressL0Compactions(d.getInProgressCompactionInfoLocked(c))
1634 2 : func() {
1635 2 : // InitCompactingFileInfo requires that no other manifest writes be
1636 2 : // happening in parallel with it, i.e. we're not in the midst of installing
1637 2 : // another version. Otherwise, it's possible that we've created another
1638 2 : // L0Sublevels instance, but not added it to the versions list, causing
1639 2 : // all the indices in FileMetadata to be inaccurate. To ensure this,
1640 2 : // grab the manifest lock.
1641 2 : d.mu.versions.logLock()
1642 2 : defer d.mu.versions.logUnlock()
1643 2 : d.mu.versions.currentVersion().L0Sublevels.InitCompactingFileInfo(l0InProgress)
1644 2 : }()
1645 : }
1646 :
1647 2 : func (d *DB) calculateDiskAvailableBytes() uint64 {
1648 2 : if space, err := d.opts.FS.GetDiskUsage(d.dirname); err == nil {
1649 1 : d.diskAvailBytes.Store(space.AvailBytes)
1650 1 : return space.AvailBytes
1651 2 : } else if !errors.Is(err, vfs.ErrUnsupported) {
1652 1 : d.opts.EventListener.BackgroundError(err)
1653 1 : }
1654 2 : return d.diskAvailBytes.Load()
1655 : }
1656 :
1657 2 : func (d *DB) getDeletionPacerInfo() deletionPacerInfo {
1658 2 : var pacerInfo deletionPacerInfo
1659 2 : // Call GetDiskUsage after every file deletion. This may seem inefficient,
1660 2 : // but in practice this was observed to take constant time, regardless of
1661 2 : // volume size used, at least on linux with ext4 and zfs. All invocations
1662 2 : // take 10 microseconds or less.
1663 2 : pacerInfo.freeBytes = d.calculateDiskAvailableBytes()
1664 2 : d.mu.Lock()
1665 2 : pacerInfo.obsoleteBytes = d.mu.versions.metrics.Table.ObsoleteSize
1666 2 : pacerInfo.liveBytes = uint64(d.mu.versions.metrics.Total().Size)
1667 2 : d.mu.Unlock()
1668 2 : return pacerInfo
1669 2 : }
1670 :
1671 : // onObsoleteTableDelete is called to update metrics when an sstable is deleted.
1672 2 : func (d *DB) onObsoleteTableDelete(fileSize uint64) {
1673 2 : d.mu.Lock()
1674 2 : d.mu.versions.metrics.Table.ObsoleteCount--
1675 2 : d.mu.versions.metrics.Table.ObsoleteSize -= fileSize
1676 2 : d.mu.Unlock()
1677 2 : }
1678 :
1679 : // maybeScheduleFlush schedules a flush if necessary.
1680 : //
1681 : // d.mu must be held when calling this.
1682 2 : func (d *DB) maybeScheduleFlush() {
1683 2 : if d.mu.compact.flushing || d.closed.Load() != nil || d.opts.ReadOnly {
1684 2 : return
1685 2 : }
1686 2 : if len(d.mu.mem.queue) <= 1 {
1687 2 : return
1688 2 : }
1689 :
1690 2 : if !d.passedFlushThreshold() {
1691 2 : return
1692 2 : }
1693 :
1694 2 : d.mu.compact.flushing = true
1695 2 : go d.flush()
1696 : }
1697 :
1698 2 : func (d *DB) passedFlushThreshold() bool {
1699 2 : var n int
1700 2 : var size uint64
1701 2 : for ; n < len(d.mu.mem.queue)-1; n++ {
1702 2 : if !d.mu.mem.queue[n].readyForFlush() {
1703 2 : break
1704 : }
1705 2 : if d.mu.mem.queue[n].flushForced {
1706 2 : // A flush was forced. Pretend the memtable size is the configured
1707 2 : // size. See minFlushSize below.
1708 2 : size += d.opts.MemTableSize
1709 2 : } else {
1710 2 : size += d.mu.mem.queue[n].totalBytes()
1711 2 : }
1712 : }
1713 2 : if n == 0 {
1714 2 : // None of the immutable memtables are ready for flushing.
1715 2 : return false
1716 2 : }
1717 :
1718 : // Only flush once the sum of the queued memtable sizes exceeds half the
1719 : // configured memtable size. This prevents flushing of memtables at startup
1720 : // while we're undergoing the ramp period on the memtable size. See
1721 : // DB.newMemTable().
1722 2 : minFlushSize := d.opts.MemTableSize / 2
1723 2 : return size >= minFlushSize
1724 : }
1725 :
1726 2 : func (d *DB) maybeScheduleDelayedFlush(tbl *memTable, dur time.Duration) {
1727 2 : var mem *flushableEntry
1728 2 : for _, m := range d.mu.mem.queue {
1729 2 : if m.flushable == tbl {
1730 2 : mem = m
1731 2 : break
1732 : }
1733 : }
1734 2 : if mem == nil || mem.flushForced {
1735 2 : return
1736 2 : }
1737 2 : deadline := d.timeNow().Add(dur)
1738 2 : if !mem.delayedFlushForcedAt.IsZero() && deadline.After(mem.delayedFlushForcedAt) {
1739 2 : // Already scheduled to flush sooner than within `dur`.
1740 2 : return
1741 2 : }
1742 2 : mem.delayedFlushForcedAt = deadline
1743 2 : go func() {
1744 2 : timer := time.NewTimer(dur)
1745 2 : defer timer.Stop()
1746 2 :
1747 2 : select {
1748 2 : case <-d.closedCh:
1749 2 : return
1750 2 : case <-mem.flushed:
1751 2 : return
1752 2 : case <-timer.C:
1753 2 : d.commit.mu.Lock()
1754 2 : defer d.commit.mu.Unlock()
1755 2 : d.mu.Lock()
1756 2 : defer d.mu.Unlock()
1757 2 :
1758 2 : // NB: The timer may fire concurrently with a call to Close. If a
1759 2 : // Close call beat us to acquiring d.mu, d.closed holds ErrClosed,
1760 2 : // and it's too late to flush anything. Otherwise, the Close call
1761 2 : // will block on locking d.mu until we've finished scheduling the
1762 2 : // flush and set `d.mu.compact.flushing` to true. Close will wait
1763 2 : // for the current flush to complete.
1764 2 : if d.closed.Load() != nil {
1765 2 : return
1766 2 : }
1767 :
1768 2 : if d.mu.mem.mutable == tbl {
1769 2 : d.makeRoomForWrite(nil)
1770 2 : } else {
1771 2 : mem.flushForced = true
1772 2 : }
1773 2 : d.maybeScheduleFlush()
1774 : }
1775 : }()
1776 : }
1777 :
1778 2 : func (d *DB) flush() {
1779 2 : pprof.Do(context.Background(), flushLabels, func(context.Context) {
1780 2 : flushingWorkStart := time.Now()
1781 2 : d.mu.Lock()
1782 2 : defer d.mu.Unlock()
1783 2 : idleDuration := flushingWorkStart.Sub(d.mu.compact.noOngoingFlushStartTime)
1784 2 : var bytesFlushed uint64
1785 2 : var err error
1786 2 : if bytesFlushed, err = d.flush1(); err != nil {
1787 1 : // TODO(peter): count consecutive flush errors and backoff.
1788 1 : d.opts.EventListener.BackgroundError(err)
1789 1 : }
1790 2 : d.mu.compact.flushing = false
1791 2 : d.mu.compact.noOngoingFlushStartTime = time.Now()
1792 2 : workDuration := d.mu.compact.noOngoingFlushStartTime.Sub(flushingWorkStart)
1793 2 : d.mu.compact.flushWriteThroughput.Bytes += int64(bytesFlushed)
1794 2 : d.mu.compact.flushWriteThroughput.WorkDuration += workDuration
1795 2 : d.mu.compact.flushWriteThroughput.IdleDuration += idleDuration
1796 2 : // More flush work may have arrived while we were flushing, so schedule
1797 2 : // another flush if needed.
1798 2 : d.maybeScheduleFlush()
1799 2 : // The flush may have produced too many files in a level, so schedule a
1800 2 : // compaction if needed.
1801 2 : d.maybeScheduleCompaction()
1802 2 : d.mu.compact.cond.Broadcast()
1803 : })
1804 : }
1805 :
1806 : // runIngestFlush is used to generate a flush version edit for sstables which
1807 : // were ingested as flushables. Both DB.mu and the manifest lock must be held
1808 : // while runIngestFlush is called.
1809 2 : func (d *DB) runIngestFlush(c *compaction) (*manifest.VersionEdit, error) {
1810 2 : if len(c.flushing) != 1 {
1811 0 : panic("pebble: ingestedFlushable must be flushed one at a time.")
1812 : }
1813 :
1814 : // Construct the VersionEdit, levelMetrics etc.
1815 2 : c.metrics = make(map[int]*LevelMetrics, numLevels)
1816 2 : // Finding the target level for ingestion must use the latest version
1817 2 : // after the logLock has been acquired.
1818 2 : c.version = d.mu.versions.currentVersion()
1819 2 :
1820 2 : baseLevel := d.mu.versions.picker.getBaseLevel()
1821 2 : iterOpts := IterOptions{logger: d.opts.Logger}
1822 2 : ve := &versionEdit{}
1823 2 : var level int
1824 2 : var err error
1825 2 : var fileToSplit *fileMetadata
1826 2 : var ingestSplitFiles []ingestSplitFile
1827 2 : for _, file := range c.flushing[0].flushable.(*ingestedFlushable).files {
1828 2 : suggestSplit := d.opts.Experimental.IngestSplit != nil && d.opts.Experimental.IngestSplit() &&
1829 2 : d.FormatMajorVersion() >= FormatVirtualSSTables
1830 2 : level, fileToSplit, err = ingestTargetLevel(
1831 2 : d.newIters, d.tableNewRangeKeyIter, iterOpts, d.opts.Comparer,
1832 2 : c.version, baseLevel, d.mu.compact.inProgress, file.FileMetadata,
1833 2 : suggestSplit,
1834 2 : )
1835 2 : if err != nil {
1836 0 : return nil, err
1837 0 : }
1838 2 : ve.NewFiles = append(ve.NewFiles, newFileEntry{Level: level, Meta: file.FileMetadata})
1839 2 : if fileToSplit != nil {
1840 0 : ingestSplitFiles = append(ingestSplitFiles, ingestSplitFile{
1841 0 : ingestFile: file.FileMetadata,
1842 0 : splitFile: fileToSplit,
1843 0 : level: level,
1844 0 : })
1845 0 : }
1846 2 : levelMetrics := c.metrics[level]
1847 2 : if levelMetrics == nil {
1848 2 : levelMetrics = &LevelMetrics{}
1849 2 : c.metrics[level] = levelMetrics
1850 2 : }
1851 2 : levelMetrics.BytesIngested += file.Size
1852 2 : levelMetrics.TablesIngested++
1853 : }
1854 :
1855 2 : updateLevelMetricsOnExcise := func(m *fileMetadata, level int, added []newFileEntry) {
1856 0 : levelMetrics := c.metrics[level]
1857 0 : if levelMetrics == nil {
1858 0 : levelMetrics = &LevelMetrics{}
1859 0 : c.metrics[level] = levelMetrics
1860 0 : }
1861 0 : levelMetrics.NumFiles--
1862 0 : levelMetrics.Size -= int64(m.Size)
1863 0 : for i := range added {
1864 0 : levelMetrics.NumFiles++
1865 0 : levelMetrics.Size += int64(added[i].Meta.Size)
1866 0 : }
1867 : }
1868 :
1869 2 : if len(ingestSplitFiles) > 0 {
1870 0 : ve.DeletedFiles = make(map[manifest.DeletedFileEntry]*manifest.FileMetadata)
1871 0 : replacedFiles := make(map[base.FileNum][]newFileEntry)
1872 0 : if err := d.ingestSplit(ve, updateLevelMetricsOnExcise, ingestSplitFiles, replacedFiles); err != nil {
1873 0 : return nil, err
1874 0 : }
1875 : }
1876 :
1877 2 : return ve, nil
1878 : }
1879 :
1880 : // flush runs a compaction that copies the immutable memtables from memory to
1881 : // disk.
1882 : //
1883 : // d.mu must be held when calling this, but the mutex may be dropped and
1884 : // re-acquired during the course of this method.
1885 2 : func (d *DB) flush1() (bytesFlushed uint64, err error) {
1886 2 : // NB: The flushable queue can contain flushables of type ingestedFlushable.
1887 2 : // The sstables in ingestedFlushable.files must be placed into the appropriate
1888 2 : // level in the lsm. Let's say the flushable queue contains a prefix of
1889 2 : // regular immutable memtables, then an ingestedFlushable, and then the
1890 2 : // mutable memtable. When the flush of the ingestedFlushable is performed,
1891 2 : // it needs an updated view of the lsm. That is, the prefix of immutable
1892 2 : // memtables must have already been flushed. Similarly, if there are two
1893 2 : // contiguous ingestedFlushables in the queue, then the first flushable must
1894 2 : // be flushed, so that the second flushable can see an updated view of the
1895 2 : // lsm.
1896 2 : //
1897 2 : // Given the above, we restrict flushes to either some prefix of regular
1898 2 : // memtables, or a single flushable of type ingestedFlushable. The DB.flush
1899 2 : // function will call DB.maybeScheduleFlush again, so a new flush to finish
1900 2 : // the remaining flush work should be scheduled right away.
1901 2 : //
1902 2 : // NB: Large batches placed in the flushable queue share the WAL with the
1903 2 : // previous memtable in the queue. We must ensure the property that both the
1904 2 : // large batch and the memtable with which it shares a WAL are flushed
1905 2 : // together. The property ensures that the minimum unflushed log number
1906 2 : // isn't incremented incorrectly. Since a flushableBatch.readyToFlush always
1907 2 : // returns true, and since the large batch will always be placed right after
1908 2 : // the memtable with which it shares a WAL, the property is naturally
1909 2 : // ensured. The large batch will always be placed after the memtable with
1910 2 : // which it shares a WAL because we ensure it in DB.commitWrite by holding
1911 2 : // the commitPipeline.mu and then holding DB.mu. As an extra defensive
1912 2 : // measure, if we try to flush the memtable without also flushing the
1913 2 : // flushable batch in the same flush, since the memtable and flushableBatch
1914 2 : // have the same logNum, the logNum invariant check below will trigger.
1915 2 : var n, inputs int
1916 2 : var inputBytes uint64
1917 2 : var ingest bool
1918 2 : for ; n < len(d.mu.mem.queue)-1; n++ {
1919 2 : if f, ok := d.mu.mem.queue[n].flushable.(*ingestedFlushable); ok {
1920 2 : if n == 0 {
1921 2 : // The first flushable is of type ingestedFlushable. Since these
1922 2 : // must be flushed individually, we perform a flush for just
1923 2 : // this.
1924 2 : if !f.readyForFlush() {
1925 0 : // This check is almost unnecessary, but we guard against it
1926 0 : // just in case this invariant changes in the future.
1927 0 : panic("pebble: ingestedFlushable should always be ready to flush.")
1928 : }
1929 : // By setting n = 1, we ensure that the first flushable(n == 0)
1930 : // is scheduled for a flush. The number of tables added is equal to the
1931 : // number of files in the ingest operation.
1932 2 : n = 1
1933 2 : inputs = len(f.files)
1934 2 : ingest = true
1935 2 : break
1936 2 : } else {
1937 2 : // There was some prefix of flushables which weren't of type
1938 2 : // ingestedFlushable. So, perform a flush for those.
1939 2 : break
1940 : }
1941 : }
1942 2 : if !d.mu.mem.queue[n].readyForFlush() {
1943 1 : break
1944 : }
1945 2 : inputBytes += d.mu.mem.queue[n].inuseBytes()
1946 : }
1947 2 : if n == 0 {
1948 0 : // None of the immutable memtables are ready for flushing.
1949 0 : return 0, nil
1950 0 : }
1951 2 : if !ingest {
1952 2 : // Flushes of memtables add the prefix of n memtables from the flushable
1953 2 : // queue.
1954 2 : inputs = n
1955 2 : }
1956 :
1957 : // Require that every memtable being flushed has a log number less than the
1958 : // new minimum unflushed log number.
1959 2 : minUnflushedLogNum := d.mu.mem.queue[n].logNum
1960 2 : if !d.opts.DisableWAL {
1961 2 : for i := 0; i < n; i++ {
1962 2 : if logNum := d.mu.mem.queue[i].logNum; logNum >= minUnflushedLogNum {
1963 0 : panic(errors.AssertionFailedf("logNum invariant violated: flushing %d items; %d:type=%T,logNum=%d; %d:type=%T,logNum=%d",
1964 0 : n,
1965 0 : i, d.mu.mem.queue[i].flushable, logNum,
1966 0 : n, d.mu.mem.queue[n].flushable, minUnflushedLogNum))
1967 : }
1968 : }
1969 : }
1970 :
1971 2 : c := newFlush(d.opts, d.mu.versions.currentVersion(),
1972 2 : d.mu.versions.picker.getBaseLevel(), d.mu.mem.queue[:n], d.timeNow())
1973 2 : d.addInProgressCompaction(c)
1974 2 :
1975 2 : jobID := d.mu.nextJobID
1976 2 : d.mu.nextJobID++
1977 2 : d.opts.EventListener.FlushBegin(FlushInfo{
1978 2 : JobID: jobID,
1979 2 : Input: inputs,
1980 2 : InputBytes: inputBytes,
1981 2 : Ingest: ingest,
1982 2 : })
1983 2 : startTime := d.timeNow()
1984 2 :
1985 2 : var ve *manifest.VersionEdit
1986 2 : var pendingOutputs []physicalMeta
1987 2 : var stats compactStats
1988 2 : // To determine the target level of the files in the ingestedFlushable, we
1989 2 : // need to acquire the logLock, and not release it for that duration. Since,
1990 2 : // we need to acquire the logLock below to perform the logAndApply step
1991 2 : // anyway, we create the VersionEdit for ingestedFlushable outside of
1992 2 : // runCompaction. For all other flush cases, we construct the VersionEdit
1993 2 : // inside runCompaction.
1994 2 : if c.kind != compactionKindIngestedFlushable {
1995 2 : ve, pendingOutputs, stats, err = d.runCompaction(jobID, c)
1996 2 : }
1997 :
1998 : // Acquire logLock. This will be released either on an error, by way of
1999 : // logUnlock, or through a call to logAndApply if there is no error.
2000 2 : d.mu.versions.logLock()
2001 2 :
2002 2 : if c.kind == compactionKindIngestedFlushable {
2003 2 : ve, err = d.runIngestFlush(c)
2004 2 : }
2005 :
2006 2 : info := FlushInfo{
2007 2 : JobID: jobID,
2008 2 : Input: inputs,
2009 2 : InputBytes: inputBytes,
2010 2 : Duration: d.timeNow().Sub(startTime),
2011 2 : Done: true,
2012 2 : Ingest: ingest,
2013 2 : Err: err,
2014 2 : }
2015 2 : if err == nil {
2016 2 : for i := range ve.NewFiles {
2017 2 : e := &ve.NewFiles[i]
2018 2 : info.Output = append(info.Output, e.Meta.TableInfo())
2019 2 : // Ingested tables are not necessarily flushed to L0. Record the level of
2020 2 : // each ingested file explicitly.
2021 2 : if ingest {
2022 2 : info.IngestLevels = append(info.IngestLevels, e.Level)
2023 2 : }
2024 : }
2025 2 : if len(ve.NewFiles) == 0 {
2026 2 : info.Err = errEmptyTable
2027 2 : }
2028 :
2029 : // The flush succeeded or it produced an empty sstable. In either case we
2030 : // want to bump the minimum unflushed log number to the log number of the
2031 : // oldest unflushed memtable.
2032 2 : ve.MinUnflushedLogNum = minUnflushedLogNum
2033 2 : if c.kind != compactionKindIngestedFlushable {
2034 2 : metrics := c.metrics[0]
2035 2 : if d.opts.DisableWAL {
2036 2 : // If the WAL is disabled, every flushable has a zero [logSize],
2037 2 : // resulting in zero bytes in. Instead, use the number of bytes we
2038 2 : // flushed as the BytesIn. This ensures we get a reasonable w-amp
2039 2 : // calculation even when the WAL is disabled.
2040 2 : metrics.BytesIn = metrics.BytesFlushed
2041 2 : } else {
2042 2 : metrics := c.metrics[0]
2043 2 : for i := 0; i < n; i++ {
2044 2 : metrics.BytesIn += d.mu.mem.queue[i].logSize
2045 2 : }
2046 : }
2047 2 : } else if len(ve.DeletedFiles) > 0 {
2048 0 : // c.kind == compactionKindIngestedFlushable && we have deleted files due
2049 0 : // to ingest-time splits.
2050 0 : //
2051 0 : // Iterate through all other compactions, and check if their inputs have
2052 0 : // been replaced due to an ingest-time split. In that case, cancel the
2053 0 : // compaction.
2054 0 : for c2 := range d.mu.compact.inProgress {
2055 0 : for i := range c2.inputs {
2056 0 : iter := c2.inputs[i].files.Iter()
2057 0 : for f := iter.First(); f != nil; f = iter.Next() {
2058 0 : if _, ok := ve.DeletedFiles[deletedFileEntry{FileNum: f.FileNum, Level: c2.inputs[i].level}]; ok {
2059 0 : c2.cancel.Store(true)
2060 0 : break
2061 : }
2062 : }
2063 : }
2064 : }
2065 : }
2066 2 : err = d.mu.versions.logAndApply(jobID, ve, c.metrics, false, /* forceRotation */
2067 2 : func() []compactionInfo { return d.getInProgressCompactionInfoLocked(c) })
2068 2 : if err != nil {
2069 1 : info.Err = err
2070 1 : // TODO(peter): untested.
2071 1 : for _, f := range pendingOutputs {
2072 1 : // Note that the FileBacking for the file metadata might not have
2073 1 : // been set yet. So, we directly use the FileNum. Since these
2074 1 : // files were generated as compaction outputs, these must be
2075 1 : // physical files on disk. This property might not hold once
2076 1 : // https://github.com/cockroachdb/pebble/issues/389 is
2077 1 : // implemented if #389 creates virtual sstables as output files.
2078 1 : d.mu.versions.obsoleteTables = append(
2079 1 : d.mu.versions.obsoleteTables,
2080 1 : fileInfo{f.FileNum.DiskFileNum(), f.Size},
2081 1 : )
2082 1 : }
2083 1 : d.mu.versions.updateObsoleteTableMetricsLocked()
2084 : }
2085 1 : } else {
2086 1 : // We won't be performing the logAndApply step because of the error,
2087 1 : // so logUnlock.
2088 1 : d.mu.versions.logUnlock()
2089 1 : }
2090 :
2091 2 : bytesFlushed = c.bytesIterated
2092 2 :
2093 2 : // If err != nil, then the flush will be retried, and we will recalculate
2094 2 : // these metrics.
2095 2 : if err == nil {
2096 2 : d.mu.snapshots.cumulativePinnedCount += stats.cumulativePinnedKeys
2097 2 : d.mu.snapshots.cumulativePinnedSize += stats.cumulativePinnedSize
2098 2 : d.mu.versions.metrics.Keys.MissizedTombstonesCount += stats.countMissizedDels
2099 2 : d.maybeUpdateDeleteCompactionHints(c)
2100 2 : }
2101 :
2102 2 : d.clearCompactingState(c, err != nil)
2103 2 : delete(d.mu.compact.inProgress, c)
2104 2 : d.mu.versions.incrementCompactions(c.kind, c.extraLevels, c.pickerMetrics)
2105 2 :
2106 2 : var flushed flushableList
2107 2 : if err == nil {
2108 2 : flushed = d.mu.mem.queue[:n]
2109 2 : d.mu.mem.queue = d.mu.mem.queue[n:]
2110 2 : d.updateReadStateLocked(d.opts.DebugCheck)
2111 2 : d.updateTableStatsLocked(ve.NewFiles)
2112 2 : if ingest {
2113 2 : d.mu.versions.metrics.Flush.AsIngestCount++
2114 2 : for _, l := range c.metrics {
2115 2 : d.mu.versions.metrics.Flush.AsIngestBytes += l.BytesIngested
2116 2 : d.mu.versions.metrics.Flush.AsIngestTableCount += l.TablesIngested
2117 2 : }
2118 : }
2119 :
2120 : // Update if any eventually file-only snapshots have now transitioned to
2121 : // being file-only.
2122 2 : earliestUnflushedSeqNum := d.getEarliestUnflushedSeqNumLocked()
2123 2 : currentVersion := d.mu.versions.currentVersion()
2124 2 : for s := d.mu.snapshots.root.next; s != &d.mu.snapshots.root; {
2125 2 : if s.efos == nil {
2126 2 : s = s.next
2127 2 : continue
2128 : }
2129 2 : if base.Visible(earliestUnflushedSeqNum, s.efos.seqNum, InternalKeySeqNumMax) {
2130 1 : s = s.next
2131 1 : continue
2132 : }
2133 2 : if s.efos.excised.Load() {
2134 1 : // If a concurrent excise has happened that overlaps with one of the key
2135 1 : // ranges this snapshot is interested in, this EFOS cannot transition to
2136 1 : // a file-only snapshot as keys in that range could now be deleted. Move
2137 1 : // onto the next snapshot.
2138 1 : s = s.next
2139 1 : continue
2140 : }
2141 2 : currentVersion.Ref()
2142 2 :
2143 2 : // NB: s.efos.transitionToFileOnlySnapshot could close s, in which
2144 2 : // case s.next would be nil. Save it before calling it.
2145 2 : next := s.next
2146 2 : _ = s.efos.transitionToFileOnlySnapshot(currentVersion)
2147 2 : s = next
2148 : }
2149 : }
2150 : // Signal FlushEnd after installing the new readState. This helps for unit
2151 : // tests that use the callback to trigger a read using an iterator with
2152 : // IterOptions.OnlyReadGuaranteedDurable.
2153 2 : info.TotalDuration = d.timeNow().Sub(startTime)
2154 2 : d.opts.EventListener.FlushEnd(info)
2155 2 :
2156 2 : // The order of these operations matters here for ease of testing.
2157 2 : // Removing the reader reference first allows tests to be guaranteed that
2158 2 : // the memtable reservation has been released by the time a synchronous
2159 2 : // flush returns. readerUnrefLocked may also produce obsolete files so the
2160 2 : // call to deleteObsoleteFiles must happen after it.
2161 2 : for i := range flushed {
2162 2 : flushed[i].readerUnrefLocked(true)
2163 2 : }
2164 :
2165 2 : d.deleteObsoleteFiles(jobID)
2166 2 :
2167 2 : // Mark all the memtables we flushed as flushed.
2168 2 : for i := range flushed {
2169 2 : close(flushed[i].flushed)
2170 2 : }
2171 :
2172 2 : return bytesFlushed, err
2173 : }
2174 :
2175 : // maybeScheduleCompactionAsync should be used when
2176 : // we want to possibly schedule a compaction, but don't
2177 : // want to eat the cost of running maybeScheduleCompaction.
2178 : // This method should be launched in a separate goroutine.
2179 : // d.mu must not be held when this is called.
2180 0 : func (d *DB) maybeScheduleCompactionAsync() {
2181 0 : defer d.compactionSchedulers.Done()
2182 0 :
2183 0 : d.mu.Lock()
2184 0 : d.maybeScheduleCompaction()
2185 0 : d.mu.Unlock()
2186 0 : }
2187 :
2188 : // maybeScheduleCompaction schedules a compaction if necessary.
2189 : //
2190 : // d.mu must be held when calling this.
2191 2 : func (d *DB) maybeScheduleCompaction() {
2192 2 : d.maybeScheduleCompactionPicker(pickAuto)
2193 2 : }
2194 :
2195 2 : func pickAuto(picker compactionPicker, env compactionEnv) *pickedCompaction {
2196 2 : return picker.pickAuto(env)
2197 2 : }
2198 :
2199 2 : func pickElisionOnly(picker compactionPicker, env compactionEnv) *pickedCompaction {
2200 2 : return picker.pickElisionOnlyCompaction(env)
2201 2 : }
2202 :
2203 : // maybeScheduleDownloadCompaction schedules a download compaction.
2204 : //
2205 : // Requires d.mu to be held.
2206 2 : func (d *DB) maybeScheduleDownloadCompaction(env compactionEnv, maxConcurrentCompactions int) {
2207 2 : for len(d.mu.compact.downloads) > 0 && d.mu.compact.compactingCount < maxConcurrentCompactions {
2208 1 : v := d.mu.versions.currentVersion()
2209 1 : download := d.mu.compact.downloads[0]
2210 1 : env.inProgressCompactions = d.getInProgressCompactionInfoLocked(nil)
2211 1 : var externalFile *fileMetadata
2212 1 : var err error
2213 1 : var level int
2214 1 : for i := range v.Levels {
2215 1 : overlaps := v.Overlaps(i, d.cmp, download.start, download.end, true /* exclusiveEnd */)
2216 1 : iter := overlaps.Iter()
2217 1 : provider := d.objProvider
2218 1 : for f := iter.First(); f != nil; f = iter.Next() {
2219 1 : var objMeta objstorage.ObjectMetadata
2220 1 : objMeta, err = provider.Lookup(fileTypeTable, f.FileBacking.DiskFileNum)
2221 1 : if err != nil {
2222 0 : break
2223 : }
2224 1 : if objMeta.IsExternal() {
2225 1 : if f.IsCompacting() {
2226 0 : continue
2227 : }
2228 1 : externalFile = f
2229 1 : level = i
2230 1 : break
2231 : }
2232 : }
2233 1 : if externalFile != nil || err != nil {
2234 1 : break
2235 : }
2236 : }
2237 1 : if err != nil {
2238 0 : d.mu.compact.downloads = d.mu.compact.downloads[1:]
2239 0 : download.doneChans[download.compactionsStarted] <- err
2240 0 : continue
2241 : }
2242 1 : if externalFile == nil {
2243 1 : // The entirety of this span is downloaded, or is being downloaded right
2244 1 : // now. No need to schedule additional downloads for this span.
2245 1 : d.mu.compact.downloads = d.mu.compact.downloads[1:]
2246 1 : continue
2247 : }
2248 1 : pc := pickDownloadCompaction(v, d.opts, env, d.mu.versions.picker.getBaseLevel(), download, level, externalFile)
2249 1 : if pc != nil {
2250 1 : doneCh := download.doneChans[download.compactionsStarted]
2251 1 : download.compactionsStarted++
2252 1 : // Create another doneChan for the next compaction.
2253 1 : download.doneChans = append(download.doneChans, make(chan error, 1))
2254 1 :
2255 1 : c := newCompaction(pc, d.opts, d.timeNow(), d.ObjProvider())
2256 1 : d.mu.compact.compactingCount++
2257 1 : d.addInProgressCompaction(c)
2258 1 : go d.compact(c, doneCh)
2259 1 : }
2260 : }
2261 : }
2262 :
2263 : // maybeScheduleCompactionPicker schedules a compaction if necessary,
2264 : // calling `pickFunc` to pick automatic compactions.
2265 : //
2266 : // d.mu must be held when calling this.
2267 : func (d *DB) maybeScheduleCompactionPicker(
2268 : pickFunc func(compactionPicker, compactionEnv) *pickedCompaction,
2269 2 : ) {
2270 2 : if d.closed.Load() != nil || d.opts.ReadOnly {
2271 2 : return
2272 2 : }
2273 2 : maxConcurrentCompactions := d.opts.MaxConcurrentCompactions()
2274 2 : if d.mu.compact.compactingCount >= maxConcurrentCompactions {
2275 2 : if len(d.mu.compact.manual) > 0 {
2276 2 : // Inability to run head blocks later manual compactions.
2277 2 : d.mu.compact.manual[0].retries++
2278 2 : }
2279 2 : return
2280 : }
2281 :
2282 : // Compaction picking needs a coherent view of a Version. In particular, we
2283 : // need to exlude concurrent ingestions from making a decision on which level
2284 : // to ingest into that conflicts with our compaction
2285 : // decision. versionSet.logLock provides the necessary mutual exclusion.
2286 2 : d.mu.versions.logLock()
2287 2 : defer d.mu.versions.logUnlock()
2288 2 :
2289 2 : // Check for the closed flag again, in case the DB was closed while we were
2290 2 : // waiting for logLock().
2291 2 : if d.closed.Load() != nil {
2292 2 : return
2293 2 : }
2294 :
2295 2 : env := compactionEnv{
2296 2 : diskAvailBytes: d.diskAvailBytes.Load(),
2297 2 : earliestSnapshotSeqNum: d.mu.snapshots.earliest(),
2298 2 : earliestUnflushedSeqNum: d.getEarliestUnflushedSeqNumLocked(),
2299 2 : }
2300 2 :
2301 2 : // Check for delete-only compactions first, because they're expected to be
2302 2 : // cheap and reduce future compaction work.
2303 2 : if !d.opts.private.disableDeleteOnlyCompactions &&
2304 2 : len(d.mu.compact.deletionHints) > 0 &&
2305 2 : !d.opts.DisableAutomaticCompactions {
2306 2 : v := d.mu.versions.currentVersion()
2307 2 : snapshots := d.mu.snapshots.toSlice()
2308 2 : inputs, unresolvedHints := checkDeleteCompactionHints(d.cmp, v, d.mu.compact.deletionHints, snapshots)
2309 2 : d.mu.compact.deletionHints = unresolvedHints
2310 2 :
2311 2 : if len(inputs) > 0 {
2312 2 : c := newDeleteOnlyCompaction(d.opts, v, inputs, d.timeNow())
2313 2 : d.mu.compact.compactingCount++
2314 2 : d.addInProgressCompaction(c)
2315 2 : go d.compact(c, nil)
2316 2 : }
2317 : }
2318 :
2319 2 : for len(d.mu.compact.manual) > 0 && d.mu.compact.compactingCount < maxConcurrentCompactions {
2320 2 : v := d.mu.versions.currentVersion()
2321 2 : manual := d.mu.compact.manual[0]
2322 2 : env.inProgressCompactions = d.getInProgressCompactionInfoLocked(nil)
2323 2 : pc, retryLater := pickManualCompaction(v, d.opts, env, d.mu.versions.picker.getBaseLevel(), manual)
2324 2 : if pc != nil {
2325 2 : c := newCompaction(pc, d.opts, d.timeNow(), d.ObjProvider())
2326 2 : d.mu.compact.manual = d.mu.compact.manual[1:]
2327 2 : d.mu.compact.compactingCount++
2328 2 : d.addInProgressCompaction(c)
2329 2 : go d.compact(c, manual.done)
2330 2 : } else if !retryLater {
2331 2 : // Noop
2332 2 : d.mu.compact.manual = d.mu.compact.manual[1:]
2333 2 : manual.done <- nil
2334 2 : } else {
2335 2 : // Inability to run head blocks later manual compactions.
2336 2 : manual.retries++
2337 2 : break
2338 : }
2339 : }
2340 :
2341 2 : for !d.opts.DisableAutomaticCompactions && d.mu.compact.compactingCount < maxConcurrentCompactions {
2342 2 : env.inProgressCompactions = d.getInProgressCompactionInfoLocked(nil)
2343 2 : env.readCompactionEnv = readCompactionEnv{
2344 2 : readCompactions: &d.mu.compact.readCompactions,
2345 2 : flushing: d.mu.compact.flushing || d.passedFlushThreshold(),
2346 2 : rescheduleReadCompaction: &d.mu.compact.rescheduleReadCompaction,
2347 2 : }
2348 2 : pc := pickFunc(d.mu.versions.picker, env)
2349 2 : if pc == nil {
2350 2 : break
2351 : }
2352 2 : c := newCompaction(pc, d.opts, d.timeNow(), d.ObjProvider())
2353 2 : d.mu.compact.compactingCount++
2354 2 : d.addInProgressCompaction(c)
2355 2 : go d.compact(c, nil)
2356 : }
2357 :
2358 2 : d.maybeScheduleDownloadCompaction(env, maxConcurrentCompactions)
2359 : }
2360 :
2361 : // deleteCompactionHintType indicates whether the deleteCompactionHint was
2362 : // generated from a span containing a range del (point key only), a range key
2363 : // delete (range key only), or both a point and range key.
2364 : type deleteCompactionHintType uint8
2365 :
2366 : const (
2367 : // NOTE: While these are primarily used as enumeration types, they are also
2368 : // used for some bitwise operations. Care should be taken when updating.
2369 : deleteCompactionHintTypeUnknown deleteCompactionHintType = iota
2370 : deleteCompactionHintTypePointKeyOnly
2371 : deleteCompactionHintTypeRangeKeyOnly
2372 : deleteCompactionHintTypePointAndRangeKey
2373 : )
2374 :
2375 : // String implements fmt.Stringer.
2376 1 : func (h deleteCompactionHintType) String() string {
2377 1 : switch h {
2378 0 : case deleteCompactionHintTypeUnknown:
2379 0 : return "unknown"
2380 1 : case deleteCompactionHintTypePointKeyOnly:
2381 1 : return "point-key-only"
2382 1 : case deleteCompactionHintTypeRangeKeyOnly:
2383 1 : return "range-key-only"
2384 1 : case deleteCompactionHintTypePointAndRangeKey:
2385 1 : return "point-and-range-key"
2386 0 : default:
2387 0 : panic(fmt.Sprintf("unknown hint type: %d", h))
2388 : }
2389 : }
2390 :
2391 : // compactionHintFromKeys returns a deleteCompactionHintType given a slice of
2392 : // keyspan.Keys.
2393 2 : func compactionHintFromKeys(keys []keyspan.Key) deleteCompactionHintType {
2394 2 : var hintType deleteCompactionHintType
2395 2 : for _, k := range keys {
2396 2 : switch k.Kind() {
2397 2 : case base.InternalKeyKindRangeDelete:
2398 2 : hintType |= deleteCompactionHintTypePointKeyOnly
2399 2 : case base.InternalKeyKindRangeKeyDelete:
2400 2 : hintType |= deleteCompactionHintTypeRangeKeyOnly
2401 0 : default:
2402 0 : panic(fmt.Sprintf("unsupported key kind: %s", k.Kind()))
2403 : }
2404 : }
2405 2 : return hintType
2406 : }
2407 :
2408 : // A deleteCompactionHint records a user key and sequence number span that has been
2409 : // deleted by a range tombstone. A hint is recorded if at least one sstable
2410 : // falls completely within both the user key and sequence number spans.
2411 : // Once the tombstones and the observed completely-contained sstables fall
2412 : // into the same snapshot stripe, a delete-only compaction may delete any
2413 : // sstables within the range.
2414 : type deleteCompactionHint struct {
2415 : // The type of key span that generated this hint (point key, range key, or
2416 : // both).
2417 : hintType deleteCompactionHintType
2418 : // start and end are user keys specifying a key range [start, end) of
2419 : // deleted keys.
2420 : start []byte
2421 : end []byte
2422 : // The level of the file containing the range tombstone(s) when the hint
2423 : // was created. Only lower levels need to be searched for files that may
2424 : // be deleted.
2425 : tombstoneLevel int
2426 : // The file containing the range tombstone(s) that created the hint.
2427 : tombstoneFile *fileMetadata
2428 : // The smallest and largest sequence numbers of the abutting tombstones
2429 : // merged to form this hint. All of a tables' keys must be less than the
2430 : // tombstone smallest sequence number to be deleted. All of a tables'
2431 : // sequence numbers must fall into the same snapshot stripe as the
2432 : // tombstone largest sequence number to be deleted.
2433 : tombstoneLargestSeqNum uint64
2434 : tombstoneSmallestSeqNum uint64
2435 : // The smallest sequence number of a sstable that was found to be covered
2436 : // by this hint. The hint cannot be resolved until this sequence number is
2437 : // in the same snapshot stripe as the largest tombstone sequence number.
2438 : // This is set when a hint is created, so the LSM may look different and
2439 : // notably no longer contain the sstable that contained the key at this
2440 : // sequence number.
2441 : fileSmallestSeqNum uint64
2442 : }
2443 :
2444 1 : func (h deleteCompactionHint) String() string {
2445 1 : return fmt.Sprintf(
2446 1 : "L%d.%s %s-%s seqnums(tombstone=%d-%d, file-smallest=%d, type=%s)",
2447 1 : h.tombstoneLevel, h.tombstoneFile.FileNum, h.start, h.end,
2448 1 : h.tombstoneSmallestSeqNum, h.tombstoneLargestSeqNum, h.fileSmallestSeqNum,
2449 1 : h.hintType,
2450 1 : )
2451 1 : }
2452 :
2453 2 : func (h *deleteCompactionHint) canDelete(cmp Compare, m *fileMetadata, snapshots []uint64) bool {
2454 2 : // The file can only be deleted if all of its keys are older than the
2455 2 : // earliest tombstone aggregated into the hint.
2456 2 : if m.LargestSeqNum >= h.tombstoneSmallestSeqNum || m.SmallestSeqNum < h.fileSmallestSeqNum {
2457 2 : return false
2458 2 : }
2459 :
2460 : // The file's oldest key must be in the same snapshot stripe as the
2461 : // newest tombstone. NB: We already checked the hint's sequence numbers,
2462 : // but this file's oldest sequence number might be lower than the hint's
2463 : // smallest sequence number despite the file falling within the key range
2464 : // if this file was constructed after the hint by a compaction.
2465 2 : ti, _ := snapshotIndex(h.tombstoneLargestSeqNum, snapshots)
2466 2 : fi, _ := snapshotIndex(m.SmallestSeqNum, snapshots)
2467 2 : if ti != fi {
2468 0 : return false
2469 0 : }
2470 :
2471 2 : switch h.hintType {
2472 2 : case deleteCompactionHintTypePointKeyOnly:
2473 2 : // A hint generated by a range del span cannot delete tables that contain
2474 2 : // range keys.
2475 2 : if m.HasRangeKeys {
2476 1 : return false
2477 1 : }
2478 2 : case deleteCompactionHintTypeRangeKeyOnly:
2479 2 : // A hint generated by a range key del span cannot delete tables that
2480 2 : // contain point keys.
2481 2 : if m.HasPointKeys {
2482 2 : return false
2483 2 : }
2484 2 : case deleteCompactionHintTypePointAndRangeKey:
2485 : // A hint from a span that contains both range dels *and* range keys can
2486 : // only be deleted if both bounds fall within the hint. The next check takes
2487 : // care of this.
2488 0 : default:
2489 0 : panic(fmt.Sprintf("pebble: unknown delete compaction hint type: %d", h.hintType))
2490 : }
2491 :
2492 : // The file's keys must be completely contained within the hint range.
2493 2 : return cmp(h.start, m.Smallest.UserKey) <= 0 && cmp(m.Largest.UserKey, h.end) < 0
2494 : }
2495 :
2496 2 : func (d *DB) maybeUpdateDeleteCompactionHints(c *compaction) {
2497 2 : // Compactions that zero sequence numbers can interfere with compaction
2498 2 : // deletion hints. Deletion hints apply to tables containing keys older
2499 2 : // than a threshold. If a key more recent than the threshold is zeroed in
2500 2 : // a compaction, a delete-only compaction may mistake it as meeting the
2501 2 : // threshold and drop a table containing live data.
2502 2 : //
2503 2 : // To avoid this scenario, compactions that zero sequence numbers remove
2504 2 : // any conflicting deletion hints. A deletion hint is conflicting if both
2505 2 : // of the following conditions apply:
2506 2 : // * its key space overlaps with the compaction
2507 2 : // * at least one of its inputs contains a key as recent as one of the
2508 2 : // hint's tombstones.
2509 2 : //
2510 2 : if !c.allowedZeroSeqNum {
2511 2 : return
2512 2 : }
2513 :
2514 2 : updatedHints := d.mu.compact.deletionHints[:0]
2515 2 : for _, h := range d.mu.compact.deletionHints {
2516 2 : // If the compaction's key space is disjoint from the hint's key
2517 2 : // space, the zeroing of sequence numbers won't affect the hint. Keep
2518 2 : // the hint.
2519 2 : keysDisjoint := d.cmp(h.end, c.smallest.UserKey) < 0 || d.cmp(h.start, c.largest.UserKey) > 0
2520 2 : if keysDisjoint {
2521 1 : updatedHints = append(updatedHints, h)
2522 1 : continue
2523 : }
2524 :
2525 : // All of the compaction's inputs must be older than the hint's
2526 : // tombstones.
2527 2 : inputsOlder := true
2528 2 : for _, in := range c.inputs {
2529 2 : iter := in.files.Iter()
2530 2 : for f := iter.First(); f != nil; f = iter.Next() {
2531 2 : inputsOlder = inputsOlder && f.LargestSeqNum < h.tombstoneSmallestSeqNum
2532 2 : }
2533 : }
2534 2 : if inputsOlder {
2535 1 : updatedHints = append(updatedHints, h)
2536 1 : continue
2537 : }
2538 :
2539 : // Drop h, because the compaction c may have zeroed sequence numbers
2540 : // of keys more recent than some of h's tombstones.
2541 : }
2542 2 : d.mu.compact.deletionHints = updatedHints
2543 : }
2544 :
2545 : func checkDeleteCompactionHints(
2546 : cmp Compare, v *version, hints []deleteCompactionHint, snapshots []uint64,
2547 2 : ) ([]compactionLevel, []deleteCompactionHint) {
2548 2 : var files map[*fileMetadata]bool
2549 2 : var byLevel [numLevels][]*fileMetadata
2550 2 :
2551 2 : unresolvedHints := hints[:0]
2552 2 : for _, h := range hints {
2553 2 : // Check each compaction hint to see if it's resolvable. Resolvable
2554 2 : // hints are removed and trigger a delete-only compaction if any files
2555 2 : // in the current LSM still meet their criteria. Unresolvable hints
2556 2 : // are saved and don't trigger a delete-only compaction.
2557 2 : //
2558 2 : // When a compaction hint is created, the sequence numbers of the
2559 2 : // range tombstones and the covered file with the oldest key are
2560 2 : // recorded. The largest tombstone sequence number and the smallest
2561 2 : // file sequence number must be in the same snapshot stripe for the
2562 2 : // hint to be resolved. The below graphic models a compaction hint
2563 2 : // covering the keyspace [b, r). The hint completely contains two
2564 2 : // files, 000002 and 000003. The file 000003 contains the lowest
2565 2 : // covered sequence number at #90. The tombstone b.RANGEDEL.230:h has
2566 2 : // the highest tombstone sequence number incorporated into the hint.
2567 2 : // The hint may be resolved only once the snapshots at #100, #180 and
2568 2 : // #210 are all closed. File 000001 is not included within the hint
2569 2 : // because it extends beyond the range tombstones in user key space.
2570 2 : //
2571 2 : // 250
2572 2 : //
2573 2 : // |-b...230:h-|
2574 2 : // _____________________________________________________ snapshot #210
2575 2 : // 200 |--h.RANGEDEL.200:r--|
2576 2 : //
2577 2 : // _____________________________________________________ snapshot #180
2578 2 : //
2579 2 : // 150 +--------+
2580 2 : // +---------+ | 000003 |
2581 2 : // | 000002 | | |
2582 2 : // +_________+ | |
2583 2 : // 100_____________________|________|___________________ snapshot #100
2584 2 : // +--------+
2585 2 : // _____________________________________________________ snapshot #70
2586 2 : // +---------------+
2587 2 : // 50 | 000001 |
2588 2 : // | |
2589 2 : // +---------------+
2590 2 : // ______________________________________________________________
2591 2 : // 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
2592 2 :
2593 2 : ti, _ := snapshotIndex(h.tombstoneLargestSeqNum, snapshots)
2594 2 : fi, _ := snapshotIndex(h.fileSmallestSeqNum, snapshots)
2595 2 : if ti != fi {
2596 2 : // Cannot resolve yet.
2597 2 : unresolvedHints = append(unresolvedHints, h)
2598 2 : continue
2599 : }
2600 :
2601 : // The hint h will be resolved and dropped, regardless of whether
2602 : // there are any tables that can be deleted.
2603 2 : for l := h.tombstoneLevel + 1; l < numLevels; l++ {
2604 2 : overlaps := v.Overlaps(l, cmp, h.start, h.end, true /* exclusiveEnd */)
2605 2 : iter := overlaps.Iter()
2606 2 : for m := iter.First(); m != nil; m = iter.Next() {
2607 2 : if m.IsCompacting() || !h.canDelete(cmp, m, snapshots) || files[m] {
2608 2 : continue
2609 : }
2610 2 : if files == nil {
2611 2 : // Construct files lazily, assuming most calls will not
2612 2 : // produce delete-only compactions.
2613 2 : files = make(map[*fileMetadata]bool)
2614 2 : }
2615 2 : files[m] = true
2616 2 : byLevel[l] = append(byLevel[l], m)
2617 : }
2618 : }
2619 : }
2620 :
2621 2 : var compactLevels []compactionLevel
2622 2 : for l, files := range byLevel {
2623 2 : if len(files) == 0 {
2624 2 : continue
2625 : }
2626 2 : compactLevels = append(compactLevels, compactionLevel{
2627 2 : level: l,
2628 2 : files: manifest.NewLevelSliceKeySorted(cmp, files),
2629 2 : })
2630 : }
2631 2 : return compactLevels, unresolvedHints
2632 : }
2633 :
2634 : // compact runs one compaction and maybe schedules another call to compact.
2635 2 : func (d *DB) compact(c *compaction, errChannel chan error) {
2636 2 : pprof.Do(context.Background(), compactLabels, func(context.Context) {
2637 2 : d.mu.Lock()
2638 2 : defer d.mu.Unlock()
2639 2 : if err := d.compact1(c, errChannel); err != nil {
2640 2 : // TODO(peter): count consecutive compaction errors and backoff.
2641 2 : d.opts.EventListener.BackgroundError(err)
2642 2 : }
2643 2 : d.mu.compact.compactingCount--
2644 2 : delete(d.mu.compact.inProgress, c)
2645 2 : // Add this compaction's duration to the cumulative duration. NB: This
2646 2 : // must be atomic with the above removal of c from
2647 2 : // d.mu.compact.InProgress to ensure Metrics.Compact.Duration does not
2648 2 : // miss or double count a completing compaction's duration.
2649 2 : d.mu.compact.duration += d.timeNow().Sub(c.beganAt)
2650 2 :
2651 2 : // The previous compaction may have produced too many files in a
2652 2 : // level, so reschedule another compaction if needed.
2653 2 : d.maybeScheduleCompaction()
2654 2 : d.mu.compact.cond.Broadcast()
2655 : })
2656 : }
2657 :
2658 : // compact1 runs one compaction.
2659 : //
2660 : // d.mu must be held when calling this, but the mutex may be dropped and
2661 : // re-acquired during the course of this method.
2662 2 : func (d *DB) compact1(c *compaction, errChannel chan error) (err error) {
2663 2 : if errChannel != nil {
2664 2 : defer func() {
2665 2 : errChannel <- err
2666 2 : }()
2667 : }
2668 :
2669 2 : jobID := d.mu.nextJobID
2670 2 : d.mu.nextJobID++
2671 2 : info := c.makeInfo(jobID)
2672 2 : d.opts.EventListener.CompactionBegin(info)
2673 2 : startTime := d.timeNow()
2674 2 :
2675 2 : ve, pendingOutputs, stats, err := d.runCompaction(jobID, c)
2676 2 :
2677 2 : info.Duration = d.timeNow().Sub(startTime)
2678 2 : if err == nil {
2679 2 : err = func() error {
2680 2 : var err error
2681 2 : d.mu.versions.logLock()
2682 2 : // Check if this compaction had a conflicting operation (eg. a d.excise())
2683 2 : // that necessitates it restarting from scratch. Note that since we hold
2684 2 : // the manifest lock, we don't expect this bool to change its value
2685 2 : // as only the holder of the manifest lock will ever write to it.
2686 2 : if c.cancel.Load() {
2687 2 : err = firstError(err, ErrCancelledCompaction)
2688 2 : }
2689 2 : if err != nil {
2690 2 : // logAndApply calls logUnlock. If we didn't call it, we need to call
2691 2 : // logUnlock ourselves.
2692 2 : d.mu.versions.logUnlock()
2693 2 : return err
2694 2 : }
2695 2 : return d.mu.versions.logAndApply(jobID, ve, c.metrics, false /* forceRotation */, func() []compactionInfo {
2696 2 : return d.getInProgressCompactionInfoLocked(c)
2697 2 : })
2698 : }()
2699 2 : if err != nil {
2700 2 : // TODO(peter): untested.
2701 2 : for _, f := range pendingOutputs {
2702 2 : // Note that the FileBacking for the file metadata might not have
2703 2 : // been set yet. So, we directly use the FileNum. Since these
2704 2 : // files were generated as compaction outputs, these must be
2705 2 : // physical files on disk. This property might not hold once
2706 2 : // https://github.com/cockroachdb/pebble/issues/389 is
2707 2 : // implemented if #389 creates virtual sstables as output files.
2708 2 : d.mu.versions.obsoleteTables = append(
2709 2 : d.mu.versions.obsoleteTables,
2710 2 : fileInfo{f.FileNum.DiskFileNum(), f.Size},
2711 2 : )
2712 2 : }
2713 2 : d.mu.versions.updateObsoleteTableMetricsLocked()
2714 : }
2715 : }
2716 :
2717 2 : info.Done = true
2718 2 : info.Err = err
2719 2 : if err == nil {
2720 2 : for i := range ve.NewFiles {
2721 2 : e := &ve.NewFiles[i]
2722 2 : info.Output.Tables = append(info.Output.Tables, e.Meta.TableInfo())
2723 2 : }
2724 2 : d.mu.snapshots.cumulativePinnedCount += stats.cumulativePinnedKeys
2725 2 : d.mu.snapshots.cumulativePinnedSize += stats.cumulativePinnedSize
2726 2 : d.mu.versions.metrics.Keys.MissizedTombstonesCount += stats.countMissizedDels
2727 2 : d.maybeUpdateDeleteCompactionHints(c)
2728 : }
2729 :
2730 : // NB: clearing compacting state must occur before updating the read state;
2731 : // L0Sublevels initialization depends on it.
2732 2 : d.clearCompactingState(c, err != nil)
2733 2 : d.mu.versions.incrementCompactions(c.kind, c.extraLevels, c.pickerMetrics)
2734 2 : d.mu.versions.incrementCompactionBytes(-c.bytesWritten)
2735 2 :
2736 2 : info.TotalDuration = d.timeNow().Sub(c.beganAt)
2737 2 : d.opts.EventListener.CompactionEnd(info)
2738 2 :
2739 2 : // Update the read state before deleting obsolete files because the
2740 2 : // read-state update will cause the previous version to be unref'd and if
2741 2 : // there are no references obsolete tables will be added to the obsolete
2742 2 : // table list.
2743 2 : if err == nil {
2744 2 : d.updateReadStateLocked(d.opts.DebugCheck)
2745 2 : d.updateTableStatsLocked(ve.NewFiles)
2746 2 : }
2747 2 : d.deleteObsoleteFiles(jobID)
2748 2 :
2749 2 : return err
2750 : }
2751 :
2752 : type compactStats struct {
2753 : cumulativePinnedKeys uint64
2754 : cumulativePinnedSize uint64
2755 : countMissizedDels uint64
2756 : }
2757 :
2758 : // runCopyCompaction runs a copy compaction where a new FileNum is created that
2759 : // is a byte-for-byte copy of the input file. This is used in lieu of a move
2760 : // compaction when a file is being moved across the local/remote storage
2761 : // boundary.
2762 : //
2763 : // d.mu must be held when calling this method.
2764 : func (d *DB) runCopyCompaction(
2765 : jobID int,
2766 : c *compaction,
2767 : meta *fileMetadata,
2768 : objMeta objstorage.ObjectMetadata,
2769 : versionEdit *versionEdit,
2770 2 : ) (ve *versionEdit, pendingOutputs []physicalMeta, retErr error) {
2771 2 : ve = versionEdit
2772 2 : if objMeta.IsRemote() || !remote.ShouldCreateShared(d.opts.Experimental.CreateOnShared, c.outputLevel.level) {
2773 0 : panic("pebble: scheduled a copy compaction that is not actually moving files to shared storage")
2774 : }
2775 : // Note that based on logic in the compaction picker, we're guaranteed
2776 : // meta.Virtual is false.
2777 2 : if meta.Virtual {
2778 0 : panic(errors.AssertionFailedf("cannot do a copy compaction of a virtual sstable across local/remote storage"))
2779 : }
2780 : // We are in the relatively more complex case where we need to copy this
2781 : // file to remote/shared storage. Drop the db mutex while we do the
2782 : // copy.
2783 : //
2784 : // To ease up cleanup of the local file and tracking of refs, we create
2785 : // a new FileNum. This has the potential of making the block cache less
2786 : // effective, however.
2787 2 : metaCopy := new(fileMetadata)
2788 2 : *metaCopy = fileMetadata{
2789 2 : Size: meta.Size,
2790 2 : CreationTime: meta.CreationTime,
2791 2 : SmallestSeqNum: meta.SmallestSeqNum,
2792 2 : LargestSeqNum: meta.LargestSeqNum,
2793 2 : Stats: meta.Stats,
2794 2 : Virtual: meta.Virtual,
2795 2 : }
2796 2 : if meta.HasPointKeys {
2797 2 : metaCopy.ExtendPointKeyBounds(c.cmp, meta.SmallestPointKey, meta.LargestPointKey)
2798 2 : }
2799 2 : if meta.HasRangeKeys {
2800 2 : metaCopy.ExtendRangeKeyBounds(c.cmp, meta.SmallestRangeKey, meta.LargestRangeKey)
2801 2 : }
2802 2 : metaCopy.FileNum = d.mu.versions.getNextFileNum()
2803 2 : metaCopy.InitPhysicalBacking()
2804 2 : c.metrics = map[int]*LevelMetrics{
2805 2 : c.outputLevel.level: {
2806 2 : BytesIn: meta.Size,
2807 2 : BytesCompacted: meta.Size,
2808 2 : TablesCompacted: 1,
2809 2 : },
2810 2 : }
2811 2 : pendingOutputs = append(pendingOutputs, metaCopy.PhysicalMeta())
2812 2 : // Before dropping the db mutex, grab a ref to the current version. This
2813 2 : // prevents any concurrent excises from deleting files that this compaction
2814 2 : // needs to read/maintain a reference to.
2815 2 : vers := d.mu.versions.currentVersion()
2816 2 : vers.Ref()
2817 2 : defer vers.UnrefLocked()
2818 2 :
2819 2 : d.mu.Unlock()
2820 2 : defer d.mu.Lock()
2821 2 : _, err := d.objProvider.LinkOrCopyFromLocal(context.TODO(), d.opts.FS,
2822 2 : d.objProvider.Path(objMeta), fileTypeTable, metaCopy.FileBacking.DiskFileNum,
2823 2 : objstorage.CreateOptions{PreferSharedStorage: true})
2824 2 : if err != nil {
2825 0 : return ve, pendingOutputs, err
2826 0 : }
2827 2 : ve.NewFiles[0].Meta = metaCopy
2828 2 :
2829 2 : if err := d.objProvider.Sync(); err != nil {
2830 0 : return nil, pendingOutputs, err
2831 0 : }
2832 2 : return ve, pendingOutputs, nil
2833 : }
2834 :
2835 : // runCompactions runs a compaction that produces new on-disk tables from
2836 : // memtables or old on-disk tables.
2837 : //
2838 : // d.mu must be held when calling this, but the mutex may be dropped and
2839 : // re-acquired during the course of this method.
2840 : func (d *DB) runCompaction(
2841 : jobID int, c *compaction,
2842 2 : ) (ve *versionEdit, pendingOutputs []physicalMeta, stats compactStats, retErr error) {
2843 2 : // As a sanity check, confirm that the smallest / largest keys for new and
2844 2 : // deleted files in the new versionEdit pass a validation function before
2845 2 : // returning the edit.
2846 2 : defer func() {
2847 2 : // If we're handling a panic, don't expect the version edit to validate.
2848 2 : if r := recover(); r != nil {
2849 0 : panic(r)
2850 2 : } else if ve != nil {
2851 2 : err := validateVersionEdit(ve, d.opts.Experimental.KeyValidationFunc, d.opts.Comparer.FormatKey)
2852 2 : if err != nil {
2853 0 : d.opts.Logger.Fatalf("pebble: version edit validation failed: %s", err)
2854 0 : }
2855 : }
2856 : }()
2857 :
2858 : // Check for a delete-only compaction. This can occur when wide range
2859 : // tombstones completely contain sstables.
2860 2 : if c.kind == compactionKindDeleteOnly {
2861 2 : c.metrics = make(map[int]*LevelMetrics, len(c.inputs))
2862 2 : ve := &versionEdit{
2863 2 : DeletedFiles: map[deletedFileEntry]*fileMetadata{},
2864 2 : }
2865 2 : for _, cl := range c.inputs {
2866 2 : levelMetrics := &LevelMetrics{}
2867 2 : iter := cl.files.Iter()
2868 2 : for f := iter.First(); f != nil; f = iter.Next() {
2869 2 : ve.DeletedFiles[deletedFileEntry{
2870 2 : Level: cl.level,
2871 2 : FileNum: f.FileNum,
2872 2 : }] = f
2873 2 : }
2874 2 : c.metrics[cl.level] = levelMetrics
2875 : }
2876 2 : return ve, nil, stats, nil
2877 : }
2878 :
2879 2 : if c.kind == compactionKindIngestedFlushable {
2880 0 : panic("pebble: runCompaction cannot handle compactionKindIngestedFlushable.")
2881 : }
2882 :
2883 : // Check for a move or copy of one table from one level to the next. We avoid
2884 : // such a move if there is lots of overlapping grandparent data. Otherwise,
2885 : // the move could create a parent file that will require a very expensive
2886 : // merge later on.
2887 2 : if c.kind == compactionKindMove || c.kind == compactionKindCopy {
2888 2 : iter := c.startLevel.files.Iter()
2889 2 : meta := iter.First()
2890 2 : if invariants.Enabled {
2891 2 : if iter.Next() != nil {
2892 0 : panic("got more than one file for a move or copy compaction")
2893 : }
2894 : }
2895 2 : if c.cancel.Load() {
2896 1 : return ve, nil, stats, ErrCancelledCompaction
2897 1 : }
2898 2 : objMeta, err := d.objProvider.Lookup(fileTypeTable, meta.FileBacking.DiskFileNum)
2899 2 : if err != nil {
2900 0 : return ve, pendingOutputs, stats, err
2901 0 : }
2902 2 : c.metrics = map[int]*LevelMetrics{
2903 2 : c.outputLevel.level: {
2904 2 : BytesMoved: meta.Size,
2905 2 : TablesMoved: 1,
2906 2 : },
2907 2 : }
2908 2 : ve := &versionEdit{
2909 2 : DeletedFiles: map[deletedFileEntry]*fileMetadata{
2910 2 : {Level: c.startLevel.level, FileNum: meta.FileNum}: meta,
2911 2 : },
2912 2 : NewFiles: []newFileEntry{
2913 2 : {Level: c.outputLevel.level, Meta: meta},
2914 2 : },
2915 2 : }
2916 2 : if c.kind == compactionKindCopy {
2917 2 : ve, pendingOutputs, retErr = d.runCopyCompaction(jobID, c, meta, objMeta, ve)
2918 2 : if retErr != nil {
2919 0 : return ve, pendingOutputs, stats, retErr
2920 0 : }
2921 : }
2922 2 : return ve, nil, stats, nil
2923 : }
2924 :
2925 2 : defer func() {
2926 2 : if retErr != nil {
2927 2 : pendingOutputs = nil
2928 2 : }
2929 : }()
2930 :
2931 2 : snapshots := d.mu.snapshots.toSlice()
2932 2 : formatVers := d.FormatMajorVersion()
2933 2 :
2934 2 : if c.flushing == nil {
2935 2 : // Before dropping the db mutex, grab a ref to the current version. This
2936 2 : // prevents any concurrent excises from deleting files that this compaction
2937 2 : // needs to read/maintain a reference to.
2938 2 : //
2939 2 : // Note that unlike user iterators, compactionIter does not maintain a ref
2940 2 : // of the version or read state.
2941 2 : vers := d.mu.versions.currentVersion()
2942 2 : vers.Ref()
2943 2 : defer vers.UnrefLocked()
2944 2 : }
2945 :
2946 2 : if c.cancel.Load() {
2947 1 : return ve, nil, stats, ErrCancelledCompaction
2948 1 : }
2949 :
2950 : // Release the d.mu lock while doing I/O.
2951 : // Note the unusual order: Unlock and then Lock.
2952 2 : d.mu.Unlock()
2953 2 : defer d.mu.Lock()
2954 2 :
2955 2 : // Compactions use a pool of buffers to read blocks, avoiding polluting the
2956 2 : // block cache with blocks that will not be read again. We initialize the
2957 2 : // buffer pool with a size 12. This initial size does not need to be
2958 2 : // accurate, because the pool will grow to accommodate the maximum number of
2959 2 : // blocks allocated at a given time over the course of the compaction. But
2960 2 : // choosing a size larger than that working set avoids any additional
2961 2 : // allocations to grow the size of the pool over the course of iteration.
2962 2 : //
2963 2 : // Justification for initial size 12: In a two-level compaction, at any
2964 2 : // given moment we'll have 2 index blocks in-use and 2 data blocks in-use.
2965 2 : // Additionally, when decoding a compressed block, we'll temporarily
2966 2 : // allocate 1 additional block to hold the compressed buffer. In the worst
2967 2 : // case that all input sstables have two-level index blocks (+2), value
2968 2 : // blocks (+2), range deletion blocks (+n) and range key blocks (+n), we'll
2969 2 : // additionally require 2n+4 blocks where n is the number of input sstables.
2970 2 : // Range deletion and range key blocks are relatively rare, and the cost of
2971 2 : // an additional allocation or two over the course of the compaction is
2972 2 : // considered to be okay. A larger initial size would cause the pool to hold
2973 2 : // on to more memory, even when it's not in-use because the pool will
2974 2 : // recycle buffers up to the current capacity of the pool. The memory use of
2975 2 : // a 12-buffer pool is expected to be within reason, even if all the buffers
2976 2 : // grow to the typical size of an index block (256 KiB) which would
2977 2 : // translate to 3 MiB per compaction.
2978 2 : c.bufferPool.Init(12)
2979 2 : defer c.bufferPool.Release()
2980 2 :
2981 2 : iiter, err := c.newInputIter(d.newIters, d.tableNewRangeKeyIter, snapshots)
2982 2 : if err != nil {
2983 0 : return nil, pendingOutputs, stats, err
2984 0 : }
2985 2 : c.allowedZeroSeqNum = c.allowZeroSeqNum()
2986 2 : iiter = invalidating.MaybeWrapIfInvariants(iiter)
2987 2 : iter := newCompactionIter(c.cmp, c.equal, c.formatKey, d.merge, iiter, snapshots,
2988 2 : &c.rangeDelFrag, &c.rangeKeyFrag, c.allowedZeroSeqNum, c.elideTombstone,
2989 2 : c.elideRangeTombstone, d.opts.Experimental.IneffectualSingleDeleteCallback,
2990 2 : d.opts.Experimental.SingleDeleteInvariantViolationCallback,
2991 2 : d.FormatMajorVersion())
2992 2 :
2993 2 : var (
2994 2 : createdFiles []base.DiskFileNum
2995 2 : tw *sstable.Writer
2996 2 : pinnedKeySize uint64
2997 2 : pinnedValueSize uint64
2998 2 : pinnedCount uint64
2999 2 : )
3000 2 : defer func() {
3001 2 : if iter != nil {
3002 2 : retErr = firstError(retErr, iter.Close())
3003 2 : }
3004 2 : if tw != nil {
3005 0 : retErr = firstError(retErr, tw.Close())
3006 0 : }
3007 2 : if retErr != nil {
3008 2 : for _, fileNum := range createdFiles {
3009 2 : _ = d.objProvider.Remove(fileTypeTable, fileNum)
3010 2 : }
3011 : }
3012 2 : for _, closer := range c.closers {
3013 2 : retErr = firstError(retErr, closer.Close())
3014 2 : }
3015 : }()
3016 :
3017 2 : ve = &versionEdit{
3018 2 : DeletedFiles: map[deletedFileEntry]*fileMetadata{},
3019 2 : }
3020 2 :
3021 2 : startLevelBytes := c.startLevel.files.SizeSum()
3022 2 : outputMetrics := &LevelMetrics{
3023 2 : BytesIn: startLevelBytes,
3024 2 : BytesRead: c.outputLevel.files.SizeSum(),
3025 2 : }
3026 2 : if len(c.extraLevels) > 0 {
3027 2 : outputMetrics.BytesIn += c.extraLevels[0].files.SizeSum()
3028 2 : }
3029 2 : outputMetrics.BytesRead += outputMetrics.BytesIn
3030 2 :
3031 2 : c.metrics = map[int]*LevelMetrics{
3032 2 : c.outputLevel.level: outputMetrics,
3033 2 : }
3034 2 : if len(c.flushing) == 0 && c.metrics[c.startLevel.level] == nil {
3035 2 : c.metrics[c.startLevel.level] = &LevelMetrics{}
3036 2 : }
3037 2 : if len(c.extraLevels) > 0 {
3038 2 : c.metrics[c.extraLevels[0].level] = &LevelMetrics{}
3039 2 : outputMetrics.MultiLevel.BytesInTop = startLevelBytes
3040 2 : outputMetrics.MultiLevel.BytesIn = outputMetrics.BytesIn
3041 2 : outputMetrics.MultiLevel.BytesRead = outputMetrics.BytesRead
3042 2 : }
3043 :
3044 : // The table is typically written at the maximum allowable format implied by
3045 : // the current format major version of the DB.
3046 2 : tableFormat := formatVers.MaxTableFormat()
3047 2 :
3048 2 : // In format major versions with maximum table formats of Pebblev3, value
3049 2 : // blocks were conditional on an experimental setting. In format major
3050 2 : // versions with maximum table formats of Pebblev4 and higher, value blocks
3051 2 : // are always enabled.
3052 2 : if tableFormat == sstable.TableFormatPebblev3 &&
3053 2 : (d.opts.Experimental.EnableValueBlocks == nil || !d.opts.Experimental.EnableValueBlocks()) {
3054 2 : tableFormat = sstable.TableFormatPebblev2
3055 2 : }
3056 :
3057 2 : writerOpts := d.opts.MakeWriterOptions(c.outputLevel.level, tableFormat)
3058 2 :
3059 2 : // prevPointKey is a sstable.WriterOption that provides access to
3060 2 : // the last point key written to a writer's sstable. When a new
3061 2 : // output begins in newOutput, prevPointKey is updated to point to
3062 2 : // the new output's sstable.Writer. This allows the compaction loop
3063 2 : // to access the last written point key without requiring the
3064 2 : // compaction loop to make a copy of each key ahead of time. Users
3065 2 : // must be careful, because the byte slice returned by UnsafeKey
3066 2 : // points directly into the Writer's block buffer.
3067 2 : var prevPointKey sstable.PreviousPointKeyOpt
3068 2 : var cpuWorkHandle CPUWorkHandle
3069 2 : defer func() {
3070 2 : if cpuWorkHandle != nil {
3071 1 : d.opts.Experimental.CPUWorkPermissionGranter.CPUWorkDone(cpuWorkHandle)
3072 1 : }
3073 : }()
3074 :
3075 2 : newOutput := func() error {
3076 2 : // Check if we've been cancelled by a concurrent operation.
3077 2 : if c.cancel.Load() {
3078 1 : return ErrCancelledCompaction
3079 1 : }
3080 2 : fileMeta := &fileMetadata{}
3081 2 : d.mu.Lock()
3082 2 : fileNum := d.mu.versions.getNextFileNum()
3083 2 : fileMeta.FileNum = fileNum
3084 2 : pendingOutputs = append(pendingOutputs, fileMeta.PhysicalMeta())
3085 2 : d.mu.Unlock()
3086 2 :
3087 2 : ctx := context.TODO()
3088 2 : if objiotracing.Enabled {
3089 0 : ctx = objiotracing.WithLevel(ctx, c.outputLevel.level)
3090 0 : switch c.kind {
3091 0 : case compactionKindFlush:
3092 0 : ctx = objiotracing.WithReason(ctx, objiotracing.ForFlush)
3093 0 : case compactionKindIngestedFlushable:
3094 0 : ctx = objiotracing.WithReason(ctx, objiotracing.ForIngestion)
3095 0 : default:
3096 0 : ctx = objiotracing.WithReason(ctx, objiotracing.ForCompaction)
3097 : }
3098 : }
3099 : // Prefer shared storage if present.
3100 2 : createOpts := objstorage.CreateOptions{
3101 2 : PreferSharedStorage: remote.ShouldCreateShared(d.opts.Experimental.CreateOnShared, c.outputLevel.level),
3102 2 : }
3103 2 : writable, objMeta, err := d.objProvider.Create(ctx, fileTypeTable, fileNum.DiskFileNum(), createOpts)
3104 2 : if err != nil {
3105 1 : return err
3106 1 : }
3107 :
3108 2 : reason := "flushing"
3109 2 : if c.flushing == nil {
3110 2 : reason = "compacting"
3111 2 : }
3112 2 : d.opts.EventListener.TableCreated(TableCreateInfo{
3113 2 : JobID: jobID,
3114 2 : Reason: reason,
3115 2 : Path: d.objProvider.Path(objMeta),
3116 2 : FileNum: fileNum,
3117 2 : })
3118 2 : if c.kind != compactionKindFlush {
3119 2 : writable = &compactionWritable{
3120 2 : Writable: writable,
3121 2 : versions: d.mu.versions,
3122 2 : written: &c.bytesWritten,
3123 2 : }
3124 2 : }
3125 2 : createdFiles = append(createdFiles, fileNum.DiskFileNum())
3126 2 : cacheOpts := private.SSTableCacheOpts(d.cacheID, fileNum.DiskFileNum()).(sstable.WriterOption)
3127 2 :
3128 2 : const MaxFileWriteAdditionalCPUTime = time.Millisecond * 100
3129 2 : cpuWorkHandle = d.opts.Experimental.CPUWorkPermissionGranter.GetPermission(
3130 2 : MaxFileWriteAdditionalCPUTime,
3131 2 : )
3132 2 : writerOpts.Parallelism =
3133 2 : d.opts.Experimental.MaxWriterConcurrency > 0 &&
3134 2 : (cpuWorkHandle.Permitted() || d.opts.Experimental.ForceWriterParallelism)
3135 2 :
3136 2 : tw = sstable.NewWriter(writable, writerOpts, cacheOpts, &prevPointKey)
3137 2 :
3138 2 : fileMeta.CreationTime = time.Now().Unix()
3139 2 : ve.NewFiles = append(ve.NewFiles, newFileEntry{
3140 2 : Level: c.outputLevel.level,
3141 2 : Meta: fileMeta,
3142 2 : })
3143 2 : return nil
3144 : }
3145 :
3146 : // splitL0Outputs is true during flushes and intra-L0 compactions with flush
3147 : // splits enabled.
3148 2 : splitL0Outputs := c.outputLevel.level == 0 && d.opts.FlushSplitBytes > 0
3149 2 :
3150 2 : // finishOutput is called with the a user key up to which all tombstones
3151 2 : // should be flushed. Typically, this is the first key of the next
3152 2 : // sstable or an empty key if this output is the final sstable.
3153 2 : finishOutput := func(splitKey []byte) error {
3154 2 : // If we haven't output any point records to the sstable (tw == nil) then the
3155 2 : // sstable will only contain range tombstones and/or range keys. The smallest
3156 2 : // key in the sstable will be the start key of the first range tombstone or
3157 2 : // range key added. We need to ensure that this start key is distinct from
3158 2 : // the splitKey passed to finishOutput (if set), otherwise we would generate
3159 2 : // an sstable where the largest key is smaller than the smallest key due to
3160 2 : // how the largest key boundary is set below. NB: It is permissible for the
3161 2 : // range tombstone / range key start key to be the empty string.
3162 2 : //
3163 2 : // TODO: It is unfortunate that we have to do this check here rather than
3164 2 : // when we decide to finish the sstable in the runCompaction loop. A better
3165 2 : // structure currently eludes us.
3166 2 : if tw == nil {
3167 2 : startKey := c.rangeDelFrag.Start()
3168 2 : if len(iter.tombstones) > 0 {
3169 2 : startKey = iter.tombstones[0].Start
3170 2 : }
3171 2 : if startKey == nil {
3172 2 : startKey = c.rangeKeyFrag.Start()
3173 2 : if len(iter.rangeKeys) > 0 {
3174 2 : startKey = iter.rangeKeys[0].Start
3175 2 : }
3176 : }
3177 2 : if splitKey != nil && d.cmp(startKey, splitKey) == 0 {
3178 0 : return nil
3179 0 : }
3180 : }
3181 :
3182 : // NB: clone the key because the data can be held on to by the call to
3183 : // compactionIter.Tombstones via keyspan.Fragmenter.FlushTo, and by the
3184 : // WriterMetadata.LargestRangeDel.UserKey.
3185 2 : splitKey = append([]byte(nil), splitKey...)
3186 2 : for _, v := range iter.Tombstones(splitKey) {
3187 2 : if tw == nil {
3188 2 : if err := newOutput(); err != nil {
3189 1 : return err
3190 1 : }
3191 : }
3192 : // The tombstone being added could be completely outside the
3193 : // eventual bounds of the sstable. Consider this example (bounds
3194 : // in square brackets next to table filename):
3195 : //
3196 : // ./000240.sst [tmgc#391,MERGE-tmgc#391,MERGE]
3197 : // tmgc#391,MERGE [786e627a]
3198 : // tmgc-udkatvs#331,RANGEDEL
3199 : //
3200 : // ./000241.sst [tmgc#384,MERGE-tmgc#384,MERGE]
3201 : // tmgc#384,MERGE [666c7070]
3202 : // tmgc-tvsalezade#383,RANGEDEL
3203 : // tmgc-tvsalezade#331,RANGEDEL
3204 : //
3205 : // ./000242.sst [tmgc#383,RANGEDEL-tvsalezade#72057594037927935,RANGEDEL]
3206 : // tmgc-tvsalezade#383,RANGEDEL
3207 : // tmgc#375,SET [72646c78766965616c72776865676e79]
3208 : // tmgc-tvsalezade#356,RANGEDEL
3209 : //
3210 : // Note that both of the top two SSTables have range tombstones
3211 : // that start after the file's end keys. Since the file bound
3212 : // computation happens well after all range tombstones have been
3213 : // added to the writer, eliding out-of-file range tombstones based
3214 : // on sequence number at this stage is difficult, and necessitates
3215 : // read-time logic to ignore range tombstones outside file bounds.
3216 2 : if err := rangedel.Encode(&v, tw.Add); err != nil {
3217 0 : return err
3218 0 : }
3219 : }
3220 2 : for _, v := range iter.RangeKeys(splitKey) {
3221 2 : // Same logic as for range tombstones, except added using tw.AddRangeKey.
3222 2 : if tw == nil {
3223 2 : if err := newOutput(); err != nil {
3224 1 : return err
3225 1 : }
3226 : }
3227 2 : if err := rangekey.Encode(&v, tw.AddRangeKey); err != nil {
3228 0 : return err
3229 0 : }
3230 : }
3231 :
3232 2 : if tw == nil {
3233 2 : return nil
3234 2 : }
3235 2 : {
3236 2 : // Set internal sstable properties.
3237 2 : p := getInternalWriterProperties(tw)
3238 2 : // Set the external sst version to 0. This is what RocksDB expects for
3239 2 : // db-internal sstables; otherwise, it could apply a global sequence number.
3240 2 : p.ExternalFormatVersion = 0
3241 2 : // Set the snapshot pinned totals.
3242 2 : p.SnapshotPinnedKeys = pinnedCount
3243 2 : p.SnapshotPinnedKeySize = pinnedKeySize
3244 2 : p.SnapshotPinnedValueSize = pinnedValueSize
3245 2 : stats.cumulativePinnedKeys += pinnedCount
3246 2 : stats.cumulativePinnedSize += pinnedKeySize + pinnedValueSize
3247 2 : pinnedCount = 0
3248 2 : pinnedKeySize = 0
3249 2 : pinnedValueSize = 0
3250 2 : }
3251 2 : if err := tw.Close(); err != nil {
3252 1 : tw = nil
3253 1 : return err
3254 1 : }
3255 2 : d.opts.Experimental.CPUWorkPermissionGranter.CPUWorkDone(cpuWorkHandle)
3256 2 : cpuWorkHandle = nil
3257 2 : writerMeta, err := tw.Metadata()
3258 2 : if err != nil {
3259 0 : tw = nil
3260 0 : return err
3261 0 : }
3262 2 : tw = nil
3263 2 : meta := ve.NewFiles[len(ve.NewFiles)-1].Meta
3264 2 : meta.Size = writerMeta.Size
3265 2 : meta.SmallestSeqNum = writerMeta.SmallestSeqNum
3266 2 : meta.LargestSeqNum = writerMeta.LargestSeqNum
3267 2 : meta.InitPhysicalBacking()
3268 2 :
3269 2 : // If the file didn't contain any range deletions, we can fill its
3270 2 : // table stats now, avoiding unnecessarily loading the table later.
3271 2 : maybeSetStatsFromProperties(
3272 2 : meta.PhysicalMeta(), &writerMeta.Properties,
3273 2 : )
3274 2 :
3275 2 : if c.flushing == nil {
3276 2 : outputMetrics.TablesCompacted++
3277 2 : outputMetrics.BytesCompacted += meta.Size
3278 2 : } else {
3279 2 : outputMetrics.TablesFlushed++
3280 2 : outputMetrics.BytesFlushed += meta.Size
3281 2 : }
3282 2 : outputMetrics.Size += int64(meta.Size)
3283 2 : outputMetrics.NumFiles++
3284 2 : outputMetrics.Additional.BytesWrittenDataBlocks += writerMeta.Properties.DataSize
3285 2 : outputMetrics.Additional.BytesWrittenValueBlocks += writerMeta.Properties.ValueBlocksSize
3286 2 :
3287 2 : if n := len(ve.NewFiles); n > 1 {
3288 2 : // This is not the first output file. Ensure the sstable boundaries
3289 2 : // are nonoverlapping.
3290 2 : prevMeta := ve.NewFiles[n-2].Meta
3291 2 : if writerMeta.SmallestRangeDel.UserKey != nil {
3292 2 : c := d.cmp(writerMeta.SmallestRangeDel.UserKey, prevMeta.Largest.UserKey)
3293 2 : if c < 0 {
3294 0 : return errors.Errorf(
3295 0 : "pebble: smallest range tombstone start key is less than previous sstable largest key: %s < %s",
3296 0 : writerMeta.SmallestRangeDel.Pretty(d.opts.Comparer.FormatKey),
3297 0 : prevMeta.Largest.Pretty(d.opts.Comparer.FormatKey))
3298 2 : } else if c == 0 && !prevMeta.Largest.IsExclusiveSentinel() {
3299 0 : // The user key portion of the range boundary start key is
3300 0 : // equal to the previous table's largest key user key, and
3301 0 : // the previous table's largest key is not exclusive. This
3302 0 : // violates the invariant that tables are key-space
3303 0 : // partitioned.
3304 0 : return errors.Errorf(
3305 0 : "pebble: invariant violation: previous sstable largest key %s, current sstable smallest rangedel: %s",
3306 0 : prevMeta.Largest.Pretty(d.opts.Comparer.FormatKey),
3307 0 : writerMeta.SmallestRangeDel.Pretty(d.opts.Comparer.FormatKey),
3308 0 : )
3309 0 : }
3310 : }
3311 : }
3312 :
3313 : // Verify that all range deletions outputted to the sstable are
3314 : // truncated to split key.
3315 2 : if splitKey != nil && writerMeta.LargestRangeDel.UserKey != nil &&
3316 2 : d.cmp(writerMeta.LargestRangeDel.UserKey, splitKey) > 0 {
3317 0 : return errors.Errorf(
3318 0 : "pebble: invariant violation: rangedel largest key %q extends beyond split key %q",
3319 0 : writerMeta.LargestRangeDel.Pretty(d.opts.Comparer.FormatKey),
3320 0 : d.opts.Comparer.FormatKey(splitKey),
3321 0 : )
3322 0 : }
3323 :
3324 2 : if writerMeta.HasPointKeys {
3325 2 : meta.ExtendPointKeyBounds(d.cmp, writerMeta.SmallestPoint, writerMeta.LargestPoint)
3326 2 : }
3327 2 : if writerMeta.HasRangeDelKeys {
3328 2 : meta.ExtendPointKeyBounds(d.cmp, writerMeta.SmallestRangeDel, writerMeta.LargestRangeDel)
3329 2 : }
3330 2 : if writerMeta.HasRangeKeys {
3331 2 : meta.ExtendRangeKeyBounds(d.cmp, writerMeta.SmallestRangeKey, writerMeta.LargestRangeKey)
3332 2 : }
3333 :
3334 : // Verify that the sstable bounds fall within the compaction input
3335 : // bounds. This is a sanity check that we don't have a logic error
3336 : // elsewhere that causes the sstable bounds to accidentally expand past the
3337 : // compaction input bounds as doing so could lead to various badness such
3338 : // as keys being deleted by a range tombstone incorrectly.
3339 2 : if c.smallest.UserKey != nil {
3340 2 : switch v := d.cmp(meta.Smallest.UserKey, c.smallest.UserKey); {
3341 2 : case v >= 0:
3342 : // Nothing to do.
3343 0 : case v < 0:
3344 0 : return errors.Errorf("pebble: compaction output grew beyond bounds of input: %s < %s",
3345 0 : meta.Smallest.Pretty(d.opts.Comparer.FormatKey),
3346 0 : c.smallest.Pretty(d.opts.Comparer.FormatKey))
3347 : }
3348 : }
3349 2 : if c.largest.UserKey != nil {
3350 2 : switch v := d.cmp(meta.Largest.UserKey, c.largest.UserKey); {
3351 2 : case v <= 0:
3352 : // Nothing to do.
3353 0 : case v > 0:
3354 0 : return errors.Errorf("pebble: compaction output grew beyond bounds of input: %s > %s",
3355 0 : meta.Largest.Pretty(d.opts.Comparer.FormatKey),
3356 0 : c.largest.Pretty(d.opts.Comparer.FormatKey))
3357 : }
3358 : }
3359 : // Verify that we never split different revisions of the same user key
3360 : // across two different sstables.
3361 2 : if err := c.errorOnUserKeyOverlap(ve); err != nil {
3362 0 : return err
3363 0 : }
3364 2 : if err := meta.Validate(d.cmp, d.opts.Comparer.FormatKey); err != nil {
3365 0 : return err
3366 0 : }
3367 2 : return nil
3368 : }
3369 :
3370 : // Build a compactionOutputSplitter that contains all logic to determine
3371 : // whether the compaction loop should stop writing to one output sstable and
3372 : // switch to a new one. Some splitters can wrap other splitters, and the
3373 : // splitterGroup can be composed of multiple splitters. In this case, we
3374 : // start off with splitters for file sizes, grandparent limits, and (for L0
3375 : // splits) L0 limits, before wrapping them in an splitterGroup.
3376 2 : sizeSplitter := newFileSizeSplitter(&iter.frontiers, c.maxOutputFileSize, c.grandparents.Iter())
3377 2 : unsafePrevUserKey := func() []byte {
3378 2 : // Return the largest point key written to tw or the start of
3379 2 : // the current range deletion in the fragmenter, whichever is
3380 2 : // greater.
3381 2 : prevPoint := prevPointKey.UnsafeKey()
3382 2 : if c.cmp(prevPoint.UserKey, c.rangeDelFrag.Start()) > 0 {
3383 2 : return prevPoint.UserKey
3384 2 : }
3385 2 : return c.rangeDelFrag.Start()
3386 : }
3387 2 : outputSplitters := []compactionOutputSplitter{
3388 2 : // We do not split the same user key across different sstables within
3389 2 : // one flush or compaction. The fileSizeSplitter may request a split in
3390 2 : // the middle of a user key, so the userKeyChangeSplitter ensures we are
3391 2 : // at a user key change boundary when doing a split.
3392 2 : &userKeyChangeSplitter{
3393 2 : cmp: c.cmp,
3394 2 : splitter: sizeSplitter,
3395 2 : unsafePrevUserKey: unsafePrevUserKey,
3396 2 : },
3397 2 : newLimitFuncSplitter(&iter.frontiers, c.findGrandparentLimit),
3398 2 : }
3399 2 : if splitL0Outputs {
3400 2 : outputSplitters = append(outputSplitters, newLimitFuncSplitter(&iter.frontiers, c.findL0Limit))
3401 2 : }
3402 2 : splitter := &splitterGroup{cmp: c.cmp, splitters: outputSplitters}
3403 2 :
3404 2 : // Each outer loop iteration produces one output file. An iteration that
3405 2 : // produces a file containing point keys (and optionally range tombstones)
3406 2 : // guarantees that the input iterator advanced. An iteration that produces
3407 2 : // a file containing only range tombstones guarantees the limit passed to
3408 2 : // `finishOutput()` advanced to a strictly greater user key corresponding
3409 2 : // to a grandparent file largest key, or nil. Taken together, these
3410 2 : // progress guarantees ensure that eventually the input iterator will be
3411 2 : // exhausted and the range tombstone fragments will all be flushed.
3412 2 : for key, val := iter.First(); key != nil || !c.rangeDelFrag.Empty() || !c.rangeKeyFrag.Empty(); {
3413 2 : var firstKey []byte
3414 2 : if key != nil {
3415 2 : firstKey = key.UserKey
3416 2 : } else if startKey := c.rangeDelFrag.Start(); startKey != nil {
3417 2 : // Pass the start key of the first pending tombstone to find the
3418 2 : // next limit. All pending tombstones have the same start key. We
3419 2 : // use this as opposed to the end key of the last written sstable to
3420 2 : // effectively handle cases like these:
3421 2 : //
3422 2 : // a.SET.3
3423 2 : // (lf.limit at b)
3424 2 : // d.RANGEDEL.4:f
3425 2 : //
3426 2 : // In this case, the partition after b has only range deletions, so
3427 2 : // if we were to find the limit after the last written key at the
3428 2 : // split point (key a), we'd get the limit b again, and
3429 2 : // finishOutput() would not advance any further because the next
3430 2 : // range tombstone to write does not start until after the L0 split
3431 2 : // point.
3432 2 : firstKey = startKey
3433 2 : }
3434 2 : splitterSuggestion := splitter.onNewOutput(firstKey)
3435 2 :
3436 2 : // Each inner loop iteration processes one key from the input iterator.
3437 2 : for ; key != nil; key, val = iter.Next() {
3438 2 : if split := splitter.shouldSplitBefore(key, tw); split == splitNow {
3439 2 : break
3440 : }
3441 :
3442 2 : switch key.Kind() {
3443 2 : case InternalKeyKindRangeDelete:
3444 2 : // Range tombstones are handled specially. They are fragmented,
3445 2 : // and they're not written until later during `finishOutput()`.
3446 2 : // We add them to the `Fragmenter` now to make them visible to
3447 2 : // `compactionIter` so covered keys in the same snapshot stripe
3448 2 : // can be elided.
3449 2 :
3450 2 : // The interleaved range deletion might only be one of many with
3451 2 : // these bounds. Some fragmenting is performed ahead of time by
3452 2 : // keyspan.MergingIter.
3453 2 : if s := c.rangeDelIter.Span(); !s.Empty() {
3454 2 : // The memory management here is subtle. Range deletions
3455 2 : // blocks do NOT use prefix compression, which ensures that
3456 2 : // range deletion spans' memory is available as long we keep
3457 2 : // the iterator open. However, the keyspan.MergingIter that
3458 2 : // merges spans across levels only guarantees the lifetime
3459 2 : // of the [start, end) bounds until the next positioning
3460 2 : // method is called.
3461 2 : //
3462 2 : // Additionally, the Span.Keys slice is owned by the the
3463 2 : // range deletion iterator stack, and it may be overwritten
3464 2 : // when we advance.
3465 2 : //
3466 2 : // Clone the Keys slice and the start and end keys.
3467 2 : //
3468 2 : // TODO(jackson): Avoid the clone by removing c.rangeDelFrag
3469 2 : // and performing explicit truncation of the pending
3470 2 : // rangedel span as necessary.
3471 2 : clone := keyspan.Span{
3472 2 : Start: iter.cloneKey(s.Start),
3473 2 : End: iter.cloneKey(s.End),
3474 2 : Keys: make([]keyspan.Key, len(s.Keys)),
3475 2 : }
3476 2 : copy(clone.Keys, s.Keys)
3477 2 : c.rangeDelFrag.Add(clone)
3478 2 : }
3479 2 : continue
3480 2 : case InternalKeyKindRangeKeySet, InternalKeyKindRangeKeyUnset, InternalKeyKindRangeKeyDelete:
3481 2 : // Range keys are handled in the same way as range tombstones, except
3482 2 : // with a dedicated fragmenter.
3483 2 : if s := c.rangeKeyInterleaving.Span(); !s.Empty() {
3484 2 : clone := keyspan.Span{
3485 2 : Start: iter.cloneKey(s.Start),
3486 2 : End: iter.cloneKey(s.End),
3487 2 : Keys: make([]keyspan.Key, len(s.Keys)),
3488 2 : }
3489 2 : // Since the keys' Suffix and Value fields are not deep cloned, the
3490 2 : // underlying blockIter must be kept open for the lifetime of the
3491 2 : // compaction.
3492 2 : copy(clone.Keys, s.Keys)
3493 2 : c.rangeKeyFrag.Add(clone)
3494 2 : }
3495 2 : continue
3496 : }
3497 2 : if tw == nil {
3498 2 : if err := newOutput(); err != nil {
3499 2 : return nil, pendingOutputs, stats, err
3500 2 : }
3501 : }
3502 2 : if err := tw.AddWithForceObsolete(*key, val, iter.forceObsoleteDueToRangeDel); err != nil {
3503 0 : return nil, pendingOutputs, stats, err
3504 0 : }
3505 2 : if iter.snapshotPinned {
3506 2 : // The kv pair we just added to the sstable was only surfaced by
3507 2 : // the compaction iterator because an open snapshot prevented
3508 2 : // its elision. Increment the stats.
3509 2 : pinnedCount++
3510 2 : pinnedKeySize += uint64(len(key.UserKey)) + base.InternalTrailerLen
3511 2 : pinnedValueSize += uint64(len(val))
3512 2 : }
3513 : }
3514 :
3515 : // A splitter requested a split, and we're ready to finish the output.
3516 : // We need to choose the key at which to split any pending range
3517 : // tombstones. There are two options:
3518 : // 1. splitterSuggestion — The key suggested by the splitter. This key
3519 : // is guaranteed to be greater than the last key written to the
3520 : // current output.
3521 : // 2. key.UserKey — the first key of the next sstable output. This user
3522 : // key is also guaranteed to be greater than the last user key
3523 : // written to the current output (see userKeyChangeSplitter).
3524 : //
3525 : // Use whichever is smaller. Using the smaller of the two limits
3526 : // overlap with grandparents. Consider the case where the
3527 : // grandparent limit is calculated to be 'b', key is 'x', and
3528 : // there exist many sstables between 'b' and 'x'. If the range
3529 : // deletion fragmenter has a pending tombstone [a,x), splitting
3530 : // at 'x' would cause the output table to overlap many
3531 : // grandparents well beyond the calculated grandparent limit
3532 : // 'b'. Splitting at the smaller `splitterSuggestion` avoids
3533 : // this unbounded overlap with grandparent tables.
3534 2 : splitKey := splitterSuggestion
3535 2 : if key != nil && (splitKey == nil || c.cmp(splitKey, key.UserKey) > 0) {
3536 2 : splitKey = key.UserKey
3537 2 : }
3538 2 : if err := finishOutput(splitKey); err != nil {
3539 2 : return nil, pendingOutputs, stats, err
3540 2 : }
3541 : }
3542 :
3543 2 : for _, cl := range c.inputs {
3544 2 : iter := cl.files.Iter()
3545 2 : for f := iter.First(); f != nil; f = iter.Next() {
3546 2 : ve.DeletedFiles[deletedFileEntry{
3547 2 : Level: cl.level,
3548 2 : FileNum: f.FileNum,
3549 2 : }] = f
3550 2 : }
3551 : }
3552 :
3553 : // The compaction iterator keeps track of a count of the number of DELSIZED
3554 : // keys that encoded an incorrect size. Propagate it up as a part of
3555 : // compactStats.
3556 2 : stats.countMissizedDels = iter.stats.countMissizedDels
3557 2 :
3558 2 : if err := d.objProvider.Sync(); err != nil {
3559 0 : return nil, pendingOutputs, stats, err
3560 0 : }
3561 :
3562 : // Refresh the disk available statistic whenever a compaction/flush
3563 : // completes, before re-acquiring the mutex.
3564 2 : _ = d.calculateDiskAvailableBytes()
3565 2 :
3566 2 : return ve, pendingOutputs, stats, nil
3567 : }
3568 :
3569 : // validateVersionEdit validates that start and end keys across new and deleted
3570 : // files in a versionEdit pass the given validation function.
3571 : func validateVersionEdit(
3572 : ve *versionEdit, validateFn func([]byte) error, format base.FormatKey,
3573 2 : ) error {
3574 2 : validateMetaFn := func(f *manifest.FileMetadata) error {
3575 2 : for _, key := range []InternalKey{f.Smallest, f.Largest} {
3576 2 : if err := validateFn(key.UserKey); err != nil {
3577 1 : return errors.Wrapf(err, "key=%q; file=%s", format(key.UserKey), f)
3578 1 : }
3579 : }
3580 2 : return nil
3581 : }
3582 :
3583 : // Validate both new and deleted files.
3584 2 : for _, f := range ve.NewFiles {
3585 2 : if err := validateMetaFn(f.Meta); err != nil {
3586 1 : return err
3587 1 : }
3588 : }
3589 2 : for _, m := range ve.DeletedFiles {
3590 2 : if err := validateMetaFn(m); err != nil {
3591 1 : return err
3592 1 : }
3593 : }
3594 :
3595 2 : return nil
3596 : }
3597 :
3598 : // scanObsoleteFiles scans the filesystem for files that are no longer needed
3599 : // and adds those to the internal lists of obsolete files. Note that the files
3600 : // are not actually deleted by this method. A subsequent call to
3601 : // deleteObsoleteFiles must be performed. Must be not be called concurrently
3602 : // with compactions and flushes. db.mu must be held when calling this function.
3603 2 : func (d *DB) scanObsoleteFiles(list []string) {
3604 2 : // Disable automatic compactions temporarily to avoid concurrent compactions /
3605 2 : // flushes from interfering. The original value is restored on completion.
3606 2 : disabledPrev := d.opts.DisableAutomaticCompactions
3607 2 : defer func() {
3608 2 : d.opts.DisableAutomaticCompactions = disabledPrev
3609 2 : }()
3610 2 : d.opts.DisableAutomaticCompactions = true
3611 2 :
3612 2 : // Wait for any ongoing compaction to complete before continuing.
3613 2 : for d.mu.compact.compactingCount > 0 || d.mu.compact.flushing {
3614 0 : d.mu.compact.cond.Wait()
3615 0 : }
3616 :
3617 2 : liveFileNums := make(map[base.DiskFileNum]struct{})
3618 2 : d.mu.versions.addLiveFileNums(liveFileNums)
3619 2 : // Protect against files which are only referred to by the ingestedFlushable
3620 2 : // from being deleted. These are added to the flushable queue on WAL replay
3621 2 : // during read only mode and aren't part of the Version. Note that if
3622 2 : // !d.opts.ReadOnly, then all flushables of type ingestedFlushable have
3623 2 : // already been flushed.
3624 2 : for _, fEntry := range d.mu.mem.queue {
3625 2 : if f, ok := fEntry.flushable.(*ingestedFlushable); ok {
3626 0 : for _, file := range f.files {
3627 0 : liveFileNums[file.FileBacking.DiskFileNum] = struct{}{}
3628 0 : }
3629 : }
3630 : }
3631 :
3632 2 : minUnflushedLogNum := d.mu.versions.minUnflushedLogNum
3633 2 : manifestFileNum := d.mu.versions.manifestFileNum
3634 2 :
3635 2 : var obsoleteLogs []fileInfo
3636 2 : var obsoleteTables []fileInfo
3637 2 : var obsoleteManifests []fileInfo
3638 2 : var obsoleteOptions []fileInfo
3639 2 :
3640 2 : for _, filename := range list {
3641 2 : fileType, diskFileNum, ok := base.ParseFilename(d.opts.FS, filename)
3642 2 : if !ok {
3643 2 : continue
3644 : }
3645 2 : switch fileType {
3646 2 : case fileTypeLog:
3647 2 : if diskFileNum >= minUnflushedLogNum {
3648 0 : continue
3649 : }
3650 2 : fi := fileInfo{fileNum: diskFileNum}
3651 2 : if stat, err := d.opts.FS.Stat(filename); err == nil {
3652 1 : fi.fileSize = uint64(stat.Size())
3653 1 : }
3654 2 : obsoleteLogs = append(obsoleteLogs, fi)
3655 2 : case fileTypeManifest:
3656 2 : if diskFileNum >= manifestFileNum {
3657 2 : continue
3658 : }
3659 2 : fi := fileInfo{fileNum: diskFileNum}
3660 2 : if stat, err := d.opts.FS.Stat(filename); err == nil {
3661 1 : fi.fileSize = uint64(stat.Size())
3662 1 : }
3663 2 : obsoleteManifests = append(obsoleteManifests, fi)
3664 2 : case fileTypeOptions:
3665 2 : if diskFileNum.FileNum() >= d.optionsFileNum.FileNum() {
3666 0 : continue
3667 : }
3668 2 : fi := fileInfo{fileNum: diskFileNum}
3669 2 : if stat, err := d.opts.FS.Stat(filename); err == nil {
3670 1 : fi.fileSize = uint64(stat.Size())
3671 1 : }
3672 2 : obsoleteOptions = append(obsoleteOptions, fi)
3673 2 : case fileTypeTable:
3674 : // Objects are handled through the objstorage provider below.
3675 2 : default:
3676 : // Don't delete files we don't know about.
3677 : }
3678 : }
3679 :
3680 2 : objects := d.objProvider.List()
3681 2 : for _, obj := range objects {
3682 2 : switch obj.FileType {
3683 2 : case fileTypeTable:
3684 2 : if _, ok := liveFileNums[obj.DiskFileNum]; ok {
3685 2 : continue
3686 : }
3687 2 : fileInfo := fileInfo{
3688 2 : fileNum: obj.DiskFileNum,
3689 2 : }
3690 2 : if size, err := d.objProvider.Size(obj); err == nil {
3691 2 : fileInfo.fileSize = uint64(size)
3692 2 : }
3693 2 : obsoleteTables = append(obsoleteTables, fileInfo)
3694 :
3695 0 : default:
3696 : // Ignore object types we don't know about.
3697 : }
3698 : }
3699 :
3700 2 : d.mu.log.queue = merge(d.mu.log.queue, obsoleteLogs)
3701 2 : d.mu.versions.metrics.WAL.Files = int64(len(d.mu.log.queue))
3702 2 : d.mu.versions.obsoleteTables = merge(d.mu.versions.obsoleteTables, obsoleteTables)
3703 2 : d.mu.versions.updateObsoleteTableMetricsLocked()
3704 2 : d.mu.versions.obsoleteManifests = merge(d.mu.versions.obsoleteManifests, obsoleteManifests)
3705 2 : d.mu.versions.obsoleteOptions = merge(d.mu.versions.obsoleteOptions, obsoleteOptions)
3706 : }
3707 :
3708 : // disableFileDeletions disables file deletions and then waits for any
3709 : // in-progress deletion to finish. The caller is required to call
3710 : // enableFileDeletions in order to enable file deletions again. It is ok for
3711 : // multiple callers to disable file deletions simultaneously, though they must
3712 : // all invoke enableFileDeletions in order for file deletions to be re-enabled
3713 : // (there is an internal reference count on file deletion disablement).
3714 : //
3715 : // d.mu must be held when calling this method.
3716 2 : func (d *DB) disableFileDeletions() {
3717 2 : d.mu.disableFileDeletions++
3718 2 : d.mu.Unlock()
3719 2 : defer d.mu.Lock()
3720 2 : d.cleanupManager.Wait()
3721 2 : }
3722 :
3723 : // enableFileDeletions enables previously disabled file deletions. A cleanup job
3724 : // is queued if necessary.
3725 : //
3726 : // d.mu must be held when calling this method.
3727 2 : func (d *DB) enableFileDeletions() {
3728 2 : if d.mu.disableFileDeletions <= 0 {
3729 1 : panic("pebble: file deletion disablement invariant violated")
3730 : }
3731 2 : d.mu.disableFileDeletions--
3732 2 : if d.mu.disableFileDeletions > 0 {
3733 0 : return
3734 0 : }
3735 2 : jobID := d.mu.nextJobID
3736 2 : d.mu.nextJobID++
3737 2 : d.deleteObsoleteFiles(jobID)
3738 : }
3739 :
3740 : type fileInfo struct {
3741 : fileNum base.DiskFileNum
3742 : fileSize uint64
3743 : }
3744 :
3745 : // deleteObsoleteFiles enqueues a cleanup job to the cleanup manager, if necessary.
3746 : //
3747 : // d.mu must be held when calling this. The function will release and re-aquire the mutex.
3748 : //
3749 : // Does nothing if file deletions are disabled (see disableFileDeletions). A
3750 : // cleanup job will be scheduled when file deletions are re-enabled.
3751 2 : func (d *DB) deleteObsoleteFiles(jobID int) {
3752 2 : if d.mu.disableFileDeletions > 0 {
3753 2 : return
3754 2 : }
3755 :
3756 2 : var obsoleteLogs []fileInfo
3757 2 : for i := range d.mu.log.queue {
3758 2 : // NB: d.mu.versions.minUnflushedLogNum is the log number of the earliest
3759 2 : // log that has not had its contents flushed to an sstable. We can recycle
3760 2 : // the prefix of d.mu.log.queue with log numbers less than
3761 2 : // minUnflushedLogNum.
3762 2 : if d.mu.log.queue[i].fileNum >= d.mu.versions.minUnflushedLogNum {
3763 2 : obsoleteLogs = d.mu.log.queue[:i]
3764 2 : d.mu.log.queue = d.mu.log.queue[i:]
3765 2 : d.mu.versions.metrics.WAL.Files -= int64(len(obsoleteLogs))
3766 2 : break
3767 : }
3768 : }
3769 :
3770 2 : obsoleteTables := append([]fileInfo(nil), d.mu.versions.obsoleteTables...)
3771 2 : d.mu.versions.obsoleteTables = nil
3772 2 :
3773 2 : for _, tbl := range obsoleteTables {
3774 2 : delete(d.mu.versions.zombieTables, tbl.fileNum)
3775 2 : }
3776 :
3777 : // Sort the manifests cause we want to delete some contiguous prefix
3778 : // of the older manifests.
3779 2 : slices.SortFunc(d.mu.versions.obsoleteManifests, func(a, b fileInfo) int {
3780 2 : return cmp.Compare(a.fileNum, b.fileNum)
3781 2 : })
3782 :
3783 2 : var obsoleteManifests []fileInfo
3784 2 : manifestsToDelete := len(d.mu.versions.obsoleteManifests) - d.opts.NumPrevManifest
3785 2 : if manifestsToDelete > 0 {
3786 2 : obsoleteManifests = d.mu.versions.obsoleteManifests[:manifestsToDelete]
3787 2 : d.mu.versions.obsoleteManifests = d.mu.versions.obsoleteManifests[manifestsToDelete:]
3788 2 : if len(d.mu.versions.obsoleteManifests) == 0 {
3789 0 : d.mu.versions.obsoleteManifests = nil
3790 0 : }
3791 : }
3792 :
3793 2 : obsoleteOptions := d.mu.versions.obsoleteOptions
3794 2 : d.mu.versions.obsoleteOptions = nil
3795 2 :
3796 2 : // Release d.mu while preparing the cleanup job and possibly waiting.
3797 2 : // Note the unusual order: Unlock and then Lock.
3798 2 : d.mu.Unlock()
3799 2 : defer d.mu.Lock()
3800 2 :
3801 2 : files := [4]struct {
3802 2 : fileType fileType
3803 2 : obsolete []fileInfo
3804 2 : }{
3805 2 : {fileTypeLog, obsoleteLogs},
3806 2 : {fileTypeTable, obsoleteTables},
3807 2 : {fileTypeManifest, obsoleteManifests},
3808 2 : {fileTypeOptions, obsoleteOptions},
3809 2 : }
3810 2 : _, noRecycle := d.opts.Cleaner.(base.NeedsFileContents)
3811 2 : filesToDelete := make([]obsoleteFile, 0, len(obsoleteLogs)+len(obsoleteTables)+len(obsoleteManifests)+len(obsoleteOptions))
3812 2 : for _, f := range files {
3813 2 : // We sort to make the order of deletions deterministic, which is nice for
3814 2 : // tests.
3815 2 : slices.SortFunc(f.obsolete, func(a, b fileInfo) int {
3816 2 : return cmp.Compare(a.fileNum, b.fileNum)
3817 2 : })
3818 2 : for _, fi := range f.obsolete {
3819 2 : dir := d.dirname
3820 2 : switch f.fileType {
3821 2 : case fileTypeLog:
3822 2 : if !noRecycle && d.logRecycler.add(fi) {
3823 1 : continue
3824 : }
3825 2 : dir = d.walDirname
3826 2 : case fileTypeTable:
3827 2 : d.tableCache.evict(fi.fileNum)
3828 : }
3829 :
3830 2 : filesToDelete = append(filesToDelete, obsoleteFile{
3831 2 : dir: dir,
3832 2 : fileNum: fi.fileNum,
3833 2 : fileType: f.fileType,
3834 2 : fileSize: fi.fileSize,
3835 2 : })
3836 : }
3837 : }
3838 2 : if len(filesToDelete) > 0 {
3839 2 : d.cleanupManager.EnqueueJob(jobID, filesToDelete)
3840 2 : }
3841 2 : if d.opts.private.testingAlwaysWaitForCleanup {
3842 1 : d.cleanupManager.Wait()
3843 1 : }
3844 : }
3845 :
3846 2 : func (d *DB) maybeScheduleObsoleteTableDeletion() {
3847 2 : d.mu.Lock()
3848 2 : defer d.mu.Unlock()
3849 2 : d.maybeScheduleObsoleteTableDeletionLocked()
3850 2 : }
3851 :
3852 2 : func (d *DB) maybeScheduleObsoleteTableDeletionLocked() {
3853 2 : if len(d.mu.versions.obsoleteTables) > 0 {
3854 2 : jobID := d.mu.nextJobID
3855 2 : d.mu.nextJobID++
3856 2 : d.deleteObsoleteFiles(jobID)
3857 2 : }
3858 : }
3859 :
3860 2 : func merge(a, b []fileInfo) []fileInfo {
3861 2 : if len(b) == 0 {
3862 2 : return a
3863 2 : }
3864 :
3865 2 : a = append(a, b...)
3866 2 : slices.SortFunc(a, func(a, b fileInfo) int {
3867 2 : return cmp.Compare(a.fileNum, b.fileNum)
3868 2 : })
3869 2 : return slices.CompactFunc(a, func(a, b fileInfo) bool {
3870 2 : return a.fileNum == b.fileNum
3871 2 : })
3872 : }
|