-
Notifications
You must be signed in to change notification settings - Fork 7
/
Copy pathstarskey.go
1733 lines (1460 loc) · 43.5 KB
/
starskey.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
// Package starskey
//
// (C) Copyright Starskey
//
// Original Author: Alex Gaetano Padula
//
// Licensed under the Mozilla Public License, v. 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// https://www.mozilla.org/en-US/MPL/2.0/
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package starskey
import (
"bytes"
"errors"
"fmt"
"github.com/klauspost/compress/s2"
"github.com/klauspost/compress/snappy"
"github.com/starskey-io/starskey/bloomfilter"
"github.com/starskey-io/starskey/pager"
"github.com/starskey-io/starskey/ttree"
"go.mongodb.org/mongo-driver/bson" // It's fast and simple for our use case
"log"
"os"
"strings"
"sync"
"time"
)
// Global variables
var (
WALExtension = ".wal" // Write ahead log extension
VLogExtension = ".vlog" // value log extension
KLogExtension = ".klog" // key log extension
LogExtension = ".log" // debug log extension
BloomFilterExtension = ".bf" // bloom filter extension
SSTPrefix = "sst_" // SSTable prefix
LevelPrefix = "l" // Level prefix
PageSize = 128 // Page size, smaller is better. The pager handles overflowing in sequence. 1024, or 1024 will cause VERY large files.
SyncInterval = time.Millisecond * 512 // File sync interval
Tombstone = []byte{0xDE, 0xAD, 0xBE, 0xEF} // Tombstone value
TTreeMin = 12 // Minimum degree of the T-Tree
TTreeMax = 32 // Maximum degree of the T-Tree
BloomFilterProbability = 0.01 // Bloom filter probability
)
// Config represents the configuration for starskey instance
type Config struct {
Permission os.FileMode // Directory and file permissions
Directory string // Directory to store the starskey files
FlushThreshold uint64 // Flush threshold for memtable
MaxLevel uint64 // Maximum number of levels
SizeFactor uint64 // Size factor for each level
BloomFilter bool // Enable bloom filter
Logging bool // Enable log file
Compression bool // Enable compression
CompressionOption CompressionOption // Desired compression option
}
// Level represents a disk level
type Level struct {
id int // Level number
sstables []*SSTable // SSTables in the level
maxSize int // Maximum size of the level
sizeFactor int // Size factor, is multiplied by the flush threshold
}
// WAL represents a write-ahead log
type WAL struct {
pager *pager.Pager // Pager for the write-ahead log
}
// OperationType represents the type of operation for a WAL record and transactions
type OperationType int
const (
Put OperationType = iota
Delete
Get
)
type CompressionOption int
// Compression options
const (
NoCompression CompressionOption = iota
SnappyCompression
S2Compression
)
// WALRecord represents a WAL record
type WALRecord struct {
Key []byte // Key
Value []byte // Value
Op OperationType // Operation type
}
// SSTable represents a sorted string table
type SSTable struct {
klog *pager.Pager // Key log, stores KLogRecord records
vlog *pager.Pager // Value log, stores VLogRecord records
bloomfilter *bloomfilter.BloomFilter // In-memory bloom filter for the SSTable, can be nil if not configured
}
// KLogRecord represents a key log record
type KLogRecord struct {
Key []byte // The key
ValPageNum uint64 // The page number of the value in the value log
}
// VLogRecord represents a value log record
type VLogRecord struct {
Value []byte // The value
}
// Starskey represents the main struct for the package
type Starskey struct {
wal *pager.Pager // Write-ahead log
memtable *ttree.TTree // Memtable
levels []*Level // Disk levels
config *Config // Starskey configuration
lock *sync.Mutex // Mutex for thread safety
logFile *os.File // Debug log file
}
// Txn represents a transaction
type Txn struct {
db *Starskey // The db instance
operations []*TxnOperation // Operations in the transaction
lock *sync.Mutex // Mutex for thread safety
}
// TxnOperation represents an operation in a transaction
type TxnOperation struct {
key []byte // Key
value []byte // Value
op OperationType // Operation type
rollback *TxnRollbackOperation // The rollback for the operation
commited bool // Transaction status
}
// TxnRollbackOperation represents a rollback operation in a transaction
type TxnRollbackOperation struct {
key []byte // Key
value []byte // Value
op OperationType // Operation type
}
// Open opens a new Starskey instance with the given configuration
func Open(config *Config) (*Starskey, error) {
// Check if config is nil
if config == nil {
return nil, errors.New("config cannot be nil")
}
// Create new starskey instance
skey := &Starskey{
config: config,
}
// Validate configs
if config.FlushThreshold == 0 {
return nil, errors.New("flush threshold cannot be zero")
}
if config.MaxLevel < 3 {
return nil, errors.New("max level cannot be less than 3")
}
if config.SizeFactor < 4 {
return nil, errors.New("size factor cannot be less than 4")
}
if len(config.Directory) == 0 {
return nil, errors.New("directory cannot be empty")
}
if config.Permission == 0 {
config.Permission = 750 // Default permission
}
// If compression is configured we check if option is valid
if config.Compression {
switch config.CompressionOption {
case SnappyCompression, S2Compression: // All good
default:
return nil, errors.New("invalid compression option")
}
}
// We check if configured directory ends with a slash, if not we add it
if string(config.Directory[len(config.Directory)-1]) != string(os.PathSeparator) {
config.Directory += string(os.PathSeparator)
}
// We create the configured directory
// (will not create if it already exists)
if err := os.MkdirAll(config.Directory, config.Permission); err != nil {
return nil, err
}
// We check if logging is configured,
// If so we log to file instead of standard output
if skey.config.Logging {
logFile, err := os.OpenFile(fmt.Sprintf("%s%s", skey.config.Directory, LogExtension), os.O_CREATE|os.O_APPEND|os.O_WRONLY, config.Permission)
if err != nil {
return nil, err
}
log.SetOutput(logFile) // We set the log output to the file
skey.logFile = logFile
}
// We log the configuration
log.Println("Opening Starskey with config:")
log.Println("Directory: ", config.Directory)
log.Println("FlushThreshold: ", config.FlushThreshold)
log.Println("MaxLevel: ", config.MaxLevel)
log.Println("SizeFactor: ", config.SizeFactor)
log.Println("BloomFilter: ", config.BloomFilter)
log.Println("Logging: ", config.Logging)
log.Println("Opening write ahead log")
// We create/open the write-ahead log within the configured directory
walPath := config.Directory + WALExtension
wal, err := pager.Open(walPath, os.O_RDWR|os.O_CREATE, config.Permission, PageSize, true, SyncInterval)
if err != nil {
return nil, err
}
// We set the write-ahead log
skey.wal = wal
log.Println("Write-ahead log opened successfully")
log.Println("Creating memory table")
// We create the memtable
skey.memtable = ttree.New(TTreeMin, TTreeMax)
log.Println("Memory table created successfully")
log.Println("Opening levels")
// We open disk levels and their SSTables
skey.levels, err = openLevels(config)
if err != nil {
return nil, err
}
log.Println("Levels opened successfully")
skey.lock = &sync.Mutex{}
log.Println("Replaying WAL")
// We replay the write-ahead log and populate the memtable
if err = skey.replayWAL(); err != nil {
return nil, err
}
log.Println("WAL replayed successfully")
log.Println("Starskey opened successfully")
return skey, nil
}
// Close closes the Starskey instance
func (skey *Starskey) Close() error {
log.Println("Closing WAL")
// Close the write-ahead log
if err := skey.wal.Close(); err != nil {
return err
}
log.Println("Closed WAL")
log.Println("Closing levels")
for _, level := range skey.levels {
log.Println("Closing level", level.id)
for _, sstable := range level.sstables {
// We close opened sstable files
if err := sstable.klog.Close(); err != nil {
return err
}
if err := sstable.vlog.Close(); err != nil {
return err
}
}
log.Println("Level", level.id, "closed")
}
log.Println("Levels closed")
log.Println("Starskey closed")
if skey.logFile != nil { // If log configured, we close it
if err := skey.logFile.Close(); err != nil {
return err
}
}
return nil
}
func (skey *Starskey) appendToWal(record *WALRecord) error {
// Serialize the WAL record
walSerialized, err := serializeWalRecord(record, skey.config.Compression, skey.config.CompressionOption)
if err != nil {
return err
}
// Write the WAL record to the write-ahead log
if _, err = skey.wal.Write(walSerialized); err != nil {
return err
}
return nil
}
// Put puts a key-value pair into the database
func (skey *Starskey) Put(key, value []byte) error {
// We validate the key and value
if len(key) == 0 {
return errors.New("key cannot be empty")
}
if len(value) == 0 {
return errors.New("value cannot be empty")
}
// Lock for thread safety
skey.lock.Lock()
defer skey.lock.Unlock()
// Append to WAL
err := skey.appendToWal(&WALRecord{
Key: key,
Value: value,
Op: Put,
})
if err != nil {
return err
}
// Put the key-value pair into the memtable
err = skey.memtable.Put(key, value)
if err != nil {
return err
}
// If the memtable size exceeds the flush threshold we trigger a sorted run to level 1
if skey.memtable.SizeOfTree >= skey.config.FlushThreshold {
// Sorted run to level 1
if err := skey.run(); err != nil {
return err
}
}
return nil
}
// Get retrieves a key from the database
func (skey *Starskey) Get(key []byte) ([]byte, error) {
// We validate the key
if len(key) == 0 {
return nil, errors.New("key cannot be empty")
}
// Lock for thread safety
skey.lock.Lock()
defer skey.lock.Unlock()
// Check memtable first
if value, exists := skey.memtable.Get(key); exists {
// Check for tombstone
if bytes.Equal(value.Value, Tombstone) {
return nil, nil
}
return value.Value, nil
}
// Search through levels
for _, level := range skey.levels {
for _, sstable := range level.sstables {
if sstable == nil {
continue
}
klog := sstable.klog
vlog := sstable.vlog
// Create a new iterator for the key log
it := pager.NewIterator(klog)
// If bloom filter is configured we check if key is in the bloom filter
if skey.config.BloomFilter {
// We check in-memory bloom filter first
if !sstable.bloomfilter.Contains(key) {
continue
}
}
for it.Next() {
data, err := it.Read()
if err != nil {
break
}
klogRecord, err := deserializeKLogRecord(data, skey.config.Compression, skey.config.CompressionOption)
if err != nil {
return nil, err
}
if bytes.Equal(klogRecord.Key, key) {
// We found the key
// We read the value from the value log
read, _, err := vlog.Read(int(klogRecord.ValPageNum))
if err != nil {
return nil, err
}
vlogRecord, err := deserializeVLogRecord(read, skey.config.Compression, skey.config.CompressionOption)
if err != nil {
return nil, err
}
// Check if the value is a tombstone
if bytes.Equal(vlogRecord.Value, Tombstone) {
return nil, nil
}
return vlogRecord.Value, nil
}
}
}
}
return nil, nil
}
// Delete deletes a key from the database
func (skey *Starskey) Delete(key []byte) error {
return skey.Put(key, Tombstone) // We simply put a tombstone value
}
// Range retrieves a range of values from the database
func (skey *Starskey) Range(startKey, endKey []byte) ([][]byte, error) {
// We validate the keys
if len(startKey) == 0 {
return nil, errors.New("start key cannot be empty")
}
if len(endKey) == 0 {
return nil, errors.New("end key cannot be empty")
}
// Start key cannot be greater than end key
if bytes.Compare(startKey, endKey) > 0 {
return nil, errors.New("start key cannot be greater than end key")
}
// Lock for thread safety
skey.lock.Lock()
defer skey.lock.Unlock()
// We create a slice to store the values
var result [][]byte
seenKeys := make(map[string]struct{}) // We use a map to keep track of seen keys
// Check memtable first
entries := skey.memtable.Range(startKey, endKey)
for _, entry := range entries {
result = append(result, entry.Value)
seenKeys[string(entry.Key)] = struct{}{}
}
// Search through levels
for _, level := range skey.levels {
for _, sstable := range level.sstables {
klog := sstable.klog
vlog := sstable.vlog
it := pager.NewIterator(klog)
// If bloom is configured skip first page which is the bloom filter
if skey.config.BloomFilter {
if !it.Next() {
continue
}
}
for it.Next() {
data, err := it.Read()
if err != nil {
return nil, err
}
klogRecord, err := deserializeKLogRecord(data, skey.config.Compression, skey.config.CompressionOption)
if err != nil {
return nil, err
}
if bytes.Compare(klogRecord.Key, startKey) >= 0 && bytes.Compare(klogRecord.Key, endKey) <= 0 {
if _, seen := seenKeys[string(klogRecord.Key)]; seen {
continue
}
read, _, err := vlog.Read(int(klogRecord.ValPageNum))
if err != nil {
return nil, err
}
vlogRecord, err := deserializeVLogRecord(read, skey.config.Compression, skey.config.CompressionOption)
if err != nil {
return nil, err
}
// Check if the value is a tombstone
if bytes.Equal(vlogRecord.Value, Tombstone) {
continue
}
result = append(result, vlogRecord.Value)
seenKeys[string(klogRecord.Key)] = struct{}{}
}
}
}
}
return result, nil
}
// FilterKeys retrieves values from the database that match a key filter
func (skey *Starskey) FilterKeys(compare func(key []byte) bool) ([][]byte, error) {
// We validate the compare function
if compare == nil {
return nil, errors.New("compare function cannot be nil")
}
// Lock for thread safety
skey.lock.Lock()
defer skey.lock.Unlock()
var result [][]byte
seenKeys := make(map[string]struct{})
// Check memtable first
iter := skey.memtable.NewIterator(false)
for iter.Valid() {
if entry, ok := iter.Current(); ok {
if compare(entry.Key) {
result = append(result, entry.Value)
seenKeys[string(entry.Key)] = struct{}{}
}
}
if !iter.HasNext() {
break
}
iter.Next()
}
// Search through levels
for _, level := range skey.levels {
for _, sstable := range level.sstables {
klog := sstable.klog
vlog := sstable.vlog
it := pager.NewIterator(klog)
if skey.config.BloomFilter {
if !it.Next() {
continue
}
}
for it.Next() {
data, err := it.Read()
if err != nil {
return nil, err
}
klogRecord, err := deserializeKLogRecord(data, skey.config.Compression, skey.config.CompressionOption)
if err != nil {
return nil, err
}
if compare(klogRecord.Key) {
if _, seen := seenKeys[string(klogRecord.Key)]; seen {
continue
}
read, _, err := vlog.Read(int(klogRecord.ValPageNum))
if err != nil {
return nil, err
}
vlogRecord, err := deserializeVLogRecord(read, skey.config.Compression, skey.config.CompressionOption)
if err != nil {
return nil, err
}
// Check if the value is a tombstone
if bytes.Equal(vlogRecord.Value, Tombstone) {
continue
}
result = append(result, vlogRecord.Value)
seenKeys[string(klogRecord.Key)] = struct{}{}
}
}
}
}
return result, nil
}
// BeginTxn begins a new transaction
func (skey *Starskey) BeginTxn() *Txn {
return &Txn{
operations: make([]*TxnOperation, 0),
lock: &sync.Mutex{},
db: skey,
}
}
// Get retrieves a key-value pair from a transaction
func (txn *Txn) Get(key []byte) ([]byte, error) {
// We validate the key
if len(key) == 0 {
return nil, errors.New("key cannot be empty")
}
// Lock for thread safety
txn.lock.Lock()
defer txn.lock.Unlock()
// Check if the key is in the transaction operations
for _, op := range txn.operations {
if bytes.Equal(op.key, key) {
if op.op == Delete {
return nil, nil // Key is marked for deletion
}
return op.value, nil
}
}
// If not found in transaction, check the database
value, err := txn.db.Get(key)
if err != nil {
return nil, err
}
return value, nil
}
// Put puts a key-value pair into the database from a transaction
func (txn *Txn) Put(key, value []byte) {
txn.lock.Lock()
defer txn.lock.Unlock()
txn.operations = append(txn.operations, &TxnOperation{
key: key,
value: value,
op: Put,
commited: false,
rollback: &TxnRollbackOperation{
key: key,
value: Tombstone,
op: Delete,
},
})
}
// Delete deletes a key from the database from a transaction
func (txn *Txn) Delete(key []byte) {
// Lock for thread safety
txn.lock.Lock()
defer txn.lock.Unlock()
currentValue, exists := txn.db.memtable.Get(key)
if exists {
txn.operations = append(txn.operations, &TxnOperation{
key: key,
value: currentValue.Value,
op: Delete,
commited: false,
rollback: &TxnRollbackOperation{
key: key,
value: currentValue.Value,
op: Put,
},
})
return
}
txn.operations = append(txn.operations, &TxnOperation{
key: key,
value: Tombstone,
op: Delete,
commited: false,
rollback: nil,
})
}
// Commit commits a transaction
func (txn *Txn) Commit() error {
// Lock for thread safety
txn.db.lock.Lock()
defer txn.db.lock.Unlock()
txn.lock.Lock()
defer txn.lock.Unlock()
for _, op := range txn.operations {
var record *WALRecord
switch op.op {
case Put:
// Create a WAL record
record = &WALRecord{
Key: op.key,
Value: op.value,
Op: Put,
}
case Delete:
record = &WALRecord{
Key: op.key,
Value: op.value,
Op: Delete,
}
case Get:
continue
}
// Append to WAL
err := txn.db.appendToWal(record)
if err != nil {
_ = txn.Rollback()
return err
}
// Escalate write
txn.db.wal.EscalateFSync()
// Put the key-value pair into the memtable
err = txn.db.memtable.Put(op.key, op.value)
if err != nil {
_ = txn.Rollback()
return err
}
op.commited = true
}
if txn.db.memtable.SizeOfTree >= txn.db.config.FlushThreshold {
// Sorted run to level 1
if err := txn.db.run(); err != nil {
_ = txn.Rollback()
return err
}
}
return nil
}
// Update runs a function within a transaction.
func (skey *Starskey) Update(fn func(tx *Txn) error) error {
// Begin a new transaction
txn := skey.BeginTxn()
if txn == nil {
return errors.New("failed to begin transaction")
}
// Call the provided function with the transaction
err := fn(txn)
if err != nil {
// If the function returns an error, roll back the transaction..
if rollbackErr := txn.Rollback(); rollbackErr != nil {
return fmt.Errorf("transaction rollback failed: %v, original error: %v", rollbackErr, err)
}
return err
}
// If the function succeeds, commit the transaction
if commitErr := txn.Commit(); commitErr != nil {
return fmt.Errorf("transaction commit failed: %v", commitErr)
}
return nil
}
// Rollback rolls back a transaction
func (txn *Txn) Rollback() error {
// Lock for thread safety
txn.db.lock.Lock()
defer txn.db.lock.Unlock()
txn.lock.Lock()
defer txn.lock.Unlock()
for _, op := range txn.operations {
if op.commited {
if op.rollback != nil {
// Create a WAL record
record := &WALRecord{
Key: op.rollback.key,
Value: op.rollback.value,
Op: op.rollback.op,
}
// Serialize the WAL record
walSerialized, err := serializeWalRecord(record, txn.db.config.Compression, txn.db.config.CompressionOption)
if err != nil {
return err
}
// Write the WAL record to the write-ahead log
if _, err = txn.db.wal.Write(walSerialized); err != nil {
return err
}
// Escalate write
txn.db.wal.EscalateFSync()
// Put the key-value pair into the memtable
err = txn.db.memtable.Put(op.rollback.key, op.rollback.value)
if err != nil {
return err
}
}
}
}
return nil
}
// openLevels opens disk levels and their SSTables and returns a slice of Level
func openLevels(config *Config) ([]*Level, error) {
levels := make([]*Level, config.MaxLevel) // We create a slice of levels
// We iterate over the number of levels
for i := 0; i < int(config.MaxLevel); i++ {
// We create level
levels[i] = &Level{
id: i + 1,
sstables: make([]*SSTable, 0),
maxSize: int(config.FlushThreshold) * int(config.SizeFactor) * (1 << uint(i)), // Size increases exponentially
sizeFactor: int(config.SizeFactor), // Size factor
}
// Open the SSTables
sstables, err := openSSTables(fmt.Sprintf("%s%s%d", config.Directory, LevelPrefix, i+1), config.BloomFilter, config.Permission)
if err != nil {
return nil, err
}
// Set the SSTables
levels[i].sstables = sstables
// Log that sh
log.Println("Level", i+1, "opened successfully")
}
return levels, nil
}
// openSSTables opens SSTables in a directory and returns a slice of SSTable
func openSSTables(directory string, bf bool, perm os.FileMode) ([]*SSTable, error) {
log.Println("Opening SSTables for level", directory)
sstables := make([]*SSTable, 0)
// We check if configured directory ends with a slash
if string(directory[len(directory)-1]) != string(os.PathSeparator) {
directory += string(os.PathSeparator)
}
// We create or the configured directory
if err := os.MkdirAll(directory, perm); err != nil {
return nil, err
}
// We read all files in the directory
files, err := os.ReadDir(directory)
if err != nil {
return nil, err
}
// We iterate over all files in the directory
for _, file := range files {
// If the file starts with the SST prefix there will be a key log and a value log
if file.IsDir() {
continue
}
if strings.HasPrefix(file.Name(), SSTPrefix) {
if strings.HasSuffix(file.Name(), KLogExtension) {
// Open the key log
klogPath := fmt.Sprintf("%s%s", directory, file.Name())
log.Println("Opening SSTable klog", klogPath)
klog, err := pager.Open(klogPath, os.O_CREATE|os.O_RDWR, perm, PageSize, true, SyncInterval)
if err != nil {
return nil, err
}
// Open the value log for the key log
vlogPath := strings.TrimRight(klogPath, KLogExtension) + VLogExtension
log.Println("Opening SSTable vlog", vlogPath)
vlog, err := pager.Open(vlogPath, os.O_CREATE|os.O_RDWR, perm, PageSize, true, SyncInterval)
if err != nil {
return nil, err
}
sst := &SSTable{
klog: klog,
vlog: vlog,
}
if bf {
log.Println("Opening bloom filter for SSTable", strings.TrimRight(klogPath, KLogExtension)+BloomFilterExtension)
bloomFilterFile, err := os.ReadFile(strings.TrimRight(klogPath, KLogExtension) + BloomFilterExtension)
if err != nil {
return nil, err
}
deserializedBf, err := bloomfilter.Deserialize(bloomFilterFile)
if err != nil {
return nil, err
}
sst.bloomfilter = deserializedBf
log.Println("Bloom filter opened successfully for SSTable")
}
// Append the SSTable to the list
sstables = append(sstables, sst)
}
}
}
return sstables, nil
}
// replayWal replays write ahead log and rebuilds the last memtable state
func (skey *Starskey) replayWAL() error {
if skey.wal.PageCount() == 0 {
log.Println("No records in WAL to replay")
return nil
}
// We create an iter for the write-ahead log
iter := pager.NewIterator(skey.wal)
// We iterate over all records in the write-ahead log
for iter.Next() {
data, err := iter.Read()
if err != nil {
return err
}
// Deserialize the WAL record
record, err := deserializeWalRecord(data, skey.config.Compression, skey.config.CompressionOption)
if err != nil {
return err
}
// We apply the operation in the WAL record
switch record.Op {
case Put:
err = skey.memtable.Put(record.Key, record.Value)
if err != nil {
return err
}
case Delete:
err = skey.memtable.Put(record.Key, Tombstone)
if err != nil {
return err
}
}
}
return nil
}
// run runs a sorted flush to disk level 1
func (skey *Starskey) run() error {
log.Println("Running sorted run to l1")