-
Notifications
You must be signed in to change notification settings - Fork 1.2k
/
partition.go
1948 lines (1690 loc) · 53.1 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 storage
import (
"encoding/json"
"errors"
"fmt"
"os"
"path/filepath"
"sort"
"strings"
"sync"
"sync/atomic"
"time"
"unsafe"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/cgroup"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/encoding"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/fasttime"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/fs"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/logger"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/memory"
"github.com/VictoriaMetrics/VictoriaMetrics/lib/mergeset"
)
// The maximum size of big part.
//
// This number limits the maximum time required for building big part.
// This time shouldn't exceed a few days.
const maxBigPartSize = 1e12
// The maximum number of inmemory parts in the partition.
//
// If the number of inmemory parts reaches this value, then assisted merge runs during data ingestion.
const maxInmemoryPartsPerPartition = 20
// The maximum number of small parts in the partition.
//
// If the number of small parts reaches this value, then assisted merge runs during data ingestion.
const maxSmallPartsPerPartition = 30
// Default number of parts to merge at once.
//
// This number has been obtained empirically - it gives the lowest possible overhead.
// See appendPartsToMerge tests for details.
const defaultPartsToMerge = 15
// The final number of parts to merge at once.
//
// It must be smaller than defaultPartsToMerge.
// Lower value improves select performance at the cost of increased
// write amplification.
const finalPartsToMerge = 3
// The number of shards for rawRow entries per partition.
//
// Higher number of shards reduces CPU contention and increases the max bandwidth on multi-core systems.
var rawRowsShardsPerPartition = (cgroup.AvailableCPUs() + 1) / 2
// The interval for flushing buffered rows into parts, so they become visible to search.
const pendingRowsFlushInterval = time.Second
// The interval for guaranteed flush of recently ingested data from memory to on-disk parts,
// so they survive process crash.
var dataFlushInterval = 5 * time.Second
// SetDataFlushInterval sets the interval for guaranteed flush of recently ingested data from memory to disk.
//
// The data can be flushed from memory to disk more frequently if it doesn't fit the memory limit.
//
// This function must be called before initializing the storage.
func SetDataFlushInterval(d time.Duration) {
if d > pendingRowsFlushInterval {
dataFlushInterval = d
mergeset.SetDataFlushInterval(d)
}
}
// getMaxRawRowsPerShard returns the maximum number of rows that haven't been converted into parts yet.
func getMaxRawRowsPerShard() int {
maxRawRowsPerPartitionOnce.Do(func() {
n := memory.Allowed() / rawRowsShardsPerPartition / 256 / int(unsafe.Sizeof(rawRow{}))
if n < 1e4 {
n = 1e4
}
if n > 500e3 {
n = 500e3
}
maxRawRowsPerPartition = n
})
return maxRawRowsPerPartition
}
var (
maxRawRowsPerPartition int
maxRawRowsPerPartitionOnce sync.Once
)
// partition represents a partition.
type partition struct {
// Put atomic counters to the top of struct, so they are aligned to 8 bytes on 32-bit arch.
// See https://github.com/VictoriaMetrics/VictoriaMetrics/issues/212
activeInmemoryMerges uint64
activeSmallMerges uint64
activeBigMerges uint64
inmemoryMergesCount uint64
smallMergesCount uint64
bigMergesCount uint64
inmemoryRowsMerged uint64
smallRowsMerged uint64
bigRowsMerged uint64
inmemoryRowsDeleted uint64
smallRowsDeleted uint64
bigRowsDeleted uint64
inmemoryAssistedMerges uint64
smallAssistedMerges uint64
mergeNeedFreeDiskSpace uint64
mergeIdx uint64
smallPartsPath string
bigPartsPath string
// The parent storage.
s *Storage
// Name is the name of the partition in the form YYYY_MM.
name string
// The time range for the partition. Usually this is a whole month.
tr TimeRange
// rawRows contains recently added rows that haven't been converted into parts yet.
// rawRows are periodically converted into inmemroyParts.
// rawRows aren't used in search for performance reasons.
rawRows rawRowsShards
// partsLock protects inmemoryParts, smallParts and bigParts.
partsLock sync.Mutex
// Contains inmemory parts with recently ingested data.
// It must be merged into either smallParts or bigParts to become visible to search.
inmemoryParts []*partWrapper
// Contains file-based parts with small number of items.
smallParts []*partWrapper
// Contains file-based parts with big number of items.
bigParts []*partWrapper
// This channel is used for signaling the background mergers that there are parts,
// which may need to be merged.
needMergeCh chan struct{}
stopCh chan struct{}
wg sync.WaitGroup
}
// partWrapper is a wrapper for the part.
type partWrapper struct {
// The number of references to the part.
refCount uint32
// The flag, which is set when the part must be deleted after refCount reaches zero.
// This field should be updated only after partWrapper
// was removed from the list of active parts.
mustBeDeleted uint32
// The part itself.
p *part
// non-nil if the part is inmemoryPart.
mp *inmemoryPart
// Whether the part is in merge now.
isInMerge bool
// The deadline when in-memory part must be flushed to disk.
flushToDiskDeadline time.Time
}
func (pw *partWrapper) incRef() {
atomic.AddUint32(&pw.refCount, 1)
}
func (pw *partWrapper) decRef() {
n := atomic.AddUint32(&pw.refCount, ^uint32(0))
if int32(n) < 0 {
logger.Panicf("BUG: pw.refCount must be bigger than 0; got %d", int32(n))
}
if n > 0 {
return
}
deletePath := ""
if pw.mp == nil && atomic.LoadUint32(&pw.mustBeDeleted) != 0 {
deletePath = pw.p.path
}
if pw.mp != nil {
putInmemoryPart(pw.mp)
pw.mp = nil
}
pw.p.MustClose()
pw.p = nil
if deletePath != "" {
fs.MustRemoveAll(deletePath)
}
}
// mustCreatePartition creates new partition for the given timestamp and the given paths
// to small and big partitions.
func mustCreatePartition(timestamp int64, smallPartitionsPath, bigPartitionsPath string, s *Storage) *partition {
name := timestampToPartitionName(timestamp)
smallPartsPath := filepath.Join(filepath.Clean(smallPartitionsPath), name)
bigPartsPath := filepath.Join(filepath.Clean(bigPartitionsPath), name)
logger.Infof("creating a partition %q with smallPartsPath=%q, bigPartsPath=%q", name, smallPartsPath, bigPartsPath)
fs.MustMkdirFailIfExist(smallPartsPath)
fs.MustMkdirFailIfExist(bigPartsPath)
pt := newPartition(name, smallPartsPath, bigPartsPath, s)
pt.tr.fromPartitionTimestamp(timestamp)
pt.startBackgroundWorkers()
logger.Infof("partition %q has been created", name)
return pt
}
func (pt *partition) startBackgroundWorkers() {
pt.startMergeWorkers()
pt.startInmemoryPartsFlusher()
pt.startPendingRowsFlusher()
pt.startStalePartsRemover()
}
// Drop drops all the data on the storage for the given pt.
//
// The pt must be detached from table before calling pt.Drop.
func (pt *partition) Drop() {
logger.Infof("dropping partition %q at smallPartsPath=%q, bigPartsPath=%q", pt.name, pt.smallPartsPath, pt.bigPartsPath)
fs.MustRemoveDirAtomic(pt.smallPartsPath)
fs.MustRemoveDirAtomic(pt.bigPartsPath)
logger.Infof("partition %q has been dropped", pt.name)
}
// mustOpenPartition opens the existing partition from the given paths.
func mustOpenPartition(smallPartsPath, bigPartsPath string, s *Storage) *partition {
smallPartsPath = filepath.Clean(smallPartsPath)
bigPartsPath = filepath.Clean(bigPartsPath)
name := filepath.Base(smallPartsPath)
if !strings.HasSuffix(bigPartsPath, name) {
logger.Panicf("FATAL: patititon name in bigPartsPath %q doesn't match smallPartsPath %q; want %q", bigPartsPath, smallPartsPath, name)
}
partNamesSmall, partNamesBig := mustReadPartNames(smallPartsPath, bigPartsPath)
smallParts := mustOpenParts(smallPartsPath, partNamesSmall)
bigParts := mustOpenParts(bigPartsPath, partNamesBig)
partNamesPath := filepath.Join(smallPartsPath, partsFilename)
if !fs.IsPathExist(partNamesPath) {
// Create parts.json file if it doesn't exist yet.
// This should protect from possible carshloops just after the migration from versions below v1.90.0
// See https://github.com/VictoriaMetrics/VictoriaMetrics/issues/4336
mustWritePartNames(smallParts, bigParts, smallPartsPath)
}
pt := newPartition(name, smallPartsPath, bigPartsPath, s)
pt.smallParts = smallParts
pt.bigParts = bigParts
if err := pt.tr.fromPartitionName(name); err != nil {
logger.Panicf("FATAL: cannot obtain partition time range from smallPartsPath %q: %s", smallPartsPath, err)
}
pt.startBackgroundWorkers()
// Wake up a single background merger, so it could start merging parts if needed.
pt.notifyBackgroundMergers()
return pt
}
func newPartition(name, smallPartsPath, bigPartsPath string, s *Storage) *partition {
p := &partition{
name: name,
smallPartsPath: smallPartsPath,
bigPartsPath: bigPartsPath,
s: s,
mergeIdx: uint64(time.Now().UnixNano()),
needMergeCh: make(chan struct{}, cgroup.AvailableCPUs()),
stopCh: make(chan struct{}),
}
p.rawRows.init()
return p
}
// partitionMetrics contains essential metrics for the partition.
type partitionMetrics struct {
PendingRows uint64
IndexBlocksCacheSize uint64
IndexBlocksCacheSizeBytes uint64
IndexBlocksCacheSizeMaxBytes uint64
IndexBlocksCacheRequests uint64
IndexBlocksCacheMisses uint64
InmemorySizeBytes uint64
SmallSizeBytes uint64
BigSizeBytes uint64
InmemoryRowsCount uint64
SmallRowsCount uint64
BigRowsCount uint64
InmemoryBlocksCount uint64
SmallBlocksCount uint64
BigBlocksCount uint64
InmemoryPartsCount uint64
SmallPartsCount uint64
BigPartsCount uint64
ActiveInmemoryMerges uint64
ActiveSmallMerges uint64
ActiveBigMerges uint64
InmemoryMergesCount uint64
SmallMergesCount uint64
BigMergesCount uint64
InmemoryRowsMerged uint64
SmallRowsMerged uint64
BigRowsMerged uint64
InmemoryRowsDeleted uint64
SmallRowsDeleted uint64
BigRowsDeleted uint64
InmemoryPartsRefCount uint64
SmallPartsRefCount uint64
BigPartsRefCount uint64
InmemoryAssistedMerges uint64
SmallAssistedMerges uint64
MergeNeedFreeDiskSpace uint64
}
// TotalRowsCount returns total number of rows in tm.
func (pm *partitionMetrics) TotalRowsCount() uint64 {
return pm.PendingRows + pm.InmemoryRowsCount + pm.SmallRowsCount + pm.BigRowsCount
}
// UpdateMetrics updates m with metrics from pt.
func (pt *partition) UpdateMetrics(m *partitionMetrics) {
m.PendingRows += uint64(pt.rawRows.Len())
pt.partsLock.Lock()
for _, pw := range pt.inmemoryParts {
p := pw.p
m.InmemoryRowsCount += p.ph.RowsCount
m.InmemoryBlocksCount += p.ph.BlocksCount
m.InmemorySizeBytes += p.size
m.InmemoryPartsRefCount += uint64(atomic.LoadUint32(&pw.refCount))
}
for _, pw := range pt.smallParts {
p := pw.p
m.SmallRowsCount += p.ph.RowsCount
m.SmallBlocksCount += p.ph.BlocksCount
m.SmallSizeBytes += p.size
m.SmallPartsRefCount += uint64(atomic.LoadUint32(&pw.refCount))
}
for _, pw := range pt.bigParts {
p := pw.p
m.BigRowsCount += p.ph.RowsCount
m.BigBlocksCount += p.ph.BlocksCount
m.BigSizeBytes += p.size
m.BigPartsRefCount += uint64(atomic.LoadUint32(&pw.refCount))
}
m.InmemoryPartsCount += uint64(len(pt.inmemoryParts))
m.SmallPartsCount += uint64(len(pt.smallParts))
m.BigPartsCount += uint64(len(pt.bigParts))
pt.partsLock.Unlock()
m.IndexBlocksCacheSize = uint64(ibCache.Len())
m.IndexBlocksCacheSizeBytes = uint64(ibCache.SizeBytes())
m.IndexBlocksCacheSizeMaxBytes = uint64(ibCache.SizeMaxBytes())
m.IndexBlocksCacheRequests = ibCache.Requests()
m.IndexBlocksCacheMisses = ibCache.Misses()
m.ActiveInmemoryMerges += atomic.LoadUint64(&pt.activeInmemoryMerges)
m.ActiveSmallMerges += atomic.LoadUint64(&pt.activeSmallMerges)
m.ActiveBigMerges += atomic.LoadUint64(&pt.activeBigMerges)
m.InmemoryMergesCount += atomic.LoadUint64(&pt.inmemoryMergesCount)
m.SmallMergesCount += atomic.LoadUint64(&pt.smallMergesCount)
m.BigMergesCount += atomic.LoadUint64(&pt.bigMergesCount)
m.InmemoryRowsMerged += atomic.LoadUint64(&pt.inmemoryRowsMerged)
m.SmallRowsMerged += atomic.LoadUint64(&pt.smallRowsMerged)
m.BigRowsMerged += atomic.LoadUint64(&pt.bigRowsMerged)
m.InmemoryRowsDeleted += atomic.LoadUint64(&pt.inmemoryRowsDeleted)
m.SmallRowsDeleted += atomic.LoadUint64(&pt.smallRowsDeleted)
m.BigRowsDeleted += atomic.LoadUint64(&pt.bigRowsDeleted)
m.InmemoryAssistedMerges += atomic.LoadUint64(&pt.inmemoryAssistedMerges)
m.SmallAssistedMerges += atomic.LoadUint64(&pt.smallAssistedMerges)
m.MergeNeedFreeDiskSpace += atomic.LoadUint64(&pt.mergeNeedFreeDiskSpace)
}
// AddRows adds the given rows to the partition pt.
//
// All the rows must fit the partition by timestamp range
// and must have valid PrecisionBits.
func (pt *partition) AddRows(rows []rawRow) {
if len(rows) == 0 {
return
}
if isDebug {
// Validate all the rows.
for i := range rows {
r := &rows[i]
if !pt.HasTimestamp(r.Timestamp) {
logger.Panicf("BUG: row %+v has Timestamp outside partition %q range %+v", r, pt.smallPartsPath, &pt.tr)
}
if err := encoding.CheckPrecisionBits(r.PrecisionBits); err != nil {
logger.Panicf("BUG: row %+v has invalid PrecisionBits: %s", r, err)
}
}
}
pt.rawRows.addRows(pt, rows)
}
var isDebug = false
type rawRowsShards struct {
shardIdx uint32
// Shards reduce lock contention when adding rows on multi-CPU systems.
shards []rawRowsShard
}
func (rrss *rawRowsShards) init() {
rrss.shards = make([]rawRowsShard, rawRowsShardsPerPartition)
}
func (rrss *rawRowsShards) addRows(pt *partition, rows []rawRow) {
shards := rrss.shards
shardsLen := uint32(len(shards))
for len(rows) > 0 {
n := atomic.AddUint32(&rrss.shardIdx, 1)
idx := n % shardsLen
rows = shards[idx].addRows(pt, rows)
}
}
func (rrss *rawRowsShards) Len() int {
n := 0
for i := range rrss.shards[:] {
n += rrss.shards[i].Len()
}
return n
}
type rawRowsShardNopad struct {
// Put lastFlushTime to the top in order to avoid unaligned memory access on 32-bit architectures
lastFlushTime uint64
mu sync.Mutex
rows []rawRow
}
type rawRowsShard struct {
rawRowsShardNopad
// The padding prevents false sharing on widespread platforms with
// 128 mod (cache line size) = 0 .
_ [128 - unsafe.Sizeof(rawRowsShardNopad{})%128]byte
}
func (rrs *rawRowsShard) Len() int {
rrs.mu.Lock()
n := len(rrs.rows)
rrs.mu.Unlock()
return n
}
func (rrs *rawRowsShard) addRows(pt *partition, rows []rawRow) []rawRow {
var rrb *rawRowsBlock
rrs.mu.Lock()
if cap(rrs.rows) == 0 {
rrs.rows = newRawRows()
}
n := copy(rrs.rows[len(rrs.rows):cap(rrs.rows)], rows)
rrs.rows = rrs.rows[:len(rrs.rows)+n]
rows = rows[n:]
if len(rows) > 0 {
rrb = getRawRowsBlock()
rrb.rows, rrs.rows = rrs.rows, rrb.rows
n = copy(rrs.rows[:cap(rrs.rows)], rows)
rrs.rows = rrs.rows[:n]
rows = rows[n:]
atomic.StoreUint64(&rrs.lastFlushTime, fasttime.UnixTimestamp())
}
rrs.mu.Unlock()
if rrb != nil {
pt.flushRowsToParts(rrb.rows)
putRawRowsBlock(rrb)
// Run assisted merges if needed.
flushConcurrencyCh <- struct{}{}
pt.assistedMergeForInmemoryParts()
pt.assistedMergeForSmallParts()
// There is no need in assisted merges for big parts,
// since the bottleneck is possible only at inmemory and small parts.
<-flushConcurrencyCh
}
return rows
}
type rawRowsBlock struct {
rows []rawRow
}
func newRawRows() []rawRow {
n := getMaxRawRowsPerShard()
return make([]rawRow, 0, n)
}
func getRawRowsBlock() *rawRowsBlock {
v := rawRowsBlockPool.Get()
if v == nil {
return &rawRowsBlock{
rows: newRawRows(),
}
}
return v.(*rawRowsBlock)
}
func putRawRowsBlock(rrb *rawRowsBlock) {
rrb.rows = rrb.rows[:0]
rawRowsBlockPool.Put(rrb)
}
var rawRowsBlockPool sync.Pool
func (pt *partition) flushRowsToParts(rows []rawRow) {
if len(rows) == 0 {
return
}
maxRows := getMaxRawRowsPerShard()
var pwsLock sync.Mutex
pws := make([]*partWrapper, 0, (len(rows)+maxRows-1)/maxRows)
wg := getWaitGroup()
for len(rows) > 0 {
n := maxRows
if n > len(rows) {
n = len(rows)
}
wg.Add(1)
flushConcurrencyCh <- struct{}{}
go func(rowsChunk []rawRow) {
defer func() {
<-flushConcurrencyCh
wg.Done()
}()
pw := pt.createInmemoryPart(rowsChunk)
if pw == nil {
return
}
pwsLock.Lock()
pws = append(pws, pw)
pwsLock.Unlock()
}(rows[:n])
rows = rows[n:]
}
wg.Wait()
putWaitGroup(wg)
pt.partsLock.Lock()
pt.inmemoryParts = append(pt.inmemoryParts, pws...)
for range pws {
if !pt.notifyBackgroundMergers() {
break
}
}
pt.partsLock.Unlock()
}
func (pt *partition) notifyBackgroundMergers() bool {
select {
case pt.needMergeCh <- struct{}{}:
return true
default:
return false
}
}
var flushConcurrencyLimit = func() int {
n := cgroup.AvailableCPUs()
if n < 3 {
// Allow at least 3 concurrent flushers on systems with a single CPU core
// in order to guarantee that in-memory data flushes and background merges can be continued
// when a single flusher is busy with the long merge of big parts,
// while another flusher is busy with the long merge of small parts.
n = 3
}
return n
}()
var flushConcurrencyCh = make(chan struct{}, flushConcurrencyLimit)
func needAssistedMerge(pws []*partWrapper, maxParts int) bool {
if len(pws) < maxParts {
return false
}
return getNotInMergePartsCount(pws) >= defaultPartsToMerge
}
func (pt *partition) assistedMergeForInmemoryParts() {
for {
pt.partsLock.Lock()
needMerge := needAssistedMerge(pt.inmemoryParts, maxInmemoryPartsPerPartition)
pt.partsLock.Unlock()
if !needMerge {
return
}
atomic.AddUint64(&pt.inmemoryAssistedMerges, 1)
err := pt.mergeInmemoryParts()
if err == nil {
continue
}
if errors.Is(err, errNothingToMerge) || errors.Is(err, errForciblyStopped) {
return
}
logger.Panicf("FATAL: cannot merge inmemory parts: %s", err)
}
}
func (pt *partition) assistedMergeForSmallParts() {
for {
pt.partsLock.Lock()
needMerge := needAssistedMerge(pt.smallParts, maxSmallPartsPerPartition)
pt.partsLock.Unlock()
if !needMerge {
return
}
atomic.AddUint64(&pt.smallAssistedMerges, 1)
err := pt.mergeExistingParts(false)
if err == nil {
continue
}
if errors.Is(err, errNothingToMerge) || errors.Is(err, errForciblyStopped) || errors.Is(err, errReadOnlyMode) {
return
}
logger.Panicf("FATAL: cannot merge small parts: %s", err)
}
}
func getNotInMergePartsCount(pws []*partWrapper) int {
n := 0
for _, pw := range pws {
if !pw.isInMerge {
n++
}
}
return n
}
func getWaitGroup() *sync.WaitGroup {
v := wgPool.Get()
if v == nil {
return &sync.WaitGroup{}
}
return v.(*sync.WaitGroup)
}
func putWaitGroup(wg *sync.WaitGroup) {
wgPool.Put(wg)
}
var wgPool sync.Pool
func (pt *partition) createInmemoryPart(rows []rawRow) *partWrapper {
if len(rows) == 0 {
return nil
}
mp := getInmemoryPart()
mp.InitFromRows(rows)
// Make sure the part may be added.
if mp.ph.MinTimestamp > mp.ph.MaxTimestamp {
logger.Panicf("BUG: the part %q cannot be added to partition %q because its MinTimestamp exceeds MaxTimestamp; %d vs %d",
&mp.ph, pt.smallPartsPath, mp.ph.MinTimestamp, mp.ph.MaxTimestamp)
}
if mp.ph.MinTimestamp < pt.tr.MinTimestamp {
logger.Panicf("BUG: the part %q cannot be added to partition %q because of too small MinTimestamp; got %d; want at least %d",
&mp.ph, pt.smallPartsPath, mp.ph.MinTimestamp, pt.tr.MinTimestamp)
}
if mp.ph.MaxTimestamp > pt.tr.MaxTimestamp {
logger.Panicf("BUG: the part %q cannot be added to partition %q because of too big MaxTimestamp; got %d; want at least %d",
&mp.ph, pt.smallPartsPath, mp.ph.MaxTimestamp, pt.tr.MaxTimestamp)
}
flushToDiskDeadline := time.Now().Add(dataFlushInterval)
return newPartWrapperFromInmemoryPart(mp, flushToDiskDeadline)
}
func newPartWrapperFromInmemoryPart(mp *inmemoryPart, flushToDiskDeadline time.Time) *partWrapper {
p := mp.NewPart()
pw := &partWrapper{
p: p,
mp: mp,
refCount: 1,
flushToDiskDeadline: flushToDiskDeadline,
}
return pw
}
// HasTimestamp returns true if the pt contains the given timestamp.
func (pt *partition) HasTimestamp(timestamp int64) bool {
return timestamp >= pt.tr.MinTimestamp && timestamp <= pt.tr.MaxTimestamp
}
// GetParts appends parts snapshot to dst and returns it.
//
// The appended parts must be released with PutParts.
func (pt *partition) GetParts(dst []*partWrapper, addInMemory bool) []*partWrapper {
pt.partsLock.Lock()
if addInMemory {
incRefForParts(pt.inmemoryParts)
dst = append(dst, pt.inmemoryParts...)
}
incRefForParts(pt.smallParts)
dst = append(dst, pt.smallParts...)
incRefForParts(pt.bigParts)
dst = append(dst, pt.bigParts...)
pt.partsLock.Unlock()
return dst
}
// PutParts releases the given pws obtained via GetParts.
func (pt *partition) PutParts(pws []*partWrapper) {
for _, pw := range pws {
pw.decRef()
}
}
func incRefForParts(pws []*partWrapper) {
for _, pw := range pws {
pw.incRef()
}
}
// MustClose closes the pt, so the app may safely exit.
//
// The pt must be detached from table before calling pt.MustClose.
func (pt *partition) MustClose() {
close(pt.stopCh)
// Waiting for service workers to stop
pt.wg.Wait()
pt.flushInmemoryRows()
// Remove references from inmemoryParts, smallParts and bigParts, so they may be eventually closed
// after all the searches are done.
pt.partsLock.Lock()
inmemoryParts := pt.inmemoryParts
smallParts := pt.smallParts
bigParts := pt.bigParts
pt.inmemoryParts = nil
pt.smallParts = nil
pt.bigParts = nil
pt.partsLock.Unlock()
for _, pw := range inmemoryParts {
pw.decRef()
}
for _, pw := range smallParts {
pw.decRef()
}
for _, pw := range bigParts {
pw.decRef()
}
}
func (pt *partition) startInmemoryPartsFlusher() {
pt.wg.Add(1)
go func() {
pt.inmemoryPartsFlusher()
pt.wg.Done()
}()
}
func (pt *partition) startPendingRowsFlusher() {
pt.wg.Add(1)
go func() {
pt.pendingRowsFlusher()
pt.wg.Done()
}()
}
func (pt *partition) inmemoryPartsFlusher() {
ticker := time.NewTicker(dataFlushInterval)
defer ticker.Stop()
for {
select {
case <-pt.stopCh:
return
case <-ticker.C:
pt.flushInmemoryParts(false)
}
}
}
func (pt *partition) pendingRowsFlusher() {
ticker := time.NewTicker(pendingRowsFlushInterval)
defer ticker.Stop()
var rows []rawRow
for {
select {
case <-pt.stopCh:
return
case <-ticker.C:
rows = pt.flushPendingRows(rows[:0], false)
}
}
}
func (pt *partition) flushPendingRows(dst []rawRow, isFinal bool) []rawRow {
return pt.rawRows.flush(pt, dst, isFinal)
}
func (pt *partition) flushInmemoryRows() {
pt.rawRows.flush(pt, nil, true)
pt.flushInmemoryParts(true)
}
func (pt *partition) flushInmemoryParts(isFinal bool) {
currentTime := time.Now()
var pws []*partWrapper
pt.partsLock.Lock()
for _, pw := range pt.inmemoryParts {
if !pw.isInMerge && (isFinal || pw.flushToDiskDeadline.Before(currentTime)) {
pw.isInMerge = true
pws = append(pws, pw)
}
}
pt.partsLock.Unlock()
if err := pt.mergePartsOptimal(pws, nil); err != nil {
logger.Panicf("FATAL: cannot merge in-memory parts: %s", err)
}
}
func (rrss *rawRowsShards) flush(pt *partition, dst []rawRow, isFinal bool) []rawRow {
for i := range rrss.shards {
dst = rrss.shards[i].appendRawRowsToFlush(dst, isFinal)
}
pt.flushRowsToParts(dst)
return dst
}
func (rrs *rawRowsShard) appendRawRowsToFlush(dst []rawRow, isFinal bool) []rawRow {
currentTime := fasttime.UnixTimestamp()
flushSeconds := int64(pendingRowsFlushInterval.Seconds())
if flushSeconds <= 0 {
flushSeconds = 1
}
lastFlushTime := atomic.LoadUint64(&rrs.lastFlushTime)
if !isFinal && currentTime < lastFlushTime+uint64(flushSeconds) {
// Fast path - nothing to flush
return dst
}
// Slow path - move rrs.rows to dst.
rrs.mu.Lock()
dst = append(dst, rrs.rows...)
rrs.rows = rrs.rows[:0]
atomic.StoreUint64(&rrs.lastFlushTime, currentTime)
rrs.mu.Unlock()
return dst
}
func (pt *partition) mergePartsOptimal(pws []*partWrapper, stopCh <-chan struct{}) error {
sortPartsForOptimalMerge(pws)
for len(pws) > 0 {
n := defaultPartsToMerge
if n > len(pws) {
n = len(pws)
}
pwsChunk := pws[:n]
pws = pws[n:]
err := pt.mergeParts(pwsChunk, stopCh, true)
if err == nil {
continue
}
pt.releasePartsToMerge(pws)
if errors.Is(err, errForciblyStopped) {
return nil
}
return fmt.Errorf("cannot merge parts optimally: %w", err)
}
return nil
}
// ForceMergeAllParts runs merge for all the parts in pt.
func (pt *partition) ForceMergeAllParts() error {
pws := pt.getAllPartsForMerge()
if len(pws) == 0 {
// Nothing to merge.
return nil
}
// Check whether there is enough disk space for merging pws.
newPartSize := getPartsSize(pws)
maxOutBytes := fs.MustGetFreeSpace(pt.bigPartsPath)
if newPartSize > maxOutBytes {
freeSpaceNeededBytes := newPartSize - maxOutBytes
forceMergeLogger.Warnf("cannot initiate force merge for the partition %s; additional space needed: %d bytes", pt.name, freeSpaceNeededBytes)
pt.releasePartsToMerge(pws)
return nil
}
// If len(pws) == 1, then the merge must run anyway.
// This allows applying the configured retention, removing the deleted series
// and performing de-duplication if needed.
if err := pt.mergePartsOptimal(pws, pt.stopCh); err != nil {
return fmt.Errorf("cannot force merge %d parts from partition %q: %w", len(pws), pt.name, err)
}
return nil
}
var forceMergeLogger = logger.WithThrottler("forceMerge", time.Minute)
func (pt *partition) getAllPartsForMerge() []*partWrapper {
var pws []*partWrapper
pt.partsLock.Lock()
if !hasActiveMerges(pt.inmemoryParts) && !hasActiveMerges(pt.smallParts) && !hasActiveMerges(pt.bigParts) {
pws = appendAllPartsForMerge(pws, pt.inmemoryParts)
pws = appendAllPartsForMerge(pws, pt.smallParts)
pws = appendAllPartsForMerge(pws, pt.bigParts)
}
pt.partsLock.Unlock()
return pws
}
func appendAllPartsForMerge(dst, src []*partWrapper) []*partWrapper {
for _, pw := range src {
if pw.isInMerge {
logger.Panicf("BUG: part %q is already in merge", pw.p.path)
}
pw.isInMerge = true
dst = append(dst, pw)
}
return dst
}
func hasActiveMerges(pws []*partWrapper) bool {
for _, pw := range pws {
if pw.isInMerge {
return true
}
}
return false
}
var mergeWorkersLimitCh = make(chan struct{}, getDefaultMergeConcurrency(16))
func getDefaultMergeConcurrency(max int) int {
v := (cgroup.AvailableCPUs() + 1) / 2
if v > max {
v = max
}