Line data Source code
1 : // Copyright 2011 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 sstable
6 :
7 : import (
8 : "bytes"
9 : "encoding/binary"
10 : "fmt"
11 : "math"
12 : "runtime"
13 : "slices"
14 : "sort"
15 : "sync"
16 :
17 : "github.com/cespare/xxhash/v2"
18 : "github.com/cockroachdb/errors"
19 : "github.com/cockroachdb/pebble/internal/base"
20 : "github.com/cockroachdb/pebble/internal/bytealloc"
21 : "github.com/cockroachdb/pebble/internal/cache"
22 : "github.com/cockroachdb/pebble/internal/crc"
23 : "github.com/cockroachdb/pebble/internal/invariants"
24 : "github.com/cockroachdb/pebble/internal/keyspan"
25 : "github.com/cockroachdb/pebble/internal/private"
26 : "github.com/cockroachdb/pebble/internal/rangedel"
27 : "github.com/cockroachdb/pebble/internal/rangekey"
28 : "github.com/cockroachdb/pebble/objstorage"
29 : )
30 :
31 : // encodedBHPEstimatedSize estimates the size of the encoded BlockHandleWithProperties.
32 : // It would also be nice to account for the length of the data block properties here,
33 : // but isn't necessary since this is an estimate.
34 : const encodedBHPEstimatedSize = binary.MaxVarintLen64 * 2
35 :
36 : var errWriterClosed = errors.New("pebble: writer is closed")
37 :
38 : // WriterMetadata holds info about a finished sstable.
39 : type WriterMetadata struct {
40 : Size uint64
41 : SmallestPoint InternalKey
42 : // LargestPoint, LargestRangeKey, LargestRangeDel should not be accessed
43 : // before Writer.Close is called, because they may only be set on
44 : // Writer.Close.
45 : LargestPoint InternalKey
46 : SmallestRangeDel InternalKey
47 : LargestRangeDel InternalKey
48 : SmallestRangeKey InternalKey
49 : LargestRangeKey InternalKey
50 : HasPointKeys bool
51 : HasRangeDelKeys bool
52 : HasRangeKeys bool
53 : SmallestSeqNum uint64
54 : LargestSeqNum uint64
55 : Properties Properties
56 : }
57 :
58 : // SetSmallestPointKey sets the smallest point key to the given key.
59 : // NB: this method set the "absolute" smallest point key. Any existing key is
60 : // overridden.
61 1 : func (m *WriterMetadata) SetSmallestPointKey(k InternalKey) {
62 1 : m.SmallestPoint = k
63 1 : m.HasPointKeys = true
64 1 : }
65 :
66 : // SetSmallestRangeDelKey sets the smallest rangedel key to the given key.
67 : // NB: this method set the "absolute" smallest rangedel key. Any existing key is
68 : // overridden.
69 1 : func (m *WriterMetadata) SetSmallestRangeDelKey(k InternalKey) {
70 1 : m.SmallestRangeDel = k
71 1 : m.HasRangeDelKeys = true
72 1 : }
73 :
74 : // SetSmallestRangeKey sets the smallest range key to the given key.
75 : // NB: this method set the "absolute" smallest range key. Any existing key is
76 : // overridden.
77 1 : func (m *WriterMetadata) SetSmallestRangeKey(k InternalKey) {
78 1 : m.SmallestRangeKey = k
79 1 : m.HasRangeKeys = true
80 1 : }
81 :
82 : // SetLargestPointKey sets the largest point key to the given key.
83 : // NB: this method set the "absolute" largest point key. Any existing key is
84 : // overridden.
85 1 : func (m *WriterMetadata) SetLargestPointKey(k InternalKey) {
86 1 : m.LargestPoint = k
87 1 : m.HasPointKeys = true
88 1 : }
89 :
90 : // SetLargestRangeDelKey sets the largest rangedel key to the given key.
91 : // NB: this method set the "absolute" largest rangedel key. Any existing key is
92 : // overridden.
93 1 : func (m *WriterMetadata) SetLargestRangeDelKey(k InternalKey) {
94 1 : m.LargestRangeDel = k
95 1 : m.HasRangeDelKeys = true
96 1 : }
97 :
98 : // SetLargestRangeKey sets the largest range key to the given key.
99 : // NB: this method set the "absolute" largest range key. Any existing key is
100 : // overridden.
101 1 : func (m *WriterMetadata) SetLargestRangeKey(k InternalKey) {
102 1 : m.LargestRangeKey = k
103 1 : m.HasRangeKeys = true
104 1 : }
105 :
106 1 : func (m *WriterMetadata) updateSeqNum(seqNum uint64) {
107 1 : if m.SmallestSeqNum > seqNum {
108 1 : m.SmallestSeqNum = seqNum
109 1 : }
110 1 : if m.LargestSeqNum < seqNum {
111 1 : m.LargestSeqNum = seqNum
112 1 : }
113 : }
114 :
115 : // flushDecisionOptions holds parameters to inform the sstable block flushing
116 : // heuristics.
117 : type flushDecisionOptions struct {
118 : blockSize int
119 : blockSizeThreshold int
120 : // sizeClassAwareThreshold takes precedence over blockSizeThreshold when the
121 : // Writer is aware of the allocator's size classes.
122 : sizeClassAwareThreshold int
123 : }
124 :
125 : // Writer is a table writer.
126 : type Writer struct {
127 : writable objstorage.Writable
128 : meta WriterMetadata
129 : err error
130 : // cacheID and fileNum are used to remove blocks written to the sstable from
131 : // the cache, providing a defense in depth against bugs which cause cache
132 : // collisions.
133 : cacheID uint64
134 : fileNum base.DiskFileNum
135 : // dataBlockOptions and indexBlockOptions are used to configure the sstable
136 : // block flush heuristics.
137 : dataBlockOptions flushDecisionOptions
138 : indexBlockOptions flushDecisionOptions
139 : // The following fields are copied from Options.
140 : compare Compare
141 : split Split
142 : formatKey base.FormatKey
143 : compression Compression
144 : separator Separator
145 : successor Successor
146 : tableFormat TableFormat
147 : isStrictObsolete bool
148 : writingToLowestLevel bool
149 : cache *cache.Cache
150 : restartInterval int
151 : checksumType ChecksumType
152 : // disableKeyOrderChecks disables the checks that keys are added to an
153 : // sstable in order. It is intended for internal use only in the construction
154 : // of invalid sstables for testing. See tool/make_test_sstables.go.
155 : disableKeyOrderChecks bool
156 : // With two level indexes, the index/filter of a SST file is partitioned into
157 : // smaller blocks with an additional top-level index on them. When reading an
158 : // index/filter, only the top-level index is loaded into memory. The two level
159 : // index/filter then uses the top-level index to load on demand into the block
160 : // cache the partitions that are required to perform the index/filter query.
161 : //
162 : // Two level indexes are enabled automatically when there is more than one
163 : // index block.
164 : //
165 : // This is useful when there are very large index blocks, which generally occurs
166 : // with the usage of large keys. With large index blocks, the index blocks fight
167 : // the data blocks for block cache space and the index blocks are likely to be
168 : // re-read many times from the disk. The top level index, which has a much
169 : // smaller memory footprint, can be used to prevent the entire index block from
170 : // being loaded into the block cache.
171 : twoLevelIndex bool
172 : // Internal flag to allow creation of range-del-v1 format blocks. Only used
173 : // for testing. Note that v2 format blocks are backwards compatible with v1
174 : // format blocks.
175 : rangeDelV1Format bool
176 : indexBlock *indexBlockBuf
177 : rangeDelBlock blockWriter
178 : rangeKeyBlock blockWriter
179 : topLevelIndexBlock blockWriter
180 : props Properties
181 : blockPropCollectors []BlockPropertyCollector
182 : obsoleteCollector obsoleteKeyBlockPropertyCollector
183 : blockPropsEncoder blockPropertiesEncoder
184 : // filter accumulates the filter block. If populated, the filter ingests
185 : // either the output of w.split (i.e. a prefix extractor) if w.split is not
186 : // nil, or the full keys otherwise.
187 : filter filterWriter
188 : indexPartitions []indexBlockAndBlockProperties
189 :
190 : // indexBlockAlloc is used to bulk-allocate byte slices used to store index
191 : // blocks in indexPartitions. These live until the index finishes.
192 : indexBlockAlloc []byte
193 : // indexSepAlloc is used to bulk-allocate index block separator slices stored
194 : // in indexPartitions. These live until the index finishes.
195 : indexSepAlloc bytealloc.A
196 :
197 : // To allow potentially overlapping (i.e. un-fragmented) range keys spans to
198 : // be added to the Writer, a keyspan.Fragmenter is used to retain the keys
199 : // and values, emitting fragmented, coalesced spans as appropriate. Range
200 : // keys must be added in order of their start user-key.
201 : fragmenter keyspan.Fragmenter
202 : rangeKeyEncoder rangekey.Encoder
203 : rangeKeysBySuffix keyspan.KeysBySuffix
204 : rangeKeySpan keyspan.Span
205 : rkBuf []byte
206 : // dataBlockBuf consists of the state which is currently owned by and used by
207 : // the Writer client goroutine. This state can be handed off to other goroutines.
208 : dataBlockBuf *dataBlockBuf
209 : // blockBuf consists of the state which is owned by and used by the Writer client
210 : // goroutine.
211 : blockBuf blockBuf
212 :
213 : coordination coordinationState
214 :
215 : // Information (other than the byte slice) about the last point key, to
216 : // avoid extracting it again.
217 : lastPointKeyInfo pointKeyInfo
218 :
219 : // For value blocks.
220 : shortAttributeExtractor base.ShortAttributeExtractor
221 : requiredInPlaceValueBound UserKeyPrefixBound
222 : // When w.tableFormat >= TableFormatPebblev3, valueBlockWriter is nil iff
223 : // WriterOptions.DisableValueBlocks was true.
224 : valueBlockWriter *valueBlockWriter
225 :
226 : allocatorSizeClasses []int
227 : }
228 :
229 : type pointKeyInfo struct {
230 : trailer uint64
231 : // Only computed when w.valueBlockWriter is not nil.
232 : userKeyLen int
233 : // prefixLen uses w.split, if not nil. Only computed when w.valueBlockWriter
234 : // is not nil.
235 : prefixLen int
236 : // True iff the point was marked obsolete.
237 : isObsolete bool
238 : }
239 :
240 : type coordinationState struct {
241 : parallelismEnabled bool
242 :
243 : // writeQueue is used to write data blocks to disk. The writeQueue is primarily
244 : // used to maintain the order in which data blocks must be written to disk. For
245 : // this reason, every single data block write must be done through the writeQueue.
246 : writeQueue *writeQueue
247 :
248 : sizeEstimate dataBlockEstimates
249 : }
250 :
251 1 : func (c *coordinationState) init(parallelismEnabled bool, writer *Writer) {
252 1 : c.parallelismEnabled = parallelismEnabled
253 1 : // useMutex is false regardless of parallelismEnabled, because we do not do
254 1 : // parallel compression yet.
255 1 : c.sizeEstimate.useMutex = false
256 1 :
257 1 : // writeQueueSize determines the size of the write queue, or the number
258 1 : // of items which can be added to the queue without blocking. By default, we
259 1 : // use a writeQueue size of 0, since we won't be doing any block writes in
260 1 : // parallel.
261 1 : writeQueueSize := 0
262 1 : if parallelismEnabled {
263 1 : writeQueueSize = runtime.GOMAXPROCS(0)
264 1 : }
265 1 : c.writeQueue = newWriteQueue(writeQueueSize, writer)
266 : }
267 :
268 : // sizeEstimate is a general purpose helper for estimating two kinds of sizes:
269 : // A. The compressed sstable size, which is useful for deciding when to start
270 : //
271 : // a new sstable during flushes or compactions. In practice, we use this in
272 : // estimating the data size (excluding the index).
273 : //
274 : // B. The size of index blocks to decide when to start a new index block.
275 : //
276 : // There are some terminology peculiarities which are due to the origin of
277 : // sizeEstimate for use case A with parallel compression enabled (for which
278 : // the code has not been merged). Specifically this relates to the terms
279 : // "written" and "compressed".
280 : // - The notion of "written" for case A is sufficiently defined by saying that
281 : // the data block is compressed. Waiting for the actual data block write to
282 : // happen can result in unnecessary estimation, when we already know how big
283 : // it will be in compressed form. Additionally, with the forthcoming value
284 : // blocks containing older MVCC values, these compressed block will be held
285 : // in-memory until late in the sstable writing, and we do want to accurately
286 : // account for them without waiting for the actual write.
287 : // For case B, "written" means that the index entry has been fully
288 : // generated, and has been added to the uncompressed block buffer for that
289 : // index block. It does not include actually writing a potentially
290 : // compressed index block.
291 : // - The notion of "compressed" is to differentiate between a "inflight" size
292 : // and the actual size, and is handled via computing a compression ratio
293 : // observed so far (defaults to 1).
294 : // For case A, this is actual data block compression, so the "inflight" size
295 : // is uncompressed blocks (that are no longer being written to) and the
296 : // "compressed" size is after they have been compressed.
297 : // For case B the inflight size is for a key-value pair in the index for
298 : // which the value size (the encoded size of the BlockHandleWithProperties)
299 : // is not accurately known, while the compressed size is the size of that
300 : // entry when it has been added to the (in-progress) index ssblock.
301 : //
302 : // Usage: To update state, one can optionally provide an inflight write value
303 : // using addInflight (used for case B). When something is "written" the state
304 : // can be updated using either writtenWithDelta or writtenWithTotal, which
305 : // provide the actual delta size or the total size (latter must be
306 : // monotonically non-decreasing). If there were no calls to addInflight, there
307 : // isn't any real estimation happening here. So case A does not do any real
308 : // estimation. However, when we introduce parallel compression, there will be
309 : // estimation in that the client goroutine will call addInFlight and the
310 : // compression goroutines will call writtenWithDelta.
311 : type sizeEstimate struct {
312 : // emptySize is the size when there is no inflight data, and numEntries is 0.
313 : // emptySize is constant once set.
314 : emptySize uint64
315 :
316 : // inflightSize is the estimated size of some inflight data which hasn't
317 : // been written yet.
318 : inflightSize uint64
319 :
320 : // totalSize is the total size of the data which has already been written.
321 : totalSize uint64
322 :
323 : // numWrittenEntries is the total number of entries which have already been
324 : // written.
325 : numWrittenEntries uint64
326 : // numInflightEntries is the total number of entries which are inflight, and
327 : // haven't been written.
328 : numInflightEntries uint64
329 :
330 : // maxEstimatedSize stores the maximum result returned from sizeEstimate.size.
331 : // It ensures that values returned from subsequent calls to Writer.EstimatedSize
332 : // never decrease.
333 : maxEstimatedSize uint64
334 :
335 : // We assume that the entries added to the sizeEstimate can be compressed.
336 : // For this reason, we keep track of a compressedSize and an uncompressedSize
337 : // to compute a compression ratio for the inflight entries. If the entries
338 : // aren't being compressed, then compressedSize and uncompressedSize must be
339 : // equal.
340 : compressedSize uint64
341 : uncompressedSize uint64
342 : }
343 :
344 1 : func (s *sizeEstimate) init(emptySize uint64) {
345 1 : s.emptySize = emptySize
346 1 : }
347 :
348 1 : func (s *sizeEstimate) size() uint64 {
349 1 : ratio := float64(1)
350 1 : if s.uncompressedSize > 0 {
351 1 : ratio = float64(s.compressedSize) / float64(s.uncompressedSize)
352 1 : }
353 1 : estimatedInflightSize := uint64(float64(s.inflightSize) * ratio)
354 1 : total := s.totalSize + estimatedInflightSize
355 1 : if total > s.maxEstimatedSize {
356 1 : s.maxEstimatedSize = total
357 1 : } else {
358 1 : total = s.maxEstimatedSize
359 1 : }
360 :
361 1 : if total == 0 {
362 1 : return s.emptySize
363 1 : }
364 :
365 1 : return total
366 : }
367 :
368 1 : func (s *sizeEstimate) numTotalEntries() uint64 {
369 1 : return s.numWrittenEntries + s.numInflightEntries
370 1 : }
371 :
372 1 : func (s *sizeEstimate) addInflight(size int) {
373 1 : s.numInflightEntries++
374 1 : s.inflightSize += uint64(size)
375 1 : }
376 :
377 1 : func (s *sizeEstimate) writtenWithTotal(newTotalSize uint64, inflightSize int) {
378 1 : finalEntrySize := int(newTotalSize - s.totalSize)
379 1 : s.writtenWithDelta(finalEntrySize, inflightSize)
380 1 : }
381 :
382 1 : func (s *sizeEstimate) writtenWithDelta(finalEntrySize int, inflightSize int) {
383 1 : if inflightSize > 0 {
384 1 : // This entry was previously inflight, so we should decrement inflight
385 1 : // entries and update the "compression" stats for future estimation.
386 1 : s.numInflightEntries--
387 1 : s.inflightSize -= uint64(inflightSize)
388 1 : s.uncompressedSize += uint64(inflightSize)
389 1 : s.compressedSize += uint64(finalEntrySize)
390 1 : }
391 1 : s.numWrittenEntries++
392 1 : s.totalSize += uint64(finalEntrySize)
393 : }
394 :
395 1 : func (s *sizeEstimate) clear() {
396 1 : *s = sizeEstimate{emptySize: s.emptySize}
397 1 : }
398 :
399 : type indexBlockBuf struct {
400 : // block will only be accessed from the writeQueue.
401 : block blockWriter
402 :
403 : size struct {
404 : useMutex bool
405 : mu sync.Mutex
406 : estimate sizeEstimate
407 : }
408 :
409 : // restartInterval matches indexBlockBuf.block.restartInterval. We store it twice, because the `block`
410 : // must only be accessed from the writeQueue goroutine.
411 : restartInterval int
412 : }
413 :
414 1 : func (i *indexBlockBuf) clear() {
415 1 : i.block.clear()
416 1 : if i.size.useMutex {
417 1 : i.size.mu.Lock()
418 1 : defer i.size.mu.Unlock()
419 1 : }
420 1 : i.size.estimate.clear()
421 1 : i.restartInterval = 0
422 : }
423 :
424 : var indexBlockBufPool = sync.Pool{
425 1 : New: func() interface{} {
426 1 : return &indexBlockBuf{}
427 1 : },
428 : }
429 :
430 : const indexBlockRestartInterval = 1
431 :
432 1 : func newIndexBlockBuf(useMutex bool) *indexBlockBuf {
433 1 : i := indexBlockBufPool.Get().(*indexBlockBuf)
434 1 : i.size.useMutex = useMutex
435 1 : i.restartInterval = indexBlockRestartInterval
436 1 : i.block.restartInterval = indexBlockRestartInterval
437 1 : i.size.estimate.init(emptyBlockSize)
438 1 : return i
439 1 : }
440 :
441 : func (i *indexBlockBuf) shouldFlush(
442 : sep InternalKey, valueLen int, flushOptions flushDecisionOptions, sizeClassHints []int,
443 1 : ) bool {
444 1 : if i.size.useMutex {
445 1 : i.size.mu.Lock()
446 1 : defer i.size.mu.Unlock()
447 1 : }
448 :
449 1 : nEntries := i.size.estimate.numTotalEntries()
450 1 : return shouldFlushWithHints(
451 1 : sep.Size(), valueLen, i.restartInterval, int(i.size.estimate.size()),
452 1 : int(nEntries), flushOptions, sizeClassHints)
453 : }
454 :
455 1 : func (i *indexBlockBuf) add(key InternalKey, value []byte, inflightSize int) {
456 1 : i.block.add(key, value)
457 1 : size := i.block.estimatedSize()
458 1 : if i.size.useMutex {
459 1 : i.size.mu.Lock()
460 1 : defer i.size.mu.Unlock()
461 1 : }
462 1 : i.size.estimate.writtenWithTotal(uint64(size), inflightSize)
463 : }
464 :
465 1 : func (i *indexBlockBuf) finish() []byte {
466 1 : b := i.block.finish()
467 1 : return b
468 1 : }
469 :
470 1 : func (i *indexBlockBuf) addInflight(inflightSize int) {
471 1 : if i.size.useMutex {
472 1 : i.size.mu.Lock()
473 1 : defer i.size.mu.Unlock()
474 1 : }
475 1 : i.size.estimate.addInflight(inflightSize)
476 : }
477 :
478 1 : func (i *indexBlockBuf) estimatedSize() uint64 {
479 1 : if i.size.useMutex {
480 1 : i.size.mu.Lock()
481 1 : defer i.size.mu.Unlock()
482 1 : }
483 :
484 : // Make sure that the size estimation works as expected when parallelism
485 : // is disabled.
486 1 : if invariants.Enabled && !i.size.useMutex {
487 1 : if i.size.estimate.inflightSize != 0 {
488 0 : panic("unexpected inflight entry in index block size estimation")
489 : }
490 :
491 : // NB: The i.block should only be accessed from the writeQueue goroutine,
492 : // when parallelism is enabled. We break that invariant here, but that's
493 : // okay since parallelism is disabled.
494 1 : if i.size.estimate.size() != uint64(i.block.estimatedSize()) {
495 0 : panic("index block size estimation sans parallelism is incorrect")
496 : }
497 : }
498 1 : return i.size.estimate.size()
499 : }
500 :
501 : // sizeEstimate is used for sstable size estimation. sizeEstimate can be
502 : // accessed by the Writer client and compressionQueue goroutines. Fields
503 : // should only be read/updated through the functions defined on the
504 : // *sizeEstimate type.
505 : type dataBlockEstimates struct {
506 : // If we don't do block compression in parallel, then we don't need to take
507 : // the performance hit of synchronizing using this mutex.
508 : useMutex bool
509 : mu sync.Mutex
510 :
511 : estimate sizeEstimate
512 : }
513 :
514 : // inflightSize is the uncompressed block size estimate which has been
515 : // previously provided to addInflightDataBlock(). If addInflightDataBlock()
516 : // has not been called, this must be set to 0. compressedSize is the
517 : // compressed size of the block.
518 1 : func (d *dataBlockEstimates) dataBlockCompressed(compressedSize int, inflightSize int) {
519 1 : if d.useMutex {
520 0 : d.mu.Lock()
521 0 : defer d.mu.Unlock()
522 0 : }
523 1 : d.estimate.writtenWithDelta(compressedSize+blockTrailerLen, inflightSize)
524 : }
525 :
526 : // size is an estimated size of datablock data which has been written to disk.
527 1 : func (d *dataBlockEstimates) size() uint64 {
528 1 : if d.useMutex {
529 0 : d.mu.Lock()
530 0 : defer d.mu.Unlock()
531 0 : }
532 : // If there is no parallel compression, there should not be any inflight bytes.
533 1 : if invariants.Enabled && !d.useMutex {
534 1 : if d.estimate.inflightSize != 0 {
535 0 : panic("unexpected inflight entry in data block size estimation")
536 : }
537 : }
538 1 : return d.estimate.size()
539 : }
540 :
541 : // Avoid linter unused error.
542 : var _ = (&dataBlockEstimates{}).addInflightDataBlock
543 :
544 : // NB: unused since no parallel compression.
545 0 : func (d *dataBlockEstimates) addInflightDataBlock(size int) {
546 0 : if d.useMutex {
547 0 : d.mu.Lock()
548 0 : defer d.mu.Unlock()
549 0 : }
550 :
551 0 : d.estimate.addInflight(size)
552 : }
553 :
554 : var writeTaskPool = sync.Pool{
555 1 : New: func() interface{} {
556 1 : t := &writeTask{}
557 1 : t.compressionDone = make(chan bool, 1)
558 1 : return t
559 1 : },
560 : }
561 :
562 : type checksummer struct {
563 : checksumType ChecksumType
564 : xxHasher *xxhash.Digest
565 : }
566 :
567 1 : func (c *checksummer) checksum(block []byte, blockType []byte) (checksum uint32) {
568 1 : // Calculate the checksum.
569 1 : switch c.checksumType {
570 1 : case ChecksumTypeCRC32c:
571 1 : checksum = crc.New(block).Update(blockType).Value()
572 0 : case ChecksumTypeXXHash64:
573 0 : if c.xxHasher == nil {
574 0 : c.xxHasher = xxhash.New()
575 0 : } else {
576 0 : c.xxHasher.Reset()
577 0 : }
578 0 : c.xxHasher.Write(block)
579 0 : c.xxHasher.Write(blockType)
580 0 : checksum = uint32(c.xxHasher.Sum64())
581 0 : default:
582 0 : panic(errors.Newf("unsupported checksum type: %d", c.checksumType))
583 : }
584 1 : return checksum
585 : }
586 :
587 : type blockBuf struct {
588 : // tmp is a scratch buffer, large enough to hold either footerLen bytes,
589 : // blockTrailerLen bytes, (5 * binary.MaxVarintLen64) bytes, and most
590 : // likely large enough for a block handle with properties.
591 : tmp [blockHandleLikelyMaxLen]byte
592 : // compressedBuf is the destination buffer for compression. It is re-used over the
593 : // lifetime of the blockBuf, avoiding the allocation of a temporary buffer for each block.
594 : compressedBuf []byte
595 : checksummer checksummer
596 : }
597 :
598 1 : func (b *blockBuf) clear() {
599 1 : // We can't assign b.compressedBuf[:0] to compressedBuf because snappy relies
600 1 : // on the length of the buffer, and not the capacity to determine if it needs
601 1 : // to make an allocation.
602 1 : *b = blockBuf{
603 1 : compressedBuf: b.compressedBuf, checksummer: b.checksummer,
604 1 : }
605 1 : }
606 :
607 : // A dataBlockBuf holds all the state required to compress and write a data block to disk.
608 : // A dataBlockBuf begins its lifecycle owned by the Writer client goroutine. The Writer
609 : // client goroutine adds keys to the sstable, writing directly into a dataBlockBuf's blockWriter
610 : // until the block is full. Once a dataBlockBuf's block is full, the dataBlockBuf may be passed
611 : // to other goroutines for compression and file I/O.
612 : type dataBlockBuf struct {
613 : blockBuf
614 : dataBlock blockWriter
615 :
616 : // uncompressed is a reference to a byte slice which is owned by the dataBlockBuf. It is the
617 : // next byte slice to be compressed. The uncompressed byte slice will be backed by the
618 : // dataBlock.buf.
619 : uncompressed []byte
620 : // compressed is a reference to a byte slice which is owned by the dataBlockBuf. It is the
621 : // compressed byte slice which must be written to disk. The compressed byte slice may be
622 : // backed by the dataBlock.buf, or the dataBlockBuf.compressedBuf, depending on whether
623 : // we use the result of the compression.
624 : compressed []byte
625 :
626 : // We're making calls to BlockPropertyCollectors from the Writer client goroutine. We need to
627 : // pass the encoded block properties over to the write queue. To prevent copies, and allocations,
628 : // we give each dataBlockBuf, a blockPropertiesEncoder.
629 : blockPropsEncoder blockPropertiesEncoder
630 : // dataBlockProps is set when Writer.finishDataBlockProps is called. The dataBlockProps slice is
631 : // a shallow copy of the internal buffer of the dataBlockBuf.blockPropsEncoder.
632 : dataBlockProps []byte
633 :
634 : // sepScratch is reusable scratch space for computing separator keys.
635 : sepScratch []byte
636 : }
637 :
638 1 : func (d *dataBlockBuf) clear() {
639 1 : d.blockBuf.clear()
640 1 : d.dataBlock.clear()
641 1 :
642 1 : d.uncompressed = nil
643 1 : d.compressed = nil
644 1 : d.dataBlockProps = nil
645 1 : d.sepScratch = d.sepScratch[:0]
646 1 : }
647 :
648 : var dataBlockBufPool = sync.Pool{
649 1 : New: func() interface{} {
650 1 : return &dataBlockBuf{}
651 1 : },
652 : }
653 :
654 1 : func newDataBlockBuf(restartInterval int, checksumType ChecksumType) *dataBlockBuf {
655 1 : d := dataBlockBufPool.Get().(*dataBlockBuf)
656 1 : d.dataBlock.restartInterval = restartInterval
657 1 : d.checksummer.checksumType = checksumType
658 1 : return d
659 1 : }
660 :
661 1 : func (d *dataBlockBuf) finish() {
662 1 : d.uncompressed = d.dataBlock.finish()
663 1 : }
664 :
665 1 : func (d *dataBlockBuf) compressAndChecksum(c Compression) {
666 1 : d.compressed = compressAndChecksum(d.uncompressed, c, &d.blockBuf)
667 1 : }
668 :
669 : func (d *dataBlockBuf) shouldFlush(
670 : key InternalKey, valueLen int, flushOptions flushDecisionOptions, sizeClassHints []int,
671 1 : ) bool {
672 1 : return shouldFlushWithHints(
673 1 : key.Size(), valueLen, d.dataBlock.restartInterval, d.dataBlock.estimatedSize(),
674 1 : d.dataBlock.nEntries, flushOptions, sizeClassHints)
675 1 : }
676 :
677 : type indexBlockAndBlockProperties struct {
678 : nEntries int
679 : // sep is the last key added to this block, for computing a separator later.
680 : sep InternalKey
681 : properties []byte
682 : // block is the encoded block produced by blockWriter.finish.
683 : block []byte
684 : }
685 :
686 : // Set sets the value for the given key. The sequence number is set to 0.
687 : // Intended for use to externally construct an sstable before ingestion into a
688 : // DB. For a given Writer, the keys passed to Set must be in strictly increasing
689 : // order.
690 : //
691 : // TODO(peter): untested
692 1 : func (w *Writer) Set(key, value []byte) error {
693 1 : if w.err != nil {
694 0 : return w.err
695 0 : }
696 1 : if w.isStrictObsolete {
697 0 : return errors.Errorf("use AddWithForceObsolete")
698 0 : }
699 : // forceObsolete is false based on the assumption that no RANGEDELs in the
700 : // sstable delete the added points.
701 1 : return w.addPoint(base.MakeInternalKey(key, 0, InternalKeyKindSet), value, false)
702 : }
703 :
704 : // Delete deletes the value for the given key. The sequence number is set to
705 : // 0. Intended for use to externally construct an sstable before ingestion into
706 : // a DB.
707 : //
708 : // TODO(peter): untested
709 0 : func (w *Writer) Delete(key []byte) error {
710 0 : if w.err != nil {
711 0 : return w.err
712 0 : }
713 0 : if w.isStrictObsolete {
714 0 : return errors.Errorf("use AddWithForceObsolete")
715 0 : }
716 : // forceObsolete is false based on the assumption that no RANGEDELs in the
717 : // sstable delete the added points.
718 0 : return w.addPoint(base.MakeInternalKey(key, 0, InternalKeyKindDelete), nil, false)
719 : }
720 :
721 : // DeleteRange deletes all of the keys (and values) in the range [start,end)
722 : // (inclusive on start, exclusive on end). The sequence number is set to
723 : // 0. Intended for use to externally construct an sstable before ingestion into
724 : // a DB.
725 : //
726 : // TODO(peter): untested
727 1 : func (w *Writer) DeleteRange(start, end []byte) error {
728 1 : if w.err != nil {
729 0 : return w.err
730 0 : }
731 1 : return w.addTombstone(base.MakeInternalKey(start, 0, InternalKeyKindRangeDelete), end)
732 : }
733 :
734 : // Merge adds an action to the DB that merges the value at key with the new
735 : // value. The details of the merge are dependent upon the configured merge
736 : // operator. The sequence number is set to 0. Intended for use to externally
737 : // construct an sstable before ingestion into a DB.
738 : //
739 : // TODO(peter): untested
740 0 : func (w *Writer) Merge(key, value []byte) error {
741 0 : if w.err != nil {
742 0 : return w.err
743 0 : }
744 0 : if w.isStrictObsolete {
745 0 : return errors.Errorf("use AddWithForceObsolete")
746 0 : }
747 : // forceObsolete is false based on the assumption that no RANGEDELs in the
748 : // sstable that delete the added points. If the user configured this writer
749 : // to be strict-obsolete, addPoint will reject the addition of this MERGE.
750 0 : return w.addPoint(base.MakeInternalKey(key, 0, InternalKeyKindMerge), value, false)
751 : }
752 :
753 : // Add adds a key/value pair to the table being written. For a given Writer,
754 : // the keys passed to Add must be in increasing order. The exception to this
755 : // rule is range deletion tombstones. Range deletion tombstones need to be
756 : // added ordered by their start key, but they can be added out of order from
757 : // point entries. Additionally, range deletion tombstones must be fragmented
758 : // (i.e. by keyspan.Fragmenter).
759 1 : func (w *Writer) Add(key InternalKey, value []byte) error {
760 1 : if w.isStrictObsolete {
761 0 : return errors.Errorf("use AddWithForceObsolete")
762 0 : }
763 1 : return w.AddWithForceObsolete(key, value, false)
764 : }
765 :
766 : // AddWithForceObsolete must be used when writing a strict-obsolete sstable.
767 : //
768 : // forceObsolete indicates whether the caller has determined that this key is
769 : // obsolete even though it may be the latest point key for this userkey. This
770 : // should be set to true for keys obsoleted by RANGEDELs, and is required for
771 : // strict-obsolete sstables.
772 : //
773 : // Note that there are two properties, S1 and S2 (see comment in format.go)
774 : // that strict-obsolete ssts must satisfy. S2, due to RANGEDELs, is solely the
775 : // responsibility of the caller. S1 is solely the responsibility of the
776 : // callee.
777 1 : func (w *Writer) AddWithForceObsolete(key InternalKey, value []byte, forceObsolete bool) error {
778 1 : if w.err != nil {
779 0 : return w.err
780 0 : }
781 :
782 1 : switch key.Kind() {
783 1 : case InternalKeyKindRangeDelete:
784 1 : return w.addTombstone(key, value)
785 : case base.InternalKeyKindRangeKeyDelete,
786 : base.InternalKeyKindRangeKeySet,
787 0 : base.InternalKeyKindRangeKeyUnset:
788 0 : w.err = errors.Errorf(
789 0 : "pebble: range keys must be added via one of the RangeKey* functions")
790 0 : return w.err
791 : }
792 1 : return w.addPoint(key, value, forceObsolete)
793 : }
794 :
795 1 : func (w *Writer) makeAddPointDecisionV2(key InternalKey) error {
796 1 : prevTrailer := w.lastPointKeyInfo.trailer
797 1 : w.lastPointKeyInfo.trailer = key.Trailer
798 1 : if w.dataBlockBuf.dataBlock.nEntries == 0 {
799 1 : return nil
800 1 : }
801 1 : if !w.disableKeyOrderChecks {
802 1 : prevPointUserKey := w.dataBlockBuf.dataBlock.getCurUserKey()
803 1 : cmpUser := w.compare(prevPointUserKey, key.UserKey)
804 1 : if cmpUser > 0 || (cmpUser == 0 && prevTrailer <= key.Trailer) {
805 0 : return errors.Errorf(
806 0 : "pebble: keys must be added in strictly increasing order: %s, %s",
807 0 : InternalKey{UserKey: prevPointUserKey, Trailer: prevTrailer}.Pretty(w.formatKey),
808 0 : key.Pretty(w.formatKey))
809 0 : }
810 : }
811 1 : return nil
812 : }
813 :
814 : // REQUIRES: at least one point has been written to the Writer.
815 1 : func (w *Writer) getLastPointUserKey() []byte {
816 1 : if w.dataBlockBuf.dataBlock.nEntries == 0 {
817 0 : panic(errors.AssertionFailedf("no point keys added to writer"))
818 : }
819 1 : return w.dataBlockBuf.dataBlock.getCurUserKey()
820 : }
821 :
822 : // REQUIRES: w.tableFormat >= TableFormatPebblev3
823 : func (w *Writer) makeAddPointDecisionV3(
824 : key InternalKey, valueLen int,
825 1 : ) (setHasSamePrefix bool, writeToValueBlock bool, isObsolete bool, err error) {
826 1 : prevPointKeyInfo := w.lastPointKeyInfo
827 1 : w.lastPointKeyInfo.userKeyLen = len(key.UserKey)
828 1 : w.lastPointKeyInfo.prefixLen = w.split(key.UserKey)
829 1 : w.lastPointKeyInfo.trailer = key.Trailer
830 1 : w.lastPointKeyInfo.isObsolete = false
831 1 : if !w.meta.HasPointKeys {
832 1 : return false, false, false, nil
833 1 : }
834 1 : keyKind := base.TrailerKind(key.Trailer)
835 1 : prevPointUserKey := w.getLastPointUserKey()
836 1 : prevPointKey := InternalKey{UserKey: prevPointUserKey, Trailer: prevPointKeyInfo.trailer}
837 1 : prevKeyKind := base.TrailerKind(prevPointKeyInfo.trailer)
838 1 : considerWriteToValueBlock := prevKeyKind == InternalKeyKindSet &&
839 1 : keyKind == InternalKeyKindSet
840 1 : if considerWriteToValueBlock && !w.requiredInPlaceValueBound.IsEmpty() {
841 0 : keyPrefix := key.UserKey[:w.lastPointKeyInfo.prefixLen]
842 0 : cmpUpper := w.compare(
843 0 : w.requiredInPlaceValueBound.Upper, keyPrefix)
844 0 : if cmpUpper <= 0 {
845 0 : // Common case for CockroachDB. Make it empty since all future keys in
846 0 : // this sstable will also have cmpUpper <= 0.
847 0 : w.requiredInPlaceValueBound = UserKeyPrefixBound{}
848 0 : } else if w.compare(keyPrefix, w.requiredInPlaceValueBound.Lower) >= 0 {
849 0 : considerWriteToValueBlock = false
850 0 : }
851 : }
852 : // cmpPrefix is initialized iff considerWriteToValueBlock.
853 1 : var cmpPrefix int
854 1 : var cmpUser int
855 1 : if considerWriteToValueBlock {
856 1 : // Compare the prefixes.
857 1 : cmpPrefix = w.compare(prevPointUserKey[:prevPointKeyInfo.prefixLen],
858 1 : key.UserKey[:w.lastPointKeyInfo.prefixLen])
859 1 : cmpUser = cmpPrefix
860 1 : if cmpPrefix == 0 {
861 1 : // Need to compare suffixes to compute cmpUser.
862 1 : cmpUser = w.compare(prevPointUserKey[prevPointKeyInfo.prefixLen:],
863 1 : key.UserKey[w.lastPointKeyInfo.prefixLen:])
864 1 : }
865 1 : } else {
866 1 : cmpUser = w.compare(prevPointUserKey, key.UserKey)
867 1 : }
868 : // Ensure that no one adds a point key kind without considering the obsolete
869 : // handling for that kind.
870 1 : switch keyKind {
871 : case InternalKeyKindSet, InternalKeyKindSetWithDelete, InternalKeyKindMerge,
872 1 : InternalKeyKindDelete, InternalKeyKindSingleDelete, InternalKeyKindDeleteSized:
873 0 : default:
874 0 : panic(errors.AssertionFailedf("unexpected key kind %s", keyKind.String()))
875 : }
876 : // If same user key, then the current key is obsolete if any of the
877 : // following is true:
878 : // C1 The prev key was obsolete.
879 : // C2 The prev key was not a MERGE. When the previous key is a MERGE we must
880 : // preserve SET* and MERGE since their values will be merged into the
881 : // previous key. We also must preserve DEL* since there may be an older
882 : // SET*/MERGE in a lower level that must not be merged with the MERGE --
883 : // if we omit the DEL* that lower SET*/MERGE will become visible.
884 : //
885 : // Regardless of whether it is the same user key or not
886 : // C3 The current key is some kind of point delete, and we are writing to
887 : // the lowest level, then it is also obsolete. The correctness of this
888 : // relies on the same user key not spanning multiple sstables in a level.
889 : //
890 : // C1 ensures that for a user key there is at most one transition from
891 : // !obsolete to obsolete. Consider a user key k, for which the first n keys
892 : // are not obsolete. We consider the various value of n:
893 : //
894 : // n = 0: This happens due to forceObsolete being set by the caller, or due
895 : // to C3. forceObsolete must only be set due a RANGEDEL, and that RANGEDEL
896 : // must also delete all the lower seqnums for the same user key. C3 triggers
897 : // due to a point delete and that deletes all the lower seqnums for the same
898 : // user key.
899 : //
900 : // n = 1: This is the common case. It happens when the first key is not a
901 : // MERGE, or the current key is some kind of point delete.
902 : //
903 : // n > 1: This is due to a sequence of MERGE keys, potentially followed by a
904 : // single non-MERGE key.
905 1 : isObsoleteC1AndC2 := cmpUser == 0 &&
906 1 : (prevPointKeyInfo.isObsolete || prevKeyKind != InternalKeyKindMerge)
907 1 : isObsoleteC3 := w.writingToLowestLevel &&
908 1 : (keyKind == InternalKeyKindDelete || keyKind == InternalKeyKindSingleDelete ||
909 1 : keyKind == InternalKeyKindDeleteSized)
910 1 : isObsolete = isObsoleteC1AndC2 || isObsoleteC3
911 1 : // TODO(sumeer): storing isObsolete SET and SETWITHDEL in value blocks is
912 1 : // possible, but requires some care in documenting and checking invariants.
913 1 : // There is code that assumes nothing in value blocks because of single MVCC
914 1 : // version (those should be ok). We have to ensure setHasSamePrefix is
915 1 : // correctly initialized here etc.
916 1 :
917 1 : if !w.disableKeyOrderChecks &&
918 1 : (cmpUser > 0 || (cmpUser == 0 && prevPointKeyInfo.trailer <= key.Trailer)) {
919 0 : return false, false, false, errors.Errorf(
920 0 : "pebble: keys must be added in strictly increasing order: %s, %s",
921 0 : prevPointKey.Pretty(w.formatKey), key.Pretty(w.formatKey))
922 0 : }
923 1 : if !considerWriteToValueBlock {
924 1 : return false, false, isObsolete, nil
925 1 : }
926 : // NB: it is possible that cmpUser == 0, i.e., these two SETs have identical
927 : // user keys (because of an open snapshot). This should be the rare case.
928 1 : setHasSamePrefix = cmpPrefix == 0
929 1 : // Use of 0 here is somewhat arbitrary. Given the minimum 3 byte encoding of
930 1 : // valueHandle, this should be > 3. But tiny values are common in test and
931 1 : // unlikely in production, so we use 0 here for better test coverage.
932 1 : const tinyValueThreshold = 0
933 1 : // NB: setting WriterOptions.DisableValueBlocks does not disable the
934 1 : // setHasSamePrefix optimization.
935 1 : considerWriteToValueBlock = setHasSamePrefix && valueLen > tinyValueThreshold && w.valueBlockWriter != nil
936 1 : return setHasSamePrefix, considerWriteToValueBlock, isObsolete, nil
937 : }
938 :
939 1 : func (w *Writer) addPoint(key InternalKey, value []byte, forceObsolete bool) error {
940 1 : if w.isStrictObsolete && key.Kind() == InternalKeyKindMerge {
941 0 : return errors.Errorf("MERGE not supported in a strict-obsolete sstable")
942 0 : }
943 1 : var err error
944 1 : var setHasSameKeyPrefix, writeToValueBlock, addPrefixToValueStoredWithKey bool
945 1 : var isObsolete bool
946 1 : maxSharedKeyLen := len(key.UserKey)
947 1 : if w.tableFormat >= TableFormatPebblev3 {
948 1 : // maxSharedKeyLen is limited to the prefix of the preceding key. If the
949 1 : // preceding key was in a different block, then the blockWriter will
950 1 : // ignore this maxSharedKeyLen.
951 1 : maxSharedKeyLen = w.lastPointKeyInfo.prefixLen
952 1 : setHasSameKeyPrefix, writeToValueBlock, isObsolete, err =
953 1 : w.makeAddPointDecisionV3(key, len(value))
954 1 : addPrefixToValueStoredWithKey = base.TrailerKind(key.Trailer) == InternalKeyKindSet
955 1 : } else {
956 1 : err = w.makeAddPointDecisionV2(key)
957 1 : }
958 1 : if err != nil {
959 0 : return err
960 0 : }
961 1 : isObsolete = w.tableFormat >= TableFormatPebblev4 && (isObsolete || forceObsolete)
962 1 : w.lastPointKeyInfo.isObsolete = isObsolete
963 1 : var valueStoredWithKey []byte
964 1 : var prefix valuePrefix
965 1 : var valueStoredWithKeyLen int
966 1 : if writeToValueBlock {
967 1 : vh, err := w.valueBlockWriter.addValue(value)
968 1 : if err != nil {
969 0 : return err
970 0 : }
971 1 : n := encodeValueHandle(w.blockBuf.tmp[:], vh)
972 1 : valueStoredWithKey = w.blockBuf.tmp[:n]
973 1 : valueStoredWithKeyLen = len(valueStoredWithKey) + 1
974 1 : var attribute base.ShortAttribute
975 1 : if w.shortAttributeExtractor != nil {
976 0 : // TODO(sumeer): for compactions, it is possible that the input sstable
977 0 : // already has this value in the value section and so we have already
978 0 : // extracted the ShortAttribute. Avoid extracting it again. This will
979 0 : // require changing the Writer.Add interface.
980 0 : if attribute, err = w.shortAttributeExtractor(
981 0 : key.UserKey, w.lastPointKeyInfo.prefixLen, value); err != nil {
982 0 : return err
983 0 : }
984 : }
985 1 : prefix = makePrefixForValueHandle(setHasSameKeyPrefix, attribute)
986 1 : } else {
987 1 : valueStoredWithKey = value
988 1 : valueStoredWithKeyLen = len(value)
989 1 : if addPrefixToValueStoredWithKey {
990 1 : valueStoredWithKeyLen++
991 1 : }
992 1 : prefix = makePrefixForInPlaceValue(setHasSameKeyPrefix)
993 : }
994 :
995 1 : if err := w.maybeFlush(key, valueStoredWithKeyLen); err != nil {
996 0 : return err
997 0 : }
998 :
999 1 : for i := range w.blockPropCollectors {
1000 1 : v := value
1001 1 : if addPrefixToValueStoredWithKey {
1002 1 : // Values for SET are not required to be in-place, and in the future may
1003 1 : // not even be read by the compaction, so pass nil values. Block
1004 1 : // property collectors in such Pebble DB's must not look at the value.
1005 1 : v = nil
1006 1 : }
1007 1 : if err := w.blockPropCollectors[i].Add(key, v); err != nil {
1008 0 : w.err = err
1009 0 : return err
1010 0 : }
1011 : }
1012 1 : if w.tableFormat >= TableFormatPebblev4 {
1013 1 : w.obsoleteCollector.AddPoint(isObsolete)
1014 1 : }
1015 :
1016 1 : w.maybeAddToFilter(key.UserKey)
1017 1 : w.dataBlockBuf.dataBlock.addWithOptionalValuePrefix(
1018 1 : key, isObsolete, valueStoredWithKey, maxSharedKeyLen, addPrefixToValueStoredWithKey, prefix,
1019 1 : setHasSameKeyPrefix)
1020 1 :
1021 1 : w.meta.updateSeqNum(key.SeqNum())
1022 1 :
1023 1 : if !w.meta.HasPointKeys {
1024 1 : k := w.dataBlockBuf.dataBlock.getCurKey()
1025 1 : // NB: We need to ensure that SmallestPoint.UserKey is set, so we create
1026 1 : // an InternalKey which is semantically identical to the key, but won't
1027 1 : // have a nil UserKey. We do this, because key.UserKey could be nil, and
1028 1 : // we don't want SmallestPoint.UserKey to be nil.
1029 1 : //
1030 1 : // todo(bananabrick): Determine if it's okay to have a nil SmallestPoint
1031 1 : // .UserKey now that we don't rely on a nil UserKey to determine if the
1032 1 : // key has been set or not.
1033 1 : w.meta.SetSmallestPointKey(k.Clone())
1034 1 : }
1035 :
1036 1 : w.props.NumEntries++
1037 1 : switch key.Kind() {
1038 1 : case InternalKeyKindDelete, InternalKeyKindSingleDelete:
1039 1 : w.props.NumDeletions++
1040 1 : w.props.RawPointTombstoneKeySize += uint64(len(key.UserKey))
1041 1 : case InternalKeyKindDeleteSized:
1042 1 : var size uint64
1043 1 : if len(value) > 0 {
1044 1 : var n int
1045 1 : size, n = binary.Uvarint(value)
1046 1 : if n <= 0 {
1047 0 : w.err = errors.Newf("%s key's value (%x) does not parse as uvarint",
1048 0 : errors.Safe(key.Kind().String()), value)
1049 0 : return w.err
1050 0 : }
1051 : }
1052 1 : w.props.NumDeletions++
1053 1 : w.props.NumSizedDeletions++
1054 1 : w.props.RawPointTombstoneKeySize += uint64(len(key.UserKey))
1055 1 : w.props.RawPointTombstoneValueSize += size
1056 1 : case InternalKeyKindMerge:
1057 1 : w.props.NumMergeOperands++
1058 : }
1059 1 : w.props.RawKeySize += uint64(key.Size())
1060 1 : w.props.RawValueSize += uint64(len(value))
1061 1 : return nil
1062 : }
1063 :
1064 0 : func (w *Writer) prettyTombstone(k InternalKey, value []byte) fmt.Formatter {
1065 0 : return keyspan.Span{
1066 0 : Start: k.UserKey,
1067 0 : End: value,
1068 0 : Keys: []keyspan.Key{{Trailer: k.Trailer}},
1069 0 : }.Pretty(w.formatKey)
1070 0 : }
1071 :
1072 1 : func (w *Writer) addTombstone(key InternalKey, value []byte) error {
1073 1 : if !w.disableKeyOrderChecks && !w.rangeDelV1Format && w.rangeDelBlock.nEntries > 0 {
1074 1 : // Check that tombstones are being added in fragmented order. If the two
1075 1 : // tombstones overlap, their start and end keys must be identical.
1076 1 : prevKey := w.rangeDelBlock.getCurKey()
1077 1 : switch c := w.compare(prevKey.UserKey, key.UserKey); {
1078 0 : case c > 0:
1079 0 : w.err = errors.Errorf("pebble: keys must be added in order: %s, %s",
1080 0 : prevKey.Pretty(w.formatKey), key.Pretty(w.formatKey))
1081 0 : return w.err
1082 1 : case c == 0:
1083 1 : prevValue := w.rangeDelBlock.curValue
1084 1 : if w.compare(prevValue, value) != 0 {
1085 0 : w.err = errors.Errorf("pebble: overlapping tombstones must be fragmented: %s vs %s",
1086 0 : w.prettyTombstone(prevKey, prevValue),
1087 0 : w.prettyTombstone(key, value))
1088 0 : return w.err
1089 0 : }
1090 1 : if prevKey.SeqNum() <= key.SeqNum() {
1091 0 : w.err = errors.Errorf("pebble: keys must be added in strictly increasing order: %s, %s",
1092 0 : prevKey.Pretty(w.formatKey), key.Pretty(w.formatKey))
1093 0 : return w.err
1094 0 : }
1095 1 : default:
1096 1 : prevValue := w.rangeDelBlock.curValue
1097 1 : if w.compare(prevValue, key.UserKey) > 0 {
1098 0 : w.err = errors.Errorf("pebble: overlapping tombstones must be fragmented: %s vs %s",
1099 0 : w.prettyTombstone(prevKey, prevValue),
1100 0 : w.prettyTombstone(key, value))
1101 0 : return w.err
1102 0 : }
1103 : }
1104 : }
1105 :
1106 1 : if key.Trailer == InternalKeyRangeDeleteSentinel {
1107 0 : w.err = errors.Errorf("pebble: cannot add range delete sentinel: %s", key.Pretty(w.formatKey))
1108 0 : return w.err
1109 0 : }
1110 :
1111 1 : w.meta.updateSeqNum(key.SeqNum())
1112 1 :
1113 1 : switch {
1114 0 : case w.rangeDelV1Format:
1115 0 : // Range tombstones are not fragmented in the v1 (i.e. RocksDB) range
1116 0 : // deletion block format, so we need to track the largest range tombstone
1117 0 : // end key as every range tombstone is added.
1118 0 : //
1119 0 : // Note that writing the v1 format is only supported for tests.
1120 0 : if w.props.NumRangeDeletions == 0 {
1121 0 : w.meta.SetSmallestRangeDelKey(key.Clone())
1122 0 : w.meta.SetLargestRangeDelKey(base.MakeRangeDeleteSentinelKey(value).Clone())
1123 0 : } else {
1124 0 : if base.InternalCompare(w.compare, w.meta.SmallestRangeDel, key) > 0 {
1125 0 : w.meta.SetSmallestRangeDelKey(key.Clone())
1126 0 : }
1127 0 : end := base.MakeRangeDeleteSentinelKey(value)
1128 0 : if base.InternalCompare(w.compare, w.meta.LargestRangeDel, end) < 0 {
1129 0 : w.meta.SetLargestRangeDelKey(end.Clone())
1130 0 : }
1131 : }
1132 :
1133 1 : default:
1134 1 : // Range tombstones are fragmented in the v2 range deletion block format,
1135 1 : // so the start key of the first range tombstone added will be the smallest
1136 1 : // range tombstone key. The largest range tombstone key will be determined
1137 1 : // in Writer.Close() as the end key of the last range tombstone added.
1138 1 : if w.props.NumRangeDeletions == 0 {
1139 1 : w.meta.SetSmallestRangeDelKey(key.Clone())
1140 1 : }
1141 : }
1142 :
1143 1 : w.props.NumEntries++
1144 1 : w.props.NumDeletions++
1145 1 : w.props.NumRangeDeletions++
1146 1 : w.props.RawKeySize += uint64(key.Size())
1147 1 : w.props.RawValueSize += uint64(len(value))
1148 1 : w.rangeDelBlock.add(key, value)
1149 1 : return nil
1150 : }
1151 :
1152 : // RangeKeySet sets a range between start (inclusive) and end (exclusive) with
1153 : // the given suffix to the given value. The resulting range key is given the
1154 : // sequence number zero, with the expectation that the resulting sstable will be
1155 : // ingested.
1156 : //
1157 : // Keys must be added to the table in increasing order of start key. Spans are
1158 : // not required to be fragmented. The same suffix may not be set or unset twice
1159 : // over the same keyspan, because it would result in inconsistent state. Both
1160 : // the Set and Unset would share the zero sequence number, and a key cannot be
1161 : // both simultaneously set and unset.
1162 0 : func (w *Writer) RangeKeySet(start, end, suffix, value []byte) error {
1163 0 : return w.addRangeKeySpan(keyspan.Span{
1164 0 : Start: w.tempRangeKeyCopy(start),
1165 0 : End: w.tempRangeKeyCopy(end),
1166 0 : Keys: []keyspan.Key{
1167 0 : {
1168 0 : Trailer: base.MakeTrailer(0, base.InternalKeyKindRangeKeySet),
1169 0 : Suffix: w.tempRangeKeyCopy(suffix),
1170 0 : Value: w.tempRangeKeyCopy(value),
1171 0 : },
1172 0 : },
1173 0 : })
1174 0 : }
1175 :
1176 : // RangeKeyUnset un-sets a range between start (inclusive) and end (exclusive)
1177 : // with the given suffix. The resulting range key is given the
1178 : // sequence number zero, with the expectation that the resulting sstable will be
1179 : // ingested.
1180 : //
1181 : // Keys must be added to the table in increasing order of start key. Spans are
1182 : // not required to be fragmented. The same suffix may not be set or unset twice
1183 : // over the same keyspan, because it would result in inconsistent state. Both
1184 : // the Set and Unset would share the zero sequence number, and a key cannot be
1185 : // both simultaneously set and unset.
1186 0 : func (w *Writer) RangeKeyUnset(start, end, suffix []byte) error {
1187 0 : return w.addRangeKeySpan(keyspan.Span{
1188 0 : Start: w.tempRangeKeyCopy(start),
1189 0 : End: w.tempRangeKeyCopy(end),
1190 0 : Keys: []keyspan.Key{
1191 0 : {
1192 0 : Trailer: base.MakeTrailer(0, base.InternalKeyKindRangeKeyUnset),
1193 0 : Suffix: w.tempRangeKeyCopy(suffix),
1194 0 : },
1195 0 : },
1196 0 : })
1197 0 : }
1198 :
1199 : // RangeKeyDelete deletes a range between start (inclusive) and end (exclusive).
1200 : //
1201 : // Keys must be added to the table in increasing order of start key. Spans are
1202 : // not required to be fragmented.
1203 0 : func (w *Writer) RangeKeyDelete(start, end []byte) error {
1204 0 : return w.addRangeKeySpan(keyspan.Span{
1205 0 : Start: w.tempRangeKeyCopy(start),
1206 0 : End: w.tempRangeKeyCopy(end),
1207 0 : Keys: []keyspan.Key{
1208 0 : {Trailer: base.MakeTrailer(0, base.InternalKeyKindRangeKeyDelete)},
1209 0 : },
1210 0 : })
1211 0 : }
1212 :
1213 : // AddRangeKey adds a range key set, unset, or delete key/value pair to the
1214 : // table being written.
1215 : //
1216 : // Range keys must be supplied in strictly ascending order of start key (i.e.
1217 : // user key ascending, sequence number descending, and key type descending).
1218 : // Ranges added must also be supplied in fragmented span order - i.e. other than
1219 : // spans that are perfectly aligned (same start and end keys), spans may not
1220 : // overlap. Range keys may be added out of order relative to point keys and
1221 : // range deletions.
1222 1 : func (w *Writer) AddRangeKey(key InternalKey, value []byte) error {
1223 1 : if w.err != nil {
1224 0 : return w.err
1225 0 : }
1226 1 : return w.addRangeKey(key, value)
1227 : }
1228 :
1229 0 : func (w *Writer) addRangeKeySpan(span keyspan.Span) error {
1230 0 : if w.compare(span.Start, span.End) >= 0 {
1231 0 : return errors.Errorf(
1232 0 : "pebble: start key must be strictly less than end key",
1233 0 : )
1234 0 : }
1235 0 : if w.fragmenter.Start() != nil && w.compare(w.fragmenter.Start(), span.Start) > 0 {
1236 0 : return errors.Errorf("pebble: spans must be added in order: %s > %s",
1237 0 : w.formatKey(w.fragmenter.Start()), w.formatKey(span.Start))
1238 0 : }
1239 : // Add this span to the fragmenter.
1240 0 : w.fragmenter.Add(span)
1241 0 : return w.err
1242 : }
1243 :
1244 0 : func (w *Writer) encodeRangeKeySpan(span keyspan.Span) {
1245 0 : // This method is the emit function of the Fragmenter.
1246 0 : //
1247 0 : // NB: The span should only contain range keys and be internally consistent
1248 0 : // (eg, no duplicate suffixes, no additional keys after a RANGEKEYDEL).
1249 0 : //
1250 0 : // We use w.rangeKeysBySuffix and w.rangeKeySpan to avoid allocations.
1251 0 :
1252 0 : // Sort the keys by suffix. Iteration doesn't *currently* depend on it, but
1253 0 : // we may want to in the future.
1254 0 : w.rangeKeysBySuffix.Cmp = w.compare
1255 0 : w.rangeKeysBySuffix.Keys = span.Keys
1256 0 : sort.Sort(&w.rangeKeysBySuffix)
1257 0 :
1258 0 : w.rangeKeySpan = span
1259 0 : w.rangeKeySpan.Keys = w.rangeKeysBySuffix.Keys
1260 0 : w.err = firstError(w.err, w.rangeKeyEncoder.Encode(&w.rangeKeySpan))
1261 0 : }
1262 :
1263 1 : func (w *Writer) addRangeKey(key InternalKey, value []byte) error {
1264 1 : if !w.disableKeyOrderChecks && w.rangeKeyBlock.nEntries > 0 {
1265 1 : prevStartKey := w.rangeKeyBlock.getCurKey()
1266 1 : prevEndKey, _, ok := rangekey.DecodeEndKey(prevStartKey.Kind(), w.rangeKeyBlock.curValue)
1267 1 : if !ok {
1268 0 : // We panic here as we should have previously decoded and validated this
1269 0 : // key and value when it was first added to the range key block.
1270 0 : panic(errors.Errorf("pebble: invalid end key for span: %s",
1271 0 : prevStartKey.Pretty(w.formatKey)))
1272 : }
1273 :
1274 1 : curStartKey := key
1275 1 : curEndKey, _, ok := rangekey.DecodeEndKey(curStartKey.Kind(), value)
1276 1 : if !ok {
1277 0 : w.err = errors.Errorf("pebble: invalid end key for span: %s",
1278 0 : curStartKey.Pretty(w.formatKey))
1279 0 : return w.err
1280 0 : }
1281 :
1282 : // Start keys must be strictly increasing.
1283 1 : if base.InternalCompare(w.compare, prevStartKey, curStartKey) >= 0 {
1284 0 : w.err = errors.Errorf(
1285 0 : "pebble: range keys starts must be added in increasing order: %s, %s",
1286 0 : prevStartKey.Pretty(w.formatKey), key.Pretty(w.formatKey))
1287 0 : return w.err
1288 0 : }
1289 :
1290 : // Start keys are increasing. If the start user keys are equal, the
1291 : // end keys must be equal (i.e. aligned spans).
1292 1 : if w.compare(prevStartKey.UserKey, curStartKey.UserKey) == 0 {
1293 1 : if w.compare(prevEndKey, curEndKey) != 0 {
1294 0 : w.err = errors.Errorf("pebble: overlapping range keys must be fragmented: %s, %s",
1295 0 : prevStartKey.Pretty(w.formatKey),
1296 0 : curStartKey.Pretty(w.formatKey))
1297 0 : return w.err
1298 0 : }
1299 1 : } else if w.compare(prevEndKey, curStartKey.UserKey) > 0 {
1300 0 : // If the start user keys are NOT equal, the spans must be disjoint (i.e.
1301 0 : // no overlap).
1302 0 : // NOTE: the inequality excludes zero, as we allow the end key of the
1303 0 : // lower span be the same as the start key of the upper span, because
1304 0 : // the range end key is considered an exclusive bound.
1305 0 : w.err = errors.Errorf("pebble: overlapping range keys must be fragmented: %s, %s",
1306 0 : prevStartKey.Pretty(w.formatKey),
1307 0 : curStartKey.Pretty(w.formatKey))
1308 0 : return w.err
1309 0 : }
1310 : }
1311 :
1312 : // TODO(travers): Add an invariant-gated check to ensure that suffix-values
1313 : // are sorted within coalesced spans.
1314 :
1315 : // Range-keys and point-keys are intended to live in "parallel" keyspaces.
1316 : // However, we track a single seqnum in the table metadata that spans both of
1317 : // these keyspaces.
1318 : // TODO(travers): Consider tracking range key seqnums separately.
1319 1 : w.meta.updateSeqNum(key.SeqNum())
1320 1 :
1321 1 : // Range tombstones are fragmented, so the start key of the first range key
1322 1 : // added will be the smallest. The largest range key is determined in
1323 1 : // Writer.Close() as the end key of the last range key added to the block.
1324 1 : if w.props.NumRangeKeys() == 0 {
1325 1 : w.meta.SetSmallestRangeKey(key.Clone())
1326 1 : }
1327 :
1328 : // Update block properties.
1329 1 : w.props.RawRangeKeyKeySize += uint64(key.Size())
1330 1 : w.props.RawRangeKeyValueSize += uint64(len(value))
1331 1 : switch key.Kind() {
1332 1 : case base.InternalKeyKindRangeKeyDelete:
1333 1 : w.props.NumRangeKeyDels++
1334 1 : case base.InternalKeyKindRangeKeySet:
1335 1 : w.props.NumRangeKeySets++
1336 1 : case base.InternalKeyKindRangeKeyUnset:
1337 1 : w.props.NumRangeKeyUnsets++
1338 0 : default:
1339 0 : panic(errors.Errorf("pebble: invalid range key type: %s", key.Kind()))
1340 : }
1341 :
1342 1 : for i := range w.blockPropCollectors {
1343 1 : if err := w.blockPropCollectors[i].Add(key, value); err != nil {
1344 0 : return err
1345 0 : }
1346 : }
1347 :
1348 : // Add the key to the block.
1349 1 : w.rangeKeyBlock.add(key, value)
1350 1 : return nil
1351 : }
1352 :
1353 : // tempRangeKeyBuf returns a slice of length n from the Writer's rkBuf byte
1354 : // slice. Any byte written to the returned slice is retained for the lifetime of
1355 : // the Writer.
1356 0 : func (w *Writer) tempRangeKeyBuf(n int) []byte {
1357 0 : if cap(w.rkBuf)-len(w.rkBuf) < n {
1358 0 : size := len(w.rkBuf) + 2*n
1359 0 : if size < 2*cap(w.rkBuf) {
1360 0 : size = 2 * cap(w.rkBuf)
1361 0 : }
1362 0 : buf := make([]byte, len(w.rkBuf), size)
1363 0 : copy(buf, w.rkBuf)
1364 0 : w.rkBuf = buf
1365 : }
1366 0 : b := w.rkBuf[len(w.rkBuf) : len(w.rkBuf)+n]
1367 0 : w.rkBuf = w.rkBuf[:len(w.rkBuf)+n]
1368 0 : return b
1369 : }
1370 :
1371 : // tempRangeKeyCopy returns a copy of the provided slice, stored in the Writer's
1372 : // range key buffer.
1373 0 : func (w *Writer) tempRangeKeyCopy(k []byte) []byte {
1374 0 : if len(k) == 0 {
1375 0 : return nil
1376 0 : }
1377 0 : buf := w.tempRangeKeyBuf(len(k))
1378 0 : copy(buf, k)
1379 0 : return buf
1380 : }
1381 :
1382 1 : func (w *Writer) maybeAddToFilter(key []byte) {
1383 1 : if w.filter != nil {
1384 1 : prefix := key[:w.split(key)]
1385 1 : w.filter.addKey(prefix)
1386 1 : }
1387 : }
1388 :
1389 1 : func (w *Writer) flush(key InternalKey) error {
1390 1 : // We're finishing a data block.
1391 1 : err := w.finishDataBlockProps(w.dataBlockBuf)
1392 1 : if err != nil {
1393 0 : return err
1394 0 : }
1395 1 : w.dataBlockBuf.finish()
1396 1 : w.dataBlockBuf.compressAndChecksum(w.compression)
1397 1 : // Since dataBlockEstimates.addInflightDataBlock was never called, the
1398 1 : // inflightSize is set to 0.
1399 1 : w.coordination.sizeEstimate.dataBlockCompressed(len(w.dataBlockBuf.compressed), 0)
1400 1 :
1401 1 : // Determine if the index block should be flushed. Since we're accessing the
1402 1 : // dataBlockBuf.dataBlock.curKey here, we have to make sure that once we start
1403 1 : // to pool the dataBlockBufs, the curKey isn't used by the Writer once the
1404 1 : // dataBlockBuf is added back to a sync.Pool. In this particular case, the
1405 1 : // byte slice which supports "sep" will eventually be copied when "sep" is
1406 1 : // added to the index block.
1407 1 : prevKey := w.dataBlockBuf.dataBlock.getCurKey()
1408 1 : sep := w.indexEntrySep(prevKey, key, w.dataBlockBuf)
1409 1 : // We determine that we should flush an index block from the Writer client
1410 1 : // goroutine, but we actually finish the index block from the writeQueue.
1411 1 : // When we determine that an index block should be flushed, we need to call
1412 1 : // BlockPropertyCollector.FinishIndexBlock. But block property collector
1413 1 : // calls must happen sequentially from the Writer client. Therefore, we need
1414 1 : // to determine that we are going to flush the index block from the Writer
1415 1 : // client.
1416 1 : shouldFlushIndexBlock := supportsTwoLevelIndex(w.tableFormat) && w.indexBlock.shouldFlush(
1417 1 : sep, encodedBHPEstimatedSize, w.indexBlockOptions, w.allocatorSizeClasses,
1418 1 : )
1419 1 :
1420 1 : var indexProps []byte
1421 1 : var flushableIndexBlock *indexBlockBuf
1422 1 : if shouldFlushIndexBlock {
1423 1 : flushableIndexBlock = w.indexBlock
1424 1 : w.indexBlock = newIndexBlockBuf(w.coordination.parallelismEnabled)
1425 1 : // Call BlockPropertyCollector.FinishIndexBlock, since we've decided to
1426 1 : // flush the index block.
1427 1 : indexProps, err = w.finishIndexBlockProps()
1428 1 : if err != nil {
1429 0 : return err
1430 0 : }
1431 : }
1432 :
1433 : // We've called BlockPropertyCollector.FinishDataBlock, and, if necessary,
1434 : // BlockPropertyCollector.FinishIndexBlock. Since we've decided to finish
1435 : // the data block, we can call
1436 : // BlockPropertyCollector.AddPrevDataBlockToIndexBlock.
1437 1 : w.addPrevDataBlockToIndexBlockProps()
1438 1 :
1439 1 : // Schedule a write.
1440 1 : writeTask := writeTaskPool.Get().(*writeTask)
1441 1 : // We're setting compressionDone to indicate that compression of this block
1442 1 : // has already been completed.
1443 1 : writeTask.compressionDone <- true
1444 1 : writeTask.buf = w.dataBlockBuf
1445 1 : writeTask.indexEntrySep = sep
1446 1 : writeTask.currIndexBlock = w.indexBlock
1447 1 : writeTask.indexInflightSize = sep.Size() + encodedBHPEstimatedSize
1448 1 : writeTask.finishedIndexProps = indexProps
1449 1 : writeTask.flushableIndexBlock = flushableIndexBlock
1450 1 :
1451 1 : // The writeTask corresponds to an unwritten index entry.
1452 1 : w.indexBlock.addInflight(writeTask.indexInflightSize)
1453 1 :
1454 1 : w.dataBlockBuf = nil
1455 1 : if w.coordination.parallelismEnabled {
1456 1 : w.coordination.writeQueue.add(writeTask)
1457 1 : } else {
1458 1 : err = w.coordination.writeQueue.addSync(writeTask)
1459 1 : }
1460 1 : w.dataBlockBuf = newDataBlockBuf(w.restartInterval, w.checksumType)
1461 1 :
1462 1 : return err
1463 : }
1464 :
1465 1 : func (w *Writer) maybeFlush(key InternalKey, valueLen int) error {
1466 1 : if !w.dataBlockBuf.shouldFlush(key, valueLen, w.dataBlockOptions, w.allocatorSizeClasses) {
1467 1 : return nil
1468 1 : }
1469 :
1470 1 : err := w.flush(key)
1471 1 :
1472 1 : if err != nil {
1473 0 : w.err = err
1474 0 : return err
1475 0 : }
1476 :
1477 1 : return nil
1478 : }
1479 :
1480 : // dataBlockBuf.dataBlockProps set by this method must be encoded before any future use of the
1481 : // dataBlockBuf.blockPropsEncoder, since the properties slice will get reused by the
1482 : // blockPropsEncoder.
1483 1 : func (w *Writer) finishDataBlockProps(buf *dataBlockBuf) error {
1484 1 : if len(w.blockPropCollectors) == 0 {
1485 1 : return nil
1486 1 : }
1487 1 : var err error
1488 1 : buf.blockPropsEncoder.resetProps()
1489 1 : for i := range w.blockPropCollectors {
1490 1 : scratch := buf.blockPropsEncoder.getScratchForProp()
1491 1 : if scratch, err = w.blockPropCollectors[i].FinishDataBlock(scratch); err != nil {
1492 0 : return err
1493 0 : }
1494 1 : buf.blockPropsEncoder.addProp(shortID(i), scratch)
1495 : }
1496 :
1497 1 : buf.dataBlockProps = buf.blockPropsEncoder.unsafeProps()
1498 1 : return nil
1499 : }
1500 :
1501 : // The BlockHandleWithProperties returned by this method must be encoded before any future use of
1502 : // the Writer.blockPropsEncoder, since the properties slice will get reused by the blockPropsEncoder.
1503 : // maybeAddBlockPropertiesToBlockHandle should only be called if block is being written synchronously
1504 : // with the Writer client.
1505 : func (w *Writer) maybeAddBlockPropertiesToBlockHandle(
1506 : bh BlockHandle,
1507 1 : ) (BlockHandleWithProperties, error) {
1508 1 : err := w.finishDataBlockProps(w.dataBlockBuf)
1509 1 : if err != nil {
1510 0 : return BlockHandleWithProperties{}, err
1511 0 : }
1512 1 : return BlockHandleWithProperties{BlockHandle: bh, Props: w.dataBlockBuf.dataBlockProps}, nil
1513 : }
1514 :
1515 1 : func (w *Writer) indexEntrySep(prevKey, key InternalKey, dataBlockBuf *dataBlockBuf) InternalKey {
1516 1 : // Make a rough guess that we want key-sized scratch to compute the separator.
1517 1 : if cap(dataBlockBuf.sepScratch) < key.Size() {
1518 1 : dataBlockBuf.sepScratch = make([]byte, 0, key.Size()*2)
1519 1 : }
1520 :
1521 1 : var sep InternalKey
1522 1 : if key.UserKey == nil && key.Trailer == 0 {
1523 1 : sep = prevKey.Successor(w.compare, w.successor, dataBlockBuf.sepScratch[:0])
1524 1 : } else {
1525 1 : sep = prevKey.Separator(w.compare, w.separator, dataBlockBuf.sepScratch[:0], key)
1526 1 : }
1527 1 : return sep
1528 : }
1529 :
1530 : // addIndexEntry adds an index entry for the specified key and block handle.
1531 : // addIndexEntry can be called from both the Writer client goroutine, and the
1532 : // writeQueue goroutine. If the flushIndexBuf != nil, then the indexProps, as
1533 : // they're used when the index block is finished.
1534 : //
1535 : // Invariant:
1536 : // 1. addIndexEntry must not store references to the sep InternalKey, the tmp
1537 : // byte slice, bhp.Props. That is, these must be either deep copied or
1538 : // encoded.
1539 : // 2. addIndexEntry must not hold references to the flushIndexBuf, and the writeTo
1540 : // indexBlockBufs.
1541 : func (w *Writer) addIndexEntry(
1542 : sep InternalKey,
1543 : bhp BlockHandleWithProperties,
1544 : tmp []byte,
1545 : flushIndexBuf *indexBlockBuf,
1546 : writeTo *indexBlockBuf,
1547 : inflightSize int,
1548 : indexProps []byte,
1549 1 : ) error {
1550 1 : if bhp.Length == 0 {
1551 0 : // A valid blockHandle must be non-zero.
1552 0 : // In particular, it must have a non-zero length.
1553 0 : return nil
1554 0 : }
1555 :
1556 1 : encoded := encodeBlockHandleWithProperties(tmp, bhp)
1557 1 :
1558 1 : if flushIndexBuf != nil {
1559 1 : if cap(w.indexPartitions) == 0 {
1560 1 : w.indexPartitions = make([]indexBlockAndBlockProperties, 0, 32)
1561 1 : }
1562 : // Enable two level indexes if there is more than one index block.
1563 1 : w.twoLevelIndex = true
1564 1 : if err := w.finishIndexBlock(flushIndexBuf, indexProps); err != nil {
1565 0 : return err
1566 0 : }
1567 : }
1568 :
1569 1 : writeTo.add(sep, encoded, inflightSize)
1570 1 : return nil
1571 : }
1572 :
1573 1 : func (w *Writer) addPrevDataBlockToIndexBlockProps() {
1574 1 : for i := range w.blockPropCollectors {
1575 1 : w.blockPropCollectors[i].AddPrevDataBlockToIndexBlock()
1576 1 : }
1577 : }
1578 :
1579 : // addIndexEntrySync adds an index entry for the specified key and block handle.
1580 : // Writer.addIndexEntry is only called synchronously once Writer.Close is called.
1581 : // addIndexEntrySync should only be called if we're sure that index entries
1582 : // aren't being written asynchronously.
1583 : //
1584 : // Invariant:
1585 : // 1. addIndexEntrySync must not store references to the prevKey, key InternalKey's,
1586 : // the tmp byte slice. That is, these must be either deep copied or encoded.
1587 : //
1588 : // TODO: Improve coverage of this method. e.g. tests passed without the line
1589 : // `w.twoLevelIndex = true` previously.
1590 : func (w *Writer) addIndexEntrySync(
1591 : prevKey, key InternalKey, bhp BlockHandleWithProperties, tmp []byte,
1592 1 : ) error {
1593 1 : return w.addIndexEntrySep(w.indexEntrySep(prevKey, key, w.dataBlockBuf), bhp, tmp)
1594 1 : }
1595 :
1596 : func (w *Writer) addIndexEntrySep(
1597 : sep InternalKey, bhp BlockHandleWithProperties, tmp []byte,
1598 1 : ) error {
1599 1 : shouldFlush := supportsTwoLevelIndex(
1600 1 : w.tableFormat) && w.indexBlock.shouldFlush(
1601 1 : sep, encodedBHPEstimatedSize, w.indexBlockOptions, w.allocatorSizeClasses,
1602 1 : )
1603 1 : var flushableIndexBlock *indexBlockBuf
1604 1 : var props []byte
1605 1 : var err error
1606 1 : if shouldFlush {
1607 1 : flushableIndexBlock = w.indexBlock
1608 1 : w.indexBlock = newIndexBlockBuf(w.coordination.parallelismEnabled)
1609 1 : w.twoLevelIndex = true
1610 1 : // Call BlockPropertyCollector.FinishIndexBlock, since we've decided to
1611 1 : // flush the index block.
1612 1 : props, err = w.finishIndexBlockProps()
1613 1 : if err != nil {
1614 0 : return err
1615 0 : }
1616 : }
1617 :
1618 1 : err = w.addIndexEntry(sep, bhp, tmp, flushableIndexBlock, w.indexBlock, 0, props)
1619 1 : if flushableIndexBlock != nil {
1620 1 : flushableIndexBlock.clear()
1621 1 : indexBlockBufPool.Put(flushableIndexBlock)
1622 1 : }
1623 1 : w.addPrevDataBlockToIndexBlockProps()
1624 1 : return err
1625 : }
1626 :
1627 : func shouldFlushWithHints(
1628 : keyLen, valueLen int,
1629 : restartInterval, estimatedBlockSize, numEntries int,
1630 : flushOptions flushDecisionOptions,
1631 : sizeClassHints []int,
1632 1 : ) bool {
1633 1 : if numEntries == 0 {
1634 1 : return false
1635 1 : }
1636 :
1637 : // If we are not informed about the memory allocator's size classes we fall
1638 : // back to a simple set of flush heuristics that are unaware of internal
1639 : // fragmentation in block cache allocations.
1640 1 : if len(sizeClassHints) == 0 {
1641 1 : return shouldFlushWithoutHints(
1642 1 : keyLen, valueLen, restartInterval, estimatedBlockSize, numEntries, flushOptions)
1643 1 : }
1644 :
1645 : // For size-class aware flushing we need to account for the metadata that is
1646 : // allocated when this block is loaded into the block cache. For instance, if
1647 : // a block has size 1020B it may fit within a 1024B class. However, when
1648 : // loaded into the block cache we also allocate space for the cache entry
1649 : // metadata. The new allocation of size ~1052B may now only fit within a
1650 : // 2048B class, which increases internal fragmentation.
1651 0 : blockSizeWithMetadata := estimatedBlockSize + cache.ValueMetadataSize
1652 0 :
1653 0 : // For the fast path we can avoid computing the exact varint encoded
1654 0 : // key-value pair size. Instead, we combine the key-value pair size with an
1655 0 : // upper-bound estimate of the associated metadata (4B restart point, 4B
1656 0 : // shared prefix length, 5B varint unshared key size, 5B varint value size).
1657 0 : newEstimatedSize := blockSizeWithMetadata + keyLen + valueLen + 18
1658 0 : // Our new block size estimate disregards key prefix compression. This puts
1659 0 : // us at risk of overestimating the size and flushing small blocks. We
1660 0 : // mitigate this by imposing a minimum size restriction.
1661 0 : if blockSizeWithMetadata <= flushOptions.sizeClassAwareThreshold || newEstimatedSize <= flushOptions.blockSize {
1662 0 : return false
1663 0 : }
1664 :
1665 0 : sizeClass, ok := blockSizeClass(blockSizeWithMetadata, sizeClassHints)
1666 0 : // If the block size could not be mapped to a size class we fall back to
1667 0 : // using a simpler set of flush heuristics.
1668 0 : if !ok {
1669 0 : return shouldFlushWithoutHints(
1670 0 : keyLen, valueLen, restartInterval, estimatedBlockSize, numEntries, flushOptions)
1671 0 : }
1672 :
1673 : // Tighter upper-bound estimate of the metadata stored with the next
1674 : // key-value pair.
1675 0 : newSize := blockSizeWithMetadata + keyLen + valueLen
1676 0 : if numEntries%restartInterval == 0 {
1677 0 : newSize += 4
1678 0 : }
1679 0 : newSize += 4 // varint for shared prefix length
1680 0 : newSize += uvarintLen(uint32(keyLen)) // varint for unshared key bytes
1681 0 : newSize += uvarintLen(uint32(valueLen)) // varint for value size
1682 0 :
1683 0 : if blockSizeWithMetadata < flushOptions.blockSize {
1684 0 : newSizeClass, ok := blockSizeClass(newSize, sizeClassHints)
1685 0 : if ok && newSizeClass-newSize >= sizeClass-blockSizeWithMetadata {
1686 0 : // Although the block hasn't reached the target size, waiting to insert the
1687 0 : // next entry would exceed the target and increase memory fragmentation.
1688 0 : return true
1689 0 : }
1690 0 : return false
1691 : }
1692 :
1693 : // Flush if inserting the next entry bumps the block size to the memory
1694 : // allocator's next size class.
1695 0 : return newSize > sizeClass
1696 : }
1697 :
1698 : func shouldFlushWithoutHints(
1699 : keyLen, valueLen int,
1700 : restartInterval, estimatedBlockSize, numEntries int,
1701 : flushOptions flushDecisionOptions,
1702 1 : ) bool {
1703 1 : if estimatedBlockSize >= flushOptions.blockSize {
1704 1 : return true
1705 1 : }
1706 :
1707 : // The block is currently smaller than the target size.
1708 1 : if estimatedBlockSize <= flushOptions.blockSizeThreshold {
1709 1 : // The block is smaller than the threshold size at which we'll consider
1710 1 : // flushing it.
1711 1 : return false
1712 1 : }
1713 :
1714 1 : newSize := estimatedBlockSize + keyLen + valueLen
1715 1 : if numEntries%restartInterval == 0 {
1716 1 : newSize += 4
1717 1 : }
1718 1 : newSize += 4 // varint for shared prefix length
1719 1 : newSize += uvarintLen(uint32(keyLen)) // varint for unshared key bytes
1720 1 : newSize += uvarintLen(uint32(valueLen)) // varint for value size
1721 1 : // Flush if the block plus the new entry is larger than the target size.
1722 1 : return newSize > flushOptions.blockSize
1723 : }
1724 :
1725 : // blockSizeClass returns the smallest memory allocator size class that could
1726 : // hold a block of a given size and returns a boolean indicating whether an
1727 : // appropriate size class was found. It is useful for computing the potential
1728 : // space wasted by an allocation.
1729 0 : func blockSizeClass(blockSize int, sizeClassHints []int) (int, bool) {
1730 0 : sizeClassIdx, _ := slices.BinarySearch(sizeClassHints, blockSize)
1731 0 : if sizeClassIdx == len(sizeClassHints) {
1732 0 : return -1, false
1733 0 : }
1734 0 : return sizeClassHints[sizeClassIdx], true
1735 : }
1736 :
1737 1 : func cloneKeyWithBuf(k InternalKey, a bytealloc.A) (bytealloc.A, InternalKey) {
1738 1 : if len(k.UserKey) == 0 {
1739 0 : return a, k
1740 0 : }
1741 1 : a, keyCopy := a.Copy(k.UserKey)
1742 1 : return a, InternalKey{UserKey: keyCopy, Trailer: k.Trailer}
1743 : }
1744 :
1745 : // Invariants: The byte slice returned by finishIndexBlockProps is heap-allocated
1746 : //
1747 : // and has its own lifetime, independent of the Writer and the blockPropsEncoder,
1748 : //
1749 : // and it is safe to:
1750 : // 1. Reuse w.blockPropsEncoder without first encoding the byte slice returned.
1751 : // 2. Store the byte slice in the Writer since it is a copy and not supported by
1752 : // an underlying buffer.
1753 1 : func (w *Writer) finishIndexBlockProps() ([]byte, error) {
1754 1 : w.blockPropsEncoder.resetProps()
1755 1 : for i := range w.blockPropCollectors {
1756 1 : scratch := w.blockPropsEncoder.getScratchForProp()
1757 1 : var err error
1758 1 : if scratch, err = w.blockPropCollectors[i].FinishIndexBlock(scratch); err != nil {
1759 0 : return nil, err
1760 0 : }
1761 1 : w.blockPropsEncoder.addProp(shortID(i), scratch)
1762 : }
1763 1 : return w.blockPropsEncoder.props(), nil
1764 : }
1765 :
1766 : // finishIndexBlock finishes the current index block and adds it to the top
1767 : // level index block. This is only used when two level indexes are enabled.
1768 : //
1769 : // Invariants:
1770 : // 1. The props slice passed into finishedIndexBlock must not be a
1771 : // owned by any other struct, since it will be stored in the Writer.indexPartitions
1772 : // slice.
1773 : // 2. None of the buffers owned by indexBuf will be shallow copied and stored elsewhere.
1774 : // That is, it must be safe to reuse indexBuf after finishIndexBlock has been called.
1775 1 : func (w *Writer) finishIndexBlock(indexBuf *indexBlockBuf, props []byte) error {
1776 1 : part := indexBlockAndBlockProperties{
1777 1 : nEntries: indexBuf.block.nEntries, properties: props,
1778 1 : }
1779 1 : w.indexSepAlloc, part.sep = cloneKeyWithBuf(
1780 1 : indexBuf.block.getCurKey(), w.indexSepAlloc,
1781 1 : )
1782 1 : bk := indexBuf.finish()
1783 1 : if len(w.indexBlockAlloc) < len(bk) {
1784 1 : // Allocate enough bytes for approximately 16 index blocks.
1785 1 : w.indexBlockAlloc = make([]byte, len(bk)*16)
1786 1 : }
1787 1 : n := copy(w.indexBlockAlloc, bk)
1788 1 : part.block = w.indexBlockAlloc[:n:n]
1789 1 : w.indexBlockAlloc = w.indexBlockAlloc[n:]
1790 1 : w.indexPartitions = append(w.indexPartitions, part)
1791 1 : return nil
1792 : }
1793 :
1794 1 : func (w *Writer) writeTwoLevelIndex() (BlockHandle, error) {
1795 1 : props, err := w.finishIndexBlockProps()
1796 1 : if err != nil {
1797 0 : return BlockHandle{}, err
1798 0 : }
1799 : // Add the final unfinished index.
1800 1 : if err = w.finishIndexBlock(w.indexBlock, props); err != nil {
1801 0 : return BlockHandle{}, err
1802 0 : }
1803 :
1804 1 : for i := range w.indexPartitions {
1805 1 : b := &w.indexPartitions[i]
1806 1 : w.props.NumDataBlocks += uint64(b.nEntries)
1807 1 :
1808 1 : data := b.block
1809 1 : w.props.IndexSize += uint64(len(data))
1810 1 : bh, err := w.writeBlock(data, w.compression, &w.blockBuf)
1811 1 : if err != nil {
1812 0 : return BlockHandle{}, err
1813 0 : }
1814 1 : bhp := BlockHandleWithProperties{
1815 1 : BlockHandle: bh,
1816 1 : Props: b.properties,
1817 1 : }
1818 1 : encoded := encodeBlockHandleWithProperties(w.blockBuf.tmp[:], bhp)
1819 1 : w.topLevelIndexBlock.add(b.sep, encoded)
1820 : }
1821 :
1822 : // NB: RocksDB includes the block trailer length in the index size
1823 : // property, though it doesn't include the trailer in the top level
1824 : // index size property.
1825 1 : w.props.IndexPartitions = uint64(len(w.indexPartitions))
1826 1 : w.props.TopLevelIndexSize = uint64(w.topLevelIndexBlock.estimatedSize())
1827 1 : w.props.IndexSize += w.props.TopLevelIndexSize + blockTrailerLen
1828 1 :
1829 1 : return w.writeBlock(w.topLevelIndexBlock.finish(), w.compression, &w.blockBuf)
1830 : }
1831 :
1832 1 : func compressAndChecksum(b []byte, compression Compression, blockBuf *blockBuf) []byte {
1833 1 : // Compress the buffer, discarding the result if the improvement isn't at
1834 1 : // least 12.5%.
1835 1 : blockType, compressed := compressBlock(compression, b, blockBuf.compressedBuf)
1836 1 : if blockType != noCompressionBlockType && cap(compressed) > cap(blockBuf.compressedBuf) {
1837 1 : blockBuf.compressedBuf = compressed[:cap(compressed)]
1838 1 : }
1839 1 : if len(compressed) < len(b)-len(b)/8 {
1840 1 : b = compressed
1841 1 : } else {
1842 1 : blockType = noCompressionBlockType
1843 1 : }
1844 :
1845 1 : blockBuf.tmp[0] = byte(blockType)
1846 1 :
1847 1 : // Calculate the checksum.
1848 1 : checksum := blockBuf.checksummer.checksum(b, blockBuf.tmp[:1])
1849 1 : binary.LittleEndian.PutUint32(blockBuf.tmp[1:5], checksum)
1850 1 : return b
1851 : }
1852 :
1853 1 : func (w *Writer) writeCompressedBlock(block []byte, blockTrailerBuf []byte) (BlockHandle, error) {
1854 1 : bh := BlockHandle{Offset: w.meta.Size, Length: uint64(len(block))}
1855 1 :
1856 1 : if w.cacheID != 0 && w.fileNum != 0 {
1857 1 : // Remove the block being written from the cache. This provides defense in
1858 1 : // depth against bugs which cause cache collisions.
1859 1 : //
1860 1 : // TODO(peter): Alternatively, we could add the uncompressed value to the
1861 1 : // cache.
1862 1 : w.cache.Delete(w.cacheID, w.fileNum, bh.Offset)
1863 1 : }
1864 :
1865 : // Write the bytes to the file.
1866 1 : if err := w.writable.Write(block); err != nil {
1867 0 : return BlockHandle{}, err
1868 0 : }
1869 1 : w.meta.Size += uint64(len(block))
1870 1 : if err := w.writable.Write(blockTrailerBuf[:blockTrailerLen]); err != nil {
1871 0 : return BlockHandle{}, err
1872 0 : }
1873 1 : w.meta.Size += blockTrailerLen
1874 1 :
1875 1 : return bh, nil
1876 : }
1877 :
1878 : // Write implements io.Writer. This is analogous to writeCompressedBlock for
1879 : // blocks that already incorporate the trailer, and don't need the callee to
1880 : // return a BlockHandle.
1881 1 : func (w *Writer) Write(blockWithTrailer []byte) (n int, err error) {
1882 1 : offset := w.meta.Size
1883 1 : if w.cacheID != 0 && w.fileNum != 0 {
1884 1 : // Remove the block being written from the cache. This provides defense in
1885 1 : // depth against bugs which cause cache collisions.
1886 1 : //
1887 1 : // TODO(peter): Alternatively, we could add the uncompressed value to the
1888 1 : // cache.
1889 1 : w.cache.Delete(w.cacheID, w.fileNum, offset)
1890 1 : }
1891 1 : w.meta.Size += uint64(len(blockWithTrailer))
1892 1 : if err := w.writable.Write(blockWithTrailer); err != nil {
1893 0 : return 0, err
1894 0 : }
1895 1 : return len(blockWithTrailer), nil
1896 : }
1897 :
1898 : func (w *Writer) writeBlock(
1899 : b []byte, compression Compression, blockBuf *blockBuf,
1900 1 : ) (BlockHandle, error) {
1901 1 : b = compressAndChecksum(b, compression, blockBuf)
1902 1 : return w.writeCompressedBlock(b, blockBuf.tmp[:])
1903 1 : }
1904 :
1905 : // assertFormatCompatibility ensures that the features present on the table are
1906 : // compatible with the table format version.
1907 1 : func (w *Writer) assertFormatCompatibility() error {
1908 1 : // PebbleDBv1: block properties.
1909 1 : if len(w.blockPropCollectors) > 0 && w.tableFormat < TableFormatPebblev1 {
1910 0 : return errors.Newf(
1911 0 : "table format version %s is less than the minimum required version %s for block properties",
1912 0 : w.tableFormat, TableFormatPebblev1,
1913 0 : )
1914 0 : }
1915 :
1916 : // PebbleDBv2: range keys.
1917 1 : if w.props.NumRangeKeys() > 0 && w.tableFormat < TableFormatPebblev2 {
1918 0 : return errors.Newf(
1919 0 : "table format version %s is less than the minimum required version %s for range keys",
1920 0 : w.tableFormat, TableFormatPebblev2,
1921 0 : )
1922 0 : }
1923 :
1924 : // PebbleDBv3: value blocks.
1925 1 : if (w.props.NumValueBlocks > 0 || w.props.NumValuesInValueBlocks > 0 ||
1926 1 : w.props.ValueBlocksSize > 0) && w.tableFormat < TableFormatPebblev3 {
1927 0 : return errors.Newf(
1928 0 : "table format version %s is less than the minimum required version %s for value blocks",
1929 0 : w.tableFormat, TableFormatPebblev3)
1930 0 : }
1931 :
1932 : // PebbleDBv4: DELSIZED tombstones.
1933 1 : if w.props.NumSizedDeletions > 0 && w.tableFormat < TableFormatPebblev4 {
1934 0 : return errors.Newf(
1935 0 : "table format version %s is less than the minimum required version %s for sized deletion tombstones",
1936 0 : w.tableFormat, TableFormatPebblev4)
1937 0 : }
1938 1 : return nil
1939 : }
1940 :
1941 : // UnsafeLastPointUserKey returns the last point key written to the writer to
1942 : // which this option was passed during creation. The returned key points
1943 : // directly into a buffer belonging to the Writer. The value's lifetime ends the
1944 : // next time a point key is added to the Writer.
1945 : //
1946 : // Must not be called after Writer is closed.
1947 1 : func (w *Writer) UnsafeLastPointUserKey() []byte {
1948 1 : if w != nil && w.dataBlockBuf.dataBlock.nEntries >= 1 {
1949 1 : // w.dataBlockBuf.dataBlock.curKey is guaranteed to point to the last point key
1950 1 : // which was added to the Writer.
1951 1 : return w.dataBlockBuf.dataBlock.getCurUserKey()
1952 1 : }
1953 0 : return nil
1954 : }
1955 :
1956 : // EncodeSpan encodes the keys in the given span. The span can contain either
1957 : // only RANGEDEL keys or only range keys.
1958 1 : func (w *Writer) EncodeSpan(span *keyspan.Span) error {
1959 1 : if span.Empty() {
1960 1 : return nil
1961 1 : }
1962 1 : if span.Keys[0].Kind() == base.InternalKeyKindRangeDelete {
1963 1 : return rangedel.Encode(span, w.Add)
1964 1 : }
1965 1 : return rangekey.Encode(span, w.AddRangeKey)
1966 : }
1967 :
1968 : // Close finishes writing the table and closes the underlying file that the
1969 : // table was written to.
1970 1 : func (w *Writer) Close() (err error) {
1971 1 : defer func() {
1972 1 : if w.valueBlockWriter != nil {
1973 1 : releaseValueBlockWriter(w.valueBlockWriter)
1974 1 : // Defensive code in case Close gets called again. We don't want to put
1975 1 : // the same object to a sync.Pool.
1976 1 : w.valueBlockWriter = nil
1977 1 : }
1978 1 : if w.writable != nil {
1979 0 : w.writable.Abort()
1980 0 : w.writable = nil
1981 0 : }
1982 : // Record any error in the writer (so we can exit early if Close is called
1983 : // again).
1984 1 : if err != nil {
1985 0 : w.err = err
1986 0 : }
1987 : }()
1988 :
1989 : // finish must be called before we check for an error, because finish will
1990 : // block until every single task added to the writeQueue has been processed,
1991 : // and an error could be encountered while any of those tasks are processed.
1992 1 : if err := w.coordination.writeQueue.finish(); err != nil {
1993 0 : return err
1994 0 : }
1995 :
1996 1 : if w.err != nil {
1997 0 : return w.err
1998 0 : }
1999 :
2000 : // The w.meta.LargestPointKey is only used once the Writer is closed, so it is safe to set it
2001 : // when the Writer is closed.
2002 : //
2003 : // The following invariants ensure that setting the largest key at this point of a Writer close
2004 : // is correct:
2005 : // 1. Keys must only be added to the Writer in an increasing order.
2006 : // 2. The current w.dataBlockBuf is guaranteed to have the latest key added to the Writer. This
2007 : // must be true, because a w.dataBlockBuf is only switched out when a dataBlock is flushed,
2008 : // however, if a dataBlock is flushed, then we add a key to the new w.dataBlockBuf in the
2009 : // addPoint function after the flush occurs.
2010 1 : if w.dataBlockBuf.dataBlock.nEntries >= 1 {
2011 1 : w.meta.SetLargestPointKey(w.dataBlockBuf.dataBlock.getCurKey().Clone())
2012 1 : }
2013 :
2014 : // Finish the last data block, or force an empty data block if there
2015 : // aren't any data blocks at all.
2016 1 : if w.dataBlockBuf.dataBlock.nEntries > 0 || w.indexBlock.block.nEntries == 0 {
2017 1 : bh, err := w.writeBlock(w.dataBlockBuf.dataBlock.finish(), w.compression, &w.dataBlockBuf.blockBuf)
2018 1 : if err != nil {
2019 0 : return err
2020 0 : }
2021 1 : bhp, err := w.maybeAddBlockPropertiesToBlockHandle(bh)
2022 1 : if err != nil {
2023 0 : return err
2024 0 : }
2025 1 : prevKey := w.dataBlockBuf.dataBlock.getCurKey()
2026 1 : if err := w.addIndexEntrySync(prevKey, InternalKey{}, bhp, w.dataBlockBuf.tmp[:]); err != nil {
2027 0 : return err
2028 0 : }
2029 : }
2030 1 : w.props.DataSize = w.meta.Size
2031 1 :
2032 1 : // Write the filter block.
2033 1 : var metaindex rawBlockWriter
2034 1 : metaindex.restartInterval = 1
2035 1 : if w.filter != nil {
2036 1 : b, err := w.filter.finish()
2037 1 : if err != nil {
2038 0 : return err
2039 0 : }
2040 1 : bh, err := w.writeBlock(b, NoCompression, &w.blockBuf)
2041 1 : if err != nil {
2042 0 : return err
2043 0 : }
2044 1 : n := encodeBlockHandle(w.blockBuf.tmp[:], bh)
2045 1 : metaindex.add(InternalKey{UserKey: []byte(w.filter.metaName())}, w.blockBuf.tmp[:n])
2046 1 : w.props.FilterPolicyName = w.filter.policyName()
2047 1 : w.props.FilterSize = bh.Length
2048 : }
2049 :
2050 1 : var indexBH BlockHandle
2051 1 : if w.twoLevelIndex {
2052 1 : w.props.IndexType = twoLevelIndex
2053 1 : // Write the two level index block.
2054 1 : indexBH, err = w.writeTwoLevelIndex()
2055 1 : if err != nil {
2056 0 : return err
2057 0 : }
2058 1 : } else {
2059 1 : w.props.IndexType = binarySearchIndex
2060 1 : // NB: RocksDB includes the block trailer length in the index size
2061 1 : // property, though it doesn't include the trailer in the filter size
2062 1 : // property.
2063 1 : w.props.IndexSize = uint64(w.indexBlock.estimatedSize()) + blockTrailerLen
2064 1 : w.props.NumDataBlocks = uint64(w.indexBlock.block.nEntries)
2065 1 :
2066 1 : // Write the single level index block.
2067 1 : indexBH, err = w.writeBlock(w.indexBlock.finish(), w.compression, &w.blockBuf)
2068 1 : if err != nil {
2069 0 : return err
2070 0 : }
2071 : }
2072 :
2073 : // Write the range-del block. The block handle must added to the meta index block
2074 : // after the properties block has been written. This is because the entries in the
2075 : // metaindex block must be sorted by key.
2076 1 : var rangeDelBH BlockHandle
2077 1 : if w.props.NumRangeDeletions > 0 {
2078 1 : if !w.rangeDelV1Format {
2079 1 : // Because the range tombstones are fragmented in the v2 format, the end
2080 1 : // key of the last added range tombstone will be the largest range
2081 1 : // tombstone key. Note that we need to make this into a range deletion
2082 1 : // sentinel because sstable boundaries are inclusive while the end key of
2083 1 : // a range deletion tombstone is exclusive. A Clone() is necessary as
2084 1 : // rangeDelBlock.curValue is the same slice that will get passed
2085 1 : // into w.writer, and some implementations of vfs.File mutate the
2086 1 : // slice passed into Write(). Also, w.meta will often outlive the
2087 1 : // blockWriter, and so cloning curValue allows the rangeDelBlock's
2088 1 : // internal buffer to get gc'd.
2089 1 : k := base.MakeRangeDeleteSentinelKey(w.rangeDelBlock.curValue).Clone()
2090 1 : w.meta.SetLargestRangeDelKey(k)
2091 1 : }
2092 1 : rangeDelBH, err = w.writeBlock(w.rangeDelBlock.finish(), NoCompression, &w.blockBuf)
2093 1 : if err != nil {
2094 0 : return err
2095 0 : }
2096 : }
2097 :
2098 : // Write the range-key block, flushing any remaining spans from the
2099 : // fragmenter first.
2100 1 : w.fragmenter.Finish()
2101 1 :
2102 1 : var rangeKeyBH BlockHandle
2103 1 : if w.props.NumRangeKeys() > 0 {
2104 1 : key := w.rangeKeyBlock.getCurKey()
2105 1 : kind := key.Kind()
2106 1 : endKey, _, ok := rangekey.DecodeEndKey(kind, w.rangeKeyBlock.curValue)
2107 1 : if !ok {
2108 0 : return errors.Newf("invalid end key: %s", w.rangeKeyBlock.curValue)
2109 0 : }
2110 1 : k := base.MakeExclusiveSentinelKey(kind, endKey).Clone()
2111 1 : w.meta.SetLargestRangeKey(k)
2112 1 : // TODO(travers): The lack of compression on the range key block matches the
2113 1 : // lack of compression on the range-del block. Revisit whether we want to
2114 1 : // enable compression on this block.
2115 1 : rangeKeyBH, err = w.writeBlock(w.rangeKeyBlock.finish(), NoCompression, &w.blockBuf)
2116 1 : if err != nil {
2117 0 : return err
2118 0 : }
2119 : }
2120 :
2121 1 : if w.valueBlockWriter != nil {
2122 1 : vbiHandle, vbStats, err := w.valueBlockWriter.finish(w, w.meta.Size)
2123 1 : if err != nil {
2124 0 : return err
2125 0 : }
2126 1 : w.props.NumValueBlocks = vbStats.numValueBlocks
2127 1 : w.props.NumValuesInValueBlocks = vbStats.numValuesInValueBlocks
2128 1 : w.props.ValueBlocksSize = vbStats.valueBlocksAndIndexSize
2129 1 : if vbStats.numValueBlocks > 0 {
2130 1 : n := encodeValueBlocksIndexHandle(w.blockBuf.tmp[:], vbiHandle)
2131 1 : metaindex.add(InternalKey{UserKey: []byte(metaValueIndexName)}, w.blockBuf.tmp[:n])
2132 1 : }
2133 : }
2134 :
2135 : // Add the range key block handle to the metaindex block. Note that we add the
2136 : // block handle to the metaindex block before the other meta blocks as the
2137 : // metaindex block entries must be sorted, and the range key block name sorts
2138 : // before the other block names.
2139 1 : if w.props.NumRangeKeys() > 0 {
2140 1 : n := encodeBlockHandle(w.blockBuf.tmp[:], rangeKeyBH)
2141 1 : metaindex.add(InternalKey{UserKey: []byte(metaRangeKeyName)}, w.blockBuf.tmp[:n])
2142 1 : }
2143 :
2144 1 : {
2145 1 : // Finish and record the prop collectors if props are not yet recorded.
2146 1 : // Pre-computed props might have been copied by specialized sst creators
2147 1 : // like suffix replacer.
2148 1 : if len(w.props.UserProperties) == 0 {
2149 1 : userProps := make(map[string]string)
2150 1 : for i := range w.blockPropCollectors {
2151 1 : scratch := w.blockPropsEncoder.getScratchForProp()
2152 1 : // Place the shortID in the first byte.
2153 1 : scratch = append(scratch, byte(i))
2154 1 : buf, err := w.blockPropCollectors[i].FinishTable(scratch)
2155 1 : if err != nil {
2156 0 : return err
2157 0 : }
2158 1 : var prop string
2159 1 : if len(buf) > 0 {
2160 1 : prop = string(buf)
2161 1 : }
2162 : // NB: The property is populated in the map even if it is the
2163 : // empty string, since the presence in the map is what indicates
2164 : // that the block property collector was used when writing.
2165 1 : userProps[w.blockPropCollectors[i].Name()] = prop
2166 : }
2167 1 : if len(userProps) > 0 {
2168 1 : w.props.UserProperties = userProps
2169 1 : }
2170 : }
2171 :
2172 : // Write the properties block.
2173 1 : var raw rawBlockWriter
2174 1 : // The restart interval is set to infinity because the properties block
2175 1 : // is always read sequentially and cached in a heap located object. This
2176 1 : // reduces table size without a significant impact on performance.
2177 1 : raw.restartInterval = propertiesBlockRestartInterval
2178 1 : w.props.CompressionOptions = rocksDBCompressionOptions
2179 1 : w.props.save(w.tableFormat, &raw)
2180 1 : bh, err := w.writeBlock(raw.finish(), NoCompression, &w.blockBuf)
2181 1 : if err != nil {
2182 0 : return err
2183 0 : }
2184 1 : n := encodeBlockHandle(w.blockBuf.tmp[:], bh)
2185 1 : metaindex.add(InternalKey{UserKey: []byte(metaPropertiesName)}, w.blockBuf.tmp[:n])
2186 : }
2187 :
2188 : // Add the range deletion block handle to the metaindex block.
2189 1 : if w.props.NumRangeDeletions > 0 {
2190 1 : n := encodeBlockHandle(w.blockBuf.tmp[:], rangeDelBH)
2191 1 : // The v2 range-del block encoding is backwards compatible with the v1
2192 1 : // encoding. We add meta-index entries for both the old name and the new
2193 1 : // name so that old code can continue to find the range-del block and new
2194 1 : // code knows that the range tombstones in the block are fragmented and
2195 1 : // sorted.
2196 1 : metaindex.add(InternalKey{UserKey: []byte(metaRangeDelName)}, w.blockBuf.tmp[:n])
2197 1 : if !w.rangeDelV1Format {
2198 1 : metaindex.add(InternalKey{UserKey: []byte(metaRangeDelV2Name)}, w.blockBuf.tmp[:n])
2199 1 : }
2200 : }
2201 :
2202 : // Write the metaindex block. It might be an empty block, if the filter
2203 : // policy is nil. NoCompression is specified because a) RocksDB never
2204 : // compresses the meta-index block and b) RocksDB has some code paths which
2205 : // expect the meta-index block to not be compressed.
2206 1 : metaindexBH, err := w.writeBlock(metaindex.blockWriter.finish(), NoCompression, &w.blockBuf)
2207 1 : if err != nil {
2208 0 : return err
2209 0 : }
2210 :
2211 : // Write the table footer.
2212 1 : footer := footer{
2213 1 : format: w.tableFormat,
2214 1 : checksum: w.blockBuf.checksummer.checksumType,
2215 1 : metaindexBH: metaindexBH,
2216 1 : indexBH: indexBH,
2217 1 : }
2218 1 : encoded := footer.encode(w.blockBuf.tmp[:])
2219 1 : if err := w.writable.Write(footer.encode(w.blockBuf.tmp[:])); err != nil {
2220 0 : return err
2221 0 : }
2222 1 : w.meta.Size += uint64(len(encoded))
2223 1 : w.meta.Properties = w.props
2224 1 :
2225 1 : // Check that the features present in the table are compatible with the format
2226 1 : // configured for the table.
2227 1 : if err = w.assertFormatCompatibility(); err != nil {
2228 0 : return err
2229 0 : }
2230 :
2231 1 : if err := w.writable.Finish(); err != nil {
2232 0 : w.writable = nil
2233 0 : return err
2234 0 : }
2235 1 : w.writable = nil
2236 1 :
2237 1 : w.dataBlockBuf.clear()
2238 1 : dataBlockBufPool.Put(w.dataBlockBuf)
2239 1 : w.dataBlockBuf = nil
2240 1 : w.indexBlock.clear()
2241 1 : indexBlockBufPool.Put(w.indexBlock)
2242 1 : w.indexBlock = nil
2243 1 :
2244 1 : // Make any future calls to Set or Close return an error.
2245 1 : w.err = errWriterClosed
2246 1 : return nil
2247 : }
2248 :
2249 : // EstimatedSize returns the estimated size of the sstable being written if a
2250 : // call to Finish() was made without adding additional keys.
2251 1 : func (w *Writer) EstimatedSize() uint64 {
2252 1 : if w == nil {
2253 0 : return 0
2254 0 : }
2255 1 : return w.coordination.sizeEstimate.size() +
2256 1 : uint64(w.dataBlockBuf.dataBlock.estimatedSize()) +
2257 1 : w.indexBlock.estimatedSize()
2258 : }
2259 :
2260 : // Metadata returns the metadata for the finished sstable. Only valid to call
2261 : // after the sstable has been finished.
2262 1 : func (w *Writer) Metadata() (*WriterMetadata, error) {
2263 1 : if w.writable != nil {
2264 0 : return nil, errors.New("pebble: writer is not closed")
2265 0 : }
2266 1 : return &w.meta, nil
2267 : }
2268 :
2269 : // WriterOption provide an interface to do work on Writer while it is being
2270 : // opened.
2271 : type WriterOption interface {
2272 : // writerApply is called on the writer during opening in order to set
2273 : // internal parameters.
2274 : writerApply(*Writer)
2275 : }
2276 :
2277 : // NewWriter returns a new table writer for the file. Closing the writer will
2278 : // close the file.
2279 1 : func NewWriter(writable objstorage.Writable, o WriterOptions, extraOpts ...WriterOption) *Writer {
2280 1 : o = o.ensureDefaults()
2281 1 : w := &Writer{
2282 1 : writable: writable,
2283 1 : meta: WriterMetadata{
2284 1 : SmallestSeqNum: math.MaxUint64,
2285 1 : },
2286 1 : dataBlockOptions: flushDecisionOptions{
2287 1 : blockSize: o.BlockSize,
2288 1 : blockSizeThreshold: (o.BlockSize*o.BlockSizeThreshold + 99) / 100,
2289 1 : sizeClassAwareThreshold: (o.BlockSize*o.SizeClassAwareThreshold + 99) / 100,
2290 1 : },
2291 1 : indexBlockOptions: flushDecisionOptions{
2292 1 : blockSize: o.IndexBlockSize,
2293 1 : blockSizeThreshold: (o.IndexBlockSize*o.BlockSizeThreshold + 99) / 100,
2294 1 : sizeClassAwareThreshold: (o.IndexBlockSize*o.SizeClassAwareThreshold + 99) / 100,
2295 1 : },
2296 1 : compare: o.Comparer.Compare,
2297 1 : split: o.Comparer.Split,
2298 1 : formatKey: o.Comparer.FormatKey,
2299 1 : compression: o.Compression,
2300 1 : separator: o.Comparer.Separator,
2301 1 : successor: o.Comparer.Successor,
2302 1 : tableFormat: o.TableFormat,
2303 1 : isStrictObsolete: o.IsStrictObsolete,
2304 1 : writingToLowestLevel: o.WritingToLowestLevel,
2305 1 : cache: o.Cache,
2306 1 : restartInterval: o.BlockRestartInterval,
2307 1 : checksumType: o.Checksum,
2308 1 : indexBlock: newIndexBlockBuf(o.Parallelism),
2309 1 : rangeDelBlock: blockWriter{
2310 1 : restartInterval: 1,
2311 1 : },
2312 1 : rangeKeyBlock: blockWriter{
2313 1 : restartInterval: 1,
2314 1 : },
2315 1 : topLevelIndexBlock: blockWriter{
2316 1 : restartInterval: 1,
2317 1 : },
2318 1 : fragmenter: keyspan.Fragmenter{
2319 1 : Cmp: o.Comparer.Compare,
2320 1 : Format: o.Comparer.FormatKey,
2321 1 : },
2322 1 : allocatorSizeClasses: o.AllocatorSizeClasses,
2323 1 : }
2324 1 : if w.tableFormat >= TableFormatPebblev3 {
2325 1 : w.shortAttributeExtractor = o.ShortAttributeExtractor
2326 1 : w.requiredInPlaceValueBound = o.RequiredInPlaceValueBound
2327 1 : if !o.DisableValueBlocks {
2328 1 : w.valueBlockWriter = newValueBlockWriter(
2329 1 : w.dataBlockOptions.blockSize, w.dataBlockOptions.blockSizeThreshold, w.compression, w.checksumType, func(compressedSize int) {
2330 1 : w.coordination.sizeEstimate.dataBlockCompressed(compressedSize, 0)
2331 1 : })
2332 : }
2333 : }
2334 :
2335 1 : w.dataBlockBuf = newDataBlockBuf(w.restartInterval, w.checksumType)
2336 1 :
2337 1 : w.blockBuf = blockBuf{
2338 1 : checksummer: checksummer{checksumType: o.Checksum},
2339 1 : }
2340 1 :
2341 1 : w.coordination.init(o.Parallelism, w)
2342 1 :
2343 1 : if writable == nil {
2344 0 : w.err = errors.New("pebble: nil writable")
2345 0 : return w
2346 0 : }
2347 :
2348 : // Note that WriterOptions are applied in two places; the ones with a
2349 : // preApply() method are applied here. The rest are applied down below after
2350 : // default properties are set.
2351 1 : type preApply interface{ preApply() }
2352 1 : for _, opt := range extraOpts {
2353 1 : if _, ok := opt.(preApply); ok {
2354 1 : opt.writerApply(w)
2355 1 : }
2356 : }
2357 :
2358 1 : if o.FilterPolicy != nil {
2359 1 : switch o.FilterType {
2360 1 : case TableFilter:
2361 1 : w.filter = newTableFilterWriter(o.FilterPolicy)
2362 0 : default:
2363 0 : panic(fmt.Sprintf("unknown filter type: %v", o.FilterType))
2364 : }
2365 : }
2366 :
2367 1 : w.props.ComparerName = o.Comparer.Name
2368 1 : w.props.CompressionName = o.Compression.String()
2369 1 : w.props.MergerName = o.MergerName
2370 1 : w.props.PropertyCollectorNames = "[]"
2371 1 :
2372 1 : numBlockPropertyCollectors := len(o.BlockPropertyCollectors)
2373 1 : if w.tableFormat >= TableFormatPebblev4 {
2374 1 : numBlockPropertyCollectors++
2375 1 : }
2376 :
2377 1 : if numBlockPropertyCollectors > 0 {
2378 1 : if numBlockPropertyCollectors > maxPropertyCollectors {
2379 0 : w.err = errors.New("pebble: too many block property collectors")
2380 0 : return w
2381 0 : }
2382 1 : w.blockPropCollectors = make([]BlockPropertyCollector, 0, numBlockPropertyCollectors)
2383 1 : for _, constructFn := range o.BlockPropertyCollectors {
2384 1 : w.blockPropCollectors = append(w.blockPropCollectors, constructFn())
2385 1 : }
2386 1 : if w.tableFormat >= TableFormatPebblev4 {
2387 1 : w.blockPropCollectors = append(w.blockPropCollectors, &w.obsoleteCollector)
2388 1 : }
2389 :
2390 1 : var buf bytes.Buffer
2391 1 : buf.WriteString("[")
2392 1 : for i := range w.blockPropCollectors {
2393 1 : if i > 0 {
2394 1 : buf.WriteString(",")
2395 1 : }
2396 1 : buf.WriteString(w.blockPropCollectors[i].Name())
2397 : }
2398 1 : buf.WriteString("]")
2399 1 : w.props.PropertyCollectorNames = buf.String()
2400 : }
2401 :
2402 : // Apply the remaining WriterOptions that do not have a preApply() method.
2403 1 : for _, opt := range extraOpts {
2404 1 : if _, ok := opt.(preApply); ok {
2405 1 : continue
2406 : }
2407 0 : opt.writerApply(w)
2408 : }
2409 :
2410 : // Initialize the range key fragmenter and encoder.
2411 1 : w.fragmenter.Emit = w.encodeRangeKeySpan
2412 1 : w.rangeKeyEncoder.Emit = w.addRangeKey
2413 1 : return w
2414 : }
2415 :
2416 : // internalGetProperties is a private, internal-use-only function that takes a
2417 : // Writer and returns a pointer to its Properties, allowing direct mutation.
2418 : // It's used by internal Pebble flushes and compactions to set internal
2419 : // properties. It gets installed in private.
2420 1 : func internalGetProperties(w *Writer) *Properties {
2421 1 : return &w.props
2422 1 : }
2423 :
2424 1 : func init() {
2425 1 : private.SSTableWriterDisableKeyOrderChecks = func(i interface{}) {
2426 0 : w := i.(*Writer)
2427 0 : w.disableKeyOrderChecks = true
2428 0 : }
2429 1 : private.SSTableInternalProperties = internalGetProperties
2430 : }
|