-
Notifications
You must be signed in to change notification settings - Fork 857
Expand file tree
/
Copy pathcompactor.go
More file actions
1327 lines (1141 loc) · 57.6 KB
/
compactor.go
File metadata and controls
1327 lines (1141 loc) · 57.6 KB
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 compactor
import (
"context"
crypto_rand "crypto/rand"
"flag"
"fmt"
"hash/fnv"
"math/rand"
"os"
"path/filepath"
"slices"
"strings"
"time"
"github.com/go-kit/log"
"github.com/go-kit/log/level"
"github.com/oklog/ulid/v2"
"github.com/pkg/errors"
"github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/client_golang/prometheus/promauto"
"github.com/prometheus/prometheus/tsdb"
"github.com/thanos-io/objstore"
"github.com/thanos-io/thanos/pkg/block"
"github.com/thanos-io/thanos/pkg/block/metadata"
"github.com/thanos-io/thanos/pkg/compact"
"github.com/thanos-io/thanos/pkg/compact/downsample"
"github.com/thanos-io/thanos/pkg/errutil"
"github.com/thanos-io/thanos/pkg/extprom"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/status"
"github.com/cortexproject/cortex/pkg/ring"
"github.com/cortexproject/cortex/pkg/storage/bucket"
cortex_tsdb "github.com/cortexproject/cortex/pkg/storage/tsdb"
"github.com/cortexproject/cortex/pkg/storage/tsdb/bucketindex"
"github.com/cortexproject/cortex/pkg/util"
"github.com/cortexproject/cortex/pkg/util/backoff"
"github.com/cortexproject/cortex/pkg/util/flagext"
util_log "github.com/cortexproject/cortex/pkg/util/log"
"github.com/cortexproject/cortex/pkg/util/services"
"github.com/cortexproject/cortex/pkg/util/users"
"github.com/cortexproject/cortex/pkg/util/validation"
)
const (
// ringKey is the key under which we store the compactors ring in the KVStore.
ringKey = "compactor"
blocksMarkedForDeletionName = "cortex_compactor_blocks_marked_for_deletion_total"
blocksMarkedForDeletionHelp = "Total number of blocks marked for deletion in compactor."
)
var (
errInvalidBlockRanges = "compactor block range periods should be divisible by the previous one, but %s is not divisible by %s"
RingOp = ring.NewOp([]ring.InstanceState{ring.ACTIVE}, nil)
supportedShardingStrategies = []string{util.ShardingStrategyDefault, util.ShardingStrategyShuffle}
errInvalidShardingStrategy = errors.New("invalid sharding strategy")
errInvalidTenantShardSize = errors.New("invalid tenant shard size, the value must be greater than 0")
supportedCompactionStrategies = []string{util.CompactionStrategyDefault, util.CompactionStrategyPartitioning}
errInvalidCompactionStrategy = errors.New("invalid compaction strategy")
errInvalidCompactionStrategyPartitioning = errors.New("compaction strategy partitioning can only be enabled when shuffle sharding is enabled")
DefaultBlocksGrouperFactory = func(ctx context.Context, cfg Config, bkt objstore.InstrumentedBucket, logger log.Logger, blocksMarkedForNoCompaction prometheus.Counter, _ prometheus.Counter, _ prometheus.Counter, syncerMetrics *compact.SyncerMetrics, compactorMetrics *compactorMetrics, _ *ring.Ring, _ *ring.Lifecycler, _ Limits, _ string, _ *compact.GatherNoCompactionMarkFilter, _ int) compact.Grouper {
return compact.NewDefaultGrouperWithMetrics(
logger,
bkt,
cfg.AcceptMalformedIndex,
true, // Enable vertical compaction
compactorMetrics.compactions,
compactorMetrics.compactionRunsStarted,
compactorMetrics.compactionRunsCompleted,
compactorMetrics.compactionFailures,
compactorMetrics.verticalCompactions,
syncerMetrics.BlocksMarkedForDeletion,
syncerMetrics.GarbageCollectedBlocks,
blocksMarkedForNoCompaction,
metadata.NoneFunc,
cfg.BlockFilesConcurrency,
cfg.BlocksFetchConcurrency)
}
ShuffleShardingGrouperFactory = func(ctx context.Context, cfg Config, bkt objstore.InstrumentedBucket, logger log.Logger, blocksMarkedForNoCompaction prometheus.Counter, blockVisitMarkerReadFailed prometheus.Counter, blockVisitMarkerWriteFailed prometheus.Counter, syncerMetrics *compact.SyncerMetrics, compactorMetrics *compactorMetrics, ring *ring.Ring, ringLifecycle *ring.Lifecycler, limits Limits, userID string, noCompactionMarkFilter *compact.GatherNoCompactionMarkFilter, ingestionReplicationFactor int) compact.Grouper {
if cfg.CompactionStrategy == util.CompactionStrategyPartitioning {
return NewPartitionCompactionGrouper(
ctx,
logger,
bkt,
cfg.AcceptMalformedIndex,
true, // Enable vertical compaction
blocksMarkedForNoCompaction,
syncerMetrics,
compactorMetrics,
metadata.NoneFunc,
cfg,
ring,
ringLifecycle.Addr,
ringLifecycle.ID,
limits,
userID,
cfg.BlockFilesConcurrency,
cfg.BlocksFetchConcurrency,
cfg.CompactionConcurrency,
true,
cfg.CompactionVisitMarkerTimeout,
noCompactionMarkFilter.NoCompactMarkedBlocks,
ingestionReplicationFactor)
} else {
return NewShuffleShardingGrouper(
ctx,
logger,
bkt,
cfg.AcceptMalformedIndex,
true, // Enable vertical compaction
blocksMarkedForNoCompaction,
metadata.NoneFunc,
syncerMetrics,
compactorMetrics,
cfg,
ring,
ringLifecycle.Addr,
ringLifecycle.ID,
limits,
userID,
cfg.BlockFilesConcurrency,
cfg.BlocksFetchConcurrency,
cfg.CompactionConcurrency,
cfg.CompactionVisitMarkerTimeout,
blockVisitMarkerReadFailed,
blockVisitMarkerWriteFailed,
noCompactionMarkFilter.NoCompactMarkedBlocks)
}
}
DefaultBlocksCompactorFactory = func(ctx context.Context, cfg Config, logger log.Logger, reg prometheus.Registerer) (compact.Compactor, PlannerFactory, error) {
compactor, err := tsdb.NewLeveledCompactor(ctx, reg, util_log.GoKitLogToSlog(logger), cfg.BlockRanges.ToMilliseconds(), downsample.NewPool(), nil)
if err != nil {
return nil, nil, err
}
plannerFactory := func(ctx context.Context, cancel context.CancelFunc, bkt objstore.InstrumentedBucket, logger log.Logger, cfg Config, noCompactionMarkFilter *compact.GatherNoCompactionMarkFilter, ringLifecycle *ring.Lifecycler, _ string, _ prometheus.Counter, _ prometheus.Counter, _ *compactorMetrics) compact.Planner {
return compact.NewPlanner(logger, cfg.BlockRanges.ToMilliseconds(), noCompactionMarkFilter)
}
return compactor, plannerFactory, nil
}
ShuffleShardingBlocksCompactorFactory = func(ctx context.Context, cfg Config, logger log.Logger, reg prometheus.Registerer) (compact.Compactor, PlannerFactory, error) {
compactor, err := tsdb.NewLeveledCompactor(ctx, reg, util_log.GoKitLogToSlog(logger), cfg.BlockRanges.ToMilliseconds(), downsample.NewPool(), nil)
if err != nil {
return nil, nil, err
}
plannerFactory := func(ctx context.Context, cancel context.CancelFunc, bkt objstore.InstrumentedBucket, logger log.Logger, cfg Config, noCompactionMarkFilter *compact.GatherNoCompactionMarkFilter, ringLifecycle *ring.Lifecycler, userID string, blockVisitMarkerReadFailed prometheus.Counter, blockVisitMarkerWriteFailed prometheus.Counter, compactorMetrics *compactorMetrics) compact.Planner {
if cfg.CompactionStrategy == util.CompactionStrategyPartitioning {
return NewPartitionCompactionPlanner(ctx, cancel, bkt, logger, cfg.BlockRanges.ToMilliseconds(), noCompactionMarkFilter.NoCompactMarkedBlocks, ringLifecycle.ID, userID, cfg.ShardingPlannerDelay, cfg.CompactionVisitMarkerTimeout, cfg.CompactionVisitMarkerFileUpdateInterval, compactorMetrics)
} else {
return NewShuffleShardingPlanner(ctx, bkt, logger, cfg.BlockRanges.ToMilliseconds(), noCompactionMarkFilter.NoCompactMarkedBlocks, ringLifecycle.ID, cfg.CompactionVisitMarkerTimeout, cfg.CompactionVisitMarkerFileUpdateInterval, blockVisitMarkerReadFailed, blockVisitMarkerWriteFailed)
}
}
return compactor, plannerFactory, nil
}
DefaultBlockDeletableCheckerFactory = func(_ context.Context, _ objstore.InstrumentedBucket, _ log.Logger) compact.BlockDeletableChecker {
return compact.DefaultBlockDeletableChecker{}
}
PartitionCompactionBlockDeletableCheckerFactory = func(ctx context.Context, bkt objstore.InstrumentedBucket, logger log.Logger) compact.BlockDeletableChecker {
return NewPartitionCompactionBlockDeletableChecker()
}
DefaultCompactionLifecycleCallbackFactory = func(_ context.Context, _ objstore.InstrumentedBucket, _ log.Logger, _ int, _ string, _ string, _ *compactorMetrics) compact.CompactionLifecycleCallback {
return compact.DefaultCompactionLifecycleCallback{}
}
ShardedCompactionLifecycleCallbackFactory = func(ctx context.Context, userBucket objstore.InstrumentedBucket, logger log.Logger, metaSyncConcurrency int, compactDir string, userID string, compactorMetrics *compactorMetrics) compact.CompactionLifecycleCallback {
return NewShardedCompactionLifecycleCallback(
ctx,
userBucket,
logger,
metaSyncConcurrency,
compactDir,
userID,
compactorMetrics,
)
}
)
// BlocksGrouperFactory builds and returns the grouper to use to compact a tenant's blocks.
type BlocksGrouperFactory func(
ctx context.Context,
cfg Config,
bkt objstore.InstrumentedBucket,
logger log.Logger,
blocksMarkedForNoCompact prometheus.Counter,
blockVisitMarkerReadFailed prometheus.Counter,
blockVisitMarkerWriteFailed prometheus.Counter,
syncerMetrics *compact.SyncerMetrics,
compactorMetrics *compactorMetrics,
ring *ring.Ring,
ringLifecycler *ring.Lifecycler,
limit Limits,
userID string,
noCompactionMarkFilter *compact.GatherNoCompactionMarkFilter,
ingestionReplicationFactor int,
) compact.Grouper
// BlocksCompactorFactory builds and returns the compactor and planner to use to compact a tenant's blocks.
type BlocksCompactorFactory func(
ctx context.Context,
cfg Config,
logger log.Logger,
reg prometheus.Registerer,
) (compact.Compactor, PlannerFactory, error)
type PlannerFactory func(
ctx context.Context,
cancel context.CancelFunc,
bkt objstore.InstrumentedBucket,
logger log.Logger,
cfg Config,
noCompactionMarkFilter *compact.GatherNoCompactionMarkFilter,
ringLifecycle *ring.Lifecycler,
userID string,
blockVisitMarkerReadFailed prometheus.Counter,
blockVisitMarkerWriteFailed prometheus.Counter,
compactorMetrics *compactorMetrics,
) compact.Planner
type CompactionLifecycleCallbackFactory func(
ctx context.Context,
userBucket objstore.InstrumentedBucket,
logger log.Logger,
metaSyncConcurrency int,
compactDir string,
userID string,
compactorMetrics *compactorMetrics,
) compact.CompactionLifecycleCallback
type BlockDeletableCheckerFactory func(
ctx context.Context,
bkt objstore.InstrumentedBucket,
logger log.Logger,
) compact.BlockDeletableChecker
// Limits defines limits used by the Compactor.
type Limits interface {
CompactorTenantShardSize(userID string) float64
CompactorPartitionIndexSizeBytes(userID string) int64
CompactorPartitionSeriesCount(userID string) int64
}
// Config holds the Compactor config.
type Config struct {
BlockRanges cortex_tsdb.DurationList `yaml:"block_ranges"`
BlockSyncConcurrency int `yaml:"block_sync_concurrency"`
MetaSyncConcurrency int `yaml:"meta_sync_concurrency"`
ConsistencyDelay time.Duration `yaml:"consistency_delay"`
DataDir string `yaml:"data_dir"`
CompactionInterval time.Duration `yaml:"compaction_interval"`
CompactionRetries int `yaml:"compaction_retries"`
CompactionConcurrency int `yaml:"compaction_concurrency"`
CleanupInterval time.Duration `yaml:"cleanup_interval"`
CleanupConcurrency int `yaml:"cleanup_concurrency"`
DeletionDelay time.Duration `yaml:"deletion_delay"`
TenantCleanupDelay time.Duration `yaml:"tenant_cleanup_delay"`
SkipBlocksWithOutOfOrderChunksEnabled bool `yaml:"skip_blocks_with_out_of_order_chunks_enabled"`
BlockFilesConcurrency int `yaml:"block_files_concurrency"`
BlocksFetchConcurrency int `yaml:"blocks_fetch_concurrency"`
// Whether the migration of block deletion marks to the global markers location is enabled.
BlockDeletionMarksMigrationEnabled bool `yaml:"block_deletion_marks_migration_enabled"`
EnabledTenants flagext.StringSliceCSV `yaml:"enabled_tenants"`
DisabledTenants flagext.StringSliceCSV `yaml:"disabled_tenants"`
// Compactors sharding.
ShardingEnabled bool `yaml:"sharding_enabled"`
ShardingStrategy string `yaml:"sharding_strategy"`
ShardingRing RingConfig `yaml:"sharding_ring"`
ShardingPlannerDelay time.Duration `yaml:"sharding_planner_delay"`
// Compaction strategy.
CompactionStrategy string `yaml:"compaction_strategy"`
// No need to add options to customize the retry backoff,
// given the defaults should be fine, but allow to override
// it in tests.
retryMinBackoff time.Duration `yaml:"-"`
retryMaxBackoff time.Duration `yaml:"-"`
// Allow downstream projects to customise the blocks compactor.
BlocksGrouperFactory BlocksGrouperFactory `yaml:"-"`
BlocksCompactorFactory BlocksCompactorFactory `yaml:"-"`
// Compaction visit marker file config
CompactionVisitMarkerTimeout time.Duration `yaml:"compaction_visit_marker_timeout"`
CompactionVisitMarkerFileUpdateInterval time.Duration `yaml:"compaction_visit_marker_file_update_interval"`
// Cleaner visit marker file config
CleanerVisitMarkerTimeout time.Duration `yaml:"cleaner_visit_marker_timeout"`
CleanerVisitMarkerFileUpdateInterval time.Duration `yaml:"cleaner_visit_marker_file_update_interval"`
AcceptMalformedIndex bool `yaml:"accept_malformed_index"`
CachingBucketEnabled bool `yaml:"caching_bucket_enabled"`
CleanerCachingBucketEnabled bool `yaml:"cleaner_caching_bucket_enabled"`
}
// RegisterFlags registers the Compactor flags.
func (cfg *Config) RegisterFlags(f *flag.FlagSet) {
cfg.ShardingRing.RegisterFlags(f)
cfg.BlockRanges = cortex_tsdb.DurationList{2 * time.Hour, 12 * time.Hour, 24 * time.Hour}
cfg.retryMinBackoff = 10 * time.Second
cfg.retryMaxBackoff = time.Minute
f.Var(&cfg.BlockRanges, "compactor.block-ranges", "List of compaction time ranges.")
f.DurationVar(&cfg.ConsistencyDelay, "compactor.consistency-delay", 0, fmt.Sprintf("Minimum age of fresh (non-compacted) blocks before they are being processed. Malformed blocks older than the maximum of consistency-delay and %s will be removed.", compact.PartialUploadThresholdAge))
f.IntVar(&cfg.BlockSyncConcurrency, "compactor.block-sync-concurrency", 20, "Number of Go routines to use when syncing block index and chunks files from the long term storage.")
f.IntVar(&cfg.MetaSyncConcurrency, "compactor.meta-sync-concurrency", 20, "Number of Go routines to use when syncing block meta files from the long term storage.")
f.StringVar(&cfg.DataDir, "compactor.data-dir", "./data", "Data directory in which to cache blocks and process compactions")
f.DurationVar(&cfg.CompactionInterval, "compactor.compaction-interval", time.Hour, "The frequency at which the compaction runs")
f.IntVar(&cfg.CompactionRetries, "compactor.compaction-retries", 3, "How many times to retry a failed compaction within a single compaction run.")
f.IntVar(&cfg.CompactionConcurrency, "compactor.compaction-concurrency", 1, "Max number of concurrent compactions running.")
f.DurationVar(&cfg.CleanupInterval, "compactor.cleanup-interval", 15*time.Minute, "How frequently compactor should run blocks cleanup and maintenance, as well as update the bucket index.")
f.IntVar(&cfg.CleanupConcurrency, "compactor.cleanup-concurrency", 20, "Max number of tenants for which blocks cleanup and maintenance should run concurrently.")
f.BoolVar(&cfg.ShardingEnabled, "compactor.sharding-enabled", false, "Shard tenants across multiple compactor instances. Sharding is required if you run multiple compactor instances, in order to coordinate compactions and avoid race conditions leading to the same tenant blocks simultaneously compacted by different instances.")
f.StringVar(&cfg.ShardingStrategy, "compactor.sharding-strategy", util.ShardingStrategyDefault, fmt.Sprintf("The sharding strategy to use. Supported values are: %s.", strings.Join(supportedShardingStrategies, ", ")))
f.StringVar(&cfg.CompactionStrategy, "compactor.compaction-strategy", util.CompactionStrategyDefault, fmt.Sprintf("The compaction strategy to use. Supported values are: %s.", strings.Join(supportedCompactionStrategies, ", ")))
f.DurationVar(&cfg.DeletionDelay, "compactor.deletion-delay", 12*time.Hour, "Time before a block marked for deletion is deleted from bucket. "+
"If not 0, blocks will be marked for deletion and compactor component will permanently delete blocks marked for deletion from the bucket. "+
"If 0, blocks will be deleted straight away. Note that deleting blocks immediately can cause query failures.")
f.DurationVar(&cfg.TenantCleanupDelay, "compactor.tenant-cleanup-delay", 6*time.Hour, "For tenants marked for deletion, this is time between deleting of last block, and doing final cleanup (marker files, debug files) of the tenant.")
f.BoolVar(&cfg.BlockDeletionMarksMigrationEnabled, "compactor.block-deletion-marks-migration-enabled", false, "When enabled, at compactor startup the bucket will be scanned and all found deletion marks inside the block location will be copied to the markers global location too. This option can (and should) be safely disabled as soon as the compactor has successfully run at least once.")
f.BoolVar(&cfg.SkipBlocksWithOutOfOrderChunksEnabled, "compactor.skip-blocks-with-out-of-order-chunks-enabled", false, "When enabled, mark blocks containing index with out-of-order chunks for no compact instead of halting the compaction.")
f.IntVar(&cfg.BlockFilesConcurrency, "compactor.block-files-concurrency", 10, "Number of goroutines to use when fetching/uploading block files from object storage.")
f.IntVar(&cfg.BlocksFetchConcurrency, "compactor.blocks-fetch-concurrency", 3, "Number of goroutines to use when fetching blocks from object storage when compacting.")
f.Var(&cfg.EnabledTenants, "compactor.enabled-tenants", "Comma separated list of tenants that can be compacted. If specified, only these tenants will be compacted by compactor, otherwise all tenants can be compacted. Subject to sharding.")
f.Var(&cfg.DisabledTenants, "compactor.disabled-tenants", "Comma separated list of tenants that cannot be compacted by this compactor. If specified, and compactor would normally pick given tenant for compaction (via -compactor.enabled-tenants or sharding), it will be ignored instead.")
f.DurationVar(&cfg.CompactionVisitMarkerTimeout, "compactor.compaction-visit-marker-timeout", 10*time.Minute, "How long compaction visit marker file should be considered as expired and able to be picked up by compactor again.")
f.DurationVar(&cfg.CompactionVisitMarkerFileUpdateInterval, "compactor.compaction-visit-marker-file-update-interval", 1*time.Minute, "How frequently compaction visit marker file should be updated duration compaction.")
f.DurationVar(&cfg.CleanerVisitMarkerTimeout, "compactor.cleaner-visit-marker-timeout", 10*time.Minute, "How long cleaner visit marker file should be considered as expired and able to be picked up by cleaner again. The value should be smaller than -compactor.cleanup-interval")
f.DurationVar(&cfg.CleanerVisitMarkerFileUpdateInterval, "compactor.cleaner-visit-marker-file-update-interval", 5*time.Minute, "How frequently cleaner visit marker file should be updated when cleaning user.")
f.BoolVar(&cfg.AcceptMalformedIndex, "compactor.accept-malformed-index", false, "When enabled, index verification will ignore out of order label names.")
f.BoolVar(&cfg.CachingBucketEnabled, "compactor.caching-bucket-enabled", false, "When enabled, caching bucket will be used for compactor, except cleaner service, which serves as the source of truth for block status")
f.BoolVar(&cfg.CleanerCachingBucketEnabled, "compactor.cleaner-caching-bucket-enabled", false, "When enabled, caching bucket will be used for cleaner")
f.DurationVar(&cfg.ShardingPlannerDelay, "compactor.sharding-planner-delay", 10*time.Second, "How long shuffle sharding planner would wait before running planning code. This delay would prevent double compaction when two compactors claimed same partition in grouper at same time.")
}
func (cfg *Config) Validate(limits validation.Limits) error {
if slices.Contains(cfg.BlockRanges, 0) {
return errors.New("compactor block range period cannot be zero")
}
// Each block range period should be divisible by the previous one.
for i := 1; i < len(cfg.BlockRanges); i++ {
if cfg.BlockRanges[i]%cfg.BlockRanges[i-1] != 0 {
return errors.Errorf(errInvalidBlockRanges, cfg.BlockRanges[i].String(), cfg.BlockRanges[i-1].String())
}
}
// Make sure a valid sharding strategy is being used
if !slices.Contains(supportedShardingStrategies, cfg.ShardingStrategy) {
return errInvalidShardingStrategy
}
if cfg.ShardingEnabled && cfg.ShardingStrategy == util.ShardingStrategyShuffle {
if limits.CompactorTenantShardSize <= 0 {
return errInvalidTenantShardSize
}
}
// Make sure a valid compaction strategy is being used
if !slices.Contains(supportedCompactionStrategies, cfg.CompactionStrategy) {
return errInvalidCompactionStrategy
}
if !cfg.ShardingEnabled && cfg.CompactionStrategy == util.CompactionStrategyPartitioning {
return errInvalidCompactionStrategyPartitioning
}
return nil
}
// ConfigProvider defines the per-tenant config provider for the Compactor.
type ConfigProvider interface {
bucket.TenantConfigProvider
ParquetConverterEnabled(userID string) bool
CompactorBlocksRetentionPeriod(user string) time.Duration
}
// Compactor is a multi-tenant TSDB blocks compactor based on Thanos.
type Compactor struct {
services.Service
compactorCfg Config
storageCfg cortex_tsdb.BlocksStorageConfig
logger log.Logger
parentLogger log.Logger
registerer prometheus.Registerer
allowedTenants *users.AllowedTenants
limits *validation.Overrides
// Functions that creates bucket client, grouper, planner and compactor using the context.
// Useful for injecting mock objects from tests.
bucketClientFactory func(ctx context.Context) (objstore.InstrumentedBucket, error)
blocksGrouperFactory BlocksGrouperFactory
blocksCompactorFactory BlocksCompactorFactory
// Users scanner, used to discover users from the bucket.
usersScanner users.Scanner
userIndexUpdater *users.UserIndexUpdater
// Blocks cleaner is responsible to hard delete blocks marked for deletion.
blocksCleaner *BlocksCleaner
// Underlying compactor used to compact TSDB blocks.
blocksCompactor compact.Compactor
blocksPlannerFactory PlannerFactory
blockDeletableCheckerFactory BlockDeletableCheckerFactory
compactionLifecycleCallbackFactory CompactionLifecycleCallbackFactory
// Client used to run operations on the bucket storing blocks.
bucketClient objstore.InstrumentedBucket
// Ring used for sharding compactions.
ringLifecycler *ring.Lifecycler
ring *ring.Ring
ringSubservices *services.Manager
ringSubservicesWatcher *services.FailureWatcher
// Metrics.
CompactorStartDurationSeconds prometheus.Gauge
CompactionRunsStarted prometheus.Counter
CompactionRunsInterrupted prometheus.Counter
CompactionRunsCompleted prometheus.Counter
CompactionRunsFailed prometheus.Counter
CompactionRunsLastSuccess prometheus.Gauge
CompactionRunDiscoveredTenants prometheus.Gauge
CompactionRunSkippedTenants prometheus.Gauge
CompactionRunSucceededTenants prometheus.Gauge
CompactionRunFailedTenants prometheus.Gauge
CompactionRunInterval prometheus.Gauge
BlocksMarkedForNoCompaction prometheus.Counter
blockVisitMarkerReadFailed prometheus.Counter
blockVisitMarkerWriteFailed prometheus.Counter
// Thanos compactor metrics per user
compactorMetrics *compactorMetrics
// Replication factor of ingester ring
ingestionReplicationFactor int
}
// NewCompactor makes a new Compactor.
func NewCompactor(compactorCfg Config, storageCfg cortex_tsdb.BlocksStorageConfig, logger log.Logger, registerer prometheus.Registerer, limits *validation.Overrides, ingestionReplicationFactor int) (*Compactor, error) {
bucketClientFactory := func(ctx context.Context) (objstore.InstrumentedBucket, error) {
return bucket.NewClient(ctx, storageCfg.Bucket, nil, "compactor", logger, registerer)
}
blocksGrouperFactory := compactorCfg.BlocksGrouperFactory
if blocksGrouperFactory == nil {
if compactorCfg.ShardingStrategy == util.ShardingStrategyShuffle {
blocksGrouperFactory = ShuffleShardingGrouperFactory
} else {
blocksGrouperFactory = DefaultBlocksGrouperFactory
}
}
blocksCompactorFactory := compactorCfg.BlocksCompactorFactory
if blocksCompactorFactory == nil {
if compactorCfg.ShardingStrategy == util.ShardingStrategyShuffle {
blocksCompactorFactory = ShuffleShardingBlocksCompactorFactory
} else {
blocksCompactorFactory = DefaultBlocksCompactorFactory
}
}
var blockDeletableCheckerFactory BlockDeletableCheckerFactory
if compactorCfg.ShardingStrategy == util.ShardingStrategyShuffle && compactorCfg.CompactionStrategy == util.CompactionStrategyPartitioning {
blockDeletableCheckerFactory = PartitionCompactionBlockDeletableCheckerFactory
} else {
blockDeletableCheckerFactory = DefaultBlockDeletableCheckerFactory
}
var compactionLifecycleCallbackFactory CompactionLifecycleCallbackFactory
if compactorCfg.ShardingStrategy == util.ShardingStrategyShuffle && compactorCfg.CompactionStrategy == util.CompactionStrategyPartitioning {
compactionLifecycleCallbackFactory = ShardedCompactionLifecycleCallbackFactory
} else {
compactionLifecycleCallbackFactory = DefaultCompactionLifecycleCallbackFactory
}
if ingestionReplicationFactor <= 0 {
ingestionReplicationFactor = 1
}
cortexCompactor, err := newCompactor(compactorCfg, storageCfg, logger, registerer, bucketClientFactory, blocksGrouperFactory, blocksCompactorFactory, blockDeletableCheckerFactory, compactionLifecycleCallbackFactory, limits, ingestionReplicationFactor)
if err != nil {
return nil, errors.Wrap(err, "failed to create Cortex blocks compactor")
}
return cortexCompactor, nil
}
func newCompactor(
compactorCfg Config,
storageCfg cortex_tsdb.BlocksStorageConfig,
logger log.Logger,
registerer prometheus.Registerer,
bucketClientFactory func(ctx context.Context) (objstore.InstrumentedBucket, error),
blocksGrouperFactory BlocksGrouperFactory,
blocksCompactorFactory BlocksCompactorFactory,
blockDeletableCheckerFactory BlockDeletableCheckerFactory,
compactionLifecycleCallbackFactory CompactionLifecycleCallbackFactory,
limits *validation.Overrides,
ingestionReplicationFactor int,
) (*Compactor, error) {
var compactorMetrics *compactorMetrics
if compactorCfg.ShardingStrategy == util.ShardingStrategyShuffle {
compactorMetrics = newCompactorMetrics(registerer)
} else {
compactorMetrics = newDefaultCompactorMetrics(registerer)
}
c := &Compactor{
compactorCfg: compactorCfg,
storageCfg: storageCfg,
parentLogger: logger,
logger: log.With(logger, "component", "compactor"),
registerer: registerer,
bucketClientFactory: bucketClientFactory,
blocksGrouperFactory: blocksGrouperFactory,
blocksCompactorFactory: blocksCompactorFactory,
blockDeletableCheckerFactory: blockDeletableCheckerFactory,
compactionLifecycleCallbackFactory: compactionLifecycleCallbackFactory,
allowedTenants: users.NewAllowedTenants(compactorCfg.EnabledTenants, compactorCfg.DisabledTenants),
CompactorStartDurationSeconds: promauto.With(registerer).NewGauge(prometheus.GaugeOpts{
Name: "cortex_compactor_start_duration_seconds",
Help: "Time in seconds spent by compactor running start function",
}),
CompactionRunsStarted: promauto.With(registerer).NewCounter(prometheus.CounterOpts{
Name: "cortex_compactor_runs_started_total",
Help: "Total number of compaction runs started.",
}),
CompactionRunsInterrupted: promauto.With(registerer).NewCounter(prometheus.CounterOpts{
Name: "cortex_compactor_runs_interrupted_total",
Help: "Total number of compaction runs interrupted.",
}),
CompactionRunsCompleted: promauto.With(registerer).NewCounter(prometheus.CounterOpts{
Name: "cortex_compactor_runs_completed_total",
Help: "Total number of compaction runs successfully completed.",
}),
CompactionRunsFailed: promauto.With(registerer).NewCounter(prometheus.CounterOpts{
Name: "cortex_compactor_runs_failed_total",
Help: "Total number of compaction runs failed.",
}),
CompactionRunsLastSuccess: promauto.With(registerer).NewGauge(prometheus.GaugeOpts{
Name: "cortex_compactor_last_successful_run_timestamp_seconds",
Help: "Unix timestamp of the last successful compaction run.",
}),
CompactionRunDiscoveredTenants: promauto.With(registerer).NewGauge(prometheus.GaugeOpts{
Name: "cortex_compactor_tenants_discovered",
Help: "Number of tenants discovered during the current compaction run. Reset to 0 when compactor is idle.",
}),
CompactionRunSkippedTenants: promauto.With(registerer).NewGauge(prometheus.GaugeOpts{
Name: "cortex_compactor_tenants_skipped",
Help: "Number of tenants skipped during the current compaction run. Reset to 0 when compactor is idle.",
}),
CompactionRunSucceededTenants: promauto.With(registerer).NewGauge(prometheus.GaugeOpts{
Name: "cortex_compactor_tenants_processing_succeeded",
Help: "Number of tenants successfully processed during the current compaction run. Reset to 0 when compactor is idle.",
}),
CompactionRunFailedTenants: promauto.With(registerer).NewGauge(prometheus.GaugeOpts{
Name: "cortex_compactor_tenants_processing_failed",
Help: "Number of tenants failed processing during the current compaction run. Reset to 0 when compactor is idle.",
}),
CompactionRunInterval: promauto.With(registerer).NewGauge(prometheus.GaugeOpts{
Name: "cortex_compactor_compaction_interval_seconds",
Help: "The configured interval on which compaction is run in seconds. Useful when compared to the last successful run metric to accurately detect multiple failed compaction runs.",
}),
BlocksMarkedForNoCompaction: promauto.With(registerer).NewCounter(prometheus.CounterOpts{
Name: "cortex_compactor_blocks_marked_for_no_compaction_total",
Help: "Total number of blocks marked for no compact during a compaction run.",
}),
blockVisitMarkerReadFailed: promauto.With(registerer).NewCounter(prometheus.CounterOpts{
Name: "cortex_compactor_block_visit_marker_read_failed",
Help: "Number of block visit marker file failed to be read.",
}),
blockVisitMarkerWriteFailed: promauto.With(registerer).NewCounter(prometheus.CounterOpts{
Name: "cortex_compactor_block_visit_marker_write_failed",
Help: "Number of block visit marker file failed to be written.",
}),
limits: limits,
compactorMetrics: compactorMetrics,
ingestionReplicationFactor: ingestionReplicationFactor,
}
if len(compactorCfg.EnabledTenants) > 0 {
level.Info(c.logger).Log("msg", "compactor using enabled users", "enabled", strings.Join(compactorCfg.EnabledTenants, ", "))
}
if len(compactorCfg.DisabledTenants) > 0 {
level.Info(c.logger).Log("msg", "compactor using disabled users", "disabled", strings.Join(compactorCfg.DisabledTenants, ", "))
}
c.Service = services.NewBasicService(c.starting, c.running, c.stopping)
if c.registerer != nil {
// Copied from Thanos, pkg/block/fetcher.go
promauto.With(c.registerer).NewGaugeFunc(prometheus.GaugeOpts{
Name: "cortex_compactor_meta_sync_consistency_delay_seconds",
Help: "Configured consistency delay in seconds.",
}, func() float64 {
return c.compactorCfg.ConsistencyDelay.Seconds()
})
}
// The last successful compaction run metric is exposed as seconds since epoch, so we need to use seconds for this metric.
c.CompactionRunInterval.Set(c.compactorCfg.CompactionInterval.Seconds())
return c, nil
}
// Start the compactor.
func (c *Compactor) starting(ctx context.Context) error {
begin := time.Now()
defer func() {
c.CompactorStartDurationSeconds.Set(time.Since(begin).Seconds())
level.Info(c.logger).Log("msg", "compactor started", "duration", time.Since(begin), "duration_ms", time.Since(begin).Milliseconds())
}()
var err error
// Create bucket client.
c.bucketClient, err = c.bucketClientFactory(ctx)
if err != nil {
return errors.Wrap(err, "failed to create bucket client")
}
// Create blocks compactor dependencies.
c.blocksCompactor, c.blocksPlannerFactory, err = c.blocksCompactorFactory(ctx, c.compactorCfg, c.logger, c.registerer)
if err != nil {
return errors.Wrap(err, "failed to initialize compactor dependencies")
}
// Wrap the bucket client to write block deletion marks in the global location too.
c.bucketClient = bucketindex.BucketWithGlobalMarkers(c.bucketClient)
cleanerBucketClient := c.bucketClient
if c.compactorCfg.CleanerCachingBucketEnabled {
cleanerBucketClient, err = cortex_tsdb.CreateCachingBucketForCompactor(c.storageCfg.BucketStore.MetadataCache, true, c.bucketClient, c.logger, extprom.WrapRegistererWith(prometheus.Labels{"component": "cleaner"}, c.registerer))
if err != nil {
return errors.Wrap(err, "create caching bucket for cleaner")
}
}
if c.compactorCfg.CachingBucketEnabled {
c.bucketClient, err = cortex_tsdb.CreateCachingBucketForCompactor(c.storageCfg.BucketStore.MetadataCache, false, c.bucketClient, c.logger, extprom.WrapRegistererWith(prometheus.Labels{"component": "compactor"}, c.registerer))
if err != nil {
return errors.Wrap(err, "create caching bucket for compactor")
}
}
// Create the users scanner.
c.usersScanner, err = users.NewScanner(c.storageCfg.UsersScanner, c.bucketClient, c.logger, extprom.WrapRegistererWith(prometheus.Labels{"component": "compactor"}, c.registerer))
if err != nil {
return errors.Wrap(err, "failed to create users scanner")
}
var cleanerRingLifecyclerID = "default-cleaner"
// Initialize the compactors ring if sharding is enabled.
if c.compactorCfg.ShardingEnabled {
lifecyclerCfg := c.compactorCfg.ShardingRing.ToLifecyclerConfig()
var delegate ring.LifecyclerDelegate
delegate = &ring.DefaultLifecyclerDelegate{}
if c.compactorCfg.ShardingRing.AutoForgetDelay > 0 {
delegate = ring.NewLifecyclerAutoForgetDelegate(c.compactorCfg.ShardingRing.AutoForgetDelay, delegate, c.logger)
}
c.ringLifecycler, err = ring.NewLifecyclerWithDelegate(lifecyclerCfg, ring.NewNoopFlushTransferer(), "compactor", ringKey, true, false, c.logger, prometheus.WrapRegistererWithPrefix("cortex_", c.registerer), delegate)
if err != nil {
return errors.Wrap(err, "unable to initialize compactor ring lifecycler")
}
cleanerRingLifecyclerID = c.ringLifecycler.ID
c.ring, err = ring.New(lifecyclerCfg.RingConfig, "compactor", ringKey, c.logger, prometheus.WrapRegistererWithPrefix("cortex_", c.registerer))
if err != nil {
return errors.Wrap(err, "unable to initialize compactor ring")
}
c.ringSubservices, err = services.NewManager(c.ringLifecycler, c.ring)
if err == nil {
c.ringSubservicesWatcher = services.NewFailureWatcher()
c.ringSubservicesWatcher.WatchManager(c.ringSubservices)
err = services.StartManagerAndAwaitHealthy(ctx, c.ringSubservices)
}
if err != nil {
return errors.Wrap(err, "unable to start compactor ring dependencies")
}
// If sharding is enabled we should wait until this instance is
// ACTIVE within the ring. This MUST be done before starting the
// any other component depending on the users scanner, because the
// users scanner depends on the ring (to check whether an user belongs
// to this shard or not).
level.Info(c.logger).Log("msg", "waiting until compactor is ACTIVE in the ring")
ctxWithTimeout, cancel := context.WithTimeout(ctx, c.compactorCfg.ShardingRing.WaitActiveInstanceTimeout)
defer cancel()
if err := ring.WaitInstanceState(ctxWithTimeout, c.ring, c.ringLifecycler.ID, ring.ACTIVE); err != nil {
level.Error(c.logger).Log("msg", "compactor failed to become ACTIVE in the ring", "err", err)
return err
}
level.Info(c.logger).Log("msg", "compactor is ACTIVE in the ring")
// In the event of a cluster cold start or scale up of 2+ compactor instances at the same
// time, we may end up in a situation where each new compactor instance starts at a slightly
// different time and thus each one starts with a different state of the ring. It's better
// to just wait the ring stability for a short time.
if c.compactorCfg.ShardingRing.WaitStabilityMinDuration > 0 {
minWaiting := c.compactorCfg.ShardingRing.WaitStabilityMinDuration
maxWaiting := c.compactorCfg.ShardingRing.WaitStabilityMaxDuration
level.Info(c.logger).Log("msg", "waiting until compactor ring topology is stable", "min_waiting", minWaiting.String(), "max_waiting", maxWaiting.String())
if err := ring.WaitRingStability(ctx, c.ring, RingOp, minWaiting, maxWaiting); err != nil {
level.Warn(c.logger).Log("msg", "compactor ring topology is not stable after the max waiting time, proceeding anyway")
} else {
level.Info(c.logger).Log("msg", "compactor ring topology is stable")
}
}
}
// Cleaner needs a users scanner that is sharded.
cleanerUsersScanner := users.NewShardedScanner(c.usersScanner, c.ownUserForCleanUp, c.logger)
// Create the blocks cleaner (service).
c.blocksCleaner = NewBlocksCleaner(BlocksCleanerConfig{
DeletionDelay: c.compactorCfg.DeletionDelay,
CleanupInterval: util.DurationWithJitter(c.compactorCfg.CleanupInterval, 0.1),
CleanupConcurrency: c.compactorCfg.CleanupConcurrency,
BlockDeletionMarksMigrationEnabled: c.compactorCfg.BlockDeletionMarksMigrationEnabled,
TenantCleanupDelay: c.compactorCfg.TenantCleanupDelay,
ShardingStrategy: c.compactorCfg.ShardingStrategy,
CompactionStrategy: c.compactorCfg.CompactionStrategy,
BlockRanges: c.compactorCfg.BlockRanges.ToMilliseconds(),
}, cleanerBucketClient, cleanerUsersScanner, c.compactorCfg.CompactionVisitMarkerTimeout, c.limits, c.parentLogger, cleanerRingLifecyclerID, c.registerer, c.compactorCfg.CleanerVisitMarkerTimeout, c.compactorCfg.CleanerVisitMarkerFileUpdateInterval,
c.compactorMetrics.syncerBlocksMarkedForDeletion, c.compactorMetrics.remainingPlannedCompactions)
// If sharding is disabled, there is no need to have every compactor to run the user index updater
// as it will be the same to fallback to list strategy.
if c.compactorCfg.ShardingEnabled && c.storageCfg.UsersScanner.Strategy == users.UserScanStrategyUserIndex {
// We hardcode strategy to be list so can ignore error.
baseScanner, _ := users.NewScanner(users.UsersScannerConfig{
Strategy: users.UserScanStrategyList,
}, c.bucketClient, c.logger, c.registerer)
c.userIndexUpdater = users.NewUserIndexUpdater(c.bucketClient, c.storageCfg.UsersScanner.UpdateInterval, baseScanner, extprom.WrapRegistererWith(prometheus.Labels{"component": "compactor"}, c.registerer))
}
return nil
}
func (c *Compactor) stopping(_ error) error {
begin := time.Now()
defer func() {
level.Info(c.logger).Log("msg", "compactor stopped", "duration", time.Since(begin), "duration_ms", time.Since(begin).Milliseconds())
}()
ctx := context.Background()
services.StopAndAwaitTerminated(ctx, c.blocksCleaner) //nolint:errcheck
if c.ringSubservices != nil {
return services.StopManagerAndAwaitStopped(ctx, c.ringSubservices)
}
return nil
}
func (c *Compactor) running(ctx context.Context) error {
// Ensure an initial cleanup occurred as first thing when running compactor.
if err := services.StartAndAwaitRunning(ctx, c.blocksCleaner); err != nil {
c.ringSubservices.StopAsync()
return errors.Wrap(err, "failed to start the blocks cleaner")
}
if c.userIndexUpdater != nil {
go c.userIndexUpdateLoop(ctx)
}
// Run an initial compaction before starting the interval.
// Insert jitter right before compaction starts to avoid multiple starting compactor to be in sync
select {
case <-ctx.Done():
return ctx.Err()
case <-time.After(time.Duration(rand.Int63n(int64(float64(c.compactorCfg.CompactionInterval) * 0.1)))):
}
c.compactUsers(ctx)
ticker := time.NewTicker(c.compactorCfg.CompactionInterval)
defer ticker.Stop()
for {
select {
case <-ticker.C:
// Insert jitter right before compaction starts, so that there will always
// have jitter even compaction time is longer than CompactionInterval
time.Sleep(time.Duration(rand.Int63n(int64(float64(c.compactorCfg.CompactionInterval) * 0.1))))
c.compactUsers(ctx)
case <-ctx.Done():
return nil
case err := <-c.ringSubservicesWatcher.Chan():
return errors.Wrap(err, "compactor subservice failed")
}
}
}
func (c *Compactor) compactUsers(ctx context.Context) {
succeeded := false
interrupted := false
compactionErrorCount := 0
c.CompactionRunsStarted.Inc()
defer func() {
if succeeded && compactionErrorCount == 0 {
c.CompactionRunsCompleted.Inc()
c.CompactionRunsLastSuccess.SetToCurrentTime()
} else if interrupted {
c.CompactionRunsInterrupted.Inc()
} else {
c.CompactionRunsFailed.Inc()
}
// Reset progress metrics once done.
c.CompactionRunDiscoveredTenants.Set(0)
c.CompactionRunSkippedTenants.Set(0)
c.CompactionRunSucceededTenants.Set(0)
c.CompactionRunFailedTenants.Set(0)
}()
level.Info(c.logger).Log("msg", "discovering users from bucket")
userIDs, err := c.discoverUsersWithRetries(ctx)
if err != nil {
level.Error(c.logger).Log("msg", "failed to discover users from bucket", "err", err)
return
}
level.Info(c.logger).Log("msg", "discovered users from bucket", "users", len(userIDs))
c.CompactionRunDiscoveredTenants.Set(float64(len(userIDs)))
// When starting multiple compactor replicas nearly at the same time, running in a cluster with
// a large number of tenants, we may end up in a situation where the 1st user is compacted by
// multiple replicas at the same time. Shuffling users helps reduce the likelihood this will happen.
rand.Shuffle(len(userIDs), func(i, j int) {
userIDs[i], userIDs[j] = userIDs[j], userIDs[i]
})
// Keep track of users owned by this shard, so that we can delete the local files for all other users.
ownedUsers := map[string]struct{}{}
for _, userID := range userIDs {
// Ensure the context has not been canceled (ie. compactor shutdown has been triggered).
if ctx.Err() != nil {
interrupted = true
level.Info(c.logger).Log("msg", "interrupting compaction of user blocks", "user", userID)
return
}
// Ensure the user ID belongs to our shard.
if owned, err := c.ownUserForCompaction(userID); err != nil {
c.CompactionRunSkippedTenants.Inc()
level.Warn(c.logger).Log("msg", "unable to check if user is owned by this shard", "user", userID, "err", err)
continue
} else if !owned {
c.CompactionRunSkippedTenants.Inc()
level.Debug(c.logger).Log("msg", "skipping user because it is not owned by this shard", "user", userID)
continue
}
// Skipping compaction if the bucket index failed to sync due to CMK errors.
if idxs, err := bucketindex.ReadSyncStatus(ctx, c.bucketClient, userID, util_log.WithUserID(userID, c.logger)); err == nil {
if idxs.Status == bucketindex.CustomerManagedKeyError {
c.CompactionRunSkippedTenants.Inc()
level.Info(c.logger).Log("msg", "skipping compactUser due CustomerManagedKeyError", "user", userID)
continue
}
}
ownedUsers[userID] = struct{}{}
if markedForDeletion, err := users.TenantDeletionMarkExists(ctx, c.bucketClient, userID); err != nil {
c.CompactionRunSkippedTenants.Inc()
level.Warn(c.logger).Log("msg", "unable to check if user is marked for deletion", "user", userID, "err", err)
continue
} else if markedForDeletion {
c.CompactionRunSkippedTenants.Inc()
c.compactorMetrics.deleteMetricsForDeletedTenant(userID)
level.Debug(c.logger).Log("msg", "skipping user because it is marked for deletion", "user", userID)
continue
}
level.Info(c.logger).Log("msg", "starting compaction of user blocks", "user", userID)
if err = c.compactUserWithRetries(ctx, userID); err != nil {
// TODO: patch thanos error types to support errors.Is(err, context.Canceled) here
if ctx.Err() != nil && ctx.Err() == context.Canceled {
interrupted = true
level.Info(c.logger).Log("msg", "interrupting compaction of user blocks", "user", userID)
return
}
c.CompactionRunFailedTenants.Inc()
compactionErrorCount++
level.Error(c.logger).Log("msg", "failed to compact user blocks", "user", userID, "err", err)
continue
}
c.CompactionRunSucceededTenants.Inc()
level.Info(c.logger).Log("msg", "successfully compacted user blocks", "user", userID)
}
// Delete local files for unowned tenants, if there are any. This cleans up
// leftover local files for tenants that belong to different compactors now,
// or have been deleted completely.
for userID := range c.listTenantsWithMetaSyncDirectories() {
if _, owned := ownedUsers[userID]; owned {
continue
}
dir := c.metaSyncDirForUser(userID)
s, err := os.Stat(dir)
if err != nil {
if !os.IsNotExist(err) {
level.Warn(c.logger).Log("msg", "failed to stat local directory with user data", "dir", dir, "err", err)
}
continue
}
if s.IsDir() {
err := os.RemoveAll(dir)
if err == nil {
level.Info(c.logger).Log("msg", "deleted directory for user not owned by this shard", "dir", dir)
} else {
level.Warn(c.logger).Log("msg", "failed to delete directory for user not owned by this shard", "dir", dir, "err", err)
}
}
}
succeeded = true
}
func (c *Compactor) compactUserWithRetries(ctx context.Context, userID string) error {
var lastErr error
retries := backoff.New(ctx, backoff.Config{
MinBackoff: c.compactorCfg.retryMinBackoff,
MaxBackoff: c.compactorCfg.retryMaxBackoff,
MaxRetries: c.compactorCfg.CompactionRetries,
})
for retries.Ongoing() {
lastErr = c.compactUser(ctx, userID)
if lastErr == nil {
return nil
}
if ctx.Err() != nil {
return ctx.Err()
}
if c.isCausedByPermissionDenied(lastErr) {
level.Warn(c.logger).Log("msg", "skipping compactUser due to PermissionDenied", "user", userID, "err", lastErr)
c.compactorMetrics.compactionErrorsCount.WithLabelValues(userID, unauthorizedError).Inc()
return nil
}
if compact.IsHaltError(lastErr) {
level.Error(c.logger).Log("msg", "compactor returned critical error", "user", userID, "err", lastErr)
c.compactorMetrics.compactionErrorsCount.WithLabelValues(userID, haltError).Inc()
return lastErr
}
c.compactorMetrics.compactionErrorsCount.WithLabelValues(userID, retriableError).Inc()
retries.Wait()
}
err := errors.Unwrap(errors.Cause(lastErr))
if errors.Is(err, plannerCompletedPartitionError) || errors.Is(err, plannerVisitedPartitionError) {
return nil
}
return lastErr
}
func (c *Compactor) compactUser(ctx context.Context, userID string) error {
bucket := bucket.NewUserBucketClient(userID, c.bucketClient, c.limits)
reg := prometheus.NewRegistry()