forked from cockroachdb/pebble
-
Notifications
You must be signed in to change notification settings - Fork 0
/
batch.go
1255 lines (1119 loc) · 34.3 KB
/
batch.go
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
// Copyright 2012 The LevelDB-Go and Pebble Authors. All rights reserved. Use
// of this source code is governed by a BSD-style license that can be found in
// the LICENSE file.
package pebble
import (
"encoding/binary"
"errors"
"fmt"
"sort"
"sync"
"sync/atomic"
"unsafe"
"github.com/petermattis/pebble/internal/base"
"github.com/petermattis/pebble/internal/batchskl"
"github.com/petermattis/pebble/internal/rangedel"
"github.com/petermattis/pebble/internal/rawalloc"
)
const (
batchHeaderLen = 12
batchInitialSize = 1 << 10 // 1 KB
batchMaxRetainedSize = 1 << 20 // 1 MB
invalidBatchCount = 1<<32 - 1
maxVarintLen32 = 5
)
// ErrNotIndexed means that a read operation on a batch failed because the
// batch is not indexed and thus doesn't support reads.
var ErrNotIndexed = errors.New("pebble: batch not indexed")
// ErrInvalidBatch indicates that a batch is invalid or otherwise corrupted.
var ErrInvalidBatch = errors.New("pebble: invalid batch")
type batchStorage struct {
// Data is the wire format of a batch's log entry:
// - 8 bytes for a sequence number of the first batch element,
// or zeroes if the batch has not yet been applied,
// - 4 bytes for the count: the number of elements in the batch,
// or "\xff\xff\xff\xff" if the batch is invalid,
// - count elements, being:
// - one byte for the kind
// - the varint-string user key,
// - the varint-string value (if kind != delete).
// The sequence number and count are stored in little-endian order.
data []byte
cmp Compare
abbreviatedKey AbbreviatedKey
}
// Get implements Storage.Get, as documented in the pebble/batchskl package.
func (s *batchStorage) Get(offset uint32) InternalKey {
kind := InternalKeyKind(s.data[offset])
_, key, ok := batchDecodeStr(s.data[offset+1:])
if !ok {
panic(fmt.Sprintf("corrupted batch entry: %d", offset))
}
return base.MakeInternalKey(key, uint64(offset)|InternalKeySeqNumBatch, kind)
}
// AbbreviatedKey implements Storage.AbbreviatedKey, as documented in the
// pebble/batchskl package.
func (s *batchStorage) AbbreviatedKey(key []byte) uint64 {
return s.abbreviatedKey(key)
}
// Compare implements Storage.Compare, as documented in the pebble/batchskl
// package.
func (s *batchStorage) Compare(a []byte, b uint32) int {
// The key "a" is always the search key or the newer key being inserted. If
// it is equal to the existing key consider it smaller so that it sorts
// first.
if s.cmp(a, s.Get(b).UserKey) <= 0 {
return -1
}
return 1
}
// A Batch is a sequence of Sets, Merges, Deletes, and/or DeleteRanges that are
// applied atomically. Batch implements the Reader interface, but only an
// indexed batch supports reading (without error) via Get or NewIter. A
// non-indexed batch will return ErrNotIndexed when read from .
//
// Indexing
//
// Batches can be optionally indexed (see DB.NewIndexedBatch). An indexed batch
// allows iteration via an Iterator (see Batch.NewIter). The iterator provides
// a merged view of the operations in the batch and the underlying
// database. This is implemented by treating the batch as an additional layer
// in the LSM where every entry in the batch is considered newer than any entry
// in the underlying database (batch entries have the InternalKeySeqNumBatch
// bit set). By treating the batch as an additional layer in the LSM, iteration
// supports all batch operations (i.e. Set, Merge, Delete, and DeleteRange)
// with minimal effort.
//
// The same key can be operated on multiple times in a batch, though only the
// latest operation will be visible. For example, Put("a", "b"), Delete("a")
// will cause the key "a" to not be visible in the batch. Put("a", "b"),
// Put("a", "c") will cause a read of "a" to return the value "c".
//
// The batch index is implemented via an skiplist (internal/batchskl). While
// the skiplist implementation is very fast, inserting into an indexed batch is
// significantly slower than inserting into a non-indexed batch. Only use an
// indexed batch if you require reading from it.
//
// Atomic commit
//
// The operations in a batch are persisted by calling Batch.Commit which is
// equivalent to calling DB.Apply(batch). A batch is committed atomically by
// writing the internal batch representation to the WAL, adding all of the
// batch operations to the memtable associated with the WAL, and then
// incrementing the visible sequence number so that subsequent reads can see
// the effects of the batch operations. If WriteOptions.Sync is true, a call to
// Batch.Commit will guarantee that the batch is persisted to disk before
// returning. See commitPipeline for more on the implementation details.
//
// Large batches
//
// The size of a batch is limited only by available memory (be aware that
// indexed batches require considerably additional memory for the skiplist
// structure). A given WAL file has a single memtable associated with it (this
// restriction could be removed, but doing so is onerous and complex). And a
// memtable has a fixed size due to the underlying fixed size arena. Note that
// this differs from RocksDB where a memtable can grow arbitrarily large using
// a list of arena chunks. In RocksDB this is accomplished by storing pointers
// in the arena memory, but that isn't possible in Go.
//
// During Batch.Commit, a batch which is larger than a threshold (>
// MemTableSize/2) is wrapped in a flushableBatch and inserted into the queue
// of memtables. A flushableBatch forces WAL to be rotated, but that happens
// anyways when the memtable becomes full so this does not cause significant
// WAL churn. Because the flushableBatch is readable as another layer in the
// LSM, Batch.Commit returns as soon as the flushableBatch has been added to
// the queue of memtables.
//
// Internally, a flushableBatch provides Iterator support by sorting the batch
// contents (the batch is sorted once, when it is added to the memtable
// queue). Sorting the batch contents and insertion of the contents into a
// memtable have the same big-O time, but the constant factor dominates
// here. Sorting is significantly faster and uses significantly less memory.
//
// Internal representation
//
// The internal batch representation is a contiguous byte buffer with a fixed
// 12-byte header, followed by a series of records.
//
// +-------------+------------+--- ... ---+
// | SeqNum (8B) | Count (4B) | Entries |
// +-------------+------------+--- ... ---+
//
// Each record has a 1-byte kind tag prefix, followed by 1 or 2 length prefixed
// strings (varstring):
//
// +-----------+-----------------+-------------------+
// | Kind (1B) | Key (varstring) | Value (varstring) |
// +-----------+-----------------+-------------------+
//
// A varstring is a varint32 followed by N bytes of data. The Kind tags are
// exactly those specified by InternalKeyKind. The following table shows the
// format for records of each kind:
//
// InternalKeyKindDelete varstring
// InternalKeyKindLogData varstring
// InternalKeyKindSet varstring varstring
// InternalKeyKindMerge varstring varstring
// InternalKeyKindRangeDelete varstring varstring
//
// The intuitive understanding here are that the arguments to Delete(), Set(),
// Merge(), and DeleteRange() are encoded into the batch.
//
// The internal batch representation is the on disk format for a batch in the
// WAL, and thus stable. New record kinds may be added, but the existing ones
// will not be modified.
type Batch struct {
storage batchStorage
memTableSize uint32
// The db to which the batch will be committed.
db *DB
// An optional skiplist keyed by offset into data of the entry.
index *batchskl.Skiplist
rangeDelIndex *batchskl.Skiplist
// Fragmented range deletion tombstones. Cached the first time a range
// deletion iterator is requested. The cache is invalidated whenever a new
// range deletion is added to the batch.
tombstones []rangedel.Tombstone
// The flushableBatch wrapper if the batch is too large to fit in the
// memtable.
flushable *flushableBatch
commit sync.WaitGroup
applied uint32 // updated atomically
}
var _ Reader = (*Batch)(nil)
var _ Writer = (*Batch)(nil)
var batchPool = sync.Pool{
New: func() interface{} {
return &Batch{}
},
}
type indexedBatch struct {
batch Batch
index batchskl.Skiplist
}
var indexedBatchPool = sync.Pool{
New: func() interface{} {
return &indexedBatch{}
},
}
func newBatch(db *DB) *Batch {
b := batchPool.Get().(*Batch)
b.db = db
return b
}
func newIndexedBatch(db *DB, comparer *Comparer) *Batch {
i := indexedBatchPool.Get().(*indexedBatch)
i.batch.storage.cmp = comparer.Compare
i.batch.storage.abbreviatedKey = comparer.AbbreviatedKey
i.batch.db = db
i.batch.index = &i.index
i.batch.index.Reset(&i.batch.storage, 0)
return &i.batch
}
func (b *Batch) release() {
// NB: This is ugly, but necessary so that we can use atomic.StoreUint32 for
// the Batch.applied field. Without using an atomic to clear that field the
// Go race detector complains.
b.reset()
b.storage.cmp = nil
b.storage.abbreviatedKey = nil
b.memTableSize = 0
b.db = nil
b.flushable = nil
b.commit = sync.WaitGroup{}
atomic.StoreUint32(&b.applied, 0)
if b.index == nil {
batchPool.Put(b)
} else {
*b.index = batchskl.Skiplist{}
b.index, b.rangeDelIndex = nil, nil
indexedBatchPool.Put((*indexedBatch)(unsafe.Pointer(b)))
}
}
func (b *Batch) refreshMemTableSize() {
b.memTableSize = 0
for r := b.Reader(); ; {
_, key, value, ok := r.Next()
if !ok {
break
}
b.memTableSize += memTableEntrySize(len(key), len(value))
}
}
// Apply the operations contained in the batch to the receiver batch.
//
// It is safe to modify the contents of the arguments after Apply returns.
func (b *Batch) Apply(batch *Batch, _ *WriteOptions) error {
if len(batch.storage.data) == 0 {
return nil
}
if len(batch.storage.data) < batchHeaderLen {
return errors.New("pebble: invalid batch")
}
offset := len(b.storage.data)
if offset == 0 {
b.init(offset)
offset = batchHeaderLen
}
b.storage.data = append(b.storage.data, batch.storage.data[batchHeaderLen:]...)
count := binary.LittleEndian.Uint32(batch.storage.data[8:12])
b.setCount(b.count() + count)
for iter := BatchReader(b.storage.data[offset:]); len(iter) > 0; {
offset := uintptr(unsafe.Pointer(&iter[0])) - uintptr(unsafe.Pointer(&b.storage.data[0]))
kind, key, value, ok := iter.Next()
if !ok {
break
}
if b.index != nil {
var err error
if kind == InternalKeyKindRangeDelete {
if b.rangeDelIndex == nil {
b.rangeDelIndex = batchskl.NewSkiplist(&b.storage, 0)
}
err = b.rangeDelIndex.Add(uint32(offset))
} else {
err = b.index.Add(uint32(offset))
}
if err != nil {
// We never add duplicate entries, so an error should never occur.
panic(err)
}
}
b.memTableSize += memTableEntrySize(len(key), len(value))
}
return nil
}
// Get gets the value for the given key. It returns ErrNotFound if the DB
// does not contain the key.
//
// The caller should not modify the contents of the returned slice, but
// it is safe to modify the contents of the argument after Get returns.
func (b *Batch) Get(key []byte) (value []byte, err error) {
if b.index == nil {
return nil, ErrNotIndexed
}
return b.db.getInternal(key, b, nil /* snapshot */)
}
func (b *Batch) encodeKeyValue(key, value []byte, kind InternalKeyKind) uint32 {
pos := len(b.storage.data)
offset := uint32(pos)
b.grow(1 + 2*maxVarintLen32 + len(key) + len(value))
b.storage.data[pos] = byte(kind)
pos, varlen1 := b.copyStr(pos+1, key)
_, varlen2 := b.copyStr(pos, value)
b.storage.data = b.storage.data[:len(b.storage.data)-(2*maxVarintLen32-varlen1-varlen2)]
return offset
}
// Set adds an action to the batch that sets the key to map to the value.
//
// It is safe to modify the contents of the arguments after Set returns.
func (b *Batch) Set(key, value []byte, _ *WriteOptions) error {
if len(b.storage.data) == 0 {
b.init(len(key) + len(value) + 2*binary.MaxVarintLen64 + batchHeaderLen)
}
if !b.increment() {
return ErrInvalidBatch
}
offset := b.encodeKeyValue(key, value, InternalKeyKindSet)
if b.index != nil {
if err := b.index.Add(offset); err != nil {
// We never add duplicate entries, so an error should never occur.
panic(err)
}
}
b.memTableSize += memTableEntrySize(len(key), len(value))
return nil
}
// Merge adds an action to the batch that merges the value at key with the new
// value. The details of the merge are dependent upon the configured merge
// operator.
//
// It is safe to modify the contents of the arguments after Merge returns.
func (b *Batch) Merge(key, value []byte, _ *WriteOptions) error {
if len(b.storage.data) == 0 {
b.init(len(key) + len(value) + 2*binary.MaxVarintLen64 + batchHeaderLen)
}
if !b.increment() {
return ErrInvalidBatch
}
offset := b.encodeKeyValue(key, value, InternalKeyKindMerge)
if b.index != nil {
if err := b.index.Add(offset); err != nil {
// We never add duplicate entries, so an error should never occur.
panic(err)
}
}
b.memTableSize += memTableEntrySize(len(key), len(value))
return nil
}
// Delete adds an action to the batch that deletes the entry for key.
//
// It is safe to modify the contents of the arguments after Delete returns.
func (b *Batch) Delete(key []byte, _ *WriteOptions) error {
if len(b.storage.data) == 0 {
b.init(len(key) + binary.MaxVarintLen64 + batchHeaderLen)
}
if !b.increment() {
return ErrInvalidBatch
}
pos := len(b.storage.data)
offset := uint32(pos)
b.grow(1 + maxVarintLen32 + len(key))
b.storage.data[pos] = byte(InternalKeyKindDelete)
pos, varlen1 := b.copyStr(pos+1, key)
b.storage.data = b.storage.data[:len(b.storage.data)-(maxVarintLen32-varlen1)]
if b.index != nil {
if err := b.index.Add(offset); err != nil {
// We never add duplicate entries, so an error should never occur.
panic(err)
}
}
b.memTableSize += memTableEntrySize(len(key), 0)
return nil
}
// DeleteRange deletes all of the keys (and values) in the range [start,end)
// (inclusive on start, exclusive on end).
//
// It is safe to modify the contents of the arguments after DeleteRange
// returns.
func (b *Batch) DeleteRange(start, end []byte, _ *WriteOptions) error {
if len(b.storage.data) == 0 {
b.init(len(start) + len(end) + 2*binary.MaxVarintLen64 + batchHeaderLen)
}
if !b.increment() {
return ErrInvalidBatch
}
offset := b.encodeKeyValue(start, end, InternalKeyKindRangeDelete)
if b.index != nil {
// Range deletions are rare, so we lazily allocate the index for them.
if b.rangeDelIndex == nil {
b.rangeDelIndex = batchskl.NewSkiplist(&b.storage, 0)
}
if err := b.rangeDelIndex.Add(offset); err != nil {
// We never add duplicate entries, so an error should never occur.
panic(err)
}
b.tombstones = nil
}
b.memTableSize += memTableEntrySize(len(start), len(end))
return nil
}
// LogData adds the specified to the batch. The data will be written to the
// WAL, but not added to memtables or sstables. Log data is never indexed,
// which makes it useful for testing WAL performance.
//
// It is safe to modify the contents of the argument after LogData returns.
func (b *Batch) LogData(data []byte, _ *WriteOptions) error {
if len(b.storage.data) == 0 {
b.init(len(data) + binary.MaxVarintLen64 + batchHeaderLen)
}
if !b.increment() {
return ErrInvalidBatch
}
pos := len(b.storage.data)
b.grow(1 + maxVarintLen32 + len(data))
b.storage.data[pos] = byte(InternalKeyKindLogData)
pos, varlen1 := b.copyStr(pos+1, data)
b.storage.data = b.storage.data[:len(b.storage.data)-(maxVarintLen32-varlen1)]
return nil
}
// Repr returns the underlying batch representation. It is not safe to modify
// the contents.
func (b *Batch) Repr() []byte {
return b.storage.data
}
// NewIter returns an iterator that is unpositioned (Iterator.Valid() will
// return false). The iterator can be positioned via a call to SeekGE,
// SeekPrefixGE, SeekLT, First or Last. Only indexed batches support iterators.
func (b *Batch) NewIter(o *IterOptions) *Iterator {
if b.index == nil {
return &Iterator{err: ErrNotIndexed}
}
return b.db.newIterInternal(b.newInternalIter(o),
b.newRangeDelIter(o), nil /* snapshot */, o)
}
// newInternalIter creates a new internalIterator that iterates over the
// contents of the batch.
func (b *Batch) newInternalIter(o *IterOptions) internalIterator {
if b.index == nil {
return newErrorIter(ErrNotIndexed)
}
return &batchIter{
cmp: b.storage.cmp,
batch: b,
iter: b.index.NewIter(o.GetLowerBound(), o.GetUpperBound()),
}
}
func (b *Batch) newRangeDelIter(o *IterOptions) internalIterator {
if b.index == nil {
return newErrorIter(ErrNotIndexed)
}
if b.rangeDelIndex == nil {
return nil
}
// Fragment the range tombstones the first time a range deletion iterator is
// requested. The cached tombstones are invalidated if another range deletion
// tombstone is added to the batch.
if b.tombstones == nil {
frag := &rangedel.Fragmenter{
Cmp: b.storage.cmp,
Emit: func(fragmented []rangedel.Tombstone) {
b.tombstones = append(b.tombstones, fragmented...)
},
}
it := &batchIter{
cmp: b.storage.cmp,
batch: b,
iter: b.rangeDelIndex.NewIter(nil, nil),
}
for {
key, val := it.Next()
if key == nil {
break
}
frag.Add(*key, val)
}
frag.Finish()
}
return rangedel.NewIter(b.storage.cmp, b.tombstones)
}
// Commit applies the batch to its parent writer.
func (b *Batch) Commit(o *WriteOptions) error {
return b.db.Apply(b, o)
}
// Close closes the batch without committing it.
func (b *Batch) Close() error {
b.release()
return nil
}
// Indexed returns true if the batch is indexed (i.e. supports read
// operations).
func (b *Batch) Indexed() bool {
return b.index != nil
}
func (b *Batch) init(cap int) {
n := batchInitialSize
for n < cap {
n *= 2
}
b.storage.data = rawalloc.New(batchHeaderLen, n)
b.setCount(0)
b.setSeqNum(0)
b.storage.data = b.storage.data[:batchHeaderLen]
}
func (b *Batch) reset() {
if b.storage.data != nil {
if cap(b.storage.data) > batchMaxRetainedSize {
// If the capacity of the buffer is larger than our maximum
// retention size, don't re-use it. Let it be GC-ed instead.
// This prevents the memory from an unusually large batch from
// being held on to indefinitely.
b.storage.data = nil
} else {
// Otherwise, reset the buffer for re-use.
b.storage.data = b.storage.data[:batchHeaderLen]
b.setCount(0)
b.setSeqNum(0)
}
}
}
// seqNumData returns the 8 byte little-endian sequence number. Zero means that
// the batch has not yet been applied.
func (b *Batch) seqNumData() []byte {
return b.storage.data[:8]
}
// countData returns the 4 byte little-endian count data. "\xff\xff\xff\xff"
// means that the batch is invalid.
func (b *Batch) countData() []byte {
return b.storage.data[8:12]
}
func (b *Batch) increment() (ok bool) {
p := b.countData()
for i := range p {
p[i]++
if p[i] != 0x00 {
return true
}
}
// The countData was "\xff\xff\xff\xff". Leave it as it was.
p[0] = 0xff
p[1] = 0xff
p[2] = 0xff
p[3] = 0xff
return false
}
func (b *Batch) grow(n int) {
newSize := len(b.storage.data) + n
if newSize > cap(b.storage.data) {
newCap := 2 * cap(b.storage.data)
for newCap < newSize {
newCap *= 2
}
newData := rawalloc.New(len(b.storage.data), newCap)
copy(newData, b.storage.data)
b.storage.data = newData
}
b.storage.data = b.storage.data[:newSize]
}
func putUvarint32(buf []byte, x uint32) int {
i := 0
for x >= 0x80 {
buf[i] = byte(x) | 0x80
x >>= 7
i++
}
buf[i] = byte(x)
return i + 1
}
func (b *Batch) copyStr(pos int, s []byte) (int, int) {
n := putUvarint32(b.storage.data[pos:], uint32(len(s)))
return pos + n + copy(b.storage.data[pos+n:], s), n
}
func (b *Batch) setSeqNum(seqNum uint64) {
binary.LittleEndian.PutUint64(b.seqNumData(), seqNum)
}
func (b *Batch) seqNum() uint64 {
return binary.LittleEndian.Uint64(b.seqNumData())
}
func (b *Batch) setCount(v uint32) {
binary.LittleEndian.PutUint32(b.countData(), v)
}
func (b *Batch) count() uint32 {
return binary.LittleEndian.Uint32(b.countData())
}
// Reader returns a BatchReader for the current batch contents. If the batch is
// mutated, the new entries will not be visible to the reader.
func (b *Batch) Reader() BatchReader {
return b.storage.data[batchHeaderLen:]
}
func batchDecode(data []byte, offset uint32) (kind InternalKeyKind, ukey []byte, value []byte, ok bool) {
p := data[offset:]
if len(p) == 0 {
return 0, nil, nil, false
}
kind, p = InternalKeyKind(p[0]), p[1:]
if kind > InternalKeyKindMax {
return 0, nil, nil, false
}
p, ukey, ok = batchDecodeStr(p)
if !ok {
return 0, nil, nil, false
}
switch kind {
case InternalKeyKindSet, InternalKeyKindMerge, InternalKeyKindRangeDelete:
_, value, ok = batchDecodeStr(p)
if !ok {
return 0, nil, nil, false
}
}
return kind, ukey, value, true
}
func batchDecodeStr(data []byte) (odata []byte, s []byte, ok bool) {
v, n := binary.Uvarint(data)
if n <= 0 {
return nil, nil, false
}
data = data[n:]
if v > uint64(len(data)) {
return nil, nil, false
}
return data[v:], data[:v], true
}
// BatchReader iterates over the entries contained in a batch.
type BatchReader []byte
// MakeBatchReader constructs a BatchReader from a batch representation. The
// header (containing the batch count and seqnum) is ignored.
func MakeBatchReader(repr []byte) BatchReader {
return repr[batchHeaderLen:]
}
// Next returns the next entry in this batch. The final return value is false
// if the batch is corrupt. The end of batch is reached when len(r)==0.
func (r *BatchReader) Next() (kind InternalKeyKind, ukey []byte, value []byte, ok bool) {
p := *r
if len(p) == 0 {
return 0, nil, nil, false
}
kind, *r = InternalKeyKind(p[0]), p[1:]
if kind > InternalKeyKindMax {
return 0, nil, nil, false
}
ukey, ok = r.nextStr()
if !ok {
return 0, nil, nil, false
}
switch kind {
case InternalKeyKindSet, InternalKeyKindMerge, InternalKeyKindRangeDelete:
value, ok = r.nextStr()
if !ok {
return 0, nil, nil, false
}
}
return kind, ukey, value, true
}
func (r *BatchReader) nextStr() (s []byte, ok bool) {
p := *r
u, numBytes := binary.Uvarint(p)
if numBytes <= 0 {
return nil, false
}
p = p[numBytes:]
if u > uint64(len(p)) {
return nil, false
}
s, *r = p[:u], p[u:]
return s, true
}
// Note: batchIter mirrors the implementation of flushableBatchIter. Keep the
// two in sync.
type batchIter struct {
cmp Compare
batch *Batch
reverse bool
iter batchskl.Iterator
err error
}
// batchIter implements the internalIterator interface.
var _ internalIterator = (*batchIter)(nil)
func (i *batchIter) SeekGE(key []byte) (*InternalKey, []byte) {
ikey := i.iter.SeekGE(key)
if ikey == nil {
return nil, nil
}
return ikey, i.Value()
}
func (i *batchIter) SeekPrefixGE(prefix, key []byte) (*InternalKey, []byte) {
return i.SeekGE(key)
}
func (i *batchIter) SeekLT(key []byte) (*InternalKey, []byte) {
ikey := i.iter.SeekLT(key)
if ikey == nil {
return nil, nil
}
return ikey, i.Value()
}
func (i *batchIter) First() (*InternalKey, []byte) {
ikey := i.iter.First()
if ikey == nil {
return nil, nil
}
return ikey, i.Value()
}
func (i *batchIter) Last() (*InternalKey, []byte) {
ikey := i.iter.Last()
if ikey == nil {
return nil, nil
}
return ikey, i.Value()
}
func (i *batchIter) Next() (*InternalKey, []byte) {
ikey := i.iter.Next()
if ikey == nil {
return nil, nil
}
return ikey, i.Value()
}
func (i *batchIter) Prev() (*InternalKey, []byte) {
ikey := i.iter.Prev()
if ikey == nil {
return nil, nil
}
return ikey, i.Value()
}
func (i *batchIter) Key() *InternalKey {
return i.iter.Key()
}
func (i *batchIter) Value() []byte {
_, _, value, ok := batchDecode(i.batch.storage.data, i.iter.KeyOffset())
if !ok {
i.err = fmt.Errorf("corrupted batch")
}
return value
}
func (i *batchIter) Valid() bool {
return i.iter.Valid()
}
func (i *batchIter) Error() error {
return i.err
}
func (i *batchIter) Close() error {
_ = i.iter.Close()
return i.err
}
func (i *batchIter) SetBounds(lower, upper []byte) {
i.iter.SetBounds(lower, upper)
}
type flushableBatchEntry struct {
offset uint32
index uint32
keyStart uint32
keyEnd uint32
}
// flushableBatch wraps an existing batch and provides the interfaces needed
// for making the batch flushable (i.e. able to mimic a memtable).
type flushableBatch struct {
cmp Compare
data []byte
// The base sequence number for the entries in the batch. This is the same
// value as Batch.seqNum() and is cached here for performance.
seqNum uint64
// A slice of offsets and indices for the entries in the batch. Used to
// implement flushableBatchIter. Unlike the indexing on a normal batch, a
// flushable batch is indexed such that batch entry i will be given the
// sequence number flushableBatch.seqNum+i.
offsets []flushableBatchEntry
// Fragmented range deletion tombstones.
tombstones []rangedel.Tombstone
flushedCh chan struct{}
logNum uint64
}
var _ flushable = (*flushableBatch)(nil)
// newFlushableBatch creates a new batch that implements the flushable
// interface. This allows the batch to act like a memtable and be placed in the
// queue of flushable memtables. Note that the flushable batch takes ownership
// of the batch data.
func newFlushableBatch(batch *Batch, comparer *Comparer) *flushableBatch {
b := &flushableBatch{
data: batch.storage.data,
cmp: comparer.Compare,
offsets: make([]flushableBatchEntry, 0, batch.count()),
flushedCh: make(chan struct{}),
}
var index uint32
var rangeDelOffsets []flushableBatchEntry
for iter := BatchReader(b.data[batchHeaderLen:]); len(iter) > 0; index++ {
offset := uintptr(unsafe.Pointer(&iter[0])) - uintptr(unsafe.Pointer(&b.data[0]))
kind, key, _, ok := iter.Next()
if !ok {
break
}
entry := flushableBatchEntry{
offset: uint32(offset),
index: uint32(index),
}
if keySize := uint32(len(key)); keySize == 0 {
// Must add 2 to the offset. One byte encodes `kind` and the next
// byte encodes `0`, which is the length of the key.
entry.keyStart = uint32(offset) + 2
entry.keyEnd = entry.keyStart
} else {
entry.keyStart = uint32(uintptr(unsafe.Pointer(&key[0])) -
uintptr(unsafe.Pointer(&b.data[0])))
entry.keyEnd = entry.keyStart + keySize
}
if kind == InternalKeyKindRangeDelete {
rangeDelOffsets = append(rangeDelOffsets, entry)
} else {
b.offsets = append(b.offsets, entry)
}
}
// Sort both offsets and rangeDelOffsets.
sort.Sort(b)
rangeDelOffsets, b.offsets = b.offsets, rangeDelOffsets
sort.Sort(b)
rangeDelOffsets, b.offsets = b.offsets, rangeDelOffsets
if len(rangeDelOffsets) > 0 {
frag := &rangedel.Fragmenter{
Cmp: b.cmp,
Emit: func(fragmented []rangedel.Tombstone) {
b.tombstones = append(b.tombstones, fragmented...)
},
}
it := &flushableBatchIter{
batch: b,
data: b.data,
offsets: rangeDelOffsets,
cmp: b.cmp,
index: -1,
}
for {
key, val := it.Next()
if key == nil {
break
}
frag.Add(*key, val)
}
frag.Finish()
}
return b
}
func (b *flushableBatch) Len() int {
return len(b.offsets)
}
func (b *flushableBatch) Less(i, j int) bool {
ei := &b.offsets[i]
ej := &b.offsets[j]
ki := b.data[ei.keyStart:ei.keyEnd]
kj := b.data[ej.keyStart:ej.keyEnd]
switch c := b.cmp(ki, kj); {
case c < 0:
return true
case c > 0:
return false
default:
return ei.offset > ej.offset
}
}
func (b *flushableBatch) Swap(i, j int) {
b.offsets[i], b.offsets[j] = b.offsets[j], b.offsets[i]
}
func (b *flushableBatch) newIter(o *IterOptions) internalIterator {
return &flushableBatchIter{
batch: b,
data: b.data,
offsets: b.offsets,
cmp: b.cmp,
index: -1,
lower: o.GetLowerBound(),
upper: o.GetUpperBound(),
}
}
func (b *flushableBatch) newFlushIter(o *IterOptions, bytesFlushed *uint64) internalIterator {
return &flushFlushableBatchIter{
flushableBatchIter: flushableBatchIter{
batch: b,
data: b.data,
offsets: b.offsets,
cmp: b.cmp,
index: -1,
},
bytesIterated: bytesFlushed,
}
}
func (b *flushableBatch) newRangeDelIter(o *IterOptions) internalIterator {
if len(b.tombstones) == 0 {
return nil
}
return rangedel.NewIter(b.cmp, b.tombstones)
}
func (b *flushableBatch) totalBytes() uint64 {
return uint64(len(b.data) - batchHeaderLen)
}
func (b *flushableBatch) flushed() chan struct{} {