forked from influxdata/influxdb
/
tsm1.go
2371 lines (1995 loc) · 61.9 KB
/
tsm1.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
import (
"encoding/binary"
"encoding/json"
"fmt"
"hash/fnv"
"io"
"io/ioutil"
"log"
"math"
"os"
"path/filepath"
"sort"
"strings"
"sync"
"syscall"
"time"
"github.com/golang/snappy"
"github.com/influxdb/influxdb/models"
"github.com/influxdb/influxdb/tsdb"
)
const (
// Format is the file format name of this engine.
Format = "tsm1"
//IDsFileExtension is the extension for the file that keeps the compressed map
// of keys to uint64 IDs.
IDsFileExtension = "ids"
// FieldsFileExtension is the extension for the file that stores compressed field
// encoding data for this db
FieldsFileExtension = "fields"
// SeriesFileExtension is the extension for the file that stores the compressed
// series metadata for series in this db
SeriesFileExtension = "series"
// CollisionsFileExtension is the extension for the file that keeps a map of which
// keys have hash collisions and what their actual IDs are
CollisionsFileExtension = "collisions"
// CheckpointExtension is the extension given to files that checkpoint a rewrite or compaction.
// The checkpoint files are created when a new file is first created. They
// are removed after the file has been synced and is safe for use. If a file
// has an associated checkpoint file, it wasn't safely written and both should be removed
CheckpointExtension = "check"
// CompactionExtension is the extension given to the file that marks when a compaction has been
// fully written, but the compacted files have not yet been deleted. It is used for cleanup
// if the server was not cleanly shutdown before the compacted files could be deleted.
CompactionExtension = "compact"
// keyFieldSeparator separates the series key from the field name in the composite key
// that identifies a specific field in series
keyFieldSeparator = "#!~#"
blockBufferSize = 1024 * 1024
)
type TimePrecision uint8
const (
Seconds TimePrecision = iota
Milliseconds
Microseconds
Nanoseconds
)
func init() {
tsdb.RegisterEngine(Format, NewEngine)
}
const (
MaxDataFileSize = 1024 * 1024 * 1024 * 2 // 2GB
DefaultRotateFileSize = 5 * 1024 * 1024 // 5MB
DefaultMaxPointsPerBlock = 1000
// MAP_POPULATE is for the mmap syscall. For some reason this isn't defined in golang's syscall
MAP_POPULATE = 0x8000
)
// Ensure Engine implements the interface.
var _ tsdb.Engine = &Engine{}
// Engine represents a storage engine with compressed blocks.
type Engine struct {
writeLock *WriteLock
metaLock sync.Mutex
path string
logger *log.Logger
// deletesPending mark how many old data files are waiting to be deleted. This will
// keep a close from returning until all deletes finish
deletesPending sync.WaitGroup
// HashSeriesField is a function that takes a series key and a field name
// and returns a hash identifier. It's not guaranteed to be unique.
HashSeriesField func(key string) uint64
WAL *Log
RotateFileSize uint32
MaxFileSize uint32
SkipCompaction bool
CompactionAge time.Duration
MinCompactionFileCount int
IndexCompactionFullAge time.Duration
IndexMinCompactionInterval time.Duration
MaxPointsPerBlock int
// filesLock is only for modifying and accessing the files slice
filesLock sync.RWMutex
files dataFiles
currentFileID int
compactionRunning bool
lastCompactionTime time.Time
// deletes is a map of keys that are deleted, but haven't yet been
// compacted and flushed. They map the ID to the corresponding key
deletes map[uint64]string
// deleteMeasurements is a map of the measurements that are deleted
// but haven't yet been compacted and flushed
deleteMeasurements map[string]bool
collisionsLock sync.RWMutex
collisions map[string]uint64
// queryLock keeps data files from being deleted or the store from
// being closed while queries are running
queryLock sync.RWMutex
}
// NewEngine returns a new instance of Engine.
func NewEngine(path string, walPath string, opt tsdb.EngineOptions) tsdb.Engine {
w := NewLog(path)
w.FlushColdInterval = time.Duration(opt.Config.WALFlushColdInterval)
w.FlushMemorySizeThreshold = opt.Config.WALFlushMemorySizeThreshold
w.MaxMemorySizeThreshold = opt.Config.WALMaxMemorySizeThreshold
w.LoggingEnabled = opt.Config.WALLoggingEnabled
e := &Engine{
path: path,
writeLock: &WriteLock{},
logger: log.New(os.Stderr, "[tsm1] ", log.LstdFlags),
// TODO: this is the function where we can inject a check against the in memory collisions
HashSeriesField: hashSeriesField,
WAL: w,
RotateFileSize: DefaultRotateFileSize,
MaxFileSize: MaxDataFileSize,
CompactionAge: opt.Config.IndexCompactionAge,
MinCompactionFileCount: opt.Config.IndexMinCompactionFileCount,
IndexCompactionFullAge: opt.Config.IndexCompactionFullAge,
IndexMinCompactionInterval: opt.Config.IndexMinCompactionInterval,
MaxPointsPerBlock: DefaultMaxPointsPerBlock,
}
e.WAL.IndexWriter = e
return e
}
// Path returns the path the engine was opened with.
func (e *Engine) Path() string { return e.path }
// PerformMaintenance is for periodic maintenance of the store. A no-op for b1
func (e *Engine) PerformMaintenance() {
if f := e.WAL.shouldFlush(); f != noFlush {
go func() {
if err := e.WAL.flush(f); err != nil {
e.logger.Printf("PerformMaintenance: WAL flush failed: %v", err)
}
}()
return
}
// don't do a full compaction if the WAL received writes in the time window
if time.Since(e.WAL.LastWriteTime()) < e.IndexCompactionFullAge {
return
}
e.filesLock.RLock()
running := e.compactionRunning
deletesPending := len(e.deletes) > 0
e.filesLock.RUnlock()
if running || deletesPending {
return
}
// do a full compaction if all the index files are older than the compaction time
for _, f := range e.copyFilesCollection() {
if time.Since(f.modTime) < e.IndexCompactionFullAge {
return
}
}
go func() {
if err := e.Compact(true); err != nil {
e.logger.Printf("PerformMaintenance: error during compaction: %v", err)
}
}()
}
// Format returns the format type of this engine
func (e *Engine) Format() tsdb.EngineFormat {
return tsdb.TSM1Format
}
// Open opens and initializes the engine.
func (e *Engine) Open() error {
if err := os.MkdirAll(e.path, 0777); err != nil {
return err
}
// perform any cleanup on metafiles that were halfway written
e.cleanupMetafile(SeriesFileExtension)
e.cleanupMetafile(FieldsFileExtension)
e.cleanupMetafile(IDsFileExtension)
e.cleanupMetafile(CollisionsFileExtension)
e.cleanupUnfinishedCompaction()
files, err := filepath.Glob(filepath.Join(e.path, fmt.Sprintf("*.%s", Format)))
if err != nil {
return err
}
for _, fn := range files {
// if the file has a checkpoint it's not valid, so remove it
if removed := e.removeFileIfCheckpointExists(fn); removed {
continue
}
id, err := idFromFileName(fn)
if err != nil {
return err
}
if id >= e.currentFileID {
e.currentFileID = id + 1
}
f, err := os.OpenFile(fn, os.O_RDONLY, 0666)
if err != nil {
return fmt.Errorf("error opening file %s: %s", fn, err.Error())
}
df, err := NewDataFile(f)
if err != nil {
return fmt.Errorf("error opening memory map for file %s: %s", fn, err.Error())
}
e.files = append(e.files, df)
}
sort.Sort(e.files)
if err := e.readCollisions(); err != nil {
return err
}
e.deletes = make(map[uint64]string)
e.deleteMeasurements = make(map[string]bool)
// mark the last compaction as now so it doesn't try to compact while
// flushing the WAL on load
e.lastCompactionTime = time.Now()
if err := e.WAL.Open(); err != nil {
return err
}
e.lastCompactionTime = time.Now()
return nil
}
// cleanupUnfinishedConpaction will read any compaction markers. If the marker exists, the compaction finished successfully,
// but didn't get fully cleaned up. Remove the old files and their checkpoints
func (e *Engine) cleanupUnfinishedCompaction() {
files, err := filepath.Glob(filepath.Join(e.path, fmt.Sprintf("*.%s", CompactionExtension)))
if err != nil {
panic(fmt.Sprintf("error getting compaction checkpoints: %s", err.Error()))
}
for _, fn := range files {
f, err := os.OpenFile(fn, os.O_RDONLY, 0666)
if err != nil {
panic(fmt.Sprintf("error opening compaction info file: %s", err.Error()))
}
data, err := ioutil.ReadAll(f)
if err != nil {
panic(fmt.Sprintf("error reading compaction info file: %s", err.Error()))
}
c := &compactionCheckpoint{}
err = json.Unmarshal(data, c)
if err == nil {
c.cleanup()
}
if err := f.Close(); err != nil {
panic(fmt.Sprintf("error closing compaction checkpoint: %s", err.Error()))
}
if err := os.RemoveAll(f.Name()); err != nil {
panic(fmt.Sprintf("error removing compaction checkpoint: %s", err.Error()))
}
}
}
// Close closes the engine.
func (e *Engine) Close() error {
// get all the locks so queries, writes, and compactions stop before closing
e.queryLock.Lock()
defer e.queryLock.Unlock()
e.metaLock.Lock()
defer e.metaLock.Unlock()
min, max := int64(math.MinInt64), int64(math.MaxInt64)
e.writeLock.LockRange(min, max)
defer e.writeLock.UnlockRange(min, max)
e.filesLock.Lock()
defer e.filesLock.Unlock()
e.WAL.Close()
// ensure all deletes have been processed
e.deletesPending.Wait()
for _, df := range e.files {
_ = df.Close()
}
e.files = nil
e.currentFileID = 0
e.collisions = nil
e.deletes = nil
e.deleteMeasurements = nil
return nil
}
// DataFileCount returns the number of data files in the database
func (e *Engine) DataFileCount() int {
e.filesLock.RLock()
defer e.filesLock.RUnlock()
return len(e.files)
}
// SetLogOutput is a no-op.
func (e *Engine) SetLogOutput(w io.Writer) {}
// LoadMetadataIndex loads the shard metadata into memory.
func (e *Engine) LoadMetadataIndex(shard *tsdb.Shard, index *tsdb.DatabaseIndex, measurementFields map[string]*tsdb.MeasurementFields) error {
// Load measurement metadata
fields, err := e.readFields()
if err != nil {
return err
}
for k, mf := range fields {
m := index.CreateMeasurementIndexIfNotExists(string(k))
for name := range mf.Fields {
m.SetFieldName(name)
}
mf.Codec = tsdb.NewFieldCodec(mf.Fields)
measurementFields[m.Name] = mf
}
// Load series metadata
series, err := e.readSeries()
if err != nil {
return err
}
// Load the series into the in-memory index in sorted order to ensure
// it's always consistent for testing purposes
a := make([]string, 0, len(series))
for k := range series {
a = append(a, k)
}
sort.Strings(a)
for _, key := range a {
s := series[key]
s.InitializeShards()
index.CreateSeriesIndexIfNotExists(tsdb.MeasurementFromSeriesKey(string(key)), s)
}
return nil
}
// WritePoints writes metadata and point data into the engine.
// Returns an error if new points are added to an existing key.
func (e *Engine) WritePoints(points []models.Point, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error {
return e.WAL.WritePoints(points, measurementFieldsToSave, seriesToCreate)
}
func (e *Engine) Write(pointsByKey map[string]Values, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) error {
// Flush any deletes before writing new data from the WAL
e.filesLock.RLock()
hasDeletes := len(e.deletes) > 0
e.filesLock.RUnlock()
if hasDeletes {
e.flushDeletes()
}
startTime, endTime, valuesByID, err := e.convertKeysAndWriteMetadata(pointsByKey, measurementFieldsToSave, seriesToCreate)
if err != nil {
return err
}
if len(valuesByID) == 0 {
return nil
}
files, lockStart, lockEnd := e.filesAndLock(startTime, endTime)
defer e.writeLock.UnlockRange(lockStart, lockEnd)
if len(files) == 0 {
return e.rewriteFile(nil, valuesByID)
}
maxTime := int64(math.MaxInt64)
// do the file rewrites in parallel
var mu sync.Mutex
var writes sync.WaitGroup
var errors []error
// reverse through the data files and write in the data
for i := len(files) - 1; i >= 0; i-- {
f := files[i]
// max times are exclusive, so add 1 to it
fileMax := f.MaxTime() + 1
fileMin := f.MinTime()
// if the file is < rotate, write all data between fileMin and maxTime
if f.size < e.RotateFileSize {
writes.Add(1)
go func(df *dataFile, vals map[uint64]Values) {
if err := e.rewriteFile(df, vals); err != nil {
mu.Lock()
errors = append(errors, err)
mu.Unlock()
}
writes.Done()
}(f, e.filterDataBetweenTimes(valuesByID, fileMin, maxTime))
continue
}
// if the file is > rotate:
// write all data between fileMax and maxTime into new file
// write all data between fileMin and fileMax into old file
writes.Add(1)
go func(vals map[uint64]Values) {
if err := e.rewriteFile(nil, vals); err != nil {
mu.Lock()
errors = append(errors, err)
mu.Unlock()
}
writes.Done()
}(e.filterDataBetweenTimes(valuesByID, fileMax, maxTime))
writes.Add(1)
go func(df *dataFile, vals map[uint64]Values) {
if err := e.rewriteFile(df, vals); err != nil {
mu.Lock()
errors = append(errors, err)
mu.Unlock()
}
writes.Done()
}(f, e.filterDataBetweenTimes(valuesByID, fileMin, fileMax))
maxTime = fileMin
}
// for any data leftover, write into a new file since it's all older
// than any file we currently have
writes.Add(1)
go func() {
if err := e.rewriteFile(nil, valuesByID); err != nil {
mu.Lock()
errors = append(errors, err)
mu.Unlock()
}
writes.Done()
}()
writes.Wait()
if len(errors) > 0 {
// TODO: log errors
return errors[0]
}
if !e.SkipCompaction && e.shouldCompact() {
go func() {
if err := e.Compact(false); err != nil {
e.logger.Printf("Write: error during compaction: %v", err)
}
}()
}
return nil
}
// MarkDeletes will mark the given keys for deletion in memory. They will be deleted from data
// files on the next flush. This mainly for the WAL to use on startup
func (e *Engine) MarkDeletes(keys []string) {
e.filesLock.Lock()
defer e.filesLock.Unlock()
for _, k := range keys {
e.deletes[e.keyToID(k)] = k
}
}
func (e *Engine) MarkMeasurementDelete(name string) {
e.filesLock.Lock()
defer e.filesLock.Unlock()
e.deleteMeasurements[name] = true
}
// filesAndLock returns the data files that match the given range and
// ensures that the write lock will hold for the entire range
func (e *Engine) filesAndLock(min, max int64) (a dataFiles, lockStart, lockEnd int64) {
for {
a = make([]*dataFile, 0)
files := e.copyFilesCollection()
e.filesLock.RLock()
for _, f := range e.files {
fmin, fmax := f.MinTime(), f.MaxTime()
if min < fmax && fmin >= fmin {
a = append(a, f)
} else if max >= fmin && max < fmax {
a = append(a, f)
}
}
e.filesLock.RUnlock()
if len(a) > 0 {
lockStart = a[0].MinTime()
lockEnd = a[len(a)-1].MaxTime()
if max > lockEnd {
lockEnd = max
}
} else {
lockStart = min
lockEnd = max
}
e.writeLock.LockRange(lockStart, lockEnd)
// it's possible for compaction to change the files collection while we
// were waiting for a write lock on the range. Make sure the files are still the
// same after we got the lock, otherwise try again. This shouldn't happen often.
filesAfterLock := e.copyFilesCollection()
if dataFilesEquals(files, filesAfterLock) {
return
}
e.writeLock.UnlockRange(lockStart, lockEnd)
}
}
// getCompactionFiles will return the list of files ready to be compacted along with the min and
// max time of the write lock obtained for compaction
func (e *Engine) getCompactionFiles(fullCompaction bool) (minTime, maxTime int64, files dataFiles) {
// we're looping here to ensure that the files we've marked to compact are
// still there after we've obtained the write lock
for {
if fullCompaction {
files = e.copyFilesCollection()
} else {
files = e.filesToCompact()
}
if len(files) < 2 {
return minTime, maxTimeOffset, nil
}
minTime = files[0].MinTime()
maxTime = files[len(files)-1].MaxTime()
e.writeLock.LockRange(minTime, maxTime)
// if the files are different after obtaining the write lock, one or more
// was rewritten. Release the lock and try again. This shouldn't happen really.
var filesAfterLock dataFiles
if fullCompaction {
filesAfterLock = e.copyFilesCollection()
} else {
filesAfterLock = e.filesToCompact()
}
if !dataFilesEquals(files, filesAfterLock) {
e.writeLock.UnlockRange(minTime, maxTime)
continue
}
// we've got the write lock and the files are all there
return
}
}
// compactToNewFiles will compact the passed in data files into as few files as possible
func (e *Engine) compactToNewFiles(minTime, maxTime int64, files dataFiles) []*os.File {
fileName := e.nextFileName()
e.logger.Printf("Starting compaction in %s of %d files to new file %s", e.path, len(files), fileName)
compaction := newCompactionJob(files, minTime, maxTime, e.MaxFileSize, e.MaxPointsPerBlock)
compaction.newCurrentFile(fileName)
// loop writing data until we've read through all the files
for {
nextID := compaction.nextID()
if nextID == dataFileEOF {
break
}
// write data for this ID while rotating to new files if necessary
for {
moreToWrite := compaction.writeBlocksForID(nextID)
if !moreToWrite {
break
}
compaction.newCurrentFile(e.nextFileName())
}
}
// close out the current compacted file
compaction.writeOutCurrentFile()
return compaction.newFiles
}
// Compact will compact data files in the directory into the fewest possible data files they
// can be combined into
func (e *Engine) Compact(fullCompaction bool) error {
minTime, maxTime, files := e.getCompactionFiles(fullCompaction)
if len(files) < 2 {
return nil
}
// mark the compaction as running
e.filesLock.Lock()
if e.compactionRunning {
e.filesLock.Unlock()
return nil
}
e.compactionRunning = true
e.filesLock.Unlock()
defer func() {
//release the lock
e.writeLock.UnlockRange(minTime, maxTime)
e.filesLock.Lock()
e.lastCompactionTime = time.Now()
e.compactionRunning = false
e.filesLock.Unlock()
}()
st := time.Now()
newFiles := e.compactToNewFiles(minTime, maxTime, files)
newDataFiles := make(dataFiles, len(newFiles))
for i, f := range newFiles {
// now open it as a memory mapped data file
newDF, err := NewDataFile(f)
if err != nil {
return err
}
newDataFiles[i] = newDF
}
// write the compaction file to note that we've successfully commpleted the write portion of compaction
compactedFileNames := make([]string, len(files))
newFileNames := make([]string, len(newFiles))
for i, f := range files {
compactedFileNames[i] = f.f.Name()
}
for i, f := range newFiles {
newFileNames[i] = f.Name()
}
compactionCheckpointName, err := e.writeCompactionCheckpointFile(compactedFileNames, newFileNames)
if err != nil {
return err
}
// update engine with new file pointers
e.filesLock.Lock()
var replacementFiles dataFiles
for _, df := range e.files {
// exclude any files that were compacted
include := true
for _, f := range files {
if f == df {
include = false
break
}
}
if include {
replacementFiles = append(replacementFiles, df)
}
}
replacementFiles = append(replacementFiles, newDataFiles...)
sort.Sort(replacementFiles)
e.files = replacementFiles
e.filesLock.Unlock()
e.logger.Printf("Compaction of %s took %s", e.path, time.Since(st))
e.clearCompactedFiles(compactionCheckpointName, newFiles, files)
return nil
}
// clearCompactedFiles will remove the compaction checkpoints for new files, remove the old compacted files, and
// finally remove the compaction checkpoint
func (e *Engine) clearCompactedFiles(compactionCheckpointName string, newFiles []*os.File, oldFiles dataFiles) {
// delete the old files in a goroutine so running queries won't block the write
// from completing
e.deletesPending.Add(1)
go func() {
// first clear out the compaction checkpoints
for _, f := range newFiles {
if err := removeCheckpoint(f.Name()); err != nil {
// panic here since continuing could cause data loss. It's better to fail hard so
// everything can be recovered on restart
panic(fmt.Sprintf("error removing checkpoint file %s: %s", f.Name(), err.Error()))
}
}
// now delete the underlying data files
for _, f := range oldFiles {
if err := f.Delete(); err != nil {
panic(fmt.Sprintf("error deleting old file after compaction %s: %s", f.f.Name(), err.Error()))
}
}
// finally remove the compaction marker
if err := os.RemoveAll(compactionCheckpointName); err != nil {
e.logger.Printf("error removing %s: %s", compactionCheckpointName, err.Error())
}
e.deletesPending.Done()
}()
}
// writeCompactionCheckpointFile will save the compacted filenames and new filenames in
// a file. This is used on startup to clean out files that weren't deleted if the server
// wasn't shut down cleanly.
func (e *Engine) writeCompactionCheckpointFile(compactedFiles, newFiles []string) (string, error) {
m := &compactionCheckpoint{
CompactedFiles: compactedFiles,
NewFiles: newFiles,
}
data, err := json.Marshal(m)
if err != nil {
return "", err
}
// make the compacted filename the same name as the first compacted file, but with the compacted extension
name := strings.Split(filepath.Base(compactedFiles[0]), ".")[0]
fn := fmt.Sprintf("%s.%s", name, CompactionExtension)
fileName := filepath.Join(filepath.Dir(compactedFiles[0]), fn)
f, err := os.OpenFile(fileName, os.O_CREATE|os.O_RDWR, 0666)
if err != nil {
return "", err
}
if _, err := f.Write(data); err != nil {
f.Close()
return fileName, err
}
return fileName, f.Close()
}
func (e *Engine) writeIndexAndGetDataFile(f *os.File, minTime, maxTime int64, ids []uint64, newPositions []uint32) (*dataFile, error) {
if err := writeIndex(f, minTime, maxTime, ids, newPositions); err != nil {
return nil, err
}
if err := removeCheckpoint(f.Name()); err != nil {
return nil, err
}
// now open it as a memory mapped data file
newDF, err := NewDataFile(f)
if err != nil {
return nil, err
}
return newDF, nil
}
func (e *Engine) shouldCompact() bool {
e.filesLock.RLock()
running := e.compactionRunning
since := time.Since(e.lastCompactionTime)
deletesPending := len(e.deletes) > 0
e.filesLock.RUnlock()
if running || since < e.IndexMinCompactionInterval || deletesPending {
return false
}
return len(e.filesToCompact()) >= e.MinCompactionFileCount
}
func (e *Engine) filesToCompact() dataFiles {
e.filesLock.RLock()
defer e.filesLock.RUnlock()
var a dataFiles
for _, df := range e.files {
if time.Since(df.modTime) > e.CompactionAge && df.size < e.MaxFileSize {
a = append(a, df)
} else if len(a) > 0 {
// only compact contiguous ranges. If we hit the negative case and
// there are files to compact, stop here
break
}
}
return a
}
func (e *Engine) convertKeysAndWriteMetadata(pointsByKey map[string]Values, measurementFieldsToSave map[string]*tsdb.MeasurementFields, seriesToCreate []*tsdb.SeriesCreate) (minTime, maxTime int64, valuesByID map[uint64]Values, err error) {
e.metaLock.Lock()
defer e.metaLock.Unlock()
if err := e.writeNewFields(measurementFieldsToSave); err != nil {
return 0, 0, nil, err
}
if err := e.writeNewSeries(seriesToCreate); err != nil {
return 0, 0, nil, err
}
if len(pointsByKey) == 0 {
return 0, 0, nil, nil
}
// read in keys and assign any that aren't defined
b, err := e.readCompressedFile(IDsFileExtension)
if err != nil {
return 0, 0, nil, err
}
ids := make(map[string]uint64)
if b != nil {
if err := json.Unmarshal(b, &ids); err != nil {
return 0, 0, nil, err
}
}
// these are values that are newer than anything stored in the shard
valuesByID = make(map[uint64]Values)
idToKey := make(map[uint64]string) // we only use this map if new ids are being created
collisions := make(map[string]uint64) // we only use this if a collision is encountered
newKeys := false
// track the min and max time of values being inserted so we can lock that time range
minTime = int64(math.MaxInt64)
maxTime = int64(math.MinInt64)
for k, values := range pointsByKey {
var id uint64
var ok bool
if id, ok = ids[k]; !ok {
// populate the map if we haven't already
if len(idToKey) == 0 {
for n, id := range ids {
idToKey[id] = n
}
}
// now see if the hash id collides with a different key
hashID := e.HashSeriesField(k)
existingKey, idInMap := idToKey[hashID]
// we only care if the keys are different. if so, it's a hash collision we have to keep track of
if idInMap && k != existingKey {
// we have a collision, find this new key the next available id
hashID = 0
for {
hashID++
if _, ok := idToKey[hashID]; !ok {
// next ID is available, use it
break
}
}
collisions[k] = hashID
}
newKeys = true
ids[k] = hashID
idToKey[hashID] = k
id = hashID
}
if minTime > values.MinTime() {
minTime = values.MinTime()
}
if maxTime < values.MaxTime() {
maxTime = values.MaxTime()
}
valuesByID[id] = values
}
if newKeys {
b, err := json.Marshal(ids)
if err != nil {
return 0, 0, nil, err
}
if err := e.replaceCompressedFile(IDsFileExtension, b); err != nil {
return 0, 0, nil, err
}
}
if len(collisions) > 0 {
e.saveNewCollisions(collisions)
}
return
}
func (e *Engine) saveNewCollisions(collisions map[string]uint64) error {
e.collisionsLock.Lock()
defer e.collisionsLock.Unlock()
for k, v := range collisions {
e.collisions[k] = v
}
data, err := json.Marshal(e.collisions)
if err != nil {
return err
}
return e.replaceCompressedFile(CollisionsFileExtension, data)
}
func (e *Engine) readCollisions() error {
e.collisions = make(map[string]uint64)
data, err := e.readCompressedFile(CollisionsFileExtension)
if err != nil {
return err
}
if len(data) == 0 {
return nil
}
return json.Unmarshal(data, &e.collisions)
}
// filterDataBetweenTimes will create a new map with data between
// the minTime (inclusive) and maxTime (exclusive) while removing that
// data from the passed in map. It is assume that the Values arrays
// are sorted in time ascending order
func (e *Engine) filterDataBetweenTimes(valuesByID map[uint64]Values, minTime, maxTime int64) map[uint64]Values {
filteredValues := make(map[uint64]Values)
for id, values := range valuesByID {
maxIndex := len(values)
minIndex := -1
// find the index of the first value in the range
for i, v := range values {
t := v.UnixNano()
if t >= minTime && t < maxTime {
minIndex = i
break
}
}
if minIndex == -1 {
continue
}
// go backwards to find the index of the last value in the range
for i := len(values) - 1; i >= 0; i-- {
t := values[i].UnixNano()
if t < maxTime {
maxIndex = i + 1
break
}
}
// write into the result map and filter the passed in map
filteredValues[id] = values[minIndex:maxIndex]
// if we grabbed all the values, remove them from the passed in map
if minIndex == len(values) || (minIndex == 0 && maxIndex == len(values)) {
delete(valuesByID, id)
continue
}
valuesByID[id] = values[0:minIndex]
if maxIndex < len(values) {
valuesByID[id] = append(valuesByID[id], values[maxIndex:]...)
}
}
return filteredValues
}
// rewriteFile will read in the old data file, if provided and merge the values
// in the passed map into a new data file
func (e *Engine) rewriteFile(oldDF *dataFile, valuesByID map[uint64]Values) error {
if len(valuesByID) == 0 {
return nil
}
// we need the values in sorted order so that we can merge them into the
// new file as we read the old file
ids := make([]uint64, 0, len(valuesByID))
for id := range valuesByID {
ids = append(ids, id)
}