forked from influxdata/influxdb
/
engine.go
1515 lines (1261 loc) · 45.5 KB
/
engine.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 tsm1 provides a TSDB in the Time Structured Merge tree format.
package tsm1 // import "github.com/influxdata/influxdb/tsdb/tsm1"
import (
"bytes"
"context"
"fmt"
"io/ioutil"
"os"
"path/filepath"
"regexp"
"runtime"
"strings"
"sync"
"sync/atomic"
"time"
"github.com/influxdata/influxdb"
"github.com/influxdata/influxdb/kit/tracing"
"github.com/influxdata/influxdb/logger"
"github.com/influxdata/influxdb/models"
"github.com/influxdata/influxdb/pkg/lifecycle"
"github.com/influxdata/influxdb/pkg/limiter"
"github.com/influxdata/influxdb/pkg/metrics"
"github.com/influxdata/influxdb/query"
"github.com/influxdata/influxdb/tsdb"
"github.com/influxdata/influxdb/tsdb/tsi1"
"github.com/influxdata/influxql"
"github.com/prometheus/client_golang/prometheus"
"go.uber.org/zap"
)
//go:generate env GO111MODULE=on go run github.com/benbjohnson/tmpl -data=@array_cursor.gen.go.tmpldata array_cursor.gen.go.tmpl array_cursor_iterator.gen.go.tmpl
//go:generate env GO111MODULE=on go run github.com/influxdata/influxdb/tools/tmpl -i -data=file_store.gen.go.tmpldata file_store.gen.go.tmpl=file_store.gen.go
//go:generate env GO111MODULE=on go run github.com/influxdata/influxdb/tools/tmpl -i -d isArray=y -data=file_store.gen.go.tmpldata file_store.gen.go.tmpl=file_store_array.gen.go
//go:generate env GO111MODULE=on go run github.com/benbjohnson/tmpl -data=@encoding.gen.go.tmpldata encoding.gen.go.tmpl
//go:generate env GO111MODULE=on go run github.com/benbjohnson/tmpl -data=@compact.gen.go.tmpldata compact.gen.go.tmpl
//go:generate env GO111MODULE=on go run github.com/benbjohnson/tmpl -data=@reader.gen.go.tmpldata reader.gen.go.tmpl
//go:generate stringer -type=CacheStatus
var (
// Static objects to prevent small allocs.
KeyFieldSeparatorBytes = []byte(keyFieldSeparator)
)
var (
tsmGroup = metrics.MustRegisterGroup("platform-tsm1")
numberOfRefCursorsCounter = metrics.MustRegisterCounter("cursors_ref", metrics.WithGroup(tsmGroup))
)
// NewContextWithMetricsGroup creates a new context with a tsm1 metrics.Group for tracking
// various metrics when accessing TSM data.
func NewContextWithMetricsGroup(ctx context.Context) context.Context {
group := metrics.NewGroup(tsmGroup)
return metrics.NewContextWithGroup(ctx, group)
}
// MetricsGroupFromContext returns the tsm1 metrics.Group associated with the context
// or nil if no group has been assigned.
func MetricsGroupFromContext(ctx context.Context) *metrics.Group {
return metrics.GroupFromContext(ctx)
}
const (
// keyFieldSeparator separates the series key from the field name in the composite key
// that identifies a specific field in series
keyFieldSeparator = "#!~#"
// MaxPointsPerBlock is the maximum number of points in an encoded block in a TSM file
MaxPointsPerBlock = 1000
)
// An EngineOption is a functional option for changing the configuration of
// an Engine.
type EngineOption func(i *Engine)
// WithCompactionPlanner sets the compaction planner for the engine.
func WithCompactionPlanner(planner CompactionPlanner) EngineOption {
return func(e *Engine) {
planner.SetFileStore(e.FileStore)
e.CompactionPlan = planner
}
}
// Snapshotter allows upward signaling of the tsm1 engine to the storage engine. Hopefully
// it can be removed one day. The weird interface is due to the weird inversion of locking
// that has to happen.
type Snapshotter interface {
AcquireSegments(context.Context, func(segments []string) error) error
CommitSegments(ctx context.Context, segments []string, fn func() error) error
}
type noSnapshotter struct{}
func (noSnapshotter) AcquireSegments(_ context.Context, fn func([]string) error) error { return fn(nil) }
func (noSnapshotter) CommitSegments(_ context.Context, _ []string, fn func() error) error { return fn() }
// WithSnapshotter sets the callbacks for the engine to use when creating snapshots.
func WithSnapshotter(snapshotter Snapshotter) EngineOption {
return func(e *Engine) {
e.snapshotter = snapshotter
}
}
// Engine represents a storage engine with compressed blocks.
type Engine struct {
mu sync.RWMutex
index *tsi1.Index
indexref *lifecycle.Reference
// The following group of fields is used to track the state of level compactions within the
// Engine. The WaitGroup is used to monitor the compaction goroutines, the 'done' channel is
// used to signal those goroutines to shutdown. Every request to disable level compactions will
// call 'Wait' on 'wg', with the first goroutine to arrive (levelWorkers == 0 while holding the
// lock) will close the done channel and re-assign 'nil' to the variable. Re-enabling will
// decrease 'levelWorkers', and when it decreases to zero, level compactions will be started
// back up again.
wg *sync.WaitGroup // waitgroup for active level compaction goroutines
done chan struct{} // channel to signal level compactions to stop
levelWorkers int // Number of "workers" that expect compactions to be in a disabled state
snapDone chan struct{} // channel to signal snapshot compactions to stop
snapWG *sync.WaitGroup // waitgroup for running snapshot compactions
path string
sfile *tsdb.SeriesFile
sfileref *lifecycle.Reference
logger *zap.Logger // Logger to be used for important messages
Cache *Cache
Compactor *Compactor
CompactionPlan CompactionPlanner
FileStore *FileStore
MaxPointsPerBlock int
// CacheFlushMemorySizeThreshold specifies the minimum size threshold for
// the cache when the engine should write a snapshot to a TSM file
CacheFlushMemorySizeThreshold uint64
// CacheFlushAgeDurationThreshold specified the maximum age a cache can reach
// before it is snapshotted, regardless of its size.
CacheFlushAgeDurationThreshold time.Duration
// CacheFlushWriteColdDuration specifies the length of time after which if
// no writes have been committed to the WAL, the engine will write
// a snapshot of the cache to a TSM file
CacheFlushWriteColdDuration time.Duration
// Invoked when creating a backup file "as new".
formatFileName FormatFileNameFunc
// Controls whether to enabled compactions when the engine is open
enableCompactionsOnOpen bool
compactionTracker *compactionTracker // Used to track state of compactions.
readTracker *readTracker // Used to track number of reads.
defaultMetricLabels prometheus.Labels // N.B this must not be mutated after Open is called.
// Limiter for concurrent compactions.
compactionLimiter limiter.Fixed
// A semaphore for limiting full compactions across multiple engines.
fullCompactionSemaphore influxdb.Semaphore
// Tracks how long the last full compaction took. Should be accessed atomically.
lastFullCompactionDuration int64
scheduler *scheduler
snapshotter Snapshotter
}
// NewEngine returns a new instance of Engine.
func NewEngine(path string, idx *tsi1.Index, config Config, options ...EngineOption) *Engine {
fs := NewFileStore(path)
fs.openLimiter = limiter.NewFixed(config.MaxConcurrentOpens)
fs.tsmMMAPWillNeed = config.MADVWillNeed
cache := NewCache(uint64(config.Cache.MaxMemorySize))
c := NewCompactor()
c.Dir = path
c.FileStore = fs
c.RateLimit = limiter.NewRate(
int(config.Compaction.Throughput),
int(config.Compaction.ThroughputBurst))
// determine max concurrent compactions informed by the system
maxCompactions := config.Compaction.MaxConcurrent
if maxCompactions == 0 {
maxCompactions = runtime.GOMAXPROCS(0) / 2 // Default to 50% of cores for compactions
// On systems with more cores, cap at 4 to reduce disk utilization.
if maxCompactions > 4 {
maxCompactions = 4
}
if maxCompactions < 1 {
maxCompactions = 1
}
}
// Don't allow more compactions to run than cores.
if maxCompactions > runtime.GOMAXPROCS(0) {
maxCompactions = runtime.GOMAXPROCS(0)
}
logger := zap.NewNop()
e := &Engine{
path: path,
index: idx,
sfile: idx.SeriesFile(),
logger: logger,
Cache: cache,
FileStore: fs,
Compactor: c,
CompactionPlan: NewDefaultPlanner(fs,
time.Duration(config.Compaction.FullWriteColdDuration)),
CacheFlushMemorySizeThreshold: uint64(config.Cache.SnapshotMemorySize),
CacheFlushWriteColdDuration: time.Duration(config.Cache.SnapshotWriteColdDuration),
CacheFlushAgeDurationThreshold: time.Duration(config.Cache.SnapshotAgeDuration),
enableCompactionsOnOpen: true,
formatFileName: DefaultFormatFileName,
compactionLimiter: limiter.NewFixed(maxCompactions),
fullCompactionSemaphore: influxdb.NopSemaphore,
scheduler: newScheduler(maxCompactions),
snapshotter: new(noSnapshotter),
}
for _, option := range options {
option(e)
}
return e
}
// SetSemaphore sets the semaphore used to coordinate full compactions across
// multiple engines.
func (e *Engine) SetSemaphore(s influxdb.Semaphore) {
e.fullCompactionSemaphore = s
}
// WithCompactionLimiter sets the compaction limiter, which is used to limit the
// number of concurrent compactions.
func (e *Engine) WithCompactionLimiter(limiter limiter.Fixed) {
e.compactionLimiter = limiter
}
func (e *Engine) WithFormatFileNameFunc(formatFileNameFunc FormatFileNameFunc) {
e.Compactor.WithFormatFileNameFunc(formatFileNameFunc)
e.formatFileName = formatFileNameFunc
}
func (e *Engine) WithParseFileNameFunc(parseFileNameFunc ParseFileNameFunc) {
e.FileStore.WithParseFileNameFunc(parseFileNameFunc)
e.Compactor.WithParseFileNameFunc(parseFileNameFunc)
}
func (e *Engine) WithCurrentGenerationFunc(fn func() int) {
e.Compactor.FileStore.SetCurrentGenerationFunc(fn)
}
func (e *Engine) WithFileStoreObserver(obs FileStoreObserver) {
e.FileStore.WithObserver(obs)
}
func (e *Engine) WithCompactionPlanner(planner CompactionPlanner) {
planner.SetFileStore(e.FileStore)
e.CompactionPlan = planner
}
// SetDefaultMetricLabels sets the default labels for metrics on the engine.
// It must be called before the Engine is opened.
func (e *Engine) SetDefaultMetricLabels(labels prometheus.Labels) {
e.defaultMetricLabels = labels
}
// SetEnabled sets whether the engine is enabled.
func (e *Engine) SetEnabled(enabled bool) {
e.enableCompactionsOnOpen = enabled
e.SetCompactionsEnabled(enabled)
}
// SetCompactionsEnabled enables compactions on the engine. When disabled
// all running compactions are aborted and new compactions stop running.
func (e *Engine) SetCompactionsEnabled(enabled bool) {
if enabled {
e.enableSnapshotCompactions()
e.enableLevelCompactions(false)
} else {
e.disableSnapshotCompactions()
e.disableLevelCompactions(false)
}
}
// enableLevelCompactions will request that level compactions start back up again
//
// 'wait' signifies that a corresponding call to disableLevelCompactions(true) was made at some
// point, and the associated task that required disabled compactions is now complete
func (e *Engine) enableLevelCompactions(wait bool) {
// If we don't need to wait, see if we're already enabled
if !wait {
e.mu.RLock()
if e.done != nil {
e.mu.RUnlock()
return
}
e.mu.RUnlock()
}
e.mu.Lock()
if wait {
e.levelWorkers -= 1
}
if e.levelWorkers != 0 || e.done != nil {
// still waiting on more workers or already enabled
e.mu.Unlock()
return
}
// last one to enable, start things back up
e.Compactor.EnableCompactions()
e.done = make(chan struct{})
wg := new(sync.WaitGroup)
wg.Add(1)
e.wg = wg
e.mu.Unlock()
go func() { defer wg.Done(); e.compact(wg) }()
}
// disableLevelCompactions will stop level compactions before returning.
//
// If 'wait' is set to true, then a corresponding call to enableLevelCompactions(true) will be
// required before level compactions will start back up again.
func (e *Engine) disableLevelCompactions(wait bool) {
e.mu.Lock()
old := e.levelWorkers
if wait {
e.levelWorkers += 1
}
// Hold onto the current done channel so we can wait on it if necessary
waitCh := e.done
wg := e.wg
if old == 0 && e.done != nil {
// It's possible we have closed the done channel and released the lock and another
// goroutine has attempted to disable compactions. We're current in the process of
// disabling them so check for this and wait until the original completes.
select {
case <-e.done:
e.mu.Unlock()
return
default:
}
// Prevent new compactions from starting
e.Compactor.DisableCompactions()
// Stop all background compaction goroutines
close(e.done)
e.mu.Unlock()
wg.Wait()
// Signal that all goroutines have exited.
e.mu.Lock()
e.done = nil
e.mu.Unlock()
return
}
e.mu.Unlock()
// Compaction were already disabled.
if waitCh == nil {
return
}
// We were not the first caller to disable compactions and they were in the process
// of being disabled. Wait for them to complete before returning.
<-waitCh
wg.Wait()
}
func (e *Engine) enableSnapshotCompactions() {
// Check if already enabled under read lock
e.mu.RLock()
if e.snapDone != nil {
e.mu.RUnlock()
return
}
e.mu.RUnlock()
// Check again under write lock
e.mu.Lock()
if e.snapDone != nil {
e.mu.Unlock()
return
}
e.Compactor.EnableSnapshots()
e.snapDone = make(chan struct{})
wg := new(sync.WaitGroup)
wg.Add(1)
e.snapWG = wg
e.mu.Unlock()
go func() { defer wg.Done(); e.compactCache() }()
}
func (e *Engine) disableSnapshotCompactions() {
e.mu.Lock()
if e.snapDone == nil {
e.mu.Unlock()
return
}
// We may be in the process of stopping snapshots. See if the channel
// was closed.
select {
case <-e.snapDone:
e.mu.Unlock()
return
default:
}
// first one here, disable and wait for completion
close(e.snapDone)
e.Compactor.DisableSnapshots()
wg := e.snapWG
e.mu.Unlock()
// Wait for the snapshot goroutine to exit.
wg.Wait()
// Signal that the goroutines are exit and everything is stopped by setting
// snapDone to nil.
e.mu.Lock()
e.snapDone = nil
e.mu.Unlock()
}
// ScheduleFullCompaction will force the engine to fully compact all data stored.
// This will cancel and running compactions and snapshot any data in the cache to
// TSM files. This is an expensive operation.
func (e *Engine) ScheduleFullCompaction(ctx context.Context) error {
// Snapshot any data in the cache
if err := e.WriteSnapshot(ctx, CacheStatusFullCompaction); err != nil {
return err
}
// Cancel running compactions
e.SetCompactionsEnabled(false)
// Ensure compactions are restarted
defer e.SetCompactionsEnabled(true)
// Force the planner to only create a full plan.
e.CompactionPlan.ForceFull()
return nil
}
// Path returns the path the engine was opened with.
func (e *Engine) Path() string { return e.path }
func (e *Engine) SetFieldName(measurement []byte, name string) {
e.index.SetFieldName(measurement, name)
}
func (e *Engine) MeasurementExists(name []byte) (bool, error) {
return e.index.MeasurementExists(name)
}
func (e *Engine) MeasurementNamesByRegex(re *regexp.Regexp) ([][]byte, error) {
return e.index.MeasurementNamesByRegex(re)
}
func (e *Engine) HasTagKey(name, key []byte) (bool, error) {
return e.index.HasTagKey(name, key)
}
func (e *Engine) MeasurementTagKeysByExpr(name []byte, expr influxql.Expr) (map[string]struct{}, error) {
return e.index.MeasurementTagKeysByExpr(name, expr)
}
func (e *Engine) TagKeyCardinality(name, key []byte) int {
return e.index.TagKeyCardinality(name, key)
}
// SeriesN returns the unique number of series in the index.
func (e *Engine) SeriesN() int64 {
return e.index.SeriesN()
}
// MeasurementStats returns the current measurement stats for the engine.
func (e *Engine) MeasurementStats() (MeasurementStats, error) {
return e.FileStore.MeasurementStats()
}
func (e *Engine) initTrackers() {
mmu.Lock()
defer mmu.Unlock()
if bms == nil {
// Initialise metrics if an engine has not done so already.
bms = newBlockMetrics(e.defaultMetricLabels)
}
// Propagate prometheus metrics down into trackers.
e.compactionTracker = newCompactionTracker(bms.compactionMetrics, e.defaultMetricLabels)
e.FileStore.tracker = newFileTracker(bms.fileMetrics, e.defaultMetricLabels)
e.Cache.tracker = newCacheTracker(bms.cacheMetrics, e.defaultMetricLabels)
e.readTracker = newReadTracker(bms.readMetrics, e.defaultMetricLabels)
e.scheduler.setCompactionTracker(e.compactionTracker)
}
// Open opens and initializes the engine.
func (e *Engine) Open(ctx context.Context) (err error) {
span, ctx := tracing.StartSpanFromContext(ctx)
defer span.Finish()
defer func() {
if err != nil {
e.Close()
}
}()
e.indexref, err = e.index.Acquire()
if err != nil {
return err
}
e.sfileref, err = e.sfile.Acquire()
if err != nil {
return err
}
e.initTrackers()
if err := os.MkdirAll(e.path, 0777); err != nil {
return err
}
if err := e.cleanup(); err != nil {
return err
}
if err := e.FileStore.Open(ctx); err != nil {
return err
}
e.Compactor.Open()
if e.enableCompactionsOnOpen {
e.SetCompactionsEnabled(true)
}
return nil
}
// Close closes the engine. Subsequent calls to Close are a nop.
func (e *Engine) Close() error {
e.SetCompactionsEnabled(false)
// Lock now and close everything else down.
e.mu.Lock()
defer e.mu.Unlock()
// Ensures that the channel will not be closed again.
e.done = nil
if err := e.FileStore.Close(); err != nil {
return err
}
// Release our references.
if e.sfileref != nil {
e.sfileref.Release()
e.sfileref = nil
}
if e.indexref != nil {
e.indexref.Release()
e.indexref = nil
}
return nil
}
// WithLogger sets the logger for the engine.
func (e *Engine) WithLogger(log *zap.Logger) {
e.logger = log.With(zap.String("engine", "tsm1"))
e.FileStore.WithLogger(e.logger)
}
// IsIdle returns true if the cache is empty, there are no running compactions and the
// shard is fully compacted.
func (e *Engine) IsIdle() bool {
cacheEmpty := e.Cache.Size() == 0
return cacheEmpty && e.compactionTracker.AllActive() == 0 && e.CompactionPlan.FullyCompacted()
}
// WritePoints saves the set of points in the engine.
func (e *Engine) WritePoints(points []models.Point) error {
collection := tsdb.NewSeriesCollection(points)
values, err := CollectionToValues(collection)
if err != nil {
return err
}
if err := e.WriteValues(values); err != nil {
return err
}
return collection.PartialWriteError()
}
// WriteValues saves the set of values in the engine.
func (e *Engine) WriteValues(values map[string][]Value) error {
e.mu.RLock()
defer e.mu.RUnlock()
if err := e.Cache.WriteMulti(values); err != nil {
return err
}
return nil
}
// ForEachMeasurementName iterates over each measurement name in the engine.
func (e *Engine) ForEachMeasurementName(fn func(name []byte) error) error {
return e.index.ForEachMeasurementName(fn)
}
// compactionLevel describes a snapshot or levelled compaction.
type compactionLevel int
func (l compactionLevel) String() string {
switch l {
case 0:
return "snapshot"
case 1, 2, 3:
return fmt.Sprint(int(l))
case 4:
return "optimize"
case 5:
return "full"
default:
panic("unsupported compaction level")
}
}
// compactionTracker tracks compactions and snapshots within the Engine.
//
// As well as being responsible for providing atomic reads and writes to the
// statistics tracking the various compaction operations, compactionTracker also
// mirrors any writes to the prometheus block metrics, which the Engine exposes.
//
// *NOTE* - compactionTracker fields should not be directory modified. Doing so
// could result in the Engine exposing inaccurate metrics.
type compactionTracker struct {
metrics *compactionMetrics
labels prometheus.Labels
// Note: Compactions are levelled as follows:
// 0 – Snapshots
// 1-3 – Levelled compactions
// 4 – Optimize compactions
// 5 – Full compactions
ok [6]uint64 // Counter of TSM compactions (by level) that have successfully completed.
active [6]uint64 // Gauge of TSM compactions (by level) currently running.
errors [6]uint64 // Counter of TSM compcations (by level) that have failed due to error.
queue [6]uint64 // Gauge of TSM compactions queues (by level).
}
func newCompactionTracker(metrics *compactionMetrics, defaultLables prometheus.Labels) *compactionTracker {
return &compactionTracker{metrics: metrics, labels: defaultLables}
}
// Labels returns a copy of the default labels used by the tracker's metrics.
// The returned map is safe for modification.
func (t *compactionTracker) Labels(level compactionLevel) prometheus.Labels {
labels := make(prometheus.Labels, len(t.labels))
for k, v := range t.labels {
labels[k] = v
}
// All metrics have a level label.
labels["level"] = fmt.Sprint(level)
return labels
}
// Completed returns the total number of compactions for the provided level.
func (t *compactionTracker) Completed(level int) uint64 { return atomic.LoadUint64(&t.ok[level]) }
// Active returns the number of active snapshots (level 0),
// level 1, 2 or 3 compactions, optimize compactions (level 4), or full
// compactions (level 5).
func (t *compactionTracker) Active(level int) uint64 {
return atomic.LoadUint64(&t.active[level])
}
// AllActive returns the number of active snapshots and compactions.
func (t *compactionTracker) AllActive() uint64 {
var total uint64
for i := 0; i < len(t.active); i++ {
total += atomic.LoadUint64(&t.active[i])
}
return total
}
// ActiveOptimise returns the number of active Optimise compactions.
//
// ActiveOptimise is a helper for Active(4).
func (t *compactionTracker) ActiveOptimise() uint64 { return t.Active(4) }
// ActiveFull returns the number of active Full compactions.
//
// ActiveFull is a helper for Active(5).
func (t *compactionTracker) ActiveFull() uint64 { return t.Active(5) }
// Errors returns the total number of errors encountered attempting compactions
// for the provided level.
func (t *compactionTracker) Errors(level int) uint64 { return atomic.LoadUint64(&t.errors[level]) }
// IncActive increments the number of active compactions for the provided level.
func (t *compactionTracker) IncActive(level compactionLevel) {
atomic.AddUint64(&t.active[level], 1)
labels := t.Labels(level)
t.metrics.CompactionsActive.With(labels).Inc()
}
// IncFullActive increments the number of active Full compactions.
func (t *compactionTracker) IncFullActive() { t.IncActive(5) }
// DecActive decrements the number of active compactions for the provided level.
func (t *compactionTracker) DecActive(level compactionLevel) {
atomic.AddUint64(&t.active[level], ^uint64(0))
labels := t.Labels(level)
t.metrics.CompactionsActive.With(labels).Dec()
}
// DecFullActive decrements the number of active Full compactions.
func (t *compactionTracker) DecFullActive() { t.DecActive(5) }
// Attempted updates the number of compactions attempted for the provided level.
func (t *compactionTracker) Attempted(level compactionLevel, success bool, reason string, duration time.Duration) {
if success {
atomic.AddUint64(&t.ok[level], 1)
labels := t.Labels(level)
t.metrics.CompactionDuration.With(labels).Observe(duration.Seconds())
// Total compactions metric has reason and status.
labels["reason"] = reason
labels["status"] = "ok"
t.metrics.Compactions.With(labels).Inc()
return
}
atomic.AddUint64(&t.errors[level], 1)
labels := t.Labels(level)
labels["status"] = "error"
labels["reason"] = reason
t.metrics.Compactions.With(labels).Inc()
}
// SnapshotAttempted updates the number of snapshots attempted.
func (t *compactionTracker) SnapshotAttempted(success bool, reason CacheStatus, duration time.Duration) {
t.Attempted(0, success, reason.String(), duration)
}
// SetQueue sets the compaction queue depth for the provided level.
func (t *compactionTracker) SetQueue(level compactionLevel, length uint64) {
atomic.StoreUint64(&t.queue[level], length)
labels := t.Labels(level)
t.metrics.CompactionQueue.With(labels).Set(float64(length))
}
// SetOptimiseQueue sets the queue depth for Optimisation compactions.
func (t *compactionTracker) SetOptimiseQueue(length uint64) { t.SetQueue(4, length) }
// SetFullQueue sets the queue depth for Full compactions.
func (t *compactionTracker) SetFullQueue(length uint64) { t.SetQueue(5, length) }
func (e *Engine) WriteSnapshot(ctx context.Context, status CacheStatus) error {
start := time.Now()
err := e.writeSnapshot(ctx)
if err != nil && err != errCompactionsDisabled {
e.logger.Info("Error writing snapshot", zap.Error(err))
}
e.compactionTracker.SnapshotAttempted(
err == nil || err == errCompactionsDisabled || err == ErrSnapshotInProgress,
status, time.Since(start))
if err != nil {
return err
}
return nil
}
// WriteSnapshot will snapshot the cache and write a new TSM file with its contents, releasing the snapshot when done.
func (e *Engine) writeSnapshot(ctx context.Context) error {
span, ctx := tracing.StartSpanFromContext(ctx)
defer span.Finish()
// Lock and grab the cache snapshot along with all the closed WAL
// filenames associated with the snapshot
started := time.Now()
log, logEnd := logger.NewOperation(ctx, e.logger, "Cache snapshot", "tsm1_cache_snapshot")
defer func() {
elapsed := time.Since(started)
log.Info("Snapshot for path written",
zap.String("path", e.path),
zap.Duration("duration", elapsed))
logEnd()
}()
var (
snapshot *Cache
segments []string
)
if err := e.snapshotter.AcquireSegments(ctx, func(segs []string) (err error) {
segments = segs
e.mu.Lock()
snapshot, err = e.Cache.Snapshot()
e.mu.Unlock()
return err
}); err != nil {
return err
}
if snapshot.Size() == 0 {
e.Cache.ClearSnapshot(true)
return nil
}
// The snapshotted cache may have duplicate points and unsorted data. We need to deduplicate
// it before writing the snapshot. This can be very expensive so it's done while we are not
// holding the engine write lock.
snapshot.Deduplicate()
return e.writeSnapshotAndCommit(ctx, log, snapshot, segments)
}
// writeSnapshotAndCommit will write the passed cache to a new TSM file and remove the closed WAL segments.
func (e *Engine) writeSnapshotAndCommit(ctx context.Context, log *zap.Logger, snapshot *Cache, segments []string) (err error) {
defer func() {
if err != nil {
e.Cache.ClearSnapshot(false)
}
}()
// write the new snapshot files
newFiles, err := e.Compactor.WriteSnapshot(ctx, snapshot)
if err != nil {
log.Info("Error writing snapshot from compactor", zap.Error(err))
return err
}
return e.snapshotter.CommitSegments(ctx, segments, func() error {
e.mu.RLock()
defer e.mu.RUnlock()
// update the file store with these new files
if err := e.FileStore.Replace(nil, newFiles); err != nil {
log.Info("Error adding new TSM files from snapshot", zap.Error(err))
return err
}
// clear the snapshot from the in-memory cache
e.Cache.ClearSnapshot(true)
return nil
})
}
// compactCache checks once per second if the in-memory cache should be
// snapshotted to a TSM file.
func (e *Engine) compactCache() {
t := time.NewTicker(time.Second)
defer t.Stop()
for {
e.mu.RLock()
quit := e.snapDone
e.mu.RUnlock()
select {
case <-quit:
return
case <-t.C:
e.Cache.UpdateAge()
status := e.ShouldCompactCache(time.Now())
if status == CacheStatusOkay {
continue
}
span, ctx := tracing.StartSpanFromContextWithOperationName(context.Background(), "compact cache")
span.LogKV("path", e.path)
err := e.WriteSnapshot(ctx, status)
if err != nil && err != errCompactionsDisabled && err != ErrSnapshotInProgress {
e.logger.Info("Error writing snapshot", zap.Error(err))
}
span.Finish()
}
}
}
// CacheStatus describes the current state of the cache, with respect to whether
// it is ready to be snapshotted or not.
type CacheStatus int
// Possible types of Cache status
const (
CacheStatusOkay CacheStatus = iota // Cache is Okay - do not snapshot.
CacheStatusSizeExceeded // The cache is large enough to be snapshotted.
CacheStatusAgeExceeded // The cache is past the age threshold to be snapshotted.
CacheStatusColdNoWrites // The cache has not been written to for long enough that it should be snapshotted.
CacheStatusRetention // The cache was snapshotted before running retention.
CacheStatusFullCompaction // The cache was snapshotted as part of a full compaction.
CacheStatusBackup // The cache was snapshotted before running backup.
)
// ShouldCompactCache returns a status indicating if the Cache should be
// snapshotted. There are three situations when the cache should be snapshotted:
//
// - the Cache size is over its flush size threshold;
// - the Cache has not been snapshotted for longer than its flush time threshold; or
// - the Cache has not been written since the write cold threshold.
//
func (e *Engine) ShouldCompactCache(t time.Time) CacheStatus {
sz := e.Cache.Size()
if sz == 0 {
return 0
}
// Cache is now big enough to snapshot.
if sz > e.CacheFlushMemorySizeThreshold {
return CacheStatusSizeExceeded
}
// Cache is now old enough to snapshot, regardless of last write or age.
if e.CacheFlushAgeDurationThreshold > 0 && e.Cache.Age() > e.CacheFlushAgeDurationThreshold {
return CacheStatusAgeExceeded
}
// Cache has not been written to for a long time.
if t.Sub(e.Cache.LastWriteTime()) > e.CacheFlushWriteColdDuration {
return CacheStatusColdNoWrites
}
return CacheStatusOkay
}
func (e *Engine) lastModified() time.Time {
fsTime := e.FileStore.LastModified()
cacheTime := e.Cache.LastWriteTime()
if cacheTime.After(fsTime) {
return cacheTime
}
return fsTime
}
func (e *Engine) compact(wg *sync.WaitGroup) {
t := time.NewTicker(time.Second)
defer t.Stop()
for {
e.mu.RLock()
quit := e.done
e.mu.RUnlock()
select {
case <-quit:
return
case <-t.C:
span, ctx := tracing.StartSpanFromContext(context.Background())
// Find our compaction plans
level1Groups := e.CompactionPlan.PlanLevel(1)
level2Groups := e.CompactionPlan.PlanLevel(2)
level3Groups := e.CompactionPlan.PlanLevel(3)