forked from polarsignals/frostdb
-
Notifications
You must be signed in to change notification settings - Fork 0
/
db.go
1340 lines (1196 loc) · 34.2 KB
/
db.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 frostdb
import (
"bytes"
"context"
"errors"
"fmt"
"io"
"math"
"os"
"path/filepath"
"runtime"
"strings"
"sync"
"sync/atomic"
"time"
"github.com/apache/arrow/go/v14/arrow/ipc"
"github.com/go-kit/log"
"github.com/go-kit/log/level"
"github.com/oklog/ulid"
"github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/client_golang/prometheus/promauto"
"go.opentelemetry.io/otel/trace"
"golang.org/x/exp/maps"
"golang.org/x/sync/errgroup"
"google.golang.org/protobuf/proto"
"github.com/polarsignals/frostdb/dynparquet"
schemapb "github.com/polarsignals/frostdb/gen/proto/go/frostdb/schema/v1alpha1"
schemav2pb "github.com/polarsignals/frostdb/gen/proto/go/frostdb/schema/v1alpha2"
tablepb "github.com/polarsignals/frostdb/gen/proto/go/frostdb/table/v1alpha1"
walpb "github.com/polarsignals/frostdb/gen/proto/go/frostdb/wal/v1alpha1"
"github.com/polarsignals/frostdb/index"
"github.com/polarsignals/frostdb/query/logicalplan"
"github.com/polarsignals/frostdb/wal"
)
const (
B = 1
KiB = 1024 * B
MiB = 1024 * KiB
GiB = 1024 * MiB
TiB = 1024 * GiB
)
type ColumnStore struct {
mtx sync.RWMutex
dbs map[string]*DB
dbReplaysInProgress map[string]chan struct{}
reg prometheus.Registerer
logger log.Logger
tracer trace.Tracer
granuleSizeBytes int64
activeMemorySize int64
storagePath string
enableWAL bool
manualBlockRotation bool
snapshotTriggerSize int64
metrics metrics
recoveryConcurrency int
// indexDegree is the degree of the btree index (default = 2)
indexDegree int
// splitSize is the number of new granules that are created when granules are split (default =2)
splitSize int
// indexConfig is the configuration settings for the lsm index
indexConfig []*IndexConfig
sources []DataSource
sinks []DataSink
compactAfterRecovery bool
compactAfterRecoveryTableNames []string
// testingOptions are options only used for testing purposes.
testingOptions struct {
disableReclaimDiskSpaceOnSnapshot bool
}
}
type metrics struct {
shutdownDuration prometheus.Histogram
shutdownStarted prometheus.Counter
shutdownCompleted prometheus.Counter
}
type Option func(*ColumnStore) error
func New(
options ...Option,
) (*ColumnStore, error) {
s := &ColumnStore{
dbs: make(map[string]*DB),
dbReplaysInProgress: make(map[string]chan struct{}),
reg: prometheus.NewRegistry(),
logger: log.NewNopLogger(),
tracer: trace.NewNoopTracerProvider().Tracer(""),
indexConfig: DefaultIndexConfig(),
indexDegree: 2,
splitSize: 2,
granuleSizeBytes: 1 * MiB,
activeMemorySize: 512 * MiB,
}
for _, option := range options {
if err := option(s); err != nil {
return nil, err
}
}
s.metrics = metrics{
shutdownDuration: promauto.With(s.reg).NewHistogram(prometheus.HistogramOpts{
Name: "frostdb_shutdown_duration",
Help: "time it takes for the columnarstore to complete a full shutdown.",
}),
shutdownStarted: promauto.With(s.reg).NewCounter(prometheus.CounterOpts{
Name: "frostdb_shutdown_started",
Help: "Indicates a shutdown of the columnarstore has started.",
}),
shutdownCompleted: promauto.With(s.reg).NewCounter(prometheus.CounterOpts{
Name: "frostdb_shutdown_completed",
Help: "Indicates a shutdown of the columnarstore has completed.",
}),
}
if s.enableWAL && s.storagePath == "" {
return nil, fmt.Errorf("storage path must be configured if WAL is enabled")
}
if err := s.recoverDBsFromStorage(context.Background()); err != nil {
return nil, err
}
return s, nil
}
func WithLogger(logger log.Logger) Option {
return func(s *ColumnStore) error {
s.logger = logger
return nil
}
}
func WithTracer(tracer trace.Tracer) Option {
return func(s *ColumnStore) error {
s.tracer = tracer
return nil
}
}
func WithRegistry(reg prometheus.Registerer) Option {
return func(s *ColumnStore) error {
s.reg = reg
return nil
}
}
func WithGranuleSizeBytes(bytes int64) Option {
return func(s *ColumnStore) error {
s.granuleSizeBytes = bytes
return nil
}
}
func WithActiveMemorySize(size int64) Option {
return func(s *ColumnStore) error {
s.activeMemorySize = size
return nil
}
}
func WithIndexDegree(indexDegree int) Option {
return func(s *ColumnStore) error {
s.indexDegree = indexDegree
return nil
}
}
func WithSplitSize(size int) Option {
return func(s *ColumnStore) error {
s.splitSize = size
return nil
}
}
func WithReadWriteStorage(ds DataSinkSource) Option {
return func(s *ColumnStore) error {
s.sources = append(s.sources, ds)
s.sinks = append(s.sinks, ds)
return nil
}
}
func WithReadOnlyStorage(ds DataSource) Option {
return func(s *ColumnStore) error {
s.sources = append(s.sources, ds)
return nil
}
}
func WithWriteOnlyStorage(ds DataSink) Option {
return func(s *ColumnStore) error {
s.sinks = append(s.sinks, ds)
return nil
}
}
func WithManualBlockRotation() Option {
return func(s *ColumnStore) error {
s.manualBlockRotation = true
return nil
}
}
func WithWAL() Option {
return func(s *ColumnStore) error {
s.enableWAL = true
return nil
}
}
func WithStoragePath(path string) Option {
return func(s *ColumnStore) error {
s.storagePath = path
return nil
}
}
func WithIndexConfig(indexConfig []*IndexConfig) Option {
return func(s *ColumnStore) error {
s.indexConfig = indexConfig
return nil
}
}
func WithCompactionAfterRecovery(tableNames []string) Option {
return func(s *ColumnStore) error {
s.compactAfterRecovery = true
s.compactAfterRecoveryTableNames = tableNames
return nil
}
}
// WithSnapshotTriggerSize specifies a size in bytes of uncompressed inserts
// that will trigger a snapshot of the whole database. This can be larger than
// the active memory size given that the active memory size tracks the size of
// *compressed* data, while snapshots are triggered based on the *uncompressed*
// data inserted into the database. The reason this choice was made is that
// if a database instance crashes, it is forced to reread all uncompressed
// inserts since the last snapshot from the WAL, which could potentially lead
// to unrecoverable OOMs on startup. Defining the snapshot trigger in terms of
// uncompressed bytes limits the memory usage on recovery to at most the
// snapshot trigger size (as long as snapshots were successful).
// If 0, snapshots are disabled. Note that snapshots (if enabled) are also
// triggered on block rotation of any database table.
// Snapshots are complementary to the WAL and will also be disabled if the WAL
// is disabled.
func WithSnapshotTriggerSize(size int64) Option {
return func(s *ColumnStore) error {
s.snapshotTriggerSize = size
return nil
}
}
// WithRecoveryConcurrency limits the number of databases that are recovered
// simultaneously when calling frostdb.New. This helps limit memory usage on
// recovery.
func WithRecoveryConcurrency(concurrency int) Option {
return func(s *ColumnStore) error {
s.recoveryConcurrency = concurrency
return nil
}
}
// Close persists all data from the columnstore to storage.
// It is no longer valid to use the coumnstore for reads or writes, and the object should not longer be reused.
func (s *ColumnStore) Close() error {
s.mtx.Lock()
defer s.mtx.Unlock()
s.metrics.shutdownStarted.Inc()
defer s.metrics.shutdownCompleted.Inc()
defer func(ts time.Time) {
s.metrics.shutdownDuration.Observe(float64(time.Since(ts)))
}(time.Now())
errg := &errgroup.Group{}
errg.SetLimit(runtime.GOMAXPROCS(0))
for _, db := range s.dbs {
toClose := db
errg.Go(func() error {
err := toClose.Close()
if err != nil {
level.Error(s.logger).Log("msg", "error closing DB", "db", toClose.name, "err", err)
}
return err
})
}
return errg.Wait()
}
func (s *ColumnStore) DatabasesDir() string {
return filepath.Join(s.storagePath, "databases")
}
// recoverDBsFromStorage replays the snapshots and write-ahead logs for each database.
func (s *ColumnStore) recoverDBsFromStorage(ctx context.Context) error {
if !s.enableWAL {
return nil
}
dir := s.DatabasesDir()
if _, err := os.Stat(dir); err != nil {
if os.IsNotExist(err) {
level.Debug(s.logger).Log("msg", "WAL directory does not exist, no WAL to replay")
return nil
}
return err
}
files, err := os.ReadDir(dir)
if err != nil {
return err
}
g, ctx := errgroup.WithContext(ctx)
// Limit this operation since WAL recovery could be very memory intensive.
if s.recoveryConcurrency == 0 {
s.recoveryConcurrency = runtime.GOMAXPROCS(0)
}
g.SetLimit(s.recoveryConcurrency)
for _, f := range files {
databaseName := f.Name()
g.Go(func() error {
// Open the DB for the side effect of the snapshot and WALs being loaded as part of the open operation.
_, err := s.DB(ctx,
databaseName,
WithCompactionAfterOpen(
s.compactAfterRecovery, s.compactAfterRecoveryTableNames,
),
)
return err
})
}
return g.Wait()
}
type dbMetrics struct {
txHighWatermark prometheus.GaugeFunc
snapshotMetrics *snapshotMetrics
}
type DB struct {
columnStore *ColumnStore
reg prometheus.Registerer
logger log.Logger
tracer trace.Tracer
name string
mtx *sync.RWMutex
roTables map[string]*Table
tables map[string]*Table
storagePath string
wal WAL
// The database supports multiple data sources and sinks.
sources []DataSource
sinks []DataSink
// Databases monotonically increasing transaction id
tx atomic.Uint64
// highWatermark maintains the highest consecutively completed txn.
highWatermark atomic.Uint64
// TxPool is a waiting area for finished transactions that haven't been added to the watermark
txPool *TxPool
compactAfterRecovery bool
compactAfterRecoveryTableNames []string
snapshotInProgress atomic.Bool
metrics *dbMetrics
}
// DataSinkSource is a convenience interface for a data source and sink.
type DataSinkSource interface {
DataSink
DataSource
}
// DataSource is remote source of data that can be queried.
type DataSource interface {
fmt.Stringer
Scan(ctx context.Context, prefix string, schema *dynparquet.Schema, filter logicalplan.Expr, lastBlockTimestamp uint64, callback func(context.Context, any) error) error
Prefixes(ctx context.Context, prefix string) ([]string, error)
}
// DataSink is a remote destination for data.
type DataSink interface {
fmt.Stringer
Upload(ctx context.Context, name string, r io.Reader) error
Delete(ctx context.Context, name string) error
}
type DBOption func(*DB) error
func WithCompactionAfterOpen(compact bool, tableNames []string) DBOption {
return func(db *DB) error {
db.compactAfterRecovery = compact
db.compactAfterRecoveryTableNames = tableNames
return nil
}
}
// DB gets or creates a database on the given ColumnStore with the given
// options. Note that if the database already exists, the options will be
// applied cumulatively to the database.
func (s *ColumnStore) DB(ctx context.Context, name string, opts ...DBOption) (*DB, error) {
if !validateName(name) {
return nil, errors.New("invalid database name")
}
applyOptsToDB := func(db *DB) error {
db.mtx.Lock()
defer db.mtx.Unlock()
for _, opt := range opts {
if err := opt(db); err != nil {
return err
}
}
return nil
}
s.mtx.RLock()
db, ok := s.dbs[name]
s.mtx.RUnlock()
if ok {
if err := applyOptsToDB(db); err != nil {
return nil, err
}
return db, nil
}
s.mtx.Lock()
defer s.mtx.Unlock()
// Need to double-check that in the meantime a database with the same name
// wasn't concurrently created.
for {
db, ok = s.dbs[name]
if ok {
if err := applyOptsToDB(db); err != nil {
return nil, err
}
return db, nil
}
// DB has not yet been created. However, another goroutine might be
// replaying the WAL in the background (the store mutex is released
// during replay.).
waitForReplay, ok := s.dbReplaysInProgress[name]
if !ok {
// No replay in progress, it is safe to create the DB.
break
}
s.mtx.Unlock()
<-waitForReplay
s.mtx.Lock()
}
reg := prometheus.WrapRegistererWith(prometheus.Labels{"db": name}, s.reg)
logger := log.WithPrefix(s.logger, "db", name)
db = &DB{
columnStore: s,
name: name,
mtx: &sync.RWMutex{},
tables: map[string]*Table{},
roTables: map[string]*Table{},
reg: reg,
logger: logger,
tracer: s.tracer,
storagePath: filepath.Join(s.DatabasesDir(), name),
wal: &wal.NopWAL{},
sources: s.sources,
sinks: s.sinks,
}
if err := applyOptsToDB(db); err != nil {
return nil, err
}
if dbSetupErr := func() error {
if err := os.RemoveAll(db.trashDir()); err != nil {
return err
}
db.txPool = NewTxPool(&db.highWatermark)
// Wait to start the compactor pool since benchmarks show that WAL
// replay is a lot more efficient if it is not competing against
// compaction. Additionally, if the CompactAfterRecovery option is
// specified, we don't want the user-specified compaction to race with
// our compactor pool.
if len(db.sources) != 0 {
for _, source := range db.sources {
prefixes, err := source.Prefixes(ctx, name)
if err != nil {
return err
}
for _, prefix := range prefixes {
_, err := db.readOnlyTable(prefix)
if err != nil {
return err
}
}
}
}
if s.enableWAL {
if err := func() error {
// Unlock the store mutex while the WAL is replayed, otherwise
// if multiple DBs are opened in parallel, WAL replays will not
// happen in parallel. However, create a channel for any
// goroutines that might concurrently try to open the same DB
// to listen on.
s.dbReplaysInProgress[name] = make(chan struct{})
s.mtx.Unlock()
defer func() {
s.mtx.Lock()
close(s.dbReplaysInProgress[name])
delete(s.dbReplaysInProgress, name)
}()
var err error
db.wal, err = db.openWAL(ctx)
return err
}(); err != nil {
return err
}
// WAL pointers of tables need to be updated to the DB WAL since
// they are loaded from object storage and snapshots with a no-op
// WAL by default.
for _, table := range db.tables {
if !table.config.Load().DisableWal {
table.wal = db.wal
}
}
for _, table := range db.roTables {
if !table.config.Load().DisableWal {
table.wal = db.wal
}
}
}
// Register metrics last to avoid duplicate registration should and of the WAL or storage replay errors occur
db.metrics = &dbMetrics{
txHighWatermark: promauto.With(reg).NewGaugeFunc(prometheus.GaugeOpts{
Name: "frostdb_tx_high_watermark",
Help: "The highest transaction number that has been released to be read",
}, func() float64 {
return float64(db.highWatermark.Load())
}),
snapshotMetrics: newSnapshotMetrics(reg),
}
return nil
}(); dbSetupErr != nil {
level.Warn(s.logger).Log(
"msg", "error setting up db",
"name", name,
"err", dbSetupErr,
)
// closeInternal handles closing partially set fields in the db without
// rotating blocks etc... that the public Close method does.
_ = db.closeInternal()
return nil, dbSetupErr
}
// Compact tables after recovery if requested.
if db.compactAfterRecovery {
tables := db.compactAfterRecoveryTableNames
if len(tables) == 0 {
// Run compaction on all tables.
tables = maps.Keys(db.tables)
}
for _, name := range tables {
tbl, err := db.GetTable(name)
if err != nil {
level.Warn(db.logger).Log("msg", "get table during db setup", "err", err)
continue
}
start := time.Now()
if err := tbl.EnsureCompaction(); err != nil {
level.Warn(db.logger).Log("msg", "compaction during setup", "err", err)
}
level.Info(db.logger).Log(
"msg", "compacted table after recovery", "table", name, "took", time.Since(start),
)
}
}
s.dbs[name] = db
return db, nil
}
// DBs returns all the DB names of this column store.
func (s *ColumnStore) DBs() []string {
s.mtx.RLock()
defer s.mtx.RUnlock()
return maps.Keys(s.dbs)
}
func (s *ColumnStore) GetDB(name string) (*DB, error) {
s.mtx.RLock()
defer s.mtx.RUnlock()
db, ok := s.dbs[name]
if !ok {
return nil, fmt.Errorf("db %s not found", name)
}
return db, nil
}
func (s *ColumnStore) DropDB(name string) error {
db, err := s.GetDB(name)
if err != nil {
return err
}
if err := db.Close(WithClearStorage()); err != nil {
return err
}
s.mtx.Lock()
defer s.mtx.Unlock()
delete(s.dbs, name)
return os.Remove(filepath.Join(s.DatabasesDir(), name))
}
func (db *DB) openWAL(ctx context.Context) (WAL, error) {
wal, err := wal.Open(
db.logger,
db.reg,
db.walDir(),
)
if err != nil {
return nil, err
}
if err := db.recover(ctx, wal); err != nil {
return nil, err
}
wal.RunAsync()
return wal, nil
}
const (
walPath = "wal"
snapshotsPath = "snapshots"
)
func (db *DB) walDir() string {
return filepath.Join(db.storagePath, walPath)
}
func (db *DB) snapshotsDir() string {
return filepath.Join(db.storagePath, snapshotsPath)
}
func (db *DB) trashDir() string {
return filepath.Join(db.storagePath, "trash")
}
// recover attempts to recover database state from a combination of snapshots
// and the WAL.
func (db *DB) recover(ctx context.Context, wal WAL) error {
level.Info(db.logger).Log(
"msg", "recovering db",
"name", db.name,
)
snapshotLoadStart := time.Now()
snapshotTx, err := db.loadLatestSnapshot(ctx)
if err != nil {
level.Info(db.logger).Log(
"msg", "failed to load latest snapshot", "db", db.name, "err", err,
)
snapshotTx = 0
}
snapshotLogArgs := make([]any, 0)
if snapshotTx != 0 {
snapshotLogArgs = append(
snapshotLogArgs,
"snapshot_tx", snapshotTx,
"snapshot_load_duration", time.Since(snapshotLoadStart),
)
if err := db.cleanupSnapshotDir(ctx, snapshotTx); err != nil {
// Truncation is best-effort. If it fails, move on.
level.Info(db.logger).Log(
"msg", "failed to truncate snapshots not equal to loaded snapshot",
"err", err,
"snapshot_tx", snapshotTx,
)
}
if err := wal.Truncate(snapshotTx); err != nil {
level.Info(db.logger).Log(
"msg", "failed to truncate WAL after loading snapshot",
"err", err,
"snapshot_tx", snapshotTx,
)
}
}
// persistedTables is a map from a table name to the last transaction
// persisted.
persistedTables := map[string]uint64{}
var lastTx uint64
start := time.Now()
if err := wal.Replay(snapshotTx, func(tx uint64, record *walpb.Record) error {
if err := ctx.Err(); err != nil {
return err
}
switch e := record.Entry.EntryType.(type) {
case *walpb.Entry_TableBlockPersisted_:
persistedTables[e.TableBlockPersisted.TableName] = tx
if tx > snapshotTx {
// The loaded snapshot has data in a table that has been
// persisted. Delete all data in this table, since it has
// already been persisted.
db.mtx.Lock()
if table, ok := db.tables[e.TableBlockPersisted.TableName]; ok {
table.ActiveBlock().index, err = index.NewLSM(
table.name,
table.schema,
table.configureLSMLevels(db.columnStore.indexConfig),
index.LSMWithMetrics(table.metrics.indexMetrics),
)
if err != nil {
return err
}
}
db.mtx.Unlock()
}
return nil
default:
return nil
}
}); err != nil {
return err
}
// performSnapshot is set to true if a snapshot should be performed after
// replay. This is set in cases where there could be "dead bytes" in the
// WAL (i.e. entries that occupy space on disk but are useless).
performSnapshot := false
// Writes are performed concurrently to speed up replay.
var writeWg errgroup.Group
writeWg.SetLimit(runtime.GOMAXPROCS(0))
if err := wal.Replay(snapshotTx, func(tx uint64, record *walpb.Record) error {
if err := ctx.Err(); err != nil {
return err
}
lastTx = tx
switch e := record.Entry.EntryType.(type) {
case *walpb.Entry_NewTableBlock_:
entry := e.NewTableBlock
var schema proto.Message
switch v := entry.Config.Schema.(type) {
case *tablepb.TableConfig_DeprecatedSchema:
schema = v.DeprecatedSchema
case *tablepb.TableConfig_SchemaV2:
schema = v.SchemaV2
default:
return fmt.Errorf("unhandled schema type: %T", v)
}
var id ulid.ULID
if err := id.UnmarshalBinary(entry.BlockId); err != nil {
return err
}
if lastPersistedTx, ok := persistedTables[entry.TableName]; ok && tx < lastPersistedTx {
// This block has already been successfully persisted, so we can
// skip it.
return nil
}
tableName := entry.TableName
table, err := db.GetTable(tableName)
var tableErr ErrTableNotFound
if errors.As(err, &tableErr) {
return func() error {
db.mtx.Lock()
defer db.mtx.Unlock()
config := NewTableConfig(schema, FromConfig(entry.Config))
if _, ok := db.roTables[tableName]; ok {
table, err = db.promoteReadOnlyTableLocked(tableName, config)
if err != nil {
return fmt.Errorf("promoting read only table: %w", err)
}
} else {
table, err = newTable(
db,
tableName,
config,
db.reg,
db.logger,
db.tracer,
wal,
)
if err != nil {
return fmt.Errorf("instantiate table: %w", err)
}
}
table.active, err = newTableBlock(table, 0, tx, id)
if err != nil {
return err
}
db.tables[tableName] = table
return nil
}()
}
if err != nil {
return fmt.Errorf("get table: %w", err)
}
// If we get to this point it means a block was finished but did
// not get persisted.
level.Info(db.logger).Log(
"msg", "writing unfinished block in recovery",
"table", tableName,
"tx", tx,
)
table.pendingBlocks[table.active] = struct{}{}
go table.writeBlock(table.active, db.columnStore.manualBlockRotation, false)
protoEqual := false
switch schema.(type) {
case *schemav2pb.Schema:
protoEqual = proto.Equal(schema, table.config.Load().GetSchemaV2())
case *schemapb.Schema:
protoEqual = proto.Equal(schema, table.config.Load().GetDeprecatedSchema())
}
if !protoEqual {
// If schemas are identical from block to block we should we
// reuse the previous schema in order to retain pooled memory
// for it.
schema, err := dynparquet.SchemaFromDefinition(schema)
if err != nil {
return fmt.Errorf("initialize schema: %w", err)
}
table.schema = schema
}
table.active, err = newTableBlock(table, table.active.minTx, tx, id)
if err != nil {
return err
}
case *walpb.Entry_Write_:
entry := e.Write
tableName := entry.TableName
if lastPersistedTx, ok := persistedTables[tableName]; ok && tx < lastPersistedTx {
// This write has already been successfully persisted, so we can
// skip it.
return nil
}
table, err := db.GetTable(tableName)
var tableErr ErrTableNotFound
if errors.As(err, &tableErr) {
// This means the WAL was truncated at a point where this write
// was already successfully persisted to disk in more optimized
// form than the WAL.
return nil
}
if err != nil {
return fmt.Errorf("get table: %w", err)
}
writeWg.Go(func() error {
switch e.Write.Arrow {
case true:
reader, err := ipc.NewReader(bytes.NewReader(entry.Data))
if err != nil {
return fmt.Errorf("create ipc reader: %w", err)
}
record, err := reader.Read()
if err != nil {
return fmt.Errorf("read record: %w", err)
}
if err := table.active.InsertRecord(ctx, tx, record); err != nil {
return fmt.Errorf("insert record into block: %w", err)
}
default:
panic("parquet writes are deprecated")
}
return nil
})
case *walpb.Entry_TableBlockPersisted_:
// If a block was persisted but the entry still exists in the WAL,
// a snapshot was not performed after persisting the block. Perform
// one now to clean up the WAL.
performSnapshot = true
return nil
case *walpb.Entry_Snapshot_:
return nil
default:
return fmt.Errorf("unexpected WAL entry type: %t", e)
}
return nil
}); err != nil {
return err
}
if err := writeWg.Wait(); err != nil {
return fmt.Errorf("write error during replay: %w", err)
}
resetTxn := snapshotTx
if lastTx > resetTxn {
resetTxn = lastTx
}
db.mtx.Lock()
for _, table := range db.tables {
block := table.ActiveBlock()
block.uncompressedInsertsSize.Store(block.Index().LevelSize(index.L0))
}
db.mtx.Unlock()
db.resetToTxn(resetTxn, nil)
if performSnapshot && db.columnStore.snapshotTriggerSize != 0 {
level.Info(db.logger).Log(
"msg", "performing snapshot after recovery",
)
db.snapshot(ctx, false, func() {
if err := db.reclaimDiskSpace(ctx, wal); err != nil {
level.Error(db.logger).Log(
"msg", "failed to reclaim disk space after snapshot during recovery",
"err", err,
)
}
})
}
level.Info(db.logger).Log(
append(
[]any{
"msg", "db recovered",
"wal_replay_duration", time.Since(start),
"watermark", resetTxn,
},
snapshotLogArgs...,
)...,
)
return nil
}
type CloseOption func(*closeOptions)
type closeOptions struct {
clearStorage bool
}
func WithClearStorage() CloseOption {
return func(o *closeOptions) {
o.clearStorage = true
}
}
func (db *DB) Close(options ...CloseOption) error {
opts := &closeOptions{}
for _, opt := range options {
opt(opts)
}
level.Info(db.logger).Log("msg", "closing DB")
shouldPersist := len(db.sinks) > 0 && !db.columnStore.manualBlockRotation
for _, table := range db.tables {
table.close()
if shouldPersist {
// Write the blocks but no snapshots since they are long-running
// jobs.
// TODO(asubiotto): Maybe we should snapshot in any case since it
// should be faster to write to local disk than upload to object
// storage. This would avoid a slow WAL replay on startup if we
// don't manage to persist in time.
table.writeBlock(table.ActiveBlock(), false, false)
}
}
level.Info(db.logger).Log("msg", "closed all tables")
if !shouldPersist && db.columnStore.snapshotTriggerSize != 0 && !opts.clearStorage {
start := time.Now()
db.snapshot(context.Background(), false, func() {
level.Info(db.logger).Log("msg", "snapshot on close completed", "duration", time.Since(start))
if err := db.reclaimDiskSpace(context.Background(), nil); err != nil {
level.Error(db.logger).Log(
"msg", "failed to reclaim disk space after snapshot",