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