Line data Source code
1 : // Copyright 2012 The LevelDB-Go and Pebble Authors. All rights reserved. Use
2 : // of this source code is governed by a BSD-style license that can be found in
3 : // the LICENSE file.
4 :
5 : package pebble
6 :
7 : import (
8 : "bytes"
9 : "context"
10 : "encoding/binary"
11 : "fmt"
12 : "io"
13 : "math"
14 : "sort"
15 : "sync"
16 : "sync/atomic"
17 : "time"
18 : "unsafe"
19 :
20 : "github.com/cockroachdb/errors"
21 : "github.com/cockroachdb/pebble/batchrepr"
22 : "github.com/cockroachdb/pebble/internal/base"
23 : "github.com/cockroachdb/pebble/internal/batchskl"
24 : "github.com/cockroachdb/pebble/internal/humanize"
25 : "github.com/cockroachdb/pebble/internal/invariants"
26 : "github.com/cockroachdb/pebble/internal/keyspan"
27 : "github.com/cockroachdb/pebble/internal/private"
28 : "github.com/cockroachdb/pebble/internal/rangedel"
29 : "github.com/cockroachdb/pebble/internal/rangekey"
30 : "github.com/cockroachdb/pebble/internal/rawalloc"
31 : "github.com/cockroachdb/pebble/internal/treeprinter"
32 : )
33 :
34 : const (
35 : invalidBatchCount = 1<<32 - 1
36 : maxVarintLen32 = 5
37 :
38 : defaultBatchInitialSize = 1 << 10 // 1 KB
39 : defaultBatchMaxRetainedSize = 1 << 20 // 1 MB
40 : )
41 :
42 : // ErrNotIndexed means that a read operation on a batch failed because the
43 : // batch is not indexed and thus doesn't support reads.
44 : var ErrNotIndexed = errors.New("pebble: batch not indexed")
45 :
46 : // ErrInvalidBatch indicates that a batch is invalid or otherwise corrupted.
47 : var ErrInvalidBatch = batchrepr.ErrInvalidBatch
48 :
49 : // ErrBatchTooLarge indicates that a batch is invalid or otherwise corrupted.
50 : var ErrBatchTooLarge = base.MarkCorruptionError(errors.Newf("pebble: batch too large: >= %s", humanize.Bytes.Uint64(maxBatchSize)))
51 :
52 : // DeferredBatchOp represents a batch operation (eg. set, merge, delete) that is
53 : // being inserted into the batch. Indexing is not performed on the specified key
54 : // until Finish is called, hence the name deferred. This struct lets the caller
55 : // copy or encode keys/values directly into the batch representation instead of
56 : // copying into an intermediary buffer then having pebble.Batch copy off of it.
57 : type DeferredBatchOp struct {
58 : index *batchskl.Skiplist
59 :
60 : // Key and Value point to parts of the binary batch representation where
61 : // keys and values should be encoded/copied into. len(Key) and len(Value)
62 : // bytes must be copied into these slices respectively before calling
63 : // Finish(). Changing where these slices point to is not allowed.
64 : Key, Value []byte
65 : offset uint32
66 : }
67 :
68 : // Finish completes the addition of this batch operation, and adds it to the
69 : // index if necessary. Must be called once (and exactly once) keys/values
70 : // have been filled into Key and Value. Not calling Finish or not
71 : // copying/encoding keys will result in an incomplete index, and calling Finish
72 : // twice may result in a panic.
73 0 : func (d DeferredBatchOp) Finish() error {
74 0 : if d.index != nil {
75 0 : if err := d.index.Add(d.offset); err != nil {
76 0 : return err
77 0 : }
78 : }
79 0 : return nil
80 : }
81 :
82 : // A Batch is a sequence of Sets, Merges, Deletes, DeleteRanges, RangeKeySets,
83 : // RangeKeyUnsets, and/or RangeKeyDeletes that are applied atomically. Batch
84 : // implements the Reader interface, but only an indexed batch supports reading
85 : // (without error) via Get or NewIter. A non-indexed batch will return
86 : // ErrNotIndexed when read from. A batch is not safe for concurrent use, and
87 : // consumers should use a batch per goroutine or provide their own
88 : // synchronization.
89 : //
90 : // # Indexing
91 : //
92 : // Batches can be optionally indexed (see DB.NewIndexedBatch). An indexed batch
93 : // allows iteration via an Iterator (see Batch.NewIter). The iterator provides
94 : // a merged view of the operations in the batch and the underlying
95 : // database. This is implemented by treating the batch as an additional layer
96 : // in the LSM where every entry in the batch is considered newer than any entry
97 : // in the underlying database (batch entries have the InternalKeySeqNumBatch
98 : // bit set). By treating the batch as an additional layer in the LSM, iteration
99 : // supports all batch operations (i.e. Set, Merge, Delete, DeleteRange,
100 : // RangeKeySet, RangeKeyUnset, RangeKeyDelete) with minimal effort.
101 : //
102 : // The same key can be operated on multiple times in a batch, though only the
103 : // latest operation will be visible. For example, Put("a", "b"), Delete("a")
104 : // will cause the key "a" to not be visible in the batch. Put("a", "b"),
105 : // Put("a", "c") will cause a read of "a" to return the value "c".
106 : //
107 : // The batch index is implemented via an skiplist (internal/batchskl). While
108 : // the skiplist implementation is very fast, inserting into an indexed batch is
109 : // significantly slower than inserting into a non-indexed batch. Only use an
110 : // indexed batch if you require reading from it.
111 : //
112 : // # Atomic commit
113 : //
114 : // The operations in a batch are persisted by calling Batch.Commit which is
115 : // equivalent to calling DB.Apply(batch). A batch is committed atomically by
116 : // writing the internal batch representation to the WAL, adding all of the
117 : // batch operations to the memtable associated with the WAL, and then
118 : // incrementing the visible sequence number so that subsequent reads can see
119 : // the effects of the batch operations. If WriteOptions.Sync is true, a call to
120 : // Batch.Commit will guarantee that the batch is persisted to disk before
121 : // returning. See commitPipeline for more on the implementation details.
122 : //
123 : // # Large batches
124 : //
125 : // The size of a batch is limited only by available memory (be aware that
126 : // indexed batches require considerably additional memory for the skiplist
127 : // structure). A given WAL file has a single memtable associated with it (this
128 : // restriction could be removed, but doing so is onerous and complex). And a
129 : // memtable has a fixed size due to the underlying fixed size arena. Note that
130 : // this differs from RocksDB where a memtable can grow arbitrarily large using
131 : // a list of arena chunks. In RocksDB this is accomplished by storing pointers
132 : // in the arena memory, but that isn't possible in Go.
133 : //
134 : // During Batch.Commit, a batch which is larger than a threshold (>
135 : // MemTableSize/2) is wrapped in a flushableBatch and inserted into the queue
136 : // of memtables. A flushableBatch forces WAL to be rotated, but that happens
137 : // anyways when the memtable becomes full so this does not cause significant
138 : // WAL churn. Because the flushableBatch is readable as another layer in the
139 : // LSM, Batch.Commit returns as soon as the flushableBatch has been added to
140 : // the queue of memtables.
141 : //
142 : // Internally, a flushableBatch provides Iterator support by sorting the batch
143 : // contents (the batch is sorted once, when it is added to the memtable
144 : // queue). Sorting the batch contents and insertion of the contents into a
145 : // memtable have the same big-O time, but the constant factor dominates
146 : // here. Sorting is significantly faster and uses significantly less memory.
147 : //
148 : // # Internal representation
149 : //
150 : // The internal batch representation is a contiguous byte buffer with a fixed
151 : // 12-byte header, followed by a series of records.
152 : //
153 : // +-------------+------------+--- ... ---+
154 : // | SeqNum (8B) | Count (4B) | Entries |
155 : // +-------------+------------+--- ... ---+
156 : //
157 : // Each record has a 1-byte kind tag prefix, followed by 1 or 2 length prefixed
158 : // strings (varstring):
159 : //
160 : // +-----------+-----------------+-------------------+
161 : // | Kind (1B) | Key (varstring) | Value (varstring) |
162 : // +-----------+-----------------+-------------------+
163 : //
164 : // A varstring is a varint32 followed by N bytes of data. The Kind tags are
165 : // exactly those specified by InternalKeyKind. The following table shows the
166 : // format for records of each kind:
167 : //
168 : // InternalKeyKindDelete varstring
169 : // InternalKeyKindLogData varstring
170 : // InternalKeyKindIngestSST varstring
171 : // InternalKeyKindSet varstring varstring
172 : // InternalKeyKindMerge varstring varstring
173 : // InternalKeyKindRangeDelete varstring varstring
174 : // InternalKeyKindRangeKeySet varstring varstring
175 : // InternalKeyKindRangeKeyUnset varstring varstring
176 : // InternalKeyKindRangeKeyDelete varstring varstring
177 : //
178 : // The intuitive understanding here are that the arguments to Delete, Set,
179 : // Merge, DeleteRange and RangeKeyDelete are encoded into the batch. The
180 : // RangeKeySet and RangeKeyUnset operations are slightly more complicated,
181 : // encoding their end key, suffix and value [in the case of RangeKeySet] within
182 : // the Value varstring. For more information on the value encoding for
183 : // RangeKeySet and RangeKeyUnset, see the internal/rangekey package.
184 : //
185 : // The internal batch representation is the on disk format for a batch in the
186 : // WAL, and thus stable. New record kinds may be added, but the existing ones
187 : // will not be modified.
188 : type Batch struct {
189 : batchInternal
190 : applied atomic.Bool
191 : // lifecycle is used to negotiate the lifecycle of a Batch. A Batch and its
192 : // underlying batchInternal.data byte slice may be reused. There are two
193 : // mechanisms for reuse:
194 : //
195 : // 1. The caller may explicitly call [Batch.Reset] to reset the batch to be
196 : // empty (while retaining the underlying repr's buffer).
197 : // 2. The caller may call [Batch.Close], passing ownership off to Pebble,
198 : // which may reuse the batch's memory to service new callers to
199 : // [DB.NewBatch].
200 : //
201 : // There's a complication to reuse: When WAL failover is configured, the
202 : // Pebble commit pipeline may retain a pointer to the batch.data beyond the
203 : // return of [Batch.Commit]. The user of the Batch may commit their batch
204 : // and call Close or Reset before the commit pipeline is finished reading
205 : // the data slice. Recycling immediately would cause a data race.
206 : //
207 : // To resolve this data race, this [lifecycle] atomic is used to determine
208 : // safety and responsibility of reusing a batch. The low bits of the atomic
209 : // are used as a reference count (really just the lowest bit—in practice
210 : // there's only 1 code path that references). The [Batch] is passed into
211 : // [wal.Writer]'s WriteRecord method as a [RefCount] implementation. The
212 : // wal.Writer guarantees that if it will read [Batch.data] after the call to
213 : // WriteRecord returns, it will increment the reference count. When it's
214 : // complete, it'll unreference through invoking [Batch.Unref].
215 : //
216 : // When the committer of a batch indicates intent to recycle a Batch through
217 : // calling [Batch.Reset] or [Batch.Close], the lifecycle atomic is read. If
218 : // an outstanding reference remains, it's unsafe to reuse Batch.data yet. In
219 : // [Batch.Reset] the caller wants to reuse the [Batch] immediately, so we
220 : // discard b.data to recycle the struct but not the underlying byte slice.
221 : // In [Batch.Close], we set a special high bit [batchClosedBit] on lifecycle
222 : // that indicates that the user will not use [Batch] again and we're free to
223 : // recycle it when safe. When the commit pipeline eventually calls
224 : // [Batch.Unref], the [batchClosedBit] is noticed and the batch is
225 : // recycled.
226 : lifecycle atomic.Int32
227 : }
228 :
229 : // batchClosedBit is a bit stored on Batch.lifecycle to indicate that the user
230 : // called [Batch.Close] to release a Batch, but an open reference count
231 : // prevented immediate recycling.
232 : const batchClosedBit = 1 << 30
233 :
234 : // TODO(jackson): Hide the wal.RefCount implementation from the public Batch interface.
235 :
236 : // Ref implements wal.RefCount. If the WAL writer may need to read b.data after
237 : // it returns, it invokes Ref to increment the lifecycle's reference count. When
238 : // it's finished, it invokes Unref.
239 1 : func (b *Batch) Ref() {
240 1 : b.lifecycle.Add(+1)
241 1 : }
242 :
243 : // Unref implemets wal.RefCount.
244 1 : func (b *Batch) Unref() {
245 1 : if v := b.lifecycle.Add(-1); (v ^ batchClosedBit) == 0 {
246 1 : // The [batchClosedBit] high bit is set, and there are no outstanding
247 1 : // references. The user of the Batch called [Batch.Close], expecting the
248 1 : // batch to be recycled. However, our outstanding reference count
249 1 : // prevented recycling. As the last to dereference, we're now
250 1 : // responsible for releasing the batch.
251 1 : b.lifecycle.Store(0)
252 1 : b.release()
253 1 : }
254 : }
255 :
256 : // batchInternal contains the set of fields within Batch that are non-atomic and
257 : // capable of being reset using a *b = batchInternal{} struct copy.
258 : type batchInternal struct {
259 : // Data is the wire format of a batch's log entry:
260 : // - 8 bytes for a sequence number of the first batch element,
261 : // or zeroes if the batch has not yet been applied,
262 : // - 4 bytes for the count: the number of elements in the batch,
263 : // or "\xff\xff\xff\xff" if the batch is invalid,
264 : // - count elements, being:
265 : // - one byte for the kind
266 : // - the varint-string user key,
267 : // - the varint-string value (if kind != delete).
268 : // The sequence number and count are stored in little-endian order.
269 : //
270 : // The data field can be (but is not guaranteed to be) nil for new
271 : // batches. Large batches will set the data field to nil when committed as
272 : // the data has been moved to a flushableBatch and inserted into the queue of
273 : // memtables.
274 : data []byte
275 : comparer *base.Comparer
276 : opts batchOptions
277 :
278 : // An upper bound on required space to add this batch to a memtable.
279 : // Note that although batches are limited to 4 GiB in size, that limit
280 : // applies to len(data), not the memtable size. The upper bound on the
281 : // size of a memtable node is larger than the overhead of the batch's log
282 : // encoding, so memTableSize is larger than len(data) and may overflow a
283 : // uint32.
284 : memTableSize uint64
285 :
286 : // The db to which the batch will be committed. Do not change this field
287 : // after the batch has been created as it might invalidate internal state.
288 : // Batch.memTableSize is only refreshed if Batch.db is set. Setting db to
289 : // nil once it has been set implies that the Batch has encountered an error.
290 : db *DB
291 :
292 : // The count of records in the batch. This count will be stored in the batch
293 : // data whenever Repr() is called.
294 : count uint64
295 :
296 : // The count of range deletions in the batch. Updated every time a range
297 : // deletion is added.
298 : countRangeDels uint64
299 :
300 : // The count of range key sets, unsets and deletes in the batch. Updated
301 : // every time a RANGEKEYSET, RANGEKEYUNSET or RANGEKEYDEL key is added.
302 : countRangeKeys uint64
303 :
304 : // A deferredOp struct, stored in the Batch so that a pointer can be returned
305 : // from the *Deferred() methods rather than a value.
306 : deferredOp DeferredBatchOp
307 :
308 : // An optional skiplist keyed by offset into data of the entry.
309 : index *batchskl.Skiplist
310 : rangeDelIndex *batchskl.Skiplist
311 : rangeKeyIndex *batchskl.Skiplist
312 :
313 : // Fragmented range deletion tombstones. Cached the first time a range
314 : // deletion iterator is requested. The cache is invalidated whenever a new
315 : // range deletion is added to the batch. This cache can only be used when
316 : // opening an iterator to read at a batch sequence number >=
317 : // tombstonesSeqNum. This is the case for all new iterators created over a
318 : // batch but it's not the case for all cloned iterators.
319 : tombstones []keyspan.Span
320 : tombstonesSeqNum base.SeqNum
321 :
322 : // Fragmented range key spans. Cached the first time a range key iterator is
323 : // requested. The cache is invalidated whenever a new range key
324 : // (RangeKey{Set,Unset,Del}) is added to the batch. This cache can only be
325 : // used when opening an iterator to read at a batch sequence number >=
326 : // tombstonesSeqNum. This is the case for all new iterators created over a
327 : // batch but it's not the case for all cloned iterators.
328 : rangeKeys []keyspan.Span
329 : rangeKeysSeqNum base.SeqNum
330 :
331 : // The flushableBatch wrapper if the batch is too large to fit in the
332 : // memtable.
333 : flushable *flushableBatch
334 :
335 : // minimumFormatMajorVersion indicates the format major version required in
336 : // order to commit this batch. If an operation requires a particular format
337 : // major version, it ratchets the batch's minimumFormatMajorVersion. When
338 : // the batch is committed, this is validated against the database's current
339 : // format major version.
340 : minimumFormatMajorVersion FormatMajorVersion
341 :
342 : // Synchronous Apply uses the commit WaitGroup for both publishing the
343 : // seqnum and waiting for the WAL fsync (if needed). Asynchronous
344 : // ApplyNoSyncWait, which implies WriteOptions.Sync is true, uses the commit
345 : // WaitGroup for publishing the seqnum and the fsyncWait WaitGroup for
346 : // waiting for the WAL fsync.
347 : //
348 : // TODO(sumeer): if we find that ApplyNoSyncWait in conjunction with
349 : // SyncWait is causing higher memory usage because of the time duration
350 : // between when the sync is already done, and a goroutine calls SyncWait
351 : // (followed by Batch.Close), we could separate out {fsyncWait, commitErr}
352 : // into a separate struct that is allocated separately (using another
353 : // sync.Pool), and only that struct needs to outlive Batch.Close (which
354 : // could then be called immediately after ApplyNoSyncWait). commitStats
355 : // will also need to be in this separate struct.
356 : commit sync.WaitGroup
357 : fsyncWait sync.WaitGroup
358 :
359 : commitStats BatchCommitStats
360 :
361 : commitErr error
362 :
363 : // Position bools together to reduce the sizeof the struct.
364 :
365 : // ingestedSSTBatch indicates that the batch contains one or more key kinds
366 : // of InternalKeyKindIngestSST. If the batch contains key kinds of IngestSST
367 : // then it will only contain key kinds of IngestSST.
368 : ingestedSSTBatch bool
369 :
370 : // committing is set to true when a batch begins to commit. It's used to
371 : // ensure the batch is not mutated concurrently. It is not an atomic
372 : // deliberately, so as to avoid the overhead on batch mutations. This is
373 : // okay, because under correct usage this field will never be accessed
374 : // concurrently. It's only under incorrect usage the memory accesses of this
375 : // variable may violate memory safety. Since we don't use atomics here,
376 : // false negatives are possible.
377 : committing bool
378 : }
379 :
380 : // BatchCommitStats exposes stats related to committing a batch.
381 : //
382 : // NB: there is no Pebble internal tracing (using LoggerAndTracer) of slow
383 : // batch commits. The caller can use these stats to do their own tracing as
384 : // needed.
385 : type BatchCommitStats struct {
386 : // TotalDuration is the time spent in DB.{Apply,ApplyNoSyncWait} or
387 : // Batch.Commit, plus the time waiting in Batch.SyncWait. If there is a gap
388 : // between calling ApplyNoSyncWait and calling SyncWait, that gap could
389 : // include some duration in which real work was being done for the commit
390 : // and will not be included here. This missing time is considered acceptable
391 : // since the goal of these stats is to understand user-facing latency.
392 : //
393 : // TotalDuration includes time spent in various queues both inside Pebble
394 : // and outside Pebble (I/O queues, goroutine scheduler queue, mutex wait
395 : // etc.). For some of these queues (which we consider important) the wait
396 : // times are included below -- these expose low-level implementation detail
397 : // and are meant for expert diagnosis and subject to change. There may be
398 : // unaccounted time after subtracting those values from TotalDuration.
399 : TotalDuration time.Duration
400 : // SemaphoreWaitDuration is the wait time for semaphores in
401 : // commitPipeline.Commit.
402 : SemaphoreWaitDuration time.Duration
403 : // WALQueueWaitDuration is the wait time for allocating memory blocks in the
404 : // LogWriter (due to the LogWriter not writing fast enough). At the moment
405 : // this is duration is always zero because a single WAL will allow
406 : // allocating memory blocks up to the entire memtable size. In the future,
407 : // we may pipeline WALs and bound the WAL queued blocks separately, so this
408 : // field is preserved for that possibility.
409 : WALQueueWaitDuration time.Duration
410 : // MemTableWriteStallDuration is the wait caused by a write stall due to too
411 : // many memtables (due to not flushing fast enough).
412 : MemTableWriteStallDuration time.Duration
413 : // L0ReadAmpWriteStallDuration is the wait caused by a write stall due to
414 : // high read amplification in L0 (due to not compacting fast enough out of
415 : // L0).
416 : L0ReadAmpWriteStallDuration time.Duration
417 : // WALRotationDuration is the wait time for WAL rotation, which includes
418 : // syncing and closing the old WAL and creating (or reusing) a new one.
419 : WALRotationDuration time.Duration
420 : // CommitWaitDuration is the wait for publishing the seqnum plus the
421 : // duration for the WAL sync (if requested). The former should be tiny and
422 : // one can assume that this is all due to the WAL sync.
423 : CommitWaitDuration time.Duration
424 : }
425 :
426 : var _ Reader = (*Batch)(nil)
427 : var _ Writer = (*Batch)(nil)
428 :
429 : var batchPool = sync.Pool{
430 1 : New: func() interface{} {
431 1 : return &Batch{}
432 1 : },
433 : }
434 :
435 : type indexedBatch struct {
436 : batch Batch
437 : index batchskl.Skiplist
438 : }
439 :
440 : var indexedBatchPool = sync.Pool{
441 1 : New: func() interface{} {
442 1 : return &indexedBatch{}
443 1 : },
444 : }
445 :
446 1 : func newBatch(db *DB, opts ...BatchOption) *Batch {
447 1 : b := batchPool.Get().(*Batch)
448 1 : b.db = db
449 1 : b.opts.ensureDefaults()
450 1 : for _, opt := range opts {
451 0 : opt(&b.opts)
452 0 : }
453 1 : return b
454 : }
455 :
456 0 : func newBatchWithSize(db *DB, size int, opts ...BatchOption) *Batch {
457 0 : b := newBatch(db, opts...)
458 0 : if cap(b.data) < size {
459 0 : b.data = rawalloc.New(0, size)
460 0 : }
461 0 : return b
462 : }
463 :
464 1 : func newIndexedBatch(db *DB, comparer *Comparer) *Batch {
465 1 : i := indexedBatchPool.Get().(*indexedBatch)
466 1 : i.batch.comparer = comparer
467 1 : i.batch.db = db
468 1 : i.batch.index = &i.index
469 1 : i.batch.index.Init(&i.batch.data, comparer.Compare, comparer.AbbreviatedKey)
470 1 : i.batch.opts.ensureDefaults()
471 1 : return &i.batch
472 1 : }
473 :
474 0 : func newIndexedBatchWithSize(db *DB, comparer *Comparer, size int) *Batch {
475 0 : b := newIndexedBatch(db, comparer)
476 0 : if cap(b.data) < size {
477 0 : b.data = rawalloc.New(0, size)
478 0 : }
479 0 : return b
480 : }
481 :
482 : // nextSeqNum returns the batch "sequence number" that will be given to the next
483 : // key written to the batch. During iteration keys within an indexed batch are
484 : // given a sequence number consisting of their offset within the batch combined
485 : // with the base.SeqNumBatchBit bit. These sequence numbers are only
486 : // used during iteration, and the keys are assigned ordinary sequence numbers
487 : // when the batch is committed.
488 1 : func (b *Batch) nextSeqNum() base.SeqNum {
489 1 : return base.SeqNum(len(b.data)) | base.SeqNumBatchBit
490 1 : }
491 :
492 1 : func (b *Batch) release() {
493 1 : if b.db == nil {
494 1 : // The batch was not created using newBatch or newIndexedBatch, or an error
495 1 : // was encountered. We don't try to reuse batches that encountered an error
496 1 : // because they might be stuck somewhere in the system and attempting to
497 1 : // reuse such batches is a recipe for onerous debugging sessions. Instead,
498 1 : // let the GC do its job.
499 1 : return
500 1 : }
501 1 : b.db = nil
502 1 :
503 1 : // NB: This is ugly (it would be cleaner if we could just assign a Batch{}),
504 1 : // but necessary so that we can use atomic.StoreUint32 for the Batch.applied
505 1 : // field. Without using an atomic to clear that field the Go race detector
506 1 : // complains.
507 1 : b.reset()
508 1 : b.comparer = nil
509 1 :
510 1 : if b.index == nil {
511 1 : batchPool.Put(b)
512 1 : } else {
513 1 : b.index, b.rangeDelIndex, b.rangeKeyIndex = nil, nil, nil
514 1 : indexedBatchPool.Put((*indexedBatch)(unsafe.Pointer(b)))
515 1 : }
516 : }
517 :
518 1 : func (b *Batch) refreshMemTableSize() error {
519 1 : b.memTableSize = 0
520 1 : if len(b.data) < batchrepr.HeaderLen {
521 0 : return nil
522 0 : }
523 :
524 1 : b.countRangeDels = 0
525 1 : b.countRangeKeys = 0
526 1 : b.minimumFormatMajorVersion = 0
527 1 : for r := b.Reader(); ; {
528 1 : kind, key, value, ok, err := r.Next()
529 1 : if !ok {
530 1 : if err != nil {
531 0 : return err
532 0 : }
533 1 : break
534 : }
535 1 : switch kind {
536 1 : case InternalKeyKindRangeDelete:
537 1 : b.countRangeDels++
538 1 : case InternalKeyKindRangeKeySet, InternalKeyKindRangeKeyUnset, InternalKeyKindRangeKeyDelete:
539 1 : b.countRangeKeys++
540 1 : case InternalKeyKindSet, InternalKeyKindDelete, InternalKeyKindMerge, InternalKeyKindSingleDelete, InternalKeyKindSetWithDelete:
541 : // fallthrough
542 1 : case InternalKeyKindDeleteSized:
543 1 : if b.minimumFormatMajorVersion < FormatDeleteSizedAndObsolete {
544 1 : b.minimumFormatMajorVersion = FormatDeleteSizedAndObsolete
545 1 : }
546 0 : case InternalKeyKindLogData:
547 0 : // LogData does not contribute to memtable size.
548 0 : continue
549 1 : case InternalKeyKindIngestSST:
550 1 : if b.minimumFormatMajorVersion < FormatFlushableIngest {
551 1 : b.minimumFormatMajorVersion = FormatFlushableIngest
552 1 : }
553 : // This key kind doesn't contribute to the memtable size.
554 1 : continue
555 0 : default:
556 0 : // Note In some circumstances this might be temporary memory
557 0 : // corruption that can be recovered by discarding the batch and
558 0 : // trying again. In other cases, the batch repr might've been
559 0 : // already persisted elsewhere, and we'll loop continuously trying
560 0 : // to commit the same corrupted batch. The caller is responsible for
561 0 : // distinguishing.
562 0 : return errors.Wrapf(ErrInvalidBatch, "unrecognized kind %v", kind)
563 : }
564 1 : b.memTableSize += memTableEntrySize(len(key), len(value))
565 : }
566 1 : return nil
567 : }
568 :
569 : // Apply the operations contained in the batch to the receiver batch.
570 : //
571 : // It is safe to modify the contents of the arguments after Apply returns.
572 : //
573 : // Apply returns ErrInvalidBatch if the provided batch is invalid in any way.
574 1 : func (b *Batch) Apply(batch *Batch, _ *WriteOptions) error {
575 1 : if b.ingestedSSTBatch {
576 0 : panic("pebble: invalid batch application")
577 : }
578 1 : if len(batch.data) == 0 {
579 0 : return nil
580 0 : }
581 1 : if len(batch.data) < batchrepr.HeaderLen {
582 0 : return ErrInvalidBatch
583 0 : }
584 :
585 1 : offset := len(b.data)
586 1 : if offset == 0 {
587 0 : b.init(offset)
588 0 : offset = batchrepr.HeaderLen
589 0 : }
590 1 : b.data = append(b.data, batch.data[batchrepr.HeaderLen:]...)
591 1 :
592 1 : b.setCount(b.Count() + batch.Count())
593 1 :
594 1 : if b.db != nil || b.index != nil {
595 1 : // Only iterate over the new entries if we need to track memTableSize or in
596 1 : // order to update the index.
597 1 : for iter := batchrepr.Reader(b.data[offset:]); len(iter) > 0; {
598 1 : offset := uintptr(unsafe.Pointer(&iter[0])) - uintptr(unsafe.Pointer(&b.data[0]))
599 1 : kind, key, value, ok, err := iter.Next()
600 1 : if !ok {
601 0 : if err != nil {
602 0 : return err
603 0 : }
604 0 : break
605 : }
606 1 : switch kind {
607 0 : case InternalKeyKindRangeDelete:
608 0 : b.countRangeDels++
609 0 : case InternalKeyKindRangeKeySet, InternalKeyKindRangeKeyUnset, InternalKeyKindRangeKeyDelete:
610 0 : b.countRangeKeys++
611 0 : case InternalKeyKindIngestSST:
612 0 : panic("pebble: invalid key kind for batch")
613 0 : case InternalKeyKindLogData:
614 0 : // LogData does not contribute to memtable size.
615 0 : continue
616 : case InternalKeyKindSet, InternalKeyKindDelete, InternalKeyKindMerge,
617 1 : InternalKeyKindSingleDelete, InternalKeyKindSetWithDelete, InternalKeyKindDeleteSized:
618 : // fallthrough
619 0 : default:
620 0 : // Note In some circumstances this might be temporary memory
621 0 : // corruption that can be recovered by discarding the batch and
622 0 : // trying again. In other cases, the batch repr might've been
623 0 : // already persisted elsewhere, and we'll loop continuously
624 0 : // trying to commit the same corrupted batch. The caller is
625 0 : // responsible for distinguishing.
626 0 : return errors.Wrapf(ErrInvalidBatch, "unrecognized kind %v", kind)
627 : }
628 1 : if b.index != nil {
629 1 : var err error
630 1 : switch kind {
631 0 : case InternalKeyKindRangeDelete:
632 0 : b.tombstones = nil
633 0 : b.tombstonesSeqNum = 0
634 0 : if b.rangeDelIndex == nil {
635 0 : b.rangeDelIndex = batchskl.NewSkiplist(&b.data, b.comparer.Compare, b.comparer.AbbreviatedKey)
636 0 : }
637 0 : err = b.rangeDelIndex.Add(uint32(offset))
638 0 : case InternalKeyKindRangeKeySet, InternalKeyKindRangeKeyUnset, InternalKeyKindRangeKeyDelete:
639 0 : b.rangeKeys = nil
640 0 : b.rangeKeysSeqNum = 0
641 0 : if b.rangeKeyIndex == nil {
642 0 : b.rangeKeyIndex = batchskl.NewSkiplist(&b.data, b.comparer.Compare, b.comparer.AbbreviatedKey)
643 0 : }
644 0 : err = b.rangeKeyIndex.Add(uint32(offset))
645 1 : default:
646 1 : err = b.index.Add(uint32(offset))
647 : }
648 1 : if err != nil {
649 0 : return err
650 0 : }
651 : }
652 1 : b.memTableSize += memTableEntrySize(len(key), len(value))
653 : }
654 : }
655 1 : return nil
656 : }
657 :
658 : // Get gets the value for the given key. It returns ErrNotFound if the Batch
659 : // does not contain the key.
660 : //
661 : // The caller should not modify the contents of the returned slice, but it is
662 : // safe to modify the contents of the argument after Get returns. The returned
663 : // slice will remain valid until the returned Closer is closed. On success, the
664 : // caller MUST call closer.Close() or a memory leak will occur.
665 1 : func (b *Batch) Get(key []byte) ([]byte, io.Closer, error) {
666 1 : if b.index == nil {
667 0 : return nil, nil, ErrNotIndexed
668 0 : }
669 1 : return b.db.getInternal(key, b, nil /* snapshot */)
670 : }
671 :
672 1 : func (b *Batch) prepareDeferredKeyValueRecord(keyLen, valueLen int, kind InternalKeyKind) {
673 1 : if b.committing {
674 0 : panic("pebble: batch already committing")
675 : }
676 1 : if len(b.data) == 0 {
677 1 : b.init(keyLen + valueLen + 2*binary.MaxVarintLen64 + batchrepr.HeaderLen)
678 1 : }
679 1 : b.count++
680 1 : b.memTableSize += memTableEntrySize(keyLen, valueLen)
681 1 :
682 1 : pos := len(b.data)
683 1 : b.deferredOp.offset = uint32(pos)
684 1 : b.grow(1 + 2*maxVarintLen32 + keyLen + valueLen)
685 1 : b.data[pos] = byte(kind)
686 1 : pos++
687 1 :
688 1 : {
689 1 : // TODO(peter): Manually inlined version binary.PutUvarint(). This is 20%
690 1 : // faster on BenchmarkBatchSet on go1.13. Remove if go1.14 or future
691 1 : // versions show this to not be a performance win.
692 1 : x := uint32(keyLen)
693 1 : for x >= 0x80 {
694 0 : b.data[pos] = byte(x) | 0x80
695 0 : x >>= 7
696 0 : pos++
697 0 : }
698 1 : b.data[pos] = byte(x)
699 1 : pos++
700 : }
701 :
702 1 : b.deferredOp.Key = b.data[pos : pos+keyLen]
703 1 : pos += keyLen
704 1 :
705 1 : {
706 1 : // TODO(peter): Manually inlined version binary.PutUvarint(). This is 20%
707 1 : // faster on BenchmarkBatchSet on go1.13. Remove if go1.14 or future
708 1 : // versions show this to not be a performance win.
709 1 : x := uint32(valueLen)
710 1 : for x >= 0x80 {
711 0 : b.data[pos] = byte(x) | 0x80
712 0 : x >>= 7
713 0 : pos++
714 0 : }
715 1 : b.data[pos] = byte(x)
716 1 : pos++
717 : }
718 :
719 1 : b.deferredOp.Value = b.data[pos : pos+valueLen]
720 1 : // Shrink data since varints may be shorter than the upper bound.
721 1 : b.data = b.data[:pos+valueLen]
722 : }
723 :
724 1 : func (b *Batch) prepareDeferredKeyRecord(keyLen int, kind InternalKeyKind) {
725 1 : if b.committing {
726 0 : panic("pebble: batch already committing")
727 : }
728 1 : if len(b.data) == 0 {
729 1 : b.init(keyLen + binary.MaxVarintLen64 + batchrepr.HeaderLen)
730 1 : }
731 1 : b.count++
732 1 : b.memTableSize += memTableEntrySize(keyLen, 0)
733 1 :
734 1 : pos := len(b.data)
735 1 : b.deferredOp.offset = uint32(pos)
736 1 : b.grow(1 + maxVarintLen32 + keyLen)
737 1 : b.data[pos] = byte(kind)
738 1 : pos++
739 1 :
740 1 : {
741 1 : // TODO(peter): Manually inlined version binary.PutUvarint(). Remove if
742 1 : // go1.13 or future versions show this to not be a performance win. See
743 1 : // BenchmarkBatchSet.
744 1 : x := uint32(keyLen)
745 1 : for x >= 0x80 {
746 0 : b.data[pos] = byte(x) | 0x80
747 0 : x >>= 7
748 0 : pos++
749 0 : }
750 1 : b.data[pos] = byte(x)
751 1 : pos++
752 : }
753 :
754 1 : b.deferredOp.Key = b.data[pos : pos+keyLen]
755 1 : b.deferredOp.Value = nil
756 1 :
757 1 : // Shrink data since varint may be shorter than the upper bound.
758 1 : b.data = b.data[:pos+keyLen]
759 : }
760 :
761 : // AddInternalKey allows the caller to add an internal key of point key or range
762 : // key kinds (but not RangeDelete) to a batch. Passing in an internal key of
763 : // kind RangeDelete will result in a panic. Note that the seqnum in the internal
764 : // key is effectively ignored, even though the Kind is preserved. This is
765 : // because the batch format does not allow for a per-key seqnum to be specified,
766 : // only a batch-wide one.
767 : //
768 : // Note that non-indexed keys (IngestKeyKind{LogData,IngestSST}) are not
769 : // supported with this method as they require specialized logic.
770 1 : func (b *Batch) AddInternalKey(key *base.InternalKey, value []byte, _ *WriteOptions) error {
771 1 : keyLen := len(key.UserKey)
772 1 : hasValue := false
773 1 : switch kind := key.Kind(); kind {
774 0 : case InternalKeyKindRangeDelete:
775 0 : panic("unexpected range delete in AddInternalKey")
776 0 : case InternalKeyKindSingleDelete, InternalKeyKindDelete:
777 0 : b.prepareDeferredKeyRecord(keyLen, kind)
778 0 : b.deferredOp.index = b.index
779 1 : case InternalKeyKindRangeKeySet, InternalKeyKindRangeKeyUnset, InternalKeyKindRangeKeyDelete:
780 1 : b.prepareDeferredKeyValueRecord(keyLen, len(value), kind)
781 1 : hasValue = true
782 1 : b.incrementRangeKeysCount()
783 0 : default:
784 0 : b.prepareDeferredKeyValueRecord(keyLen, len(value), kind)
785 0 : hasValue = true
786 0 : b.deferredOp.index = b.index
787 : }
788 1 : copy(b.deferredOp.Key, key.UserKey)
789 1 : if hasValue {
790 1 : copy(b.deferredOp.Value, value)
791 1 : }
792 :
793 : // TODO(peter): Manually inline DeferredBatchOp.Finish(). Mid-stack inlining
794 : // in go1.13 will remove the need for this.
795 1 : if b.index != nil {
796 0 : if err := b.index.Add(b.deferredOp.offset); err != nil {
797 0 : return err
798 0 : }
799 : }
800 1 : return nil
801 : }
802 :
803 : // Set adds an action to the batch that sets the key to map to the value.
804 : //
805 : // It is safe to modify the contents of the arguments after Set returns.
806 1 : func (b *Batch) Set(key, value []byte, _ *WriteOptions) error {
807 1 : deferredOp := b.SetDeferred(len(key), len(value))
808 1 : copy(deferredOp.Key, key)
809 1 : copy(deferredOp.Value, value)
810 1 : // TODO(peter): Manually inline DeferredBatchOp.Finish(). Mid-stack inlining
811 1 : // in go1.13 will remove the need for this.
812 1 : if b.index != nil {
813 1 : if err := b.index.Add(deferredOp.offset); err != nil {
814 0 : return err
815 0 : }
816 : }
817 1 : return nil
818 : }
819 :
820 : // SetDeferred is similar to Set in that it adds a set operation to the batch,
821 : // except it only takes in key/value lengths instead of complete slices,
822 : // letting the caller encode into those objects and then call Finish() on the
823 : // returned object.
824 1 : func (b *Batch) SetDeferred(keyLen, valueLen int) *DeferredBatchOp {
825 1 : b.prepareDeferredKeyValueRecord(keyLen, valueLen, InternalKeyKindSet)
826 1 : b.deferredOp.index = b.index
827 1 : return &b.deferredOp
828 1 : }
829 :
830 : // Merge adds an action to the batch that merges the value at key with the new
831 : // value. The details of the merge are dependent upon the configured merge
832 : // operator.
833 : //
834 : // It is safe to modify the contents of the arguments after Merge returns.
835 1 : func (b *Batch) Merge(key, value []byte, _ *WriteOptions) error {
836 1 : deferredOp := b.MergeDeferred(len(key), len(value))
837 1 : copy(deferredOp.Key, key)
838 1 : copy(deferredOp.Value, value)
839 1 : // TODO(peter): Manually inline DeferredBatchOp.Finish(). Mid-stack inlining
840 1 : // in go1.13 will remove the need for this.
841 1 : if b.index != nil {
842 1 : if err := b.index.Add(deferredOp.offset); err != nil {
843 0 : return err
844 0 : }
845 : }
846 1 : return nil
847 : }
848 :
849 : // MergeDeferred is similar to Merge in that it adds a merge operation to the
850 : // batch, except it only takes in key/value lengths instead of complete slices,
851 : // letting the caller encode into those objects and then call Finish() on the
852 : // returned object.
853 1 : func (b *Batch) MergeDeferred(keyLen, valueLen int) *DeferredBatchOp {
854 1 : b.prepareDeferredKeyValueRecord(keyLen, valueLen, InternalKeyKindMerge)
855 1 : b.deferredOp.index = b.index
856 1 : return &b.deferredOp
857 1 : }
858 :
859 : // Delete adds an action to the batch that deletes the entry for key.
860 : //
861 : // It is safe to modify the contents of the arguments after Delete returns.
862 1 : func (b *Batch) Delete(key []byte, _ *WriteOptions) error {
863 1 : deferredOp := b.DeleteDeferred(len(key))
864 1 : copy(deferredOp.Key, key)
865 1 : // TODO(peter): Manually inline DeferredBatchOp.Finish(). Mid-stack inlining
866 1 : // in go1.13 will remove the need for this.
867 1 : if b.index != nil {
868 1 : if err := b.index.Add(deferredOp.offset); err != nil {
869 0 : return err
870 0 : }
871 : }
872 1 : return nil
873 : }
874 :
875 : // DeleteDeferred is similar to Delete in that it adds a delete operation to
876 : // the batch, except it only takes in key/value lengths instead of complete
877 : // slices, letting the caller encode into those objects and then call Finish()
878 : // on the returned object.
879 1 : func (b *Batch) DeleteDeferred(keyLen int) *DeferredBatchOp {
880 1 : b.prepareDeferredKeyRecord(keyLen, InternalKeyKindDelete)
881 1 : b.deferredOp.index = b.index
882 1 : return &b.deferredOp
883 1 : }
884 :
885 : // DeleteSized behaves identically to Delete, but takes an additional
886 : // argument indicating the size of the value being deleted. DeleteSized
887 : // should be preferred when the caller has the expectation that there exists
888 : // a single internal KV pair for the key (eg, the key has not been
889 : // overwritten recently), and the caller knows the size of its value.
890 : //
891 : // DeleteSized will record the value size within the tombstone and use it to
892 : // inform compaction-picking heuristics which strive to reduce space
893 : // amplification in the LSM. This "calling your shot" mechanic allows the
894 : // storage engine to more accurately estimate and reduce space amplification.
895 : //
896 : // It is safe to modify the contents of the arguments after DeleteSized
897 : // returns.
898 1 : func (b *Batch) DeleteSized(key []byte, deletedValueSize uint32, _ *WriteOptions) error {
899 1 : deferredOp := b.DeleteSizedDeferred(len(key), deletedValueSize)
900 1 : copy(b.deferredOp.Key, key)
901 1 : // TODO(peter): Manually inline DeferredBatchOp.Finish(). Check if in a
902 1 : // later Go release this is unnecessary.
903 1 : if b.index != nil {
904 1 : if err := b.index.Add(deferredOp.offset); err != nil {
905 0 : return err
906 0 : }
907 : }
908 1 : return nil
909 : }
910 :
911 : // DeleteSizedDeferred is similar to DeleteSized in that it adds a sized delete
912 : // operation to the batch, except it only takes in key length instead of a
913 : // complete key slice, letting the caller encode into the DeferredBatchOp.Key
914 : // slice and then call Finish() on the returned object.
915 1 : func (b *Batch) DeleteSizedDeferred(keyLen int, deletedValueSize uint32) *DeferredBatchOp {
916 1 : if b.minimumFormatMajorVersion < FormatDeleteSizedAndObsolete {
917 1 : b.minimumFormatMajorVersion = FormatDeleteSizedAndObsolete
918 1 : }
919 :
920 : // Encode the sum of the key length and the value in the value.
921 1 : v := uint64(deletedValueSize) + uint64(keyLen)
922 1 :
923 1 : // Encode `v` as a varint.
924 1 : var buf [binary.MaxVarintLen64]byte
925 1 : n := 0
926 1 : {
927 1 : x := v
928 1 : for x >= 0x80 {
929 0 : buf[n] = byte(x) | 0x80
930 0 : x >>= 7
931 0 : n++
932 0 : }
933 1 : buf[n] = byte(x)
934 1 : n++
935 : }
936 :
937 : // NB: In batch entries and sstable entries, values are stored as
938 : // varstrings. Here, the value is itself a simple varint. This results in an
939 : // unnecessary double layer of encoding:
940 : // varint(n) varint(deletedValueSize)
941 : // The first varint will always be 1-byte, since a varint-encoded uint64
942 : // will never exceed 128 bytes. This unnecessary extra byte and wrapping is
943 : // preserved to avoid special casing across the database, and in particular
944 : // in sstable block decoding which is performance sensitive.
945 1 : b.prepareDeferredKeyValueRecord(keyLen, n, InternalKeyKindDeleteSized)
946 1 : b.deferredOp.index = b.index
947 1 : copy(b.deferredOp.Value, buf[:n])
948 1 : return &b.deferredOp
949 : }
950 :
951 : // SingleDelete adds an action to the batch that single deletes the entry for key.
952 : // See Writer.SingleDelete for more details on the semantics of SingleDelete.
953 : //
954 : // It is safe to modify the contents of the arguments after SingleDelete returns.
955 1 : func (b *Batch) SingleDelete(key []byte, _ *WriteOptions) error {
956 1 : deferredOp := b.SingleDeleteDeferred(len(key))
957 1 : copy(deferredOp.Key, key)
958 1 : // TODO(peter): Manually inline DeferredBatchOp.Finish(). Mid-stack inlining
959 1 : // in go1.13 will remove the need for this.
960 1 : if b.index != nil {
961 1 : if err := b.index.Add(deferredOp.offset); err != nil {
962 0 : return err
963 0 : }
964 : }
965 1 : return nil
966 : }
967 :
968 : // SingleDeleteDeferred is similar to SingleDelete in that it adds a single delete
969 : // operation to the batch, except it only takes in key/value lengths instead of
970 : // complete slices, letting the caller encode into those objects and then call
971 : // Finish() on the returned object.
972 1 : func (b *Batch) SingleDeleteDeferred(keyLen int) *DeferredBatchOp {
973 1 : b.prepareDeferredKeyRecord(keyLen, InternalKeyKindSingleDelete)
974 1 : b.deferredOp.index = b.index
975 1 : return &b.deferredOp
976 1 : }
977 :
978 : // DeleteRange deletes all of the point keys (and values) in the range
979 : // [start,end) (inclusive on start, exclusive on end). DeleteRange does NOT
980 : // delete overlapping range keys (eg, keys set via RangeKeySet).
981 : //
982 : // It is safe to modify the contents of the arguments after DeleteRange
983 : // returns.
984 1 : func (b *Batch) DeleteRange(start, end []byte, _ *WriteOptions) error {
985 1 : deferredOp := b.DeleteRangeDeferred(len(start), len(end))
986 1 : copy(deferredOp.Key, start)
987 1 : copy(deferredOp.Value, end)
988 1 : // TODO(peter): Manually inline DeferredBatchOp.Finish(). Mid-stack inlining
989 1 : // in go1.13 will remove the need for this.
990 1 : if deferredOp.index != nil {
991 1 : if err := deferredOp.index.Add(deferredOp.offset); err != nil {
992 0 : return err
993 0 : }
994 : }
995 1 : return nil
996 : }
997 :
998 : // DeleteRangeDeferred is similar to DeleteRange in that it adds a delete range
999 : // operation to the batch, except it only takes in key lengths instead of
1000 : // complete slices, letting the caller encode into those objects and then call
1001 : // Finish() on the returned object. Note that DeferredBatchOp.Key should be
1002 : // populated with the start key, and DeferredBatchOp.Value should be populated
1003 : // with the end key.
1004 1 : func (b *Batch) DeleteRangeDeferred(startLen, endLen int) *DeferredBatchOp {
1005 1 : b.prepareDeferredKeyValueRecord(startLen, endLen, InternalKeyKindRangeDelete)
1006 1 : b.countRangeDels++
1007 1 : if b.index != nil {
1008 1 : b.tombstones = nil
1009 1 : b.tombstonesSeqNum = 0
1010 1 : // Range deletions are rare, so we lazily allocate the index for them.
1011 1 : if b.rangeDelIndex == nil {
1012 1 : b.rangeDelIndex = batchskl.NewSkiplist(&b.data, b.comparer.Compare, b.comparer.AbbreviatedKey)
1013 1 : }
1014 1 : b.deferredOp.index = b.rangeDelIndex
1015 : }
1016 1 : return &b.deferredOp
1017 : }
1018 :
1019 : // RangeKeySet sets a range key mapping the key range [start, end) at the MVCC
1020 : // timestamp suffix to value. The suffix is optional. If any portion of the key
1021 : // range [start, end) is already set by a range key with the same suffix value,
1022 : // RangeKeySet overrides it.
1023 : //
1024 : // It is safe to modify the contents of the arguments after RangeKeySet returns.
1025 1 : func (b *Batch) RangeKeySet(start, end, suffix, value []byte, _ *WriteOptions) error {
1026 1 : if invariants.Enabled && b.db != nil {
1027 1 : // RangeKeySet is only supported on prefix keys.
1028 1 : if b.db.opts.Comparer.Split(start) != len(start) {
1029 0 : panic("RangeKeySet called with suffixed start key")
1030 : }
1031 1 : if b.db.opts.Comparer.Split(end) != len(end) {
1032 0 : panic("RangeKeySet called with suffixed end key")
1033 : }
1034 : }
1035 1 : suffixValues := [1]rangekey.SuffixValue{{Suffix: suffix, Value: value}}
1036 1 : internalValueLen := rangekey.EncodedSetValueLen(end, suffixValues[:])
1037 1 :
1038 1 : deferredOp := b.rangeKeySetDeferred(len(start), internalValueLen)
1039 1 : copy(deferredOp.Key, start)
1040 1 : n := rangekey.EncodeSetValue(deferredOp.Value, end, suffixValues[:])
1041 1 : if n != internalValueLen {
1042 0 : panic("unexpected internal value length mismatch")
1043 : }
1044 :
1045 : // Manually inline DeferredBatchOp.Finish().
1046 1 : if deferredOp.index != nil {
1047 1 : if err := deferredOp.index.Add(deferredOp.offset); err != nil {
1048 0 : return err
1049 0 : }
1050 : }
1051 1 : return nil
1052 : }
1053 :
1054 1 : func (b *Batch) rangeKeySetDeferred(startLen, internalValueLen int) *DeferredBatchOp {
1055 1 : b.prepareDeferredKeyValueRecord(startLen, internalValueLen, InternalKeyKindRangeKeySet)
1056 1 : b.incrementRangeKeysCount()
1057 1 : return &b.deferredOp
1058 1 : }
1059 :
1060 1 : func (b *Batch) incrementRangeKeysCount() {
1061 1 : b.countRangeKeys++
1062 1 : if b.index != nil {
1063 1 : b.rangeKeys = nil
1064 1 : b.rangeKeysSeqNum = 0
1065 1 : // Range keys are rare, so we lazily allocate the index for them.
1066 1 : if b.rangeKeyIndex == nil {
1067 1 : b.rangeKeyIndex = batchskl.NewSkiplist(&b.data, b.comparer.Compare, b.comparer.AbbreviatedKey)
1068 1 : }
1069 1 : b.deferredOp.index = b.rangeKeyIndex
1070 : }
1071 : }
1072 :
1073 : // RangeKeyUnset removes a range key mapping the key range [start, end) at the
1074 : // MVCC timestamp suffix. The suffix may be omitted to remove an unsuffixed
1075 : // range key. RangeKeyUnset only removes portions of range keys that fall within
1076 : // the [start, end) key span, and only range keys with suffixes that exactly
1077 : // match the unset suffix.
1078 : //
1079 : // It is safe to modify the contents of the arguments after RangeKeyUnset
1080 : // returns.
1081 1 : func (b *Batch) RangeKeyUnset(start, end, suffix []byte, _ *WriteOptions) error {
1082 1 : if invariants.Enabled && b.db != nil {
1083 1 : // RangeKeyUnset is only supported on prefix keys.
1084 1 : if b.db.opts.Comparer.Split(start) != len(start) {
1085 0 : panic("RangeKeyUnset called with suffixed start key")
1086 : }
1087 1 : if b.db.opts.Comparer.Split(end) != len(end) {
1088 0 : panic("RangeKeyUnset called with suffixed end key")
1089 : }
1090 : }
1091 1 : suffixes := [1][]byte{suffix}
1092 1 : internalValueLen := rangekey.EncodedUnsetValueLen(end, suffixes[:])
1093 1 :
1094 1 : deferredOp := b.rangeKeyUnsetDeferred(len(start), internalValueLen)
1095 1 : copy(deferredOp.Key, start)
1096 1 : n := rangekey.EncodeUnsetValue(deferredOp.Value, end, suffixes[:])
1097 1 : if n != internalValueLen {
1098 0 : panic("unexpected internal value length mismatch")
1099 : }
1100 :
1101 : // Manually inline DeferredBatchOp.Finish()
1102 1 : if deferredOp.index != nil {
1103 1 : if err := deferredOp.index.Add(deferredOp.offset); err != nil {
1104 0 : return err
1105 0 : }
1106 : }
1107 1 : return nil
1108 : }
1109 :
1110 1 : func (b *Batch) rangeKeyUnsetDeferred(startLen, internalValueLen int) *DeferredBatchOp {
1111 1 : b.prepareDeferredKeyValueRecord(startLen, internalValueLen, InternalKeyKindRangeKeyUnset)
1112 1 : b.incrementRangeKeysCount()
1113 1 : return &b.deferredOp
1114 1 : }
1115 :
1116 : // RangeKeyDelete deletes all of the range keys in the range [start,end)
1117 : // (inclusive on start, exclusive on end). It does not delete point keys (for
1118 : // that use DeleteRange). RangeKeyDelete removes all range keys within the
1119 : // bounds, including those with or without suffixes.
1120 : //
1121 : // It is safe to modify the contents of the arguments after RangeKeyDelete
1122 : // returns.
1123 1 : func (b *Batch) RangeKeyDelete(start, end []byte, _ *WriteOptions) error {
1124 1 : if invariants.Enabled && b.db != nil {
1125 1 : // RangeKeyDelete is only supported on prefix keys.
1126 1 : if b.db.opts.Comparer.Split(start) != len(start) {
1127 0 : panic("RangeKeyDelete called with suffixed start key")
1128 : }
1129 1 : if b.db.opts.Comparer.Split(end) != len(end) {
1130 0 : panic("RangeKeyDelete called with suffixed end key")
1131 : }
1132 : }
1133 1 : deferredOp := b.RangeKeyDeleteDeferred(len(start), len(end))
1134 1 : copy(deferredOp.Key, start)
1135 1 : copy(deferredOp.Value, end)
1136 1 : // Manually inline DeferredBatchOp.Finish().
1137 1 : if deferredOp.index != nil {
1138 1 : if err := deferredOp.index.Add(deferredOp.offset); err != nil {
1139 0 : return err
1140 0 : }
1141 : }
1142 1 : return nil
1143 : }
1144 :
1145 : // RangeKeyDeleteDeferred is similar to RangeKeyDelete in that it adds an
1146 : // operation to delete range keys to the batch, except it only takes in key
1147 : // lengths instead of complete slices, letting the caller encode into those
1148 : // objects and then call Finish() on the returned object. Note that
1149 : // DeferredBatchOp.Key should be populated with the start key, and
1150 : // DeferredBatchOp.Value should be populated with the end key.
1151 1 : func (b *Batch) RangeKeyDeleteDeferred(startLen, endLen int) *DeferredBatchOp {
1152 1 : b.prepareDeferredKeyValueRecord(startLen, endLen, InternalKeyKindRangeKeyDelete)
1153 1 : b.incrementRangeKeysCount()
1154 1 : return &b.deferredOp
1155 1 : }
1156 :
1157 : // LogData adds the specified to the batch. The data will be written to the
1158 : // WAL, but not added to memtables or sstables. Log data is never indexed,
1159 : // which makes it useful for testing WAL performance.
1160 : //
1161 : // It is safe to modify the contents of the argument after LogData returns.
1162 1 : func (b *Batch) LogData(data []byte, _ *WriteOptions) error {
1163 1 : origCount, origMemTableSize := b.count, b.memTableSize
1164 1 : b.prepareDeferredKeyRecord(len(data), InternalKeyKindLogData)
1165 1 : copy(b.deferredOp.Key, data)
1166 1 : // Since LogData only writes to the WAL and does not affect the memtable, we
1167 1 : // restore b.count and b.memTableSize to their origin values. Note that
1168 1 : // Batch.count only refers to records that are added to the memtable.
1169 1 : b.count, b.memTableSize = origCount, origMemTableSize
1170 1 : return nil
1171 1 : }
1172 :
1173 : // IngestSST adds the FileNum for an sstable to the batch. The data will only be
1174 : // written to the WAL (not added to memtables or sstables).
1175 1 : func (b *Batch) ingestSST(fileNum base.FileNum) {
1176 1 : if b.Empty() {
1177 1 : b.ingestedSSTBatch = true
1178 1 : } else if !b.ingestedSSTBatch {
1179 0 : // Batch contains other key kinds.
1180 0 : panic("pebble: invalid call to ingestSST")
1181 : }
1182 :
1183 1 : origMemTableSize := b.memTableSize
1184 1 : var buf [binary.MaxVarintLen64]byte
1185 1 : length := binary.PutUvarint(buf[:], uint64(fileNum))
1186 1 : b.prepareDeferredKeyRecord(length, InternalKeyKindIngestSST)
1187 1 : copy(b.deferredOp.Key, buf[:length])
1188 1 : // Since IngestSST writes only to the WAL and does not affect the memtable,
1189 1 : // we restore b.memTableSize to its original value. Note that Batch.count
1190 1 : // is not reset because for the InternalKeyKindIngestSST the count is the
1191 1 : // number of sstable paths which have been added to the batch.
1192 1 : b.memTableSize = origMemTableSize
1193 1 : b.minimumFormatMajorVersion = FormatFlushableIngest
1194 : }
1195 :
1196 : // Empty returns true if the batch is empty, and false otherwise.
1197 1 : func (b *Batch) Empty() bool {
1198 1 : return batchrepr.IsEmpty(b.data)
1199 1 : }
1200 :
1201 : // Len returns the current size of the batch in bytes.
1202 0 : func (b *Batch) Len() int {
1203 0 : return max(batchrepr.HeaderLen, len(b.data))
1204 0 : }
1205 :
1206 : // Repr returns the underlying batch representation. It is not safe to modify
1207 : // the contents. Reset() will not change the contents of the returned value,
1208 : // though any other mutation operation may do so.
1209 1 : func (b *Batch) Repr() []byte {
1210 1 : if len(b.data) == 0 {
1211 0 : b.init(batchrepr.HeaderLen)
1212 0 : }
1213 1 : batchrepr.SetCount(b.data, b.Count())
1214 1 : return b.data
1215 : }
1216 :
1217 : // SetRepr sets the underlying batch representation. The batch takes ownership
1218 : // of the supplied slice. It is not safe to modify it afterwards until the
1219 : // Batch is no longer in use.
1220 : //
1221 : // SetRepr may return ErrInvalidBatch if the supplied slice fails to decode in
1222 : // any way. It will not return an error in any other circumstance.
1223 1 : func (b *Batch) SetRepr(data []byte) error {
1224 1 : h, ok := batchrepr.ReadHeader(data)
1225 1 : if !ok {
1226 0 : return ErrInvalidBatch
1227 0 : }
1228 1 : b.data = data
1229 1 : b.count = uint64(h.Count)
1230 1 : var err error
1231 1 : if b.db != nil {
1232 1 : // Only track memTableSize for batches that will be committed to the DB.
1233 1 : err = b.refreshMemTableSize()
1234 1 : }
1235 1 : return err
1236 : }
1237 :
1238 : // NewIter returns an iterator that is unpositioned (Iterator.Valid() will
1239 : // return false). The iterator can be positioned via a call to SeekGE,
1240 : // SeekPrefixGE, SeekLT, First or Last. Only indexed batches support iterators.
1241 : //
1242 : // The returned Iterator observes all of the Batch's existing mutations, but no
1243 : // later mutations. Its view can be refreshed via RefreshBatchSnapshot or
1244 : // SetOptions().
1245 1 : func (b *Batch) NewIter(o *IterOptions) (*Iterator, error) {
1246 1 : return b.NewIterWithContext(context.Background(), o)
1247 1 : }
1248 :
1249 : // NewIterWithContext is like NewIter, and additionally accepts a context for
1250 : // tracing.
1251 1 : func (b *Batch) NewIterWithContext(ctx context.Context, o *IterOptions) (*Iterator, error) {
1252 1 : if b.index == nil {
1253 0 : return nil, ErrNotIndexed
1254 0 : }
1255 1 : return b.db.newIter(ctx, b, newIterOpts{}, o), nil
1256 : }
1257 :
1258 : // NewBatchOnlyIter constructs an iterator that only reads the contents of the
1259 : // batch, and does not overlay the batch mutations on top of the DB state.
1260 : //
1261 : // The returned Iterator observes all of the Batch's existing mutations, but
1262 : // no later mutations. Its view can be refreshed via RefreshBatchSnapshot or
1263 : // SetOptions().
1264 0 : func (b *Batch) NewBatchOnlyIter(ctx context.Context, o *IterOptions) (*Iterator, error) {
1265 0 : if b.index == nil {
1266 0 : return nil, ErrNotIndexed
1267 0 : }
1268 0 : return b.db.newIter(ctx, b, newIterOpts{batch: batchIterOpts{batchOnly: true}}, o), nil
1269 : }
1270 :
1271 : // newInternalIter creates a new internalIterator that iterates over the
1272 : // contents of the batch.
1273 1 : func (b *Batch) newInternalIter(o *IterOptions) *batchIter {
1274 1 : iter := &batchIter{}
1275 1 : b.initInternalIter(o, iter)
1276 1 : return iter
1277 1 : }
1278 :
1279 1 : func (b *Batch) initInternalIter(o *IterOptions, iter *batchIter) {
1280 1 : *iter = batchIter{
1281 1 : batch: b,
1282 1 : iter: b.index.NewIter(o.GetLowerBound(), o.GetUpperBound()),
1283 1 : // NB: We explicitly do not propagate the batch snapshot to the point
1284 1 : // key iterator. Filtering point keys within the batch iterator can
1285 1 : // cause pathological behavior where a batch iterator advances
1286 1 : // significantly farther than necessary filtering many batch keys that
1287 1 : // are not visible at the batch sequence number. Instead, the merging
1288 1 : // iterator enforces bounds.
1289 1 : //
1290 1 : // For example, consider an engine that contains the committed keys
1291 1 : // 'bar' and 'bax', with no keys between them. Consider a batch
1292 1 : // containing keys 1,000 keys within the range [a,z]. All of the
1293 1 : // batch keys were added to the batch after the iterator was
1294 1 : // constructed, so they are not visible to the iterator. A call to
1295 1 : // SeekGE('bax') would seek the LSM iterators and discover the key
1296 1 : // 'bax'. It would also seek the batch iterator, landing on the key
1297 1 : // 'baz' but discover it that it's not visible. The batch iterator would
1298 1 : // next through the rest of the batch's keys, only to discover there are
1299 1 : // no visible keys greater than or equal to 'bax'.
1300 1 : //
1301 1 : // Filtering these batch points within the merging iterator ensures that
1302 1 : // the batch iterator never needs to iterate beyond 'baz', because it
1303 1 : // already found a smaller, visible key 'bax'.
1304 1 : snapshot: base.SeqNumMax,
1305 1 : }
1306 1 : }
1307 :
1308 1 : func (b *Batch) newRangeDelIter(o *IterOptions, batchSnapshot base.SeqNum) *keyspan.Iter {
1309 1 : // Construct an iterator even if rangeDelIndex is nil, because it is allowed
1310 1 : // to refresh later, so we need the container to exist.
1311 1 : iter := new(keyspan.Iter)
1312 1 : b.initRangeDelIter(o, iter, batchSnapshot)
1313 1 : return iter
1314 1 : }
1315 :
1316 1 : func (b *Batch) initRangeDelIter(_ *IterOptions, iter *keyspan.Iter, batchSnapshot base.SeqNum) {
1317 1 : if b.rangeDelIndex == nil {
1318 1 : iter.Init(b.comparer.Compare, nil)
1319 1 : return
1320 1 : }
1321 :
1322 : // Fragment the range tombstones the first time a range deletion iterator is
1323 : // requested. The cached tombstones are invalidated if another range
1324 : // deletion tombstone is added to the batch. This cache is only guaranteed
1325 : // to be correct if we're opening an iterator to read at a batch sequence
1326 : // number at least as high as tombstonesSeqNum. The cache is guaranteed to
1327 : // include all tombstones up to tombstonesSeqNum, and if any additional
1328 : // tombstones were added after that sequence number the cache would've been
1329 : // cleared.
1330 1 : nextSeqNum := b.nextSeqNum()
1331 1 : if b.tombstones != nil && b.tombstonesSeqNum <= batchSnapshot {
1332 1 : iter.Init(b.comparer.Compare, b.tombstones)
1333 1 : return
1334 1 : }
1335 :
1336 1 : tombstones := make([]keyspan.Span, 0, b.countRangeDels)
1337 1 : frag := &keyspan.Fragmenter{
1338 1 : Cmp: b.comparer.Compare,
1339 1 : Format: b.comparer.FormatKey,
1340 1 : Emit: func(s keyspan.Span) {
1341 1 : tombstones = append(tombstones, s)
1342 1 : },
1343 : }
1344 1 : it := &batchIter{
1345 1 : batch: b,
1346 1 : iter: b.rangeDelIndex.NewIter(nil, nil),
1347 1 : snapshot: batchSnapshot,
1348 1 : }
1349 1 : fragmentRangeDels(frag, it, int(b.countRangeDels))
1350 1 : iter.Init(b.comparer.Compare, tombstones)
1351 1 :
1352 1 : // If we just read all the tombstones in the batch (eg, batchSnapshot was
1353 1 : // set to b.nextSeqNum()), then cache the tombstones so that a subsequent
1354 1 : // call to initRangeDelIter may use them without refragmenting.
1355 1 : if nextSeqNum == batchSnapshot {
1356 1 : b.tombstones = tombstones
1357 1 : b.tombstonesSeqNum = nextSeqNum
1358 1 : }
1359 : }
1360 :
1361 1 : func fragmentRangeDels(frag *keyspan.Fragmenter, it internalIterator, count int) {
1362 1 : // The memory management here is a bit subtle. The keys and values returned
1363 1 : // by the iterator are slices in Batch.data. Thus the fragmented tombstones
1364 1 : // are slices within Batch.data. If additional entries are added to the
1365 1 : // Batch, Batch.data may be reallocated. The references in the fragmented
1366 1 : // tombstones will remain valid, pointing into the old Batch.data. GC for
1367 1 : // the win.
1368 1 :
1369 1 : // Use a single []keyspan.Key buffer to avoid allocating many
1370 1 : // individual []keyspan.Key slices with a single element each.
1371 1 : keyBuf := make([]keyspan.Key, 0, count)
1372 1 : for kv := it.First(); kv != nil; kv = it.Next() {
1373 1 : s := rangedel.Decode(kv.K, kv.InPlaceValue(), keyBuf)
1374 1 : keyBuf = s.Keys[len(s.Keys):]
1375 1 :
1376 1 : // Set a fixed capacity to avoid accidental overwriting.
1377 1 : s.Keys = s.Keys[:len(s.Keys):len(s.Keys)]
1378 1 : frag.Add(s)
1379 1 : }
1380 1 : frag.Finish()
1381 : }
1382 :
1383 1 : func (b *Batch) newRangeKeyIter(o *IterOptions, batchSnapshot base.SeqNum) *keyspan.Iter {
1384 1 : // Construct an iterator even if rangeKeyIndex is nil, because it is allowed
1385 1 : // to refresh later, so we need the container to exist.
1386 1 : iter := new(keyspan.Iter)
1387 1 : b.initRangeKeyIter(o, iter, batchSnapshot)
1388 1 : return iter
1389 1 : }
1390 :
1391 1 : func (b *Batch) initRangeKeyIter(_ *IterOptions, iter *keyspan.Iter, batchSnapshot base.SeqNum) {
1392 1 : if b.rangeKeyIndex == nil {
1393 1 : iter.Init(b.comparer.Compare, nil)
1394 1 : return
1395 1 : }
1396 :
1397 : // Fragment the range keys the first time a range key iterator is requested.
1398 : // The cached spans are invalidated if another range key is added to the
1399 : // batch. This cache is only guaranteed to be correct if we're opening an
1400 : // iterator to read at a batch sequence number at least as high as
1401 : // rangeKeysSeqNum. The cache is guaranteed to include all range keys up to
1402 : // rangeKeysSeqNum, and if any additional range keys were added after that
1403 : // sequence number the cache would've been cleared.
1404 1 : nextSeqNum := b.nextSeqNum()
1405 1 : if b.rangeKeys != nil && b.rangeKeysSeqNum <= batchSnapshot {
1406 0 : iter.Init(b.comparer.Compare, b.rangeKeys)
1407 0 : return
1408 0 : }
1409 :
1410 1 : rangeKeys := make([]keyspan.Span, 0, b.countRangeKeys)
1411 1 : frag := &keyspan.Fragmenter{
1412 1 : Cmp: b.comparer.Compare,
1413 1 : Format: b.comparer.FormatKey,
1414 1 : Emit: func(s keyspan.Span) {
1415 1 : rangeKeys = append(rangeKeys, s)
1416 1 : },
1417 : }
1418 1 : it := &batchIter{
1419 1 : batch: b,
1420 1 : iter: b.rangeKeyIndex.NewIter(nil, nil),
1421 1 : snapshot: batchSnapshot,
1422 1 : }
1423 1 : fragmentRangeKeys(frag, it, int(b.countRangeKeys))
1424 1 : iter.Init(b.comparer.Compare, rangeKeys)
1425 1 :
1426 1 : // If we just read all the range keys in the batch (eg, batchSnapshot was
1427 1 : // set to b.nextSeqNum()), then cache the range keys so that a subsequent
1428 1 : // call to initRangeKeyIter may use them without refragmenting.
1429 1 : if nextSeqNum == batchSnapshot {
1430 1 : b.rangeKeys = rangeKeys
1431 1 : b.rangeKeysSeqNum = nextSeqNum
1432 1 : }
1433 : }
1434 :
1435 1 : func fragmentRangeKeys(frag *keyspan.Fragmenter, it internalIterator, count int) error {
1436 1 : // The memory management here is a bit subtle. The keys and values
1437 1 : // returned by the iterator are slices in Batch.data. Thus the
1438 1 : // fragmented key spans are slices within Batch.data. If additional
1439 1 : // entries are added to the Batch, Batch.data may be reallocated. The
1440 1 : // references in the fragmented keys will remain valid, pointing into
1441 1 : // the old Batch.data. GC for the win.
1442 1 :
1443 1 : // Use a single []keyspan.Key buffer to avoid allocating many
1444 1 : // individual []keyspan.Key slices with a single element each.
1445 1 : keyBuf := make([]keyspan.Key, 0, count)
1446 1 : for kv := it.First(); kv != nil; kv = it.Next() {
1447 1 : s, err := rangekey.Decode(kv.K, kv.InPlaceValue(), keyBuf)
1448 1 : if err != nil {
1449 0 : return err
1450 0 : }
1451 1 : keyBuf = s.Keys[len(s.Keys):]
1452 1 :
1453 1 : // Set a fixed capacity to avoid accidental overwriting.
1454 1 : s.Keys = s.Keys[:len(s.Keys):len(s.Keys)]
1455 1 : frag.Add(s)
1456 : }
1457 1 : frag.Finish()
1458 1 : return nil
1459 : }
1460 :
1461 : // Commit applies the batch to its parent writer.
1462 1 : func (b *Batch) Commit(o *WriteOptions) error {
1463 1 : return b.db.Apply(b, o)
1464 1 : }
1465 :
1466 : // Close closes the batch without committing it.
1467 1 : func (b *Batch) Close() error {
1468 1 : // The storage engine commit pipeline may retain a pointer to b.data beyond
1469 1 : // when Commit() returns. This is possible when configured for WAL failover;
1470 1 : // we don't know if we might need to read the batch data again until the
1471 1 : // batch has been durably synced [even if the committer doesn't care to wait
1472 1 : // for the sync and Sync()=false].
1473 1 : //
1474 1 : // We still want to recycle these batches. The b.lifecycle atomic negotiates
1475 1 : // the batch's lifecycle. If the commit pipeline still might read b.data,
1476 1 : // b.lifecycle will be nonzeroed [the low bits hold a ref count].
1477 1 : for {
1478 1 : v := b.lifecycle.Load()
1479 1 : switch {
1480 1 : case v == 0:
1481 1 : // A zero value indicates that the commit pipeline has no
1482 1 : // outstanding references to the batch. The commit pipeline is
1483 1 : // required to acquire a ref synchronously, so there is no risk that
1484 1 : // the commit pipeline will grab a ref after the call to release. We
1485 1 : // can simply release the batch.
1486 1 : b.release()
1487 1 : return nil
1488 0 : case (v & batchClosedBit) != 0:
1489 0 : // The batch has a batchClosedBit: This batch has already been closed.
1490 0 : return ErrClosed
1491 1 : default:
1492 1 : // There's an outstanding reference. Set the batch released bit so
1493 1 : // that the commit pipeline knows it should release the batch when
1494 1 : // it unrefs.
1495 1 : if b.lifecycle.CompareAndSwap(v, v|batchClosedBit) {
1496 1 : return nil
1497 1 : }
1498 : // CAS Failed—this indicates the outstanding reference just
1499 : // decremented (or the caller illegally closed the batch twice).
1500 : // Loop to reload.
1501 : }
1502 : }
1503 : }
1504 :
1505 : // Indexed returns true if the batch is indexed (i.e. supports read
1506 : // operations).
1507 1 : func (b *Batch) Indexed() bool {
1508 1 : return b.index != nil
1509 1 : }
1510 :
1511 : // init ensures that the batch data slice is initialized to meet the
1512 : // minimum required size and allocates space for the batch header.
1513 1 : func (b *Batch) init(size int) {
1514 1 : b.opts.ensureDefaults()
1515 1 : n := b.opts.initialSizeBytes
1516 1 : for n < size {
1517 0 : n *= 2
1518 0 : }
1519 1 : if cap(b.data) < n {
1520 1 : b.data = rawalloc.New(batchrepr.HeaderLen, n)
1521 1 : }
1522 1 : b.data = b.data[:batchrepr.HeaderLen]
1523 1 : clear(b.data) // Zero the sequence number in the header
1524 : }
1525 :
1526 : // Reset resets the batch for reuse. The underlying byte slice (that is
1527 : // returned by Repr()) may not be modified. It is only necessary to call this
1528 : // method if a batch is explicitly being reused. Close automatically takes are
1529 : // of releasing resources when appropriate for batches that are internally
1530 : // being reused.
1531 0 : func (b *Batch) Reset() {
1532 0 : // In some configurations (WAL failover) the commit pipeline may retain
1533 0 : // b.data beyond a call to commit the batch. When this happens, b.lifecycle
1534 0 : // is nonzero (see the comment above b.lifecycle). In this case it's unsafe
1535 0 : // to mutate b.data, so we discard it. Note that Reset must not be called on
1536 0 : // a closed batch, so v > 0 implies a non-zero ref count and not
1537 0 : // batchClosedBit being set.
1538 0 : if v := b.lifecycle.Load(); v > 0 {
1539 0 : b.data = nil
1540 0 : }
1541 0 : b.reset()
1542 : }
1543 :
1544 1 : func (b *Batch) reset() {
1545 1 : // Zero out the struct, retaining only the fields necessary for manual
1546 1 : // reuse.
1547 1 : b.batchInternal = batchInternal{
1548 1 : data: b.data,
1549 1 : comparer: b.comparer,
1550 1 : opts: b.opts,
1551 1 : index: b.index,
1552 1 : db: b.db,
1553 1 : }
1554 1 : b.applied.Store(false)
1555 1 : if b.data != nil {
1556 1 : if cap(b.data) > b.opts.maxRetainedSizeBytes {
1557 0 : // If the capacity of the buffer is larger than our maximum
1558 0 : // retention size, don't re-use it. Let it be GC-ed instead.
1559 0 : // This prevents the memory from an unusually large batch from
1560 0 : // being held on to indefinitely.
1561 0 : b.data = nil
1562 1 : } else {
1563 1 : // Otherwise, reset the buffer for re-use.
1564 1 : b.data = b.data[:batchrepr.HeaderLen]
1565 1 : clear(b.data)
1566 1 : }
1567 : }
1568 1 : if b.index != nil {
1569 1 : b.index.Init(&b.data, b.comparer.Compare, b.comparer.AbbreviatedKey)
1570 1 : }
1571 : }
1572 :
1573 1 : func (b *Batch) grow(n int) {
1574 1 : newSize := len(b.data) + n
1575 1 : if uint64(newSize) >= maxBatchSize {
1576 0 : panic(ErrBatchTooLarge)
1577 : }
1578 1 : if newSize > cap(b.data) {
1579 0 : newCap := 2 * cap(b.data)
1580 0 : for newCap < newSize {
1581 0 : newCap *= 2
1582 0 : }
1583 0 : newData := rawalloc.New(len(b.data), newCap)
1584 0 : copy(newData, b.data)
1585 0 : b.data = newData
1586 : }
1587 1 : b.data = b.data[:newSize]
1588 : }
1589 :
1590 1 : func (b *Batch) setSeqNum(seqNum base.SeqNum) {
1591 1 : batchrepr.SetSeqNum(b.data, seqNum)
1592 1 : }
1593 :
1594 : // SeqNum returns the batch sequence number which is applied to the first
1595 : // record in the batch. The sequence number is incremented for each subsequent
1596 : // record. It returns zero if the batch is empty.
1597 1 : func (b *Batch) SeqNum() base.SeqNum {
1598 1 : if len(b.data) == 0 {
1599 0 : b.init(batchrepr.HeaderLen)
1600 0 : }
1601 1 : return batchrepr.ReadSeqNum(b.data)
1602 : }
1603 :
1604 1 : func (b *Batch) setCount(v uint32) {
1605 1 : b.count = uint64(v)
1606 1 : }
1607 :
1608 : // Count returns the count of memtable-modifying operations in this batch. All
1609 : // operations with the except of LogData increment this count. For IngestSSTs,
1610 : // count is only used to indicate the number of SSTs ingested in the record, the
1611 : // batch isn't applied to the memtable.
1612 1 : func (b *Batch) Count() uint32 {
1613 1 : if b.count > math.MaxUint32 {
1614 0 : panic(batchrepr.ErrInvalidBatch)
1615 : }
1616 1 : return uint32(b.count)
1617 : }
1618 :
1619 : // Reader returns a batchrepr.Reader for the current batch contents. If the
1620 : // batch is mutated, the new entries will not be visible to the reader.
1621 1 : func (b *Batch) Reader() batchrepr.Reader {
1622 1 : if len(b.data) == 0 {
1623 1 : b.init(batchrepr.HeaderLen)
1624 1 : }
1625 1 : return batchrepr.Read(b.data)
1626 : }
1627 :
1628 : // SyncWait is to be used in conjunction with DB.ApplyNoSyncWait.
1629 1 : func (b *Batch) SyncWait() error {
1630 1 : now := time.Now()
1631 1 : b.fsyncWait.Wait()
1632 1 : if b.commitErr != nil {
1633 0 : b.db = nil // prevent batch reuse on error
1634 0 : }
1635 1 : waitDuration := time.Since(now)
1636 1 : b.commitStats.CommitWaitDuration += waitDuration
1637 1 : b.commitStats.TotalDuration += waitDuration
1638 1 : return b.commitErr
1639 : }
1640 :
1641 : // CommitStats returns stats related to committing the batch. Should be called
1642 : // after Batch.Commit, DB.Apply. If DB.ApplyNoSyncWait is used, should be
1643 : // called after Batch.SyncWait.
1644 0 : func (b *Batch) CommitStats() BatchCommitStats {
1645 0 : return b.commitStats
1646 0 : }
1647 :
1648 : // Note: batchIter mirrors the implementation of flushableBatchIter. Keep the
1649 : // two in sync.
1650 : type batchIter struct {
1651 : batch *Batch
1652 : iter batchskl.Iterator
1653 : kv base.InternalKV
1654 : err error
1655 : // snapshot holds a batch "sequence number" at which the batch is being
1656 : // read. This sequence number has the InternalKeySeqNumBatch bit set, so it
1657 : // encodes an offset within the batch. Only batch entries earlier than the
1658 : // offset are visible during iteration.
1659 : snapshot base.SeqNum
1660 : }
1661 :
1662 : // batchIter implements the base.InternalIterator interface.
1663 : var _ base.InternalIterator = (*batchIter)(nil)
1664 :
1665 0 : func (i *batchIter) String() string {
1666 0 : return "batch"
1667 0 : }
1668 :
1669 1 : func (i *batchIter) SeekGE(key []byte, flags base.SeekGEFlags) *base.InternalKV {
1670 1 : // Ignore TrySeekUsingNext if the view of the batch changed.
1671 1 : if flags.TrySeekUsingNext() && flags.BatchJustRefreshed() {
1672 0 : flags = flags.DisableTrySeekUsingNext()
1673 0 : }
1674 :
1675 1 : i.err = nil // clear cached iteration error
1676 1 : ikey := i.iter.SeekGE(key, flags)
1677 1 : for ikey != nil && ikey.SeqNum() >= i.snapshot {
1678 0 : ikey = i.iter.Next()
1679 0 : }
1680 1 : if ikey == nil {
1681 1 : i.kv = base.InternalKV{}
1682 1 : return nil
1683 1 : }
1684 1 : i.kv.K = *ikey
1685 1 : i.kv.V = base.MakeInPlaceValue(i.value())
1686 1 : return &i.kv
1687 : }
1688 :
1689 1 : func (i *batchIter) SeekPrefixGE(prefix, key []byte, flags base.SeekGEFlags) *base.InternalKV {
1690 1 : kv := i.SeekGE(key, flags)
1691 1 : if kv == nil {
1692 1 : return nil
1693 1 : }
1694 : // If the key doesn't have the sought prefix, return nil.
1695 1 : if !bytes.Equal(i.batch.comparer.Split.Prefix(kv.K.UserKey), prefix) {
1696 1 : i.kv = base.InternalKV{}
1697 1 : return nil
1698 1 : }
1699 1 : return kv
1700 : }
1701 :
1702 1 : func (i *batchIter) SeekLT(key []byte, flags base.SeekLTFlags) *base.InternalKV {
1703 1 : i.err = nil // clear cached iteration error
1704 1 : ikey := i.iter.SeekLT(key)
1705 1 : for ikey != nil && ikey.SeqNum() >= i.snapshot {
1706 0 : ikey = i.iter.Prev()
1707 0 : }
1708 1 : if ikey == nil {
1709 1 : i.kv = base.InternalKV{}
1710 1 : return nil
1711 1 : }
1712 1 : i.kv.K = *ikey
1713 1 : i.kv.V = base.MakeInPlaceValue(i.value())
1714 1 : return &i.kv
1715 : }
1716 :
1717 1 : func (i *batchIter) First() *base.InternalKV {
1718 1 : i.err = nil // clear cached iteration error
1719 1 : ikey := i.iter.First()
1720 1 : for ikey != nil && ikey.SeqNum() >= i.snapshot {
1721 0 : ikey = i.iter.Next()
1722 0 : }
1723 1 : if ikey == nil {
1724 1 : i.kv = base.InternalKV{}
1725 1 : return nil
1726 1 : }
1727 1 : i.kv.K = *ikey
1728 1 : i.kv.V = base.MakeInPlaceValue(i.value())
1729 1 : return &i.kv
1730 : }
1731 :
1732 1 : func (i *batchIter) Last() *base.InternalKV {
1733 1 : i.err = nil // clear cached iteration error
1734 1 : ikey := i.iter.Last()
1735 1 : for ikey != nil && ikey.SeqNum() >= i.snapshot {
1736 0 : ikey = i.iter.Prev()
1737 0 : }
1738 1 : if ikey == nil {
1739 1 : i.kv = base.InternalKV{}
1740 1 : return nil
1741 1 : }
1742 1 : i.kv.K = *ikey
1743 1 : i.kv.V = base.MakeInPlaceValue(i.value())
1744 1 : return &i.kv
1745 : }
1746 :
1747 1 : func (i *batchIter) Next() *base.InternalKV {
1748 1 : ikey := i.iter.Next()
1749 1 : for ikey != nil && ikey.SeqNum() >= i.snapshot {
1750 0 : ikey = i.iter.Next()
1751 0 : }
1752 1 : if ikey == nil {
1753 1 : i.kv = base.InternalKV{}
1754 1 : return nil
1755 1 : }
1756 1 : i.kv.K = *ikey
1757 1 : i.kv.V = base.MakeInPlaceValue(i.value())
1758 1 : return &i.kv
1759 : }
1760 :
1761 0 : func (i *batchIter) NextPrefix(succKey []byte) *base.InternalKV {
1762 0 : // Because NextPrefix was invoked `succKey` must be ≥ the key at i's current
1763 0 : // position. Seek the arena iterator using TrySeekUsingNext.
1764 0 : ikey := i.iter.SeekGE(succKey, base.SeekGEFlagsNone.EnableTrySeekUsingNext())
1765 0 : for ikey != nil && ikey.SeqNum() >= i.snapshot {
1766 0 : ikey = i.iter.Next()
1767 0 : }
1768 0 : if ikey == nil {
1769 0 : i.kv = base.InternalKV{}
1770 0 : return nil
1771 0 : }
1772 0 : i.kv.K = *ikey
1773 0 : i.kv.V = base.MakeInPlaceValue(i.value())
1774 0 : return &i.kv
1775 : }
1776 :
1777 1 : func (i *batchIter) Prev() *base.InternalKV {
1778 1 : ikey := i.iter.Prev()
1779 1 : for ikey != nil && ikey.SeqNum() >= i.snapshot {
1780 0 : ikey = i.iter.Prev()
1781 0 : }
1782 1 : if ikey == nil {
1783 1 : i.kv = base.InternalKV{}
1784 1 : return nil
1785 1 : }
1786 0 : i.kv.K = *ikey
1787 0 : i.kv.V = base.MakeInPlaceValue(i.value())
1788 0 : return &i.kv
1789 : }
1790 :
1791 1 : func (i *batchIter) value() []byte {
1792 1 : offset, _, keyEnd := i.iter.KeyInfo()
1793 1 : data := i.batch.data
1794 1 : if len(data[offset:]) == 0 {
1795 0 : i.err = base.CorruptionErrorf("corrupted batch")
1796 0 : return nil
1797 0 : }
1798 :
1799 1 : switch InternalKeyKind(data[offset]) {
1800 : case InternalKeyKindSet, InternalKeyKindMerge, InternalKeyKindRangeDelete,
1801 : InternalKeyKindRangeKeySet, InternalKeyKindRangeKeyUnset, InternalKeyKindRangeKeyDelete,
1802 1 : InternalKeyKindDeleteSized:
1803 1 : _, value, ok := batchrepr.DecodeStr(data[keyEnd:])
1804 1 : if !ok {
1805 0 : return nil
1806 0 : }
1807 1 : return value
1808 1 : default:
1809 1 : return nil
1810 : }
1811 : }
1812 :
1813 1 : func (i *batchIter) Error() error {
1814 1 : return i.err
1815 1 : }
1816 :
1817 1 : func (i *batchIter) Close() error {
1818 1 : _ = i.iter.Close()
1819 1 : return i.err
1820 1 : }
1821 :
1822 1 : func (i *batchIter) SetBounds(lower, upper []byte) {
1823 1 : i.iter.SetBounds(lower, upper)
1824 1 : }
1825 :
1826 0 : func (i *batchIter) SetContext(_ context.Context) {}
1827 :
1828 : // DebugTree is part of the InternalIterator interface.
1829 0 : func (i *batchIter) DebugTree(tp treeprinter.Node) {
1830 0 : tp.Childf("%T(%p)", i, i)
1831 0 : }
1832 :
1833 : type flushableBatchEntry struct {
1834 : // offset is the byte offset of the record within the batch repr.
1835 : offset uint32
1836 : // index is the 0-based ordinal number of the record within the batch. Used
1837 : // to compute the seqnum for the record.
1838 : index uint32
1839 : // key{Start,End} are the start and end byte offsets of the key within the
1840 : // batch repr. Cached to avoid decoding the key length on every
1841 : // comparison. The value is stored starting at keyEnd.
1842 : keyStart uint32
1843 : keyEnd uint32
1844 : }
1845 :
1846 : // flushableBatch wraps an existing batch and provides the interfaces needed
1847 : // for making the batch flushable (i.e. able to mimic a memtable).
1848 : type flushableBatch struct {
1849 : cmp Compare
1850 : comparer *base.Comparer
1851 : data []byte
1852 :
1853 : // The base sequence number for the entries in the batch. This is the same
1854 : // value as Batch.seqNum() and is cached here for performance.
1855 : seqNum base.SeqNum
1856 :
1857 : // A slice of offsets and indices for the entries in the batch. Used to
1858 : // implement flushableBatchIter. Unlike the indexing on a normal batch, a
1859 : // flushable batch is indexed such that batch entry i will be given the
1860 : // sequence number flushableBatch.seqNum+i.
1861 : //
1862 : // Sorted in increasing order of key and decreasing order of offset (since
1863 : // higher offsets correspond to higher sequence numbers).
1864 : //
1865 : // Does not include range deletion entries or range key entries.
1866 : offsets []flushableBatchEntry
1867 :
1868 : // Fragmented range deletion tombstones.
1869 : tombstones []keyspan.Span
1870 :
1871 : // Fragmented range keys.
1872 : rangeKeys []keyspan.Span
1873 : }
1874 :
1875 : var _ flushable = (*flushableBatch)(nil)
1876 :
1877 : // newFlushableBatch creates a new batch that implements the flushable
1878 : // interface. This allows the batch to act like a memtable and be placed in the
1879 : // queue of flushable memtables. Note that the flushable batch takes ownership
1880 : // of the batch data.
1881 1 : func newFlushableBatch(batch *Batch, comparer *Comparer) (*flushableBatch, error) {
1882 1 : b := &flushableBatch{
1883 1 : data: batch.data,
1884 1 : cmp: comparer.Compare,
1885 1 : comparer: comparer,
1886 1 : offsets: make([]flushableBatchEntry, 0, batch.Count()),
1887 1 : }
1888 1 : if b.data != nil {
1889 1 : // Note that this sequence number is not correct when this batch has not
1890 1 : // been applied since the sequence number has not been assigned yet. The
1891 1 : // correct sequence number will be set later. But it is correct when the
1892 1 : // batch is being replayed from the WAL.
1893 1 : b.seqNum = batch.SeqNum()
1894 1 : }
1895 1 : var rangeDelOffsets []flushableBatchEntry
1896 1 : var rangeKeyOffsets []flushableBatchEntry
1897 1 : if len(b.data) > batchrepr.HeaderLen {
1898 1 : // Non-empty batch.
1899 1 : var index uint32
1900 1 : for iter := batchrepr.Read(b.data); len(iter) > 0; {
1901 1 : offset := uintptr(unsafe.Pointer(&iter[0])) - uintptr(unsafe.Pointer(&b.data[0]))
1902 1 : kind, key, _, ok, err := iter.Next()
1903 1 : if !ok {
1904 0 : if err != nil {
1905 0 : return nil, err
1906 0 : }
1907 0 : break
1908 : }
1909 1 : entry := flushableBatchEntry{
1910 1 : offset: uint32(offset),
1911 1 : index: uint32(index),
1912 1 : }
1913 1 : if keySize := uint32(len(key)); keySize == 0 {
1914 1 : // Must add 2 to the offset. One byte encodes `kind` and the next
1915 1 : // byte encodes `0`, which is the length of the key.
1916 1 : entry.keyStart = uint32(offset) + 2
1917 1 : entry.keyEnd = entry.keyStart
1918 1 : } else {
1919 1 : entry.keyStart = uint32(uintptr(unsafe.Pointer(&key[0])) -
1920 1 : uintptr(unsafe.Pointer(&b.data[0])))
1921 1 : entry.keyEnd = entry.keyStart + keySize
1922 1 : }
1923 1 : switch kind {
1924 1 : case InternalKeyKindRangeDelete:
1925 1 : rangeDelOffsets = append(rangeDelOffsets, entry)
1926 1 : case InternalKeyKindRangeKeySet, InternalKeyKindRangeKeyUnset, InternalKeyKindRangeKeyDelete:
1927 1 : rangeKeyOffsets = append(rangeKeyOffsets, entry)
1928 1 : case InternalKeyKindLogData:
1929 1 : // Skip it; we never want to iterate over LogDatas.
1930 1 : continue
1931 : case InternalKeyKindSet, InternalKeyKindDelete, InternalKeyKindMerge,
1932 1 : InternalKeyKindSingleDelete, InternalKeyKindSetWithDelete, InternalKeyKindDeleteSized:
1933 1 : b.offsets = append(b.offsets, entry)
1934 0 : default:
1935 0 : // Note In some circumstances this might be temporary memory
1936 0 : // corruption that can be recovered by discarding the batch and
1937 0 : // trying again. In other cases, the batch repr might've been
1938 0 : // already persisted elsewhere, and we'll loop continuously trying
1939 0 : // to commit the same corrupted batch. The caller is responsible for
1940 0 : // distinguishing.
1941 0 : return nil, errors.Wrapf(ErrInvalidBatch, "unrecognized kind %v", kind)
1942 : }
1943 : // NB: index (used for entry.offset above) must not reach the
1944 : // batch.count, because the offset is used in conjunction with the
1945 : // batch's sequence number to assign sequence numbers to keys within
1946 : // the batch. If we assign KV's indexes as high as batch.count,
1947 : // we'll begin assigning keys sequence numbers that weren't
1948 : // allocated.
1949 1 : if index >= uint32(batch.count) {
1950 0 : return nil, base.AssertionFailedf("pebble: batch entry index %d ≥ batch.count %d", index, batch.count)
1951 0 : }
1952 1 : index++
1953 : }
1954 : }
1955 :
1956 : // Sort all of offsets, rangeDelOffsets and rangeKeyOffsets, using *batch's
1957 : // sort.Interface implementation.
1958 1 : pointOffsets := b.offsets
1959 1 : sort.Sort(b)
1960 1 : b.offsets = rangeDelOffsets
1961 1 : sort.Sort(b)
1962 1 : b.offsets = rangeKeyOffsets
1963 1 : sort.Sort(b)
1964 1 : b.offsets = pointOffsets
1965 1 :
1966 1 : if len(rangeDelOffsets) > 0 {
1967 1 : frag := &keyspan.Fragmenter{
1968 1 : Cmp: b.cmp,
1969 1 : Format: b.comparer.FormatKey,
1970 1 : Emit: func(s keyspan.Span) {
1971 1 : b.tombstones = append(b.tombstones, s)
1972 1 : },
1973 : }
1974 1 : it := &flushableBatchIter{
1975 1 : batch: b,
1976 1 : data: b.data,
1977 1 : offsets: rangeDelOffsets,
1978 1 : cmp: b.cmp,
1979 1 : index: -1,
1980 1 : }
1981 1 : fragmentRangeDels(frag, it, len(rangeDelOffsets))
1982 : }
1983 1 : if len(rangeKeyOffsets) > 0 {
1984 1 : frag := &keyspan.Fragmenter{
1985 1 : Cmp: b.cmp,
1986 1 : Format: b.comparer.FormatKey,
1987 1 : Emit: func(s keyspan.Span) {
1988 1 : b.rangeKeys = append(b.rangeKeys, s)
1989 1 : },
1990 : }
1991 1 : it := &flushableBatchIter{
1992 1 : batch: b,
1993 1 : data: b.data,
1994 1 : offsets: rangeKeyOffsets,
1995 1 : cmp: b.cmp,
1996 1 : index: -1,
1997 1 : }
1998 1 : fragmentRangeKeys(frag, it, len(rangeKeyOffsets))
1999 : }
2000 1 : return b, nil
2001 : }
2002 :
2003 1 : func (b *flushableBatch) setSeqNum(seqNum base.SeqNum) {
2004 1 : if b.seqNum != 0 {
2005 0 : panic(fmt.Sprintf("pebble: flushableBatch.seqNum already set: %d", b.seqNum))
2006 : }
2007 1 : b.seqNum = seqNum
2008 1 : for i := range b.tombstones {
2009 1 : for j := range b.tombstones[i].Keys {
2010 1 : b.tombstones[i].Keys[j].Trailer = base.MakeTrailer(
2011 1 : b.tombstones[i].Keys[j].SeqNum()+seqNum,
2012 1 : b.tombstones[i].Keys[j].Kind(),
2013 1 : )
2014 1 : }
2015 : }
2016 1 : for i := range b.rangeKeys {
2017 1 : for j := range b.rangeKeys[i].Keys {
2018 1 : b.rangeKeys[i].Keys[j].Trailer = base.MakeTrailer(
2019 1 : b.rangeKeys[i].Keys[j].SeqNum()+seqNum,
2020 1 : b.rangeKeys[i].Keys[j].Kind(),
2021 1 : )
2022 1 : }
2023 : }
2024 : }
2025 :
2026 1 : func (b *flushableBatch) Len() int {
2027 1 : return len(b.offsets)
2028 1 : }
2029 :
2030 1 : func (b *flushableBatch) Less(i, j int) bool {
2031 1 : ei := &b.offsets[i]
2032 1 : ej := &b.offsets[j]
2033 1 : ki := b.data[ei.keyStart:ei.keyEnd]
2034 1 : kj := b.data[ej.keyStart:ej.keyEnd]
2035 1 : switch c := b.cmp(ki, kj); {
2036 1 : case c < 0:
2037 1 : return true
2038 1 : case c > 0:
2039 1 : return false
2040 1 : default:
2041 1 : return ei.offset > ej.offset
2042 : }
2043 : }
2044 :
2045 1 : func (b *flushableBatch) Swap(i, j int) {
2046 1 : b.offsets[i], b.offsets[j] = b.offsets[j], b.offsets[i]
2047 1 : }
2048 :
2049 : // newIter is part of the flushable interface.
2050 1 : func (b *flushableBatch) newIter(o *IterOptions) internalIterator {
2051 1 : return &flushableBatchIter{
2052 1 : batch: b,
2053 1 : data: b.data,
2054 1 : offsets: b.offsets,
2055 1 : cmp: b.cmp,
2056 1 : index: -1,
2057 1 : lower: o.GetLowerBound(),
2058 1 : upper: o.GetUpperBound(),
2059 1 : }
2060 1 : }
2061 :
2062 : // newFlushIter is part of the flushable interface.
2063 1 : func (b *flushableBatch) newFlushIter(o *IterOptions) internalIterator {
2064 1 : return &flushFlushableBatchIter{
2065 1 : flushableBatchIter: flushableBatchIter{
2066 1 : batch: b,
2067 1 : data: b.data,
2068 1 : offsets: b.offsets,
2069 1 : cmp: b.cmp,
2070 1 : index: -1,
2071 1 : },
2072 1 : }
2073 1 : }
2074 :
2075 : // newRangeDelIter is part of the flushable interface.
2076 1 : func (b *flushableBatch) newRangeDelIter(o *IterOptions) keyspan.FragmentIterator {
2077 1 : if len(b.tombstones) == 0 {
2078 1 : return nil
2079 1 : }
2080 1 : return keyspan.NewIter(b.cmp, b.tombstones)
2081 : }
2082 :
2083 : // newRangeKeyIter is part of the flushable interface.
2084 1 : func (b *flushableBatch) newRangeKeyIter(o *IterOptions) keyspan.FragmentIterator {
2085 1 : if len(b.rangeKeys) == 0 {
2086 1 : return nil
2087 1 : }
2088 1 : return keyspan.NewIter(b.cmp, b.rangeKeys)
2089 : }
2090 :
2091 : // containsRangeKeys is part of the flushable interface.
2092 1 : func (b *flushableBatch) containsRangeKeys() bool { return len(b.rangeKeys) > 0 }
2093 :
2094 : // inuseBytes is part of the flushable interface.
2095 1 : func (b *flushableBatch) inuseBytes() uint64 {
2096 1 : return uint64(len(b.data) - batchrepr.HeaderLen)
2097 1 : }
2098 :
2099 : // totalBytes is part of the flushable interface.
2100 1 : func (b *flushableBatch) totalBytes() uint64 {
2101 1 : return uint64(cap(b.data))
2102 1 : }
2103 :
2104 : // readyForFlush is part of the flushable interface.
2105 1 : func (b *flushableBatch) readyForFlush() bool {
2106 1 : // A flushable batch is always ready for flush; it must be flushed together
2107 1 : // with the previous memtable.
2108 1 : return true
2109 1 : }
2110 :
2111 : // computePossibleOverlaps is part of the flushable interface.
2112 : func (b *flushableBatch) computePossibleOverlaps(
2113 : fn func(bounded) shouldContinue, bounded ...bounded,
2114 1 : ) {
2115 1 : computePossibleOverlapsGenericImpl[*flushableBatch](b, b.cmp, fn, bounded)
2116 1 : }
2117 :
2118 : // Note: flushableBatchIter mirrors the implementation of batchIter. Keep the
2119 : // two in sync.
2120 : type flushableBatchIter struct {
2121 : // Members to be initialized by creator.
2122 : batch *flushableBatch
2123 : // The bytes backing the batch. Always the same as batch.data?
2124 : data []byte
2125 : // The sorted entries. This is not always equal to batch.offsets.
2126 : offsets []flushableBatchEntry
2127 : cmp Compare
2128 : // Must be initialized to -1. It is the index into offsets that represents
2129 : // the current iterator position.
2130 : index int
2131 :
2132 : // For internal use by the implementation.
2133 : kv base.InternalKV
2134 : err error
2135 :
2136 : // Optionally initialize to bounds of iteration, if any.
2137 : lower []byte
2138 : upper []byte
2139 : }
2140 :
2141 : // flushableBatchIter implements the base.InternalIterator interface.
2142 : var _ base.InternalIterator = (*flushableBatchIter)(nil)
2143 :
2144 1 : func (i *flushableBatchIter) String() string {
2145 1 : return "flushable-batch"
2146 1 : }
2147 :
2148 : // SeekGE implements internalIterator.SeekGE, as documented in the pebble
2149 : // package. Ignore flags.TrySeekUsingNext() since we don't expect this
2150 : // optimization to provide much benefit here at the moment.
2151 1 : func (i *flushableBatchIter) SeekGE(key []byte, flags base.SeekGEFlags) *base.InternalKV {
2152 1 : i.err = nil // clear cached iteration error
2153 1 : ikey := base.MakeSearchKey(key)
2154 1 : i.index = sort.Search(len(i.offsets), func(j int) bool {
2155 1 : return base.InternalCompare(i.cmp, ikey, i.getKey(j)) <= 0
2156 1 : })
2157 1 : if i.index >= len(i.offsets) {
2158 1 : return nil
2159 1 : }
2160 1 : kv := i.getKV(i.index)
2161 1 : if i.upper != nil && i.cmp(kv.K.UserKey, i.upper) >= 0 {
2162 1 : i.index = len(i.offsets)
2163 1 : return nil
2164 1 : }
2165 1 : return kv
2166 : }
2167 :
2168 : // SeekPrefixGE implements internalIterator.SeekPrefixGE, as documented in the
2169 : // pebble package.
2170 : func (i *flushableBatchIter) SeekPrefixGE(
2171 : prefix, key []byte, flags base.SeekGEFlags,
2172 1 : ) *base.InternalKV {
2173 1 : kv := i.SeekGE(key, flags)
2174 1 : if kv == nil {
2175 1 : return nil
2176 1 : }
2177 : // If the key doesn't have the sought prefix, return nil.
2178 1 : if !bytes.Equal(i.batch.comparer.Split.Prefix(kv.K.UserKey), prefix) {
2179 1 : return nil
2180 1 : }
2181 1 : return kv
2182 : }
2183 :
2184 : // SeekLT implements internalIterator.SeekLT, as documented in the pebble
2185 : // package.
2186 1 : func (i *flushableBatchIter) SeekLT(key []byte, flags base.SeekLTFlags) *base.InternalKV {
2187 1 : i.err = nil // clear cached iteration error
2188 1 : ikey := base.MakeSearchKey(key)
2189 1 : i.index = sort.Search(len(i.offsets), func(j int) bool {
2190 1 : return base.InternalCompare(i.cmp, ikey, i.getKey(j)) <= 0
2191 1 : })
2192 1 : i.index--
2193 1 : if i.index < 0 {
2194 1 : return nil
2195 1 : }
2196 1 : kv := i.getKV(i.index)
2197 1 : if i.lower != nil && i.cmp(kv.K.UserKey, i.lower) < 0 {
2198 1 : i.index = -1
2199 1 : return nil
2200 1 : }
2201 1 : return kv
2202 : }
2203 :
2204 : // First implements internalIterator.First, as documented in the pebble
2205 : // package.
2206 1 : func (i *flushableBatchIter) First() *base.InternalKV {
2207 1 : i.err = nil // clear cached iteration error
2208 1 : if len(i.offsets) == 0 {
2209 1 : return nil
2210 1 : }
2211 1 : i.index = 0
2212 1 : kv := i.getKV(i.index)
2213 1 : if i.upper != nil && i.cmp(kv.K.UserKey, i.upper) >= 0 {
2214 1 : i.index = len(i.offsets)
2215 1 : return nil
2216 1 : }
2217 1 : return kv
2218 : }
2219 :
2220 : // Last implements internalIterator.Last, as documented in the pebble
2221 : // package.
2222 1 : func (i *flushableBatchIter) Last() *base.InternalKV {
2223 1 : i.err = nil // clear cached iteration error
2224 1 : if len(i.offsets) == 0 {
2225 1 : return nil
2226 1 : }
2227 1 : i.index = len(i.offsets) - 1
2228 1 : kv := i.getKV(i.index)
2229 1 : if i.lower != nil && i.cmp(kv.K.UserKey, i.lower) < 0 {
2230 0 : i.index = -1
2231 0 : return nil
2232 0 : }
2233 1 : return kv
2234 : }
2235 :
2236 : // Note: flushFlushableBatchIter.Next mirrors the implementation of
2237 : // flushableBatchIter.Next due to performance. Keep the two in sync.
2238 1 : func (i *flushableBatchIter) Next() *base.InternalKV {
2239 1 : if i.index == len(i.offsets) {
2240 0 : return nil
2241 0 : }
2242 1 : i.index++
2243 1 : if i.index == len(i.offsets) {
2244 1 : return nil
2245 1 : }
2246 1 : kv := i.getKV(i.index)
2247 1 : if i.upper != nil && i.cmp(kv.K.UserKey, i.upper) >= 0 {
2248 1 : i.index = len(i.offsets)
2249 1 : return nil
2250 1 : }
2251 1 : return kv
2252 : }
2253 :
2254 1 : func (i *flushableBatchIter) Prev() *base.InternalKV {
2255 1 : if i.index < 0 {
2256 0 : return nil
2257 0 : }
2258 1 : i.index--
2259 1 : if i.index < 0 {
2260 1 : return nil
2261 1 : }
2262 1 : kv := i.getKV(i.index)
2263 1 : if i.lower != nil && i.cmp(kv.K.UserKey, i.lower) < 0 {
2264 1 : i.index = -1
2265 1 : return nil
2266 1 : }
2267 1 : return kv
2268 : }
2269 :
2270 : // Note: flushFlushableBatchIter.NextPrefix mirrors the implementation of
2271 : // flushableBatchIter.NextPrefix due to performance. Keep the two in sync.
2272 0 : func (i *flushableBatchIter) NextPrefix(succKey []byte) *base.InternalKV {
2273 0 : return i.SeekGE(succKey, base.SeekGEFlagsNone.EnableTrySeekUsingNext())
2274 0 : }
2275 :
2276 1 : func (i *flushableBatchIter) getKey(index int) InternalKey {
2277 1 : e := &i.offsets[index]
2278 1 : kind := InternalKeyKind(i.data[e.offset])
2279 1 : key := i.data[e.keyStart:e.keyEnd]
2280 1 : return base.MakeInternalKey(key, i.batch.seqNum+base.SeqNum(e.index), kind)
2281 1 : }
2282 :
2283 1 : func (i *flushableBatchIter) getKV(index int) *base.InternalKV {
2284 1 : i.kv = base.InternalKV{
2285 1 : K: i.getKey(index),
2286 1 : V: i.extractValue(),
2287 1 : }
2288 1 : return &i.kv
2289 1 : }
2290 :
2291 1 : func (i *flushableBatchIter) extractValue() base.LazyValue {
2292 1 : p := i.data[i.offsets[i.index].offset:]
2293 1 : if len(p) == 0 {
2294 0 : i.err = base.CorruptionErrorf("corrupted batch")
2295 0 : return base.LazyValue{}
2296 0 : }
2297 1 : kind := InternalKeyKind(p[0])
2298 1 : if kind > InternalKeyKindMax {
2299 0 : i.err = base.CorruptionErrorf("corrupted batch")
2300 0 : return base.LazyValue{}
2301 0 : }
2302 1 : var value []byte
2303 1 : var ok bool
2304 1 : switch kind {
2305 : case InternalKeyKindSet, InternalKeyKindMerge, InternalKeyKindRangeDelete,
2306 : InternalKeyKindRangeKeySet, InternalKeyKindRangeKeyUnset, InternalKeyKindRangeKeyDelete,
2307 1 : InternalKeyKindDeleteSized:
2308 1 : keyEnd := i.offsets[i.index].keyEnd
2309 1 : _, value, ok = batchrepr.DecodeStr(i.data[keyEnd:])
2310 1 : if !ok {
2311 0 : i.err = base.CorruptionErrorf("corrupted batch")
2312 0 : return base.LazyValue{}
2313 0 : }
2314 : }
2315 1 : return base.MakeInPlaceValue(value)
2316 : }
2317 :
2318 0 : func (i *flushableBatchIter) Valid() bool {
2319 0 : return i.index >= 0 && i.index < len(i.offsets)
2320 0 : }
2321 :
2322 1 : func (i *flushableBatchIter) Error() error {
2323 1 : return i.err
2324 1 : }
2325 :
2326 1 : func (i *flushableBatchIter) Close() error {
2327 1 : return i.err
2328 1 : }
2329 :
2330 1 : func (i *flushableBatchIter) SetBounds(lower, upper []byte) {
2331 1 : i.lower = lower
2332 1 : i.upper = upper
2333 1 : }
2334 :
2335 0 : func (i *flushableBatchIter) SetContext(_ context.Context) {}
2336 :
2337 : // DebugTree is part of the InternalIterator interface.
2338 0 : func (i *flushableBatchIter) DebugTree(tp treeprinter.Node) {
2339 0 : tp.Childf("%T(%p)", i, i)
2340 0 : }
2341 :
2342 : // flushFlushableBatchIter is similar to flushableBatchIter but it keeps track
2343 : // of number of bytes iterated.
2344 : type flushFlushableBatchIter struct {
2345 : flushableBatchIter
2346 : }
2347 :
2348 : // flushFlushableBatchIter implements the base.InternalIterator interface.
2349 : var _ base.InternalIterator = (*flushFlushableBatchIter)(nil)
2350 :
2351 0 : func (i *flushFlushableBatchIter) String() string {
2352 0 : return "flushable-batch"
2353 0 : }
2354 :
2355 0 : func (i *flushFlushableBatchIter) SeekGE(key []byte, flags base.SeekGEFlags) *base.InternalKV {
2356 0 : panic("pebble: SeekGE unimplemented")
2357 : }
2358 :
2359 : func (i *flushFlushableBatchIter) SeekPrefixGE(
2360 : prefix, key []byte, flags base.SeekGEFlags,
2361 0 : ) *base.InternalKV {
2362 0 : panic("pebble: SeekPrefixGE unimplemented")
2363 : }
2364 :
2365 0 : func (i *flushFlushableBatchIter) SeekLT(key []byte, flags base.SeekLTFlags) *base.InternalKV {
2366 0 : panic("pebble: SeekLT unimplemented")
2367 : }
2368 :
2369 1 : func (i *flushFlushableBatchIter) First() *base.InternalKV {
2370 1 : i.err = nil // clear cached iteration error
2371 1 : return i.flushableBatchIter.First()
2372 1 : }
2373 :
2374 0 : func (i *flushFlushableBatchIter) NextPrefix(succKey []byte) *base.InternalKV {
2375 0 : panic("pebble: Prev unimplemented")
2376 : }
2377 :
2378 : // Note: flushFlushableBatchIter.Next mirrors the implementation of
2379 : // flushableBatchIter.Next due to performance. Keep the two in sync.
2380 1 : func (i *flushFlushableBatchIter) Next() *base.InternalKV {
2381 1 : if i.index == len(i.offsets) {
2382 0 : return nil
2383 0 : }
2384 1 : i.index++
2385 1 : if i.index == len(i.offsets) {
2386 1 : return nil
2387 1 : }
2388 1 : return i.getKV(i.index)
2389 : }
2390 :
2391 0 : func (i flushFlushableBatchIter) Prev() *base.InternalKV {
2392 0 : panic("pebble: Prev unimplemented")
2393 : }
2394 :
2395 : // batchOptions holds the parameters to configure batch.
2396 : type batchOptions struct {
2397 : initialSizeBytes int
2398 : maxRetainedSizeBytes int
2399 : }
2400 :
2401 : // ensureDefaults creates batch options with default values.
2402 1 : func (o *batchOptions) ensureDefaults() {
2403 1 : if o.initialSizeBytes <= 0 {
2404 1 : o.initialSizeBytes = defaultBatchInitialSize
2405 1 : }
2406 1 : if o.maxRetainedSizeBytes <= 0 {
2407 1 : o.maxRetainedSizeBytes = defaultBatchMaxRetainedSize
2408 1 : }
2409 : }
2410 :
2411 : // BatchOption allows customizing the batch.
2412 : type BatchOption func(*batchOptions)
2413 :
2414 : // WithInitialSizeBytes sets a custom initial size for the batch. Defaults
2415 : // to 1KB.
2416 0 : func WithInitialSizeBytes(s int) BatchOption {
2417 0 : return func(opts *batchOptions) {
2418 0 : opts.initialSizeBytes = s
2419 0 : }
2420 : }
2421 :
2422 : // WithMaxRetainedSizeBytes sets a custom max size for the batch to be
2423 : // re-used. Any batch which exceeds the max retained size would be GC-ed.
2424 : // Defaults to 1MB.
2425 0 : func WithMaxRetainedSizeBytes(s int) BatchOption {
2426 0 : return func(opts *batchOptions) {
2427 0 : opts.maxRetainedSizeBytes = s
2428 0 : }
2429 : }
2430 :
2431 : // batchSort returns iterators for the sorted contents of the batch. It is
2432 : // intended for testing use only. The batch.Sort dance is done to prevent
2433 : // exposing this method in the public pebble interface.
2434 : func batchSort(
2435 : i interface{},
2436 : ) (
2437 : points internalIterator,
2438 : rangeDels keyspan.FragmentIterator,
2439 : rangeKeys keyspan.FragmentIterator,
2440 1 : ) {
2441 1 : b := i.(*Batch)
2442 1 : if b.Indexed() {
2443 1 : pointIter := b.newInternalIter(nil)
2444 1 : rangeDelIter := b.newRangeDelIter(nil, math.MaxUint64)
2445 1 : rangeKeyIter := b.newRangeKeyIter(nil, math.MaxUint64)
2446 1 : return pointIter, rangeDelIter, rangeKeyIter
2447 1 : }
2448 1 : f, err := newFlushableBatch(b, b.db.opts.Comparer)
2449 1 : if err != nil {
2450 0 : panic(err)
2451 : }
2452 1 : return f.newIter(nil), f.newRangeDelIter(nil), f.newRangeKeyIter(nil)
2453 : }
2454 :
2455 1 : func init() {
2456 1 : private.BatchSort = batchSort
2457 1 : }
|