-
Notifications
You must be signed in to change notification settings - Fork 3.3k
/
compactor.go
849 lines (720 loc) · 32.8 KB
/
compactor.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
package compactor
import (
"context"
"flag"
"fmt"
"net/http"
"os"
"path/filepath"
"sort"
"sync"
"time"
"github.com/go-kit/log/level"
"github.com/grafana/dskit/flagext"
"github.com/grafana/dskit/kv"
"github.com/grafana/dskit/ring"
"github.com/grafana/dskit/services"
"github.com/pkg/errors"
"github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/common/model"
"github.com/grafana/loki/pkg/analytics"
"github.com/grafana/loki/pkg/compactor/deletion"
"github.com/grafana/loki/pkg/compactor/retention"
"github.com/grafana/loki/pkg/storage/chunk/client"
"github.com/grafana/loki/pkg/storage/chunk/client/local"
chunk_util "github.com/grafana/loki/pkg/storage/chunk/client/util"
"github.com/grafana/loki/pkg/storage/config"
"github.com/grafana/loki/pkg/storage/stores/shipper/indexshipper/storage"
"github.com/grafana/loki/pkg/util/filter"
util_log "github.com/grafana/loki/pkg/util/log"
lokiring "github.com/grafana/loki/pkg/util/ring"
"github.com/grafana/loki/pkg/validation"
)
// Here is how the generic compactor works:
// 1. Find the index type from table name using schemaPeriodForTable.
// 2. Find the registered IndexCompactor for the index type.
// 3. Build an instance of TableCompactor using IndexCompactor.NewIndexCompactor, with all the required information to do a compaction.
// 4. Run the compaction using TableCompactor.Compact, which would set the new/updated CompactedIndex for each IndexSet.
// 5. If retention is enabled, run retention on the CompactedIndex using its retention.IndexProcessor implementation.
// 6. Convert the CompactedIndex to a file using the IndexCompactor.ToIndexFile for uploading.
// 7. If we uploaded successfully, delete the old index files.
const (
// ringAutoForgetUnhealthyPeriods is how many consecutive timeout periods an unhealthy instance
// in the ring will be automatically removed.
ringAutoForgetUnhealthyPeriods = 10
// ringKey is the key under which we store the store gateways ring in the KVStore.
ringKey = "compactor"
// ringNameForServer is the name of the ring used by the compactor server.
ringNameForServer = "compactor"
// ringKeyOfLeader is a somewhat arbitrary ID to pull from the ring to see who will be elected the leader
ringKeyOfLeader = 0
// ringReplicationFactor should be 1 because we only want to pull back one node from the Ring
ringReplicationFactor = 1
// ringNumTokens sets our single token in the ring,
// we only need to insert 1 token to be used for leader election purposes.
ringNumTokens = 1
)
var (
retentionEnabledStats = analytics.NewString("compactor_retention_enabled")
defaultRetentionStats = analytics.NewString("compactor_default_retention")
)
type Config struct {
WorkingDirectory string `yaml:"working_directory"`
CompactionInterval time.Duration `yaml:"compaction_interval"`
ApplyRetentionInterval time.Duration `yaml:"apply_retention_interval"`
RetentionEnabled bool `yaml:"retention_enabled"`
RetentionDeleteDelay time.Duration `yaml:"retention_delete_delay"`
RetentionDeleteWorkCount int `yaml:"retention_delete_worker_count"`
RetentionTableTimeout time.Duration `yaml:"retention_table_timeout"`
DeleteRequestStore string `yaml:"delete_request_store"`
DeleteRequestStoreKeyPrefix string `yaml:"delete_request_store_key_prefix"`
DeleteBatchSize int `yaml:"delete_batch_size"`
DeleteRequestCancelPeriod time.Duration `yaml:"delete_request_cancel_period"`
DeleteMaxInterval time.Duration `yaml:"delete_max_interval"`
MaxCompactionParallelism int `yaml:"max_compaction_parallelism"`
UploadParallelism int `yaml:"upload_parallelism"`
CompactorRing lokiring.RingConfig `yaml:"compactor_ring,omitempty" doc:"description=The hash ring configuration used by compactors to elect a single instance for running compactions. The CLI flags prefix for this block config is: compactor.ring"`
RunOnce bool `yaml:"_" doc:"hidden"`
TablesToCompact int `yaml:"tables_to_compact"`
SkipLatestNTables int `yaml:"skip_latest_n_tables"`
// Deprecated
DeletionMode string `yaml:"deletion_mode" doc:"deprecated|description=Use deletion_mode per tenant configuration instead."`
}
// RegisterFlags registers flags.
func (cfg *Config) RegisterFlagsWithPrefix(prefix string, f *flag.FlagSet) {
var deprecated = ""
if prefix != "" {
deprecated = "Deprecated: "
}
f.StringVar(&cfg.WorkingDirectory, prefix+"compactor.working-directory", "", deprecated+"Directory where files can be downloaded for compaction.")
f.DurationVar(&cfg.CompactionInterval, prefix+"compactor.compaction-interval", 10*time.Minute, deprecated+"Interval at which to re-run the compaction operation.")
f.DurationVar(&cfg.ApplyRetentionInterval, prefix+"compactor.apply-retention-interval", 0, deprecated+"Interval at which to apply/enforce retention. 0 means run at same interval as compaction. If non-zero, it should always be a multiple of compaction interval.")
f.DurationVar(&cfg.RetentionDeleteDelay, prefix+"compactor.retention-delete-delay", 2*time.Hour, deprecated+"Delay after which chunks will be fully deleted during retention.")
f.BoolVar(&cfg.RetentionEnabled, prefix+"compactor.retention-enabled", false, deprecated+"(Experimental) Activate custom (per-stream,per-tenant) retention.")
f.IntVar(&cfg.RetentionDeleteWorkCount, prefix+"compactor.retention-delete-worker-count", 150, deprecated+"The total amount of worker to use to delete chunks.")
f.StringVar(&cfg.DeleteRequestStore, prefix+"compactor.delete-request-store", "", deprecated+"Store used for managing delete requests.")
f.IntVar(&cfg.DeleteBatchSize, prefix+"compactor.delete-batch-size", 70, deprecated+"The max number of delete requests to run per compaction cycle.")
f.DurationVar(&cfg.DeleteRequestCancelPeriod, prefix+"compactor.delete-request-cancel-period", 24*time.Hour, deprecated+"Allow cancellation of delete request until duration after they are created. Data would be deleted only after delete requests have been older than this duration. Ideally this should be set to at least 24h.")
f.DurationVar(&cfg.DeleteMaxInterval, prefix+"compactor.delete-max-interval", 24*time.Hour, deprecated+"Constrain the size of any single delete request. When a delete request > delete_max_interval is input, the request is sharded into smaller requests of no more than delete_max_interval")
f.DurationVar(&cfg.RetentionTableTimeout, prefix+"compactor.retention-table-timeout", 0, deprecated+"The maximum amount of time to spend running retention and deletion on any given table in the index.")
f.IntVar(&cfg.MaxCompactionParallelism, prefix+"compactor.max-compaction-parallelism", 1, deprecated+"Maximum number of tables to compact in parallel. While increasing this value, please make sure compactor has enough disk space allocated to be able to store and compact as many tables.")
f.IntVar(&cfg.UploadParallelism, prefix+"compactor.upload-parallelism", 10, deprecated+"Number of upload/remove operations to execute in parallel when finalizing a compaction. NOTE: This setting is per compaction operation, which can be executed in parallel. The upper bound on the number of concurrent uploads is upload_parallelism * max_compaction_parallelism.")
f.BoolVar(&cfg.RunOnce, prefix+"compactor.run-once", false, deprecated+"Run the compactor one time to cleanup and compact index files only (no retention applied)")
cfg.CompactorRing.RegisterFlagsWithPrefix(prefix+"compactor.", "collectors/", f)
f.IntVar(&cfg.TablesToCompact, prefix+"compactor.tables-to-compact", 0, deprecated+"Number of tables that compactor will try to compact. Newer tables are chosen when this is less than the number of tables available.")
f.IntVar(&cfg.SkipLatestNTables, prefix+"compactor.skip-latest-n-tables", 0, deprecated+"Do not compact N latest tables. Together with -compactor.run-once and -compactor.tables-to-compact, this is useful when clearing compactor backlogs.")
}
// RegisterFlags registers flags.
func (cfg *Config) RegisterFlags(f *flag.FlagSet) {
cfg.RegisterFlagsWithPrefix("", f)
f.StringVar(&cfg.DeleteRequestStoreKeyPrefix, "compactor.delete-request-store.key-prefix", "index/", "Path prefix for storing delete requests.")
// Deprecated. CLI flags with boltdb.shipper. prefix will be removed in the next major version.
cfg.RegisterFlagsWithPrefix("boltdb.shipper.", f)
// Deprecated
flagext.DeprecatedFlag(f, "boltdb.shipper.compactor.deletion-mode", "Deprecated. This has been moved to the deletion_mode per tenant configuration.", util_log.Logger)
}
// Validate verifies the config does not contain inappropriate values
func (cfg *Config) Validate() error {
if cfg.MaxCompactionParallelism < 1 {
return errors.New("max compaction parallelism must be >= 1")
}
if cfg.RetentionEnabled {
if cfg.DeleteRequestStore == "" {
return fmt.Errorf("compactor.delete-request-store should be configured when retention is enabled")
}
if cfg.ApplyRetentionInterval != 0 && cfg.ApplyRetentionInterval%cfg.CompactionInterval != 0 {
return fmt.Errorf("interval for applying retention should either be set to a 0 or a multiple of compaction interval")
}
if err := config.ValidatePathPrefix(cfg.DeleteRequestStoreKeyPrefix); err != nil {
return fmt.Errorf("validate delete store path prefix: %w", err)
}
}
if cfg.DeletionMode != "" {
level.Warn(util_log.Logger).Log("msg", "boltdb.shipper.compactor.deletion-mode has been deprecated and will be ignored. This has been moved to the deletion_mode per tenant configuration.")
}
return nil
}
type Compactor struct {
services.Service
cfg Config
indexStorageClient storage.Client
tableMarker retention.TableMarker
sweeper *retention.Sweeper
deleteRequestsStore deletion.DeleteRequestsStore
DeleteRequestsHandler *deletion.DeleteRequestHandler
DeleteRequestsGRPCHandler *deletion.GRPCRequestHandler
deleteRequestsManager *deletion.DeleteRequestsManager
expirationChecker retention.ExpirationChecker
metrics *metrics
running bool
wg sync.WaitGroup
indexCompactors map[string]IndexCompactor
schemaConfig config.SchemaConfig
// Ring used for running a single compactor
ringLifecycler *ring.BasicLifecycler
ring *ring.Ring
ringPollPeriod time.Duration
// Subservices manager.
subservices *services.Manager
subservicesWatcher *services.FailureWatcher
// one for each period
storeContainers map[config.DayTime]storeContainer
}
type storeContainer struct {
tableMarker retention.TableMarker
sweeper *retention.Sweeper
indexStorageClient storage.Client
}
type Limits interface {
deletion.Limits
retention.Limits
DefaultLimits() *validation.Limits
}
func NewCompactor(cfg Config, objectStoreClients map[config.DayTime]client.ObjectClient, deleteStoreClient client.ObjectClient, schemaConfig config.SchemaConfig, limits Limits, r prometheus.Registerer, metricsNamespace string) (*Compactor, error) {
retentionEnabledStats.Set("false")
if cfg.RetentionEnabled {
retentionEnabledStats.Set("true")
}
if limits != nil {
defaultRetentionStats.Set(limits.DefaultLimits().RetentionPeriod.String())
}
compactor := &Compactor{
cfg: cfg,
ringPollPeriod: 5 * time.Second,
indexCompactors: map[string]IndexCompactor{},
schemaConfig: schemaConfig,
}
ringStore, err := kv.NewClient(
cfg.CompactorRing.KVStore,
ring.GetCodec(),
kv.RegistererWithKVName(prometheus.WrapRegistererWithPrefix("loki_", r), "compactor"),
util_log.Logger,
)
if err != nil {
return nil, errors.Wrap(err, "create KV store client")
}
lifecyclerCfg, err := cfg.CompactorRing.ToLifecyclerConfig(ringNumTokens, util_log.Logger)
if err != nil {
return nil, errors.Wrap(err, "invalid ring lifecycler config")
}
// Define lifecycler delegates in reverse order (last to be called defined first because they're
// chained via "next delegate").
delegate := ring.BasicLifecyclerDelegate(compactor)
delegate = ring.NewLeaveOnStoppingDelegate(delegate, util_log.Logger)
delegate = ring.NewTokensPersistencyDelegate(cfg.CompactorRing.TokensFilePath, ring.JOINING, delegate, util_log.Logger)
delegate = ring.NewAutoForgetDelegate(ringAutoForgetUnhealthyPeriods*cfg.CompactorRing.HeartbeatTimeout, delegate, util_log.Logger)
compactor.ringLifecycler, err = ring.NewBasicLifecycler(lifecyclerCfg, ringNameForServer, ringKey, ringStore, delegate, util_log.Logger, r)
if err != nil {
return nil, errors.Wrap(err, "create ring lifecycler")
}
ringCfg := cfg.CompactorRing.ToRingConfig(ringReplicationFactor)
compactor.ring, err = ring.NewWithStoreClientAndStrategy(ringCfg, ringNameForServer, ringKey, ringStore, ring.NewIgnoreUnhealthyInstancesReplicationStrategy(), prometheus.WrapRegistererWithPrefix(metricsNamespace+"_", r), util_log.Logger)
if err != nil {
return nil, errors.Wrap(err, "create ring client")
}
compactor.subservices, err = services.NewManager(compactor.ringLifecycler, compactor.ring)
if err != nil {
return nil, err
}
compactor.subservicesWatcher = services.NewFailureWatcher()
compactor.subservicesWatcher.WatchManager(compactor.subservices)
if err := compactor.init(objectStoreClients, deleteStoreClient, schemaConfig, limits, r); err != nil {
return nil, fmt.Errorf("init compactor: %w", err)
}
compactor.Service = services.NewBasicService(compactor.starting, compactor.loop, compactor.stopping)
return compactor, nil
}
func (c *Compactor) init(objectStoreClients map[config.DayTime]client.ObjectClient, deleteStoreClient client.ObjectClient, schemaConfig config.SchemaConfig, limits Limits, r prometheus.Registerer) error {
err := chunk_util.EnsureDirectory(c.cfg.WorkingDirectory)
if err != nil {
return err
}
if c.cfg.RetentionEnabled {
if deleteStoreClient == nil {
return fmt.Errorf("delete store client not initialised when retention is enabled")
}
if err := c.initDeletes(deleteStoreClient, r, limits); err != nil {
return fmt.Errorf("failed to init delete store: %w", err)
}
}
legacyMarkerDirs := make(map[string]struct{})
c.storeContainers = make(map[config.DayTime]storeContainer, len(objectStoreClients))
for from, objectClient := range objectStoreClients {
period, err := schemaConfig.SchemaForTime(from.Time)
if err != nil {
return err
}
var sc storeContainer
sc.indexStorageClient = storage.NewIndexStorageClient(objectClient, period.IndexTables.PathPrefix)
if c.cfg.RetentionEnabled {
var (
raw client.ObjectClient
encoder client.KeyEncoder
name = fmt.Sprintf("%s_%s", period.ObjectType, period.From.String())
retentionWorkDir = filepath.Join(c.cfg.WorkingDirectory, "retention", name)
r = prometheus.WrapRegistererWith(prometheus.Labels{"from": name}, r)
)
// given that compaction can now run on multiple periods, marker files are stored under /retention/{objectStoreType}_{periodFrom}/markers/
// if any markers are found in the common markers dir (/retention/markers/) or store specific markers dir (/retention/{objectStoreType}/markers/), copy them to the period specific dirs
// chunk would be removed by the sweeper if it belongs to a given period or no-op if it doesn't exist.
if err := retention.CopyMarkers(filepath.Join(c.cfg.WorkingDirectory, "retention"), retentionWorkDir); err != nil {
return fmt.Errorf("failed to move common markers to period specific dir: %w", err)
}
if err := retention.CopyMarkers(filepath.Join(c.cfg.WorkingDirectory, "retention", period.ObjectType), retentionWorkDir); err != nil {
return fmt.Errorf("failed to move store markers to period specific dir: %w", err)
}
// remove markers from the store dir after copying them to period specific dirs.
legacyMarkerDirs[period.ObjectType] = struct{}{}
if casted, ok := objectClient.(client.PrefixedObjectClient); ok {
raw = casted.GetDownstream()
} else {
raw = objectClient
}
if _, ok := raw.(*local.FSObjectClient); ok {
encoder = client.FSEncoder
}
chunkClient := client.NewClient(objectClient, encoder, schemaConfig)
sc.sweeper, err = retention.NewSweeper(retentionWorkDir, chunkClient, c.cfg.RetentionDeleteWorkCount, c.cfg.RetentionDeleteDelay, r)
if err != nil {
return fmt.Errorf("failed to init sweeper: %w", err)
}
sc.tableMarker, err = retention.NewMarker(retentionWorkDir, c.expirationChecker, c.cfg.RetentionTableTimeout, chunkClient, r)
if err != nil {
return fmt.Errorf("failed to init table marker: %w", err)
}
}
c.storeContainers[from] = sc
}
if c.cfg.RetentionEnabled {
// remove legacy markers
for store := range legacyMarkerDirs {
if err := os.RemoveAll(filepath.Join(c.cfg.WorkingDirectory, "retention", store, retention.MarkersFolder)); err != nil {
return fmt.Errorf("remove old markers from store dir: %w", err)
}
}
if err := os.RemoveAll(filepath.Join(c.cfg.WorkingDirectory, "retention", retention.MarkersFolder)); err != nil {
return fmt.Errorf("remove old markers: %w", err)
}
}
c.metrics = newMetrics(r)
return nil
}
func (c *Compactor) initDeletes(objectClient client.ObjectClient, r prometheus.Registerer, limits Limits) error {
deletionWorkDir := filepath.Join(c.cfg.WorkingDirectory, "deletion")
store, err := deletion.NewDeleteStore(deletionWorkDir, storage.NewIndexStorageClient(objectClient, c.cfg.DeleteRequestStoreKeyPrefix))
if err != nil {
return err
}
c.deleteRequestsStore = store
c.DeleteRequestsHandler = deletion.NewDeleteRequestHandler(
c.deleteRequestsStore,
c.cfg.DeleteMaxInterval,
r,
)
c.DeleteRequestsGRPCHandler = deletion.NewGRPCRequestHandler(c.deleteRequestsStore, limits)
c.deleteRequestsManager = deletion.NewDeleteRequestsManager(
c.deleteRequestsStore,
c.cfg.DeleteRequestCancelPeriod,
c.cfg.DeleteBatchSize,
limits,
r,
)
c.expirationChecker = newExpirationChecker(retention.NewExpirationChecker(limits), c.deleteRequestsManager)
return nil
}
func (c *Compactor) starting(ctx context.Context) (err error) {
// In case this function will return error we want to unregister the instance
// from the ring. We do it ensuring dependencies are gracefully stopped if they
// were already started.
defer func() {
if err == nil || c.subservices == nil {
return
}
if stopErr := services.StopManagerAndAwaitStopped(context.Background(), c.subservices); stopErr != nil {
level.Error(util_log.Logger).Log("msg", "failed to gracefully stop compactor dependencies", "err", stopErr)
}
}()
if err := services.StartManagerAndAwaitHealthy(ctx, c.subservices); err != nil {
return errors.Wrap(err, "unable to start compactor subservices")
}
// The BasicLifecycler does not automatically move state to ACTIVE such that any additional work that
// someone wants to do can be done before becoming ACTIVE. For the query compactor we don't currently
// have any additional work so we can become ACTIVE right away.
// Wait until the ring client detected this instance in the JOINING state to
// make sure that when we'll run the initial sync we already know the tokens
// assigned to this instance.
level.Info(util_log.Logger).Log("msg", "waiting until compactor is JOINING in the ring")
if err := ring.WaitInstanceState(ctx, c.ring, c.ringLifecycler.GetInstanceID(), ring.JOINING); err != nil {
return err
}
level.Info(util_log.Logger).Log("msg", "compactor is JOINING in the ring")
// Change ring state to ACTIVE
if err = c.ringLifecycler.ChangeState(ctx, ring.ACTIVE); err != nil {
return errors.Wrapf(err, "switch instance to %s in the ring", ring.ACTIVE)
}
// Wait until the ring client detected this instance in the ACTIVE state to
// make sure that when we'll run the loop it won't be detected as a ring
// topology change.
level.Info(util_log.Logger).Log("msg", "waiting until compactor is ACTIVE in the ring")
if err := ring.WaitInstanceState(ctx, c.ring, c.ringLifecycler.GetInstanceID(), ring.ACTIVE); err != nil {
return err
}
level.Info(util_log.Logger).Log("msg", "compactor is ACTIVE in the ring")
return nil
}
func (c *Compactor) loop(ctx context.Context) error {
if c.cfg.RunOnce {
level.Info(util_log.Logger).Log("msg", "running single compaction")
err := c.RunCompaction(ctx, false)
if err != nil {
level.Error(util_log.Logger).Log("msg", "compaction encountered an error", "err", err)
}
level.Info(util_log.Logger).Log("msg", "single compaction finished")
level.Info(util_log.Logger).Log("msg", "interrupt or terminate the process to finish")
// Wait for Loki to shutdown.
<-ctx.Done()
level.Info(util_log.Logger).Log("msg", "compactor exiting")
return nil
}
if c.cfg.RetentionEnabled {
if c.deleteRequestsStore != nil {
defer c.deleteRequestsStore.Stop()
}
if c.deleteRequestsManager != nil {
defer c.deleteRequestsManager.Stop()
}
}
syncTicker := time.NewTicker(c.ringPollPeriod)
defer syncTicker.Stop()
var runningCtx context.Context
var runningCancel context.CancelFunc
for {
select {
case <-ctx.Done():
if runningCancel != nil {
runningCancel()
}
c.wg.Wait()
level.Info(util_log.Logger).Log("msg", "compactor exiting")
return nil
case <-syncTicker.C:
bufDescs, bufHosts, bufZones := ring.MakeBuffersForGet()
rs, err := c.ring.Get(ringKeyOfLeader, ring.Write, bufDescs, bufHosts, bufZones)
if err != nil {
level.Error(util_log.Logger).Log("msg", "error asking ring for who should run the compactor, will check again", "err", err)
continue
}
addrs := rs.GetAddresses()
if len(addrs) != 1 {
level.Error(util_log.Logger).Log("msg", "too many addresses (more that one) return when asking the ring who should run the compactor, will check again")
continue
}
if c.ringLifecycler.GetInstanceAddr() == addrs[0] {
// If not running, start
if !c.running {
level.Info(util_log.Logger).Log("msg", "this instance has been chosen to run the compactor, starting compactor")
runningCtx, runningCancel = context.WithCancel(ctx)
go c.runCompactions(runningCtx)
c.running = true
c.metrics.compactorRunning.Set(1)
}
} else {
// If running, shutdown
if c.running {
level.Info(util_log.Logger).Log("msg", "this instance should no longer run the compactor, stopping compactor")
runningCancel()
c.wg.Wait()
c.running = false
c.metrics.compactorRunning.Set(0)
level.Info(util_log.Logger).Log("msg", "compactor stopped")
}
}
}
}
}
func (c *Compactor) runCompactions(ctx context.Context) {
// To avoid races, wait 1 compaction interval before actually starting the compactor
// this allows the ring to settle if there are a lot of ring changes and gives
// time for existing compactors to shutdown before this starts to avoid
// multiple compactors running at the same time.
func() {
t := time.NewTimer(c.cfg.CompactionInterval)
defer t.Stop()
level.Info(util_log.Logger).Log("msg", fmt.Sprintf("waiting %v for ring to stay stable and previous compactions to finish before starting compactor", c.cfg.CompactionInterval))
select {
case <-ctx.Done():
return
case <-t.C:
level.Info(util_log.Logger).Log("msg", "compactor startup delay completed")
break
}
}()
lastRetentionRunAt := time.Unix(0, 0)
runCompaction := func() {
applyRetention := false
if c.cfg.RetentionEnabled && time.Since(lastRetentionRunAt) >= c.cfg.ApplyRetentionInterval {
level.Info(util_log.Logger).Log("msg", "applying retention with compaction")
applyRetention = true
}
err := c.RunCompaction(ctx, applyRetention)
if err != nil {
level.Error(util_log.Logger).Log("msg", "failed to run compaction", "err", err)
}
if applyRetention {
lastRetentionRunAt = time.Now()
}
}
c.wg.Add(1)
go func() {
defer c.wg.Done()
runCompaction()
ticker := time.NewTicker(c.cfg.CompactionInterval)
defer ticker.Stop()
for {
select {
case <-ticker.C:
runCompaction()
case <-ctx.Done():
return
}
}
}()
if c.cfg.RetentionEnabled {
for _, container := range c.storeContainers {
c.wg.Add(1)
go func(sc storeContainer) {
// starts the chunk sweeper
defer func() {
sc.sweeper.Stop()
c.wg.Done()
}()
sc.sweeper.Start()
<-ctx.Done()
}(container)
}
}
level.Info(util_log.Logger).Log("msg", "compactor started")
}
func (c *Compactor) stopping(_ error) error {
return services.StopManagerAndAwaitStopped(context.Background(), c.subservices)
}
func (c *Compactor) CompactTable(ctx context.Context, tableName string, applyRetention bool) error {
schemaCfg, ok := schemaPeriodForTable(c.schemaConfig, tableName)
if !ok {
level.Error(util_log.Logger).Log("msg", "skipping compaction since we can't find schema for table", "table", tableName)
return nil
}
indexCompactor, ok := c.indexCompactors[schemaCfg.IndexType]
if !ok {
return fmt.Errorf("index processor not found for index type %s", schemaCfg.IndexType)
}
sc, ok := c.storeContainers[schemaCfg.From]
if !ok {
return fmt.Errorf("index store client not found for period starting at %s", schemaCfg.From.String())
}
table, err := newTable(ctx, filepath.Join(c.cfg.WorkingDirectory, tableName), sc.indexStorageClient, indexCompactor,
schemaCfg, sc.tableMarker, c.expirationChecker, c.cfg.UploadParallelism)
if err != nil {
level.Error(util_log.Logger).Log("msg", "failed to initialize table for compaction", "table", tableName, "err", err)
return err
}
interval := retention.ExtractIntervalFromTableName(tableName)
intervalMayHaveExpiredChunks := false
if applyRetention {
intervalMayHaveExpiredChunks = c.expirationChecker.IntervalMayHaveExpiredChunks(interval, "")
}
err = table.compact(intervalMayHaveExpiredChunks)
if err != nil {
level.Error(util_log.Logger).Log("msg", "failed to compact files", "table", tableName, "err", err)
return err
}
return nil
}
func (c *Compactor) RegisterIndexCompactor(indexType string, indexCompactor IndexCompactor) {
c.indexCompactors[indexType] = indexCompactor
}
func (c *Compactor) RunCompaction(ctx context.Context, applyRetention bool) error {
status := statusSuccess
start := time.Now()
if applyRetention {
c.expirationChecker.MarkPhaseStarted()
}
defer func() {
c.metrics.compactTablesOperationTotal.WithLabelValues(status).Inc()
runtime := time.Since(start)
if status == statusSuccess {
c.metrics.compactTablesOperationDurationSeconds.Set(runtime.Seconds())
c.metrics.compactTablesOperationLastSuccess.SetToCurrentTime()
if applyRetention {
c.metrics.applyRetentionLastSuccess.SetToCurrentTime()
}
}
if applyRetention {
if status == statusSuccess {
c.expirationChecker.MarkPhaseFinished()
} else {
c.expirationChecker.MarkPhaseFailed()
}
}
if runtime > c.cfg.CompactionInterval {
level.Warn(util_log.Logger).Log("msg", fmt.Sprintf("last compaction took %s which is longer than the compaction interval of %s, this can lead to duplicate compactors running if not running a standalone compactor instance.", runtime, c.cfg.CompactionInterval))
}
}()
var (
tables []string
// it possible for two periods to use the same storage bucket and path prefix (different indexType or schema version)
// so more than one index storage client may end up listing the same set of buckets
// avoid including the same table twice in the compact tables list.
seen = make(map[string]struct{})
)
for _, sc := range c.storeContainers {
// refresh index list cache since previous compaction would have changed the index files in the object store
sc.indexStorageClient.RefreshIndexTableNamesCache(ctx)
tbls, err := sc.indexStorageClient.ListTables(ctx)
if err != nil {
status = statusFailure
return fmt.Errorf("failed to list tables: %w", err)
}
for _, table := range tbls {
if _, ok := seen[table]; ok {
continue
}
tables = append(tables, table)
seen[table] = struct{}{}
}
}
// process most recent tables first
sortTablesByRange(tables)
// apply passed in compaction limits
if c.cfg.SkipLatestNTables <= len(tables) {
tables = tables[c.cfg.SkipLatestNTables:]
}
if c.cfg.TablesToCompact > 0 && c.cfg.TablesToCompact < len(tables) {
tables = tables[:c.cfg.TablesToCompact]
}
compactTablesChan := make(chan string)
errChan := make(chan error)
for i := 0; i < c.cfg.MaxCompactionParallelism; i++ {
go func() {
var err error
defer func() {
errChan <- err
}()
for {
select {
case tableName, ok := <-compactTablesChan:
if !ok {
return
}
level.Info(util_log.Logger).Log("msg", "compacting table", "table-name", tableName)
err = c.CompactTable(ctx, tableName, applyRetention)
if err != nil {
return
}
level.Info(util_log.Logger).Log("msg", "finished compacting table", "table-name", tableName)
case <-ctx.Done():
return
}
}
}()
}
go func() {
for _, tableName := range tables {
if tableName == deletion.DeleteRequestsTableName {
// we do not want to compact or apply retention on delete requests table
continue
}
select {
case compactTablesChan <- tableName:
case <-ctx.Done():
return
}
}
close(compactTablesChan)
}()
var firstErr error
// read all the errors
for i := 0; i < c.cfg.MaxCompactionParallelism; i++ {
err := <-errChan
if err != nil && firstErr == nil {
status = statusFailure
firstErr = err
}
}
return firstErr
}
type expirationChecker struct {
retentionExpiryChecker retention.ExpirationChecker
deletionExpiryChecker retention.ExpirationChecker
}
func newExpirationChecker(retentionExpiryChecker, deletionExpiryChecker retention.ExpirationChecker) retention.ExpirationChecker {
return &expirationChecker{retentionExpiryChecker, deletionExpiryChecker}
}
func (e *expirationChecker) Expired(ref retention.ChunkEntry, now model.Time) (bool, filter.Func) {
if expired, nonDeletedIntervals := e.retentionExpiryChecker.Expired(ref, now); expired {
return expired, nonDeletedIntervals
}
return e.deletionExpiryChecker.Expired(ref, now)
}
func (e *expirationChecker) MarkPhaseStarted() {
e.retentionExpiryChecker.MarkPhaseStarted()
e.deletionExpiryChecker.MarkPhaseStarted()
}
func (e *expirationChecker) MarkPhaseFailed() {
e.retentionExpiryChecker.MarkPhaseFailed()
e.deletionExpiryChecker.MarkPhaseFailed()
}
func (e *expirationChecker) MarkPhaseFinished() {
e.retentionExpiryChecker.MarkPhaseFinished()
e.deletionExpiryChecker.MarkPhaseFinished()
}
func (e *expirationChecker) MarkPhaseTimedOut() {
e.retentionExpiryChecker.MarkPhaseTimedOut()
e.deletionExpiryChecker.MarkPhaseTimedOut()
}
func (e *expirationChecker) IntervalMayHaveExpiredChunks(interval model.Interval, userID string) bool {
return e.retentionExpiryChecker.IntervalMayHaveExpiredChunks(interval, userID) || e.deletionExpiryChecker.IntervalMayHaveExpiredChunks(interval, userID)
}
func (e *expirationChecker) DropFromIndex(ref retention.ChunkEntry, tableEndTime model.Time, now model.Time) bool {
return e.retentionExpiryChecker.DropFromIndex(ref, tableEndTime, now) || e.deletionExpiryChecker.DropFromIndex(ref, tableEndTime, now)
}
func (c *Compactor) OnRingInstanceRegister(_ *ring.BasicLifecycler, ringDesc ring.Desc, instanceExists bool, _ string, instanceDesc ring.InstanceDesc) (ring.InstanceState, ring.Tokens) {
// When we initialize the compactor instance in the ring we want to start from
// a clean situation, so whatever is the state we set it JOINING, while we keep existing
// tokens (if any) or the ones loaded from file.
var tokens []uint32
if instanceExists {
tokens = instanceDesc.GetTokens()
}
takenTokens := ringDesc.GetTokens()
gen := ring.NewRandomTokenGenerator()
newTokens := gen.GenerateTokens(ringNumTokens-len(tokens), takenTokens)
// Tokens sorting will be enforced by the parent caller.
tokens = append(tokens, newTokens...)
return ring.JOINING, tokens
}
func (c *Compactor) OnRingInstanceTokens(_ *ring.BasicLifecycler, _ ring.Tokens) {}
func (c *Compactor) OnRingInstanceStopping(_ *ring.BasicLifecycler) {}
func (c *Compactor) OnRingInstanceHeartbeat(_ *ring.BasicLifecycler, _ *ring.Desc, _ *ring.InstanceDesc) {
}
func (c *Compactor) ServeHTTP(w http.ResponseWriter, req *http.Request) {
c.ring.ServeHTTP(w, req)
}
func sortTablesByRange(tables []string) {
tableRanges := make(map[string]model.Interval)
for _, table := range tables {
tableRanges[table] = retention.ExtractIntervalFromTableName(table)
}
sort.Slice(tables, func(i, j int) bool {
// less than if start time is after produces a most recent first sort order
return tableRanges[tables[i]].Start.After(tableRanges[tables[j]].Start)
})
}
func schemaPeriodForTable(cfg config.SchemaConfig, tableName string) (config.PeriodConfig, bool) {
tableInterval := retention.ExtractIntervalFromTableName(tableName)
schemaCfg, err := cfg.SchemaForTime(tableInterval.Start)
if err != nil || schemaCfg.IndexTables.TableFor(tableInterval.Start) != tableName {
return config.PeriodConfig{}, false
}
return schemaCfg, true
}