From 912172b0c7c7fe60e359954e4da03c755fd55a7d Mon Sep 17 00:00:00 2001 From: Filip Petkovski Date: Thu, 1 Dec 2022 15:24:00 +0100 Subject: [PATCH 01/22] Parallelize vertical compactions The current compaction implementation allocates one goroutine per compaction stream. This means that compaction can run as fast as the slowest stream. As a result, as soon as one stream starts to fall behind, the all other streams can become affected. In addition, despite setting a high compact-concurrency, CPU utilization can still be low because of the one-goroutine-per-stream limit. The compaction algorithm also prioritizes vertical compactions over horizontal ones. As soon as it detects any overlapping blocks, it will compact those blocks and reevaluate the plan in a subsequent iteration. This commit enables parallel execution of vertical compactions within a single compaction stream. It does that by first changing the Planner interface to allow it to return multiple compaction tasks per group instead of a single one. It also adapts the algorithm for detecting overlapping blocks to be able to detect multiple independent groups. These groups are then returned as distinct compaction tasks and the compactor can execute them in separate goroutines. By modifying the planner interface, this commit also enables parallelizing horizontal compactions in the future. Signed-off-by: Filip Petkovski --- pkg/compact/compact.go | 152 +++++++++++++++++---------------- pkg/compact/planner.go | 164 ++++++++++++++++++++++-------------- pkg/compact/planner_test.go | 163 ++++++++++++++++++++--------------- 3 files changed, 276 insertions(+), 203 deletions(-) diff --git a/pkg/compact/compact.go b/pkg/compact/compact.go index c6c36bf3cc..a56562b863 100644 --- a/pkg/compact/compact.go +++ b/pkg/compact/compact.go @@ -542,26 +542,28 @@ func (ps *CompactionProgressCalculator) ProgressCalculate(ctx context.Context, g if len(plan) == 0 { continue } - groupCompactions[g.key]++ - toRemove := make(map[ulid.ULID]struct{}, len(plan)) - metas := make([]*tsdb.BlockMeta, 0, len(plan)) - for _, p := range plan { - metas = append(metas, &p.BlockMeta) - toRemove[p.BlockMeta.ULID] = struct{}{} - } - g.deleteFromGroup(toRemove) + for _, groupTask := range plan { + groupCompactions[g.key]++ + toRemove := make(map[ulid.ULID]struct{}, len(groupTask)) + metas := make([]*tsdb.BlockMeta, 0, len(groupTask)) + for _, meta := range groupTask { + metas = append(metas, &meta.BlockMeta) + toRemove[meta.BlockMeta.ULID] = struct{}{} + } + g.deleteFromGroup(toRemove) + groupBlocks[g.key] += len(groupTask) - groupBlocks[g.key] += len(plan) + newMeta := tsdb.CompactBlockMetas(ulid.MustNew(uint64(time.Now().Unix()), nil), metas...) + if err := g.AppendMeta(&metadata.Meta{BlockMeta: *newMeta, Thanos: metadata.Thanos{Downsample: metadata.ThanosDownsample{Resolution: g.Resolution()}, Labels: g.Labels().Map()}}); err != nil { + return errors.Wrapf(err, "append meta") + } + } if len(g.metasByMinTime) == 0 { continue } - newMeta := tsdb.CompactBlockMetas(ulid.MustNew(uint64(time.Now().Unix()), nil), metas...) - if err := g.AppendMeta(&metadata.Meta{BlockMeta: *newMeta, Thanos: metadata.Thanos{Downsample: metadata.ThanosDownsample{Resolution: g.Resolution()}, Labels: g.Labels().Map()}}); err != nil { - return errors.Wrapf(err, "append meta") - } tmpGroups = append(tmpGroups, g) } @@ -727,7 +729,7 @@ func (rs *RetentionProgressCalculator) ProgressCalculate(ctx context.Context, gr type Planner interface { // Plan returns a list of blocks that should be compacted into single one. // The blocks can be overlapping. The provided metadata has to be ordered by minTime. - Plan(ctx context.Context, metasByMinTime []*metadata.Meta) ([]*metadata.Meta, error) + Plan(ctx context.Context, metasByMinTime []*metadata.Meta) ([]compactionTask, error) } // Compactor provides compaction against an underlying storage of time series data. @@ -990,19 +992,19 @@ func (cg *Group) compact(ctx context.Context, dir string, planner Planner, comp overlappingBlocks = true } - var toCompact []*metadata.Meta + var tasks []compactionTask if err := tracing.DoInSpanWithErr(ctx, "compaction_planning", func(ctx context.Context) (e error) { - toCompact, e = planner.Plan(ctx, cg.metasByMinTime) + tasks, e = planner.Plan(ctx, cg.metasByMinTime) return e }); err != nil { return false, ulid.ULID{}, errors.Wrap(err, "plan compaction") } - if len(toCompact) == 0 { + if len(tasks) == 0 { // Nothing to do. return false, ulid.ULID{}, nil } - level.Info(cg.logger).Log("msg", "compaction available and planned; downloading blocks", "plan", fmt.Sprintf("%v", toCompact)) + level.Info(cg.logger).Log("msg", "compaction available and planned; downloading blocks", "plan", fmt.Sprintf("%v", tasks)) // Due to #183 we verify that none of the blocks in the plan have overlapping sources. // This is one potential source of how we could end up with duplicated chunks. @@ -1014,53 +1016,55 @@ func (cg *Group) compact(ctx context.Context, dir string, planner Planner, comp g, errCtx := errgroup.WithContext(ctx) g.SetLimit(cg.compactBlocksFetchConcurrency) - toCompactDirs := make([]string, 0, len(toCompact)) - for _, m := range toCompact { - bdir := filepath.Join(dir, m.ULID.String()) - for _, s := range m.Compaction.Sources { - if _, ok := uniqueSources[s]; ok { - return false, ulid.ULID{}, halt(errors.Errorf("overlapping sources detected for plan %v", toCompact)) - } - uniqueSources[s] = struct{}{} - } - func(ctx context.Context, meta *metadata.Meta) { - g.Go(func() error { - if err := tracing.DoInSpanWithErr(ctx, "compaction_block_download", func(ctx context.Context) error { - return block.Download(ctx, cg.logger, cg.bkt, meta.ULID, bdir, objstore.WithFetchConcurrency(cg.blockFilesConcurrency)) - }, opentracing.Tags{"block.id": meta.ULID}); err != nil { - return retry(errors.Wrapf(err, "download block %s", meta.ULID)) + toCompactDirs := make([]string, 0, len(tasks)) + for _, task := range tasks { + for _, m := range task { + bdir := filepath.Join(dir, m.ULID.String()) + for _, s := range m.Compaction.Sources { + if _, ok := uniqueSources[s]; ok { + return false, ulid.ULID{}, halt(errors.Errorf("overlapping sources detected for plan %v", tasks)) } + uniqueSources[s] = struct{}{} + } + func(ctx context.Context, meta *metadata.Meta) { + g.Go(func() error { + if err := tracing.DoInSpanWithErr(ctx, "compaction_block_download", func(ctx context.Context) error { + return block.Download(ctx, cg.logger, cg.bkt, meta.ULID, bdir, objstore.WithFetchConcurrency(cg.blockFilesConcurrency)) + }, opentracing.Tags{"block.id": meta.ULID}); err != nil { + return retry(errors.Wrapf(err, "download block %s", meta.ULID)) + } - // Ensure all input blocks are valid. - var stats block.HealthStats - if err := tracing.DoInSpanWithErr(ctx, "compaction_block_health_stats", func(ctx context.Context) (e error) { - stats, e = block.GatherIndexHealthStats(cg.logger, filepath.Join(bdir, block.IndexFilename), meta.MinTime, meta.MaxTime) - return e - }, opentracing.Tags{"block.id": meta.ULID}); err != nil { - return errors.Wrapf(err, "gather index issues for block %s", bdir) - } + // Ensure all input blocks are valid. + var stats block.HealthStats + if err := tracing.DoInSpanWithErr(ctx, "compaction_block_health_stats", func(ctx context.Context) (e error) { + stats, e = block.GatherIndexHealthStats(cg.logger, filepath.Join(bdir, block.IndexFilename), meta.MinTime, meta.MaxTime) + return e + }, opentracing.Tags{"block.id": meta.ULID}); err != nil { + return errors.Wrapf(err, "gather index issues for block %s", bdir) + } - if err := stats.CriticalErr(); err != nil { - return halt(errors.Wrapf(err, "block with not healthy index found %s; Compaction level %v; Labels: %v", bdir, meta.Compaction.Level, meta.Thanos.Labels)) - } + if err := stats.CriticalErr(); err != nil { + return halt(errors.Wrapf(err, "block with not healthy index found %s; Compaction level %v; Labels: %v", bdir, meta.Compaction.Level, meta.Thanos.Labels)) + } - if err := stats.OutOfOrderChunksErr(); err != nil { - return outOfOrderChunkError(errors.Wrapf(err, "blocks with out-of-order chunks are dropped from compaction: %s", bdir), meta.ULID) - } + if err := stats.OutOfOrderChunksErr(); err != nil { + return outOfOrderChunkError(errors.Wrapf(err, "blocks with out-of-order chunks are dropped from compaction: %s", bdir), meta.ULID) + } - if err := stats.Issue347OutsideChunksErr(); err != nil { - return issue347Error(errors.Wrapf(err, "invalid, but reparable block %s", bdir), meta.ULID) - } + if err := stats.Issue347OutsideChunksErr(); err != nil { + return issue347Error(errors.Wrapf(err, "invalid, but reparable block %s", bdir), meta.ULID) + } - if err := stats.OutOfOrderLabelsErr(); !cg.acceptMalformedIndex && err != nil { - return errors.Wrapf(err, - "block id %s, try running with --debug.accept-malformed-index", meta.ULID) - } - return nil - }) - }(errCtx, m) + if err := stats.OutOfOrderLabelsErr(); !cg.acceptMalformedIndex && err != nil { + return errors.Wrapf(err, + "block id %s, try running with --debug.accept-malformed-index", meta.ULID) + } + return nil + }) + }(errCtx, m) - toCompactDirs = append(toCompactDirs, bdir) + toCompactDirs = append(toCompactDirs, bdir) + } } sourceBlockStr := fmt.Sprintf("%v", toCompactDirs) @@ -1080,10 +1084,12 @@ func (cg *Group) compact(ctx context.Context, dir string, planner Planner, comp if compID == (ulid.ULID{}) { // Prometheus compactor found that the compacted block would have no samples. level.Info(cg.logger).Log("msg", "compacted block would have no samples, deleting source blocks", "blocks", sourceBlockStr) - for _, meta := range toCompact { - if meta.Stats.NumSamples == 0 { - if err := cg.deleteBlock(meta.ULID, filepath.Join(dir, meta.ULID.String())); err != nil { - level.Warn(cg.logger).Log("msg", "failed to mark for deletion an empty block found during compaction", "block", meta.ULID) + for _, task := range tasks { + for _, meta := range task { + if meta.Stats.NumSamples == 0 { + if err := cg.deleteBlock(meta.ULID, filepath.Join(dir, meta.ULID.String())); err != nil { + level.Warn(cg.logger).Log("msg", "failed to mark for deletion an empty block found during compaction", "block", meta.ULID) + } } } } @@ -1125,8 +1131,10 @@ func (cg *Group) compact(ctx context.Context, dir string, planner Planner, comp // Ensure the output block is not overlapping with anything else, // unless vertical compaction is enabled. if !cg.enableVerticalCompaction { - if err := cg.areBlocksOverlapping(newMeta, toCompact...); err != nil { - return false, ulid.ULID{}, halt(errors.Wrapf(err, "resulted compacted block %s overlaps with something", bdir)) + for _, task := range tasks { + if err := cg.areBlocksOverlapping(newMeta, task...); err != nil { + return false, ulid.ULID{}, halt(errors.Wrapf(err, "resulted compacted block %s overlaps with something", bdir)) + } } } @@ -1143,14 +1151,16 @@ func (cg *Group) compact(ctx context.Context, dir string, planner Planner, comp // Mark for deletion the blocks we just compacted from the group and bucket so they do not get included // into the next planning cycle. // Eventually the block we just uploaded should get synced into the group again (including sync-delay). - for _, meta := range toCompact { - err = tracing.DoInSpanWithErr(ctx, "compaction_block_delete", func(ctx context.Context) error { - return cg.deleteBlock(meta.ULID, filepath.Join(dir, meta.ULID.String())) - }, opentracing.Tags{"block.id": meta.ULID}) - if err != nil { - return false, ulid.ULID{}, retry(errors.Wrapf(err, "mark old block for deletion from bucket")) + for _, task := range tasks { + for _, meta := range task { + err = tracing.DoInSpanWithErr(ctx, "compaction_block_delete", func(ctx context.Context) error { + return cg.deleteBlock(meta.ULID, filepath.Join(dir, meta.ULID.String())) + }, opentracing.Tags{"block.id": meta.ULID}) + if err != nil { + return false, ulid.ULID{}, retry(errors.Wrapf(err, "mark old block for deletion from bucket")) + } + cg.groupGarbageCollectedBlocks.Inc() } - cg.groupGarbageCollectedBlocks.Inc() } level.Info(cg.logger).Log("msg", "finished compacting blocks", "result_block", compID, "source_blocks", sourceBlockStr, diff --git a/pkg/compact/planner.go b/pkg/compact/planner.go index 5c2a93df8d..6d9c66a5af 100644 --- a/pkg/compact/planner.go +++ b/pkg/compact/planner.go @@ -8,6 +8,7 @@ import ( "fmt" "math" "path/filepath" + "sort" "github.com/go-kit/log" "github.com/oklog/ulid" @@ -49,11 +50,13 @@ func NewPlanner(logger log.Logger, ranges []int64, noCompBlocks *GatherNoCompact } // TODO(bwplotka): Consider smarter algorithm, this prefers smaller iterative compactions vs big single one: https://github.com/thanos-io/thanos/issues/3405 -func (p *tsdbBasedPlanner) Plan(_ context.Context, metasByMinTime []*metadata.Meta) ([]*metadata.Meta, error) { +func (p *tsdbBasedPlanner) Plan(_ context.Context, metasByMinTime []*metadata.Meta) ([]compactionTask, error) { return p.plan(p.noCompBlocksFunc(), metasByMinTime) } -func (p *tsdbBasedPlanner) plan(noCompactMarked map[ulid.ULID]*metadata.NoCompactMark, metasByMinTime []*metadata.Meta) ([]*metadata.Meta, error) { +type compactionTask []*metadata.Meta + +func (p *tsdbBasedPlanner) plan(noCompactMarked map[ulid.ULID]*metadata.NoCompactMark, metasByMinTime []*metadata.Meta) ([]compactionTask, error) { notExcludedMetasByMinTime := make([]*metadata.Meta, 0, len(metasByMinTime)) for _, meta := range metasByMinTime { if _, excluded := noCompactMarked[meta.ULID]; excluded { @@ -62,9 +65,9 @@ func (p *tsdbBasedPlanner) plan(noCompactMarked map[ulid.ULID]*metadata.NoCompac notExcludedMetasByMinTime = append(notExcludedMetasByMinTime, meta) } - res := selectOverlappingMetas(notExcludedMetasByMinTime) - if len(res) > 0 { - return res, nil + verticalCompactions := selectOverlappingMetas(notExcludedMetasByMinTime) + if len(verticalCompactions) > 0 { + return verticalCompactions, nil } // No overlapping blocks, do compaction the usual way. @@ -74,9 +77,9 @@ func (p *tsdbBasedPlanner) plan(noCompactMarked map[ulid.ULID]*metadata.NoCompac notExcludedMetasByMinTime = notExcludedMetasByMinTime[:len(notExcludedMetasByMinTime)-1] } metasByMinTime = metasByMinTime[:len(metasByMinTime)-1] - res = append(res, selectMetas(p.ranges, noCompactMarked, metasByMinTime)...) + res := selectMetas(p.ranges, noCompactMarked, metasByMinTime) if len(res) > 0 { - return res, nil + return []compactionTask{res}, nil } // Compact any blocks with big enough time range that have >5% tombstones. @@ -86,7 +89,8 @@ func (p *tsdbBasedPlanner) plan(noCompactMarked map[ulid.ULID]*metadata.NoCompac break } if float64(meta.Stats.NumTombstones)/float64(meta.Stats.NumSeries+1) > 0.05 { - return []*metadata.Meta{notExcludedMetasByMinTime[i]}, nil + task := []*metadata.Meta{notExcludedMetasByMinTime[i]} + return []compactionTask{task}, nil } } @@ -155,28 +159,62 @@ func selectMetas(ranges []int64, noCompactMarked map[ulid.ULID]*metadata.NoCompa // selectOverlappingMetas returns all dirs with overlapping time ranges. // It expects sorted input by mint and returns the overlapping dirs in the same order as received. // Copied and adjusted from https://github.com/prometheus/prometheus/blob/3d8826a3d42566684283a9b7f7e812e412c24407/tsdb/compact.go#L268. -func selectOverlappingMetas(metasByMinTime []*metadata.Meta) []*metadata.Meta { +func selectOverlappingMetas(metasByMinTime []*metadata.Meta) []compactionTask { if len(metasByMinTime) < 2 { return nil } - var overlappingMetas []*metadata.Meta - globalMaxt := metasByMinTime[0].MaxTime - for i, m := range metasByMinTime[1:] { - if m.MinTime < globalMaxt { - if len(overlappingMetas) == 0 { - // When it is the first overlap, need to add the last one as well. - overlappingMetas = append(overlappingMetas, metasByMinTime[i]) + + groups := make([][]*metadata.Meta, len(metasByMinTime)) + for i, m := range metasByMinTime { + groups[i] = []*metadata.Meta{m} + } + + // Iterate through all metas and merge overlapping blocks into groups. + // We compare each block's max time with the next block's min time. + // If we detect an overlap, we merge all blocks from the current block's group + // into the group of the next block. + // We also adjust the head of the next group 's head(zero-th element) to be the block with the + // highest max-time. This allows groups to be used as heaps, and helps us detect + // overlaps when the next block is contained in the current block. + // See test case https://github.com/thanos-io/thanos/blob/04106d7a7add7f47025c00422c80f746650c1b97/pkg/compact/planner_test.go#L310-L321. +loopMetas: + for i := range metasByMinTime { + currentGroup := groups[i] + currentBlock := currentGroup[0] + for j := i + 1; j < len(groups); j++ { + if currentBlock.MaxTime <= groups[j][0].MinTime { + continue } - overlappingMetas = append(overlappingMetas, m) - } else if len(overlappingMetas) > 0 { - break + // If the current block has an overlap with the next block, + // merge the current block's group of the overlapping block's group. + for _, blockToMerge := range currentGroup { + groups[j] = append(groups[j], blockToMerge) + // Set the block with the highest max time as the head of the group. + if blockToMerge.MaxTime > groups[j][0].MaxTime { + n := len(groups[j]) - 1 + groups[j][0], groups[j][n] = groups[j][n], groups[j][0] + } + } + // Empty the current block's group. + groups[i] = nil + + // Move on to the next block. + continue loopMetas } + } - if m.MaxTime > globalMaxt { - globalMaxt = m.MaxTime + overlappingGroups := make([]compactionTask, 0, len(groups)) + for _, group := range groups { + if len(group) < 2 { + continue } + sort.Slice(group, func(i, j int) bool { + return group[i].MinTime < group[j].MinTime + }) + overlappingGroups = append(overlappingGroups, group) } - return overlappingMetas + + return overlappingGroups } // splitByRange splits the directories by the time range. The range sequence starts at 0. @@ -243,7 +281,7 @@ func WithLargeTotalIndexSizeFilter(with *tsdbBasedPlanner, bkt objstore.Bucket, return &largeTotalIndexSizeFilter{tsdbBasedPlanner: with, bkt: bkt, totalMaxIndexSizeBytes: totalMaxIndexSizeBytes, markedForNoCompact: markedForNoCompact} } -func (t *largeTotalIndexSizeFilter) Plan(ctx context.Context, metasByMinTime []*metadata.Meta) ([]*metadata.Meta, error) { +func (t *largeTotalIndexSizeFilter) Plan(ctx context.Context, metasByMinTime []*metadata.Meta) ([]compactionTask, error) { noCompactMarked := t.noCompBlocksFunc() copiedNoCompactMarked := make(map[ulid.ULID]*metadata.NoCompactMark, len(noCompactMarked)) for k, v := range noCompactMarked { @@ -252,54 +290,56 @@ func (t *largeTotalIndexSizeFilter) Plan(ctx context.Context, metasByMinTime []* PlanLoop: for { - plan, err := t.plan(copiedNoCompactMarked, metasByMinTime) + tasks, err := t.plan(copiedNoCompactMarked, metasByMinTime) if err != nil { return nil, err } - var totalIndexBytes, maxIndexSize int64 = 0, math.MinInt64 - var biggestIndex int - for i, p := range plan { - indexSize := int64(-1) - for _, f := range p.Thanos.Files { - if f.RelPath == block.IndexFilename { - indexSize = f.SizeBytes + for _, task := range tasks { + var totalIndexBytes, maxIndexSize int64 = 0, math.MinInt64 + var biggestIndex int + for i, meta := range task { + indexSize := int64(-1) + for _, f := range meta.Thanos.Files { + if f.RelPath == block.IndexFilename { + indexSize = f.SizeBytes + } } - } - if indexSize <= 0 { - // Get size from bkt instead. - attr, err := t.bkt.Attributes(ctx, filepath.Join(p.ULID.String(), block.IndexFilename)) - if err != nil { - return nil, errors.Wrapf(err, "get attr of %v", filepath.Join(p.ULID.String(), block.IndexFilename)) + if indexSize <= 0 { + // Get size from bkt instead. + attr, err := t.bkt.Attributes(ctx, filepath.Join(meta.ULID.String(), block.IndexFilename)) + if err != nil { + return nil, errors.Wrapf(err, "get attr of %v", filepath.Join(meta.ULID.String(), block.IndexFilename)) + } + indexSize = attr.Size } - indexSize = attr.Size - } - if maxIndexSize < indexSize { - maxIndexSize = indexSize - biggestIndex = i - } - totalIndexBytes += indexSize - // Leave 15% headroom for index compaction bloat. - if totalIndexBytes >= int64(float64(t.totalMaxIndexSizeBytes)*0.85) { - // Marking blocks for no compact to limit size. - // TODO(bwplotka): Make sure to reset cache once this is done: https://github.com/thanos-io/thanos/issues/3408 - if err := block.MarkForNoCompact( - ctx, - t.logger, - t.bkt, - plan[biggestIndex].ULID, - metadata.IndexSizeExceedingNoCompactReason, - fmt.Sprintf("largeTotalIndexSizeFilter: Total compacted block's index size could exceed: %v with this block. See https://github.com/thanos-io/thanos/issues/1424", t.totalMaxIndexSizeBytes), - t.markedForNoCompact, - ); err != nil { - return nil, errors.Wrapf(err, "mark %v for no compaction", plan[biggestIndex].ULID.String()) + if maxIndexSize < indexSize { + maxIndexSize = indexSize + biggestIndex = i + } + totalIndexBytes += indexSize + // Leave 15% headroom for index compaction bloat. + if totalIndexBytes >= int64(float64(t.totalMaxIndexSizeBytes)*0.85) { + // Marking blocks for no compact to limit size. + // TODO(bwplotka): Make sure to reset cache once this is done: https://github.com/thanos-io/thanos/issues/3408 + if err := block.MarkForNoCompact( + ctx, + t.logger, + t.bkt, + task[biggestIndex].ULID, + metadata.IndexSizeExceedingNoCompactReason, + fmt.Sprintf("largeTotalIndexSizeFilter: Total compacted block's index size could exceed: %v with this block. See https://github.com/thanos-io/thanos/issues/1424", t.totalMaxIndexSizeBytes), + t.markedForNoCompact, + ); err != nil { + return nil, errors.Wrapf(err, "mark %v for no compaction", task[biggestIndex].ULID.String()) + } + // Make sure wrapped planner exclude this block. + copiedNoCompactMarked[task[biggestIndex].ULID] = &metadata.NoCompactMark{ID: task[biggestIndex].ULID, Version: metadata.NoCompactMarkVersion1} + continue PlanLoop } - // Make sure wrapped planner exclude this block. - copiedNoCompactMarked[plan[biggestIndex].ULID] = &metadata.NoCompactMark{ID: plan[biggestIndex].ULID, Version: metadata.NoCompactMarkVersion1} - continue PlanLoop } } // Planned blocks should not exceed limit. - return plan, nil + return tasks, nil } } diff --git a/pkg/compact/planner_test.go b/pkg/compact/planner_test.go index d5c253be72..0cdf48a5ec 100644 --- a/pkg/compact/planner_test.go +++ b/pkg/compact/planner_test.go @@ -30,7 +30,7 @@ type tsdbPlannerAdapter struct { comp tsdb.Compactor } -func (p *tsdbPlannerAdapter) Plan(_ context.Context, metasByMinTime []*metadata.Meta) ([]*metadata.Meta, error) { +func (p *tsdbPlannerAdapter) Plan(_ context.Context, metasByMinTime []*metadata.Meta) ([]compactionTask, error) { // TSDB planning works based on the meta.json files in the given dir. Mock it up. for _, meta := range metasByMinTime { bdir := filepath.Join(p.dir, meta.ULID.String()) @@ -46,6 +46,10 @@ func (p *tsdbPlannerAdapter) Plan(_ context.Context, metasByMinTime []*metadata. return nil, err } + if len(plan) == 0 { + return nil, nil + } + var res []*metadata.Meta for _, pdir := range plan { meta, err := metadata.ReadFromDir(pdir) @@ -54,7 +58,7 @@ func (p *tsdbPlannerAdapter) Plan(_ context.Context, metasByMinTime []*metadata. } res = append(res, meta) } - return res, nil + return []compactionTask{res}, nil } // Adapted from https://github.com/prometheus/prometheus/blob/6c56a1faaaad07317ff585bda75b99bdba0517ad/tsdb/compact_test.go#L167 @@ -76,7 +80,7 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { for _, c := range []struct { name string metas []*metadata.Meta - expected []*metadata.Meta + expected []compactionTask }{ { name: "Outside range", @@ -108,11 +112,11 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 40, MaxTime: 60}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(4, nil), MinTime: 60, MaxTime: 80}}, }, - expected: []*metadata.Meta{ + expected: []compactionTask{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(1, nil), MinTime: 0, MaxTime: 20}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 20, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 40, MaxTime: 60}}, - }, + }}, }, { name: "There are blocks to fill the entire 2nd parent range.", @@ -123,10 +127,12 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(9, nil), MinTime: 180, MaxTime: 200}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(10, nil), MinTime: 200, MaxTime: 220}}, }, - expected: []*metadata.Meta{ - {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(6, nil), MinTime: 0, MaxTime: 60}}, - {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(7, nil), MinTime: 60, MaxTime: 120}}, - {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(8, nil), MinTime: 120, MaxTime: 180}}, + expected: []compactionTask{ + { + {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(6, nil), MinTime: 0, MaxTime: 60}}, + {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(7, nil), MinTime: 60, MaxTime: 120}}, + {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(8, nil), MinTime: 120, MaxTime: 180}}, + }, }, }, { @@ -147,10 +153,10 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(4, nil), MinTime: 60, MaxTime: 80}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(5, nil), MinTime: 80, MaxTime: 100}}, }, - expected: []*metadata.Meta{ + expected: []compactionTask{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(1, nil), MinTime: 0, MaxTime: 20}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 20, MaxTime: 40}}, - }, + }}, }, { name: "We have 20, 20, 20, 60, 60 range blocks. '5' is marked as fresh one", @@ -161,11 +167,11 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(4, nil), MinTime: 60, MaxTime: 120}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(5, nil), MinTime: 120, MaxTime: 180}}, }, - expected: []*metadata.Meta{ + expected: []compactionTask{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(1, nil), MinTime: 0, MaxTime: 20}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 20, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 40, MaxTime: 60}}, - }, + }}, }, { name: "There are blocks to fill the entire 2nd parent range, but there is a gap", @@ -175,10 +181,10 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(9, nil), MinTime: 180, MaxTime: 200}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(10, nil), MinTime: 200, MaxTime: 220}}, }, - expected: []*metadata.Meta{ + expected: []compactionTask{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(6, nil), MinTime: 0, MaxTime: 60}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(8, nil), MinTime: 120, MaxTime: 180}}, - }, + }}, }, { name: "We have 20, 60, 20, 60, 240 range blocks. We can compact 20 + 60 + 60", @@ -189,20 +195,18 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(6, nil), MinTime: 120, MaxTime: 180}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(7, nil), MinTime: 720, MaxTime: 960}}, }, - expected: []*metadata.Meta{ + expected: []compactionTask{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 20, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(4, nil), MinTime: 60, MaxTime: 120}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(6, nil), MinTime: 120, MaxTime: 180}}, - }, + }}, }, { name: "Do not select large blocks that have many tombstones when there is no fresh block", - metas: []*metadata.Meta{ - {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(1, nil), MinTime: 0, MaxTime: 540, Stats: tsdb.BlockStats{ - NumSeries: 10, - NumTombstones: 3, - }}}, - }, + metas: []*metadata.Meta{{BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(1, nil), MinTime: 0, MaxTime: 540, Stats: tsdb.BlockStats{ + NumSeries: 10, + NumTombstones: 3, + }}}}, }, { name: "Select large blocks that have many tombstones when fresh appears", @@ -213,10 +217,12 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { }}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 540, MaxTime: 560}}, }, - expected: []*metadata.Meta{{BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(1, nil), MinTime: 0, MaxTime: 540, Stats: tsdb.BlockStats{ - NumSeries: 10, - NumTombstones: 3, - }}}}, + expected: []compactionTask{{ + {BlockMeta: tsdb.BlockMeta{ + Version: 1, ULID: ulid.MustNew(1, nil), MinTime: 0, MaxTime: 540, Stats: tsdb.BlockStats{ + NumSeries: 10, + NumTombstones: 3, + }}}}}, }, { name: "For small blocks, do not compact tombstones, even when fresh appears.", @@ -251,10 +257,10 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(7, nil), MinTime: 360, MaxTime: 420}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(8, nil), MinTime: 420, MaxTime: 540}}, }, - expected: []*metadata.Meta{ + expected: []compactionTask{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(7, nil), MinTime: 360, MaxTime: 420}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(8, nil), MinTime: 420, MaxTime: 540}}, - }, + }}, }, // |--------------| // |----------------| @@ -266,10 +272,10 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 19, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 40, MaxTime: 60}}, }, - expected: []*metadata.Meta{ + expected: []compactionTask{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(1, nil), MinTime: 0, MaxTime: 20}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 19, MaxTime: 40}}, - }, + }}, }, // |--------------| // |--------------| @@ -281,10 +287,10 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 20, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 30, MaxTime: 50}}, }, - expected: []*metadata.Meta{ + expected: []compactionTask{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 20, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 30, MaxTime: 50}}, - }, + }}, }, // |--------------| // |---------------------| @@ -296,11 +302,11 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 10, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 30, MaxTime: 50}}, }, - expected: []*metadata.Meta{ + expected: []compactionTask{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(1, nil), MinTime: 0, MaxTime: 20}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 10, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 30, MaxTime: 50}}, - }, + }}, }, // |--------------| // |--------------------------------| @@ -314,35 +320,46 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(7, nil), MinTime: 360, MaxTime: 420}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(8, nil), MinTime: 420, MaxTime: 540}}, }, - expected: []*metadata.Meta{ + expected: []compactionTask{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(5, nil), MinTime: 0, MaxTime: 360}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(6, nil), MinTime: 340, MaxTime: 560}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(7, nil), MinTime: 360, MaxTime: 420}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(8, nil), MinTime: 420, MaxTime: 540}}, - }, + }}, }, // |--------------| // |--------------| + // |--------------| // |--------------| // |--------------| { - name: "Overlapping blocks 5", + name: "Multiple independent groups of overlapping blocks", metas: []*metadata.Meta{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(1, nil), MinTime: 0, MaxTime: 10}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 9, MaxTime: 20}}, - {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 30, MaxTime: 40}}, - {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(4, nil), MinTime: 39, MaxTime: 50}}, - }, - expected: []*metadata.Meta{ - {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(1, nil), MinTime: 0, MaxTime: 10}}, - {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 9, MaxTime: 20}}, + {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 17, MaxTime: 35}}, + {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(4, nil), MinTime: 40, MaxTime: 50}}, + {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(5, nil), MinTime: 47, MaxTime: 60}}, + }, + expected: []compactionTask{ + { + {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(1, nil), MinTime: 0, MaxTime: 10}}, + {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 9, MaxTime: 20}}, + {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 17, MaxTime: 35}}, + }, + { + {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(4, nil), MinTime: 40, MaxTime: 50}}, + {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(5, nil), MinTime: 47, MaxTime: 60}}, + }, }, }, } { t.Run(c.name, func(t *testing.T) { for _, e := range c.expected { - // Add here to avoid boilerplate. - e.Thanos.Labels = make(map[string]string) + for _, meta := range e { + // Add here to avoid boilerplate. + meta.Thanos.Labels = make(map[string]string) + } } for _, e := range c.metas { // Add here to avoid boilerplate. @@ -366,7 +383,11 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { tsdbPlanner.dir = dir plan, err := tsdbPlanner.Plan(context.Background(), metasByMinTime) testutil.Ok(t, err) - testutil.Equals(t, c.expected, plan) + if len(c.expected) == 0 { + testutil.Equals(t, len(plan), 0) + } else { + testutil.Equals(t, c.expected[0], plan[0]) + } }) t.Run("tsdbBasedPlanner", func(t *testing.T) { metasByMinTime := make([]*metadata.Meta, len(c.metas)) @@ -442,12 +463,12 @@ func TestRangeWithFailedCompactionWontGetSelected(t *testing.T) { tsdbPlanner.dir = dir plan, err := tsdbPlanner.Plan(context.Background(), c.metas) testutil.Ok(t, err) - testutil.Equals(t, []*metadata.Meta(nil), plan) + testutil.Equals(t, []compactionTask(nil), plan) }) t.Run("tsdbBasedPlanner", func(t *testing.T) { plan, err := tsdbBasedPlanner.Plan(context.Background(), c.metas) testutil.Ok(t, err) - testutil.Equals(t, []*metadata.Meta(nil), plan) + testutil.Equals(t, []compactionTask(nil), plan) }) }) } @@ -470,7 +491,7 @@ func TestTSDBBasedPlanner_PlanWithNoCompactMarks(t *testing.T) { metas []*metadata.Meta noCompactMarks map[ulid.ULID]*metadata.NoCompactMark - expected []*metadata.Meta + expected []compactionTask }{ { name: "Outside range and excluded", @@ -492,10 +513,10 @@ func TestTSDBBasedPlanner_PlanWithNoCompactMarks(t *testing.T) { noCompactMarks: map[ulid.ULID]*metadata.NoCompactMark{ ulid.MustNew(1, nil): {}, }, - expected: []*metadata.Meta{ + expected: []compactionTask{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 20, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 40, MaxTime: 60}}, - }, + }}, }, { name: "Blocks to fill the entire parent, but with second one excluded.", @@ -520,11 +541,11 @@ func TestTSDBBasedPlanner_PlanWithNoCompactMarks(t *testing.T) { noCompactMarks: map[ulid.ULID]*metadata.NoCompactMark{ ulid.MustNew(4, nil): {}, }, - expected: []*metadata.Meta{ + expected: []compactionTask{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(1, nil), MinTime: 0, MaxTime: 20}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 20, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 40, MaxTime: 60}}, - }, + }}, }, { name: "Blocks to fill the entire parent, but with last one fist excluded.", @@ -538,10 +559,10 @@ func TestTSDBBasedPlanner_PlanWithNoCompactMarks(t *testing.T) { ulid.MustNew(1, nil): {}, ulid.MustNew(4, nil): {}, }, - expected: []*metadata.Meta{ + expected: []compactionTask{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 20, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 40, MaxTime: 60}}, - }, + }}, }, { name: "Blocks to fill the entire parent, but with all of them excluded.", @@ -583,10 +604,10 @@ func TestTSDBBasedPlanner_PlanWithNoCompactMarks(t *testing.T) { noCompactMarks: map[ulid.ULID]*metadata.NoCompactMark{ ulid.MustNew(6, nil): {}, }, - expected: []*metadata.Meta{ + expected: []compactionTask{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 20, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(4, nil), MinTime: 60, MaxTime: 120}}, - }, + }}, }, { name: "We have 20, 60, 20, 60, 240 range blocks. We could compact 20 + 60 + 60, but 4th is excluded", @@ -664,7 +685,7 @@ func TestLargeTotalIndexSizeFilter_Plan(t *testing.T) { name string metas []*metadata.Meta - expected []*metadata.Meta + expected []compactionTask expectedMarks float64 }{ { @@ -688,10 +709,10 @@ func TestLargeTotalIndexSizeFilter_Plan(t *testing.T) { BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(4, nil), MinTime: 60, MaxTime: 80}}, }, expectedMarks: 1, - expected: []*metadata.Meta{ + expected: []compactionTask{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 20, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 40, MaxTime: 60}}, - }, + }}, }, { name: "Blocks to fill the entire parent, but with second one too large.", @@ -719,11 +740,11 @@ func TestLargeTotalIndexSizeFilter_Plan(t *testing.T) { {Thanos: metadata.Thanos{Files: []metadata.File{{RelPath: block.IndexFilename, SizeBytes: 90}}}, BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(4, nil), MinTime: 60, MaxTime: 80}}, }, - expected: []*metadata.Meta{ + expected: []compactionTask{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(1, nil), MinTime: 0, MaxTime: 20}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 20, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 40, MaxTime: 60}}, - }, + }}, }, { name: "Blocks to fill the entire parent, but with pre-last one and first too large.", @@ -739,10 +760,10 @@ func TestLargeTotalIndexSizeFilter_Plan(t *testing.T) { {Thanos: metadata.Thanos{Files: []metadata.File{{RelPath: block.IndexFilename, SizeBytes: 90}}}, BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(5, nil), MinTime: 60, MaxTime: 80}}, }, - expected: []*metadata.Meta{ + expected: []compactionTask{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 20, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 40, MaxTime: 50}}, - }, + }}, expectedMarks: 2, }, { @@ -774,10 +795,10 @@ func TestLargeTotalIndexSizeFilter_Plan(t *testing.T) { {Thanos: metadata.Thanos{Files: []metadata.File{{RelPath: block.IndexFilename, SizeBytes: 30}}}, BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(7, nil), MinTime: 720, MaxTime: 960}}, }, - expected: []*metadata.Meta{ + expected: []compactionTask{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 20, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(4, nil), MinTime: 60, MaxTime: 120}}, - }, + }}, expectedMarks: 1, }, // |--------------| @@ -817,9 +838,11 @@ func TestLargeTotalIndexSizeFilter_Plan(t *testing.T) { plan, err := planner.Plan(context.Background(), metasByMinTime) testutil.Ok(t, err) - for _, m := range plan { - // For less boilerplate. - m.Thanos = metadata.Thanos{} + for _, task := range plan { + for _, m := range task { + // For less boilerplate. + m.Thanos = metadata.Thanos{} + } } testutil.Equals(t, c.expected, plan) testutil.Equals(t, c.expectedMarks, promtest.ToFloat64(marked)-lastMarkValue) From d71c4d1b48f0a09a4899fb5a64f39327e6387f36 Mon Sep 17 00:00:00 2001 From: Filip Petkovski Date: Fri, 2 Dec 2022 07:56:05 +0100 Subject: [PATCH 02/22] Run all tasks independently Signed-off-by: Filip Petkovski --- pkg/compact/compact.go | 178 ++++++++++++++++++++---------------- pkg/compact/planner.go | 20 ++-- pkg/compact/planner_test.go | 64 ++++++------- 3 files changed, 144 insertions(+), 118 deletions(-) diff --git a/pkg/compact/compact.go b/pkg/compact/compact.go index a56562b863..9d8f20a243 100644 --- a/pkg/compact/compact.go +++ b/pkg/compact/compact.go @@ -25,6 +25,7 @@ import ( "github.com/thanos-io/objstore" "golang.org/x/sync/errgroup" + "github.com/thanos-io/thanos/internal/cortex/util/multierror" "github.com/thanos-io/thanos/pkg/block" "github.com/thanos-io/thanos/pkg/block/metadata" "github.com/thanos-io/thanos/pkg/compact/downsample" @@ -729,7 +730,7 @@ func (rs *RetentionProgressCalculator) ProgressCalculate(ctx context.Context, gr type Planner interface { // Plan returns a list of blocks that should be compacted into single one. // The blocks can be overlapping. The provided metadata has to be ordered by minTime. - Plan(ctx context.Context, metasByMinTime []*metadata.Meta) ([]compactionTask, error) + Plan(ctx context.Context, metasByMinTime []*metadata.Meta) ([]CompactionTask, error) } // Compactor provides compaction against an underlying storage of time series data. @@ -753,7 +754,7 @@ type Compactor interface { // Compact plans and runs a single compaction against the group. The compacted result // is uploaded into the bucket the blocks were retrieved from. -func (cg *Group) Compact(ctx context.Context, dir string, planner Planner, comp Compactor) (shouldRerun bool, compID ulid.ULID, rerr error) { +func (cg *Group) Compact(ctx context.Context, dir string, planner Planner, comp Compactor) (shouldRerun bool, rerr error) { cg.compactionRunsStarted.Inc() subDir := filepath.Join(dir, cg.Key()) @@ -770,19 +771,19 @@ func (cg *Group) Compact(ctx context.Context, dir string, planner Planner, comp }() if err := os.MkdirAll(subDir, 0750); err != nil { - return false, ulid.ULID{}, errors.Wrap(err, "create compaction group dir") + return false, errors.Wrap(err, "create compaction group dir") } err := tracing.DoInSpanWithErr(ctx, "compaction_group", func(ctx context.Context) (err error) { - shouldRerun, compID, err = cg.compact(ctx, subDir, planner, comp) + shouldRerun, err = cg.compact(ctx, subDir, planner, comp) return err }, opentracing.Tags{"group.key": cg.Key()}) if err != nil { cg.compactionFailures.Inc() - return false, ulid.ULID{}, err + return false, err } cg.compactionRunsCompleted.Inc() - return shouldRerun, compID, nil + return shouldRerun, nil } // Issue347Error is a type wrapper for errors that should invoke repair process for broken block. @@ -976,7 +977,7 @@ func RepairIssue347(ctx context.Context, logger log.Logger, bkt objstore.Bucket, return nil } -func (cg *Group) compact(ctx context.Context, dir string, planner Planner, comp Compactor) (shouldRerun bool, compID ulid.ULID, _ error) { +func (cg *Group) compact(ctx context.Context, dir string, planner Planner, comp Compactor) (shouldRerun bool, _ error) { cg.mtx.Lock() defer cg.mtx.Unlock() @@ -986,86 +987,114 @@ func (cg *Group) compact(ctx context.Context, dir string, planner Planner, comp // TODO(bwplotka): It would really nice if we could still check for other overlaps than replica. In fact this should be checked // in syncer itself. Otherwise with vertical compaction enabled we will sacrifice this important check. if !cg.enableVerticalCompaction { - return false, ulid.ULID{}, halt(errors.Wrap(err, "pre compaction overlap check")) + return false, halt(errors.Wrap(err, "pre compaction overlap check")) } overlappingBlocks = true } - var tasks []compactionTask + var tasks []CompactionTask if err := tracing.DoInSpanWithErr(ctx, "compaction_planning", func(ctx context.Context) (e error) { tasks, e = planner.Plan(ctx, cg.metasByMinTime) return e }); err != nil { - return false, ulid.ULID{}, errors.Wrap(err, "plan compaction") + return false, errors.Wrap(err, "plan compaction") } if len(tasks) == 0 { // Nothing to do. - return false, ulid.ULID{}, nil + return false, nil } - level.Info(cg.logger).Log("msg", "compaction available and planned; downloading blocks", "plan", fmt.Sprintf("%v", tasks)) - // Due to #183 we verify that none of the blocks in the plan have overlapping sources. // This is one potential source of how we could end up with duplicated chunks. uniqueSources := map[ulid.ULID]struct{}{} - - // Once we have a plan we need to download the actual data. - groupCompactionBegin := time.Now() - begin := groupCompactionBegin - g, errCtx := errgroup.WithContext(ctx) - g.SetLimit(cg.compactBlocksFetchConcurrency) - - toCompactDirs := make([]string, 0, len(tasks)) for _, task := range tasks { for _, m := range task { - bdir := filepath.Join(dir, m.ULID.String()) for _, s := range m.Compaction.Sources { if _, ok := uniqueSources[s]; ok { - return false, ulid.ULID{}, halt(errors.Errorf("overlapping sources detected for plan %v", tasks)) + return false, halt(errors.Errorf("overlapping sources detected for plan %v", task)) } uniqueSources[s] = struct{}{} } - func(ctx context.Context, meta *metadata.Meta) { - g.Go(func() error { - if err := tracing.DoInSpanWithErr(ctx, "compaction_block_download", func(ctx context.Context) error { - return block.Download(ctx, cg.logger, cg.bkt, meta.ULID, bdir, objstore.WithFetchConcurrency(cg.blockFilesConcurrency)) - }, opentracing.Tags{"block.id": meta.ULID}); err != nil { - return retry(errors.Wrapf(err, "download block %s", meta.ULID)) - } + } + } - // Ensure all input blocks are valid. - var stats block.HealthStats - if err := tracing.DoInSpanWithErr(ctx, "compaction_block_health_stats", func(ctx context.Context) (e error) { - stats, e = block.GatherIndexHealthStats(cg.logger, filepath.Join(bdir, block.IndexFilename), meta.MinTime, meta.MaxTime) - return e - }, opentracing.Tags{"block.id": meta.ULID}); err != nil { - return errors.Wrapf(err, "gather index issues for block %s", bdir) - } + level.Info(cg.logger).Log("msg", "compaction available and planned; downloading blocks", "plan", fmt.Sprintf("%v", tasks)) - if err := stats.CriticalErr(); err != nil { - return halt(errors.Wrapf(err, "block with not healthy index found %s; Compaction level %v; Labels: %v", bdir, meta.Compaction.Level, meta.Thanos.Labels)) - } + var ( + wg sync.WaitGroup + mu sync.Mutex + groupErr multierror.MultiError + rerunGroup bool + ) + for _, task := range tasks { + wg.Add(1) + go func(task CompactionTask) { + defer wg.Done() + rerunTask, _, err := cg.compactBlocks(ctx, dir, task, comp, overlappingBlocks) - if err := stats.OutOfOrderChunksErr(); err != nil { - return outOfOrderChunkError(errors.Wrapf(err, "blocks with out-of-order chunks are dropped from compaction: %s", bdir), meta.ULID) - } + mu.Lock() + defer mu.Unlock() + rerunGroup = rerunGroup || rerunTask + groupErr.Add(err) + }(task) + } + wg.Wait() - if err := stats.Issue347OutsideChunksErr(); err != nil { - return issue347Error(errors.Wrapf(err, "invalid, but reparable block %s", bdir), meta.ULID) - } + return rerunGroup, groupErr.Err() +} - if err := stats.OutOfOrderLabelsErr(); !cg.acceptMalformedIndex && err != nil { - return errors.Wrapf(err, - "block id %s, try running with --debug.accept-malformed-index", meta.ULID) - } - return nil - }) - }(errCtx, m) +func (cg *Group) compactBlocks(ctx context.Context, dir string, task CompactionTask, comp Compactor, overlappingBlocks bool) (bool, ulid.ULID, error) { + // Once we have a plan we need to download the actual data. + compactionBegin := time.Now() + begin := compactionBegin - toCompactDirs = append(toCompactDirs, bdir) - } + g, errCtx := errgroup.WithContext(ctx) + g.SetLimit(cg.compactBlocksFetchConcurrency) + + toCompactDirs := make([]string, 0, len(task)) + for _, m := range task { + bdir := filepath.Join(dir, m.ULID.String()) + func(ctx context.Context, meta *metadata.Meta) { + g.Go(func() error { + if err := tracing.DoInSpanWithErr(ctx, "compaction_block_download", func(ctx context.Context) error { + return block.Download(ctx, cg.logger, cg.bkt, meta.ULID, bdir, objstore.WithFetchConcurrency(cg.blockFilesConcurrency)) + }, opentracing.Tags{"block.id": meta.ULID}); err != nil { + return retry(errors.Wrapf(err, "download block %s", meta.ULID)) + } + + // Ensure all input blocks are valid. + var stats block.HealthStats + if err := tracing.DoInSpanWithErr(ctx, "compaction_block_health_stats", func(ctx context.Context) (e error) { + stats, e = block.GatherIndexHealthStats(cg.logger, filepath.Join(bdir, block.IndexFilename), meta.MinTime, meta.MaxTime) + return e + }, opentracing.Tags{"block.id": meta.ULID}); err != nil { + return errors.Wrapf(err, "gather index issues for block %s", bdir) + } + + if err := stats.CriticalErr(); err != nil { + return halt(errors.Wrapf(err, "block with not healthy index found %s; Compaction level %v; Labels: %v", bdir, meta.Compaction.Level, meta.Thanos.Labels)) + } + + if err := stats.OutOfOrderChunksErr(); err != nil { + return outOfOrderChunkError(errors.Wrapf(err, "blocks with out-of-order chunks are dropped from compaction: %s", bdir), meta.ULID) + } + + if err := stats.Issue347OutsideChunksErr(); err != nil { + return issue347Error(errors.Wrapf(err, "invalid, but reparable block %s", bdir), meta.ULID) + } + + if err := stats.OutOfOrderLabelsErr(); !cg.acceptMalformedIndex && err != nil { + return errors.Wrapf(err, + "block id %s, try running with --debug.accept-malformed-index", meta.ULID) + } + return nil + }) + }(errCtx, m) + + toCompactDirs = append(toCompactDirs, bdir) } + sourceBlockStr := fmt.Sprintf("%v", toCompactDirs) if err := g.Wait(); err != nil { @@ -1074,6 +1103,7 @@ func (cg *Group) compact(ctx context.Context, dir string, planner Planner, comp level.Info(cg.logger).Log("msg", "downloaded and verified blocks; compacting blocks", "plan", sourceBlockStr, "duration", time.Since(begin), "duration_ms", time.Since(begin).Milliseconds()) + var compID ulid.ULID begin = time.Now() if err := tracing.DoInSpanWithErr(ctx, "compaction", func(ctx context.Context) (e error) { compID, e = comp.Compact(dir, toCompactDirs, nil) @@ -1084,12 +1114,10 @@ func (cg *Group) compact(ctx context.Context, dir string, planner Planner, comp if compID == (ulid.ULID{}) { // Prometheus compactor found that the compacted block would have no samples. level.Info(cg.logger).Log("msg", "compacted block would have no samples, deleting source blocks", "blocks", sourceBlockStr) - for _, task := range tasks { - for _, meta := range task { - if meta.Stats.NumSamples == 0 { - if err := cg.deleteBlock(meta.ULID, filepath.Join(dir, meta.ULID.String())); err != nil { - level.Warn(cg.logger).Log("msg", "failed to mark for deletion an empty block found during compaction", "block", meta.ULID) - } + for _, meta := range task { + if meta.Stats.NumSamples == 0 { + if err := cg.deleteBlock(meta.ULID, filepath.Join(dir, meta.ULID.String())); err != nil { + level.Warn(cg.logger).Log("msg", "failed to mark for deletion an empty block found during compaction", "block", meta.ULID) } } } @@ -1131,10 +1159,8 @@ func (cg *Group) compact(ctx context.Context, dir string, planner Planner, comp // Ensure the output block is not overlapping with anything else, // unless vertical compaction is enabled. if !cg.enableVerticalCompaction { - for _, task := range tasks { - if err := cg.areBlocksOverlapping(newMeta, task...); err != nil { - return false, ulid.ULID{}, halt(errors.Wrapf(err, "resulted compacted block %s overlaps with something", bdir)) - } + if err := cg.areBlocksOverlapping(newMeta, task...); err != nil { + return false, ulid.ULID{}, halt(errors.Wrapf(err, "resulted compacted block %s overlaps with something", bdir)) } } @@ -1151,20 +1177,18 @@ func (cg *Group) compact(ctx context.Context, dir string, planner Planner, comp // Mark for deletion the blocks we just compacted from the group and bucket so they do not get included // into the next planning cycle. // Eventually the block we just uploaded should get synced into the group again (including sync-delay). - for _, task := range tasks { - for _, meta := range task { - err = tracing.DoInSpanWithErr(ctx, "compaction_block_delete", func(ctx context.Context) error { - return cg.deleteBlock(meta.ULID, filepath.Join(dir, meta.ULID.String())) - }, opentracing.Tags{"block.id": meta.ULID}) - if err != nil { - return false, ulid.ULID{}, retry(errors.Wrapf(err, "mark old block for deletion from bucket")) - } - cg.groupGarbageCollectedBlocks.Inc() + for _, meta := range task { + err = tracing.DoInSpanWithErr(ctx, "compaction_block_delete", func(ctx context.Context) error { + return cg.deleteBlock(meta.ULID, filepath.Join(dir, meta.ULID.String())) + }, opentracing.Tags{"block.id": meta.ULID}) + if err != nil { + return false, ulid.ULID{}, retry(errors.Wrapf(err, "mark old block for deletion from bucket")) } + cg.groupGarbageCollectedBlocks.Inc() } level.Info(cg.logger).Log("msg", "finished compacting blocks", "result_block", compID, "source_blocks", sourceBlockStr, - "duration", time.Since(groupCompactionBegin), "duration_ms", time.Since(groupCompactionBegin).Milliseconds()) + "duration", time.Since(compactionBegin), "duration_ms", time.Since(compactionBegin).Milliseconds()) return true, compID, nil } @@ -1257,7 +1281,7 @@ func (c *BucketCompactor) Compact(ctx context.Context) (rerr error) { go func() { defer wg.Done() for g := range groupChan { - shouldRerunGroup, _, err := g.Compact(workCtx, c.compactDir, c.planner, c.comp) + shouldRerunGroup, err := g.Compact(workCtx, c.compactDir, c.planner, c.comp) if err == nil { if shouldRerunGroup { mtx.Lock() diff --git a/pkg/compact/planner.go b/pkg/compact/planner.go index 6d9c66a5af..ae59689cb3 100644 --- a/pkg/compact/planner.go +++ b/pkg/compact/planner.go @@ -50,13 +50,15 @@ func NewPlanner(logger log.Logger, ranges []int64, noCompBlocks *GatherNoCompact } // TODO(bwplotka): Consider smarter algorithm, this prefers smaller iterative compactions vs big single one: https://github.com/thanos-io/thanos/issues/3405 -func (p *tsdbBasedPlanner) Plan(_ context.Context, metasByMinTime []*metadata.Meta) ([]compactionTask, error) { +func (p *tsdbBasedPlanner) Plan(_ context.Context, metasByMinTime []*metadata.Meta) ([]CompactionTask, error) { return p.plan(p.noCompBlocksFunc(), metasByMinTime) } -type compactionTask []*metadata.Meta +// CompactionTask is a set of blocks that should be compacted together in a single compaction. +// Multiple compaction tasks can be run in parallel even inside a single compaction group. +type CompactionTask []*metadata.Meta -func (p *tsdbBasedPlanner) plan(noCompactMarked map[ulid.ULID]*metadata.NoCompactMark, metasByMinTime []*metadata.Meta) ([]compactionTask, error) { +func (p *tsdbBasedPlanner) plan(noCompactMarked map[ulid.ULID]*metadata.NoCompactMark, metasByMinTime []*metadata.Meta) ([]CompactionTask, error) { notExcludedMetasByMinTime := make([]*metadata.Meta, 0, len(metasByMinTime)) for _, meta := range metasByMinTime { if _, excluded := noCompactMarked[meta.ULID]; excluded { @@ -79,7 +81,7 @@ func (p *tsdbBasedPlanner) plan(noCompactMarked map[ulid.ULID]*metadata.NoCompac metasByMinTime = metasByMinTime[:len(metasByMinTime)-1] res := selectMetas(p.ranges, noCompactMarked, metasByMinTime) if len(res) > 0 { - return []compactionTask{res}, nil + return []CompactionTask{res}, nil } // Compact any blocks with big enough time range that have >5% tombstones. @@ -90,7 +92,7 @@ func (p *tsdbBasedPlanner) plan(noCompactMarked map[ulid.ULID]*metadata.NoCompac } if float64(meta.Stats.NumTombstones)/float64(meta.Stats.NumSeries+1) > 0.05 { task := []*metadata.Meta{notExcludedMetasByMinTime[i]} - return []compactionTask{task}, nil + return []CompactionTask{task}, nil } } @@ -159,7 +161,7 @@ func selectMetas(ranges []int64, noCompactMarked map[ulid.ULID]*metadata.NoCompa // selectOverlappingMetas returns all dirs with overlapping time ranges. // It expects sorted input by mint and returns the overlapping dirs in the same order as received. // Copied and adjusted from https://github.com/prometheus/prometheus/blob/3d8826a3d42566684283a9b7f7e812e412c24407/tsdb/compact.go#L268. -func selectOverlappingMetas(metasByMinTime []*metadata.Meta) []compactionTask { +func selectOverlappingMetas(metasByMinTime []*metadata.Meta) []CompactionTask { if len(metasByMinTime) < 2 { return nil } @@ -173,7 +175,7 @@ func selectOverlappingMetas(metasByMinTime []*metadata.Meta) []compactionTask { // We compare each block's max time with the next block's min time. // If we detect an overlap, we merge all blocks from the current block's group // into the group of the next block. - // We also adjust the head of the next group 's head(zero-th element) to be the block with the + // We also adjust the head of the next group's head (zero-th element) to be the block with the // highest max-time. This allows groups to be used as heaps, and helps us detect // overlaps when the next block is contained in the current block. // See test case https://github.com/thanos-io/thanos/blob/04106d7a7add7f47025c00422c80f746650c1b97/pkg/compact/planner_test.go#L310-L321. @@ -203,7 +205,7 @@ loopMetas: } } - overlappingGroups := make([]compactionTask, 0, len(groups)) + overlappingGroups := make([]CompactionTask, 0, len(groups)) for _, group := range groups { if len(group) < 2 { continue @@ -281,7 +283,7 @@ func WithLargeTotalIndexSizeFilter(with *tsdbBasedPlanner, bkt objstore.Bucket, return &largeTotalIndexSizeFilter{tsdbBasedPlanner: with, bkt: bkt, totalMaxIndexSizeBytes: totalMaxIndexSizeBytes, markedForNoCompact: markedForNoCompact} } -func (t *largeTotalIndexSizeFilter) Plan(ctx context.Context, metasByMinTime []*metadata.Meta) ([]compactionTask, error) { +func (t *largeTotalIndexSizeFilter) Plan(ctx context.Context, metasByMinTime []*metadata.Meta) ([]CompactionTask, error) { noCompactMarked := t.noCompBlocksFunc() copiedNoCompactMarked := make(map[ulid.ULID]*metadata.NoCompactMark, len(noCompactMarked)) for k, v := range noCompactMarked { diff --git a/pkg/compact/planner_test.go b/pkg/compact/planner_test.go index 0cdf48a5ec..c760d69271 100644 --- a/pkg/compact/planner_test.go +++ b/pkg/compact/planner_test.go @@ -30,7 +30,7 @@ type tsdbPlannerAdapter struct { comp tsdb.Compactor } -func (p *tsdbPlannerAdapter) Plan(_ context.Context, metasByMinTime []*metadata.Meta) ([]compactionTask, error) { +func (p *tsdbPlannerAdapter) Plan(_ context.Context, metasByMinTime []*metadata.Meta) ([]CompactionTask, error) { // TSDB planning works based on the meta.json files in the given dir. Mock it up. for _, meta := range metasByMinTime { bdir := filepath.Join(p.dir, meta.ULID.String()) @@ -58,7 +58,7 @@ func (p *tsdbPlannerAdapter) Plan(_ context.Context, metasByMinTime []*metadata. } res = append(res, meta) } - return []compactionTask{res}, nil + return []CompactionTask{res}, nil } // Adapted from https://github.com/prometheus/prometheus/blob/6c56a1faaaad07317ff585bda75b99bdba0517ad/tsdb/compact_test.go#L167 @@ -80,7 +80,7 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { for _, c := range []struct { name string metas []*metadata.Meta - expected []compactionTask + expected []CompactionTask }{ { name: "Outside range", @@ -112,7 +112,7 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 40, MaxTime: 60}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(4, nil), MinTime: 60, MaxTime: 80}}, }, - expected: []compactionTask{{ + expected: []CompactionTask{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(1, nil), MinTime: 0, MaxTime: 20}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 20, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 40, MaxTime: 60}}, @@ -127,7 +127,7 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(9, nil), MinTime: 180, MaxTime: 200}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(10, nil), MinTime: 200, MaxTime: 220}}, }, - expected: []compactionTask{ + expected: []CompactionTask{ { {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(6, nil), MinTime: 0, MaxTime: 60}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(7, nil), MinTime: 60, MaxTime: 120}}, @@ -153,7 +153,7 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(4, nil), MinTime: 60, MaxTime: 80}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(5, nil), MinTime: 80, MaxTime: 100}}, }, - expected: []compactionTask{{ + expected: []CompactionTask{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(1, nil), MinTime: 0, MaxTime: 20}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 20, MaxTime: 40}}, }}, @@ -167,7 +167,7 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(4, nil), MinTime: 60, MaxTime: 120}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(5, nil), MinTime: 120, MaxTime: 180}}, }, - expected: []compactionTask{{ + expected: []CompactionTask{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(1, nil), MinTime: 0, MaxTime: 20}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 20, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 40, MaxTime: 60}}, @@ -181,7 +181,7 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(9, nil), MinTime: 180, MaxTime: 200}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(10, nil), MinTime: 200, MaxTime: 220}}, }, - expected: []compactionTask{{ + expected: []CompactionTask{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(6, nil), MinTime: 0, MaxTime: 60}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(8, nil), MinTime: 120, MaxTime: 180}}, }}, @@ -195,7 +195,7 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(6, nil), MinTime: 120, MaxTime: 180}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(7, nil), MinTime: 720, MaxTime: 960}}, }, - expected: []compactionTask{{ + expected: []CompactionTask{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 20, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(4, nil), MinTime: 60, MaxTime: 120}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(6, nil), MinTime: 120, MaxTime: 180}}, @@ -217,7 +217,7 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { }}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 540, MaxTime: 560}}, }, - expected: []compactionTask{{ + expected: []CompactionTask{{ {BlockMeta: tsdb.BlockMeta{ Version: 1, ULID: ulid.MustNew(1, nil), MinTime: 0, MaxTime: 540, Stats: tsdb.BlockStats{ NumSeries: 10, @@ -257,7 +257,7 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(7, nil), MinTime: 360, MaxTime: 420}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(8, nil), MinTime: 420, MaxTime: 540}}, }, - expected: []compactionTask{{ + expected: []CompactionTask{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(7, nil), MinTime: 360, MaxTime: 420}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(8, nil), MinTime: 420, MaxTime: 540}}, }}, @@ -272,7 +272,7 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 19, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 40, MaxTime: 60}}, }, - expected: []compactionTask{{ + expected: []CompactionTask{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(1, nil), MinTime: 0, MaxTime: 20}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 19, MaxTime: 40}}, }}, @@ -287,7 +287,7 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 20, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 30, MaxTime: 50}}, }, - expected: []compactionTask{{ + expected: []CompactionTask{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 20, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 30, MaxTime: 50}}, }}, @@ -302,7 +302,7 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 10, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 30, MaxTime: 50}}, }, - expected: []compactionTask{{ + expected: []CompactionTask{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(1, nil), MinTime: 0, MaxTime: 20}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 10, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 30, MaxTime: 50}}, @@ -320,7 +320,7 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(7, nil), MinTime: 360, MaxTime: 420}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(8, nil), MinTime: 420, MaxTime: 540}}, }, - expected: []compactionTask{{ + expected: []CompactionTask{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(5, nil), MinTime: 0, MaxTime: 360}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(6, nil), MinTime: 340, MaxTime: 560}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(7, nil), MinTime: 360, MaxTime: 420}}, @@ -330,25 +330,25 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { // |--------------| // |--------------| // |--------------| - // |--------------| - // |--------------| + // |--------------| + // |--------------| { name: "Multiple independent groups of overlapping blocks", metas: []*metadata.Meta{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(1, nil), MinTime: 0, MaxTime: 10}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 9, MaxTime: 20}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 17, MaxTime: 35}}, - {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(4, nil), MinTime: 40, MaxTime: 50}}, + {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(4, nil), MinTime: 35, MaxTime: 50}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(5, nil), MinTime: 47, MaxTime: 60}}, }, - expected: []compactionTask{ + expected: []CompactionTask{ { {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(1, nil), MinTime: 0, MaxTime: 10}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 9, MaxTime: 20}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 17, MaxTime: 35}}, }, { - {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(4, nil), MinTime: 40, MaxTime: 50}}, + {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(4, nil), MinTime: 35, MaxTime: 50}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(5, nil), MinTime: 47, MaxTime: 60}}, }, }, @@ -463,12 +463,12 @@ func TestRangeWithFailedCompactionWontGetSelected(t *testing.T) { tsdbPlanner.dir = dir plan, err := tsdbPlanner.Plan(context.Background(), c.metas) testutil.Ok(t, err) - testutil.Equals(t, []compactionTask(nil), plan) + testutil.Equals(t, []CompactionTask(nil), plan) }) t.Run("tsdbBasedPlanner", func(t *testing.T) { plan, err := tsdbBasedPlanner.Plan(context.Background(), c.metas) testutil.Ok(t, err) - testutil.Equals(t, []compactionTask(nil), plan) + testutil.Equals(t, []CompactionTask(nil), plan) }) }) } @@ -491,7 +491,7 @@ func TestTSDBBasedPlanner_PlanWithNoCompactMarks(t *testing.T) { metas []*metadata.Meta noCompactMarks map[ulid.ULID]*metadata.NoCompactMark - expected []compactionTask + expected []CompactionTask }{ { name: "Outside range and excluded", @@ -513,7 +513,7 @@ func TestTSDBBasedPlanner_PlanWithNoCompactMarks(t *testing.T) { noCompactMarks: map[ulid.ULID]*metadata.NoCompactMark{ ulid.MustNew(1, nil): {}, }, - expected: []compactionTask{{ + expected: []CompactionTask{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 20, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 40, MaxTime: 60}}, }}, @@ -541,7 +541,7 @@ func TestTSDBBasedPlanner_PlanWithNoCompactMarks(t *testing.T) { noCompactMarks: map[ulid.ULID]*metadata.NoCompactMark{ ulid.MustNew(4, nil): {}, }, - expected: []compactionTask{{ + expected: []CompactionTask{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(1, nil), MinTime: 0, MaxTime: 20}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 20, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 40, MaxTime: 60}}, @@ -559,7 +559,7 @@ func TestTSDBBasedPlanner_PlanWithNoCompactMarks(t *testing.T) { ulid.MustNew(1, nil): {}, ulid.MustNew(4, nil): {}, }, - expected: []compactionTask{{ + expected: []CompactionTask{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 20, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 40, MaxTime: 60}}, }}, @@ -604,7 +604,7 @@ func TestTSDBBasedPlanner_PlanWithNoCompactMarks(t *testing.T) { noCompactMarks: map[ulid.ULID]*metadata.NoCompactMark{ ulid.MustNew(6, nil): {}, }, - expected: []compactionTask{{ + expected: []CompactionTask{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 20, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(4, nil), MinTime: 60, MaxTime: 120}}, }}, @@ -685,7 +685,7 @@ func TestLargeTotalIndexSizeFilter_Plan(t *testing.T) { name string metas []*metadata.Meta - expected []compactionTask + expected []CompactionTask expectedMarks float64 }{ { @@ -709,7 +709,7 @@ func TestLargeTotalIndexSizeFilter_Plan(t *testing.T) { BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(4, nil), MinTime: 60, MaxTime: 80}}, }, expectedMarks: 1, - expected: []compactionTask{{ + expected: []CompactionTask{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 20, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 40, MaxTime: 60}}, }}, @@ -740,7 +740,7 @@ func TestLargeTotalIndexSizeFilter_Plan(t *testing.T) { {Thanos: metadata.Thanos{Files: []metadata.File{{RelPath: block.IndexFilename, SizeBytes: 90}}}, BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(4, nil), MinTime: 60, MaxTime: 80}}, }, - expected: []compactionTask{{ + expected: []CompactionTask{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(1, nil), MinTime: 0, MaxTime: 20}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 20, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 40, MaxTime: 60}}, @@ -760,7 +760,7 @@ func TestLargeTotalIndexSizeFilter_Plan(t *testing.T) { {Thanos: metadata.Thanos{Files: []metadata.File{{RelPath: block.IndexFilename, SizeBytes: 90}}}, BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(5, nil), MinTime: 60, MaxTime: 80}}, }, - expected: []compactionTask{{ + expected: []CompactionTask{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 20, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 40, MaxTime: 50}}, }}, @@ -795,7 +795,7 @@ func TestLargeTotalIndexSizeFilter_Plan(t *testing.T) { {Thanos: metadata.Thanos{Files: []metadata.File{{RelPath: block.IndexFilename, SizeBytes: 30}}}, BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(7, nil), MinTime: 720, MaxTime: 960}}, }, - expected: []compactionTask{{ + expected: []CompactionTask{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 20, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(4, nil), MinTime: 60, MaxTime: 120}}, }}, From c91122918c7b325efdb36cc25faef719b4478825 Mon Sep 17 00:00:00 2001 From: Filip Petkovski Date: Fri, 2 Dec 2022 08:05:49 +0100 Subject: [PATCH 03/22] Use errutil.MultiError instead of cortex MultiError Signed-off-by: Filip Petkovski --- pkg/compact/compact.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/pkg/compact/compact.go b/pkg/compact/compact.go index 9d8f20a243..ba6881b91b 100644 --- a/pkg/compact/compact.go +++ b/pkg/compact/compact.go @@ -25,7 +25,6 @@ import ( "github.com/thanos-io/objstore" "golang.org/x/sync/errgroup" - "github.com/thanos-io/thanos/internal/cortex/util/multierror" "github.com/thanos-io/thanos/pkg/block" "github.com/thanos-io/thanos/pkg/block/metadata" "github.com/thanos-io/thanos/pkg/compact/downsample" @@ -1024,7 +1023,7 @@ func (cg *Group) compact(ctx context.Context, dir string, planner Planner, comp var ( wg sync.WaitGroup mu sync.Mutex - groupErr multierror.MultiError + groupErr errutil.MultiError rerunGroup bool ) for _, task := range tasks { From afe27df449c48b579044591d0d71ca37aa2ee485 Mon Sep 17 00:00:00 2001 From: Filip Petkovski Date: Fri, 2 Dec 2022 08:24:11 +0100 Subject: [PATCH 04/22] Add test case for progress calculator Signed-off-by: Filip Petkovski --- pkg/compact/compact_test.go | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/pkg/compact/compact_test.go b/pkg/compact/compact_test.go index d5485f02bb..6ca8f74239 100644 --- a/pkg/compact/compact_test.go +++ b/pkg/compact/compact_test.go @@ -451,6 +451,21 @@ func TestCompactProgressCalculate(t *testing.T) { }, }, }, + { + testName: "multiple_vertical_compactions", + input: []*metadata.Meta{ + createBlockMeta(1, 0, 10, map[string]string{"a": "1"}, 0, []uint64{}), + createBlockMeta(2, 5, 15, map[string]string{"a": "1"}, 0, []uint64{}), + createBlockMeta(3, 20, 30, map[string]string{"a": "1"}, 0, []uint64{}), + createBlockMeta(4, 25, 40, map[string]string{"a": "1"}, 0, []uint64{}), + }, + expected: map[string]planResult{ + keys[0]: { + compactionRuns: 2.0, + compactionBlocks: 4.0, + }, + }, + }, } { if ok := t.Run(tcase.testName, func(t *testing.T) { blocks := make(map[ulid.ULID]*metadata.Meta, len(tcase.input)) From 321efdc3e0584cc8eed524ac72d463c6e27f998d Mon Sep 17 00:00:00 2001 From: Filip Petkovski Date: Fri, 2 Dec 2022 09:18:13 +0100 Subject: [PATCH 05/22] Remove ULID return value Signed-off-by: Filip Petkovski --- pkg/compact/compact.go | 24 ++++++++++++------------ 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/pkg/compact/compact.go b/pkg/compact/compact.go index ba6881b91b..ebbf18c0d0 100644 --- a/pkg/compact/compact.go +++ b/pkg/compact/compact.go @@ -1030,7 +1030,7 @@ func (cg *Group) compact(ctx context.Context, dir string, planner Planner, comp wg.Add(1) go func(task CompactionTask) { defer wg.Done() - rerunTask, _, err := cg.compactBlocks(ctx, dir, task, comp, overlappingBlocks) + rerunTask, err := cg.compactBlocks(ctx, dir, task, comp, overlappingBlocks) mu.Lock() defer mu.Unlock() @@ -1043,7 +1043,7 @@ func (cg *Group) compact(ctx context.Context, dir string, planner Planner, comp return rerunGroup, groupErr.Err() } -func (cg *Group) compactBlocks(ctx context.Context, dir string, task CompactionTask, comp Compactor, overlappingBlocks bool) (bool, ulid.ULID, error) { +func (cg *Group) compactBlocks(ctx context.Context, dir string, task CompactionTask, comp Compactor, overlappingBlocks bool) (bool, error) { // Once we have a plan we need to download the actual data. compactionBegin := time.Now() begin := compactionBegin @@ -1097,7 +1097,7 @@ func (cg *Group) compactBlocks(ctx context.Context, dir string, task CompactionT sourceBlockStr := fmt.Sprintf("%v", toCompactDirs) if err := g.Wait(); err != nil { - return false, ulid.ULID{}, err + return false, err } level.Info(cg.logger).Log("msg", "downloaded and verified blocks; compacting blocks", "plan", sourceBlockStr, "duration", time.Since(begin), "duration_ms", time.Since(begin).Milliseconds()) @@ -1108,7 +1108,7 @@ func (cg *Group) compactBlocks(ctx context.Context, dir string, task CompactionT compID, e = comp.Compact(dir, toCompactDirs, nil) return e }); err != nil { - return false, ulid.ULID{}, halt(errors.Wrapf(err, "compact blocks %v", toCompactDirs)) + return false, halt(errors.Wrapf(err, "compact blocks %v", toCompactDirs)) } if compID == (ulid.ULID{}) { // Prometheus compactor found that the compacted block would have no samples. @@ -1121,7 +1121,7 @@ func (cg *Group) compactBlocks(ctx context.Context, dir string, task CompactionT } } // Even though this block was empty, there may be more work to do. - return true, ulid.ULID{}, nil + return true, nil } cg.compactions.Inc() if overlappingBlocks { @@ -1140,11 +1140,11 @@ func (cg *Group) compactBlocks(ctx context.Context, dir string, task CompactionT SegmentFiles: block.GetSegmentFiles(bdir), }, nil) if err != nil { - return false, ulid.ULID{}, errors.Wrapf(err, "failed to finalize the block %s", bdir) + return false, errors.Wrapf(err, "failed to finalize the block %s", bdir) } if err = os.Remove(filepath.Join(bdir, "tombstones")); err != nil { - return false, ulid.ULID{}, errors.Wrap(err, "remove tombstones") + return false, errors.Wrap(err, "remove tombstones") } // Ensure the output block is valid. @@ -1152,14 +1152,14 @@ func (cg *Group) compactBlocks(ctx context.Context, dir string, task CompactionT return block.VerifyIndex(cg.logger, index, newMeta.MinTime, newMeta.MaxTime) }) if !cg.acceptMalformedIndex && err != nil { - return false, ulid.ULID{}, halt(errors.Wrapf(err, "invalid result block %s", bdir)) + return false, halt(errors.Wrapf(err, "invalid result block %s", bdir)) } // Ensure the output block is not overlapping with anything else, // unless vertical compaction is enabled. if !cg.enableVerticalCompaction { if err := cg.areBlocksOverlapping(newMeta, task...); err != nil { - return false, ulid.ULID{}, halt(errors.Wrapf(err, "resulted compacted block %s overlaps with something", bdir)) + return false, halt(errors.Wrapf(err, "resulted compacted block %s overlaps with something", bdir)) } } @@ -1169,7 +1169,7 @@ func (cg *Group) compactBlocks(ctx context.Context, dir string, task CompactionT return block.Upload(ctx, cg.logger, cg.bkt, bdir, cg.hashFunc, objstore.WithUploadConcurrency(cg.blockFilesConcurrency)) }) if err != nil { - return false, ulid.ULID{}, retry(errors.Wrapf(err, "upload of %s failed", compID)) + return false, retry(errors.Wrapf(err, "upload of %s failed", compID)) } level.Info(cg.logger).Log("msg", "uploaded block", "result_block", compID, "duration", time.Since(begin), "duration_ms", time.Since(begin).Milliseconds()) @@ -1181,14 +1181,14 @@ func (cg *Group) compactBlocks(ctx context.Context, dir string, task CompactionT return cg.deleteBlock(meta.ULID, filepath.Join(dir, meta.ULID.String())) }, opentracing.Tags{"block.id": meta.ULID}) if err != nil { - return false, ulid.ULID{}, retry(errors.Wrapf(err, "mark old block for deletion from bucket")) + return false, retry(errors.Wrapf(err, "mark old block for deletion from bucket")) } cg.groupGarbageCollectedBlocks.Inc() } level.Info(cg.logger).Log("msg", "finished compacting blocks", "result_block", compID, "source_blocks", sourceBlockStr, "duration", time.Since(compactionBegin), "duration_ms", time.Since(compactionBegin).Milliseconds()) - return true, compID, nil + return true, nil } func (cg *Group) deleteBlock(id ulid.ULID, bdir string) error { From 31885e1350ed4ce784cf247182b17b80f53a345f Mon Sep 17 00:00:00 2001 From: Filip Petkovski Date: Fri, 2 Dec 2022 10:28:29 +0100 Subject: [PATCH 06/22] Iterate through all errors for a compaction group Signed-off-by: Filip Petkovski --- pkg/compact/compact.go | 66 +++++++++++++++++++++++++----------------- 1 file changed, 40 insertions(+), 26 deletions(-) diff --git a/pkg/compact/compact.go b/pkg/compact/compact.go index ebbf18c0d0..6611a4a18f 100644 --- a/pkg/compact/compact.go +++ b/pkg/compact/compact.go @@ -1280,8 +1280,8 @@ func (c *BucketCompactor) Compact(ctx context.Context) (rerr error) { go func() { defer wg.Done() for g := range groupChan { - shouldRerunGroup, err := g.Compact(workCtx, c.compactDir, c.planner, c.comp) - if err == nil { + shouldRerunGroup, compactErrs := g.Compact(workCtx, c.compactDir, c.planner, c.comp) + if compactErrs == nil { if shouldRerunGroup { mtx.Lock() finishedAllGroups = false @@ -1289,34 +1289,48 @@ func (c *BucketCompactor) Compact(ctx context.Context) (rerr error) { } continue } + errs, ok := compactErrs.(errutil.NonNilMultiError) + if !ok { + errs = []error{compactErrs} + } - if IsIssue347Error(err) { - if err := RepairIssue347(workCtx, c.logger, c.bkt, c.sy.metrics.blocksMarkedForDeletion, err); err == nil { - mtx.Lock() - finishedAllGroups = false - mtx.Unlock() - continue + var nonRecoverableErrs errutil.MultiError + for _, err := range errs { + if IsIssue347Error(err) { + if err := RepairIssue347(workCtx, c.logger, c.bkt, c.sy.metrics.blocksMarkedForDeletion, err); err == nil { + mtx.Lock() + finishedAllGroups = false + mtx.Unlock() + continue + } } - } - // If block has out of order chunk and it has been configured to skip it, - // then we can mark the block for no compaction so that the next compaction run - // will skip it. - if IsOutOfOrderChunkError(err) && c.skipBlocksWithOutOfOrderChunks { - if err := block.MarkForNoCompact( - ctx, - c.logger, - c.bkt, - err.(OutOfOrderChunksError).id, - metadata.OutOfOrderChunksNoCompactReason, - "OutofOrderChunk: marking block with out-of-order series/chunks to as no compact to unblock compaction", g.blocksMarkedForNoCompact); err == nil { - mtx.Lock() - finishedAllGroups = false - mtx.Unlock() - continue + + // If block has out of order chunk and it has been configured to skip it, + // then we can mark the block for no compaction so that the next compaction run + // will skip it. + if IsOutOfOrderChunkError(err) && c.skipBlocksWithOutOfOrderChunks { + if err := block.MarkForNoCompact( + ctx, + c.logger, + c.bkt, + err.(OutOfOrderChunksError).id, + metadata.OutOfOrderChunksNoCompactReason, + "OutofOrderChunk: marking block with out-of-order series/chunks to as no compact to unblock compaction", g.blocksMarkedForNoCompact, + ); err == nil { + mtx.Lock() + finishedAllGroups = false + mtx.Unlock() + continue + } } + + nonRecoverableErrs.Add(err) + } + + if nonRecoverableErrs.Err() != nil { + errChan <- errors.Wrapf(nonRecoverableErrs.Err(), "group %s", g.Key()) + return } - errChan <- errors.Wrapf(err, "group %s", g.Key()) - return } }() } From 86af2a86384c526faf0c0c9fa12708ea224e4ddd Mon Sep 17 00:00:00 2001 From: Filip Petkovski Date: Fri, 2 Dec 2022 11:48:23 +0100 Subject: [PATCH 07/22] Add test case for large index size Signed-off-by: Filip Petkovski --- pkg/compact/planner_test.go | 25 +++++++++++++++++++++++++ 1 file changed, 25 insertions(+) diff --git a/pkg/compact/planner_test.go b/pkg/compact/planner_test.go index c760d69271..e584f69fcd 100644 --- a/pkg/compact/planner_test.go +++ b/pkg/compact/planner_test.go @@ -816,6 +816,31 @@ func TestLargeTotalIndexSizeFilter_Plan(t *testing.T) { }, expectedMarks: 1, }, + // |--------------| + // |----------------| + // |--------------| + // |----------------| + // |--------------| + { + name: "Two groups of overlapping blocks, first group total is too large", + metas: []*metadata.Meta{ + {Thanos: metadata.Thanos{Files: []metadata.File{{RelPath: block.IndexFilename, SizeBytes: 90}}}, + BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(1, nil), MinTime: 0, MaxTime: 20}}, + {Thanos: metadata.Thanos{Files: []metadata.File{{RelPath: block.IndexFilename, SizeBytes: 30}}}, + BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 19, MaxTime: 40}}, + {Thanos: metadata.Thanos{Files: []metadata.File{{RelPath: block.IndexFilename, SizeBytes: 30}}}, + BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 40, MaxTime: 60}}, + {Thanos: metadata.Thanos{Files: []metadata.File{{RelPath: block.IndexFilename, SizeBytes: 30}}}, + BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(4, nil), MinTime: 70, MaxTime: 80}}, + {Thanos: metadata.Thanos{Files: []metadata.File{{RelPath: block.IndexFilename, SizeBytes: 30}}}, + BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(5, nil), MinTime: 75, MaxTime: 90}}, + }, + expectedMarks: 1, + expected: []CompactionTask{{ + {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(4, nil), MinTime: 70, MaxTime: 80}}, + {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(5, nil), MinTime: 75, MaxTime: 90}}, + }}, + }, } { if !t.Run(c.name, func(t *testing.T) { t.Run("from meta", func(t *testing.T) { From 1248a962e29a5fcfcde7d6d939e08e44e0089b64 Mon Sep 17 00:00:00 2001 From: Filip Petkovski Date: Fri, 2 Dec 2022 14:08:12 +0100 Subject: [PATCH 08/22] Parametrize concurrency inside single group Signed-off-by: Filip Petkovski --- cmd/thanos/compact.go | 7 ++++++- pkg/compact/compact_e2e_test.go | 2 +- pkg/compact/compact_test.go | 2 +- pkg/compact/planner.go | 23 +++++++++++++++++------ pkg/compact/planner_test.go | 10 ++++++---- 5 files changed, 31 insertions(+), 13 deletions(-) diff --git a/cmd/thanos/compact.go b/cmd/thanos/compact.go index 6cb4eae3e9..c4cf19bbb6 100644 --- a/cmd/thanos/compact.go +++ b/cmd/thanos/compact.go @@ -352,7 +352,7 @@ func runCompact( conf.blockFilesConcurrency, conf.compactBlocksFetchConcurrency, ) - tsdbPlanner := compact.NewPlanner(logger, levels, noCompactMarkerFilter) + tsdbPlanner := compact.NewPlanner(logger, levels, noCompactMarkerFilter, conf.groupCompactionConcurrency) planner := compact.WithLargeTotalIndexSizeFilter( tsdbPlanner, bkt, @@ -671,6 +671,7 @@ type compactConfig struct { maxBlockIndexSize units.Base2Bytes hashFunc string enableVerticalCompaction bool + groupCompactionConcurrency int dedupFunc string skipBlockWithOutOfOrderChunks bool progressCalculateInterval time.Duration @@ -746,6 +747,10 @@ func (cc *compactConfig) registerFlag(cmd extkingpin.FlagClause) { "NOTE: This flag is ignored and (enabled) when --deduplication.replica-label flag is set."). Hidden().Default("false").BoolVar(&cc.enableVerticalCompaction) + cmd.Flag("compact.group-compaction-concurrency", "The maximum number of concurrent compactions that can run within a single compaction group."+ + "A higher number means the compactor will use more resources at peak."). + Default("1").IntVar(&cc.groupCompactionConcurrency) + cmd.Flag("deduplication.func", "Experimental. Deduplication algorithm for merging overlapping blocks. "+ "Possible values are: \"\", \"penalty\". If no value is specified, the default compact deduplication merger is used, which performs 1:1 deduplication for samples. "+ "When set to penalty, penalty based deduplication algorithm will be used. At least one replica label has to be set via --deduplication.replica-label flag."). diff --git a/pkg/compact/compact_e2e_test.go b/pkg/compact/compact_e2e_test.go index 485753432e..74e680924d 100644 --- a/pkg/compact/compact_e2e_test.go +++ b/pkg/compact/compact_e2e_test.go @@ -210,7 +210,7 @@ func testGroupCompactE2e(t *testing.T, mergeFunc storage.VerticalChunkSeriesMerg comp, err := tsdb.NewLeveledCompactor(ctx, reg, logger, []int64{1000, 3000}, nil, mergeFunc) testutil.Ok(t, err) - planner := NewPlanner(logger, []int64{1000, 3000}, noCompactMarkerFilter) + planner := NewPlanner(logger, []int64{1000, 3000}, noCompactMarkerFilter, testGroupConcurrency) grouper := NewDefaultGrouper(logger, bkt, false, false, reg, blocksMarkedForDeletion, garbageCollectedBlocks, blocksMaredForNoCompact, metadata.NoneFunc, 10, 10) bComp, err := NewBucketCompactor(logger, sy, grouper, planner, comp, dir, bkt, 2, true) testutil.Ok(t, err) diff --git a/pkg/compact/compact_test.go b/pkg/compact/compact_test.go index 6ca8f74239..c811b938ed 100644 --- a/pkg/compact/compact_test.go +++ b/pkg/compact/compact_test.go @@ -360,7 +360,7 @@ func TestCompactProgressCalculate(t *testing.T) { int64(2 * time.Hour / time.Millisecond), int64(4 * time.Hour / time.Millisecond), int64(8 * time.Hour / time.Millisecond), - }) + }, testGroupConcurrency) keys := make([]string, 3) m := make([]metadata.Meta, 3) diff --git a/pkg/compact/planner.go b/pkg/compact/planner.go index ae59689cb3..e176faf7fc 100644 --- a/pkg/compact/planner.go +++ b/pkg/compact/planner.go @@ -25,7 +25,8 @@ type tsdbBasedPlanner struct { ranges []int64 - noCompBlocksFunc func() map[ulid.ULID]*metadata.NoCompactMark + noCompBlocksFunc func() map[ulid.ULID]*metadata.NoCompactMark + groupCompactionConcurrency int } var _ Planner = &tsdbBasedPlanner{} @@ -33,20 +34,26 @@ var _ Planner = &tsdbBasedPlanner{} // NewTSDBBasedPlanner is planner with the same functionality as Prometheus' TSDB. // TODO(bwplotka): Consider upstreaming this to Prometheus. // It's the same functionality just without accessing filesystem. -func NewTSDBBasedPlanner(logger log.Logger, ranges []int64) *tsdbBasedPlanner { +func NewTSDBBasedPlanner(logger log.Logger, ranges []int64, groupCompactionConcurrency int) *tsdbBasedPlanner { return &tsdbBasedPlanner{ logger: logger, ranges: ranges, noCompBlocksFunc: func() map[ulid.ULID]*metadata.NoCompactMark { return make(map[ulid.ULID]*metadata.NoCompactMark) }, + groupCompactionConcurrency: groupCompactionConcurrency, } } // NewPlanner is a default Thanos planner with the same functionality as Prometheus' TSDB plus special handling of excluded blocks. // It's the same functionality just without accessing filesystem, and special handling of excluded blocks. -func NewPlanner(logger log.Logger, ranges []int64, noCompBlocks *GatherNoCompactionMarkFilter) *tsdbBasedPlanner { - return &tsdbBasedPlanner{logger: logger, ranges: ranges, noCompBlocksFunc: noCompBlocks.NoCompactMarkedBlocks} +func NewPlanner(logger log.Logger, ranges []int64, noCompBlocks *GatherNoCompactionMarkFilter, groupCompactionConcurrency int) *tsdbBasedPlanner { + return &tsdbBasedPlanner{ + logger: logger, + ranges: ranges, + noCompBlocksFunc: noCompBlocks.NoCompactMarkedBlocks, + groupCompactionConcurrency: groupCompactionConcurrency, + } } // TODO(bwplotka): Consider smarter algorithm, this prefers smaller iterative compactions vs big single one: https://github.com/thanos-io/thanos/issues/3405 @@ -67,7 +74,7 @@ func (p *tsdbBasedPlanner) plan(noCompactMarked map[ulid.ULID]*metadata.NoCompac notExcludedMetasByMinTime = append(notExcludedMetasByMinTime, meta) } - verticalCompactions := selectOverlappingMetas(notExcludedMetasByMinTime) + verticalCompactions := selectOverlappingMetas(notExcludedMetasByMinTime, p.groupCompactionConcurrency) if len(verticalCompactions) > 0 { return verticalCompactions, nil } @@ -161,7 +168,7 @@ func selectMetas(ranges []int64, noCompactMarked map[ulid.ULID]*metadata.NoCompa // selectOverlappingMetas returns all dirs with overlapping time ranges. // It expects sorted input by mint and returns the overlapping dirs in the same order as received. // Copied and adjusted from https://github.com/prometheus/prometheus/blob/3d8826a3d42566684283a9b7f7e812e412c24407/tsdb/compact.go#L268. -func selectOverlappingMetas(metasByMinTime []*metadata.Meta) []CompactionTask { +func selectOverlappingMetas(metasByMinTime []*metadata.Meta, maxTasks int) []CompactionTask { if len(metasByMinTime) < 2 { return nil } @@ -216,6 +223,10 @@ loopMetas: overlappingGroups = append(overlappingGroups, group) } + if len(overlappingGroups) > maxTasks { + return overlappingGroups[:maxTasks] + } + return overlappingGroups } diff --git a/pkg/compact/planner_test.go b/pkg/compact/planner_test.go index e584f69fcd..94fc7e83e7 100644 --- a/pkg/compact/planner_test.go +++ b/pkg/compact/planner_test.go @@ -25,6 +25,8 @@ import ( "github.com/thanos-io/thanos/pkg/testutil" ) +const testGroupConcurrency = 3 + type tsdbPlannerAdapter struct { dir string comp tsdb.Compactor @@ -75,7 +77,7 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { tsdbComp, err := tsdb.NewLeveledCompactor(context.Background(), nil, nil, ranges, nil, nil) testutil.Ok(t, err) tsdbPlanner := &tsdbPlannerAdapter{comp: tsdbComp} - tsdbBasedPlanner := NewTSDBBasedPlanner(log.NewNopLogger(), ranges) + tsdbBasedPlanner := NewTSDBBasedPlanner(log.NewNopLogger(), ranges, testGroupConcurrency) for _, c := range []struct { name string @@ -420,7 +422,7 @@ func TestRangeWithFailedCompactionWontGetSelected(t *testing.T) { tsdbComp, err := tsdb.NewLeveledCompactor(context.Background(), nil, nil, ranges, nil, nil) testutil.Ok(t, err) tsdbPlanner := &tsdbPlannerAdapter{comp: tsdbComp} - tsdbBasedPlanner := NewTSDBBasedPlanner(log.NewNopLogger(), ranges) + tsdbBasedPlanner := NewTSDBBasedPlanner(log.NewNopLogger(), ranges, testGroupConcurrency) for _, c := range []struct { metas []*metadata.Meta @@ -484,7 +486,7 @@ func TestTSDBBasedPlanner_PlanWithNoCompactMarks(t *testing.T) { } g := &GatherNoCompactionMarkFilter{} - tsdbBasedPlanner := NewPlanner(log.NewNopLogger(), ranges, g) + tsdbBasedPlanner := NewPlanner(log.NewNopLogger(), ranges, g, testGroupConcurrency) for _, c := range []struct { name string @@ -679,7 +681,7 @@ func TestLargeTotalIndexSizeFilter_Plan(t *testing.T) { g := &GatherNoCompactionMarkFilter{} marked := promauto.With(nil).NewCounter(prometheus.CounterOpts{}) - planner := WithLargeTotalIndexSizeFilter(NewPlanner(log.NewNopLogger(), ranges, g), bkt, 100, marked) + planner := WithLargeTotalIndexSizeFilter(NewPlanner(log.NewNopLogger(), ranges, g, testGroupConcurrency), bkt, 100, marked) var lastMarkValue float64 for _, c := range []struct { name string From 09bdb1afa7c457ade7a34d1fe36aca0770deb222 Mon Sep 17 00:00:00 2001 From: Filip Petkovski Date: Sat, 3 Dec 2022 08:33:34 +0100 Subject: [PATCH 09/22] Change test name Signed-off-by: Filip Petkovski --- pkg/compact/planner_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/compact/planner_test.go b/pkg/compact/planner_test.go index 94fc7e83e7..7fc0f67a6a 100644 --- a/pkg/compact/planner_test.go +++ b/pkg/compact/planner_test.go @@ -335,7 +335,7 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { // |--------------| // |--------------| { - name: "Multiple independent groups of overlapping blocks", + name: "Multiple independent overlapping blocks", metas: []*metadata.Meta{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(1, nil), MinTime: 0, MaxTime: 10}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 9, MaxTime: 20}}, From b58d2bffc9568de7eb72fd445fc1fb55ab35b72b Mon Sep 17 00:00:00 2001 From: Filip Petkovski Date: Sun, 4 Dec 2022 08:10:33 +0100 Subject: [PATCH 10/22] Limit concurrency of tasks to global concurrency Signed-off-by: Filip Petkovski --- pkg/compact/compact.go | 238 ++++++++++++++++-------------------- pkg/compact/planner.go | 20 +-- pkg/compact/planner_test.go | 58 ++++----- 3 files changed, 141 insertions(+), 175 deletions(-) diff --git a/pkg/compact/compact.go b/pkg/compact/compact.go index 6611a4a18f..b567e76748 100644 --- a/pkg/compact/compact.go +++ b/pkg/compact/compact.go @@ -8,8 +8,10 @@ import ( "fmt" "math" "os" + "path" "path/filepath" "sort" + "strconv" "sync" "time" @@ -729,7 +731,7 @@ func (rs *RetentionProgressCalculator) ProgressCalculate(ctx context.Context, gr type Planner interface { // Plan returns a list of blocks that should be compacted into single one. // The blocks can be overlapping. The provided metadata has to be ordered by minTime. - Plan(ctx context.Context, metasByMinTime []*metadata.Meta) ([]CompactionTask, error) + Plan(ctx context.Context, metasByMinTime []*metadata.Meta) ([]CompactionBlocks, error) } // Compactor provides compaction against an underlying storage of time series data. @@ -751,38 +753,49 @@ type Compactor interface { Compact(dest string, dirs []string, open []*tsdb.Block) (ulid.ULID, error) } +// CompactionTask is an independent compaction task. +type CompactionTask struct { + Group *Group + Blocks CompactionBlocks + Dir string + logger log.Logger +} + // Compact plans and runs a single compaction against the group. The compacted result // is uploaded into the bucket the blocks were retrieved from. -func (cg *Group) Compact(ctx context.Context, dir string, planner Planner, comp Compactor) (shouldRerun bool, rerr error) { - cg.compactionRunsStarted.Inc() - - subDir := filepath.Join(dir, cg.Key()) +func (task *CompactionTask) Compact(ctx context.Context, comp Compactor) (shouldRerun bool, cerr error) { + overlappingBlocks := false + if err := task.Group.areBlocksOverlapping(nil); err != nil { + overlappingBlocks = true + } + task.Group.compactionRunsStarted.Inc() defer func() { // Leave the compact directory for inspection if it is a halt error // or if it is not then so that possibly we would not have to download everything again. - if rerr != nil { + if cerr != nil { + task.Group.compactionFailures.Inc() return } - if err := os.RemoveAll(subDir); err != nil { - level.Error(cg.logger).Log("msg", "failed to remove compaction group work directory", "path", subDir, "err", err) + task.Group.compactionRunsCompleted.Inc() + if err := os.RemoveAll(task.Dir); err != nil { + level.Error(task.logger).Log("msg", "failed to remove compaction group work directory", "path", task.Dir, "err", err) } }() - if err := os.MkdirAll(subDir, 0750); err != nil { + if err := os.MkdirAll(task.Dir, 0750); err != nil { return false, errors.Wrap(err, "create compaction group dir") } - err := tracing.DoInSpanWithErr(ctx, "compaction_group", func(ctx context.Context) (err error) { - shouldRerun, err = cg.compact(ctx, subDir, planner, comp) - return err - }, opentracing.Tags{"group.key": cg.Key()}) - if err != nil { - cg.compactionFailures.Inc() - return false, err + terr := tracing.DoInSpanWithErr(ctx, "compaction_group", func(ctx context.Context) (cerr error) { + shouldRerun, cerr = task.Group.compactBlocks(ctx, task.Dir, task.Blocks, comp, overlappingBlocks) + return cerr + }, opentracing.Tags{"group.key": task.Group.Key()}) + if terr != nil { + return false, terr } - cg.compactionRunsCompleted.Inc() - return shouldRerun, nil + + return shouldRerun, cerr } // Issue347Error is a type wrapper for errors that should invoke repair process for broken block. @@ -976,74 +989,26 @@ func RepairIssue347(ctx context.Context, logger log.Logger, bkt objstore.Bucket, return nil } -func (cg *Group) compact(ctx context.Context, dir string, planner Planner, comp Compactor) (shouldRerun bool, _ error) { - cg.mtx.Lock() - defer cg.mtx.Unlock() - - // Check for overlapped blocks. - overlappingBlocks := false - if err := cg.areBlocksOverlapping(nil); err != nil { - // TODO(bwplotka): It would really nice if we could still check for other overlaps than replica. In fact this should be checked - // in syncer itself. Otherwise with vertical compaction enabled we will sacrifice this important check. - if !cg.enableVerticalCompaction { - return false, halt(errors.Wrap(err, "pre compaction overlap check")) - } - - overlappingBlocks = true - } - - var tasks []CompactionTask - if err := tracing.DoInSpanWithErr(ctx, "compaction_planning", func(ctx context.Context) (e error) { - tasks, e = planner.Plan(ctx, cg.metasByMinTime) - return e - }); err != nil { - return false, errors.Wrap(err, "plan compaction") - } - if len(tasks) == 0 { - // Nothing to do. - return false, nil - } - - // Due to #183 we verify that none of the blocks in the plan have overlapping sources. - // This is one potential source of how we could end up with duplicated chunks. - uniqueSources := map[ulid.ULID]struct{}{} - for _, task := range tasks { - for _, m := range task { - for _, s := range m.Compaction.Sources { - if _, ok := uniqueSources[s]; ok { - return false, halt(errors.Errorf("overlapping sources detected for plan %v", task)) - } - uniqueSources[s] = struct{}{} - } - } +func (cg *Group) GetCompactionTasks(ctx context.Context, dir string, planner Planner) ([]CompactionTask, error) { + plannedCompactions, err := planner.Plan(ctx, cg.metasByMinTime) + if err != nil { + return nil, err } - level.Info(cg.logger).Log("msg", "compaction available and planned; downloading blocks", "plan", fmt.Sprintf("%v", tasks)) - - var ( - wg sync.WaitGroup - mu sync.Mutex - groupErr errutil.MultiError - rerunGroup bool - ) - for _, task := range tasks { - wg.Add(1) - go func(task CompactionTask) { - defer wg.Done() - rerunTask, err := cg.compactBlocks(ctx, dir, task, comp, overlappingBlocks) - - mu.Lock() - defer mu.Unlock() - rerunGroup = rerunGroup || rerunTask - groupErr.Add(err) - }(task) + tasks := make([]CompactionTask, 0, len(plannedCompactions)) + for i, blocks := range plannedCompactions { + tasks = append(tasks, CompactionTask{ + Group: cg, + Blocks: blocks, + Dir: path.Join(dir, cg.Key(), strconv.Itoa(i)), + logger: cg.logger, + }) } - wg.Wait() - return rerunGroup, groupErr.Err() + return tasks, nil } -func (cg *Group) compactBlocks(ctx context.Context, dir string, task CompactionTask, comp Compactor, overlappingBlocks bool) (bool, error) { +func (cg *Group) compactBlocks(ctx context.Context, dir string, blocks CompactionBlocks, comp Compactor, overlappingBlocks bool) (bool, error) { // Once we have a plan we need to download the actual data. compactionBegin := time.Now() begin := compactionBegin @@ -1051,8 +1016,8 @@ func (cg *Group) compactBlocks(ctx context.Context, dir string, task CompactionT g, errCtx := errgroup.WithContext(ctx) g.SetLimit(cg.compactBlocksFetchConcurrency) - toCompactDirs := make([]string, 0, len(task)) - for _, m := range task { + toCompactDirs := make([]string, 0, len(blocks)) + for _, m := range blocks { bdir := filepath.Join(dir, m.ULID.String()) func(ctx context.Context, meta *metadata.Meta) { g.Go(func() error { @@ -1113,7 +1078,7 @@ func (cg *Group) compactBlocks(ctx context.Context, dir string, task CompactionT if compID == (ulid.ULID{}) { // Prometheus compactor found that the compacted block would have no samples. level.Info(cg.logger).Log("msg", "compacted block would have no samples, deleting source blocks", "blocks", sourceBlockStr) - for _, meta := range task { + for _, meta := range blocks { if meta.Stats.NumSamples == 0 { if err := cg.deleteBlock(meta.ULID, filepath.Join(dir, meta.ULID.String())); err != nil { level.Warn(cg.logger).Log("msg", "failed to mark for deletion an empty block found during compaction", "block", meta.ULID) @@ -1158,13 +1123,12 @@ func (cg *Group) compactBlocks(ctx context.Context, dir string, task CompactionT // Ensure the output block is not overlapping with anything else, // unless vertical compaction is enabled. if !cg.enableVerticalCompaction { - if err := cg.areBlocksOverlapping(newMeta, task...); err != nil { + if err := cg.areBlocksOverlapping(newMeta, blocks...); err != nil { return false, halt(errors.Wrapf(err, "resulted compacted block %s overlaps with something", bdir)) } } begin = time.Now() - err = tracing.DoInSpanWithErr(ctx, "compaction_block_upload", func(ctx context.Context) error { return block.Upload(ctx, cg.logger, cg.bkt, bdir, cg.hashFunc, objstore.WithUploadConcurrency(cg.blockFilesConcurrency)) }) @@ -1176,7 +1140,7 @@ func (cg *Group) compactBlocks(ctx context.Context, dir string, task CompactionT // Mark for deletion the blocks we just compacted from the group and bucket so they do not get included // into the next planning cycle. // Eventually the block we just uploaded should get synced into the group again (including sync-delay). - for _, meta := range task { + for _, meta := range blocks { err = tracing.DoInSpanWithErr(ctx, "compaction_block_delete", func(ctx context.Context) error { return cg.deleteBlock(meta.ULID, filepath.Join(dir, meta.ULID.String())) }, opentracing.Tags{"block.id": meta.ULID}) @@ -1266,9 +1230,9 @@ func (c *BucketCompactor) Compact(ctx context.Context) (rerr error) { var ( wg sync.WaitGroup workCtx, workCtxCancel = context.WithCancel(ctx) - groupChan = make(chan *Group) + taskChan = make(chan CompactionTask) errChan = make(chan error, c.concurrency) - finishedAllGroups = true + finishedAllTasks = true mtx sync.Mutex ) defer workCtxCancel() @@ -1279,56 +1243,47 @@ func (c *BucketCompactor) Compact(ctx context.Context) (rerr error) { wg.Add(1) go func() { defer wg.Done() - for g := range groupChan { - shouldRerunGroup, compactErrs := g.Compact(workCtx, c.compactDir, c.planner, c.comp) - if compactErrs == nil { + for task := range taskChan { + shouldRerunGroup, err := task.Compact(workCtx, c.comp) + if err == nil { if shouldRerunGroup { mtx.Lock() - finishedAllGroups = false + finishedAllTasks = false mtx.Unlock() } continue } - errs, ok := compactErrs.(errutil.NonNilMultiError) - if !ok { - errs = []error{compactErrs} - } - var nonRecoverableErrs errutil.MultiError - for _, err := range errs { - if IsIssue347Error(err) { - if err := RepairIssue347(workCtx, c.logger, c.bkt, c.sy.metrics.blocksMarkedForDeletion, err); err == nil { - mtx.Lock() - finishedAllGroups = false - mtx.Unlock() - continue - } + if IsIssue347Error(err) { + if err := RepairIssue347(workCtx, c.logger, c.bkt, c.sy.metrics.blocksMarkedForDeletion, err); err == nil { + mtx.Lock() + finishedAllTasks = false + mtx.Unlock() + continue } + } - // If block has out of order chunk and it has been configured to skip it, - // then we can mark the block for no compaction so that the next compaction run - // will skip it. - if IsOutOfOrderChunkError(err) && c.skipBlocksWithOutOfOrderChunks { - if err := block.MarkForNoCompact( - ctx, - c.logger, - c.bkt, - err.(OutOfOrderChunksError).id, - metadata.OutOfOrderChunksNoCompactReason, - "OutofOrderChunk: marking block with out-of-order series/chunks to as no compact to unblock compaction", g.blocksMarkedForNoCompact, - ); err == nil { - mtx.Lock() - finishedAllGroups = false - mtx.Unlock() - continue - } + // If block has out of order chunk and it has been configured to skip it, + // then we can mark the block for no compaction so that the next compaction run + // will skip it. + if IsOutOfOrderChunkError(err) && c.skipBlocksWithOutOfOrderChunks { + if err := block.MarkForNoCompact( + ctx, + c.logger, + c.bkt, + err.(OutOfOrderChunksError).id, + metadata.OutOfOrderChunksNoCompactReason, + "OutofOrderChunk: marking block with out-of-order series/chunks to as no compact to unblock compaction", task.Group.blocksMarkedForNoCompact, + ); err == nil { + mtx.Lock() + finishedAllTasks = false + mtx.Unlock() + continue } - - nonRecoverableErrs.Add(err) } - if nonRecoverableErrs.Err() != nil { - errChan <- errors.Wrapf(nonRecoverableErrs.Err(), "group %s", g.Key()) + if err != nil { + errChan <- errors.Wrapf(err, "group %s", task.Group.Key()) return } } @@ -1365,37 +1320,48 @@ func (c *BucketCompactor) Compact(ctx context.Context) (rerr error) { level.Info(c.logger).Log("msg", "start of compactions") + tasks := make([]CompactionTask, 0) + for _, g := range groups { + groupTasks, err := g.GetCompactionTasks(ctx, c.compactDir, c.planner) + if err != nil { + return errors.Wrapf(err, "get compaction group tasks: %s", g.Key()) + } + for _, task := range groupTasks { + tasks = append(tasks, task) + } + } + // Send all groups found during this pass to the compaction workers. - var groupErrs errutil.MultiError + var taskErrs errutil.MultiError groupLoop: - for _, g := range groups { + for _, task := range tasks { // Ignore groups with only one block because there is nothing to compact. - if len(g.IDs()) == 1 { + if len(task.Blocks) == 1 { continue } select { case groupErr := <-errChan: - groupErrs.Add(groupErr) + taskErrs.Add(groupErr) break groupLoop - case groupChan <- g: + case taskChan <- task: } } - close(groupChan) + close(taskChan) wg.Wait() // Collect any other error reported by the workers, or any error reported // while we were waiting for the last batch of groups to run the compaction. close(errChan) for groupErr := range errChan { - groupErrs.Add(groupErr) + taskErrs.Add(groupErr) } workCtxCancel() - if len(groupErrs) > 0 { - return groupErrs.Err() + if len(taskErrs) > 0 { + return taskErrs.Err() } - if finishedAllGroups { + if finishedAllTasks { break } } diff --git a/pkg/compact/planner.go b/pkg/compact/planner.go index e176faf7fc..b1195ca14f 100644 --- a/pkg/compact/planner.go +++ b/pkg/compact/planner.go @@ -57,15 +57,15 @@ func NewPlanner(logger log.Logger, ranges []int64, noCompBlocks *GatherNoCompact } // TODO(bwplotka): Consider smarter algorithm, this prefers smaller iterative compactions vs big single one: https://github.com/thanos-io/thanos/issues/3405 -func (p *tsdbBasedPlanner) Plan(_ context.Context, metasByMinTime []*metadata.Meta) ([]CompactionTask, error) { +func (p *tsdbBasedPlanner) Plan(_ context.Context, metasByMinTime []*metadata.Meta) ([]CompactionBlocks, error) { return p.plan(p.noCompBlocksFunc(), metasByMinTime) } -// CompactionTask is a set of blocks that should be compacted together in a single compaction. -// Multiple compaction tasks can be run in parallel even inside a single compaction group. -type CompactionTask []*metadata.Meta +// CompactionBlocks is a set of blocks that should be compacted together in a single compaction. +// Multiple compaction tasks can be run in parallel even within a single compaction group. +type CompactionBlocks []*metadata.Meta -func (p *tsdbBasedPlanner) plan(noCompactMarked map[ulid.ULID]*metadata.NoCompactMark, metasByMinTime []*metadata.Meta) ([]CompactionTask, error) { +func (p *tsdbBasedPlanner) plan(noCompactMarked map[ulid.ULID]*metadata.NoCompactMark, metasByMinTime []*metadata.Meta) ([]CompactionBlocks, error) { notExcludedMetasByMinTime := make([]*metadata.Meta, 0, len(metasByMinTime)) for _, meta := range metasByMinTime { if _, excluded := noCompactMarked[meta.ULID]; excluded { @@ -88,7 +88,7 @@ func (p *tsdbBasedPlanner) plan(noCompactMarked map[ulid.ULID]*metadata.NoCompac metasByMinTime = metasByMinTime[:len(metasByMinTime)-1] res := selectMetas(p.ranges, noCompactMarked, metasByMinTime) if len(res) > 0 { - return []CompactionTask{res}, nil + return []CompactionBlocks{res}, nil } // Compact any blocks with big enough time range that have >5% tombstones. @@ -99,7 +99,7 @@ func (p *tsdbBasedPlanner) plan(noCompactMarked map[ulid.ULID]*metadata.NoCompac } if float64(meta.Stats.NumTombstones)/float64(meta.Stats.NumSeries+1) > 0.05 { task := []*metadata.Meta{notExcludedMetasByMinTime[i]} - return []CompactionTask{task}, nil + return []CompactionBlocks{task}, nil } } @@ -168,7 +168,7 @@ func selectMetas(ranges []int64, noCompactMarked map[ulid.ULID]*metadata.NoCompa // selectOverlappingMetas returns all dirs with overlapping time ranges. // It expects sorted input by mint and returns the overlapping dirs in the same order as received. // Copied and adjusted from https://github.com/prometheus/prometheus/blob/3d8826a3d42566684283a9b7f7e812e412c24407/tsdb/compact.go#L268. -func selectOverlappingMetas(metasByMinTime []*metadata.Meta, maxTasks int) []CompactionTask { +func selectOverlappingMetas(metasByMinTime []*metadata.Meta, maxTasks int) []CompactionBlocks { if len(metasByMinTime) < 2 { return nil } @@ -212,7 +212,7 @@ loopMetas: } } - overlappingGroups := make([]CompactionTask, 0, len(groups)) + overlappingGroups := make([]CompactionBlocks, 0, len(groups)) for _, group := range groups { if len(group) < 2 { continue @@ -294,7 +294,7 @@ func WithLargeTotalIndexSizeFilter(with *tsdbBasedPlanner, bkt objstore.Bucket, return &largeTotalIndexSizeFilter{tsdbBasedPlanner: with, bkt: bkt, totalMaxIndexSizeBytes: totalMaxIndexSizeBytes, markedForNoCompact: markedForNoCompact} } -func (t *largeTotalIndexSizeFilter) Plan(ctx context.Context, metasByMinTime []*metadata.Meta) ([]CompactionTask, error) { +func (t *largeTotalIndexSizeFilter) Plan(ctx context.Context, metasByMinTime []*metadata.Meta) ([]CompactionBlocks, error) { noCompactMarked := t.noCompBlocksFunc() copiedNoCompactMarked := make(map[ulid.ULID]*metadata.NoCompactMark, len(noCompactMarked)) for k, v := range noCompactMarked { diff --git a/pkg/compact/planner_test.go b/pkg/compact/planner_test.go index 7fc0f67a6a..3998bb0a75 100644 --- a/pkg/compact/planner_test.go +++ b/pkg/compact/planner_test.go @@ -32,7 +32,7 @@ type tsdbPlannerAdapter struct { comp tsdb.Compactor } -func (p *tsdbPlannerAdapter) Plan(_ context.Context, metasByMinTime []*metadata.Meta) ([]CompactionTask, error) { +func (p *tsdbPlannerAdapter) Plan(_ context.Context, metasByMinTime []*metadata.Meta) ([]CompactionBlocks, error) { // TSDB planning works based on the meta.json files in the given dir. Mock it up. for _, meta := range metasByMinTime { bdir := filepath.Join(p.dir, meta.ULID.String()) @@ -60,7 +60,7 @@ func (p *tsdbPlannerAdapter) Plan(_ context.Context, metasByMinTime []*metadata. } res = append(res, meta) } - return []CompactionTask{res}, nil + return []CompactionBlocks{res}, nil } // Adapted from https://github.com/prometheus/prometheus/blob/6c56a1faaaad07317ff585bda75b99bdba0517ad/tsdb/compact_test.go#L167 @@ -82,7 +82,7 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { for _, c := range []struct { name string metas []*metadata.Meta - expected []CompactionTask + expected []CompactionBlocks }{ { name: "Outside range", @@ -114,7 +114,7 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 40, MaxTime: 60}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(4, nil), MinTime: 60, MaxTime: 80}}, }, - expected: []CompactionTask{{ + expected: []CompactionBlocks{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(1, nil), MinTime: 0, MaxTime: 20}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 20, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 40, MaxTime: 60}}, @@ -129,7 +129,7 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(9, nil), MinTime: 180, MaxTime: 200}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(10, nil), MinTime: 200, MaxTime: 220}}, }, - expected: []CompactionTask{ + expected: []CompactionBlocks{ { {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(6, nil), MinTime: 0, MaxTime: 60}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(7, nil), MinTime: 60, MaxTime: 120}}, @@ -155,7 +155,7 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(4, nil), MinTime: 60, MaxTime: 80}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(5, nil), MinTime: 80, MaxTime: 100}}, }, - expected: []CompactionTask{{ + expected: []CompactionBlocks{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(1, nil), MinTime: 0, MaxTime: 20}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 20, MaxTime: 40}}, }}, @@ -169,7 +169,7 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(4, nil), MinTime: 60, MaxTime: 120}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(5, nil), MinTime: 120, MaxTime: 180}}, }, - expected: []CompactionTask{{ + expected: []CompactionBlocks{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(1, nil), MinTime: 0, MaxTime: 20}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 20, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 40, MaxTime: 60}}, @@ -183,7 +183,7 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(9, nil), MinTime: 180, MaxTime: 200}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(10, nil), MinTime: 200, MaxTime: 220}}, }, - expected: []CompactionTask{{ + expected: []CompactionBlocks{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(6, nil), MinTime: 0, MaxTime: 60}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(8, nil), MinTime: 120, MaxTime: 180}}, }}, @@ -197,7 +197,7 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(6, nil), MinTime: 120, MaxTime: 180}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(7, nil), MinTime: 720, MaxTime: 960}}, }, - expected: []CompactionTask{{ + expected: []CompactionBlocks{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 20, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(4, nil), MinTime: 60, MaxTime: 120}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(6, nil), MinTime: 120, MaxTime: 180}}, @@ -219,7 +219,7 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { }}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 540, MaxTime: 560}}, }, - expected: []CompactionTask{{ + expected: []CompactionBlocks{{ {BlockMeta: tsdb.BlockMeta{ Version: 1, ULID: ulid.MustNew(1, nil), MinTime: 0, MaxTime: 540, Stats: tsdb.BlockStats{ NumSeries: 10, @@ -259,7 +259,7 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(7, nil), MinTime: 360, MaxTime: 420}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(8, nil), MinTime: 420, MaxTime: 540}}, }, - expected: []CompactionTask{{ + expected: []CompactionBlocks{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(7, nil), MinTime: 360, MaxTime: 420}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(8, nil), MinTime: 420, MaxTime: 540}}, }}, @@ -274,7 +274,7 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 19, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 40, MaxTime: 60}}, }, - expected: []CompactionTask{{ + expected: []CompactionBlocks{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(1, nil), MinTime: 0, MaxTime: 20}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 19, MaxTime: 40}}, }}, @@ -289,7 +289,7 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 20, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 30, MaxTime: 50}}, }, - expected: []CompactionTask{{ + expected: []CompactionBlocks{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 20, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 30, MaxTime: 50}}, }}, @@ -304,7 +304,7 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 10, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 30, MaxTime: 50}}, }, - expected: []CompactionTask{{ + expected: []CompactionBlocks{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(1, nil), MinTime: 0, MaxTime: 20}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 10, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 30, MaxTime: 50}}, @@ -322,7 +322,7 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(7, nil), MinTime: 360, MaxTime: 420}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(8, nil), MinTime: 420, MaxTime: 540}}, }, - expected: []CompactionTask{{ + expected: []CompactionBlocks{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(5, nil), MinTime: 0, MaxTime: 360}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(6, nil), MinTime: 340, MaxTime: 560}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(7, nil), MinTime: 360, MaxTime: 420}}, @@ -343,7 +343,7 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(4, nil), MinTime: 35, MaxTime: 50}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(5, nil), MinTime: 47, MaxTime: 60}}, }, - expected: []CompactionTask{ + expected: []CompactionBlocks{ { {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(1, nil), MinTime: 0, MaxTime: 10}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 9, MaxTime: 20}}, @@ -465,12 +465,12 @@ func TestRangeWithFailedCompactionWontGetSelected(t *testing.T) { tsdbPlanner.dir = dir plan, err := tsdbPlanner.Plan(context.Background(), c.metas) testutil.Ok(t, err) - testutil.Equals(t, []CompactionTask(nil), plan) + testutil.Equals(t, []CompactionBlocks(nil), plan) }) t.Run("tsdbBasedPlanner", func(t *testing.T) { plan, err := tsdbBasedPlanner.Plan(context.Background(), c.metas) testutil.Ok(t, err) - testutil.Equals(t, []CompactionTask(nil), plan) + testutil.Equals(t, []CompactionBlocks(nil), plan) }) }) } @@ -493,7 +493,7 @@ func TestTSDBBasedPlanner_PlanWithNoCompactMarks(t *testing.T) { metas []*metadata.Meta noCompactMarks map[ulid.ULID]*metadata.NoCompactMark - expected []CompactionTask + expected []CompactionBlocks }{ { name: "Outside range and excluded", @@ -515,7 +515,7 @@ func TestTSDBBasedPlanner_PlanWithNoCompactMarks(t *testing.T) { noCompactMarks: map[ulid.ULID]*metadata.NoCompactMark{ ulid.MustNew(1, nil): {}, }, - expected: []CompactionTask{{ + expected: []CompactionBlocks{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 20, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 40, MaxTime: 60}}, }}, @@ -543,7 +543,7 @@ func TestTSDBBasedPlanner_PlanWithNoCompactMarks(t *testing.T) { noCompactMarks: map[ulid.ULID]*metadata.NoCompactMark{ ulid.MustNew(4, nil): {}, }, - expected: []CompactionTask{{ + expected: []CompactionBlocks{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(1, nil), MinTime: 0, MaxTime: 20}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 20, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 40, MaxTime: 60}}, @@ -561,7 +561,7 @@ func TestTSDBBasedPlanner_PlanWithNoCompactMarks(t *testing.T) { ulid.MustNew(1, nil): {}, ulid.MustNew(4, nil): {}, }, - expected: []CompactionTask{{ + expected: []CompactionBlocks{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 20, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 40, MaxTime: 60}}, }}, @@ -606,7 +606,7 @@ func TestTSDBBasedPlanner_PlanWithNoCompactMarks(t *testing.T) { noCompactMarks: map[ulid.ULID]*metadata.NoCompactMark{ ulid.MustNew(6, nil): {}, }, - expected: []CompactionTask{{ + expected: []CompactionBlocks{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 20, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(4, nil), MinTime: 60, MaxTime: 120}}, }}, @@ -687,7 +687,7 @@ func TestLargeTotalIndexSizeFilter_Plan(t *testing.T) { name string metas []*metadata.Meta - expected []CompactionTask + expected []CompactionBlocks expectedMarks float64 }{ { @@ -711,7 +711,7 @@ func TestLargeTotalIndexSizeFilter_Plan(t *testing.T) { BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(4, nil), MinTime: 60, MaxTime: 80}}, }, expectedMarks: 1, - expected: []CompactionTask{{ + expected: []CompactionBlocks{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 20, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 40, MaxTime: 60}}, }}, @@ -742,7 +742,7 @@ func TestLargeTotalIndexSizeFilter_Plan(t *testing.T) { {Thanos: metadata.Thanos{Files: []metadata.File{{RelPath: block.IndexFilename, SizeBytes: 90}}}, BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(4, nil), MinTime: 60, MaxTime: 80}}, }, - expected: []CompactionTask{{ + expected: []CompactionBlocks{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(1, nil), MinTime: 0, MaxTime: 20}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 20, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 40, MaxTime: 60}}, @@ -762,7 +762,7 @@ func TestLargeTotalIndexSizeFilter_Plan(t *testing.T) { {Thanos: metadata.Thanos{Files: []metadata.File{{RelPath: block.IndexFilename, SizeBytes: 90}}}, BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(5, nil), MinTime: 60, MaxTime: 80}}, }, - expected: []CompactionTask{{ + expected: []CompactionBlocks{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 20, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 40, MaxTime: 50}}, }}, @@ -797,7 +797,7 @@ func TestLargeTotalIndexSizeFilter_Plan(t *testing.T) { {Thanos: metadata.Thanos{Files: []metadata.File{{RelPath: block.IndexFilename, SizeBytes: 30}}}, BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(7, nil), MinTime: 720, MaxTime: 960}}, }, - expected: []CompactionTask{{ + expected: []CompactionBlocks{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 20, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(4, nil), MinTime: 60, MaxTime: 120}}, }}, @@ -838,7 +838,7 @@ func TestLargeTotalIndexSizeFilter_Plan(t *testing.T) { BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(5, nil), MinTime: 75, MaxTime: 90}}, }, expectedMarks: 1, - expected: []CompactionTask{{ + expected: []CompactionBlocks{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(4, nil), MinTime: 70, MaxTime: 80}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(5, nil), MinTime: 75, MaxTime: 90}}, }}, From c8040450fc2417a907ee3f0cce9d59500957160d Mon Sep 17 00:00:00 2001 From: Filip Petkovski Date: Sun, 4 Dec 2022 08:37:26 +0100 Subject: [PATCH 11/22] Improve flag description Signed-off-by: Filip Petkovski --- cmd/thanos/compact.go | 10 +++---- pkg/compact/compact.go | 26 ++++++++--------- pkg/compact/planner.go | 38 ++++++++++++------------ pkg/compact/planner_test.go | 58 ++++++++++++++++++------------------- 4 files changed, 65 insertions(+), 67 deletions(-) diff --git a/cmd/thanos/compact.go b/cmd/thanos/compact.go index c4cf19bbb6..9b32f9fc01 100644 --- a/cmd/thanos/compact.go +++ b/cmd/thanos/compact.go @@ -352,7 +352,7 @@ func runCompact( conf.blockFilesConcurrency, conf.compactBlocksFetchConcurrency, ) - tsdbPlanner := compact.NewPlanner(logger, levels, noCompactMarkerFilter, conf.groupCompactionConcurrency) + tsdbPlanner := compact.NewPlanner(logger, levels, noCompactMarkerFilter, conf.groupTasksConcurrency) planner := compact.WithLargeTotalIndexSizeFilter( tsdbPlanner, bkt, @@ -661,6 +661,7 @@ type compactConfig struct { cleanupBlocksInterval time.Duration compactionConcurrency int downsampleConcurrency int + groupTasksConcurrency int compactBlocksFetchConcurrency int deleteDelay model.Duration dedupReplicaLabels []string @@ -671,7 +672,6 @@ type compactConfig struct { maxBlockIndexSize units.Base2Bytes hashFunc string enableVerticalCompaction bool - groupCompactionConcurrency int dedupFunc string skipBlockWithOutOfOrderChunks bool progressCalculateInterval time.Duration @@ -747,9 +747,9 @@ func (cc *compactConfig) registerFlag(cmd extkingpin.FlagClause) { "NOTE: This flag is ignored and (enabled) when --deduplication.replica-label flag is set."). Hidden().Default("false").BoolVar(&cc.enableVerticalCompaction) - cmd.Flag("compact.group-compaction-concurrency", "The maximum number of concurrent compactions that can run within a single compaction group."+ - "A higher number means the compactor will use more resources at peak."). - Default("1").IntVar(&cc.groupCompactionConcurrency) + cmd.Flag("compact.group-concurrency", "The number of concurrent compactions from a single compaction group inside one compaction iteration."+ + "The absolute concurrency between all compactions is still limited by the compact.concurrency flag."). + Default("1").IntVar(&cc.groupTasksConcurrency) cmd.Flag("deduplication.func", "Experimental. Deduplication algorithm for merging overlapping blocks. "+ "Possible values are: \"\", \"penalty\". If no value is specified, the default compact deduplication merger is used, which performs 1:1 deduplication for samples. "+ diff --git a/pkg/compact/compact.go b/pkg/compact/compact.go index b567e76748..d897310154 100644 --- a/pkg/compact/compact.go +++ b/pkg/compact/compact.go @@ -731,7 +731,7 @@ func (rs *RetentionProgressCalculator) ProgressCalculate(ctx context.Context, gr type Planner interface { // Plan returns a list of blocks that should be compacted into single one. // The blocks can be overlapping. The provided metadata has to be ordered by minTime. - Plan(ctx context.Context, metasByMinTime []*metadata.Meta) ([]CompactionBlocks, error) + Plan(ctx context.Context, metasByMinTime []*metadata.Meta) ([]CompactionTask, error) } // Compactor provides compaction against an underlying storage of time series data. @@ -753,17 +753,16 @@ type Compactor interface { Compact(dest string, dirs []string, open []*tsdb.Block) (ulid.ULID, error) } -// CompactionTask is an independent compaction task. -type CompactionTask struct { +// GroupCompactionTask is an independent compaction task for a given compaction group. +type GroupCompactionTask struct { Group *Group - Blocks CompactionBlocks + Blocks CompactionTask Dir string - logger log.Logger } // Compact plans and runs a single compaction against the group. The compacted result // is uploaded into the bucket the blocks were retrieved from. -func (task *CompactionTask) Compact(ctx context.Context, comp Compactor) (shouldRerun bool, cerr error) { +func (task *GroupCompactionTask) Compact(ctx context.Context, comp Compactor) (shouldRerun bool, cerr error) { overlappingBlocks := false if err := task.Group.areBlocksOverlapping(nil); err != nil { overlappingBlocks = true @@ -779,7 +778,7 @@ func (task *CompactionTask) Compact(ctx context.Context, comp Compactor) (should } task.Group.compactionRunsCompleted.Inc() if err := os.RemoveAll(task.Dir); err != nil { - level.Error(task.logger).Log("msg", "failed to remove compaction group work directory", "path", task.Dir, "err", err) + level.Error(task.Group.logger).Log("msg", "failed to remove compaction group work directory", "path", task.Dir, "err", err) } }() @@ -989,26 +988,25 @@ func RepairIssue347(ctx context.Context, logger log.Logger, bkt objstore.Bucket, return nil } -func (cg *Group) GetCompactionTasks(ctx context.Context, dir string, planner Planner) ([]CompactionTask, error) { +func (cg *Group) GetCompactionTasks(ctx context.Context, dir string, planner Planner) ([]GroupCompactionTask, error) { plannedCompactions, err := planner.Plan(ctx, cg.metasByMinTime) if err != nil { return nil, err } - tasks := make([]CompactionTask, 0, len(plannedCompactions)) + tasks := make([]GroupCompactionTask, 0, len(plannedCompactions)) for i, blocks := range plannedCompactions { - tasks = append(tasks, CompactionTask{ + tasks = append(tasks, GroupCompactionTask{ Group: cg, Blocks: blocks, Dir: path.Join(dir, cg.Key(), strconv.Itoa(i)), - logger: cg.logger, }) } return tasks, nil } -func (cg *Group) compactBlocks(ctx context.Context, dir string, blocks CompactionBlocks, comp Compactor, overlappingBlocks bool) (bool, error) { +func (cg *Group) compactBlocks(ctx context.Context, dir string, blocks CompactionTask, comp Compactor, overlappingBlocks bool) (bool, error) { // Once we have a plan we need to download the actual data. compactionBegin := time.Now() begin := compactionBegin @@ -1230,7 +1228,7 @@ func (c *BucketCompactor) Compact(ctx context.Context) (rerr error) { var ( wg sync.WaitGroup workCtx, workCtxCancel = context.WithCancel(ctx) - taskChan = make(chan CompactionTask) + taskChan = make(chan GroupCompactionTask) errChan = make(chan error, c.concurrency) finishedAllTasks = true mtx sync.Mutex @@ -1320,7 +1318,7 @@ func (c *BucketCompactor) Compact(ctx context.Context) (rerr error) { level.Info(c.logger).Log("msg", "start of compactions") - tasks := make([]CompactionTask, 0) + tasks := make([]GroupCompactionTask, 0) for _, g := range groups { groupTasks, err := g.GetCompactionTasks(ctx, c.compactDir, c.planner) if err != nil { diff --git a/pkg/compact/planner.go b/pkg/compact/planner.go index b1195ca14f..6da0639cf1 100644 --- a/pkg/compact/planner.go +++ b/pkg/compact/planner.go @@ -25,8 +25,8 @@ type tsdbBasedPlanner struct { ranges []int64 - noCompBlocksFunc func() map[ulid.ULID]*metadata.NoCompactMark - groupCompactionConcurrency int + maxTasks int + noCompBlocksFunc func() map[ulid.ULID]*metadata.NoCompactMark } var _ Planner = &tsdbBasedPlanner{} @@ -34,38 +34,38 @@ var _ Planner = &tsdbBasedPlanner{} // NewTSDBBasedPlanner is planner with the same functionality as Prometheus' TSDB. // TODO(bwplotka): Consider upstreaming this to Prometheus. // It's the same functionality just without accessing filesystem. -func NewTSDBBasedPlanner(logger log.Logger, ranges []int64, groupCompactionConcurrency int) *tsdbBasedPlanner { +func NewTSDBBasedPlanner(logger log.Logger, ranges []int64, maxTasks int) *tsdbBasedPlanner { return &tsdbBasedPlanner{ logger: logger, ranges: ranges, noCompBlocksFunc: func() map[ulid.ULID]*metadata.NoCompactMark { return make(map[ulid.ULID]*metadata.NoCompactMark) }, - groupCompactionConcurrency: groupCompactionConcurrency, + maxTasks: maxTasks, } } // NewPlanner is a default Thanos planner with the same functionality as Prometheus' TSDB plus special handling of excluded blocks. // It's the same functionality just without accessing filesystem, and special handling of excluded blocks. -func NewPlanner(logger log.Logger, ranges []int64, noCompBlocks *GatherNoCompactionMarkFilter, groupCompactionConcurrency int) *tsdbBasedPlanner { +func NewPlanner(logger log.Logger, ranges []int64, noCompBlocks *GatherNoCompactionMarkFilter, maxTasks int) *tsdbBasedPlanner { return &tsdbBasedPlanner{ - logger: logger, - ranges: ranges, - noCompBlocksFunc: noCompBlocks.NoCompactMarkedBlocks, - groupCompactionConcurrency: groupCompactionConcurrency, + logger: logger, + ranges: ranges, + noCompBlocksFunc: noCompBlocks.NoCompactMarkedBlocks, + maxTasks: maxTasks, } } // TODO(bwplotka): Consider smarter algorithm, this prefers smaller iterative compactions vs big single one: https://github.com/thanos-io/thanos/issues/3405 -func (p *tsdbBasedPlanner) Plan(_ context.Context, metasByMinTime []*metadata.Meta) ([]CompactionBlocks, error) { +func (p *tsdbBasedPlanner) Plan(_ context.Context, metasByMinTime []*metadata.Meta) ([]CompactionTask, error) { return p.plan(p.noCompBlocksFunc(), metasByMinTime) } -// CompactionBlocks is a set of blocks that should be compacted together in a single compaction. +// CompactionTask is a set of blocks that should be compacted together in a single compaction run. // Multiple compaction tasks can be run in parallel even within a single compaction group. -type CompactionBlocks []*metadata.Meta +type CompactionTask []*metadata.Meta -func (p *tsdbBasedPlanner) plan(noCompactMarked map[ulid.ULID]*metadata.NoCompactMark, metasByMinTime []*metadata.Meta) ([]CompactionBlocks, error) { +func (p *tsdbBasedPlanner) plan(noCompactMarked map[ulid.ULID]*metadata.NoCompactMark, metasByMinTime []*metadata.Meta) ([]CompactionTask, error) { notExcludedMetasByMinTime := make([]*metadata.Meta, 0, len(metasByMinTime)) for _, meta := range metasByMinTime { if _, excluded := noCompactMarked[meta.ULID]; excluded { @@ -74,7 +74,7 @@ func (p *tsdbBasedPlanner) plan(noCompactMarked map[ulid.ULID]*metadata.NoCompac notExcludedMetasByMinTime = append(notExcludedMetasByMinTime, meta) } - verticalCompactions := selectOverlappingMetas(notExcludedMetasByMinTime, p.groupCompactionConcurrency) + verticalCompactions := selectOverlappingMetas(notExcludedMetasByMinTime, p.maxTasks) if len(verticalCompactions) > 0 { return verticalCompactions, nil } @@ -88,7 +88,7 @@ func (p *tsdbBasedPlanner) plan(noCompactMarked map[ulid.ULID]*metadata.NoCompac metasByMinTime = metasByMinTime[:len(metasByMinTime)-1] res := selectMetas(p.ranges, noCompactMarked, metasByMinTime) if len(res) > 0 { - return []CompactionBlocks{res}, nil + return []CompactionTask{res}, nil } // Compact any blocks with big enough time range that have >5% tombstones. @@ -99,7 +99,7 @@ func (p *tsdbBasedPlanner) plan(noCompactMarked map[ulid.ULID]*metadata.NoCompac } if float64(meta.Stats.NumTombstones)/float64(meta.Stats.NumSeries+1) > 0.05 { task := []*metadata.Meta{notExcludedMetasByMinTime[i]} - return []CompactionBlocks{task}, nil + return []CompactionTask{task}, nil } } @@ -168,7 +168,7 @@ func selectMetas(ranges []int64, noCompactMarked map[ulid.ULID]*metadata.NoCompa // selectOverlappingMetas returns all dirs with overlapping time ranges. // It expects sorted input by mint and returns the overlapping dirs in the same order as received. // Copied and adjusted from https://github.com/prometheus/prometheus/blob/3d8826a3d42566684283a9b7f7e812e412c24407/tsdb/compact.go#L268. -func selectOverlappingMetas(metasByMinTime []*metadata.Meta, maxTasks int) []CompactionBlocks { +func selectOverlappingMetas(metasByMinTime []*metadata.Meta, maxTasks int) []CompactionTask { if len(metasByMinTime) < 2 { return nil } @@ -212,7 +212,7 @@ loopMetas: } } - overlappingGroups := make([]CompactionBlocks, 0, len(groups)) + overlappingGroups := make([]CompactionTask, 0, len(groups)) for _, group := range groups { if len(group) < 2 { continue @@ -294,7 +294,7 @@ func WithLargeTotalIndexSizeFilter(with *tsdbBasedPlanner, bkt objstore.Bucket, return &largeTotalIndexSizeFilter{tsdbBasedPlanner: with, bkt: bkt, totalMaxIndexSizeBytes: totalMaxIndexSizeBytes, markedForNoCompact: markedForNoCompact} } -func (t *largeTotalIndexSizeFilter) Plan(ctx context.Context, metasByMinTime []*metadata.Meta) ([]CompactionBlocks, error) { +func (t *largeTotalIndexSizeFilter) Plan(ctx context.Context, metasByMinTime []*metadata.Meta) ([]CompactionTask, error) { noCompactMarked := t.noCompBlocksFunc() copiedNoCompactMarked := make(map[ulid.ULID]*metadata.NoCompactMark, len(noCompactMarked)) for k, v := range noCompactMarked { diff --git a/pkg/compact/planner_test.go b/pkg/compact/planner_test.go index 3998bb0a75..7fc0f67a6a 100644 --- a/pkg/compact/planner_test.go +++ b/pkg/compact/planner_test.go @@ -32,7 +32,7 @@ type tsdbPlannerAdapter struct { comp tsdb.Compactor } -func (p *tsdbPlannerAdapter) Plan(_ context.Context, metasByMinTime []*metadata.Meta) ([]CompactionBlocks, error) { +func (p *tsdbPlannerAdapter) Plan(_ context.Context, metasByMinTime []*metadata.Meta) ([]CompactionTask, error) { // TSDB planning works based on the meta.json files in the given dir. Mock it up. for _, meta := range metasByMinTime { bdir := filepath.Join(p.dir, meta.ULID.String()) @@ -60,7 +60,7 @@ func (p *tsdbPlannerAdapter) Plan(_ context.Context, metasByMinTime []*metadata. } res = append(res, meta) } - return []CompactionBlocks{res}, nil + return []CompactionTask{res}, nil } // Adapted from https://github.com/prometheus/prometheus/blob/6c56a1faaaad07317ff585bda75b99bdba0517ad/tsdb/compact_test.go#L167 @@ -82,7 +82,7 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { for _, c := range []struct { name string metas []*metadata.Meta - expected []CompactionBlocks + expected []CompactionTask }{ { name: "Outside range", @@ -114,7 +114,7 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 40, MaxTime: 60}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(4, nil), MinTime: 60, MaxTime: 80}}, }, - expected: []CompactionBlocks{{ + expected: []CompactionTask{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(1, nil), MinTime: 0, MaxTime: 20}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 20, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 40, MaxTime: 60}}, @@ -129,7 +129,7 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(9, nil), MinTime: 180, MaxTime: 200}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(10, nil), MinTime: 200, MaxTime: 220}}, }, - expected: []CompactionBlocks{ + expected: []CompactionTask{ { {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(6, nil), MinTime: 0, MaxTime: 60}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(7, nil), MinTime: 60, MaxTime: 120}}, @@ -155,7 +155,7 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(4, nil), MinTime: 60, MaxTime: 80}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(5, nil), MinTime: 80, MaxTime: 100}}, }, - expected: []CompactionBlocks{{ + expected: []CompactionTask{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(1, nil), MinTime: 0, MaxTime: 20}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 20, MaxTime: 40}}, }}, @@ -169,7 +169,7 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(4, nil), MinTime: 60, MaxTime: 120}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(5, nil), MinTime: 120, MaxTime: 180}}, }, - expected: []CompactionBlocks{{ + expected: []CompactionTask{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(1, nil), MinTime: 0, MaxTime: 20}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 20, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 40, MaxTime: 60}}, @@ -183,7 +183,7 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(9, nil), MinTime: 180, MaxTime: 200}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(10, nil), MinTime: 200, MaxTime: 220}}, }, - expected: []CompactionBlocks{{ + expected: []CompactionTask{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(6, nil), MinTime: 0, MaxTime: 60}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(8, nil), MinTime: 120, MaxTime: 180}}, }}, @@ -197,7 +197,7 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(6, nil), MinTime: 120, MaxTime: 180}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(7, nil), MinTime: 720, MaxTime: 960}}, }, - expected: []CompactionBlocks{{ + expected: []CompactionTask{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 20, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(4, nil), MinTime: 60, MaxTime: 120}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(6, nil), MinTime: 120, MaxTime: 180}}, @@ -219,7 +219,7 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { }}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 540, MaxTime: 560}}, }, - expected: []CompactionBlocks{{ + expected: []CompactionTask{{ {BlockMeta: tsdb.BlockMeta{ Version: 1, ULID: ulid.MustNew(1, nil), MinTime: 0, MaxTime: 540, Stats: tsdb.BlockStats{ NumSeries: 10, @@ -259,7 +259,7 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(7, nil), MinTime: 360, MaxTime: 420}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(8, nil), MinTime: 420, MaxTime: 540}}, }, - expected: []CompactionBlocks{{ + expected: []CompactionTask{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(7, nil), MinTime: 360, MaxTime: 420}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(8, nil), MinTime: 420, MaxTime: 540}}, }}, @@ -274,7 +274,7 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 19, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 40, MaxTime: 60}}, }, - expected: []CompactionBlocks{{ + expected: []CompactionTask{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(1, nil), MinTime: 0, MaxTime: 20}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 19, MaxTime: 40}}, }}, @@ -289,7 +289,7 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 20, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 30, MaxTime: 50}}, }, - expected: []CompactionBlocks{{ + expected: []CompactionTask{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 20, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 30, MaxTime: 50}}, }}, @@ -304,7 +304,7 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 10, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 30, MaxTime: 50}}, }, - expected: []CompactionBlocks{{ + expected: []CompactionTask{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(1, nil), MinTime: 0, MaxTime: 20}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 10, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 30, MaxTime: 50}}, @@ -322,7 +322,7 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(7, nil), MinTime: 360, MaxTime: 420}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(8, nil), MinTime: 420, MaxTime: 540}}, }, - expected: []CompactionBlocks{{ + expected: []CompactionTask{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(5, nil), MinTime: 0, MaxTime: 360}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(6, nil), MinTime: 340, MaxTime: 560}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(7, nil), MinTime: 360, MaxTime: 420}}, @@ -343,7 +343,7 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(4, nil), MinTime: 35, MaxTime: 50}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(5, nil), MinTime: 47, MaxTime: 60}}, }, - expected: []CompactionBlocks{ + expected: []CompactionTask{ { {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(1, nil), MinTime: 0, MaxTime: 10}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 9, MaxTime: 20}}, @@ -465,12 +465,12 @@ func TestRangeWithFailedCompactionWontGetSelected(t *testing.T) { tsdbPlanner.dir = dir plan, err := tsdbPlanner.Plan(context.Background(), c.metas) testutil.Ok(t, err) - testutil.Equals(t, []CompactionBlocks(nil), plan) + testutil.Equals(t, []CompactionTask(nil), plan) }) t.Run("tsdbBasedPlanner", func(t *testing.T) { plan, err := tsdbBasedPlanner.Plan(context.Background(), c.metas) testutil.Ok(t, err) - testutil.Equals(t, []CompactionBlocks(nil), plan) + testutil.Equals(t, []CompactionTask(nil), plan) }) }) } @@ -493,7 +493,7 @@ func TestTSDBBasedPlanner_PlanWithNoCompactMarks(t *testing.T) { metas []*metadata.Meta noCompactMarks map[ulid.ULID]*metadata.NoCompactMark - expected []CompactionBlocks + expected []CompactionTask }{ { name: "Outside range and excluded", @@ -515,7 +515,7 @@ func TestTSDBBasedPlanner_PlanWithNoCompactMarks(t *testing.T) { noCompactMarks: map[ulid.ULID]*metadata.NoCompactMark{ ulid.MustNew(1, nil): {}, }, - expected: []CompactionBlocks{{ + expected: []CompactionTask{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 20, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 40, MaxTime: 60}}, }}, @@ -543,7 +543,7 @@ func TestTSDBBasedPlanner_PlanWithNoCompactMarks(t *testing.T) { noCompactMarks: map[ulid.ULID]*metadata.NoCompactMark{ ulid.MustNew(4, nil): {}, }, - expected: []CompactionBlocks{{ + expected: []CompactionTask{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(1, nil), MinTime: 0, MaxTime: 20}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 20, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 40, MaxTime: 60}}, @@ -561,7 +561,7 @@ func TestTSDBBasedPlanner_PlanWithNoCompactMarks(t *testing.T) { ulid.MustNew(1, nil): {}, ulid.MustNew(4, nil): {}, }, - expected: []CompactionBlocks{{ + expected: []CompactionTask{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 20, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 40, MaxTime: 60}}, }}, @@ -606,7 +606,7 @@ func TestTSDBBasedPlanner_PlanWithNoCompactMarks(t *testing.T) { noCompactMarks: map[ulid.ULID]*metadata.NoCompactMark{ ulid.MustNew(6, nil): {}, }, - expected: []CompactionBlocks{{ + expected: []CompactionTask{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 20, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(4, nil), MinTime: 60, MaxTime: 120}}, }}, @@ -687,7 +687,7 @@ func TestLargeTotalIndexSizeFilter_Plan(t *testing.T) { name string metas []*metadata.Meta - expected []CompactionBlocks + expected []CompactionTask expectedMarks float64 }{ { @@ -711,7 +711,7 @@ func TestLargeTotalIndexSizeFilter_Plan(t *testing.T) { BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(4, nil), MinTime: 60, MaxTime: 80}}, }, expectedMarks: 1, - expected: []CompactionBlocks{{ + expected: []CompactionTask{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 20, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 40, MaxTime: 60}}, }}, @@ -742,7 +742,7 @@ func TestLargeTotalIndexSizeFilter_Plan(t *testing.T) { {Thanos: metadata.Thanos{Files: []metadata.File{{RelPath: block.IndexFilename, SizeBytes: 90}}}, BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(4, nil), MinTime: 60, MaxTime: 80}}, }, - expected: []CompactionBlocks{{ + expected: []CompactionTask{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(1, nil), MinTime: 0, MaxTime: 20}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 20, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 40, MaxTime: 60}}, @@ -762,7 +762,7 @@ func TestLargeTotalIndexSizeFilter_Plan(t *testing.T) { {Thanos: metadata.Thanos{Files: []metadata.File{{RelPath: block.IndexFilename, SizeBytes: 90}}}, BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(5, nil), MinTime: 60, MaxTime: 80}}, }, - expected: []CompactionBlocks{{ + expected: []CompactionTask{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 20, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(3, nil), MinTime: 40, MaxTime: 50}}, }}, @@ -797,7 +797,7 @@ func TestLargeTotalIndexSizeFilter_Plan(t *testing.T) { {Thanos: metadata.Thanos{Files: []metadata.File{{RelPath: block.IndexFilename, SizeBytes: 30}}}, BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(7, nil), MinTime: 720, MaxTime: 960}}, }, - expected: []CompactionBlocks{{ + expected: []CompactionTask{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(2, nil), MinTime: 20, MaxTime: 40}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(4, nil), MinTime: 60, MaxTime: 120}}, }}, @@ -838,7 +838,7 @@ func TestLargeTotalIndexSizeFilter_Plan(t *testing.T) { BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(5, nil), MinTime: 75, MaxTime: 90}}, }, expectedMarks: 1, - expected: []CompactionBlocks{{ + expected: []CompactionTask{{ {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(4, nil), MinTime: 70, MaxTime: 80}}, {BlockMeta: tsdb.BlockMeta{Version: 1, ULID: ulid.MustNew(5, nil), MinTime: 75, MaxTime: 90}}, }}, From d7543706cb492807c6b2c489a736772af19d85ba Mon Sep 17 00:00:00 2001 From: Filip Petkovski Date: Sun, 4 Dec 2022 08:42:34 +0100 Subject: [PATCH 12/22] Fix linter Signed-off-by: Filip Petkovski --- pkg/compact/compact.go | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/pkg/compact/compact.go b/pkg/compact/compact.go index d897310154..ab32335df7 100644 --- a/pkg/compact/compact.go +++ b/pkg/compact/compact.go @@ -1324,9 +1324,7 @@ func (c *BucketCompactor) Compact(ctx context.Context) (rerr error) { if err != nil { return errors.Wrapf(err, "get compaction group tasks: %s", g.Key()) } - for _, task := range groupTasks { - tasks = append(tasks, task) - } + tasks = append(tasks, groupTasks...) } // Send all groups found during this pass to the compaction workers. From 24fd3361a05ddc670cee494710b5bb5c3ea53882 Mon Sep 17 00:00:00 2001 From: Filip Petkovski Date: Sun, 4 Dec 2022 12:47:57 +0100 Subject: [PATCH 13/22] Add vertical compaction check Signed-off-by: Filip Petkovski --- pkg/compact/compact.go | 32 +++++++++++++++++++------------- 1 file changed, 19 insertions(+), 13 deletions(-) diff --git a/pkg/compact/compact.go b/pkg/compact/compact.go index ab32335df7..bd13f4fd00 100644 --- a/pkg/compact/compact.go +++ b/pkg/compact/compact.go @@ -763,11 +763,6 @@ type GroupCompactionTask struct { // Compact plans and runs a single compaction against the group. The compacted result // is uploaded into the bucket the blocks were retrieved from. func (task *GroupCompactionTask) Compact(ctx context.Context, comp Compactor) (shouldRerun bool, cerr error) { - overlappingBlocks := false - if err := task.Group.areBlocksOverlapping(nil); err != nil { - overlappingBlocks = true - } - task.Group.compactionRunsStarted.Inc() defer func() { // Leave the compact directory for inspection if it is a halt error @@ -782,12 +777,23 @@ func (task *GroupCompactionTask) Compact(ctx context.Context, comp Compactor) (s } }() + hasOverlappingBlocks := false + if err := task.Group.areBlocksOverlapping(task.Blocks, task.Group.metasByMinTime...); err != nil { + // TODO(bwplotka): It would really nice if we could still check for other overlaps than replica. In fact this should be checked + // in syncer itself. Otherwise with vertical compaction enabled we will sacrifice this important check. + if !task.Group.enableVerticalCompaction { + return false, halt(errors.Wrap(err, "pre compaction overlap check")) + } + + hasOverlappingBlocks = true + } + if err := os.MkdirAll(task.Dir, 0750); err != nil { return false, errors.Wrap(err, "create compaction group dir") } terr := tracing.DoInSpanWithErr(ctx, "compaction_group", func(ctx context.Context) (cerr error) { - shouldRerun, cerr = task.Group.compactBlocks(ctx, task.Dir, task.Blocks, comp, overlappingBlocks) + shouldRerun, cerr = task.Group.compactBlocks(ctx, task.Dir, task.Blocks, comp, hasOverlappingBlocks) return cerr }, opentracing.Tags{"group.key": task.Group.Key()}) if terr != nil { @@ -900,7 +906,7 @@ func IsRetryError(err error) bool { return ok } -func (cg *Group) areBlocksOverlapping(include *metadata.Meta, exclude ...*metadata.Meta) error { +func (cg *Group) areBlocksOverlapping(include []*metadata.Meta, exclude ...*metadata.Meta) error { var ( metas []tsdb.BlockMeta excludeMap = map[ulid.ULID]struct{}{} @@ -917,8 +923,8 @@ func (cg *Group) areBlocksOverlapping(include *metadata.Meta, exclude ...*metada metas = append(metas, m.BlockMeta) } - if include != nil { - metas = append(metas, include.BlockMeta) + for _, meta := range include { + metas = append(metas, meta.BlockMeta) } sort.Slice(metas, func(i, j int) bool { @@ -1006,7 +1012,7 @@ func (cg *Group) GetCompactionTasks(ctx context.Context, dir string, planner Pla return tasks, nil } -func (cg *Group) compactBlocks(ctx context.Context, dir string, blocks CompactionTask, comp Compactor, overlappingBlocks bool) (bool, error) { +func (cg *Group) compactBlocks(ctx context.Context, dir string, blocks CompactionTask, comp Compactor, hasOverlappingBlocks bool) (bool, error) { // Once we have a plan we need to download the actual data. compactionBegin := time.Now() begin := compactionBegin @@ -1087,11 +1093,11 @@ func (cg *Group) compactBlocks(ctx context.Context, dir string, blocks Compactio return true, nil } cg.compactions.Inc() - if overlappingBlocks { + if hasOverlappingBlocks { cg.verticalCompactions.Inc() } level.Info(cg.logger).Log("msg", "compacted blocks", "new", compID, - "blocks", sourceBlockStr, "duration", time.Since(begin), "duration_ms", time.Since(begin).Milliseconds(), "overlapping_blocks", overlappingBlocks) + "blocks", sourceBlockStr, "duration", time.Since(begin), "duration_ms", time.Since(begin).Milliseconds(), "overlapping_blocks", hasOverlappingBlocks) bdir := filepath.Join(dir, compID.String()) index := filepath.Join(bdir, block.IndexFilename) @@ -1121,7 +1127,7 @@ func (cg *Group) compactBlocks(ctx context.Context, dir string, blocks Compactio // Ensure the output block is not overlapping with anything else, // unless vertical compaction is enabled. if !cg.enableVerticalCompaction { - if err := cg.areBlocksOverlapping(newMeta, blocks...); err != nil { + if err := cg.areBlocksOverlapping([]*metadata.Meta{newMeta}, blocks...); err != nil { return false, halt(errors.Wrapf(err, "resulted compacted block %s overlaps with something", bdir)) } } From a808840816fae6c26c6cd7a86ec5d1a8353ee27e Mon Sep 17 00:00:00 2001 From: Filip Petkovski Date: Sun, 4 Dec 2022 15:10:05 +0100 Subject: [PATCH 14/22] Adjust e2e tests to not account for empty planning cycles Signed-off-by: Filip Petkovski --- pkg/compact/compact.go | 23 ++++++++++++----------- pkg/compact/compact_e2e_test.go | 8 ++++---- test/e2e/compact_test.go | 14 +++++++------- 3 files changed, 23 insertions(+), 22 deletions(-) diff --git a/pkg/compact/compact.go b/pkg/compact/compact.go index bd13f4fd00..e20552cb6a 100644 --- a/pkg/compact/compact.go +++ b/pkg/compact/compact.go @@ -994,7 +994,7 @@ func RepairIssue347(ctx context.Context, logger log.Logger, bkt objstore.Bucket, return nil } -func (cg *Group) GetCompactionTasks(ctx context.Context, dir string, planner Planner) ([]GroupCompactionTask, error) { +func (cg *Group) PlanCompactionTasks(ctx context.Context, dir string, planner Planner) ([]GroupCompactionTask, error) { plannedCompactions, err := planner.Plan(ctx, cg.metasByMinTime) if err != nil { return nil, err @@ -1326,25 +1326,26 @@ func (c *BucketCompactor) Compact(ctx context.Context) (rerr error) { tasks := make([]GroupCompactionTask, 0) for _, g := range groups { - groupTasks, err := g.GetCompactionTasks(ctx, c.compactDir, c.planner) + // Ignore groups with only one block because there is nothing to compact. + if len(g.IDs()) == 1 { + continue + } + + groupTasks, err := g.PlanCompactionTasks(ctx, c.compactDir, c.planner) if err != nil { return errors.Wrapf(err, "get compaction group tasks: %s", g.Key()) } tasks = append(tasks, groupTasks...) } - // Send all groups found during this pass to the compaction workers. + // Send all tasks planned in this pass to the compaction workers. var taskErrs errutil.MultiError - groupLoop: + tasksLoop: for _, task := range tasks { - // Ignore groups with only one block because there is nothing to compact. - if len(task.Blocks) == 1 { - continue - } select { - case groupErr := <-errChan: - taskErrs.Add(groupErr) - break groupLoop + case taskErr := <-errChan: + taskErrs.Add(taskErr) + break tasksLoop case taskChan <- task: } } diff --git a/pkg/compact/compact_e2e_test.go b/pkg/compact/compact_e2e_test.go index 74e680924d..e2b16c8e97 100644 --- a/pkg/compact/compact_e2e_test.go +++ b/pkg/compact/compact_e2e_test.go @@ -330,13 +330,13 @@ func testGroupCompactE2e(t *testing.T, mergeFunc storage.VerticalChunkSeriesMerg testutil.Equals(t, 0.0, promtest.ToFloat64(grouper.compactions.WithLabelValues(metas[4].Thanos.GroupKey()))) testutil.Equals(t, 0.0, promtest.ToFloat64(grouper.compactions.WithLabelValues(metas[5].Thanos.GroupKey()))) testutil.Equals(t, 4, MetricCount(grouper.compactionRunsStarted)) - testutil.Equals(t, 3.0, promtest.ToFloat64(grouper.compactionRunsStarted.WithLabelValues(metas[0].Thanos.GroupKey()))) - testutil.Equals(t, 3.0, promtest.ToFloat64(grouper.compactionRunsStarted.WithLabelValues(metas[7].Thanos.GroupKey()))) + testutil.Equals(t, 2.0, promtest.ToFloat64(grouper.compactionRunsStarted.WithLabelValues(metas[0].Thanos.GroupKey()))) + testutil.Equals(t, 1.0, promtest.ToFloat64(grouper.compactionRunsStarted.WithLabelValues(metas[7].Thanos.GroupKey()))) testutil.Equals(t, 0.0, promtest.ToFloat64(grouper.compactionRunsStarted.WithLabelValues(metas[4].Thanos.GroupKey()))) testutil.Equals(t, 0.0, promtest.ToFloat64(grouper.compactionRunsStarted.WithLabelValues(metas[5].Thanos.GroupKey()))) testutil.Equals(t, 4, MetricCount(grouper.compactionRunsCompleted)) - testutil.Equals(t, 2.0, promtest.ToFloat64(grouper.compactionRunsCompleted.WithLabelValues(metas[0].Thanos.GroupKey()))) - testutil.Equals(t, 3.0, promtest.ToFloat64(grouper.compactionRunsCompleted.WithLabelValues(metas[7].Thanos.GroupKey()))) + testutil.Equals(t, 1.0, promtest.ToFloat64(grouper.compactionRunsCompleted.WithLabelValues(metas[0].Thanos.GroupKey()))) + testutil.Equals(t, 1.0, promtest.ToFloat64(grouper.compactionRunsCompleted.WithLabelValues(metas[7].Thanos.GroupKey()))) testutil.Equals(t, 0.0, promtest.ToFloat64(grouper.compactionRunsCompleted.WithLabelValues(metas[4].Thanos.GroupKey()))) testutil.Equals(t, 0.0, promtest.ToFloat64(grouper.compactionRunsCompleted.WithLabelValues(metas[5].Thanos.GroupKey()))) testutil.Equals(t, 4, MetricCount(grouper.compactionFailures)) diff --git a/test/e2e/compact_test.go b/test/e2e/compact_test.go index 5686d6afaf..4f758b8b31 100644 --- a/test/e2e/compact_test.go +++ b/test/e2e/compact_test.go @@ -649,8 +649,8 @@ func testCompactWithStoreGateway(t *testing.T, penaltyDedup bool) { testutil.Ok(t, c.WaitSumMetrics(e2emon.Equals(0), "thanos_compact_group_compactions_total")) testutil.Ok(t, c.WaitSumMetrics(e2emon.Equals(0), "thanos_compact_group_vertical_compactions_total")) testutil.Ok(t, c.WaitSumMetrics(e2emon.Equals(1), "thanos_compact_group_compactions_failures_total")) - testutil.Ok(t, c.WaitSumMetrics(e2emon.Equals(2), "thanos_compact_group_compaction_runs_started_total")) - testutil.Ok(t, c.WaitSumMetrics(e2emon.Equals(1), "thanos_compact_group_compaction_runs_completed_total")) + testutil.Ok(t, c.WaitSumMetrics(e2emon.Equals(1), "thanos_compact_group_compaction_runs_started_total")) + testutil.Ok(t, c.WaitSumMetrics(e2emon.Equals(0), "thanos_compact_group_compaction_runs_completed_total")) // However, the blocks have been cleaned because that happens concurrently. testutil.Ok(t, c.WaitSumMetrics(e2emon.Equals(2), "thanos_compact_aborted_partial_uploads_deletion_attempts_total")) @@ -702,8 +702,8 @@ func testCompactWithStoreGateway(t *testing.T, penaltyDedup bool) { testutil.Ok(t, c.WaitSumMetrics(e2emon.Equals(6), "thanos_compact_group_compactions_total")) testutil.Ok(t, c.WaitSumMetrics(e2emon.Equals(3), "thanos_compact_group_vertical_compactions_total")) testutil.Ok(t, c.WaitSumMetrics(e2emon.Equals(0), "thanos_compact_group_compactions_failures_total")) - testutil.Ok(t, c.WaitSumMetrics(e2emon.Equals(14), "thanos_compact_group_compaction_runs_started_total")) - testutil.Ok(t, c.WaitSumMetrics(e2emon.Equals(14), "thanos_compact_group_compaction_runs_completed_total")) + testutil.Ok(t, c.WaitSumMetrics(e2emon.Equals(6), "thanos_compact_group_compaction_runs_started_total")) + testutil.Ok(t, c.WaitSumMetrics(e2emon.Equals(6), "thanos_compact_group_compaction_runs_completed_total")) testutil.Ok(t, c.WaitSumMetrics(e2emon.Equals(2), "thanos_compact_downsample_total")) testutil.Ok(t, c.WaitSumMetrics(e2emon.Equals(0), "thanos_compact_downsample_failures_total")) @@ -723,7 +723,7 @@ func testCompactWithStoreGateway(t *testing.T, penaltyDedup bool) { operationMatcher, err := matchers.NewMatcher(matchers.MatchEqual, "operation", "get") testutil.Ok(t, err) testutil.Ok(t, c.WaitSumMetricsWithOptions( - e2emon.Equals(573), + e2emon.Equals(635), []string{"thanos_objstore_bucket_operations_total"}, e2emon.WithLabelMatchers( bucketMatcher, operationMatcher, @@ -773,8 +773,8 @@ func testCompactWithStoreGateway(t *testing.T, penaltyDedup bool) { testutil.Ok(t, c.WaitSumMetricsWithOptions(e2emon.Equals(0), []string{"thanos_compact_group_compactions_total"}, e2emon.WaitMissingMetrics())) testutil.Ok(t, c.WaitSumMetricsWithOptions(e2emon.Equals(0), []string{"thanos_compact_group_vertical_compactions_total"}, e2emon.WaitMissingMetrics())) testutil.Ok(t, c.WaitSumMetricsWithOptions(e2emon.Equals(0), []string{"thanos_compact_group_compactions_failures_total"}, e2emon.WaitMissingMetrics())) - testutil.Ok(t, c.WaitSumMetricsWithOptions(e2emon.Equals(7), []string{"thanos_compact_group_compaction_runs_started_total"}, e2emon.WaitMissingMetrics())) - testutil.Ok(t, c.WaitSumMetricsWithOptions(e2emon.Equals(7), []string{"thanos_compact_group_compaction_runs_completed_total"}, e2emon.WaitMissingMetrics())) + testutil.Ok(t, c.WaitSumMetricsWithOptions(e2emon.Equals(0), []string{"thanos_compact_group_compaction_runs_started_total"}, e2emon.WaitMissingMetrics())) + testutil.Ok(t, c.WaitSumMetricsWithOptions(e2emon.Equals(0), []string{"thanos_compact_group_compaction_runs_completed_total"}, e2emon.WaitMissingMetrics())) testutil.Ok(t, c.WaitSumMetricsWithOptions(e2emon.Equals(0), []string{"thanos_compact_downsample_total"}, e2emon.WaitMissingMetrics())) testutil.Ok(t, c.WaitSumMetricsWithOptions(e2emon.Equals(0), []string{"thanos_compact_downsample_failures_total"}, e2emon.WaitMissingMetrics())) From 7c6dba54278420bc2cb2a520c67ac14992a57875 Mon Sep 17 00:00:00 2001 From: Filip Petkovski Date: Sun, 4 Dec 2022 16:32:49 +0100 Subject: [PATCH 15/22] Run make docs Signed-off-by: Filip Petkovski --- cmd/thanos/compact.go | 2 +- docs/components/compact.md | 6 ++++++ 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/cmd/thanos/compact.go b/cmd/thanos/compact.go index 9b32f9fc01..ebddfff521 100644 --- a/cmd/thanos/compact.go +++ b/cmd/thanos/compact.go @@ -747,7 +747,7 @@ func (cc *compactConfig) registerFlag(cmd extkingpin.FlagClause) { "NOTE: This flag is ignored and (enabled) when --deduplication.replica-label flag is set."). Hidden().Default("false").BoolVar(&cc.enableVerticalCompaction) - cmd.Flag("compact.group-concurrency", "The number of concurrent compactions from a single compaction group inside one compaction iteration."+ + cmd.Flag("compact.group-concurrency", "The number of concurrent compactions from a single compaction group inside one compaction iteration. "+ "The absolute concurrency between all compactions is still limited by the compact.concurrency flag."). Default("1").IntVar(&cc.groupTasksConcurrency) diff --git a/docs/components/compact.md b/docs/components/compact.md index bf81a7f70c..5672f1d3a3 100644 --- a/docs/components/compact.md +++ b/docs/components/compact.md @@ -308,6 +308,12 @@ Flags: happen at the end of an iteration. --compact.concurrency=1 Number of goroutines to use when compacting groups. + --compact.group-concurrency=1 + The number of concurrent compactions from a + single compaction group inside one compaction + iteration. The absolute concurrency between + all compactions is still limited by the + compact.concurrency flag. --compact.progress-interval=5m Frequency of calculating the compaction progress in the background when --wait has been enabled. From a7d4a7b8c8aa9c798f21911b0b6d7fb093ba21c8 Mon Sep 17 00:00:00 2001 From: Filip Petkovski Date: Wed, 7 Dec 2022 12:08:29 +0100 Subject: [PATCH 16/22] Automatically infer group concurrency Signed-off-by: Filip Petkovski --- cmd/thanos/compact.go | 7 +------ pkg/compact/compact.go | 9 +++++++-- pkg/compact/compact_e2e_test.go | 2 +- pkg/compact/compact_test.go | 2 +- pkg/compact/planner.go | 15 ++++----------- pkg/compact/planner_test.go | 10 ++++------ 6 files changed, 18 insertions(+), 27 deletions(-) diff --git a/cmd/thanos/compact.go b/cmd/thanos/compact.go index ebddfff521..6cb4eae3e9 100644 --- a/cmd/thanos/compact.go +++ b/cmd/thanos/compact.go @@ -352,7 +352,7 @@ func runCompact( conf.blockFilesConcurrency, conf.compactBlocksFetchConcurrency, ) - tsdbPlanner := compact.NewPlanner(logger, levels, noCompactMarkerFilter, conf.groupTasksConcurrency) + tsdbPlanner := compact.NewPlanner(logger, levels, noCompactMarkerFilter) planner := compact.WithLargeTotalIndexSizeFilter( tsdbPlanner, bkt, @@ -661,7 +661,6 @@ type compactConfig struct { cleanupBlocksInterval time.Duration compactionConcurrency int downsampleConcurrency int - groupTasksConcurrency int compactBlocksFetchConcurrency int deleteDelay model.Duration dedupReplicaLabels []string @@ -747,10 +746,6 @@ func (cc *compactConfig) registerFlag(cmd extkingpin.FlagClause) { "NOTE: This flag is ignored and (enabled) when --deduplication.replica-label flag is set."). Hidden().Default("false").BoolVar(&cc.enableVerticalCompaction) - cmd.Flag("compact.group-concurrency", "The number of concurrent compactions from a single compaction group inside one compaction iteration. "+ - "The absolute concurrency between all compactions is still limited by the compact.concurrency flag."). - Default("1").IntVar(&cc.groupTasksConcurrency) - cmd.Flag("deduplication.func", "Experimental. Deduplication algorithm for merging overlapping blocks. "+ "Possible values are: \"\", \"penalty\". If no value is specified, the default compact deduplication merger is used, which performs 1:1 deduplication for samples. "+ "When set to penalty, penalty based deduplication algorithm will be used. At least one replica label has to be set via --deduplication.replica-label flag."). diff --git a/pkg/compact/compact.go b/pkg/compact/compact.go index e20552cb6a..557e2d0088 100644 --- a/pkg/compact/compact.go +++ b/pkg/compact/compact.go @@ -994,7 +994,7 @@ func RepairIssue347(ctx context.Context, logger log.Logger, bkt objstore.Bucket, return nil } -func (cg *Group) PlanCompactionTasks(ctx context.Context, dir string, planner Planner) ([]GroupCompactionTask, error) { +func (cg *Group) PlanCompactionTasks(ctx context.Context, dir string, planner Planner, maxTasks int) ([]GroupCompactionTask, error) { plannedCompactions, err := planner.Plan(ctx, cg.metasByMinTime) if err != nil { return nil, err @@ -1009,6 +1009,10 @@ func (cg *Group) PlanCompactionTasks(ctx context.Context, dir string, planner Pl }) } + if len(tasks) > maxTasks { + tasks = tasks[:maxTasks] + } + return tasks, nil } @@ -1325,13 +1329,14 @@ func (c *BucketCompactor) Compact(ctx context.Context) (rerr error) { level.Info(c.logger).Log("msg", "start of compactions") tasks := make([]GroupCompactionTask, 0) + tasksPerGroup := c.concurrency / len(groups) for _, g := range groups { // Ignore groups with only one block because there is nothing to compact. if len(g.IDs()) == 1 { continue } - groupTasks, err := g.PlanCompactionTasks(ctx, c.compactDir, c.planner) + groupTasks, err := g.PlanCompactionTasks(ctx, c.compactDir, c.planner, tasksPerGroup) if err != nil { return errors.Wrapf(err, "get compaction group tasks: %s", g.Key()) } diff --git a/pkg/compact/compact_e2e_test.go b/pkg/compact/compact_e2e_test.go index e2b16c8e97..9709b23ea3 100644 --- a/pkg/compact/compact_e2e_test.go +++ b/pkg/compact/compact_e2e_test.go @@ -210,7 +210,7 @@ func testGroupCompactE2e(t *testing.T, mergeFunc storage.VerticalChunkSeriesMerg comp, err := tsdb.NewLeveledCompactor(ctx, reg, logger, []int64{1000, 3000}, nil, mergeFunc) testutil.Ok(t, err) - planner := NewPlanner(logger, []int64{1000, 3000}, noCompactMarkerFilter, testGroupConcurrency) + planner := NewPlanner(logger, []int64{1000, 3000}, noCompactMarkerFilter) grouper := NewDefaultGrouper(logger, bkt, false, false, reg, blocksMarkedForDeletion, garbageCollectedBlocks, blocksMaredForNoCompact, metadata.NoneFunc, 10, 10) bComp, err := NewBucketCompactor(logger, sy, grouper, planner, comp, dir, bkt, 2, true) testutil.Ok(t, err) diff --git a/pkg/compact/compact_test.go b/pkg/compact/compact_test.go index c811b938ed..6ca8f74239 100644 --- a/pkg/compact/compact_test.go +++ b/pkg/compact/compact_test.go @@ -360,7 +360,7 @@ func TestCompactProgressCalculate(t *testing.T) { int64(2 * time.Hour / time.Millisecond), int64(4 * time.Hour / time.Millisecond), int64(8 * time.Hour / time.Millisecond), - }, testGroupConcurrency) + }) keys := make([]string, 3) m := make([]metadata.Meta, 3) diff --git a/pkg/compact/planner.go b/pkg/compact/planner.go index 6da0639cf1..6f767e6aba 100644 --- a/pkg/compact/planner.go +++ b/pkg/compact/planner.go @@ -25,7 +25,6 @@ type tsdbBasedPlanner struct { ranges []int64 - maxTasks int noCompBlocksFunc func() map[ulid.ULID]*metadata.NoCompactMark } @@ -34,25 +33,23 @@ var _ Planner = &tsdbBasedPlanner{} // NewTSDBBasedPlanner is planner with the same functionality as Prometheus' TSDB. // TODO(bwplotka): Consider upstreaming this to Prometheus. // It's the same functionality just without accessing filesystem. -func NewTSDBBasedPlanner(logger log.Logger, ranges []int64, maxTasks int) *tsdbBasedPlanner { +func NewTSDBBasedPlanner(logger log.Logger, ranges []int64) *tsdbBasedPlanner { return &tsdbBasedPlanner{ logger: logger, ranges: ranges, noCompBlocksFunc: func() map[ulid.ULID]*metadata.NoCompactMark { return make(map[ulid.ULID]*metadata.NoCompactMark) }, - maxTasks: maxTasks, } } // NewPlanner is a default Thanos planner with the same functionality as Prometheus' TSDB plus special handling of excluded blocks. // It's the same functionality just without accessing filesystem, and special handling of excluded blocks. -func NewPlanner(logger log.Logger, ranges []int64, noCompBlocks *GatherNoCompactionMarkFilter, maxTasks int) *tsdbBasedPlanner { +func NewPlanner(logger log.Logger, ranges []int64, noCompBlocks *GatherNoCompactionMarkFilter) *tsdbBasedPlanner { return &tsdbBasedPlanner{ logger: logger, ranges: ranges, noCompBlocksFunc: noCompBlocks.NoCompactMarkedBlocks, - maxTasks: maxTasks, } } @@ -74,7 +71,7 @@ func (p *tsdbBasedPlanner) plan(noCompactMarked map[ulid.ULID]*metadata.NoCompac notExcludedMetasByMinTime = append(notExcludedMetasByMinTime, meta) } - verticalCompactions := selectOverlappingMetas(notExcludedMetasByMinTime, p.maxTasks) + verticalCompactions := selectOverlappingMetas(notExcludedMetasByMinTime) if len(verticalCompactions) > 0 { return verticalCompactions, nil } @@ -168,7 +165,7 @@ func selectMetas(ranges []int64, noCompactMarked map[ulid.ULID]*metadata.NoCompa // selectOverlappingMetas returns all dirs with overlapping time ranges. // It expects sorted input by mint and returns the overlapping dirs in the same order as received. // Copied and adjusted from https://github.com/prometheus/prometheus/blob/3d8826a3d42566684283a9b7f7e812e412c24407/tsdb/compact.go#L268. -func selectOverlappingMetas(metasByMinTime []*metadata.Meta, maxTasks int) []CompactionTask { +func selectOverlappingMetas(metasByMinTime []*metadata.Meta) []CompactionTask { if len(metasByMinTime) < 2 { return nil } @@ -223,10 +220,6 @@ loopMetas: overlappingGroups = append(overlappingGroups, group) } - if len(overlappingGroups) > maxTasks { - return overlappingGroups[:maxTasks] - } - return overlappingGroups } diff --git a/pkg/compact/planner_test.go b/pkg/compact/planner_test.go index 7fc0f67a6a..6474c59641 100644 --- a/pkg/compact/planner_test.go +++ b/pkg/compact/planner_test.go @@ -25,8 +25,6 @@ import ( "github.com/thanos-io/thanos/pkg/testutil" ) -const testGroupConcurrency = 3 - type tsdbPlannerAdapter struct { dir string comp tsdb.Compactor @@ -77,7 +75,7 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { tsdbComp, err := tsdb.NewLeveledCompactor(context.Background(), nil, nil, ranges, nil, nil) testutil.Ok(t, err) tsdbPlanner := &tsdbPlannerAdapter{comp: tsdbComp} - tsdbBasedPlanner := NewTSDBBasedPlanner(log.NewNopLogger(), ranges, testGroupConcurrency) + tsdbBasedPlanner := NewTSDBBasedPlanner(log.NewNopLogger(), ranges) for _, c := range []struct { name string @@ -422,7 +420,7 @@ func TestRangeWithFailedCompactionWontGetSelected(t *testing.T) { tsdbComp, err := tsdb.NewLeveledCompactor(context.Background(), nil, nil, ranges, nil, nil) testutil.Ok(t, err) tsdbPlanner := &tsdbPlannerAdapter{comp: tsdbComp} - tsdbBasedPlanner := NewTSDBBasedPlanner(log.NewNopLogger(), ranges, testGroupConcurrency) + tsdbBasedPlanner := NewTSDBBasedPlanner(log.NewNopLogger(), ranges) for _, c := range []struct { metas []*metadata.Meta @@ -486,7 +484,7 @@ func TestTSDBBasedPlanner_PlanWithNoCompactMarks(t *testing.T) { } g := &GatherNoCompactionMarkFilter{} - tsdbBasedPlanner := NewPlanner(log.NewNopLogger(), ranges, g, testGroupConcurrency) + tsdbBasedPlanner := NewPlanner(log.NewNopLogger(), ranges, g) for _, c := range []struct { name string @@ -681,7 +679,7 @@ func TestLargeTotalIndexSizeFilter_Plan(t *testing.T) { g := &GatherNoCompactionMarkFilter{} marked := promauto.With(nil).NewCounter(prometheus.CounterOpts{}) - planner := WithLargeTotalIndexSizeFilter(NewPlanner(log.NewNopLogger(), ranges, g, testGroupConcurrency), bkt, 100, marked) + planner := WithLargeTotalIndexSizeFilter(NewPlanner(log.NewNopLogger(), ranges, g), bkt, 100, marked) var lastMarkValue float64 for _, c := range []struct { name string From f476edb040048a421f51315b9f3bb0ee5439932c Mon Sep 17 00:00:00 2001 From: Filip Petkovski Date: Wed, 7 Dec 2022 12:12:46 +0100 Subject: [PATCH 17/22] Run make docs Signed-off-by: Filip Petkovski --- docs/components/compact.md | 6 ------ 1 file changed, 6 deletions(-) diff --git a/docs/components/compact.md b/docs/components/compact.md index 5672f1d3a3..bf81a7f70c 100644 --- a/docs/components/compact.md +++ b/docs/components/compact.md @@ -308,12 +308,6 @@ Flags: happen at the end of an iteration. --compact.concurrency=1 Number of goroutines to use when compacting groups. - --compact.group-concurrency=1 - The number of concurrent compactions from a - single compaction group inside one compaction - iteration. The absolute concurrency between - all compactions is still limited by the - compact.concurrency flag. --compact.progress-interval=5m Frequency of calculating the compaction progress in the background when --wait has been enabled. From 2662961592c25147f9aa4b64a3282e00d6692a72 Mon Sep 17 00:00:00 2001 From: Filip Petkovski Date: Wed, 7 Dec 2022 12:42:43 +0100 Subject: [PATCH 18/22] Fix tests Signed-off-by: Filip Petkovski --- pkg/compact/compact.go | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/pkg/compact/compact.go b/pkg/compact/compact.go index 557e2d0088..9aeb297dad 100644 --- a/pkg/compact/compact.go +++ b/pkg/compact/compact.go @@ -1328,8 +1328,11 @@ func (c *BucketCompactor) Compact(ctx context.Context) (rerr error) { level.Info(c.logger).Log("msg", "start of compactions") + tasksPerGroup := 1 + if len(groups) > 0 && c.concurrency > len(groups) { + tasksPerGroup = c.concurrency / len(groups) + } tasks := make([]GroupCompactionTask, 0) - tasksPerGroup := c.concurrency / len(groups) for _, g := range groups { // Ignore groups with only one block because there is nothing to compact. if len(g.IDs()) == 1 { From 407e98a84487cf8d3357baa8768ebecc804efd6f Mon Sep 17 00:00:00 2001 From: Filip Petkovski Date: Wed, 7 Dec 2022 13:01:08 +0100 Subject: [PATCH 19/22] Distribute tasks in round-robin manner Signed-off-by: Filip Petkovski --- pkg/compact/compact.go | 63 ++++++++++++++++++++++++++++-------------- 1 file changed, 42 insertions(+), 21 deletions(-) diff --git a/pkg/compact/compact.go b/pkg/compact/compact.go index 9aeb297dad..708c165d1c 100644 --- a/pkg/compact/compact.go +++ b/pkg/compact/compact.go @@ -994,7 +994,7 @@ func RepairIssue347(ctx context.Context, logger log.Logger, bkt objstore.Bucket, return nil } -func (cg *Group) PlanCompactionTasks(ctx context.Context, dir string, planner Planner, maxTasks int) ([]GroupCompactionTask, error) { +func (cg *Group) PlanCompactionTasks(ctx context.Context, dir string, planner Planner) ([]GroupCompactionTask, error) { plannedCompactions, err := planner.Plan(ctx, cg.metasByMinTime) if err != nil { return nil, err @@ -1009,10 +1009,6 @@ func (cg *Group) PlanCompactionTasks(ctx context.Context, dir string, planner Pl }) } - if len(tasks) > maxTasks { - tasks = tasks[:maxTasks] - } - return tasks, nil } @@ -1328,22 +1324,9 @@ func (c *BucketCompactor) Compact(ctx context.Context) (rerr error) { level.Info(c.logger).Log("msg", "start of compactions") - tasksPerGroup := 1 - if len(groups) > 0 && c.concurrency > len(groups) { - tasksPerGroup = c.concurrency / len(groups) - } - tasks := make([]GroupCompactionTask, 0) - for _, g := range groups { - // Ignore groups with only one block because there is nothing to compact. - if len(g.IDs()) == 1 { - continue - } - - groupTasks, err := g.PlanCompactionTasks(ctx, c.compactDir, c.planner, tasksPerGroup) - if err != nil { - return errors.Wrapf(err, "get compaction group tasks: %s", g.Key()) - } - tasks = append(tasks, groupTasks...) + tasks, err := c.planTasks(ctx, groups) + if err != nil { + return err } // Send all tasks planned in this pass to the compaction workers. @@ -1380,6 +1363,44 @@ func (c *BucketCompactor) Compact(ctx context.Context) (rerr error) { return nil } +func (c *BucketCompactor) planTasks(ctx context.Context, groups []*Group) ([]GroupCompactionTask, error) { + // Plan tasks from all groups + allTasks := make([][]GroupCompactionTask, 0, len(groups)) + numTasks := 0 + for _, g := range groups { + // Ignore groups with only one block because there is nothing to compact. + if len(g.IDs()) == 1 { + continue + } + + groupTasks, err := g.PlanCompactionTasks(ctx, c.compactDir, c.planner) + if err != nil { + return nil, errors.Wrapf(err, "get compaction group tasks: %s", g.Key()) + } + allTasks = append(allTasks, groupTasks) + numTasks += len(groupTasks) + } + + tasksLimit := c.concurrency + if numTasks < tasksLimit { + tasksLimit = numTasks + } + + // Distribute tasks from all groups in a round-robin manner until we + // reach the concurrency limit. + tasks := make([]GroupCompactionTask, 0) + for len(tasks) < tasksLimit { + for i, groupTasks := range allTasks { + if len(groupTasks) == 0 { + continue + } + tasks = append(tasks, groupTasks[0]) + allTasks[i] = allTasks[i][1:] + } + } + return tasks, nil +} + var _ block.MetadataFilter = &GatherNoCompactionMarkFilter{} // GatherNoCompactionMarkFilter is a block.Fetcher filter that passes all metas. While doing it, it gathers all no-compact-mark.json markers. From 25705dc12302b89641b2e0c9226bdb50bc978cc1 Mon Sep 17 00:00:00 2001 From: Filip Petkovski Date: Wed, 7 Dec 2022 13:10:54 +0100 Subject: [PATCH 20/22] Add e2e test for parallel compaction Signed-off-by: Filip Petkovski --- pkg/compact/compact_e2e_test.go | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/pkg/compact/compact_e2e_test.go b/pkg/compact/compact_e2e_test.go index 9709b23ea3..830eee8fd8 100644 --- a/pkg/compact/compact_e2e_test.go +++ b/pkg/compact/compact_e2e_test.go @@ -171,15 +171,19 @@ func MetricCount(c prometheus.Collector) int { } func TestGroupCompactE2E(t *testing.T) { - testGroupCompactE2e(t, nil) + testGroupCompactE2e(t, nil, 2) +} + +func TestGroupCompactE2EWithConcurrency(t *testing.T) { + testGroupCompactE2e(t, nil, 20) } // Penalty based merger should get the same result as the blocks don't have overlap. func TestGroupCompactPenaltyDedupE2E(t *testing.T) { - testGroupCompactE2e(t, dedup.NewChunkSeriesMerger()) + testGroupCompactE2e(t, dedup.NewChunkSeriesMerger(), 2) } -func testGroupCompactE2e(t *testing.T, mergeFunc storage.VerticalChunkSeriesMergeFunc) { +func testGroupCompactE2e(t *testing.T, mergeFunc storage.VerticalChunkSeriesMergeFunc, concurrency int) { objtesting.ForeachStore(t, func(t *testing.T, bkt objstore.Bucket) { ctx, cancel := context.WithTimeout(context.Background(), 120*time.Second) defer cancel() @@ -212,7 +216,7 @@ func testGroupCompactE2e(t *testing.T, mergeFunc storage.VerticalChunkSeriesMerg planner := NewPlanner(logger, []int64{1000, 3000}, noCompactMarkerFilter) grouper := NewDefaultGrouper(logger, bkt, false, false, reg, blocksMarkedForDeletion, garbageCollectedBlocks, blocksMaredForNoCompact, metadata.NoneFunc, 10, 10) - bComp, err := NewBucketCompactor(logger, sy, grouper, planner, comp, dir, bkt, 2, true) + bComp, err := NewBucketCompactor(logger, sy, grouper, planner, comp, dir, bkt, concurrency, true) testutil.Ok(t, err) // Compaction on empty should not fail. From e3e94c6f560576656125b9e09439327477668071 Mon Sep 17 00:00:00 2001 From: Filip Petkovski Date: Wed, 7 Dec 2022 13:16:00 +0100 Subject: [PATCH 21/22] Plan at least one task from each group Signed-off-by: Filip Petkovski --- pkg/compact/compact.go | 6 ++++++ pkg/compact/compact_e2e_test.go | 6 +++++- 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/pkg/compact/compact.go b/pkg/compact/compact.go index 708c165d1c..6d16fd01e2 100644 --- a/pkg/compact/compact.go +++ b/pkg/compact/compact.go @@ -1382,6 +1382,12 @@ func (c *BucketCompactor) planTasks(ctx context.Context, groups []*Group) ([]Gro } tasksLimit := c.concurrency + // Make sure we plan at least one task from each group. + if tasksLimit < len(allTasks) { + tasksLimit = len(groups) + } + + // If there aren't enough tasks across all groups, plan all available tasks. if numTasks < tasksLimit { tasksLimit = numTasks } diff --git a/pkg/compact/compact_e2e_test.go b/pkg/compact/compact_e2e_test.go index 830eee8fd8..c3ea353f63 100644 --- a/pkg/compact/compact_e2e_test.go +++ b/pkg/compact/compact_e2e_test.go @@ -174,7 +174,11 @@ func TestGroupCompactE2E(t *testing.T) { testGroupCompactE2e(t, nil, 2) } -func TestGroupCompactE2EWithConcurrency(t *testing.T) { +func TestGroupCompactE2EWithoutConcurrency(t *testing.T) { + testGroupCompactE2e(t, nil, 1) +} + +func TestGroupCompactE2EWithHighConcurrency(t *testing.T) { testGroupCompactE2e(t, nil, 20) } From ea766ad556d40d8db082b856b2473885c7015c63 Mon Sep 17 00:00:00 2001 From: Filip Petkovski Date: Mon, 12 Dec 2022 09:21:58 +0100 Subject: [PATCH 22/22] Clean up scheduling Signed-off-by: Filip Petkovski --- pkg/compact/compact.go | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/pkg/compact/compact.go b/pkg/compact/compact.go index 6d16fd01e2..a0f0dd0047 100644 --- a/pkg/compact/compact.go +++ b/pkg/compact/compact.go @@ -1365,7 +1365,7 @@ func (c *BucketCompactor) Compact(ctx context.Context) (rerr error) { func (c *BucketCompactor) planTasks(ctx context.Context, groups []*Group) ([]GroupCompactionTask, error) { // Plan tasks from all groups - allTasks := make([][]GroupCompactionTask, 0, len(groups)) + allGroupTasks := make([][]GroupCompactionTask, 0, len(groups)) numTasks := 0 for _, g := range groups { // Ignore groups with only one block because there is nothing to compact. @@ -1377,14 +1377,16 @@ func (c *BucketCompactor) planTasks(ctx context.Context, groups []*Group) ([]Gro if err != nil { return nil, errors.Wrapf(err, "get compaction group tasks: %s", g.Key()) } - allTasks = append(allTasks, groupTasks) - numTasks += len(groupTasks) + if len(groupTasks) > 0 { + allGroupTasks = append(allGroupTasks, groupTasks) + numTasks += len(groupTasks) + } } tasksLimit := c.concurrency // Make sure we plan at least one task from each group. - if tasksLimit < len(allTasks) { - tasksLimit = len(groups) + if tasksLimit < len(allGroupTasks) { + tasksLimit = len(allGroupTasks) } // If there aren't enough tasks across all groups, plan all available tasks. @@ -1396,12 +1398,12 @@ func (c *BucketCompactor) planTasks(ctx context.Context, groups []*Group) ([]Gro // reach the concurrency limit. tasks := make([]GroupCompactionTask, 0) for len(tasks) < tasksLimit { - for i, groupTasks := range allTasks { + for i, groupTasks := range allGroupTasks { if len(groupTasks) == 0 { continue } tasks = append(tasks, groupTasks[0]) - allTasks[i] = allTasks[i][1:] + allGroupTasks[i] = allGroupTasks[i][1:] } } return tasks, nil