forked from influxdata/influxdb
/
engine.go
2306 lines (1954 loc) · 67.4 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/engine/tsm1"
import (
"archive/tar"
"bytes"
"fmt"
"io"
"io/ioutil"
"math"
"os"
"path/filepath"
"regexp"
"runtime"
"strings"
"sync"
"sync/atomic"
"time"
"github.com/influxdata/influxdb/query"
"github.com/influxdata/influxdb/tsdb/index/inmem"
"github.com/influxdata/influxdb/influxql"
"github.com/influxdata/influxdb/models"
"github.com/influxdata/influxdb/pkg/bytesutil"
"github.com/influxdata/influxdb/pkg/estimator"
"github.com/influxdata/influxdb/pkg/limiter"
"github.com/influxdata/influxdb/tsdb"
_ "github.com/influxdata/influxdb/tsdb/index"
"github.com/uber-go/zap"
)
//go:generate tmpl -data=@iterator.gen.go.tmpldata iterator.gen.go.tmpl
//go:generate tmpl -data=@file_store.gen.go.tmpldata file_store.gen.go.tmpl
//go:generate tmpl -data=@encoding.gen.go.tmpldata encoding.gen.go.tmpl
//go:generate tmpl -data=@compact.gen.go.tmpldata compact.gen.go.tmpl
func init() {
tsdb.RegisterEngine("tsm1", NewEngine)
}
var (
// Ensure Engine implements the interface.
_ tsdb.Engine = &Engine{}
// Static objects to prevent small allocs.
timeBytes = []byte("time")
keyFieldSeparatorBytes = []byte(keyFieldSeparator)
)
const (
// keyFieldSeparator separates the series key from the field name in the composite key
// that identifies a specific field in series
keyFieldSeparator = "#!~#"
)
// Statistics gathered by the engine.
const (
statCacheCompactions = "cacheCompactions"
statCacheCompactionsActive = "cacheCompactionsActive"
statCacheCompactionError = "cacheCompactionErr"
statCacheCompactionDuration = "cacheCompactionDuration"
statTSMLevel1Compactions = "tsmLevel1Compactions"
statTSMLevel1CompactionsActive = "tsmLevel1CompactionsActive"
statTSMLevel1CompactionError = "tsmLevel1CompactionErr"
statTSMLevel1CompactionDuration = "tsmLevel1CompactionDuration"
statTSMLevel2Compactions = "tsmLevel2Compactions"
statTSMLevel2CompactionsActive = "tsmLevel2CompactionsActive"
statTSMLevel2CompactionError = "tsmLevel2CompactionErr"
statTSMLevel2CompactionDuration = "tsmLevel2CompactionDuration"
statTSMLevel3Compactions = "tsmLevel3Compactions"
statTSMLevel3CompactionsActive = "tsmLevel3CompactionsActive"
statTSMLevel3CompactionError = "tsmLevel3CompactionErr"
statTSMLevel3CompactionDuration = "tsmLevel3CompactionDuration"
statTSMOptimizeCompactions = "tsmOptimizeCompactions"
statTSMOptimizeCompactionsActive = "tsmOptimizeCompactionsActive"
statTSMOptimizeCompactionError = "tsmOptimizeCompactionErr"
statTSMOptimizeCompactionDuration = "tsmOptimizeCompactionDuration"
statTSMFullCompactions = "tsmFullCompactions"
statTSMFullCompactionsActive = "tsmFullCompactionsActive"
statTSMFullCompactionError = "tsmFullCompactionErr"
statTSMFullCompactionDuration = "tsmFullCompactionDuration"
)
// Engine represents a storage engine with compressed blocks.
type Engine struct {
mu sync.RWMutex
// 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
id uint64
database string
path string
logger zap.Logger // Logger to be used for important messages
traceLogger zap.Logger // Logger to be used when trace-logging is on.
traceLogging bool
index tsdb.Index
fieldset *tsdb.MeasurementFieldSet
WAL *WAL
Cache *Cache
Compactor *Compactor
CompactionPlan CompactionPlanner
FileStore *FileStore
MaxPointsPerBlock int
// CacheFlushMemorySizeThreshold specifies the minimum size threshodl for
// the cache when the engine should write a snapshot to a TSM file
CacheFlushMemorySizeThreshold uint64
// 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
// Controls whether to enabled compactions when the engine is open
enableCompactionsOnOpen bool
stats *EngineStatistics
// The limiter for concurrent compactions
compactionLimiter limiter.Fixed
}
// NewEngine returns a new instance of Engine.
func NewEngine(id uint64, idx tsdb.Index, database, path string, walPath string, opt tsdb.EngineOptions) tsdb.Engine {
w := NewWAL(walPath)
w.syncDelay = time.Duration(opt.Config.WALFsyncDelay)
fs := NewFileStore(path)
cache := NewCache(uint64(opt.Config.CacheMaxMemorySize), path)
c := &Compactor{
Dir: path,
FileStore: fs,
}
logger := zap.New(zap.NullEncoder())
e := &Engine{
id: id,
database: database,
path: path,
index: idx,
logger: logger,
traceLogger: logger,
traceLogging: opt.Config.TraceLoggingEnabled,
fieldset: tsdb.NewMeasurementFieldSet(),
WAL: w,
Cache: cache,
FileStore: fs,
Compactor: c,
CompactionPlan: NewDefaultPlanner(fs, time.Duration(opt.Config.CompactFullWriteColdDuration)),
CacheFlushMemorySizeThreshold: opt.Config.CacheSnapshotMemorySize,
CacheFlushWriteColdDuration: time.Duration(opt.Config.CacheSnapshotWriteColdDuration),
enableCompactionsOnOpen: true,
stats: &EngineStatistics{},
compactionLimiter: opt.CompactionLimiter,
}
// Attach fieldset to index.
e.index.SetFieldSet(e.fieldset)
if e.traceLogging {
fs.enableTraceLogging(true)
w.enableTraceLogging(true)
}
return e
}
// 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()
quit := make(chan struct{})
e.done = quit
e.wg.Add(4)
e.mu.Unlock()
go func() { defer e.wg.Done(); e.compactTSMFull(quit) }()
go func() { defer e.wg.Done(); e.compactTSMLevel(true, 1, quit) }()
go func() { defer e.wg.Done(); e.compactTSMLevel(true, 2, quit) }()
go func() { defer e.wg.Done(); e.compactTSMLevel(false, 3, quit) }()
}
// 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
}
if old == 0 && e.done != nil {
// Prevent new compactions from starting
e.Compactor.DisableCompactions()
// Stop all background compaction goroutines
close(e.done)
e.done = nil
}
e.mu.Unlock()
e.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()
quit := make(chan struct{})
e.snapDone = quit
e.snapWG.Add(1)
e.mu.Unlock()
go func() { defer e.snapWG.Done(); e.compactCache(quit) }()
}
func (e *Engine) disableSnapshotCompactions() {
e.mu.Lock()
if e.snapDone != nil {
close(e.snapDone)
e.snapDone = nil
e.Compactor.DisableSnapshots()
}
e.mu.Unlock()
e.snapWG.Wait()
}
// 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) MeasurementNamesByExpr(expr influxql.Expr) ([][]byte, error) {
return e.index.MeasurementNamesByExpr(expr)
}
func (e *Engine) MeasurementNamesByRegex(re *regexp.Regexp) ([][]byte, error) {
return e.index.MeasurementNamesByRegex(re)
}
// MeasurementFields returns the measurement fields for a measurement.
func (e *Engine) MeasurementFields(measurement []byte) *tsdb.MeasurementFields {
return e.fieldset.CreateFieldsIfNotExists(measurement)
}
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)
}
// MeasurementTagKeyValuesByExpr returns a set of tag values filtered by an expression.
//
// MeasurementTagKeyValuesByExpr relies on the provided tag keys being sorted.
// The caller can indicate the tag keys have been sorted by setting the
// keysSorted argument appropriately. Tag values are returned in a slice that
// is indexible according to the sorted order of the tag keys, e.g., the values
// for the earliest tag k will be available in index 0 of the returned values
// slice.
//
func (e *Engine) MeasurementTagKeyValuesByExpr(name []byte, keys []string, expr influxql.Expr, keysSorted bool) ([][]string, error) {
return e.index.MeasurementTagKeyValuesByExpr(name, keys, expr, keysSorted)
}
func (e *Engine) ForEachMeasurementTagKey(name []byte, fn func(key []byte) error) error {
return e.index.ForEachMeasurementTagKey(name, fn)
}
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()
}
func (e *Engine) SeriesSketches() (estimator.Sketch, estimator.Sketch, error) {
return e.index.SeriesSketches()
}
func (e *Engine) MeasurementsSketches() (estimator.Sketch, estimator.Sketch, error) {
return e.index.MeasurementsSketches()
}
// LastModified returns the time when this shard was last modified.
func (e *Engine) LastModified() time.Time {
walTime := e.WAL.LastWriteTime()
fsTime := e.FileStore.LastModified()
if walTime.After(fsTime) {
return walTime
}
return fsTime
}
// EngineStatistics maintains statistics for the engine.
type EngineStatistics struct {
CacheCompactions int64 // Counter of cache compactions that have ever run.
CacheCompactionsActive int64 // Gauge of cache compactions currently running.
CacheCompactionErrors int64 // Counter of cache compactions that have failed due to error.
CacheCompactionDuration int64 // Counter of number of wall nanoseconds spent in cache compactions.
TSMCompactions [3]int64 // Counter of TSM compactions (by level) that have ever run.
TSMCompactionsActive [3]int64 // Gauge of TSM compactions (by level) currently running.
TSMCompactionErrors [3]int64 // Counter of TSM compcations (by level) that have failed due to error.
TSMCompactionDuration [3]int64 // Counter of number of wall nanoseconds spent in TSM compactions (by level).
TSMOptimizeCompactions int64 // Counter of optimize compactions that have ever run.
TSMOptimizeCompactionsActive int64 // Gauge of optimize compactions currently running.
TSMOptimizeCompactionErrors int64 // Counter of optimize compactions that have failed due to error.
TSMOptimizeCompactionDuration int64 // Counter of number of wall nanoseconds spent in optimize compactions.
TSMFullCompactions int64 // Counter of full compactions that have ever run.
TSMFullCompactionsActive int64 // Gauge of full compactions currently running.
TSMFullCompactionErrors int64 // Counter of full compactions that have failed due to error.
TSMFullCompactionDuration int64 // Counter of number of wall nanoseconds spent in full compactions.
}
// Statistics returns statistics for periodic monitoring.
func (e *Engine) Statistics(tags map[string]string) []models.Statistic {
statistics := make([]models.Statistic, 0, 4)
statistics = append(statistics, models.Statistic{
Name: "tsm1_engine",
Tags: tags,
Values: map[string]interface{}{
statCacheCompactions: atomic.LoadInt64(&e.stats.CacheCompactions),
statCacheCompactionsActive: atomic.LoadInt64(&e.stats.CacheCompactionsActive),
statCacheCompactionError: atomic.LoadInt64(&e.stats.CacheCompactionErrors),
statCacheCompactionDuration: atomic.LoadInt64(&e.stats.CacheCompactionDuration),
statTSMLevel1Compactions: atomic.LoadInt64(&e.stats.TSMCompactions[0]),
statTSMLevel1CompactionsActive: atomic.LoadInt64(&e.stats.TSMCompactionsActive[0]),
statTSMLevel1CompactionError: atomic.LoadInt64(&e.stats.TSMCompactionErrors[0]),
statTSMLevel1CompactionDuration: atomic.LoadInt64(&e.stats.TSMCompactionDuration[0]),
statTSMLevel2Compactions: atomic.LoadInt64(&e.stats.TSMCompactions[1]),
statTSMLevel2CompactionsActive: atomic.LoadInt64(&e.stats.TSMCompactionsActive[1]),
statTSMLevel2CompactionError: atomic.LoadInt64(&e.stats.TSMCompactionErrors[1]),
statTSMLevel2CompactionDuration: atomic.LoadInt64(&e.stats.TSMCompactionDuration[1]),
statTSMLevel3Compactions: atomic.LoadInt64(&e.stats.TSMCompactions[2]),
statTSMLevel3CompactionsActive: atomic.LoadInt64(&e.stats.TSMCompactionsActive[2]),
statTSMLevel3CompactionError: atomic.LoadInt64(&e.stats.TSMCompactionErrors[2]),
statTSMLevel3CompactionDuration: atomic.LoadInt64(&e.stats.TSMCompactionDuration[2]),
statTSMOptimizeCompactions: atomic.LoadInt64(&e.stats.TSMOptimizeCompactions),
statTSMOptimizeCompactionsActive: atomic.LoadInt64(&e.stats.TSMOptimizeCompactionsActive),
statTSMOptimizeCompactionError: atomic.LoadInt64(&e.stats.TSMOptimizeCompactionErrors),
statTSMOptimizeCompactionDuration: atomic.LoadInt64(&e.stats.TSMOptimizeCompactionDuration),
statTSMFullCompactions: atomic.LoadInt64(&e.stats.TSMFullCompactions),
statTSMFullCompactionsActive: atomic.LoadInt64(&e.stats.TSMFullCompactionsActive),
statTSMFullCompactionError: atomic.LoadInt64(&e.stats.TSMFullCompactionErrors),
statTSMFullCompactionDuration: atomic.LoadInt64(&e.stats.TSMFullCompactionDuration),
},
})
statistics = append(statistics, e.Cache.Statistics(tags)...)
statistics = append(statistics, e.FileStore.Statistics(tags)...)
statistics = append(statistics, e.WAL.Statistics(tags)...)
return statistics
}
// DiskSize returns the total size in bytes of all TSM and WAL segments on disk.
func (e *Engine) DiskSize() int64 {
return e.FileStore.DiskSizeBytes() + e.WAL.DiskSizeBytes()
}
// Open opens and initializes the engine.
func (e *Engine) Open() error {
if err := os.MkdirAll(e.path, 0777); err != nil {
return err
}
if err := e.cleanup(); err != nil {
return err
}
if err := e.WAL.Open(); err != nil {
return err
}
if err := e.FileStore.Open(); err != nil {
return err
}
if err := e.reloadCache(); 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()
e.done = nil // Ensures that the channel will not be closed again.
if err := e.FileStore.Close(); err != nil {
return err
}
return e.WAL.Close()
}
// WithLogger sets the logger for the engine.
func (e *Engine) WithLogger(log zap.Logger) {
e.logger = log.With(zap.String("engine", "tsm1"))
if e.traceLogging {
e.traceLogger = e.logger
}
e.WAL.WithLogger(e.logger)
e.FileStore.WithLogger(e.logger)
}
// LoadMetadataIndex loads the shard metadata into memory.
func (e *Engine) LoadMetadataIndex(shardID uint64, index tsdb.Index) error {
now := time.Now()
// Save reference to index for iterator creation.
e.index = index
if err := e.FileStore.WalkKeys(func(key []byte, typ byte) error {
fieldType, err := tsmFieldTypeToInfluxQLDataType(typ)
if err != nil {
return err
}
if err := e.addToIndexFromKey(key, fieldType); err != nil {
return err
}
return nil
}); err != nil {
return err
}
// load metadata from the Cache
if err := e.Cache.ApplyEntryFn(func(key []byte, entry *entry) error {
fieldType, err := entry.values.InfluxQLType()
if err != nil {
e.logger.Info(fmt.Sprintf("error getting the data type of values for key %s: %s", key, err.Error()))
}
if err := e.addToIndexFromKey(key, fieldType); err != nil {
return err
}
return nil
}); err != nil {
return err
}
e.traceLogger.Info(fmt.Sprintf("Meta data index for shard %d loaded in %v", shardID, time.Since(now)))
return nil
}
// 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
runningCompactions := atomic.LoadInt64(&e.stats.CacheCompactionsActive)
runningCompactions += atomic.LoadInt64(&e.stats.TSMCompactionsActive[0])
runningCompactions += atomic.LoadInt64(&e.stats.TSMCompactionsActive[1])
runningCompactions += atomic.LoadInt64(&e.stats.TSMCompactionsActive[2])
runningCompactions += atomic.LoadInt64(&e.stats.TSMFullCompactionsActive)
runningCompactions += atomic.LoadInt64(&e.stats.TSMOptimizeCompactionsActive)
return cacheEmpty && runningCompactions == 0 && e.CompactionPlan.FullyCompacted()
}
// Backup writes a tar archive of any TSM files modified since the passed
// in time to the passed in writer. The basePath will be prepended to the names
// of the files in the archive. It will force a snapshot of the WAL first
// then perform the backup with a read lock against the file store. This means
// that new TSM files will not be able to be created in this shard while the
// backup is running. For shards that are still acively getting writes, this
// could cause the WAL to backup, increasing memory usage and evenutally rejecting writes.
func (e *Engine) Backup(w io.Writer, basePath string, since time.Time) error {
path, err := e.CreateSnapshot()
if err != nil {
return err
}
if err := e.index.SnapshotTo(path); err != nil {
return err
}
tw := tar.NewWriter(w)
defer tw.Close()
// Remove the temporary snapshot dir
defer os.RemoveAll(path)
// Recursively read all files from path.
files, err := readDir(path, "")
if err != nil {
return err
}
// Filter paths to only changed files.
var filtered []string
for _, file := range files {
fi, err := os.Stat(filepath.Join(path, file))
if err != nil {
return err
} else if !fi.ModTime().After(since) {
continue
}
filtered = append(filtered, file)
}
if len(filtered) == 0 {
return nil
}
for _, f := range filtered {
if err := e.writeFileToBackup(f, basePath, filepath.Join(path, f), tw); err != nil {
return err
}
}
return nil
}
// writeFileToBackup copies the file into the tar archive. Files will use the shardRelativePath
// in their names. This should be the <db>/<retention policy>/<id> part of the path.
func (e *Engine) writeFileToBackup(name string, shardRelativePath, fullPath string, tw *tar.Writer) error {
f, err := os.Stat(fullPath)
if err != nil {
return err
}
h := &tar.Header{
Name: filepath.ToSlash(filepath.Join(shardRelativePath, name)),
ModTime: f.ModTime(),
Size: f.Size(),
Mode: int64(f.Mode()),
}
if err := tw.WriteHeader(h); err != nil {
return err
}
fr, err := os.Open(fullPath)
if err != nil {
return err
}
defer fr.Close()
_, err = io.CopyN(tw, fr, h.Size)
return err
}
// Restore reads a tar archive generated by Backup().
// Only files that match basePath will be copied into the directory. This obtains
// a write lock so no operations can be performed while restoring.
func (e *Engine) Restore(r io.Reader, basePath string) error {
return e.overlay(r, basePath, false)
}
// Import reads a tar archive generated by Backup() and adds each
// file matching basePath as a new TSM file. This obtains
// a write lock so no operations can be performed while Importing.
func (e *Engine) Import(r io.Reader, basePath string) error {
return e.overlay(r, basePath, true)
}
// overlay reads a tar archive generated by Backup() and adds each file
// from the archive matching basePath to the shard.
// If asNew is true, each file will be installed as a new TSM file even if an
// existing file with the same name in the backup exists.
func (e *Engine) overlay(r io.Reader, basePath string, asNew bool) error {
// Copy files from archive while under lock to prevent reopening.
newFiles, err := func() ([]string, error) {
e.mu.Lock()
defer e.mu.Unlock()
var newFiles []string
tr := tar.NewReader(r)
for {
if fileName, err := e.readFileFromBackup(tr, basePath, asNew); err == io.EOF {
break
} else if err != nil {
return nil, err
} else if fileName != "" {
newFiles = append(newFiles, fileName)
}
}
if err := syncDir(e.path); err != nil {
return nil, err
}
if err := e.FileStore.Replace(nil, newFiles); err != nil {
return nil, err
}
return newFiles, nil
}()
if err != nil {
return err
}
// Load any new series keys to the index
readers := make([]chan seriesKey, 0, len(newFiles))
for _, f := range newFiles {
ch := make(chan seriesKey, 1)
readers = append(readers, ch)
// If asNew is true, the files created from readFileFromBackup will be new ones
// having a temp extension.
f = strings.TrimSuffix(f, ".tmp")
fd, err := os.Open(f)
if err != nil {
return err
}
r, err := NewTSMReader(fd)
if err != nil {
return err
}
defer r.Close()
go func(c chan seriesKey, r *TSMReader) {
n := r.KeyCount()
for i := 0; i < n; i++ {
key, typ := r.KeyAt(i)
c <- seriesKey{key, typ}
}
close(c)
}(ch, r)
}
// Merge and dedup all the series keys across each reader to reduce
// lock contention on the index.
merged := merge(readers...)
for v := range merged {
fieldType, err := tsmFieldTypeToInfluxQLDataType(v.typ)
if err != nil {
return err
}
if err := e.addToIndexFromKey(v.key, fieldType); err != nil {
return err
}
}
return nil
}
// readFileFromBackup copies the next file from the archive into the shard.
// The file is skipped if it does not have a matching shardRelativePath prefix.
// If asNew is true, each file will be installed as a new TSM file even if an
// existing file with the same name in the backup exists.
func (e *Engine) readFileFromBackup(tr *tar.Reader, shardRelativePath string, asNew bool) (string, error) {
// Read next archive file.
hdr, err := tr.Next()
if err != nil {
return "", err
}
nativeFileName := filepath.FromSlash(hdr.Name)
// Skip file if it does not have a matching prefix.
if !filepath.HasPrefix(nativeFileName, shardRelativePath) {
return "", nil
}
filename, err := filepath.Rel(shardRelativePath, nativeFileName)
if err != nil {
return "", err
}
if asNew {
filename = fmt.Sprintf("%09d-%09d.%s", e.FileStore.NextGeneration(), 1, TSMFileExtension)
}
destPath := filepath.Join(e.path, filename)
tmp := destPath + ".tmp"
// Create new file on disk.
f, err := os.OpenFile(tmp, os.O_CREATE|os.O_RDWR, 0666)
if err != nil {
return "", err
}
defer f.Close()
// Copy from archive to the file.
if _, err := io.CopyN(f, tr, hdr.Size); err != nil {
return "", err
}
// Sync to disk & close.
if err := f.Sync(); err != nil {
return "", err
}
return tmp, nil
}
// addToIndexFromKey will pull the measurement name, series key, and field name from a composite key and add it to the
// database index and measurement fields
func (e *Engine) addToIndexFromKey(key []byte, fieldType influxql.DataType) error {
seriesKey, field := SeriesAndFieldFromCompositeKey(key)
name := tsdb.MeasurementFromSeriesKey(seriesKey)
mf := e.fieldset.CreateFieldsIfNotExists(name)
if err := mf.CreateFieldIfNotExists(field, fieldType, false); err != nil {
return err
}
// Build in-memory index, if necessary.
if e.index.Type() == inmem.IndexName {
tags, _ := models.ParseTags(seriesKey)
if err := e.index.InitializeSeries(seriesKey, name, tags); err != nil {
return err
}
}
return nil
}
// WritePoints writes metadata and point data into the engine.
// It returns an error if new points are added to an existing key.
func (e *Engine) WritePoints(points []models.Point) error {
values := make(map[string][]Value, len(points))
var keyBuf []byte
var baseLen int
for _, p := range points {
keyBuf = append(keyBuf[:0], p.Key()...)
keyBuf = append(keyBuf, keyFieldSeparator...)
baseLen = len(keyBuf)
iter := p.FieldIterator()
t := p.Time().UnixNano()
for iter.Next() {
// Skip fields name "time", they are illegal
if bytes.Equal(iter.FieldKey(), timeBytes) {
continue
}
keyBuf = append(keyBuf[:baseLen], iter.FieldKey()...)
var v Value
switch iter.Type() {
case models.Float:
fv, err := iter.FloatValue()
if err != nil {
return err
}
v = NewFloatValue(t, fv)
case models.Integer:
iv, err := iter.IntegerValue()
if err != nil {
return err
}
v = NewIntegerValue(t, iv)
case models.Unsigned:
iv, err := iter.UnsignedValue()
if err != nil {
return err
}
v = NewUnsignedValue(t, iv)
case models.String:
v = NewStringValue(t, iter.StringValue())
case models.Boolean:
bv, err := iter.BooleanValue()
if err != nil {
return err
}
v = NewBooleanValue(t, bv)
default:
return fmt.Errorf("unknown field type for %s: %s", string(iter.FieldKey()), p.String())
}
values[string(keyBuf)] = append(values[string(keyBuf)], v)
}
}
e.mu.RLock()
defer e.mu.RUnlock()
// first try to write to the cache
err := e.Cache.WriteMulti(values)
if err != nil {
return err
}
_, err = e.WAL.WriteMulti(values)
return err
}
// containsSeries returns a map of keys indicating whether the key exists and
// has values or not.
func (e *Engine) containsSeries(keys [][]byte) (map[string]bool, error) {
// keyMap is used to see if a given key exists. keys
// are the measurement + tagset (minus separate & field)
keyMap := map[string]bool{}
for _, k := range keys {
keyMap[string(k)] = false
}
for _, k := range e.Cache.unsortedKeys() {
seriesKey, _ := SeriesAndFieldFromCompositeKey([]byte(k))
keyMap[string(seriesKey)] = true
}
if err := e.FileStore.WalkKeys(func(k []byte, _ byte) error {
seriesKey, _ := SeriesAndFieldFromCompositeKey(k)
if _, ok := keyMap[string(seriesKey)]; ok {
keyMap[string(seriesKey)] = true
}
return nil
}); err != nil {
return nil, err
}
return keyMap, nil
}
// deleteSeries removes all series keys from the engine.
func (e *Engine) deleteSeries(seriesKeys [][]byte) error {
return e.DeleteSeriesRange(seriesKeys, math.MinInt64, math.MaxInt64)
}
// DeleteSeriesRange removes the values between min and max (inclusive) from all series.
func (e *Engine) DeleteSeriesRange(seriesKeys [][]byte, min, max int64) error {
if len(seriesKeys) == 0 {
return nil
}
// Ensure keys are sorted since lower layers require them to be.
if !bytesutil.IsSorted(seriesKeys) {
bytesutil.Sort(seriesKeys)
}
// Disable and abort running compactions so that tombstones added existing tsm
// files don't get removed. This would cause deleted measurements/series to
// re-appear once the compaction completed. We only disable the level compactions
// so that snapshotting does not stop while writing out tombstones. If it is stopped,
// and writing tombstones takes a long time, writes can get rejected due to the cache
// filling up.
e.disableLevelCompactions(true)
defer e.enableLevelCompactions(true)
tempKeys := seriesKeys[:]
deleteKeys := make([][]byte, 0, len(seriesKeys))
// go through the keys in the file store
if err := e.FileStore.WalkKeys(func(k []byte, _ byte) error {
seriesKey, _ := SeriesAndFieldFromCompositeKey(k)
// Both tempKeys and keys walked are sorted, skip any passed in keys
// that don't exist in our key set.
for len(tempKeys) > 0 && bytes.Compare(tempKeys[0], seriesKey) < 0 {
tempKeys = tempKeys[1:]
}
// Keys match, add the full series key to delete.
if len(tempKeys) > 0 && bytes.Equal(tempKeys[0], seriesKey) {
deleteKeys = append(deleteKeys, k)
}
return nil
}); err != nil {
return err
}
if err := e.FileStore.DeleteRange(deleteKeys, min, max); err != nil {
return err
}
// find the keys in the cache and remove them
walKeys := deleteKeys[:0]
// ApplySerialEntryFn cannot return an error in this invocation.
_ = e.Cache.ApplyEntryFn(func(k []byte, _ *entry) error {
seriesKey, _ := SeriesAndFieldFromCompositeKey([]byte(k))
// Cache does not walk keys in sorted order, so search the sorted
// series we need to delete to see if any of the cache keys match.
i := bytesutil.SearchBytes(seriesKeys, seriesKey)
if i < len(seriesKeys) && bytes.Equal(seriesKey, seriesKeys[i]) {
// k is the measurement + tags + sep + field
walKeys = append(walKeys, k)
}
return nil
})
e.Cache.DeleteRange(walKeys, min, max)
// delete from the WAL
if _, err := e.WAL.DeleteRange(walKeys, min, max); err != nil {
return err
}
// Have we deleted all points for the series? If so, we need to remove
// the series from the index.
existing, err := e.containsSeries(seriesKeys)