forked from influxdata/influxdb
-
Notifications
You must be signed in to change notification settings - Fork 0
/
partition.go
1285 lines (1080 loc) · 31.3 KB
/
partition.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 tsi1
import (
"encoding/json"
"errors"
"fmt"
"io/ioutil"
"os"
"path/filepath"
"regexp"
"strconv"
"strings"
"sync"
"time"
"github.com/influxdata/influxdb/logger"
"github.com/influxdata/influxdb/models"
"github.com/influxdata/influxdb/pkg/bytesutil"
"github.com/influxdata/influxdb/pkg/estimator"
"github.com/influxdata/influxdb/tsdb"
"github.com/influxdata/influxql"
"go.uber.org/zap"
)
// Version is the current version of the TSI index.
const Version = 1
// File extensions.
const (
LogFileExt = ".tsl"
IndexFileExt = ".tsi"
CompactingExt = ".compacting"
)
// ManifestFileName is the name of the index manifest file.
const ManifestFileName = "MANIFEST"
// Partition represents a collection of layered index files and WAL.
type Partition struct {
mu sync.RWMutex
opened bool
sfile *tsdb.SeriesFile // series lookup file
activeLogFile *LogFile // current log file
fileSet *FileSet // current file set
seq int // file id sequence
// Fast series lookup of series IDs in the series file that have been present
// in this partition. This set tracks both insertions and deletions of a series.
seriesIDSet *tsdb.SeriesIDSet
// Compaction management
levels []CompactionLevel // compaction levels
levelCompacting []bool // level compaction status
// Close management.
once sync.Once
closing chan struct{} // closing is used to inform iterators the partition is closing.
wg sync.WaitGroup
// Fieldset shared with engine.
fieldset *tsdb.MeasurementFieldSet
// Name of database.
Database string
// Directory of the Partition's index files.
path string
id string // id portion of path.
// Log file compaction thresholds.
MaxLogFileSize int64
// Frequency of compaction checks.
compactionInterrupt chan struct{}
compactionsDisabled int
logger *zap.Logger
// Current size of MANIFEST. Used to determine partition size.
manifestSize int64
// Index's version.
version int
}
// NewPartition returns a new instance of Partition.
func NewPartition(sfile *tsdb.SeriesFile, path string) *Partition {
return &Partition{
closing: make(chan struct{}),
path: path,
sfile: sfile,
seriesIDSet: tsdb.NewSeriesIDSet(),
// Default compaction thresholds.
MaxLogFileSize: tsdb.DefaultMaxIndexLogFileSize,
// compactionEnabled: true,
compactionInterrupt: make(chan struct{}),
logger: zap.NewNop(),
version: Version,
}
}
// ErrIncompatibleVersion is returned when attempting to read from an
// incompatible tsi1 manifest file.
var ErrIncompatibleVersion = errors.New("incompatible tsi1 index MANIFEST")
// Open opens the partition.
func (i *Partition) Open() error {
i.mu.Lock()
defer i.mu.Unlock()
i.closing = make(chan struct{})
if i.opened {
return errors.New("index partition already open")
}
// Validate path is correct.
i.id = filepath.Base(i.path)
_, err := strconv.Atoi(i.id)
if err != nil {
return err
}
// Create directory if it doesn't exist.
if err := os.MkdirAll(i.path, 0777); err != nil {
return err
}
// Read manifest file.
m, manifestSize, err := ReadManifestFile(filepath.Join(i.path, ManifestFileName))
if os.IsNotExist(err) {
m = NewManifest(i.ManifestPath())
} else if err != nil {
return err
}
// Set manifest size on the partition
i.manifestSize = manifestSize
// Check to see if the MANIFEST file is compatible with the current Index.
if err := m.Validate(); err != nil {
return err
}
// Copy compaction levels to the index.
i.levels = make([]CompactionLevel, len(m.Levels))
copy(i.levels, m.Levels)
// Set up flags to track whether a level is compacting.
i.levelCompacting = make([]bool, len(i.levels))
// Open each file in the manifest.
var files []File
for _, filename := range m.Files {
switch filepath.Ext(filename) {
case LogFileExt:
f, err := i.openLogFile(filepath.Join(i.path, filename))
if err != nil {
return err
}
files = append(files, f)
// Make first log file active, if within threshold.
sz, _ := f.Stat()
if i.activeLogFile == nil && sz < i.MaxLogFileSize {
i.activeLogFile = f
}
case IndexFileExt:
f, err := i.openIndexFile(filepath.Join(i.path, filename))
if err != nil {
return err
}
files = append(files, f)
}
}
fs, err := NewFileSet(i.Database, i.levels, i.sfile, files)
if err != nil {
return err
}
i.fileSet = fs
// Set initial sequence number.
i.seq = i.fileSet.MaxID()
// Delete any files not in the manifest.
if err := i.deleteNonManifestFiles(m); err != nil {
return err
}
// Ensure a log file exists.
if i.activeLogFile == nil {
if err := i.prependActiveLogFile(); err != nil {
return err
}
}
// Build series existance set.
if err := i.buildSeriesSet(); err != nil {
return err
}
// Mark opened.
i.opened = true
// Send a compaction request on start up.
i.compact()
return nil
}
// openLogFile opens a log file and appends it to the index.
func (i *Partition) openLogFile(path string) (*LogFile, error) {
f := NewLogFile(i.sfile, path)
if err := f.Open(); err != nil {
return nil, err
}
return f, nil
}
// openIndexFile opens a log file and appends it to the index.
func (i *Partition) openIndexFile(path string) (*IndexFile, error) {
f := NewIndexFile(i.sfile)
f.SetPath(path)
if err := f.Open(); err != nil {
return nil, err
}
return f, nil
}
// deleteNonManifestFiles removes all files not in the manifest.
func (i *Partition) deleteNonManifestFiles(m *Manifest) error {
dir, err := os.Open(i.path)
if err != nil {
return err
}
defer dir.Close()
fis, err := dir.Readdir(-1)
if err != nil {
return err
}
// Loop over all files and remove any not in the manifest.
for _, fi := range fis {
filename := filepath.Base(fi.Name())
if filename == ManifestFileName || m.HasFile(filename) {
continue
}
if err := os.RemoveAll(filename); err != nil {
return err
}
}
return nil
}
func (p *Partition) buildSeriesSet() error {
fs := p.retainFileSet()
defer fs.Release()
p.seriesIDSet = tsdb.NewSeriesIDSet()
// Read series sets from files in reverse.
for i := len(fs.files) - 1; i >= 0; i-- {
f := fs.files[i]
// Delete anything that's been tombstoned.
ts, err := f.TombstoneSeriesIDSet()
if err != nil {
return err
}
p.seriesIDSet.Diff(ts)
// Add series created within the file.
ss, err := f.SeriesIDSet()
if err != nil {
return err
}
p.seriesIDSet.Merge(ss)
}
return nil
}
// Wait returns once outstanding compactions have finished.
func (i *Partition) Wait() {
i.wg.Wait()
}
// Close closes the index.
func (i *Partition) Close() error {
// Wait for goroutines to finish outstanding compactions.
i.once.Do(func() {
close(i.closing)
close(i.compactionInterrupt)
})
i.wg.Wait()
// Lock index and close remaining
i.mu.Lock()
defer i.mu.Unlock()
// Close log files.
for _, f := range i.fileSet.files {
f.Close()
}
i.fileSet.files = nil
return nil
}
// closing returns true if the partition is currently closing. It does not require
// a lock so will always return to callers.
func (p *Partition) isClosing() bool {
select {
case <-p.closing:
return true
default:
return false
}
}
// Path returns the path to the partition.
func (i *Partition) Path() string { return i.path }
// SeriesFile returns the attached series file.
func (i *Partition) SeriesFile() *tsdb.SeriesFile { return i.sfile }
// NextSequence returns the next file identifier.
func (i *Partition) NextSequence() int {
i.mu.Lock()
defer i.mu.Unlock()
return i.nextSequence()
}
func (i *Partition) nextSequence() int {
i.seq++
return i.seq
}
// ManifestPath returns the path to the index's manifest file.
func (i *Partition) ManifestPath() string {
return filepath.Join(i.path, ManifestFileName)
}
// Manifest returns a manifest for the index.
func (i *Partition) Manifest() *Manifest {
m := &Manifest{
Levels: i.levels,
Files: make([]string, len(i.fileSet.files)),
Version: i.version,
path: i.ManifestPath(),
}
for j, f := range i.fileSet.files {
m.Files[j] = filepath.Base(f.Path())
}
return m
}
// WithLogger sets the logger for the index.
func (i *Partition) WithLogger(logger *zap.Logger) {
i.logger = logger.With(zap.String("index", "tsi"))
}
// SetFieldSet sets a shared field set from the engine.
func (i *Partition) SetFieldSet(fs *tsdb.MeasurementFieldSet) {
i.mu.Lock()
i.fieldset = fs
i.mu.Unlock()
}
// FieldSet returns the fieldset.
func (i *Partition) FieldSet() *tsdb.MeasurementFieldSet {
i.mu.Lock()
fs := i.fieldset
i.mu.Unlock()
return fs
}
// RetainFileSet returns the current fileset and adds a reference count.
func (i *Partition) RetainFileSet() (*FileSet, error) {
select {
case <-i.closing:
return nil, errors.New("index is closing")
default:
i.mu.RLock()
defer i.mu.RUnlock()
return i.retainFileSet(), nil
}
}
func (i *Partition) retainFileSet() *FileSet {
fs := i.fileSet
fs.Retain()
return fs
}
// FileN returns the active files in the file set.
func (i *Partition) FileN() int { return len(i.fileSet.files) }
// prependActiveLogFile adds a new log file so that the current log file can be compacted.
func (i *Partition) prependActiveLogFile() error {
// Open file and insert it into the first position.
f, err := i.openLogFile(filepath.Join(i.path, FormatLogFileName(i.nextSequence())))
if err != nil {
return err
}
i.activeLogFile = f
// Prepend and generate new fileset.
i.fileSet = i.fileSet.PrependLogFile(f)
// Write new manifest.
manifestSize, err := i.Manifest().Write()
if err != nil {
// TODO: Close index if write fails.
return err
}
i.manifestSize = manifestSize
return nil
}
// ForEachMeasurementName iterates over all measurement names in the index.
func (i *Partition) ForEachMeasurementName(fn func(name []byte) error) error {
fs, err := i.RetainFileSet()
if err != nil {
return err
}
defer fs.Release()
itr := fs.MeasurementIterator()
if itr == nil {
return nil
}
for e := itr.Next(); e != nil; e = itr.Next() {
if err := fn(e.Name()); err != nil {
return err
}
}
return nil
}
// MeasurementHasSeries returns true if a measurement has at least one non-tombstoned series.
func (p *Partition) MeasurementHasSeries(name []byte) (bool, error) {
fs, err := p.RetainFileSet()
if err != nil {
return false, err
}
defer fs.Release()
for _, f := range fs.files {
if f.MeasurementHasSeries(p.seriesIDSet, name) {
return true, nil
}
}
return false, nil
}
// MeasurementIterator returns an iterator over all measurement names.
func (i *Partition) MeasurementIterator() (tsdb.MeasurementIterator, error) {
fs, err := i.RetainFileSet()
if err != nil {
return nil, err
}
itr := fs.MeasurementIterator()
if itr == nil {
fs.Release()
return nil, nil
}
return newFileSetMeasurementIterator(fs, NewTSDBMeasurementIteratorAdapter(itr)), nil
}
// MeasurementExists returns true if a measurement exists.
func (i *Partition) MeasurementExists(name []byte) (bool, error) {
fs, err := i.RetainFileSet()
if err != nil {
return false, err
}
defer fs.Release()
m := fs.Measurement(name)
return m != nil && !m.Deleted(), nil
}
func (i *Partition) MeasurementNamesByRegex(re *regexp.Regexp) ([][]byte, error) {
fs, err := i.RetainFileSet()
if err != nil {
return nil, err
}
defer fs.Release()
itr := fs.MeasurementIterator()
if itr == nil {
return nil, nil
}
var a [][]byte
for e := itr.Next(); e != nil; e = itr.Next() {
if re.Match(e.Name()) {
// Clone bytes since they will be used after the fileset is released.
a = append(a, bytesutil.Clone(e.Name()))
}
}
return a, nil
}
func (i *Partition) MeasurementSeriesIDIterator(name []byte) (tsdb.SeriesIDIterator, error) {
fs, err := i.RetainFileSet()
if err != nil {
return nil, err
}
return newFileSetSeriesIDIterator(fs, fs.MeasurementSeriesIDIterator(name)), nil
}
// DropMeasurement deletes a measurement from the index. DropMeasurement does
// not remove any series from the index directly.
func (i *Partition) DropMeasurement(name []byte) error {
fs, err := i.RetainFileSet()
if err != nil {
return err
}
defer fs.Release()
// Delete all keys and values.
if kitr := fs.TagKeyIterator(name); kitr != nil {
for k := kitr.Next(); k != nil; k = kitr.Next() {
// Delete key if not already deleted.
if !k.Deleted() {
if err := func() error {
i.mu.RLock()
defer i.mu.RUnlock()
return i.activeLogFile.DeleteTagKey(name, k.Key())
}(); err != nil {
return err
}
}
// Delete each value in key.
if vitr := k.TagValueIterator(); vitr != nil {
for v := vitr.Next(); v != nil; v = vitr.Next() {
if !v.Deleted() {
if err := func() error {
i.mu.RLock()
defer i.mu.RUnlock()
return i.activeLogFile.DeleteTagValue(name, k.Key(), v.Value())
}(); err != nil {
return err
}
}
}
}
}
}
// Delete all series.
if itr := fs.MeasurementSeriesIDIterator(name); itr != nil {
defer itr.Close()
for {
elem, err := itr.Next()
if err != nil {
return err
} else if elem.SeriesID == 0 {
break
}
if err := i.activeLogFile.DeleteSeriesID(elem.SeriesID); err != nil {
return err
}
}
}
// Mark measurement as deleted.
if err := func() error {
i.mu.RLock()
defer i.mu.RUnlock()
return i.activeLogFile.DeleteMeasurement(name)
}(); err != nil {
return err
}
// Check if the log file needs to be swapped.
if err := i.CheckLogFile(); err != nil {
return err
}
return nil
}
// createSeriesListIfNotExists creates a list of series if they doesn't exist in
// bulk.
func (i *Partition) createSeriesListIfNotExists(names [][]byte, tagsSlice []models.Tags) error {
// Is there anything to do? The partition may have been sent an empty batch.
if len(names) == 0 {
return nil
} else if len(names) != len(tagsSlice) {
return fmt.Errorf("uneven batch, partition %s sent %d names and %d tags", i.id, len(names), len(tagsSlice))
}
// Maintain reference count on files in file set.
fs, err := i.RetainFileSet()
if err != nil {
return err
}
defer fs.Release()
// Ensure fileset cannot change during insert.
i.mu.RLock()
// Insert series into log file.
if err := i.activeLogFile.AddSeriesList(i.seriesIDSet, names, tagsSlice); err != nil {
i.mu.RUnlock()
return err
}
i.mu.RUnlock()
return i.CheckLogFile()
}
func (i *Partition) DropSeries(seriesID uint64) error {
// Delete series from index.
if err := i.activeLogFile.DeleteSeriesID(seriesID); err != nil {
return err
}
i.seriesIDSet.Remove(seriesID)
// Swap log file, if necessary.
return i.CheckLogFile()
}
// MeasurementsSketches returns the two sketches for the partition by merging all
// instances of the type sketch types in all the index files.
func (i *Partition) MeasurementsSketches() (estimator.Sketch, estimator.Sketch, error) {
fs, err := i.RetainFileSet()
if err != nil {
return nil, nil, err
}
defer fs.Release()
return fs.MeasurementsSketches()
}
// SeriesSketches returns the two sketches for the partition by merging all
// instances of the type sketch types in all the index files.
func (i *Partition) SeriesSketches() (estimator.Sketch, estimator.Sketch, error) {
fs, err := i.RetainFileSet()
if err != nil {
return nil, nil, err
}
defer fs.Release()
return fs.SeriesSketches()
}
// HasTagKey returns true if tag key exists.
func (i *Partition) HasTagKey(name, key []byte) (bool, error) {
fs, err := i.RetainFileSet()
if err != nil {
return false, err
}
defer fs.Release()
return fs.HasTagKey(name, key), nil
}
// HasTagValue returns true if tag value exists.
func (i *Partition) HasTagValue(name, key, value []byte) (bool, error) {
fs, err := i.RetainFileSet()
if err != nil {
return false, err
}
defer fs.Release()
return fs.HasTagValue(name, key, value), nil
}
// TagKeyIterator returns an iterator for all keys across a single measurement.
func (i *Partition) TagKeyIterator(name []byte) tsdb.TagKeyIterator {
fs, err := i.RetainFileSet()
if err != nil {
return nil // TODO(edd): this should probably return an error.
}
itr := fs.TagKeyIterator(name)
if itr == nil {
fs.Release()
return nil
}
return newFileSetTagKeyIterator(fs, NewTSDBTagKeyIteratorAdapter(itr))
}
// TagValueIterator returns an iterator for all values across a single key.
func (i *Partition) TagValueIterator(name, key []byte) tsdb.TagValueIterator {
fs, err := i.RetainFileSet()
if err != nil {
return nil // TODO(edd): this should probably return an error.
}
itr := fs.TagValueIterator(name, key)
if itr == nil {
fs.Release()
return nil
}
return newFileSetTagValueIterator(fs, NewTSDBTagValueIteratorAdapter(itr))
}
// TagKeySeriesIDIterator returns a series iterator for all values across a single key.
func (i *Partition) TagKeySeriesIDIterator(name, key []byte) tsdb.SeriesIDIterator {
fs, err := i.RetainFileSet()
if err != nil {
return nil // TODO(edd): this should probably return an error.
}
itr := fs.TagKeySeriesIDIterator(name, key)
if itr == nil {
fs.Release()
return nil
}
return newFileSetSeriesIDIterator(fs, itr)
}
// TagValueSeriesIDIterator returns a series iterator for a single key value.
func (i *Partition) TagValueSeriesIDIterator(name, key, value []byte) tsdb.SeriesIDIterator {
fs, err := i.RetainFileSet()
if err != nil {
return nil // TODO(edd): this should probably return an error.
}
itr := fs.TagValueSeriesIDIterator(name, key, value)
if itr == nil {
fs.Release()
return nil
}
return newFileSetSeriesIDIterator(fs, itr)
}
// MeasurementTagKeysByExpr extracts the tag keys wanted by the expression.
func (i *Partition) MeasurementTagKeysByExpr(name []byte, expr influxql.Expr) (map[string]struct{}, error) {
fs, err := i.RetainFileSet()
if err != nil {
return nil, err
}
defer fs.Release()
return fs.MeasurementTagKeysByExpr(name, expr)
}
// ForEachMeasurementTagKey iterates over all tag keys in a measurement.
func (i *Partition) ForEachMeasurementTagKey(name []byte, fn func(key []byte) error) error {
fs, err := i.RetainFileSet()
if err != nil {
return err
}
defer fs.Release()
itr := fs.TagKeyIterator(name)
if itr == nil {
return nil
}
for e := itr.Next(); e != nil; e = itr.Next() {
if err := fn(e.Key()); err != nil {
return err
}
}
return nil
}
// TagKeyCardinality always returns zero.
// It is not possible to determine cardinality of tags across index files.
func (i *Partition) TagKeyCardinality(name, key []byte) int {
return 0
}
func (i *Partition) SetFieldName(measurement []byte, name string) {}
func (i *Partition) RemoveShard(shardID uint64) {}
func (i *Partition) AssignShard(k string, shardID uint64) {}
// Compact requests a compaction of log files.
func (i *Partition) Compact() {
i.mu.Lock()
defer i.mu.Unlock()
i.compact()
}
func (i *Partition) DisableCompactions() {
i.mu.Lock()
defer i.mu.Unlock()
i.compactionsDisabled++
select {
case <-i.closing:
return
default:
}
if i.compactionsDisabled == 0 {
close(i.compactionInterrupt)
i.compactionInterrupt = make(chan struct{})
}
}
func (i *Partition) EnableCompactions() {
i.mu.Lock()
defer i.mu.Unlock()
// Already enabled?
if i.compactionsEnabled() {
return
}
i.compactionsDisabled--
}
func (i *Partition) compactionsEnabled() bool {
return i.compactionsDisabled == 0
}
// compact compacts continguous groups of files that are not currently compacting.
func (i *Partition) compact() {
if i.isClosing() {
return
} else if !i.compactionsEnabled() {
return
}
interrupt := i.compactionInterrupt
fs := i.retainFileSet()
defer fs.Release()
// Iterate over each level we are going to compact.
// We skip the first level (0) because it is log files and they are compacted separately.
// We skip the last level because the files have no higher level to compact into.
minLevel, maxLevel := 1, len(i.levels)-2
for level := minLevel; level <= maxLevel; level++ {
// Skip level if it is currently compacting.
if i.levelCompacting[level] {
continue
}
// Collect contiguous files from the end of the level.
files := fs.LastContiguousIndexFilesByLevel(level)
if len(files) < 2 {
continue
} else if len(files) > MaxIndexMergeCount {
files = files[len(files)-MaxIndexMergeCount:]
}
// Retain files during compaction.
IndexFiles(files).Retain()
// Mark the level as compacting.
i.levelCompacting[level] = true
// Execute in closure to save reference to the group within the loop.
func(files []*IndexFile, level int) {
// Start compacting in a separate goroutine.
i.wg.Add(1)
go func() {
defer i.wg.Done()
// Compact to a new level.
i.compactToLevel(files, level+1, interrupt)
// Ensure compaction lock for the level is released.
i.mu.Lock()
i.levelCompacting[level] = false
i.mu.Unlock()
// Check for new compactions
i.Compact()
}()
}(files, level)
}
}
// compactToLevel compacts a set of files into a new file. Replaces old files with
// compacted file on successful completion. This runs in a separate goroutine.
func (i *Partition) compactToLevel(files []*IndexFile, level int, interrupt <-chan struct{}) {
assert(len(files) >= 2, "at least two index files are required for compaction")
assert(level > 0, "cannot compact level zero")
// Build a logger for this compaction.
log, logEnd := logger.NewOperation(i.logger, "TSI level compaction", "tsi1_compact_to_level", zap.Int("tsi1_level", level))
defer logEnd()
// Check for cancellation.
select {
case <-interrupt:
log.Error("Cannot begin compaction", zap.Error(ErrCompactionInterrupted))
return
default:
}
// Files have already been retained by caller.
// Ensure files are released only once.
var once sync.Once
defer once.Do(func() { IndexFiles(files).Release() })
// Track time to compact.
start := time.Now()
// Create new index file.
path := filepath.Join(i.path, FormatIndexFileName(i.NextSequence(), level))
f, err := os.Create(path)
if err != nil {
log.Error("Cannot create compaction files", zap.Error(err))
return
}
defer f.Close()
log.Info("Performing full compaction",
zap.String("src", joinIntSlice(IndexFiles(files).IDs(), ",")),
zap.String("dst", path),
)
// Compact all index files to new index file.
lvl := i.levels[level]
n, err := IndexFiles(files).CompactTo(f, i.sfile, lvl.M, lvl.K, interrupt)
if err != nil {
log.Error("Cannot compact index files", zap.Error(err))
return
}
// Close file.
if err := f.Close(); err != nil {
log.Error("Error closing index file", zap.Error(err))
return
}
// Reopen as an index file.
file := NewIndexFile(i.sfile)
file.SetPath(path)
if err := file.Open(); err != nil {
log.Error("Cannot open new index file", zap.Error(err))
return
}
// Obtain lock to swap in index file and write manifest.
if err := func() error {
i.mu.Lock()
defer i.mu.Unlock()
// Replace previous files with new index file.
i.fileSet = i.fileSet.MustReplace(IndexFiles(files).Files(), file)
// Write new manifest.
manifestSize, err := i.Manifest().Write()
if err != nil {
// TODO: Close index if write fails.
return err
}
i.manifestSize = manifestSize
return nil
}(); err != nil {
log.Error("Cannot write manifest", zap.Error(err))
return
}
elapsed := time.Since(start)
log.Info("Full compaction complete",
zap.String("path", path),
logger.DurationLiteral("elapsed", elapsed),
zap.Int64("bytes", n),
zap.Int("kb_per_sec", int(float64(n)/elapsed.Seconds())/1024),
)
// Release old files.
once.Do(func() { IndexFiles(files).Release() })
// Close and delete all old index files.
for _, f := range files {
log.Info("Removing index file", zap.String("path", f.Path()))
if err := f.Close(); err != nil {
log.Error("Cannot close index file", zap.Error(err))
return
} else if err := os.Remove(f.Path()); err != nil {
log.Error("Cannot remove index file", zap.Error(err))
return
}
}
}
func (i *Partition) Rebuild() {}
func (i *Partition) CheckLogFile() error {
// Check log file size under read lock.
if size := func() int64 {
i.mu.RLock()
defer i.mu.RUnlock()
return i.activeLogFile.Size()
}(); size < i.MaxLogFileSize {
return nil
}
// If file size exceeded then recheck under write lock and swap files.