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