-
Notifications
You must be signed in to change notification settings - Fork 693
Expand file tree
/
Copy pathcompaction.go
More file actions
490 lines (413 loc) · 15.3 KB
/
compaction.go
File metadata and controls
490 lines (413 loc) · 15.3 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
package provider
import (
"container/heap"
"context"
"flag"
"time"
"github.com/go-kit/log"
"github.com/go-kit/log/level"
"github.com/google/uuid"
"github.com/grafana/tempo/modules/backendscheduler/work"
"github.com/grafana/tempo/modules/backendscheduler/work/tenantselector"
"github.com/grafana/tempo/modules/overrides"
"github.com/grafana/tempo/modules/storage"
"github.com/grafana/tempo/pkg/tempopb"
"github.com/grafana/tempo/pkg/util"
"github.com/grafana/tempo/tempodb"
"github.com/grafana/tempo/tempodb/backend"
"github.com/grafana/tempo/tempodb/blockselector"
"go.opentelemetry.io/otel"
"go.opentelemetry.io/otel/attribute"
"go.opentelemetry.io/otel/codes"
"go.opentelemetry.io/otel/trace"
)
var tracer = otel.Tracer("modules/backendscheduler/provider/compaction")
type CompactionConfig struct {
MeasureInterval time.Duration `yaml:"measure_interval"`
Compactor tempodb.CompactorConfig `yaml:"compaction"`
MaxJobsPerTenant int `yaml:"max_jobs_per_tenant"`
MinInputBlocks int `yaml:"min_input_blocks"`
MaxInputBlocks int `yaml:"max_input_blocks"`
MaxCompactionLevel int `yaml:"max_compaction_level"`
MinCycleInterval time.Duration `yaml:"min_cycle_interval"`
}
func (cfg *CompactionConfig) RegisterFlagsAndApplyDefaults(prefix string, f *flag.FlagSet) {
f.DurationVar(&cfg.MeasureInterval, prefix+"backend-scheduler.compaction-provider.measure-interval", time.Minute, "Interval at which to metric tenant blocklist")
f.IntVar(&cfg.MaxJobsPerTenant, prefix+"backend-scheduler.max-jobs-per-tenant", 1000, "Maximum number of jobs to run per tenant before moving on to the next tenant")
// Compaction
f.IntVar(&cfg.MinInputBlocks, prefix+".min-input-blocks", blockselector.DefaultMinInputBlocks, "Minimum number of blocks to compact in a single job.")
f.IntVar(&cfg.MaxInputBlocks, prefix+".max-input-blocks", blockselector.DefaultMaxInputBlocks, "Maximum number of blocks to compact in a single job.")
f.IntVar(&cfg.MaxCompactionLevel, prefix+".max-compaction-level", blockselector.DefaultMaxCompactionLevel, "Maximum compaction level to include in compaction jobs. 0 means no limit.")
// Tenant prioritization
f.DurationVar(&cfg.MinCycleInterval, prefix+".min-cycle-interval", 30*time.Second, "Minimum time between tenant prioritization cycles to prevent excessive CPU usage when no work is available.")
cfg.Compactor = tempodb.CompactorConfig{}
cfg.Compactor.RegisterFlagsAndApplyDefaults(util.PrefixConfig(prefix, "compaction"), f)
}
type CompactionProvider struct {
cfg CompactionConfig
logger log.Logger
// Dependencies needed for compaction job selection
store storage.Store
overrides overrides.Interface
// Scheduler calls required for this provider
sched Scheduler
// Dependencies needed for tenant selection
curPriority *tenantselector.PriorityQueue
curTenant *tenantselector.Item
curSelector blockselector.CompactionBlockSelector
lastPrioritizeTime time.Time
}
func NewCompactionProvider(
cfg CompactionConfig,
logger log.Logger,
store storage.Store,
overrides overrides.Interface,
scheduler Scheduler,
) *CompactionProvider {
return &CompactionProvider{
cfg: cfg,
logger: logger,
store: store,
overrides: overrides,
curPriority: tenantselector.NewPriorityQueue(),
sched: scheduler,
}
}
func (p *CompactionProvider) Start(ctx context.Context) <-chan *work.Job {
jobs := make(chan *work.Job, 1)
go func() {
defer close(jobs)
level.Info(p.logger).Log("msg", "compaction provider started")
var (
job *work.Job
curTenantJobCount int
span trace.Span
loopCtx context.Context
spanStarted bool
drained bool // Signal that we have drained the current work and should wait for the next poll
)
reset := func() {
metricTenantReset.WithLabelValues(p.curTenant.Value()).Inc()
span.AddEvent("tenant reset", trace.WithAttributes(
attribute.String("tenant_id", p.curTenant.Value()),
attribute.Int("job_count", curTenantJobCount),
))
p.curSelector = nil
p.curTenant = nil
curTenantJobCount = 0
span.End()
spanStarted = false
}
level.Info(p.logger).Log("msg", "compaction provider waiting for poll notification")
<-p.store.PollNotification(ctx)
for {
if ctx.Err() != nil {
level.Info(p.logger).Log("msg", "compaction provider stopping")
return
}
if !spanStarted {
loopCtx, span = tracer.Start(ctx, "compactionProviderLoop")
spanStarted = true
}
if p.curSelector == nil {
if !p.prepareNextTenant(loopCtx, drained) {
level.Info(p.logger).Log("msg", "received empty tenant")
// If we don't have a tenant with enough blocks, we signal drained to wait for next poll.
drained = true
metricEmptyTenantCycle.Inc()
span.AddEvent("no tenant selected")
} else {
// A tenant with enough blocks was selected, reset the drained state.
drained = false
}
continue
}
if curTenantJobCount >= p.cfg.MaxJobsPerTenant {
level.Info(p.logger).Log("msg", "max jobs per tenant reached, skipping to next tenant")
span.AddEvent("max jobs per tenant reached")
reset()
continue
}
job = p.createJob(loopCtx)
if job == nil {
level.Info(p.logger).Log("msg", "tenant exhausted, skipping to next tenant")
span.AddEvent("tenant exhausted")
// we don't have a job, reset the curTenant and try again
metricTenantEmptyJob.Inc()
reset()
continue
}
// Re-check after job creation: a batch may have been submitted while we were
// draining the selector that was built before the submission. Discard the job
// before it enters the recent-jobs cache or the channel.
// TenantPending is true for the full batch lifetime (submission → cleanup),
// which subsumes the HasJobsForTenant(REDACTION) condition.
if p.sched.TenantPending(p.curTenant.Value()) {
level.Info(p.logger).Log("msg", "redaction batch submitted for tenant since selector was built; abandoning remaining compaction jobs", "tenant", p.curTenant.Value())
span.AddEvent("tenant has active redaction batch")
reset()
continue
}
// Register the job so SubmitRedaction can see its input blocks
// before the job is promoted to active via AddJob.
p.sched.RegisterJob(job)
select {
case <-ctx.Done():
level.Info(p.logger).Log("msg", "compaction provider stopping")
span.AddEvent("context done")
span.End()
return
case jobs <- job:
metricJobsCreated.WithLabelValues(p.curTenant.Value()).Inc()
curTenantJobCount++
span.AddEvent("job created", trace.WithAttributes(
attribute.String("job_id", job.ID),
attribute.String("tenant_id", p.curTenant.Value()),
))
span.End()
spanStarted = false
}
}
}()
// Measure the tenants to get their current compaction status in a separate
// goroutine to avoid blocking the main job loop.
go func() {
measureTicker := time.NewTicker(p.cfg.MeasureInterval)
defer measureTicker.Stop()
for {
select {
case <-ctx.Done():
level.Info(p.logger).Log("msg", "compaction provider measure ticker stopping")
return
case <-measureTicker.C:
p.measureTenants()
}
}
}()
return jobs
}
func (p *CompactionProvider) prepareNextTenant(ctx context.Context, drained bool) bool {
_, span := tracer.Start(ctx, "prepareNextTenant")
defer span.End()
if p.curPriority.Len() == 0 {
// Rate limit calls to prioritizeTenants to prevent excessive CPU usage
// when cycling through tenants with no available work.
//
// We only expect new work for tenants after a the next blocklist poll. If
// we have been drained, wait for the next poll.
if drained {
level.Debug(p.logger).Log("msg", "rate limiting tenant prioritization; waiting for next poll")
select {
case <-ctx.Done():
return false
case <-p.store.PollNotification(ctx):
// We waited for the poll, but we may have been cancelled in the meantime.
if ctx.Err() != nil {
return false
}
// Continue to prioritizeTenants
}
}
if elapsed := time.Since(p.lastPrioritizeTime); elapsed < p.cfg.MinCycleInterval {
level.Debug(p.logger).Log("msg", "rate limiting tenant prioritization; waiting")
time.Sleep(p.cfg.MinCycleInterval - elapsed)
// Continue to prioritizeTenants
}
p.prioritizeTenants(ctx)
p.lastPrioritizeTime = time.Now()
if p.curPriority.Len() == 0 {
return false
}
}
p.curTenant = heap.Pop(p.curPriority).(*tenantselector.Item)
if p.curTenant == nil {
span.AddEvent("no more tenants to compact")
return false
}
level.Info(p.logger).Log("msg", "new tenant selected", "tenant_id", p.curTenant.Value())
p.curSelector, _ = p.newBlockSelector(p.curTenant.Value())
return true
}
func (p *CompactionProvider) createJob(ctx context.Context) *work.Job {
_, span := tracer.Start(ctx, "createJob")
defer span.End()
span.SetAttributes(attribute.String("tenant_id", p.curTenant.Value()))
input, ok := p.getNextBlockIDs(ctx)
if !ok {
span.AddEvent("not-enough-input-blocks", trace.WithAttributes(
attribute.Int("input_blocks", len(input)),
))
span.SetStatus(codes.Error, "not enough input blocks for compaction")
return nil
}
span.AddEvent("input blocks selected", trace.WithAttributes(
attribute.Int("input_blocks", len(input)),
attribute.StringSlice("input_block_ids", input),
))
span.SetStatus(codes.Ok, "compaction job created")
return &work.Job{
ID: uuid.New().String(),
Type: tempopb.JobType_JOB_TYPE_COMPACTION,
JobDetail: tempopb.JobDetail{
Tenant: p.curTenant.Value(),
Compaction: &tempopb.CompactionDetail{Input: input},
},
}
}
func (p *CompactionProvider) getNextBlockIDs(_ context.Context) ([]string, bool) {
ids := make([]string, 0, p.cfg.MaxInputBlocks)
toBeCompacted, _ := p.curSelector.BlocksToCompact()
if len(toBeCompacted) == 0 {
return nil, false
}
for _, b := range toBeCompacted {
ids = append(ids, b.BlockID.String())
}
return ids, len(ids) >= p.cfg.MinInputBlocks
}
// prioritizeTenants prioritizes tenants based on the number of outstanding blocks.
func (p *CompactionProvider) prioritizeTenants(ctx context.Context) {
tenants := []tenantselector.Tenant{}
_, span := tracer.Start(ctx, "prioritizeTenants")
defer span.End()
p.curPriority = tenantselector.NewPriorityQueue() // wipe and restart
var (
blocklistLen int
blockSelector blockselector.CompactionBlockSelector
outstandingBlocks int
toBeCompacted []*backend.BlockMeta
)
for _, tenantID := range p.store.Tenants() {
if p.overrides.CompactionDisabled(tenantID) {
continue
}
outstandingBlocks = 0
clear(toBeCompacted)
blockSelector, blocklistLen = p.newBlockSelector(tenantID)
// Measure the outstanding blocks
for {
toBeCompacted, _ = blockSelector.BlocksToCompact()
if len(toBeCompacted) == 0 {
span.AddEvent("no-more-blocks-to-compact", trace.WithAttributes(
attribute.String("tenant_id", tenantID),
attribute.Int("outstanding_blocks", len(toBeCompacted)),
))
break
}
outstandingBlocks += len(toBeCompacted)
span.AddEvent("found-blocks-to-compact", trace.WithAttributes(
attribute.String("tenant_id", tenantID),
attribute.Int("outstanding_blocks", len(toBeCompacted)),
))
}
tenants = append(tenants, tenantselector.Tenant{
ID: tenantID,
BlocklistLength: blocklistLen,
OutstandingBlocklistLength: outstandingBlocks,
})
}
var (
ts = tenantselector.NewBlockListWeightedTenantSelector(tenants)
item *tenantselector.Item
priority int
)
for _, tenant := range tenants {
priority = ts.PriorityForTenant(tenant.ID)
if priority >= p.cfg.MinInputBlocks {
item = tenantselector.NewItem(tenant.ID, priority)
heap.Push(p.curPriority, item)
}
}
}
func (p *CompactionProvider) measureTenants() {
_, span := tracer.Start(context.Background(), "measureTenants")
defer span.End()
owns := func(_ string) bool {
return true
}
var blockSelector blockselector.CompactionBlockSelector
for _, tenant := range p.store.Tenants() {
// Use the measurement selector, which ignores TenantPending, so that the
// outstanding-blocks metric reflects real work even during an active
// redaction batch. Autoscaling must not see zero blocks just because
// compaction is gated for the tenant.
blockSelector, _ = p.newBlockSelectorForMeasurement(tenant)
tempodb.MeasureOutstandingBlocks(tenant, blockSelector, owns)
}
}
// newBlockSelectorForMeasurement builds a block selector from the full tenant
// blocklist without the TenantPending guard that newBlockSelector applies.
// Used exclusively by measureTenants so that the outstanding-blocks metric
// continues to reflect real work even while a redaction batch is active.
// The selector produced here is never used to create compaction jobs.
func (p *CompactionProvider) newBlockSelectorForMeasurement(tenantID string) (blockselector.CompactionBlockSelector, int) {
var (
fullBlocklist = p.store.BlockMetas(tenantID)
window = p.overrides.MaxCompactionRange(tenantID)
blocklist = make([]*backend.BlockMeta, 0, len(fullBlocklist))
)
busyBlocks := p.sched.BusyBlocksForTenant(tenantID)
for _, block := range fullBlocklist {
if _, ok := busyBlocks[block.BlockID.String()]; ok {
continue
}
blocklist = append(blocklist, block)
}
if window == 0 {
window = p.cfg.Compactor.MaxCompactionRange
}
return blockselector.NewTimeWindowBlockSelector(
blocklist,
window,
p.cfg.Compactor.MaxCompactionObjects,
p.cfg.Compactor.MaxBlockBytes,
p.cfg.MinInputBlocks,
p.cfg.MaxInputBlocks,
p.cfg.MaxCompactionLevel,
), len(blocklist)
}
func (p *CompactionProvider) newBlockSelector(tenantID string) (blockselector.CompactionBlockSelector, int) {
// Do not start new compaction jobs for a tenant with an active redaction batch.
// Compacting blocks during a live batch would produce output blocks not yet
// covered by any pending redaction job. This guarantees that at most one rescan
// is needed per batch: once the originally-skipped compaction jobs finish and
// the rescan fires, no further compaction can have created uncovered blocks.
if p.sched.TenantPending(tenantID) {
return blockselector.NewTimeWindowBlockSelector(nil,
p.cfg.Compactor.MaxCompactionRange,
p.cfg.Compactor.MaxCompactionObjects,
p.cfg.Compactor.MaxBlockBytes,
p.cfg.MinInputBlocks,
p.cfg.MaxInputBlocks,
p.cfg.MaxCompactionLevel,
), 0
}
var (
fullBlocklist = p.store.BlockMetas(tenantID)
window = p.overrides.MaxCompactionRange(tenantID)
blocklist = make([]*backend.BlockMeta, 0, len(fullBlocklist))
)
// Take a single snapshot of all busy blocks for this tenant — one lock
// acquisition regardless of blocklist size.
busyBlocks := p.sched.BusyBlocksForTenant(tenantID)
// Build the filtered blocklist, skipping blocks already busy
// (pending redaction, active compaction input, etc.).
for _, block := range fullBlocklist {
if _, ok := busyBlocks[block.BlockID.String()]; ok {
continue
}
blocklist = append(blocklist, block)
}
if window == 0 {
window = p.cfg.Compactor.MaxCompactionRange
}
return blockselector.NewTimeWindowBlockSelector(
blocklist,
window,
p.cfg.Compactor.MaxCompactionObjects,
p.cfg.Compactor.MaxBlockBytes,
p.cfg.MinInputBlocks,
p.cfg.MaxInputBlocks,
p.cfg.MaxCompactionLevel,
), len(blocklist)
}