From 602010285525e2a9898c8a914d5cb75199846684 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Wed, 6 Sep 2023 10:26:45 +0200 Subject: [PATCH 01/74] storage: Refactor block compaction to allow shard-splitting --- pkg/phlaredb/compact.go | 446 ++++++++++++++++++++-------------- pkg/phlaredb/compact_test.go | 192 +++++++-------- pkg/phlaredb/profile_store.go | 14 +- 3 files changed, 361 insertions(+), 291 deletions(-) diff --git a/pkg/phlaredb/compact.go b/pkg/phlaredb/compact.go index f87b2f2cd5..263b6b1482 100644 --- a/pkg/phlaredb/compact.go +++ b/pkg/phlaredb/compact.go @@ -2,6 +2,8 @@ package phlaredb import ( "context" + "crypto/rand" + "fmt" "io/fs" "math" "os" @@ -9,13 +11,14 @@ import ( "sort" "github.com/oklog/ulid" - "github.com/opentracing/opentracing-go" "github.com/parquet-go/parquet-go" "github.com/pkg/errors" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/tsdb" + "github.com/grafana/dskit/multierror" + "github.com/grafana/dskit/runutil" "github.com/grafana/pyroscope/pkg/iter" phlaremodel "github.com/grafana/pyroscope/pkg/model" phlareparquet "github.com/grafana/pyroscope/pkg/parquet" @@ -35,100 +38,275 @@ type BlockReader interface { } func Compact(ctx context.Context, src []BlockReader, dst string) (meta block.Meta, err error) { - srcMetas := make([]block.Meta, len(src)) - ulids := make([]string, len(src)) + metas, err := CompactWithShard(ctx, src, 1, dst) + if err != nil { + return block.Meta{}, err + } + return metas[0], nil +} +func CompactWithShard(ctx context.Context, src []BlockReader, shardsCount uint64, dst string) ( + []block.Meta, error, +) { + if shardsCount == 0 { + shardsCount = 1 + } + if len(src) <= 1 && shardsCount == 1 { + return nil, errors.New("not enough blocks to compact") + } + var ( + writers = make([]*blockWriter, shardsCount) + shardBy = shardByFingerprint + srcMetas = make([]block.Meta, len(src)) + err error + ) for i, b := range src { srcMetas[i] = b.Meta() - ulids[i] = b.Meta().ULID.String() + // ulids[i] = b.Meta().ULID.String() } - meta = compactMetas(srcMetas...) - blockPath := filepath.Join(dst, meta.ULID.String()) - indexPath := filepath.Join(blockPath, block.IndexFilename) - profilePath := filepath.Join(blockPath, (&schemav1.ProfilePersister{}).Name()+block.ParquetSuffix) - - sp, ctx := opentracing.StartSpanFromContext(ctx, "Compact") - defer func() { - // todo: context propagation is not working through objstore - // This is because the BlockReader has no context. - sp.SetTag("src", ulids) - sp.SetTag("block_id", meta.ULID.String()) + + outBlocksTime := ulid.Now() + outMeta := compactMetas(srcMetas...) + + // create the shards writers + for i := range writers { + meta := outMeta.Clone() + meta.ULID = ulid.MustNew(outBlocksTime, rand.Reader) + writers[i], err = newBlockWriter(dst, meta) if err != nil { - sp.SetTag("error", err) + return nil, fmt.Errorf("create block writer: %w", err) } - sp.Finish() - }() + } - if len(src) <= 1 { - return block.Meta{}, errors.New("not enough blocks to compact") + rowsIt, err := newMergeRowProfileIterator(src) + if err != nil { + return nil, err + } + defer runutil.CloseWithLogOnErr(util.Logger, rowsIt, "close rows iterator") + + // iterate and splits the rows into series. + for rowsIt.Next() { + r := rowsIt.At() + shard := int(shardBy(r, shardsCount)) + if err := writers[shard].WriteRow(r); err != nil { + return nil, err + } } + if err := rowsIt.Err(); err != nil { + return nil, err + } + + // Close all blocks + errs := multierror.New() + for _, w := range writers { + if err := w.Close(ctx); err != nil { + errs.Add(err) + } + } + + out := make([]block.Meta, 0, len(writers)) + for _, w := range writers { + if w.meta.Stats.NumSamples > 0 { + out = append(out, *w.meta) + } + } + + // Returns all Metas + return out, errs.Err() +} + +var shardByFingerprint = func(r profileRow, shardsCount uint64) uint64 { + return uint64(r.fp) % shardsCount +} + +type blockWriter struct { + indexRewriter *indexRewriter + symbolsRewriter *symbolsRewriter + profilesWriter *profilesWriter + path string + meta *block.Meta + totalProfiles uint64 +} + +func newBlockWriter(dst string, meta *block.Meta) (*blockWriter, error) { + blockPath := filepath.Join(dst, meta.ULID.String()) + if err := os.MkdirAll(blockPath, 0o777); err != nil { - return block.Meta{}, err + return nil, err } - indexw, err := prepareIndexWriter(ctx, indexPath, src) + profileWriter, err := newProfileWriter(blockPath) if err != nil { - return block.Meta{}, err + return nil, err } - profileFile, err := os.OpenFile(profilePath, os.O_RDWR|os.O_CREATE|os.O_EXCL, 0o644) + return &blockWriter{ + indexRewriter: newIndexRewriter(blockPath), + symbolsRewriter: newSymbolsRewriter(blockPath), + profilesWriter: profileWriter, + path: blockPath, + meta: meta, + }, nil +} + +func (bw *blockWriter) WriteRow(r profileRow) (err error) { + r, err = bw.indexRewriter.ReWriteRow(r) if err != nil { - return block.Meta{}, err + return err + } + r, err = bw.symbolsRewriter.ReWriteRow(r) + if err != nil { + return err } - profileWriter := newProfileWriter(profileFile) - symw := symdb.NewSymDB(symdb.DefaultConfig(). - WithDirectory(filepath.Join(blockPath, symdb.DefaultDirName)). - WithParquetConfig(symdb.ParquetConfig{ - MaxBufferRowCount: defaultParquetConfig.MaxBufferRowCount, - })) + if err := bw.profilesWriter.WriteRow(r); err != nil { + return err + } + bw.totalProfiles++ + return nil +} +func (bw *blockWriter) Close(ctx context.Context) error { + if err := bw.indexRewriter.Close(ctx); err != nil { + return err + } + if err := bw.symbolsRewriter.Close(); err != nil { + return err + } + if err := bw.profilesWriter.Close(); err != nil { + return err + } + metaFiles, err := metaFilesFromDir(bw.path) if err != nil { - return block.Meta{}, err + return err + } + bw.meta.Files = metaFiles + bw.meta.Stats.NumProfiles = bw.totalProfiles + bw.meta.Stats.NumSeries = bw.indexRewriter.NumSeries() + bw.meta.Stats.NumSamples = bw.symbolsRewriter.NumSamples() + bw.meta.Compaction.Deletable = bw.totalProfiles == 0 + if _, err := bw.meta.WriteToFile(util.Logger, bw.path); err != nil { + return err } + return nil +} - rowsIt, err := newMergeRowProfileIterator(src) +type profilesWriter struct { + *parquet.GenericWriter[*schemav1.Profile] + file *os.File + + buf []parquet.Row +} + +func newProfileWriter(path string) (*profilesWriter, error) { + profilePath := filepath.Join(path, (&schemav1.ProfilePersister{}).Name()+block.ParquetSuffix) + profileFile, err := os.OpenFile(profilePath, os.O_RDWR|os.O_CREATE|os.O_EXCL, 0o644) if err != nil { - return block.Meta{}, err + return nil, err } - seriesRewriter := newSeriesRewriter(rowsIt, indexw) - symRewriter := newSymbolsRewriter(seriesRewriter, src, symw) - reader := phlareparquet.NewIteratorRowReader(newRowsIterator(symRewriter)) + return &profilesWriter{ + GenericWriter: newParquetProfileWriter(profileFile, parquet.MaxRowsPerRowGroup(int64(defaultParquetConfig.MaxBufferRowCount))), + file: profileFile, + buf: make([]parquet.Row, 1), + }, nil +} - total, _, err := phlareparquet.CopyAsRowGroups(profileWriter, reader, defaultParquetConfig.MaxBufferRowCount) +func (p *profilesWriter) WriteRow(r profileRow) error { + p.buf[0] = parquet.Row(r.row) + _, err := p.GenericWriter.WriteRows(p.buf) if err != nil { - return block.Meta{}, err + return err } + return nil +} - if err = symRewriter.Close(); err != nil { - return block.Meta{}, err +func (p *profilesWriter) Close() error { + err := p.GenericWriter.Close() + if err != nil { + return err } + return p.file.Close() +} - // flush the index file. - if err = indexw.Close(); err != nil { - return block.Meta{}, err +func newIndexRewriter(path string) *indexRewriter { + return &indexRewriter{ + symbols: make(map[string]struct{}), + path: path, } +} - if err = profileWriter.Close(); err != nil { - return block.Meta{}, err +type indexRewriter struct { + series []struct { + labels phlaremodel.Labels + fp model.Fingerprint } - if err = symw.Flush(); err != nil { - return block.Meta{}, err + symbols map[string]struct{} + chunks []index.ChunkMeta // one chunk per series + + previousFp model.Fingerprint + + path string +} + +func (idxRw *indexRewriter) ReWriteRow(r profileRow) (profileRow, error) { + if idxRw.previousFp != r.fp || len(idxRw.series) == 0 { + series := r.labels.Clone() + for _, l := range series { + idxRw.symbols[l.Name] = struct{}{} + idxRw.symbols[l.Value] = struct{}{} + } + idxRw.series = append(idxRw.series, struct { + labels phlaremodel.Labels + fp model.Fingerprint + }{ + labels: series, + fp: r.fp, + }) + idxRw.chunks = append(idxRw.chunks, index.ChunkMeta{ + MinTime: r.timeNanos, + MaxTime: r.timeNanos, + SeriesIndex: uint32(len(idxRw.series) - 1), + }) + idxRw.previousFp = r.fp } + idxRw.chunks[len(idxRw.chunks)-1].MaxTime = r.timeNanos + r.row.SetSeriesIndex(idxRw.chunks[len(idxRw.chunks)-1].SeriesIndex) + return r, nil +} + +func (idxRw *indexRewriter) NumSeries() uint64 { + return uint64(len(idxRw.series)) +} - metaFiles, err := metaFilesFromDir(blockPath) +// Close writes the index to given folder. +func (idxRw *indexRewriter) Close(ctx context.Context) error { + indexw, err := index.NewWriter(ctx, filepath.Join(idxRw.path, block.IndexFilename)) if err != nil { - return block.Meta{}, err + return err } - meta.Files = metaFiles - meta.Stats.NumProfiles = total - meta.Stats.NumSeries = seriesRewriter.NumSeries() - meta.Stats.NumSamples = symRewriter.NumSamples() - meta.Compaction.Deletable = meta.Stats.NumSamples == 0 - if _, err := meta.WriteToFile(util.Logger, blockPath); err != nil { - return block.Meta{}, err + + // Sort symbols + symbols := make([]string, 0, len(idxRw.symbols)) + for s := range idxRw.symbols { + symbols = append(symbols, s) } - return meta, nil + sort.Strings(symbols) + + // Add symbols + for _, symbol := range symbols { + if err := indexw.AddSymbol(symbol); err != nil { + return err + } + } + + // Add Series + for i, series := range idxRw.series { + if err := indexw.AddSeries(storage.SeriesRef(i), series.labels, series.fp, idxRw.chunks[i]); err != nil { + return err + } + } + + return indexw.Close() } // metaFilesFromDir returns a list of block files description from a directory. @@ -370,82 +548,6 @@ func newMergeRowProfileIterator(src []BlockReader) (iter.Iterator[profileRow], e }, nil } -type seriesRewriter struct { - iter.Iterator[profileRow] - - indexw *index.Writer - - seriesRef storage.SeriesRef - labels phlaremodel.Labels - previousFp model.Fingerprint - currentChunkMeta index.ChunkMeta - err error - - numSeries uint64 - done bool -} - -func newSeriesRewriter(it iter.Iterator[profileRow], indexw *index.Writer) *seriesRewriter { - return &seriesRewriter{ - Iterator: it, - indexw: indexw, - } -} - -func (s *seriesRewriter) NumSeries() uint64 { - return s.numSeries -} - -func (s *seriesRewriter) Next() bool { - if !s.Iterator.Next() { - if s.done { - return false - } - s.done = true - if s.previousFp != 0 { - s.currentChunkMeta.SeriesIndex = uint32(s.seriesRef) - 1 - if err := s.indexw.AddSeries(s.seriesRef-1, s.labels, s.previousFp, s.currentChunkMeta); err != nil { - s.err = err - return false - } - s.numSeries++ - } - return false - } - currentProfile := s.Iterator.At() - if s.previousFp != currentProfile.fp { - if s.previousFp != 0 { - s.currentChunkMeta.SeriesIndex = uint32(s.seriesRef) - 1 - if err := s.indexw.AddSeries(s.seriesRef-1, s.labels, s.previousFp, s.currentChunkMeta); err != nil { - s.err = err - return false - } - s.numSeries++ - } - s.seriesRef++ - s.labels = currentProfile.labels.Clone() - s.previousFp = currentProfile.fp - s.currentChunkMeta.MinTime = currentProfile.timeNanos - } - s.currentChunkMeta.MaxTime = currentProfile.timeNanos - currentProfile.row.SetSeriesIndex(uint32(s.seriesRef - 1)) - return true -} - -type rowsIterator struct { - iter.Iterator[profileRow] -} - -func newRowsIterator(it iter.Iterator[profileRow]) *rowsIterator { - return &rowsIterator{ - Iterator: it, - } -} - -func (r *rowsIterator) At() parquet.Row { - return parquet.Row(r.Iterator.At().row) -} - type dedupeProfileRowIterator struct { iter.Iterator[profileRow] @@ -469,73 +571,36 @@ func (it *dedupeProfileRowIterator) Next() bool { } } -func prepareIndexWriter(ctx context.Context, path string, readers []BlockReader) (*index.Writer, error) { - var symbols index.StringIter - indexw, err := index.NewWriter(ctx, path) - if err != nil { - return nil, err - } - for i, r := range readers { - if i == 0 { - symbols = r.Index().Symbols() - } - symbols = tsdb.NewMergedStringIter(symbols, r.Index().Symbols()) - } - - for symbols.Next() { - if err := indexw.AddSymbol(symbols.At()); err != nil { - return nil, errors.Wrap(err, "add symbol") - } - } - if symbols.Err() != nil { - return nil, errors.Wrap(symbols.Err(), "next symbol") - } - - return indexw, nil -} - type symbolsRewriter struct { - profiles iter.Iterator[profileRow] rewriters map[BlockReader]*symdb.Rewriter + w *symdb.SymDB stacktraces []uint32 - err error numSamples uint64 } -func newSymbolsRewriter(it iter.Iterator[profileRow], blocks []BlockReader, w *symdb.SymDB) *symbolsRewriter { - sr := symbolsRewriter{ - profiles: it, - rewriters: make(map[BlockReader]*symdb.Rewriter, len(blocks)), +func newSymbolsRewriter(path string) *symbolsRewriter { + return &symbolsRewriter{ + w: symdb.NewSymDB(symdb.DefaultConfig(). + WithDirectory(filepath.Join(path, symdb.DefaultDirName)). + WithParquetConfig(symdb.ParquetConfig{ + MaxBufferRowCount: defaultParquetConfig.MaxBufferRowCount, + })), + rewriters: make(map[BlockReader]*symdb.Rewriter), } - for _, r := range blocks { - sr.rewriters[r] = symdb.NewRewriter(w, r.Symbols()) - } - return &sr } func (s *symbolsRewriter) NumSamples() uint64 { return s.numSamples } -func (s *symbolsRewriter) At() profileRow { return s.profiles.At() } - -func (s *symbolsRewriter) Close() error { return s.profiles.Close() } - -func (s *symbolsRewriter) Err() error { - if s.err != nil { - return s.err - } - return s.profiles.Err() -} - -func (s *symbolsRewriter) Next() bool { - if !s.profiles.Next() { - return false - } +func (s *symbolsRewriter) ReWriteRow(profile profileRow) (profileRow, error) { var err error - profile := s.profiles.At() profile.row.ForStacktraceIDsValues(func(values []parquet.Value) { s.loadStacktracesID(values) - r := s.rewriters[profile.blockReader] + r, ok := s.rewriters[profile.blockReader] + if !ok { + r = symdb.NewRewriter(s.w, profile.blockReader.Symbols()) + s.rewriters[profile.blockReader] = r + } if err = r.Rewrite(profile.row.StacktracePartitionID(), s.stacktraces); err != nil { return } @@ -546,10 +611,13 @@ func (s *symbolsRewriter) Next() bool { } }) if err != nil { - s.err = err - return false + return profile, err } - return true + return profile, nil +} + +func (s *symbolsRewriter) Close() error { + return s.w.Flush() } func (s *symbolsRewriter) loadStacktracesID(values []parquet.Value) { diff --git a/pkg/phlaredb/compact_test.go b/pkg/phlaredb/compact_test.go index f7fe3a5685..3cc134cf55 100644 --- a/pkg/phlaredb/compact_test.go +++ b/pkg/phlaredb/compact_test.go @@ -239,102 +239,102 @@ func TestMergeRowProfileIterator(t *testing.T) { } } -func TestSeriesRewriter(t *testing.T) { - type profile struct { - timeNanos int64 - labels phlaremodel.Labels - } - - in := []profile{ - {1, phlaremodel.LabelsFromStrings("job", "a")}, - {2, phlaremodel.LabelsFromStrings("job", "a")}, - {3, phlaremodel.LabelsFromStrings("job", "a")}, - {2, phlaremodel.LabelsFromStrings("job", "b")}, - {1, phlaremodel.LabelsFromStrings("job", "c")}, - {2, phlaremodel.LabelsFromStrings("job", "c")}, - } - - blk := newBlock(t, func() []*testhelper.ProfileBuilder { - var builders []*testhelper.ProfileBuilder - for _, p := range in { - prof := testhelper.NewProfileBuilder(p.timeNanos). - CPUProfile().ForStacktraceString("foo").AddSamples(1) - for _, l := range p.labels { - prof.WithLabels(l.Name, l.Value) - } - builders = append(builders, prof) - } - return builders - }) - rows, err := newProfileRowIterator(blk) - require.NoError(t, err) - filePath := filepath.Join(t.TempDir(), block.IndexFilename) - idxw, err := prepareIndexWriter(context.Background(), filePath, []BlockReader{blk}) - require.NoError(t, err) - it := newSeriesRewriter(rows, idxw) - // tests that all rows are written to the correct series index - require.True(t, it.Next()) - require.Equal(t, uint32(0), it.At().row.SeriesIndex()) - require.True(t, it.Next()) - require.Equal(t, uint32(0), it.At().row.SeriesIndex()) - require.True(t, it.Next()) - require.Equal(t, uint32(0), it.At().row.SeriesIndex()) - require.True(t, it.Next()) - require.Equal(t, uint32(1), it.At().row.SeriesIndex()) - require.True(t, it.Next()) - require.Equal(t, uint32(2), it.At().row.SeriesIndex()) - require.True(t, it.Next()) - require.Equal(t, uint32(2), it.At().row.SeriesIndex()) - require.False(t, it.Next()) - - require.NoError(t, it.Err()) - require.NoError(t, it.Close()) - require.NoError(t, idxw.Close()) - - idxr, err := index.NewFileReader(filePath) - require.NoError(t, err) - defer idxr.Close() - - k, v := index.AllPostingsKey() - p, err := idxr.Postings(k, nil, v) - require.NoError(t, err) - - chunks := make([]index.ChunkMeta, 1) - var lbs phlaremodel.Labels - - require.True(t, p.Next()) - fp, err := idxr.Series(p.At(), &lbs, &chunks) - require.NoError(t, err) - require.Equal(t, model.Fingerprint(lbs.Hash()), model.Fingerprint(fp)) - require.Equal(t, lbs.WithoutPrivateLabels(), phlaremodel.LabelsFromStrings("job", "a")) - require.Equal(t, []index.ChunkMeta{{ - SeriesIndex: 0, - MinTime: int64(1), - MaxTime: int64(3), - }}, chunks) - - require.True(t, p.Next()) - fp, err = idxr.Series(p.At(), &lbs, &chunks) - require.NoError(t, err) - require.Equal(t, model.Fingerprint(lbs.Hash()), model.Fingerprint(fp)) - require.Equal(t, lbs.WithoutPrivateLabels(), phlaremodel.LabelsFromStrings("job", "b")) - require.Equal(t, []index.ChunkMeta{{ - SeriesIndex: 1, - MinTime: int64(2), - MaxTime: int64(2), - }}, chunks) - - require.True(t, p.Next()) - fp, err = idxr.Series(p.At(), &lbs, &chunks) - require.NoError(t, err) - require.Equal(t, model.Fingerprint(lbs.Hash()), model.Fingerprint(fp)) - require.Equal(t, lbs.WithoutPrivateLabels(), phlaremodel.LabelsFromStrings("job", "c")) - require.Equal(t, []index.ChunkMeta{{ - SeriesIndex: 2, - MinTime: int64(1), - MaxTime: int64(2), - }}, chunks) -} +// func TestSeriesRewriter(t *testing.T) { +// type profile struct { +// timeNanos int64 +// labels phlaremodel.Labels +// } + +// in := []profile{ +// {1, phlaremodel.LabelsFromStrings("job", "a")}, +// {2, phlaremodel.LabelsFromStrings("job", "a")}, +// {3, phlaremodel.LabelsFromStrings("job", "a")}, +// {2, phlaremodel.LabelsFromStrings("job", "b")}, +// {1, phlaremodel.LabelsFromStrings("job", "c")}, +// {2, phlaremodel.LabelsFromStrings("job", "c")}, +// } + +// blk := newBlock(t, func() []*testhelper.ProfileBuilder { +// var builders []*testhelper.ProfileBuilder +// for _, p := range in { +// prof := testhelper.NewProfileBuilder(p.timeNanos). +// CPUProfile().ForStacktraceString("foo").AddSamples(1) +// for _, l := range p.labels { +// prof.WithLabels(l.Name, l.Value) +// } +// builders = append(builders, prof) +// } +// return builders +// }) +// rows, err := newProfileRowIterator(blk) +// require.NoError(t, err) +// filePath := filepath.Join(t.TempDir(), block.IndexFilename) +// idxw, err := prepareIndexWriter(context.Background(), filePath, []BlockReader{blk}) +// require.NoError(t, err) +// it := newSeriesRewriter(rows, idxw) +// // tests that all rows are written to the correct series index +// require.True(t, it.Next()) +// require.Equal(t, uint32(0), it.At().row.SeriesIndex()) +// require.True(t, it.Next()) +// require.Equal(t, uint32(0), it.At().row.SeriesIndex()) +// require.True(t, it.Next()) +// require.Equal(t, uint32(0), it.At().row.SeriesIndex()) +// require.True(t, it.Next()) +// require.Equal(t, uint32(1), it.At().row.SeriesIndex()) +// require.True(t, it.Next()) +// require.Equal(t, uint32(2), it.At().row.SeriesIndex()) +// require.True(t, it.Next()) +// require.Equal(t, uint32(2), it.At().row.SeriesIndex()) +// require.False(t, it.Next()) + +// require.NoError(t, it.Err()) +// require.NoError(t, it.Close()) +// require.NoError(t, idxw.Close()) + +// idxr, err := index.NewFileReader(filePath) +// require.NoError(t, err) +// defer idxr.Close() + +// k, v := index.AllPostingsKey() +// p, err := idxr.Postings(k, nil, v) +// require.NoError(t, err) + +// chunks := make([]index.ChunkMeta, 1) +// var lbs phlaremodel.Labels + +// require.True(t, p.Next()) +// fp, err := idxr.Series(p.At(), &lbs, &chunks) +// require.NoError(t, err) +// require.Equal(t, model.Fingerprint(lbs.Hash()), model.Fingerprint(fp)) +// require.Equal(t, lbs.WithoutPrivateLabels(), phlaremodel.LabelsFromStrings("job", "a")) +// require.Equal(t, []index.ChunkMeta{{ +// SeriesIndex: 0, +// MinTime: int64(1), +// MaxTime: int64(3), +// }}, chunks) + +// require.True(t, p.Next()) +// fp, err = idxr.Series(p.At(), &lbs, &chunks) +// require.NoError(t, err) +// require.Equal(t, model.Fingerprint(lbs.Hash()), model.Fingerprint(fp)) +// require.Equal(t, lbs.WithoutPrivateLabels(), phlaremodel.LabelsFromStrings("job", "b")) +// require.Equal(t, []index.ChunkMeta{{ +// SeriesIndex: 1, +// MinTime: int64(2), +// MaxTime: int64(2), +// }}, chunks) + +// require.True(t, p.Next()) +// fp, err = idxr.Series(p.At(), &lbs, &chunks) +// require.NoError(t, err) +// require.Equal(t, model.Fingerprint(lbs.Hash()), model.Fingerprint(fp)) +// require.Equal(t, lbs.WithoutPrivateLabels(), phlaremodel.LabelsFromStrings("job", "c")) +// require.Equal(t, []index.ChunkMeta{{ +// SeriesIndex: 2, +// MinTime: int64(1), +// MaxTime: int64(2), +// }}, chunks) +// } func TestFlushMeta(t *testing.T) { b := newBlock(t, func() []*testhelper.ProfileBuilder { diff --git a/pkg/phlaredb/profile_store.go b/pkg/phlaredb/profile_store.go index 3afe503684..9b445ceec4 100644 --- a/pkg/phlaredb/profile_store.go +++ b/pkg/phlaredb/profile_store.go @@ -62,11 +62,13 @@ type profileStore struct { flushBufferLbs []phlaremodel.Labels } -func newProfileWriter(writer io.Writer) *parquet.GenericWriter[*schemav1.Profile] { - return parquet.NewGenericWriter[*schemav1.Profile](writer, schemav1.ProfilesSchema, - parquet.ColumnPageBuffers(parquet.NewFileBufferPool(os.TempDir(), "pyroscopedb-parquet-buffers*")), - parquet.CreatedBy("github.com/grafana/pyroscope/", build.Version, build.Revision), - parquet.PageBufferSize(3*1024*1024), +func newParquetProfileWriter(writer io.Writer, options ...parquet.WriterOption) *parquet.GenericWriter[*schemav1.Profile] { + options = append(options, parquet.PageBufferSize(3*1024*1024)) + options = append(options, parquet.CreatedBy("github.com/grafana/pyroscope/", build.Version, build.Revision)) + options = append(options, parquet.ColumnPageBuffers(parquet.NewFileBufferPool(os.TempDir(), "pyroscopedb-parquet-buffers*"))) + options = append(options, schemav1.ProfilesSchema) + return parquet.NewGenericWriter[*schemav1.Profile]( + writer, options..., ) } @@ -82,7 +84,7 @@ func newProfileStore(phlarectx context.Context) *profileStore { go s.cutRowGroupLoop() // Initialize writer on /dev/null // TODO: Reuse parquet.Writer beyond life time of the head. - s.writer = newProfileWriter(io.Discard) + s.writer = newParquetProfileWriter(io.Discard) return s } From 63d1beb5ee2e2029be5a5e4af15c48111238d75f Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Wed, 6 Sep 2023 10:47:50 +0200 Subject: [PATCH 02/74] Rename to CompactWithSplitting for consistency --- pkg/phlaredb/compact.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pkg/phlaredb/compact.go b/pkg/phlaredb/compact.go index 263b6b1482..34b3468a28 100644 --- a/pkg/phlaredb/compact.go +++ b/pkg/phlaredb/compact.go @@ -38,14 +38,14 @@ type BlockReader interface { } func Compact(ctx context.Context, src []BlockReader, dst string) (meta block.Meta, err error) { - metas, err := CompactWithShard(ctx, src, 1, dst) + metas, err := CompactWithSplitting(ctx, src, 1, dst) if err != nil { return block.Meta{}, err } return metas[0], nil } -func CompactWithShard(ctx context.Context, src []BlockReader, shardsCount uint64, dst string) ( +func CompactWithSplitting(ctx context.Context, src []BlockReader, shardsCount uint64, dst string) ( []block.Meta, error, ) { if shardsCount == 0 { From 8129c0667cc80a850069148501c15e30dbf64d18 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Wed, 6 Sep 2023 11:29:07 +0200 Subject: [PATCH 03/74] Introduce back compaction series testing. --- pkg/phlaredb/compact.go | 18 ++-- pkg/phlaredb/compact_test.go | 186 +++++++++++++++++------------------ 2 files changed, 99 insertions(+), 105 deletions(-) diff --git a/pkg/phlaredb/compact.go b/pkg/phlaredb/compact.go index 34b3468a28..3a045ffd93 100644 --- a/pkg/phlaredb/compact.go +++ b/pkg/phlaredb/compact.go @@ -19,6 +19,7 @@ import ( "github.com/grafana/dskit/multierror" "github.com/grafana/dskit/runutil" + "github.com/grafana/pyroscope/pkg/iter" phlaremodel "github.com/grafana/pyroscope/pkg/model" phlareparquet "github.com/grafana/pyroscope/pkg/parquet" @@ -62,7 +63,6 @@ func CompactWithSplitting(ctx context.Context, src []BlockReader, shardsCount ui ) for i, b := range src { srcMetas[i] = b.Meta() - // ulids[i] = b.Meta().ULID.String() } outBlocksTime := ulid.Now() @@ -149,12 +149,12 @@ func newBlockWriter(dst string, meta *block.Meta) (*blockWriter, error) { }, nil } -func (bw *blockWriter) WriteRow(r profileRow) (err error) { - r, err = bw.indexRewriter.ReWriteRow(r) +func (bw *blockWriter) WriteRow(r profileRow) error { + err := bw.indexRewriter.ReWriteRow(r) if err != nil { return err } - r, err = bw.symbolsRewriter.ReWriteRow(r) + err = bw.symbolsRewriter.ReWriteRow(r) if err != nil { return err } @@ -248,7 +248,7 @@ type indexRewriter struct { path string } -func (idxRw *indexRewriter) ReWriteRow(r profileRow) (profileRow, error) { +func (idxRw *indexRewriter) ReWriteRow(r profileRow) error { if idxRw.previousFp != r.fp || len(idxRw.series) == 0 { series := r.labels.Clone() for _, l := range series { @@ -271,7 +271,7 @@ func (idxRw *indexRewriter) ReWriteRow(r profileRow) (profileRow, error) { } idxRw.chunks[len(idxRw.chunks)-1].MaxTime = r.timeNanos r.row.SetSeriesIndex(idxRw.chunks[len(idxRw.chunks)-1].SeriesIndex) - return r, nil + return nil } func (idxRw *indexRewriter) NumSeries() uint64 { @@ -592,7 +592,7 @@ func newSymbolsRewriter(path string) *symbolsRewriter { func (s *symbolsRewriter) NumSamples() uint64 { return s.numSamples } -func (s *symbolsRewriter) ReWriteRow(profile profileRow) (profileRow, error) { +func (s *symbolsRewriter) ReWriteRow(profile profileRow) error { var err error profile.row.ForStacktraceIDsValues(func(values []parquet.Value) { s.loadStacktracesID(values) @@ -611,9 +611,9 @@ func (s *symbolsRewriter) ReWriteRow(profile profileRow) (profileRow, error) { } }) if err != nil { - return profile, err + return err } - return profile, nil + return nil } func (s *symbolsRewriter) Close() error { diff --git a/pkg/phlaredb/compact_test.go b/pkg/phlaredb/compact_test.go index 3cc134cf55..995b722690 100644 --- a/pkg/phlaredb/compact_test.go +++ b/pkg/phlaredb/compact_test.go @@ -239,102 +239,96 @@ func TestMergeRowProfileIterator(t *testing.T) { } } -// func TestSeriesRewriter(t *testing.T) { -// type profile struct { -// timeNanos int64 -// labels phlaremodel.Labels -// } - -// in := []profile{ -// {1, phlaremodel.LabelsFromStrings("job", "a")}, -// {2, phlaremodel.LabelsFromStrings("job", "a")}, -// {3, phlaremodel.LabelsFromStrings("job", "a")}, -// {2, phlaremodel.LabelsFromStrings("job", "b")}, -// {1, phlaremodel.LabelsFromStrings("job", "c")}, -// {2, phlaremodel.LabelsFromStrings("job", "c")}, -// } - -// blk := newBlock(t, func() []*testhelper.ProfileBuilder { -// var builders []*testhelper.ProfileBuilder -// for _, p := range in { -// prof := testhelper.NewProfileBuilder(p.timeNanos). -// CPUProfile().ForStacktraceString("foo").AddSamples(1) -// for _, l := range p.labels { -// prof.WithLabels(l.Name, l.Value) -// } -// builders = append(builders, prof) -// } -// return builders -// }) -// rows, err := newProfileRowIterator(blk) -// require.NoError(t, err) -// filePath := filepath.Join(t.TempDir(), block.IndexFilename) -// idxw, err := prepareIndexWriter(context.Background(), filePath, []BlockReader{blk}) -// require.NoError(t, err) -// it := newSeriesRewriter(rows, idxw) -// // tests that all rows are written to the correct series index -// require.True(t, it.Next()) -// require.Equal(t, uint32(0), it.At().row.SeriesIndex()) -// require.True(t, it.Next()) -// require.Equal(t, uint32(0), it.At().row.SeriesIndex()) -// require.True(t, it.Next()) -// require.Equal(t, uint32(0), it.At().row.SeriesIndex()) -// require.True(t, it.Next()) -// require.Equal(t, uint32(1), it.At().row.SeriesIndex()) -// require.True(t, it.Next()) -// require.Equal(t, uint32(2), it.At().row.SeriesIndex()) -// require.True(t, it.Next()) -// require.Equal(t, uint32(2), it.At().row.SeriesIndex()) -// require.False(t, it.Next()) - -// require.NoError(t, it.Err()) -// require.NoError(t, it.Close()) -// require.NoError(t, idxw.Close()) - -// idxr, err := index.NewFileReader(filePath) -// require.NoError(t, err) -// defer idxr.Close() - -// k, v := index.AllPostingsKey() -// p, err := idxr.Postings(k, nil, v) -// require.NoError(t, err) - -// chunks := make([]index.ChunkMeta, 1) -// var lbs phlaremodel.Labels - -// require.True(t, p.Next()) -// fp, err := idxr.Series(p.At(), &lbs, &chunks) -// require.NoError(t, err) -// require.Equal(t, model.Fingerprint(lbs.Hash()), model.Fingerprint(fp)) -// require.Equal(t, lbs.WithoutPrivateLabels(), phlaremodel.LabelsFromStrings("job", "a")) -// require.Equal(t, []index.ChunkMeta{{ -// SeriesIndex: 0, -// MinTime: int64(1), -// MaxTime: int64(3), -// }}, chunks) - -// require.True(t, p.Next()) -// fp, err = idxr.Series(p.At(), &lbs, &chunks) -// require.NoError(t, err) -// require.Equal(t, model.Fingerprint(lbs.Hash()), model.Fingerprint(fp)) -// require.Equal(t, lbs.WithoutPrivateLabels(), phlaremodel.LabelsFromStrings("job", "b")) -// require.Equal(t, []index.ChunkMeta{{ -// SeriesIndex: 1, -// MinTime: int64(2), -// MaxTime: int64(2), -// }}, chunks) - -// require.True(t, p.Next()) -// fp, err = idxr.Series(p.At(), &lbs, &chunks) -// require.NoError(t, err) -// require.Equal(t, model.Fingerprint(lbs.Hash()), model.Fingerprint(fp)) -// require.Equal(t, lbs.WithoutPrivateLabels(), phlaremodel.LabelsFromStrings("job", "c")) -// require.Equal(t, []index.ChunkMeta{{ -// SeriesIndex: 2, -// MinTime: int64(1), -// MaxTime: int64(2), -// }}, chunks) -// } +func TestSeriesRewriter(t *testing.T) { + type profile struct { + timeNanos int64 + labels phlaremodel.Labels + } + + in := []profile{ + {1, phlaremodel.LabelsFromStrings("job", "a")}, + {2, phlaremodel.LabelsFromStrings("job", "a")}, + {3, phlaremodel.LabelsFromStrings("job", "a")}, + {2, phlaremodel.LabelsFromStrings("job", "b")}, + {1, phlaremodel.LabelsFromStrings("job", "c")}, + {2, phlaremodel.LabelsFromStrings("job", "c")}, + } + + blk := newBlock(t, func() []*testhelper.ProfileBuilder { + var builders []*testhelper.ProfileBuilder + for _, p := range in { + prof := testhelper.NewProfileBuilder(p.timeNanos). + CPUProfile().ForStacktraceString("foo").AddSamples(1) + for _, l := range p.labels { + prof.WithLabels(l.Name, l.Value) + } + builders = append(builders, prof) + } + return builders + }) + rows, err := newProfileRowIterator(blk) + require.NoError(t, err) + path := t.TempDir() + filePath := filepath.Join(path, block.IndexFilename) + idxw := newIndexRewriter(path) + seriesIdx := []uint32{} + for rows.Next() { + r := rows.At() + require.NoError(t, idxw.ReWriteRow(r)) + seriesIdx = append(seriesIdx, r.row.SeriesIndex()) + } + require.NoError(t, rows.Err()) + require.NoError(t, rows.Close()) + + require.Equal(t, []uint32{0, 0, 0, 1, 2, 2}, seriesIdx) + + err = idxw.Close(context.Background()) + require.NoError(t, err) + + idxr, err := index.NewFileReader(filePath) + require.NoError(t, err) + defer idxr.Close() + + k, v := index.AllPostingsKey() + p, err := idxr.Postings(k, nil, v) + require.NoError(t, err) + + chunks := make([]index.ChunkMeta, 1) + var lbs phlaremodel.Labels + + require.True(t, p.Next()) + fp, err := idxr.Series(p.At(), &lbs, &chunks) + require.NoError(t, err) + require.Equal(t, model.Fingerprint(lbs.Hash()), model.Fingerprint(fp)) + require.Equal(t, lbs.WithoutPrivateLabels(), phlaremodel.LabelsFromStrings("job", "a")) + require.Equal(t, []index.ChunkMeta{{ + SeriesIndex: 0, + MinTime: int64(1), + MaxTime: int64(3), + }}, chunks) + + require.True(t, p.Next()) + fp, err = idxr.Series(p.At(), &lbs, &chunks) + require.NoError(t, err) + require.Equal(t, model.Fingerprint(lbs.Hash()), model.Fingerprint(fp)) + require.Equal(t, lbs.WithoutPrivateLabels(), phlaremodel.LabelsFromStrings("job", "b")) + require.Equal(t, []index.ChunkMeta{{ + SeriesIndex: 1, + MinTime: int64(2), + MaxTime: int64(2), + }}, chunks) + + require.True(t, p.Next()) + fp, err = idxr.Series(p.At(), &lbs, &chunks) + require.NoError(t, err) + require.Equal(t, model.Fingerprint(lbs.Hash()), model.Fingerprint(fp)) + require.Equal(t, lbs.WithoutPrivateLabels(), phlaremodel.LabelsFromStrings("job", "c")) + require.Equal(t, []index.ChunkMeta{{ + SeriesIndex: 2, + MinTime: int64(1), + MaxTime: int64(2), + }}, chunks) +} func TestFlushMeta(t *testing.T) { b := newBlock(t, func() []*testhelper.ProfileBuilder { From cb310f8966827c1719cb5976a6ae8a119bc8d798 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Wed, 6 Sep 2023 14:54:28 +0200 Subject: [PATCH 04/74] Add sharding compaction level --- pkg/phlaredb/compact.go | 4 +- pkg/phlaredb/sharding/label.go | 106 ++++++++++++++ pkg/phlaredb/sharding/label_test.go | 208 ++++++++++++++++++++++++++++ 3 files changed, 317 insertions(+), 1 deletion(-) create mode 100644 pkg/phlaredb/sharding/label.go create mode 100644 pkg/phlaredb/sharding/label_test.go diff --git a/pkg/phlaredb/compact.go b/pkg/phlaredb/compact.go index 3a045ffd93..57df7d06fe 100644 --- a/pkg/phlaredb/compact.go +++ b/pkg/phlaredb/compact.go @@ -25,6 +25,7 @@ import ( phlareparquet "github.com/grafana/pyroscope/pkg/parquet" "github.com/grafana/pyroscope/pkg/phlaredb/block" schemav1 "github.com/grafana/pyroscope/pkg/phlaredb/schemas/v1" + "github.com/grafana/pyroscope/pkg/phlaredb/sharding" "github.com/grafana/pyroscope/pkg/phlaredb/symdb" "github.com/grafana/pyroscope/pkg/phlaredb/tsdb/index" "github.com/grafana/pyroscope/pkg/util" @@ -105,8 +106,9 @@ func CompactWithSplitting(ctx context.Context, src []BlockReader, shardsCount ui } out := make([]block.Meta, 0, len(writers)) - for _, w := range writers { + for shard, w := range writers { if w.meta.Stats.NumSamples > 0 { + w.meta.Labels[sharding.CompactorShardIDLabel] = sharding.FormatShardIDLabelValue(uint64(shard), shardsCount) out = append(out, *w.meta) } } diff --git a/pkg/phlaredb/sharding/label.go b/pkg/phlaredb/sharding/label.go new file mode 100644 index 0000000000..d9db0df453 --- /dev/null +++ b/pkg/phlaredb/sharding/label.go @@ -0,0 +1,106 @@ +// SPDX-License-Identifier: AGPL-3.0-only + +package sharding + +import ( + "fmt" + "strconv" + "strings" + + "github.com/pkg/errors" + "github.com/prometheus/prometheus/model/labels" +) + +const ( + // ShardLabel is a reserved label referencing a shard on read path. + ShardLabel = "__query_shard__" + // CompactorShardIDLabel is the external label used to store + // the ID of a sharded block generated by the split-and-merge compactor. If a block hasn't + // this label, it means the block hasn't been split. + CompactorShardIDLabel = "__compactor_shard_id__" +) + +// ShardSelector holds information about the configured query shard. +type ShardSelector struct { + ShardIndex uint64 + ShardCount uint64 +} + +// LabelValue returns the label value to use to select this shard. +func (shard ShardSelector) LabelValue() string { + return FormatShardIDLabelValue(shard.ShardIndex, shard.ShardCount) +} + +// Label generates the ShardSelector as a label. +func (shard ShardSelector) Label() labels.Label { + return labels.Label{ + Name: ShardLabel, + Value: shard.LabelValue(), + } +} + +// Matcher converts ShardSelector to Matcher. +func (shard ShardSelector) Matcher() *labels.Matcher { + return labels.MustNewMatcher(labels.MatchEqual, ShardLabel, shard.LabelValue()) +} + +// ShardFromMatchers extracts a ShardSelector and the index it was pulled from the matcher list. +func ShardFromMatchers(matchers []*labels.Matcher) (shard *ShardSelector, idx int, err error) { + for i, matcher := range matchers { + if matcher.Name == ShardLabel && matcher.Type == labels.MatchEqual { + index, count, err := ParseShardIDLabelValue(matcher.Value) + if err != nil { + return nil, i, err + } + return &ShardSelector{ + ShardIndex: index, + ShardCount: count, + }, i, nil + } + } + return nil, 0, nil +} + +// RemoveShardFromMatchers returns the input matchers without the label matcher on the query shard (if any). +func RemoveShardFromMatchers(matchers []*labels.Matcher) (shard *ShardSelector, filtered []*labels.Matcher, err error) { + shard, idx, err := ShardFromMatchers(matchers) + if err != nil || shard == nil { + return nil, matchers, err + } + + // Create a new slice with the shard matcher removed. + filtered = make([]*labels.Matcher, 0, len(matchers)-1) + filtered = append(filtered, matchers[:idx]...) + filtered = append(filtered, matchers[idx+1:]...) + + return shard, filtered, nil +} + +// FormatShardIDLabelValue expects 0-based shardID, but uses 1-based shard in the output string. +func FormatShardIDLabelValue(shardID, shardCount uint64) string { + return fmt.Sprintf("%d_of_%d", shardID+1, shardCount) +} + +// ParseShardIDLabelValue returns original (0-based) shard index and shard count parsed from formatted value. +func ParseShardIDLabelValue(val string) (index, shardCount uint64, _ error) { + // If we fail to parse shardID, we better not consider this block fully included in successors. + matches := strings.Split(val, "_") + if len(matches) != 3 || matches[1] != "of" { + return 0, 0, errors.Errorf("invalid shard ID: %q", val) + } + + index, err := strconv.ParseUint(matches[0], 10, 64) + if err != nil { + return 0, 0, errors.Errorf("invalid shard ID: %q: %v", val, err) + } + count, err := strconv.ParseUint(matches[2], 10, 64) + if err != nil { + return 0, 0, errors.Errorf("invalid shard ID: %q: %v", val, err) + } + + if index == 0 || count == 0 || index > count { + return 0, 0, errors.Errorf("invalid shard ID: %q", val) + } + + return index - 1, count, nil +} diff --git a/pkg/phlaredb/sharding/label_test.go b/pkg/phlaredb/sharding/label_test.go new file mode 100644 index 0000000000..2ce1e6b239 --- /dev/null +++ b/pkg/phlaredb/sharding/label_test.go @@ -0,0 +1,208 @@ +// SPDX-License-Identifier: AGPL-3.0-only + +package sharding + +import ( + "fmt" + "math/rand" + "testing" + + "github.com/prometheus/prometheus/model/labels" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +func TestParseShard(t *testing.T) { + tests := map[string]struct { + input string + index, count uint64 + err bool + }{ + "should return error on invalid format": { + input: "lsdjf", + err: true, + }, + "should return error on invalid index (not an integer)": { + input: "a_of_3", + err: true, + }, + "should return error on invalid index (not positive)": { + input: "-1_of_3", + err: true, + }, + "should return error on invalid count (not positive)": { + input: "-1_of_-3", + err: true, + }, + "should return error on invalid index (too large)": { + input: "4_of_3", + err: true, + }, + "should return error on invalid index (too small)": { + input: "0_of_3", + err: true, + }, + "should return error on invalid separator": { + input: "1_out_3", + err: true, + }, + "should succeed on valid first shard ID": { + input: "1_of_2", + index: 0, // 0-based + count: 2, + }, + "should succeed on valid last shard selector": { + input: "2_of_2", + index: 1, // 0-based + count: 2, + }, + } + + for testName, testData := range tests { + t.Run(testName, func(t *testing.T) { + index, count, err := ParseShardIDLabelValue(testData.input) + if testData.err { + require.Error(t, err) + } else { + require.NoError(t, err) + require.Equal(t, testData.index, index) + require.Equal(t, testData.count, count) + } + }) + } +} + +func TestRemoveShardFromMatchers(t *testing.T) { + tests := map[string]struct { + input []*labels.Matcher + expectedShard *ShardSelector + expectedMatchers []*labels.Matcher + expectedError error + }{ + "should return no shard on empty label matchers": {}, + "should return no shard on no shard label matcher": { + input: []*labels.Matcher{ + labels.MustNewMatcher(labels.MatchEqual, labels.MetricName, "test"), + labels.MustNewMatcher(labels.MatchRegexp, "foo", "bar.*"), + }, + expectedShard: nil, + expectedMatchers: []*labels.Matcher{ + labels.MustNewMatcher(labels.MatchEqual, labels.MetricName, "test"), + labels.MustNewMatcher(labels.MatchRegexp, "foo", "bar.*"), + }, + }, + "should return matching shard and filter out its matcher": { + input: []*labels.Matcher{ + labels.MustNewMatcher(labels.MatchEqual, labels.MetricName, "test"), + labels.MustNewMatcher(labels.MatchEqual, ShardLabel, ShardSelector{ShardIndex: 1, ShardCount: 8}.LabelValue()), + labels.MustNewMatcher(labels.MatchRegexp, "foo", "bar.*"), + }, + expectedShard: &ShardSelector{ + ShardIndex: 1, + ShardCount: 8, + }, + expectedMatchers: []*labels.Matcher{ + labels.MustNewMatcher(labels.MatchEqual, labels.MetricName, "test"), + labels.MustNewMatcher(labels.MatchRegexp, "foo", "bar.*"), + }, + }, + } + + for testName, testData := range tests { + t.Run(testName, func(t *testing.T) { + actualShard, actualMatchers, actualError := RemoveShardFromMatchers(testData.input) + assert.Equal(t, testData.expectedShard, actualShard) + assert.Equal(t, testData.expectedError, actualError) + + // Assert same matchers. We do some optimizations in mimir-prometheus which make + // the label matchers not comparable with reflect.DeepEqual() so we're going to + // compare their string representation. + require.Len(t, actualMatchers, len(testData.expectedMatchers)) + for i := 0; i < len(testData.expectedMatchers); i++ { + assert.Equal(t, testData.expectedMatchers[i].String(), actualMatchers[i].String()) + } + }) + } +} + +func TestShardFromMatchers(t *testing.T) { + testExpr := []struct { + input []*labels.Matcher + shard *ShardSelector + idx int + err bool + }{ + { + input: []*labels.Matcher{ + {}, + { + Name: ShardLabel, + Type: labels.MatchEqual, + Value: ShardSelector{ + ShardIndex: 10, + ShardCount: 16, + }.LabelValue(), + }, + {}, + }, + shard: &ShardSelector{ + ShardIndex: 10, + ShardCount: 16, + }, + idx: 1, + err: false, + }, + { + input: []*labels.Matcher{ + { + Name: ShardLabel, + Type: labels.MatchEqual, + Value: "invalid-fmt", + }, + }, + shard: nil, + idx: 0, + err: true, + }, + { + input: []*labels.Matcher{}, + shard: nil, + idx: 0, + err: false, + }, + } + + for i, c := range testExpr { + t.Run(fmt.Sprint(i), func(t *testing.T) { + shard, idx, err := ShardFromMatchers(c.input) + if c.err { + require.NotNil(t, err) + } else { + require.Nil(t, err) + require.Equal(t, c.shard, shard) + require.Equal(t, c.idx, idx) + } + }) + } +} + +func TestFormatAndParseShardId(t *testing.T) { + r := rand.New(rand.NewSource(0)) + + const maxTests = 1000 + const maxShardCount = 10000 + + for i := 0; i < maxTests; i++ { + count := 1 + r.Intn(maxShardCount) + id := r.Intn(count) + + require.True(t, id < count) + + out := FormatShardIDLabelValue(uint64(id), uint64(count)) + nid, ncount, err := ParseShardIDLabelValue(out) + + require.NoError(t, err) + require.Equal(t, uint64(id), nid) + require.Equal(t, uint64(count), ncount) + } +} From 64d22fdab1196b1130cbf585d375d0ff1dd3a14e Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Wed, 6 Sep 2023 22:50:29 +0200 Subject: [PATCH 05/74] Add some tests for CompactWithSplitting --- pkg/phlaredb/compact_test.go | 147 +++++++++++++++++++++++++++++++++++ 1 file changed, 147 insertions(+) diff --git a/pkg/phlaredb/compact_test.go b/pkg/phlaredb/compact_test.go index 995b722690..822221d126 100644 --- a/pkg/phlaredb/compact_test.go +++ b/pkg/phlaredb/compact_test.go @@ -6,6 +6,7 @@ import ( _ "net/http/pprof" "os" "path/filepath" + "sort" "testing" "time" @@ -14,6 +15,7 @@ import ( "github.com/prometheus/common/model" "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/tsdb" + "github.com/samber/lo" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" @@ -23,6 +25,7 @@ import ( "github.com/grafana/pyroscope/pkg/objstore/client" "github.com/grafana/pyroscope/pkg/objstore/providers/filesystem" "github.com/grafana/pyroscope/pkg/phlaredb/block" + "github.com/grafana/pyroscope/pkg/phlaredb/sharding" "github.com/grafana/pyroscope/pkg/phlaredb/tsdb/index" "github.com/grafana/pyroscope/pkg/pprof/testhelper" ) @@ -85,6 +88,150 @@ func TestCompact(t *testing.T) { require.Equal(t, expected.String(), res.String()) } +func TestCompactWithSplitting(t *testing.T) { + ctx := context.Background() + + b1 := newBlock(t, func() []*testhelper.ProfileBuilder { + return append( + profileSeriesGenerator(t, time.Unix(1, 0), time.Unix(10, 0), time.Second, "job", "a"), + profileSeriesGenerator(t, time.Unix(11, 0), time.Unix(20, 0), time.Second, "job", "b")..., + ) + }) + b2 := newBlock(t, func() []*testhelper.ProfileBuilder { + return append( + append( + append( + profileSeriesGenerator(t, time.Unix(1, 0), time.Unix(10, 0), time.Second, "job", "c"), + profileSeriesGenerator(t, time.Unix(11, 0), time.Unix(20, 0), time.Second, "job", "d")..., + ), profileSeriesGenerator(t, time.Unix(1, 0), time.Unix(10, 0), time.Second, "job", "a")..., + ), + profileSeriesGenerator(t, time.Unix(11, 0), time.Unix(20, 0), time.Second, "job", "b")..., + ) + }) + dst := t.TempDir() + compacted, err := CompactWithSplitting(ctx, []BlockReader{b1, b2, b2, b1}, 16, dst) + require.NoError(t, err) + // todo: fix and test min/max time from split blocks + + // 4 shards one per series. + require.Equal(t, 4, len(compacted)) + require.Equal(t, "1_of_16", compacted[0].Labels[sharding.CompactorShardIDLabel]) + require.Equal(t, "6_of_16", compacted[1].Labels[sharding.CompactorShardIDLabel]) + require.Equal(t, "7_of_16", compacted[2].Labels[sharding.CompactorShardIDLabel]) + require.Equal(t, "14_of_16", compacted[3].Labels[sharding.CompactorShardIDLabel]) + + // We first verify we have all series and timestamps across querying all blocks. + queriers := make(Queriers, len(compacted)) + for i, blk := range compacted { + queriers[i] = blockQuerierFromMeta(t, dst, blk) + } + + err = queriers.Open(context.Background()) + require.NoError(t, err) + matchAll := &ingesterv1.SelectProfilesRequest{ + LabelSelector: "{}", + Type: mustParseProfileSelector(t, "process_cpu:cpu:nanoseconds:cpu:nanoseconds"), + Start: 0, + End: 40000, + } + it, err := queriers.SelectMatchingProfiles(context.Background(), matchAll) + require.NoError(t, err) + + seriesMap := make(map[model.Fingerprint]lo.Tuple2[phlaremodel.Labels, []model.Time]) + for it.Next() { + r := it.At() + seriesMap[r.Fingerprint()] = lo.T2(r.Labels().WithoutPrivateLabels(), append(seriesMap[r.Fingerprint()].B, r.Timestamp())) + } + require.NoError(t, it.Err()) + require.NoError(t, it.Close()) + series := lo.Values(seriesMap) + sort.Slice(series, func(i, j int) bool { + return phlaremodel.CompareLabelPairs(series[i].A, series[j].A) < 0 + }) + require.Equal(t, []lo.Tuple2[phlaremodel.Labels, []model.Time]{ + lo.T2(phlaremodel.LabelsFromStrings("job", "a"), + generateTimes(t, model.TimeFromUnix(1), model.TimeFromUnix(10)), + ), + lo.T2(phlaremodel.LabelsFromStrings("job", "b"), + generateTimes(t, model.TimeFromUnix(11), model.TimeFromUnix(20)), + ), + lo.T2(phlaremodel.LabelsFromStrings("job", "c"), + generateTimes(t, model.TimeFromUnix(1), model.TimeFromUnix(10)), + ), + lo.T2(phlaremodel.LabelsFromStrings("job", "d"), + generateTimes(t, model.TimeFromUnix(11), model.TimeFromUnix(20)), + ), + }, series) + + // Then we query 2 different shards and verify we have a subset of series. + it, err = queriers[0].SelectMatchingProfiles(ctx, matchAll) + require.NoError(t, err) + seriesResult, err := queriers[0].MergeByLabels(context.Background(), it, "job") + require.NoError(t, err) + require.Equal(t, + []*typesv1.Series{ + { + Labels: phlaremodel.LabelsFromStrings("job", "a"), + Points: generatePoints(t, model.TimeFromUnix(1), model.TimeFromUnix(10)), + }, + }, seriesResult) + + it, err = queriers[1].SelectMatchingProfiles(ctx, matchAll) + require.NoError(t, err) + seriesResult, err = queriers[1].MergeByLabels(context.Background(), it, "job") + require.NoError(t, err) + require.Equal(t, + []*typesv1.Series{ + { + Labels: phlaremodel.LabelsFromStrings("job", "b"), + Points: generatePoints(t, model.TimeFromUnix(11), model.TimeFromUnix(20)), + }, + }, seriesResult) + + // Finally test some stacktraces resolution. + it, err = queriers[1].SelectMatchingProfiles(ctx, matchAll) + require.NoError(t, err) + res, err := queriers[1].MergeByStacktraces(ctx, it) + require.NoError(t, err) + + expected := new(phlaremodel.Tree) + expected.InsertStack(10, "baz", "bar", "foo") + require.Equal(t, expected.String(), res.String()) +} + +// nolint:unparam +func profileSeriesGenerator(t *testing.T, from, through time.Time, interval time.Duration, lbls ...string) []*testhelper.ProfileBuilder { + t.Helper() + var builders []*testhelper.ProfileBuilder + for ts := from; ts.Before(through) || ts.Equal(through); ts = ts.Add(interval) { + builders = append(builders, + testhelper.NewProfileBuilder(ts.UnixNano()). + CPUProfile(). + WithLabels( + lbls..., + ).ForStacktraceString("foo", "bar", "baz").AddSamples(1)) + } + return builders +} + +func generatePoints(t *testing.T, from, through model.Time) []*typesv1.Point { + t.Helper() + var points []*typesv1.Point + for ts := from; ts.Before(through) || ts.Equal(through); ts = ts.Add(time.Second) { + points = append(points, &typesv1.Point{Timestamp: int64(ts), Value: 1}) + } + return points +} + +func generateTimes(t *testing.T, from, through model.Time) []model.Time { + t.Helper() + var times []model.Time + for ts := from; ts.Before(through) || ts.Equal(through); ts = ts.Add(time.Second) { + times = append(times, ts) + } + return times +} + func TestProfileRowIterator(t *testing.T) { b := newBlock(t, func() []*testhelper.ProfileBuilder { return []*testhelper.ProfileBuilder{ From 22d6271f1ddf0901c809f92239c8a30c9c1a41b3 Mon Sep 17 00:00:00 2001 From: Anton Kolesnikov Date: Thu, 7 Sep 2023 15:03:20 +0800 Subject: [PATCH 06/74] Fix symbols split-compaction (#2371) --- pkg/phlaredb/schemas/v1/functions.go | 5 +++++ pkg/phlaredb/schemas/v1/locations.go | 7 +++++++ pkg/phlaredb/schemas/v1/mappings.go | 5 +++++ pkg/phlaredb/symdb/rewriter.go | 29 ++++++++++++++++++++++++---- 4 files changed, 42 insertions(+), 4 deletions(-) diff --git a/pkg/phlaredb/schemas/v1/functions.go b/pkg/phlaredb/schemas/v1/functions.go index 31ba8b5037..62d723fc7d 100644 --- a/pkg/phlaredb/schemas/v1/functions.go +++ b/pkg/phlaredb/schemas/v1/functions.go @@ -53,3 +53,8 @@ type InMemoryFunction struct { // Line number in source file. StartLine uint32 } + +func (f *InMemoryFunction) Clone() *InMemoryFunction { + n := *f + return &n +} diff --git a/pkg/phlaredb/schemas/v1/locations.go b/pkg/phlaredb/schemas/v1/locations.go index 27dc019a9e..b9cbf91ba6 100644 --- a/pkg/phlaredb/schemas/v1/locations.go +++ b/pkg/phlaredb/schemas/v1/locations.go @@ -110,6 +110,13 @@ type InMemoryLocation struct { Line []InMemoryLine } +func (l *InMemoryLocation) Clone() *InMemoryLocation { + x := *l + x.Line = make([]InMemoryLine, len(l.Line)) + copy(x.Line, l.Line) + return &x +} + type InMemoryLine struct { // The id of the corresponding profile.Function for this line. FunctionId uint32 diff --git a/pkg/phlaredb/schemas/v1/mappings.go b/pkg/phlaredb/schemas/v1/mappings.go index 6b342c24fe..0d5503f6cb 100644 --- a/pkg/phlaredb/schemas/v1/mappings.go +++ b/pkg/phlaredb/schemas/v1/mappings.go @@ -73,3 +73,8 @@ type InMemoryMapping struct { HasLineNumbers bool HasInlineFrames bool } + +func (m *InMemoryMapping) Clone() *InMemoryMapping { + n := *m + return &n +} diff --git a/pkg/phlaredb/symdb/rewriter.go b/pkg/phlaredb/symdb/rewriter.go index cf1ba5d408..8d34b52f36 100644 --- a/pkg/phlaredb/symdb/rewriter.go +++ b/pkg/phlaredb/symdb/rewriter.go @@ -54,14 +54,15 @@ func (r *Rewriter) getOrCreatePartition(partition uint64) (_ *partitionRewriter, n := &partitionRewriter{name: partition} n.dst = r.symdb.PartitionWriter(partition) + // Note that the partition is not released: we want to keep + // it during the whole lifetime of the rewriter. pr, err := r.source.Partition(context.TODO(), partition) if err != nil { return nil, err } - - // Note that the partition is not released: we want to keep - // it during the whole lifetime of the rewriter. - n.src = pr.Symbols() + // We clone locations, functions, and mappings, + // because these object will be modified. + n.src = cloneSymbolsPartially(pr.Symbols()) var stats PartitionStats pr.WriteStats(&stats) @@ -242,6 +243,26 @@ func (p *partitionRewriter) InsertStacktrace(stacktrace uint32, locations []int3 p.stacktraces.values[idx] = n } +func cloneSymbolsPartially(x *Symbols) *Symbols { + n := Symbols{ + Stacktraces: x.Stacktraces, + Locations: make([]*schemav1.InMemoryLocation, len(x.Locations)), + Mappings: make([]*schemav1.InMemoryMapping, len(x.Mappings)), + Functions: make([]*schemav1.InMemoryFunction, len(x.Functions)), + Strings: x.Strings, + } + for i, l := range x.Locations { + n.Locations[i] = l.Clone() + } + for i, m := range x.Mappings { + n.Mappings[i] = m.Clone() + } + for i, f := range x.Functions { + n.Functions[i] = f.Clone() + } + return &n +} + const ( marker = 1 << 31 markerMask = math.MaxUint32 >> 1 From 816eaaa14b72ae80a79e8b9fd0437db7c897c42a Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Thu, 7 Sep 2023 11:51:53 +0200 Subject: [PATCH 07/74] Fixes a race that was actually surfacing another real issue. --- pkg/phlaredb/head.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/phlaredb/head.go b/pkg/phlaredb/head.go index d4ed86dec8..dd8f1fb9de 100644 --- a/pkg/phlaredb/head.go +++ b/pkg/phlaredb/head.go @@ -488,7 +488,7 @@ func (h *Head) flush(ctx context.Context) error { // It must be guaranteed that no new inserts will happen // after the call start. h.inFlightProfiles.Wait() - if len(h.profiles.slice) == 0 { + if h.profiles.index.totalProfiles.Load() == 0 { level.Info(h.logger).Log("msg", "head empty - no block written") return os.RemoveAll(h.headPath) } From d100b30ca18aa693c5f72a86a2b95d44b9cb2c6d Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Thu, 7 Sep 2023 11:59:08 +0200 Subject: [PATCH 08/74] Add a tests for meta min/max time --- pkg/phlaredb/compact_test.go | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/pkg/phlaredb/compact_test.go b/pkg/phlaredb/compact_test.go index 822221d126..248635b771 100644 --- a/pkg/phlaredb/compact_test.go +++ b/pkg/phlaredb/compact_test.go @@ -111,7 +111,6 @@ func TestCompactWithSplitting(t *testing.T) { dst := t.TempDir() compacted, err := CompactWithSplitting(ctx, []BlockReader{b1, b2, b2, b1}, 16, dst) require.NoError(t, err) - // todo: fix and test min/max time from split blocks // 4 shards one per series. require.Equal(t, 4, len(compacted)) @@ -120,6 +119,11 @@ func TestCompactWithSplitting(t *testing.T) { require.Equal(t, "7_of_16", compacted[2].Labels[sharding.CompactorShardIDLabel]) require.Equal(t, "14_of_16", compacted[3].Labels[sharding.CompactorShardIDLabel]) + // todo: fix and test min/max time from split blocks + // The series b should span from 11 to 20 and not 1 to 20. + require.Equal(t, model.TimeFromUnix(11), compacted[1].MinTime) + require.Equal(t, model.TimeFromUnix(20), compacted[1].MaxTime) + // We first verify we have all series and timestamps across querying all blocks. queriers := make(Queriers, len(compacted)) for i, blk := range compacted { From bd4f5e79a6f7e2abe6d77a76369aeb3b2ea3de6a Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Thu, 7 Sep 2023 14:19:49 +0200 Subject: [PATCH 09/74] Fixes meta min/max after split --- pkg/phlaredb/compact.go | 12 ++++++++++++ pkg/phlaredb/compact_test.go | 1 - 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/pkg/phlaredb/compact.go b/pkg/phlaredb/compact.go index 57df7d06fe..994c8fc831 100644 --- a/pkg/phlaredb/compact.go +++ b/pkg/phlaredb/compact.go @@ -128,6 +128,7 @@ type blockWriter struct { path string meta *block.Meta totalProfiles uint64 + min, max int64 } func newBlockWriter(dst string, meta *block.Meta) (*blockWriter, error) { @@ -148,6 +149,8 @@ func newBlockWriter(dst string, meta *block.Meta) (*blockWriter, error) { profilesWriter: profileWriter, path: blockPath, meta: meta, + min: math.MaxInt64, + max: math.MinInt64, }, nil } @@ -165,6 +168,12 @@ func (bw *blockWriter) WriteRow(r profileRow) error { return err } bw.totalProfiles++ + if r.timeNanos < bw.min { + bw.min = r.timeNanos + } + if r.timeNanos > bw.max { + bw.max = r.timeNanos + } return nil } @@ -187,6 +196,8 @@ func (bw *blockWriter) Close(ctx context.Context) error { bw.meta.Stats.NumSeries = bw.indexRewriter.NumSeries() bw.meta.Stats.NumSamples = bw.symbolsRewriter.NumSamples() bw.meta.Compaction.Deletable = bw.totalProfiles == 0 + bw.meta.MinTime = model.TimeFromUnixNano(bw.min) + bw.meta.MaxTime = model.TimeFromUnixNano(bw.max) if _, err := bw.meta.WriteToFile(util.Logger, bw.path); err != nil { return err } @@ -219,6 +230,7 @@ func (p *profilesWriter) WriteRow(r profileRow) error { if err != nil { return err } + return nil } diff --git a/pkg/phlaredb/compact_test.go b/pkg/phlaredb/compact_test.go index 248635b771..1966f49a3c 100644 --- a/pkg/phlaredb/compact_test.go +++ b/pkg/phlaredb/compact_test.go @@ -119,7 +119,6 @@ func TestCompactWithSplitting(t *testing.T) { require.Equal(t, "7_of_16", compacted[2].Labels[sharding.CompactorShardIDLabel]) require.Equal(t, "14_of_16", compacted[3].Labels[sharding.CompactorShardIDLabel]) - // todo: fix and test min/max time from split blocks // The series b should span from 11 to 20 and not 1 to 20. require.Equal(t, model.TimeFromUnix(11), compacted[1].MinTime) require.Equal(t, model.TimeFromUnix(20), compacted[1].MaxTime) From cbcf90aae491511a83a7d137931a994f0751577a Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Thu, 7 Sep 2023 16:42:37 +0200 Subject: [PATCH 10/74] Add mimir compactor code --- pkg/compactor/SPLIT_MERGE_COMPACTOR.md | 34 + pkg/compactor/block_upload.go | 864 +++++++ pkg/compactor/block_upload_test.go | 2116 +++++++++++++++ pkg/compactor/blocks_cleaner.go | 639 +++++ pkg/compactor/blocks_cleaner_test.go | 1121 ++++++++ pkg/compactor/bucket_compactor.go | 1032 ++++++++ pkg/compactor/bucket_compactor_e2e_test.go | 848 ++++++ pkg/compactor/bucket_compactor_test.go | 355 +++ pkg/compactor/compactor.go | 928 +++++++ pkg/compactor/compactor_http.go | 46 + pkg/compactor/compactor_ring.go | 81 + pkg/compactor/compactor_test.go | 2298 +++++++++++++++++ pkg/compactor/job.go | 188 ++ pkg/compactor/job_sorting.go | 99 + pkg/compactor/job_sorting_test.go | 146 ++ pkg/compactor/job_test.go | 135 + pkg/compactor/label_remover_filter.go | 34 + pkg/compactor/label_remover_filter_test.go | 77 + .../shard_aware_deduplicate_filter.go | 293 +++ .../shard_aware_deduplicate_filter_test.go | 460 ++++ pkg/compactor/split_merge_compactor.go | 46 + pkg/compactor/split_merge_compactor_test.go | 877 +++++++ pkg/compactor/split_merge_grouper.go | 380 +++ pkg/compactor/split_merge_grouper_test.go | 794 ++++++ pkg/compactor/split_merge_job.go | 148 ++ pkg/compactor/split_merge_job_test.go | 262 ++ pkg/compactor/split_merge_planner.go | 53 + pkg/compactor/split_merge_planner_test.go | 85 + pkg/compactor/status.gohtml | 11 + pkg/compactor/syncer_metrics.go | 90 + pkg/compactor/syncer_metrics_test.go | 148 ++ pkg/compactor/tenant_deletion_api.go | 98 + pkg/compactor/tenant_deletion_api_test.go | 100 + 33 files changed, 14886 insertions(+) create mode 100644 pkg/compactor/SPLIT_MERGE_COMPACTOR.md create mode 100644 pkg/compactor/block_upload.go create mode 100644 pkg/compactor/block_upload_test.go create mode 100644 pkg/compactor/blocks_cleaner.go create mode 100644 pkg/compactor/blocks_cleaner_test.go create mode 100644 pkg/compactor/bucket_compactor.go create mode 100644 pkg/compactor/bucket_compactor_e2e_test.go create mode 100644 pkg/compactor/bucket_compactor_test.go create mode 100644 pkg/compactor/compactor.go create mode 100644 pkg/compactor/compactor_http.go create mode 100644 pkg/compactor/compactor_ring.go create mode 100644 pkg/compactor/compactor_test.go create mode 100644 pkg/compactor/job.go create mode 100644 pkg/compactor/job_sorting.go create mode 100644 pkg/compactor/job_sorting_test.go create mode 100644 pkg/compactor/job_test.go create mode 100644 pkg/compactor/label_remover_filter.go create mode 100644 pkg/compactor/label_remover_filter_test.go create mode 100644 pkg/compactor/shard_aware_deduplicate_filter.go create mode 100644 pkg/compactor/shard_aware_deduplicate_filter_test.go create mode 100644 pkg/compactor/split_merge_compactor.go create mode 100644 pkg/compactor/split_merge_compactor_test.go create mode 100644 pkg/compactor/split_merge_grouper.go create mode 100644 pkg/compactor/split_merge_grouper_test.go create mode 100644 pkg/compactor/split_merge_job.go create mode 100644 pkg/compactor/split_merge_job_test.go create mode 100644 pkg/compactor/split_merge_planner.go create mode 100644 pkg/compactor/split_merge_planner_test.go create mode 100644 pkg/compactor/status.gohtml create mode 100644 pkg/compactor/syncer_metrics.go create mode 100644 pkg/compactor/syncer_metrics_test.go create mode 100644 pkg/compactor/tenant_deletion_api.go create mode 100644 pkg/compactor/tenant_deletion_api_test.go diff --git a/pkg/compactor/SPLIT_MERGE_COMPACTOR.md b/pkg/compactor/SPLIT_MERGE_COMPACTOR.md new file mode 100644 index 0000000000..ca7a86a4b1 --- /dev/null +++ b/pkg/compactor/SPLIT_MERGE_COMPACTOR.md @@ -0,0 +1,34 @@ +# Split and Merge Compactor + +## How the grouper works + +Given the list of all blocks in the storage: + +- Group blocks by resolution + external labels (excluding shard ID). For each group: + - Group blocks by compactable time ranges (eg. 2h, 6h, 12h, 24h) + - Given the time range, if the time range is the "smallest one" (eg. 2h) AND there are non-sharded blocks, then we should split the block + - Example: TR=2h + - Case 1: all the blocks for the 2h time range have the "shard ID" label in the meta.json. + In this case, we shouldn't run the split stage anymore. + - Case 2: there is at least 1 block for the 2h time range NOT having "shard ID" in the meta.json. + In this case, we should run the split stage on 2h blocks without the "shard ID". + - Horizontal sharding + - Each compactor will take decisions without a central coordination + - Each compactor will run the planning for ALL blocks + - Each compactor will only execute "jobs" belonging to its shard + - Splitting job (each job consists of a set of blocks which will be merged together calling CompactWithSplitting()) + - Which blocks should belong to a specific job? + - We don't want all blocks to be compacted together, otherwise doesn't scale + - We use the configured number of output shards to determine how many concurrent jobs we want to run to split/shard blocks + - For each block to split, we add it to the job with index `hash(blockID) % numShards` + - Output: up to `numShards` jobs (each job contain the set of blocks to be merged together when running CompactWithSplitting()) + - How can we know if the compactor instance should process a job or not? + - A job is owned if `hash(tenant + stage + time range + shard ID)` belongs to the compactor tokens + - If the previous check has not produced any job AT ALL it means all the blocks for the "smallest time range" are already split + (or there are no blocks at all), so we can proceed with the merging stage: + - Group blocks by "shard ID" (taking in account the case a block doesn't have the shard ID) + - Create a job for each group that contains 2+ blocks (because they need to be merged together) + - Add the job to the list of "valid jobs to execute" only if for the job shard ID there are no other + jobs already in the list of "valid jobs" overlapping its time range +- Execute the jobs that belong to the compactor instance + - Loop through jobs and filter out the jobs hashes not belonging to the compactor tokens diff --git a/pkg/compactor/block_upload.go b/pkg/compactor/block_upload.go new file mode 100644 index 0000000000..2cfdb1a7c6 --- /dev/null +++ b/pkg/compactor/block_upload.go @@ -0,0 +1,864 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/grafana/mimir/blob/main/pkg/compactor/block_upload.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Cortex Authors. + +package compactor + +import ( + "bytes" + "context" + "encoding/binary" + "encoding/hex" + "encoding/json" + "fmt" + "io" + "net/http" + "os" + "path" + "path/filepath" + "sync" + "time" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/gorilla/mux" + "github.com/grafana/dskit/tenant" + "github.com/grafana/regexp" + "github.com/oklog/ulid" + "github.com/pkg/errors" + "github.com/thanos-io/objstore" + + "github.com/grafana/mimir/pkg/storage/bucket" + "github.com/grafana/mimir/pkg/storage/sharding" + mimir_tsdb "github.com/grafana/mimir/pkg/storage/tsdb" + "github.com/grafana/mimir/pkg/storage/tsdb/block" + "github.com/grafana/mimir/pkg/util" + util_log "github.com/grafana/mimir/pkg/util/log" +) + +const ( + uploadingMetaFilename = "uploading-meta.json" // Name of the file that stores a block's meta file while it's being uploaded + validationFilename = "validation.json" // Name of the file that stores a heartbeat time and possibly an error message + validationHeartbeatInterval = 1 * time.Minute // Duration of time between heartbeats of an in-progress block upload validation + validationHeartbeatTimeout = 5 * time.Minute // Maximum duration of time to wait until a validation is able to be restarted + maximumMetaSizeBytes = 1 * 1024 * 1024 // 1 MiB, maximum allowed size of an uploaded block's meta.json file +) + +var ( + maxBlockUploadSizeBytesFormat = "block exceeds the maximum block size limit of %d bytes" + rePath = regexp.MustCompile(`^(index|chunks/\d{6})$`) +) + +// StartBlockUpload handles request for starting block upload. +// +// Starting the uploading of a block means to upload a meta file and verify that the upload can +// go ahead. In practice this means to check that the (complete) block isn't already in block +// storage, and that the meta file is valid. +func (c *MultitenantCompactor) StartBlockUpload(w http.ResponseWriter, r *http.Request) { + blockID, tenantID, err := c.parseBlockUploadParameters(r) + if err != nil { + http.Error(w, err.Error(), http.StatusBadRequest) + return + } + + ctx := r.Context() + requestID := hexTimeNowNano() + logger := log.With( + util_log.WithContext(ctx, c.logger), + "feature", "block upload", + "block", blockID, + "operation", "start block upload", + "request_id", requestID, + ) + + userBkt := bucket.NewUserBucketClient(tenantID, c.bucketClient, c.cfgProvider) + if _, _, err := c.checkBlockState(ctx, userBkt, blockID, false); err != nil { + writeBlockUploadError(err, "can't check block state", logger, w, requestID) + return + } + + content, err := io.ReadAll(http.MaxBytesReader(w, r.Body, maximumMetaSizeBytes)) + if err != nil { + if errors.As(err, new(*http.MaxBytesError)) { + err = httpError{ + message: fmt.Sprintf("The block metadata was too large (maximum size allowed is %d bytes)", maximumMetaSizeBytes), + statusCode: http.StatusRequestEntityTooLarge, + } + } + writeBlockUploadError(err, "failed reading body", logger, w, requestID) + return + } + + var meta block.Meta + if err := json.Unmarshal(content, &meta); err != nil { + err = httpError{ + message: "malformed request body", + statusCode: http.StatusBadRequest, + } + writeBlockUploadError(err, "failed unmarshaling block meta json", logger, w, requestID) + return + } + + if err := c.createBlockUpload(ctx, &meta, logger, userBkt, tenantID, blockID); err != nil { + writeBlockUploadError(err, "failed creating block upload", logger, w, requestID) + return + } + + level.Info(logger).Log("msg", "started block upload") + + w.WriteHeader(http.StatusOK) +} + +// FinishBlockUpload handles request for finishing block upload. +// +// Finishing block upload performs block validation, and if all checks pass, marks block as finished +// by uploading meta.json file. +func (c *MultitenantCompactor) FinishBlockUpload(w http.ResponseWriter, r *http.Request) { + blockID, tenantID, err := c.parseBlockUploadParameters(r) + if err != nil { + http.Error(w, err.Error(), http.StatusBadRequest) + return + } + + ctx := r.Context() + requestID := hexTimeNowNano() + logger := log.With( + util_log.WithContext(ctx, c.logger), + "feature", "block upload", + "block", blockID, + "operation", "complete block upload", + "request_id", requestID, + ) + + userBkt := bucket.NewUserBucketClient(tenantID, c.bucketClient, c.cfgProvider) + m, _, err := c.checkBlockState(ctx, userBkt, blockID, true) + if err != nil { + writeBlockUploadError(err, "can't check block state", logger, w, requestID) + return + } + + // This should not happen, as checkBlockState with requireUploadInProgress=true returns nil error + // only if uploading-meta.json file exists. + if m == nil { + http.Error(w, "internal error", http.StatusInternalServerError) + return + } + + if c.cfgProvider.CompactorBlockUploadValidationEnabled(tenantID) { + maxConcurrency := int64(c.compactorCfg.MaxBlockUploadValidationConcurrency) + currentValidations := c.blockUploadValidations.Inc() + decreaseActiveValidationsInDefer := true + defer func() { + if decreaseActiveValidationsInDefer { + c.blockUploadValidations.Dec() + } + }() + if maxConcurrency > 0 && currentValidations > maxConcurrency { + err := httpError{ + message: fmt.Sprintf("too many block upload validations in progress, limit is %d", maxConcurrency), + statusCode: http.StatusTooManyRequests, + } + writeBlockUploadError(err, "max concurrency was hit", logger, w, requestID) + return + } + // create validation file to signal that block validation has started + if err := c.uploadValidation(ctx, blockID, userBkt); err != nil { + writeBlockUploadError(err, "can't upload validation file", logger, w, requestID) + return + } + decreaseActiveValidationsInDefer = false + go c.validateAndCompleteBlockUpload(logger, tenantID, userBkt, blockID, m, func(ctx context.Context) error { + defer c.blockUploadValidations.Dec() + return c.validateBlock(ctx, logger, blockID, m, userBkt, tenantID) + }) + level.Info(logger).Log("msg", "validation process started") + } else { + if err := c.markBlockComplete(ctx, logger, tenantID, userBkt, blockID, m); err != nil { + writeBlockUploadError(err, "can't mark block as complete", logger, w, requestID) + return + } + level.Info(logger).Log("msg", "successfully finished block upload") + } + + w.WriteHeader(http.StatusOK) +} + +// parseBlockUploadParameters parses common parameters from the request: block ID, tenant and checks if tenant has uploads enabled. +func (c *MultitenantCompactor) parseBlockUploadParameters(r *http.Request) (ulid.ULID, string, error) { + blockID, err := ulid.Parse(mux.Vars(r)["block"]) + if err != nil { + return ulid.ULID{}, "", errors.New("invalid block ID") + } + + ctx := r.Context() + tenantID, err := tenant.TenantID(ctx) + if err != nil { + return ulid.ULID{}, "", errors.New("invalid tenant ID") + } + + if !c.cfgProvider.CompactorBlockUploadEnabled(tenantID) { + return ulid.ULID{}, "", errors.New("block upload is disabled") + } + + return blockID, tenantID, nil +} + +func writeBlockUploadError(err error, msg string, logger log.Logger, w http.ResponseWriter, requestID string) { + var httpErr httpError + if errors.As(err, &httpErr) { + level.Warn(logger).Log("msg", msg, "response", httpErr.message, "status", httpErr.statusCode) + http.Error(w, httpErr.message, httpErr.statusCode) + return + } + + level.Error(logger).Log("msg", msg, "err", err) + http.Error(w, fmt.Sprintf("internal server error (id %s)", requestID), http.StatusInternalServerError) +} + +// hexTimeNano returns a hex-encoded big-endian representation of the current time in nanoseconds, previously converted to uint64 and encoded as big-endian. +func hexTimeNowNano() string { + var buf [8]byte + binary.BigEndian.PutUint64(buf[:], uint64(time.Now().UTC().UnixNano())) + return hex.EncodeToString(buf[:]) +} + +func (c *MultitenantCompactor) createBlockUpload(ctx context.Context, meta *block.Meta, + logger log.Logger, userBkt objstore.Bucket, tenantID string, blockID ulid.ULID, +) error { + level.Debug(logger).Log("msg", "starting block upload") + + if msg := c.sanitizeMeta(logger, tenantID, blockID, meta); msg != "" { + return httpError{ + message: msg, + statusCode: http.StatusBadRequest, + } + } + + // validate data is within the retention period + retention := c.cfgProvider.CompactorBlocksRetentionPeriod(tenantID) + if retention > 0 { + threshold := time.Now().Add(-retention) + if time.UnixMilli(meta.MaxTime).Before(threshold) { + maxTimeStr := util.FormatTimeMillis(meta.MaxTime) + return httpError{ + message: fmt.Sprintf("block max time (%s) older than retention period", maxTimeStr), + statusCode: http.StatusUnprocessableEntity, + } + } + } + + return c.uploadMeta(ctx, logger, meta, blockID, uploadingMetaFilename, userBkt) +} + +// UploadBlockFile handles requests for uploading block files. +// It takes the mandatory query parameter "path", specifying the file's destination path. +func (c *MultitenantCompactor) UploadBlockFile(w http.ResponseWriter, r *http.Request) { + blockID, tenantID, err := c.parseBlockUploadParameters(r) + if err != nil { + http.Error(w, err.Error(), http.StatusBadRequest) + return + } + + ctx := r.Context() + requestID := hexTimeNowNano() + logger := log.With( + util_log.WithContext(ctx, c.logger), + "feature", "block upload", + "block", blockID, + "operation", "block file upload", + "request", requestID, + ) + + pth := r.URL.Query().Get("path") + if pth == "" { + err := httpError{statusCode: http.StatusBadRequest, message: "missing or invalid file path"} + writeBlockUploadError(err, "failed because file path is empty", logger, w, requestID) + return + } + + if path.Base(pth) == block.MetaFilename { + err := httpError{statusCode: http.StatusBadRequest, message: fmt.Sprintf("%s is not allowed", block.MetaFilename)} + writeBlockUploadError(err, "failed because block meta is not allowed", logger, w, requestID) + return + } + + if !rePath.MatchString(pth) { + err := httpError{statusCode: http.StatusBadRequest, message: fmt.Sprintf("invalid path: %q", pth)} + writeBlockUploadError(err, "failed because path is invalid", logger, w, requestID) + return + } + + if r.ContentLength == 0 { + err := httpError{statusCode: http.StatusBadRequest, message: "file cannot be empty"} + writeBlockUploadError(err, "failed because file is empty", logger, w, requestID) + return + } + + userBkt := bucket.NewUserBucketClient(tenantID, c.bucketClient, c.cfgProvider) + + m, _, err := c.checkBlockState(ctx, userBkt, blockID, true) + if err != nil { + writeBlockUploadError(err, "can't check block state", logger, w, requestID) + return + } + + // This should not happen. + if m == nil { + err := httpError{statusCode: http.StatusInternalServerError, message: "internal error"} + writeBlockUploadError(err, "block meta is nil but err is also nil", logger, w, requestID) + return + } + + // Check if file was specified in meta.json, and if it has expected size. + found := false + for _, f := range m.Thanos.Files { + if pth == f.RelPath { + found = true + + if r.ContentLength >= 0 && r.ContentLength != f.SizeBytes { + err := httpError{statusCode: http.StatusBadRequest, message: fmt.Sprintf("file size doesn't match %s", block.MetaFilename)} + writeBlockUploadError(err, "failed because file size didn't match", logger, w, requestID) + return + } + } + } + if !found { + err := httpError{statusCode: http.StatusBadRequest, message: "unexpected file"} + writeBlockUploadError(err, "failed because file was not found", logger, w, requestID) + return + } + + dst := path.Join(blockID.String(), pth) + + level.Debug(logger).Log("msg", "uploading block file to bucket", "destination", dst, "size", r.ContentLength) + reader := bodyReader{r: r} + if err := userBkt.Upload(ctx, dst, reader); err != nil { + // We don't know what caused the error; it could be the client's fault (e.g. killed + // connection), but internal server error is the safe choice here. + level.Error(logger).Log("msg", "failed uploading block file to bucket", "destination", dst, "err", err) + http.Error(w, fmt.Sprintf("internal server error (id %s)", requestID), http.StatusInternalServerError) + return + } + + level.Debug(logger).Log("msg", "finished uploading block file to bucket", "path", pth) + + w.WriteHeader(http.StatusOK) +} + +func (c *MultitenantCompactor) validateAndCompleteBlockUpload(logger log.Logger, tenantID string, userBkt objstore.Bucket, blockID ulid.ULID, meta *block.Meta, validation func(context.Context) error) { + level.Debug(logger).Log("msg", "completing block upload", "files", len(meta.Thanos.Files)) + + { + var wg sync.WaitGroup + ctx, cancel := context.WithCancel(context.Background()) + + // start a go routine that updates the validation file's timestamp every heartbeat interval + wg.Add(1) + go func() { + defer wg.Done() + c.periodicValidationUpdater(ctx, logger, blockID, userBkt, cancel, validationHeartbeatInterval) + }() + + if err := validation(ctx); err != nil { + level.Error(logger).Log("msg", "error while validating block", "err", err) + cancel() + wg.Wait() + err := c.uploadValidationWithError(context.Background(), blockID, userBkt, err.Error()) + if err != nil { + level.Error(logger).Log("msg", "error updating validation file after failed block validation", "err", err) + } + return + } + + cancel() + wg.Wait() // use waitgroup to ensure validation ts update is complete + } + + ctx := context.Background() + + if err := c.markBlockComplete(ctx, logger, tenantID, userBkt, blockID, meta); err != nil { + if err := c.uploadValidationWithError(ctx, blockID, userBkt, err.Error()); err != nil { + level.Error(logger).Log("msg", "error updating validation file after upload of metadata file failed", "err", err) + } + return + } + + if err := userBkt.Delete(ctx, path.Join(blockID.String(), validationFilename)); err != nil { + level.Warn(logger).Log("msg", fmt.Sprintf( + "failed to delete %s from block in object storage", validationFilename), "err", err) + return + } + + level.Info(logger).Log("msg", "successfully completed block upload") +} + +func (c *MultitenantCompactor) markBlockComplete(ctx context.Context, logger log.Logger, tenantID string, userBkt objstore.Bucket, blockID ulid.ULID, meta *block.Meta) error { + if err := c.uploadMeta(ctx, logger, meta, blockID, block.MetaFilename, userBkt); err != nil { + level.Error(logger).Log("msg", "error uploading block metadata file", "err", err) + return err + } + + if err := userBkt.Delete(ctx, path.Join(blockID.String(), uploadingMetaFilename)); err != nil { + // Not returning an error since the temporary meta file persisting is a harmless side effect + level.Warn(logger).Log("msg", fmt.Sprintf("failed to delete %s from block in object storage", uploadingMetaFilename), "err", err) + } + + // Increment metrics on successful block upload + c.blockUploadBlocks.WithLabelValues(tenantID).Inc() + c.blockUploadBytes.WithLabelValues(tenantID).Add(float64(meta.BlockBytes())) + c.blockUploadFiles.WithLabelValues(tenantID).Add(float64(len(meta.Thanos.Files))) + + return nil +} + +// sanitizeMeta sanitizes and validates a metadata.Meta object. If a validation error occurs, an error +// message gets returned, otherwise an empty string. +func (c *MultitenantCompactor) sanitizeMeta(logger log.Logger, userID string, blockID ulid.ULID, meta *block.Meta) string { + if meta == nil { + return "missing block metadata" + } + + // check that the blocks doesn't contain down-sampled data + if meta.Thanos.Downsample.Resolution > 0 { + return "block contains downsampled data" + } + + meta.ULID = blockID + for l, v := range meta.Thanos.Labels { + switch l { + // Preserve this label + case mimir_tsdb.CompactorShardIDExternalLabel: + if v == "" { + level.Debug(logger).Log("msg", "removing empty external label", + "label", l) + delete(meta.Thanos.Labels, l) + continue + } + + if _, _, err := sharding.ParseShardIDLabelValue(v); err != nil { + return fmt.Sprintf("invalid %s external label: %q", + mimir_tsdb.CompactorShardIDExternalLabel, v) + } + // Remove unused labels + case mimir_tsdb.DeprecatedTenantIDExternalLabel, mimir_tsdb.DeprecatedIngesterIDExternalLabel, mimir_tsdb.DeprecatedShardIDExternalLabel: + level.Debug(logger).Log("msg", "removing unused external label", + "label", l, "value", v) + delete(meta.Thanos.Labels, l) + default: + return fmt.Sprintf("unsupported external label: %s", l) + } + } + + meta.Compaction.Parents = nil + meta.Compaction.Sources = []ulid.ULID{blockID} + + for _, f := range meta.Thanos.Files { + if f.RelPath == block.MetaFilename { + continue + } + + if !rePath.MatchString(f.RelPath) { + return fmt.Sprintf("file with invalid path: %s", f.RelPath) + } + + if f.SizeBytes <= 0 { + return fmt.Sprintf("file with invalid size: %s", f.RelPath) + } + } + + if err := c.validateMaximumBlockSize(logger, meta.Thanos.Files, userID); err != nil { + return err.Error() + } + + if meta.Version != block.TSDBVersion1 { + return fmt.Sprintf("version must be %d", block.TSDBVersion1) + } + + // validate minTime/maxTime + // basic sanity check + if meta.MinTime < 0 || meta.MaxTime < 0 || meta.MaxTime < meta.MinTime { + return fmt.Sprintf("invalid minTime/maxTime: minTime=%d, maxTime=%d", + meta.MinTime, meta.MaxTime) + } + // validate that times are in the past + now := time.Now() + if meta.MinTime > now.UnixMilli() || meta.MaxTime > now.UnixMilli() { + return fmt.Sprintf("block time(s) greater than the present: minTime=%d, maxTime=%d", + meta.MinTime, meta.MaxTime) + } + + // Mark block source + meta.Thanos.Source = "upload" + + return "" +} + +func (c *MultitenantCompactor) uploadMeta(ctx context.Context, logger log.Logger, meta *block.Meta, blockID ulid.ULID, name string, userBkt objstore.Bucket) error { + if meta == nil { + return errors.New("missing block metadata") + } + dst := path.Join(blockID.String(), name) + level.Debug(logger).Log("msg", fmt.Sprintf("uploading %s to bucket", name), "dst", dst) + buf := bytes.NewBuffer(nil) + if err := json.NewEncoder(buf).Encode(meta); err != nil { + return errors.Wrap(err, "failed to encode block metadata") + } + if err := userBkt.Upload(ctx, dst, buf); err != nil { + return errors.Wrapf(err, "failed uploading %s to bucket", name) + } + + return nil +} + +func (c *MultitenantCompactor) createTemporaryBlockDirectory() (dir string, err error) { + blockDir, err := os.MkdirTemp(c.compactorCfg.DataDir, "upload") + if err != nil { + level.Error(c.logger).Log("msg", "failed to create temporary block directory", "err", err) + return "", errors.New("failed to create temporary block directory") + } + + level.Debug(c.logger).Log("msg", "created temporary block directory", "dir", blockDir) + return blockDir, nil +} + +func (c *MultitenantCompactor) removeTemporaryBlockDirectory(blockDir string) { + level.Debug(c.logger).Log("msg", "removing temporary block directory", "dir", blockDir) + if err := os.RemoveAll(blockDir); err != nil { + level.Warn(c.logger).Log("msg", "failed to remove temporary block directory", "path", blockDir, "err", err) + } +} + +func (c *MultitenantCompactor) prepareBlockForValidation(ctx context.Context, userBkt objstore.Bucket, blockID ulid.ULID) (string, error) { + blockDir, err := c.createTemporaryBlockDirectory() + if err != nil { + return "", err + } + + // download the block to local storage + level.Debug(c.logger).Log("msg", "downloading block from bucket", "block", blockID.String()) + err = objstore.DownloadDir(ctx, c.logger, userBkt, blockID.String(), blockID.String(), blockDir) + if err != nil { + c.removeTemporaryBlockDirectory(blockDir) + return "", errors.Wrap(err, "failed to download block") + } + + // rename the temporary meta file name to the expected one locally so that the block can be inspected + err = os.Rename(filepath.Join(blockDir, uploadingMetaFilename), filepath.Join(blockDir, block.MetaFilename)) + if err != nil { + level.Warn(c.logger).Log("msg", "could not rename temporary metadata file", "block", blockID.String(), "err", err) + c.removeTemporaryBlockDirectory(blockDir) + return "", errors.New("failed renaming while preparing block for validation") + } + + return blockDir, nil +} + +func (c *MultitenantCompactor) validateBlock(ctx context.Context, logger log.Logger, blockID ulid.ULID, blockMetadata *block.Meta, userBkt objstore.Bucket, userID string) error { + if err := c.validateMaximumBlockSize(logger, blockMetadata.Thanos.Files, userID); err != nil { + return err + } + + blockDir, err := c.prepareBlockForValidation(ctx, userBkt, blockID) + if err != nil { + return err + } + defer c.removeTemporaryBlockDirectory(blockDir) + + // check that all files listed in the metadata are present and the correct size + for _, f := range blockMetadata.Thanos.Files { + fi, err := os.Stat(filepath.Join(blockDir, filepath.FromSlash(f.RelPath))) + if err != nil { + return errors.Wrapf(err, "failed to stat %s", f.RelPath) + } + + if !fi.Mode().IsRegular() { + return errors.Errorf("not a file: %s", f.RelPath) + } + + if f.RelPath != block.MetaFilename && fi.Size() != f.SizeBytes { + return errors.Errorf("file size mismatch for %s", f.RelPath) + } + } + + // validate block + checkChunks := c.cfgProvider.CompactorBlockUploadVerifyChunks(userID) + err = block.VerifyBlock(c.logger, blockDir, blockMetadata.MinTime, blockMetadata.MaxTime, checkChunks) + if err != nil { + return errors.Wrap(err, "error validating block") + } + + return nil +} + +func (c *MultitenantCompactor) validateMaximumBlockSize(logger log.Logger, files []block.File, userID string) error { + maxBlockSizeBytes := c.cfgProvider.CompactorBlockUploadMaxBlockSizeBytes(userID) + if maxBlockSizeBytes <= 0 { + return nil + } + + blockSizeBytes := int64(0) + for _, f := range files { + if f.SizeBytes < 0 { + return errors.New("invalid negative file size in block metadata") + } + blockSizeBytes += f.SizeBytes + if blockSizeBytes < 0 { + // overflow + break + } + } + + if blockSizeBytes > maxBlockSizeBytes || blockSizeBytes < 0 { + level.Error(logger).Log("msg", "rejecting block upload for exceeding maximum size", "limit", maxBlockSizeBytes, "size", blockSizeBytes) + return fmt.Errorf(maxBlockUploadSizeBytesFormat, maxBlockSizeBytes) + } + return nil +} + +type httpError struct { + message string + statusCode int +} + +func (e httpError) Error() string { + return e.message +} + +type bodyReader struct { + r *http.Request +} + +// ObjectSize implements thanos.ObjectSizer. +func (r bodyReader) ObjectSize() (int64, error) { + if r.r.ContentLength < 0 { + return 0, fmt.Errorf("unknown size") + } + + return r.r.ContentLength, nil +} + +// Read implements io.Reader. +func (r bodyReader) Read(b []byte) (int, error) { + return r.r.Body.Read(b) +} + +type validationFile struct { + LastUpdate int64 // UnixMillis of last update time. + Error string // Error message if validation failed. +} + +type blockUploadStateResult struct { + State string `json:"result"` + Error string `json:"error,omitempty"` +} + +type blockUploadState int + +const ( + blockStateUnknown blockUploadState = iota // unknown, default value + blockIsComplete // meta.json file exists + blockUploadNotStarted // meta.json doesn't exist, uploading-meta.json doesn't exist + blockUploadInProgress // meta.json doesn't exist, but uploading-meta.json does + blockValidationInProgress // meta.json doesn't exist, uploading-meta.json exists, validation.json exists and is recent + blockValidationFailed + blockValidationStale +) + +func (c *MultitenantCompactor) GetBlockUploadStateHandler(w http.ResponseWriter, r *http.Request) { + blockID, tenantID, err := c.parseBlockUploadParameters(r) + if err != nil { + http.Error(w, err.Error(), http.StatusBadRequest) + return + } + requestID := hexTimeNowNano() + logger := log.With( + util_log.WithContext(r.Context(), c.logger), + "feature", "block upload", + "block", blockID, + "operation", "get block state", + "request_id", requestID, + ) + + userBkt := bucket.NewUserBucketClient(tenantID, c.bucketClient, c.cfgProvider) + if err != nil { + http.Error(w, err.Error(), http.StatusBadRequest) + return + } + + s, _, v, err := c.getBlockUploadState(r.Context(), userBkt, blockID) + if err != nil { + writeBlockUploadError(err, "can't get upload state", logger, w, requestID) + return + } + + res := blockUploadStateResult{} + + switch s { + case blockIsComplete: + res.State = "complete" + case blockUploadNotStarted: + http.Error(w, "block doesn't exist", http.StatusNotFound) + return + case blockValidationStale: + fallthrough + case blockUploadInProgress: + res.State = "uploading" + case blockValidationInProgress: + res.State = "validating" + case blockValidationFailed: + res.State = "failed" + res.Error = v.Error + } + + util.WriteJSONResponse(w, res) +} + +// checkBlockState checks blocks state and returns various HTTP status codes for individual states if block +// upload cannot start, finish or file cannot be uploaded to the block. +func (c *MultitenantCompactor) checkBlockState(ctx context.Context, userBkt objstore.Bucket, blockID ulid.ULID, requireUploadInProgress bool) (*block.Meta, *validationFile, error) { + s, m, v, err := c.getBlockUploadState(ctx, userBkt, blockID) + if err != nil { + return m, v, err + } + + switch s { + case blockIsComplete: + return m, v, httpError{message: "block already exists", statusCode: http.StatusConflict} + case blockValidationInProgress: + return m, v, httpError{message: "block validation in progress", statusCode: http.StatusBadRequest} + case blockUploadNotStarted: + if requireUploadInProgress { + return m, v, httpError{message: "block upload not started", statusCode: http.StatusNotFound} + } + return m, v, nil + case blockValidationStale: + // if validation is stale, we treat block as being in "upload in progress" state, and validation can start again. + fallthrough + case blockUploadInProgress: + return m, v, nil + case blockValidationFailed: + return m, v, httpError{message: "block validation failed", statusCode: http.StatusBadRequest} + } + + return m, v, httpError{message: "unknown block upload state", statusCode: http.StatusInternalServerError} +} + +// getBlockUploadState returns state of the block upload, and meta and validation objects, if they exist. +func (c *MultitenantCompactor) getBlockUploadState(ctx context.Context, userBkt objstore.Bucket, blockID ulid.ULID) (blockUploadState, *block.Meta, *validationFile, error) { + exists, err := userBkt.Exists(ctx, path.Join(blockID.String(), block.MetaFilename)) + if err != nil { + return blockStateUnknown, nil, nil, err + } + if exists { + return blockIsComplete, nil, nil, nil + } + + meta, err := c.loadUploadingMeta(ctx, userBkt, blockID) + if err != nil { + return blockStateUnknown, nil, nil, err + } + // If neither meta.json nor uploading-meta.json file exist, we say that the block doesn't exist. + if meta == nil { + return blockUploadNotStarted, nil, nil, err + } + + v, err := c.loadValidation(ctx, userBkt, blockID) + if err != nil { + return blockStateUnknown, meta, nil, err + } + if v == nil { + return blockUploadInProgress, meta, nil, err + } + if v.Error != "" { + return blockValidationFailed, meta, v, err + } + if time.Since(time.UnixMilli(v.LastUpdate)) < validationHeartbeatTimeout { + return blockValidationInProgress, meta, v, nil + } + return blockValidationStale, meta, v, nil +} + +func (c *MultitenantCompactor) loadUploadingMeta(ctx context.Context, userBkt objstore.Bucket, blockID ulid.ULID) (*block.Meta, error) { + r, err := userBkt.Get(ctx, path.Join(blockID.String(), uploadingMetaFilename)) + if err != nil { + if userBkt.IsObjNotFoundErr(err) { + return nil, nil + } + return nil, err + } + defer func() { _ = r.Close() }() + + v := &block.Meta{} + err = json.NewDecoder(r).Decode(v) + if err != nil { + return nil, err + } + + return v, nil +} + +func (c *MultitenantCompactor) loadValidation(ctx context.Context, userBkt objstore.Bucket, blockID ulid.ULID) (*validationFile, error) { + r, err := userBkt.Get(ctx, path.Join(blockID.String(), validationFilename)) + if err != nil { + if userBkt.IsObjNotFoundErr(err) { + return nil, nil + } + return nil, err + } + defer func() { _ = r.Close() }() + + v := &validationFile{} + err = json.NewDecoder(r).Decode(v) + if err != nil { + return nil, err + } + + return v, nil +} + +func (c *MultitenantCompactor) uploadValidationWithError(ctx context.Context, blockID ulid.ULID, + userBkt objstore.Bucket, errorStr string, +) error { + val := validationFile{ + LastUpdate: time.Now().UnixMilli(), + Error: errorStr, + } + dst := path.Join(blockID.String(), validationFilename) + if err := marshalAndUploadToBucket(ctx, userBkt, dst, val); err != nil { + return errors.Wrapf(err, "failed uploading %s to bucket", validationFilename) + } + return nil +} + +func (c *MultitenantCompactor) uploadValidation(ctx context.Context, blockID ulid.ULID, userBkt objstore.Bucket) error { + return c.uploadValidationWithError(ctx, blockID, userBkt, "") +} + +func (c *MultitenantCompactor) periodicValidationUpdater(ctx context.Context, logger log.Logger, blockID ulid.ULID, userBkt objstore.Bucket, cancelFn func(), interval time.Duration) { + ticker := time.NewTicker(interval) + defer ticker.Stop() + for { + select { + case <-ctx.Done(): + return + case <-ticker.C: + if err := c.uploadValidation(ctx, blockID, userBkt); err != nil { + level.Warn(logger).Log("msg", "error during periodic update of validation file", "err", err) + cancelFn() + return + } + } + } +} + +func marshalAndUploadToBucket(ctx context.Context, bkt objstore.Bucket, pth string, val interface{}) error { + buf, err := json.Marshal(val) + if err != nil { + return err + } + if err := bkt.Upload(ctx, pth, bytes.NewReader(buf)); err != nil { + return err + } + return nil +} diff --git a/pkg/compactor/block_upload_test.go b/pkg/compactor/block_upload_test.go new file mode 100644 index 0000000000..b6395e4e06 --- /dev/null +++ b/pkg/compactor/block_upload_test.go @@ -0,0 +1,2116 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/grafana/mimir/blob/main/pkg/compactor/block_upload_test.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Cortex Authors. + +package compactor + +import ( + "bytes" + "context" + "encoding/json" + "fmt" + "io" + "math" + "net/http" + "net/http/httptest" + "net/url" + "os" + "path" + "path/filepath" + "strings" + "sync" + "testing" + "time" + + "github.com/go-kit/log" + "github.com/gorilla/mux" + "github.com/grafana/dskit/test" + "github.com/grafana/dskit/user" + "github.com/oklog/ulid" + "github.com/pkg/errors" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" + promtest "github.com/prometheus/client_golang/prometheus/testutil" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/tsdb" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/mock" + "github.com/stretchr/testify/require" + "github.com/thanos-io/objstore" + + "github.com/grafana/mimir/pkg/storage/bucket" + mimir_tsdb "github.com/grafana/mimir/pkg/storage/tsdb" + "github.com/grafana/mimir/pkg/storage/tsdb/block" +) + +func verifyUploadedMeta(t *testing.T, bkt *bucket.ClientMock, expMeta block.Meta) { + var call mock.Call + for _, c := range bkt.Calls { + if c.Method == "Upload" { + call = c + break + } + } + + rdr := call.Arguments[2].(io.Reader) + var gotMeta block.Meta + require.NoError(t, json.NewDecoder(rdr).Decode(&gotMeta)) + assert.Equal(t, expMeta, gotMeta) +} + +// Test MultitenantCompactor.StartBlockUpload +func TestMultitenantCompactor_StartBlockUpload(t *testing.T) { + const tenantID = "test" + const blockID = "01G3FZ0JWJYJC0ZM6Y9778P6KD" + bULID := ulid.MustParse(blockID) + now := time.Now().UnixMilli() + validMeta := block.Meta{ + BlockMeta: tsdb.BlockMeta{ + ULID: bULID, + Version: block.TSDBVersion1, + MinTime: now - 1000, + MaxTime: now, + }, + Thanos: block.ThanosMeta{ + Labels: map[string]string{ + mimir_tsdb.CompactorShardIDExternalLabel: "1_of_3", + }, + Files: []block.File{ + { + RelPath: block.MetaFilename, + }, + { + RelPath: "index", + SizeBytes: 1, + }, + { + RelPath: "chunks/000001", + SizeBytes: 1024, + }, + }, + }, + } + + metaPath := path.Join(tenantID, blockID, block.MetaFilename) + uploadingMetaPath := path.Join(tenantID, blockID, fmt.Sprintf("uploading-%s", block.MetaFilename)) + + setUpPartialBlock := func(bkt *bucket.ClientMock) { + bkt.MockExists(path.Join(tenantID, blockID, block.MetaFilename), false, nil) + setUpGet(bkt, path.Join(tenantID, blockID, uploadingMetaFilename), nil, bucket.ErrObjectDoesNotExist) + } + setUpUpload := func(bkt *bucket.ClientMock) { + setUpPartialBlock(bkt) + bkt.MockUpload(uploadingMetaPath, nil) + } + + verifyUpload := func(t *testing.T, bkt *bucket.ClientMock, labels map[string]string) { + t.Helper() + + expMeta := validMeta + expMeta.Compaction.Parents = nil + expMeta.Compaction.Sources = []ulid.ULID{expMeta.ULID} + expMeta.Thanos.Source = "upload" + expMeta.Thanos.Labels = labels + verifyUploadedMeta(t, bkt, expMeta) + } + + testCases := []struct { + name string + tenantID string + blockID string + body string + meta *block.Meta + retention time.Duration + disableBlockUpload bool + expBadRequest string + expConflict string + expUnprocessableEntity string + expEntityTooLarge string + expInternalServerError bool + setUpBucketMock func(bkt *bucket.ClientMock) + verifyUpload func(*testing.T, *bucket.ClientMock) + maxBlockUploadSizeBytes int64 + }{ + { + name: "missing tenant ID", + tenantID: "", + blockID: blockID, + expBadRequest: "invalid tenant ID", + }, + { + name: "missing block ID", + tenantID: tenantID, + blockID: "", + expBadRequest: "invalid block ID", + }, + { + name: "invalid block ID", + tenantID: tenantID, + blockID: "1234", + expBadRequest: "invalid block ID", + }, + { + name: "missing body", + tenantID: tenantID, + blockID: blockID, + expBadRequest: "malformed request body", + setUpBucketMock: setUpPartialBlock, + }, + { + name: "malformed body", + tenantID: tenantID, + blockID: blockID, + setUpBucketMock: setUpPartialBlock, + body: "{", + expBadRequest: "malformed request body", + }, + { + name: "invalid file path", + tenantID: tenantID, + blockID: blockID, + setUpBucketMock: setUpPartialBlock, + meta: &block.Meta{ + Thanos: block.ThanosMeta{ + Files: []block.File{ + { + RelPath: "chunks/invalid-file", + SizeBytes: 1024, + }, + }, + }, + }, + expBadRequest: "file with invalid path: chunks/invalid-file", + }, + { + name: "contains downsampled data", + tenantID: tenantID, + blockID: blockID, + setUpBucketMock: setUpPartialBlock, + meta: &block.Meta{ + Thanos: block.ThanosMeta{ + Downsample: block.ThanosDownsample{ + Resolution: 1000, + }, + Files: []block.File{ + { + RelPath: block.MetaFilename, + }, + { + RelPath: "index", + SizeBytes: 1, + }, + { + RelPath: "chunks/000001", + }, + }, + }, + }, + expBadRequest: "block contains downsampled data", + }, + { + name: "missing file size", + tenantID: tenantID, + blockID: blockID, + setUpBucketMock: setUpPartialBlock, + meta: &block.Meta{ + Thanos: block.ThanosMeta{ + Files: []block.File{ + { + RelPath: block.MetaFilename, + }, + { + RelPath: "index", + SizeBytes: 1, + }, + { + RelPath: "chunks/000001", + }, + }, + }, + }, + expBadRequest: "file with invalid size: chunks/000001", + }, + { + name: "invalid minTime", + tenantID: tenantID, + blockID: blockID, + setUpBucketMock: setUpPartialBlock, + meta: &block.Meta{ + BlockMeta: tsdb.BlockMeta{ + ULID: bULID, + Version: block.TSDBVersion1, + MinTime: -1, + MaxTime: 0, + }, + }, + expBadRequest: "invalid minTime/maxTime: minTime=-1, maxTime=0", + }, + { + name: "invalid maxTime", + tenantID: tenantID, + blockID: blockID, + setUpBucketMock: setUpPartialBlock, + meta: &block.Meta{ + BlockMeta: tsdb.BlockMeta{ + ULID: bULID, + Version: block.TSDBVersion1, + MinTime: 0, + MaxTime: -1, + }, + }, + expBadRequest: "invalid minTime/maxTime: minTime=0, maxTime=-1", + }, + { + name: "maxTime before minTime", + tenantID: tenantID, + blockID: blockID, + setUpBucketMock: setUpPartialBlock, + meta: &block.Meta{ + BlockMeta: tsdb.BlockMeta{ + ULID: bULID, + Version: block.TSDBVersion1, + MinTime: 1, + MaxTime: 0, + }, + }, + expBadRequest: "invalid minTime/maxTime: minTime=1, maxTime=0", + }, + { + name: "block before retention period", + tenantID: tenantID, + blockID: blockID, + retention: 10 * time.Second, + setUpBucketMock: setUpPartialBlock, + meta: &block.Meta{ + BlockMeta: tsdb.BlockMeta{ + ULID: bULID, + Version: block.TSDBVersion1, + MinTime: 0, + MaxTime: 1000, + }, + }, + expUnprocessableEntity: "block max time (1970-01-01 00:00:01 +0000 UTC) older than retention period", + }, + { + name: "invalid version", + tenantID: tenantID, + blockID: blockID, + setUpBucketMock: setUpPartialBlock, + meta: &block.Meta{ + BlockMeta: tsdb.BlockMeta{ + ULID: bULID, + Version: 0, + }, + }, + expBadRequest: fmt.Sprintf("version must be %d", block.TSDBVersion1), + }, + { + name: "ignore retention period if == 0", + tenantID: tenantID, + blockID: blockID, + retention: 0, + setUpBucketMock: setUpUpload, + meta: &block.Meta{ + BlockMeta: tsdb.BlockMeta{ + ULID: bULID, + Version: block.TSDBVersion1, + MinTime: 0, + MaxTime: 1000, + }, + Thanos: block.ThanosMeta{ + Labels: map[string]string{ + mimir_tsdb.CompactorShardIDExternalLabel: "1_of_3", + }, + Files: []block.File{ + { + RelPath: block.MetaFilename, + }, + { + RelPath: "index", + SizeBytes: 1, + }, + { + RelPath: "chunks/000001", + SizeBytes: 1024, + }, + }, + }, + }, + }, + { + name: "ignore retention period if < 0", + tenantID: tenantID, + blockID: blockID, + retention: -1, + setUpBucketMock: setUpUpload, + meta: &block.Meta{ + BlockMeta: tsdb.BlockMeta{ + ULID: bULID, + Version: block.TSDBVersion1, + MinTime: 0, + MaxTime: 1000, + }, + Thanos: block.ThanosMeta{ + Labels: map[string]string{ + mimir_tsdb.CompactorShardIDExternalLabel: "1_of_3", + }, + Files: []block.File{ + { + RelPath: block.MetaFilename, + }, + { + RelPath: "index", + SizeBytes: 1, + }, + { + RelPath: "chunks/000001", + SizeBytes: 1024, + }, + }, + }, + }, + }, + { + name: "invalid compactor shard ID label", + tenantID: tenantID, + blockID: blockID, + setUpBucketMock: setUpPartialBlock, + meta: &block.Meta{ + BlockMeta: tsdb.BlockMeta{ + ULID: bULID, + Version: block.TSDBVersion1, + }, + Thanos: block.ThanosMeta{ + Labels: map[string]string{ + mimir_tsdb.CompactorShardIDExternalLabel: "test", + }, + }, + }, + expBadRequest: fmt.Sprintf(`invalid %s external label: "test"`, mimir_tsdb.CompactorShardIDExternalLabel), + }, + { + name: "failure checking for complete block", + tenantID: tenantID, + blockID: blockID, + setUpBucketMock: func(bkt *bucket.ClientMock) { + bkt.MockExists(path.Join(tenantID, blockID, block.MetaFilename), false, fmt.Errorf("test")) + }, + expInternalServerError: true, + }, + { + name: "complete block already exists", + tenantID: tenantID, + blockID: blockID, + setUpBucketMock: func(bkt *bucket.ClientMock) { + bkt.MockExists(path.Join(tenantID, blockID, block.MetaFilename), true, nil) + }, + expConflict: "block already exists", + }, + { + name: "failure uploading meta file", + tenantID: tenantID, + blockID: blockID, + setUpBucketMock: func(bkt *bucket.ClientMock) { + setUpPartialBlock(bkt) + bkt.MockUpload(uploadingMetaPath, fmt.Errorf("test")) + }, + meta: &validMeta, + expInternalServerError: true, + }, + { + name: "too large of a request body", + tenantID: tenantID, + blockID: blockID, + setUpBucketMock: setUpPartialBlock, + body: strings.Repeat("A", maximumMetaSizeBytes+1), + expEntityTooLarge: fmt.Sprintf("The block metadata was too large (maximum size allowed is %d bytes)", maximumMetaSizeBytes), + }, + { + name: "block upload disabled", + tenantID: tenantID, + blockID: blockID, + disableBlockUpload: true, + expBadRequest: "block upload is disabled", + }, + { + name: "max block size exceeded", + tenantID: tenantID, + blockID: blockID, + setUpBucketMock: setUpPartialBlock, + meta: &validMeta, + maxBlockUploadSizeBytes: 1, + expBadRequest: fmt.Sprintf(maxBlockUploadSizeBytesFormat, 1), + }, + { + name: "valid request", + tenantID: tenantID, + blockID: blockID, + setUpBucketMock: setUpUpload, + meta: &validMeta, + verifyUpload: func(t *testing.T, bkt *bucket.ClientMock) { + verifyUpload(t, bkt, map[string]string{ + mimir_tsdb.CompactorShardIDExternalLabel: "1_of_3", + }) + }, + }, + { + name: "valid request with empty compactor shard ID label", + tenantID: tenantID, + blockID: blockID, + setUpBucketMock: setUpUpload, + meta: &block.Meta{ + BlockMeta: tsdb.BlockMeta{ + ULID: bULID, + Version: block.TSDBVersion1, + MinTime: now - 1000, + MaxTime: now, + }, + Thanos: block.ThanosMeta{ + Labels: map[string]string{ + mimir_tsdb.CompactorShardIDExternalLabel: "", + }, + Files: []block.File{ + { + RelPath: block.MetaFilename, + }, + { + RelPath: "index", + SizeBytes: 1, + }, + { + RelPath: "chunks/000001", + SizeBytes: 1024, + }, + }, + }, + }, + verifyUpload: func(t *testing.T, bkt *bucket.ClientMock) { + verifyUpload(t, bkt, map[string]string{}) + }, + }, + { + name: "valid request without compactor shard ID label", + tenantID: tenantID, + blockID: blockID, + setUpBucketMock: setUpUpload, + meta: &block.Meta{ + BlockMeta: tsdb.BlockMeta{ + ULID: bULID, + Version: block.TSDBVersion1, + MinTime: now - 1000, + MaxTime: now, + }, + Thanos: block.ThanosMeta{ + Files: []block.File{ + { + RelPath: block.MetaFilename, + }, + { + RelPath: "index", + SizeBytes: 1, + }, + { + RelPath: "chunks/000001", + SizeBytes: 1024, + }, + }, + }, + }, + verifyUpload: func(t *testing.T, bkt *bucket.ClientMock) { + verifyUpload(t, bkt, nil) + }, + }, + { + name: "valid request with different block ID in meta file", + tenantID: tenantID, + blockID: blockID, + setUpBucketMock: setUpUpload, + meta: &block.Meta{ + BlockMeta: tsdb.BlockMeta{ + ULID: ulid.MustParse("11A2FZ0JWJYJC0ZM6Y9778P6KD"), + Version: block.TSDBVersion1, + MinTime: now - 1000, + MaxTime: now, + }, + Thanos: block.ThanosMeta{ + Files: []block.File{ + { + RelPath: block.MetaFilename, + }, + { + RelPath: "index", + SizeBytes: 1, + }, + { + RelPath: "chunks/000001", + SizeBytes: 1024, + }, + }, + }, + }, + verifyUpload: func(t *testing.T, bkt *bucket.ClientMock) { + verifyUpload(t, bkt, nil) + }, + }, + } + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + var bkt bucket.ClientMock + if tc.setUpBucketMock != nil { + tc.setUpBucketMock(&bkt) + } + + cfgProvider := newMockConfigProvider() + cfgProvider.userRetentionPeriods[tenantID] = tc.retention + cfgProvider.blockUploadEnabled[tenantID] = !tc.disableBlockUpload + cfgProvider.blockUploadMaxBlockSizeBytes[tenantID] = tc.maxBlockUploadSizeBytes + c := &MultitenantCompactor{ + logger: log.NewNopLogger(), + bucketClient: &bkt, + cfgProvider: cfgProvider, + } + var rdr io.Reader + if tc.body != "" { + rdr = strings.NewReader(tc.body) + } else if tc.meta != nil { + buf := bytes.NewBuffer(nil) + require.NoError(t, json.NewEncoder(buf).Encode(tc.meta)) + rdr = buf + } + r := httptest.NewRequest(http.MethodPost, fmt.Sprintf("/api/v1/upload/block/%s/start", tc.blockID), rdr) + if tc.tenantID != "" { + r = r.WithContext(user.InjectOrgID(r.Context(), tc.tenantID)) + } + if tc.blockID != "" { + r = mux.SetURLVars(r, map[string]string{"block": tc.blockID}) + } + w := httptest.NewRecorder() + c.StartBlockUpload(w, r) + + resp := w.Result() + body, err := io.ReadAll(resp.Body) + require.NoError(t, err) + + switch { + case tc.expInternalServerError: + assert.Equal(t, http.StatusInternalServerError, resp.StatusCode) + assert.Regexp(t, "internal server error \\(id [0-9a-f]{16}\\)\n", string(body)) + case tc.expBadRequest != "": + assert.Equal(t, http.StatusBadRequest, resp.StatusCode) + assert.Equal(t, fmt.Sprintf("%s\n", tc.expBadRequest), string(body)) + case tc.expConflict != "": + assert.Equal(t, http.StatusConflict, resp.StatusCode) + assert.Equal(t, fmt.Sprintf("%s\n", tc.expConflict), string(body)) + case tc.expUnprocessableEntity != "": + assert.Equal(t, http.StatusUnprocessableEntity, resp.StatusCode) + assert.Equal(t, fmt.Sprintf("%s\n", tc.expUnprocessableEntity), string(body)) + case tc.expEntityTooLarge != "": + assert.Equal(t, http.StatusRequestEntityTooLarge, resp.StatusCode) + assert.Equal(t, fmt.Sprintf("%s\n", tc.expEntityTooLarge), string(body)) + default: + assert.Equal(t, http.StatusOK, resp.StatusCode) + assert.Empty(t, string(body)) + } + + bkt.AssertExpectations(t) + + if tc.verifyUpload != nil { + tc.verifyUpload(t, &bkt) + } + }) + } + + downloadMeta := func(t *testing.T, bkt *objstore.InMemBucket, pth string) block.Meta { + t.Helper() + + ctx := context.Background() + rdr, err := bkt.Get(ctx, pth) + require.NoError(t, err) + t.Cleanup(func() { + _ = rdr.Close() + }) + var gotMeta block.Meta + require.NoError(t, json.NewDecoder(rdr).Decode(&gotMeta)) + return gotMeta + } + + // Additional test cases using an in-memory bucket for state testing + extraCases := []struct { + name string + setUp func(*testing.T, *objstore.InMemBucket) block.Meta + verifyBucket func(*testing.T, *objstore.InMemBucket) + expBadRequest string + expConflict string + }{ + { + name: "valid request when both in-flight meta file and complete meta file exist in object storage", + setUp: func(t *testing.T, bkt *objstore.InMemBucket) block.Meta { + marshalAndUploadJSON(t, bkt, uploadingMetaPath, validMeta) + marshalAndUploadJSON(t, bkt, metaPath, validMeta) + return validMeta + }, + verifyBucket: func(t *testing.T, bkt *objstore.InMemBucket) { + assert.Equal(t, validMeta, downloadMeta(t, bkt, uploadingMetaPath)) + assert.Equal(t, validMeta, downloadMeta(t, bkt, metaPath)) + }, + expConflict: "block already exists", + }, + { + name: "invalid request when in-flight meta file exists in object storage", + setUp: func(t *testing.T, bkt *objstore.InMemBucket) block.Meta { + marshalAndUploadJSON(t, bkt, uploadingMetaPath, validMeta) + + meta := validMeta + // Invalid version + meta.Version = 0 + return meta + }, + verifyBucket: func(t *testing.T, bkt *objstore.InMemBucket) { + assert.Equal(t, validMeta, downloadMeta(t, bkt, uploadingMetaPath)) + }, + expBadRequest: fmt.Sprintf("version must be %d", block.TSDBVersion1), + }, + { + name: "valid request when same in-flight meta file exists in object storage", + setUp: func(t *testing.T, bkt *objstore.InMemBucket) block.Meta { + marshalAndUploadJSON(t, bkt, uploadingMetaPath, validMeta) + return validMeta + }, + verifyBucket: func(t *testing.T, bkt *objstore.InMemBucket) { + expMeta := validMeta + expMeta.Compaction.Sources = []ulid.ULID{expMeta.ULID} + expMeta.Thanos.Source = "upload" + assert.Equal(t, expMeta, downloadMeta(t, bkt, uploadingMetaPath)) + }, + }, + { + name: "valid request when different in-flight meta file exists in object storage", + setUp: func(t *testing.T, bkt *objstore.InMemBucket) block.Meta { + meta := validMeta + meta.MinTime -= 1000 + meta.MaxTime -= 1000 + marshalAndUploadJSON(t, bkt, uploadingMetaPath, meta) + + // Return meta file that differs from the one in bucket + return validMeta + }, + verifyBucket: func(t *testing.T, bkt *objstore.InMemBucket) { + expMeta := validMeta + expMeta.Compaction.Sources = []ulid.ULID{expMeta.ULID} + expMeta.Thanos.Source = "upload" + assert.Equal(t, expMeta, downloadMeta(t, bkt, uploadingMetaPath)) + }, + }, + } + for _, tc := range extraCases { + t.Run(tc.name, func(t *testing.T) { + bkt := objstore.NewInMemBucket() + meta := tc.setUp(t, bkt) + metaJSON, err := json.Marshal(meta) + require.NoError(t, err) + + cfgProvider := newMockConfigProvider() + cfgProvider.blockUploadEnabled[tenantID] = true + c := &MultitenantCompactor{ + logger: log.NewNopLogger(), + bucketClient: bkt, + cfgProvider: cfgProvider, + } + r := httptest.NewRequest(http.MethodPost, fmt.Sprintf("/api/v1/upload/block/%s/start", blockID), bytes.NewReader(metaJSON)) + r = r.WithContext(user.InjectOrgID(r.Context(), tenantID)) + r = mux.SetURLVars(r, map[string]string{"block": blockID}) + w := httptest.NewRecorder() + c.StartBlockUpload(w, r) + + resp := w.Result() + body, err := io.ReadAll(resp.Body) + require.NoError(t, err) + switch { + case tc.expBadRequest != "": + assert.Equal(t, http.StatusBadRequest, resp.StatusCode) + assert.Equal(t, fmt.Sprintf("%s\n", tc.expBadRequest), string(body)) + case tc.expConflict != "": + assert.Equal(t, http.StatusConflict, resp.StatusCode) + assert.Equal(t, fmt.Sprintf("%s\n", tc.expConflict), string(body)) + default: + assert.Equal(t, http.StatusOK, resp.StatusCode) + assert.Empty(t, string(body)) + } + }) + } +} + +// Test MultitenantCompactor.UploadBlockFile +func TestMultitenantCompactor_UploadBlockFile(t *testing.T) { + const tenantID = "test" + const blockID = "01G3FZ0JWJYJC0ZM6Y9778P6KD" + uploadingMetaFilename := fmt.Sprintf("uploading-%s", block.MetaFilename) + uploadingMetaPath := path.Join(tenantID, blockID, uploadingMetaFilename) + metaPath := path.Join(tenantID, blockID, block.MetaFilename) + + chunkBodyContent := "content" + validMeta := block.Meta{ + BlockMeta: tsdb.BlockMeta{ + ULID: ulid.MustParse(blockID), + }, + Thanos: block.ThanosMeta{ + Labels: map[string]string{ + mimir_tsdb.CompactorShardIDExternalLabel: "1_of_3", + }, + Files: []block.File{ + { + RelPath: "index", + SizeBytes: 1, + }, + { + RelPath: "chunks/000001", + SizeBytes: int64(len(chunkBodyContent)), + }, + }, + }, + } + + setupFnForValidRequest := func(bkt *bucket.ClientMock) { + bkt.MockExists(metaPath, false, nil) + + b, err := json.Marshal(validMeta) + setUpGet(bkt, path.Join(tenantID, blockID, uploadingMetaFilename), b, err) + setUpGet(bkt, path.Join(tenantID, blockID, validationFilename), nil, bucket.ErrObjectDoesNotExist) + + bkt.MockUpload(path.Join(tenantID, blockID, "chunks/000001"), nil) + } + + verifyFuncForValidRequest := func(t *testing.T, bkt *bucket.ClientMock, expContent string) { + var call mock.Call + for _, c := range bkt.Calls { + if c.Method == "Upload" { + call = c + break + } + } + + rdr := call.Arguments[2].(io.Reader) + got, err := io.ReadAll(rdr) + require.NoError(t, err) + assert.Equal(t, []byte(expContent), got) + } + + testCases := []struct { + name string + tenantID string + blockID string + path string + body string + unknownContentLength bool + disableBlockUpload bool + expBadRequest string + expConflict string + expNotFound string + expInternalServerError bool + setUpBucketMock func(bkt *bucket.ClientMock) + verifyUpload func(*testing.T, *bucket.ClientMock, string) + }{ + { + name: "without tenant ID", + blockID: blockID, + path: "chunks/000001", + expBadRequest: "invalid tenant ID", + }, + { + name: "without block ID", + tenantID: tenantID, + path: "chunks/000001", + expBadRequest: "invalid block ID", + }, + { + name: "invalid block ID", + tenantID: tenantID, + blockID: "1234", + path: "chunks/000001", + expBadRequest: "invalid block ID", + }, + { + name: "without path", + tenantID: tenantID, + blockID: blockID, + expBadRequest: "missing or invalid file path", + }, + { + name: "invalid path", + tenantID: tenantID, + blockID: blockID, + path: "../chunks/000001", + expBadRequest: `invalid path: "../chunks/000001"`, + }, + { + name: "empty file", + tenantID: tenantID, + blockID: blockID, + path: "chunks/000001", + expBadRequest: "file cannot be empty", + }, + { + name: "attempt block metadata file", + tenantID: tenantID, + blockID: blockID, + path: block.MetaFilename, + body: "content", + expBadRequest: fmt.Sprintf("%s is not allowed", block.MetaFilename), + }, + { + name: "attempt in-flight block metadata file", + tenantID: tenantID, + blockID: blockID, + path: uploadingMetaFilename, + body: "content", + expBadRequest: fmt.Sprintf("invalid path: %q", uploadingMetaFilename), + }, + { + name: "block upload disabled", + tenantID: tenantID, + blockID: blockID, + disableBlockUpload: true, + path: "chunks/000001", + expBadRequest: "block upload is disabled", + }, + { + name: "complete block already exists", + tenantID: tenantID, + blockID: blockID, + path: "chunks/000001", + body: "content", + setUpBucketMock: func(bkt *bucket.ClientMock) { + bkt.MockExists(metaPath, true, nil) + }, + expConflict: "block already exists", + }, + { + name: "failure checking for complete block", + tenantID: tenantID, + blockID: blockID, + path: "chunks/000001", + body: chunkBodyContent, + setUpBucketMock: func(bkt *bucket.ClientMock) { + bkt.MockExists(metaPath, false, fmt.Errorf("test")) + }, + expInternalServerError: true, + }, + { + name: "failure checking for in-flight meta file", + tenantID: tenantID, + blockID: blockID, + path: "chunks/000001", + body: chunkBodyContent, + setUpBucketMock: func(bkt *bucket.ClientMock) { + bkt.MockExists(metaPath, false, nil) + setUpGet(bkt, path.Join(tenantID, blockID, uploadingMetaFilename), nil, fmt.Errorf("test")) + }, + expInternalServerError: true, + }, + { + name: "missing in-flight meta file", + tenantID: tenantID, + blockID: blockID, + path: "chunks/000001", + body: chunkBodyContent, + setUpBucketMock: func(bkt *bucket.ClientMock) { + bkt.MockExists(metaPath, false, nil) + setUpGet(bkt, path.Join(tenantID, blockID, uploadingMetaFilename), nil, bucket.ErrObjectDoesNotExist) + }, + expNotFound: "block upload not started", + }, + { + name: "file upload fails", + tenantID: tenantID, + blockID: blockID, + path: "chunks/000001", + body: chunkBodyContent, + setUpBucketMock: func(bkt *bucket.ClientMock) { + bkt.MockExists(metaPath, false, nil) + + b, err := json.Marshal(validMeta) + setUpGet(bkt, path.Join(tenantID, blockID, uploadingMetaFilename), b, err) + setUpGet(bkt, path.Join(tenantID, blockID, validationFilename), nil, bucket.ErrObjectDoesNotExist) + + bkt.MockUpload(path.Join(tenantID, blockID, "chunks/000001"), fmt.Errorf("test")) + }, + expInternalServerError: true, + }, + { + name: "invalid file size", + tenantID: tenantID, + blockID: blockID, + path: "chunks/000001", + body: chunkBodyContent + chunkBodyContent, + setUpBucketMock: func(bkt *bucket.ClientMock) { + bkt.MockExists(metaPath, false, nil) + + b, err := json.Marshal(validMeta) + setUpGet(bkt, path.Join(tenantID, blockID, uploadingMetaFilename), b, err) + setUpGet(bkt, path.Join(tenantID, blockID, validationFilename), nil, bucket.ErrObjectDoesNotExist) + }, + expBadRequest: "file size doesn't match meta.json", + }, + { + name: "unexpected file", + tenantID: tenantID, + blockID: blockID, + path: "chunks/111111", + body: chunkBodyContent, + setUpBucketMock: func(bkt *bucket.ClientMock) { + bkt.MockExists(metaPath, false, nil) + + b, err := json.Marshal(validMeta) + setUpGet(bkt, path.Join(tenantID, blockID, uploadingMetaFilename), b, err) + setUpGet(bkt, path.Join(tenantID, blockID, validationFilename), nil, bucket.ErrObjectDoesNotExist) + }, + expBadRequest: "unexpected file", + }, + { + name: "valid request", + tenantID: tenantID, + blockID: blockID, + path: "chunks/000001", + body: chunkBodyContent, + setUpBucketMock: setupFnForValidRequest, + verifyUpload: verifyFuncForValidRequest, + }, + { + name: "valid request, with unknown content-length", + tenantID: tenantID, + blockID: blockID, + path: "chunks/000001", + body: chunkBodyContent, + unknownContentLength: true, + setUpBucketMock: setupFnForValidRequest, + verifyUpload: verifyFuncForValidRequest, + }, + } + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + var bkt bucket.ClientMock + if tc.setUpBucketMock != nil { + tc.setUpBucketMock(&bkt) + } + + cfgProvider := newMockConfigProvider() + cfgProvider.blockUploadEnabled[tc.tenantID] = !tc.disableBlockUpload + c := &MultitenantCompactor{ + logger: log.NewNopLogger(), + bucketClient: &bkt, + cfgProvider: cfgProvider, + } + var rdr io.Reader + if tc.body != "" { + rdr = strings.NewReader(tc.body) + } + r := httptest.NewRequest(http.MethodPost, fmt.Sprintf("/api/v1/upload/block/%s/files?path=%s", blockID, url.QueryEscape(tc.path)), rdr) + if tc.tenantID != "" { + r = r.WithContext(user.InjectOrgID(r.Context(), tenantID)) + } + if tc.blockID != "" { + r = mux.SetURLVars(r, map[string]string{"block": tc.blockID}) + } + if tc.body != "" { + r.ContentLength = int64(len(tc.body)) + if tc.unknownContentLength { + r.ContentLength = -1 + } + } + w := httptest.NewRecorder() + c.UploadBlockFile(w, r) + + resp := w.Result() + body, err := io.ReadAll(resp.Body) + require.NoError(t, err) + + switch { + case tc.expBadRequest != "": + assert.Equal(t, http.StatusBadRequest, resp.StatusCode) + assert.Equal(t, fmt.Sprintf("%s\n", tc.expBadRequest), string(body)) + case tc.expConflict != "": + assert.Equal(t, http.StatusConflict, resp.StatusCode) + assert.Equal(t, fmt.Sprintf("%s\n", tc.expConflict), string(body)) + case tc.expNotFound != "": + assert.Equal(t, http.StatusNotFound, resp.StatusCode) + assert.Equal(t, fmt.Sprintf("%s\n", tc.expNotFound), string(body)) + case tc.expInternalServerError: + assert.Equal(t, http.StatusInternalServerError, resp.StatusCode) + assert.Regexp(t, "internal server error \\(id [0-9a-f]{16}\\)\n", string(body)) + default: + assert.Equal(t, http.StatusOK, resp.StatusCode) + assert.Empty(t, string(body)) + } + + bkt.AssertExpectations(t) + + if tc.verifyUpload != nil { + tc.verifyUpload(t, &bkt, tc.body) + } + }) + } + + type file struct { + path string + content string + } + + // Additional test cases using an in-memory bucket for state testing + extraCases := []struct { + name string + files []file + setUpBucket func(*testing.T, *objstore.InMemBucket) + verifyBucket func(*testing.T, *objstore.InMemBucket, []file) + }{ + { + name: "multiple sequential uploads of same file", + files: []file{ + { + path: "chunks/000001", + content: strings.Repeat("a", len(chunkBodyContent)), + }, + { + path: "chunks/000001", + content: strings.Repeat("b", len(chunkBodyContent)), + }, + }, + setUpBucket: func(t *testing.T, bkt *objstore.InMemBucket) { + marshalAndUploadJSON(t, bkt, uploadingMetaPath, validMeta) + }, + verifyBucket: func(t *testing.T, bkt *objstore.InMemBucket, files []file) { + t.Helper() + + ctx := context.Background() + rdr, err := bkt.Get(ctx, path.Join(tenantID, blockID, files[1].path)) + require.NoError(t, err) + t.Cleanup(func() { + _ = rdr.Close() + }) + + content, err := io.ReadAll(rdr) + require.NoError(t, err) + assert.Equal(t, files[1].content, string(content)) + }, + }, + } + for _, tc := range extraCases { + t.Run(tc.name, func(t *testing.T) { + bkt := objstore.NewInMemBucket() + tc.setUpBucket(t, bkt) + cfgProvider := newMockConfigProvider() + cfgProvider.blockUploadEnabled[tenantID] = true + c := &MultitenantCompactor{ + logger: log.NewNopLogger(), + bucketClient: bkt, + cfgProvider: cfgProvider, + } + + for _, f := range tc.files { + rdr := strings.NewReader(f.content) + r := httptest.NewRequest(http.MethodPost, fmt.Sprintf("/api/v1/upload/block/%s/files?path=%s", blockID, url.QueryEscape(f.path)), rdr) + urlVars := map[string]string{ + "block": blockID, + } + r = mux.SetURLVars(r, urlVars) + r = r.WithContext(user.InjectOrgID(r.Context(), tenantID)) + w := httptest.NewRecorder() + c.UploadBlockFile(w, r) + + resp := w.Result() + body, err := io.ReadAll(resp.Body) + require.NoError(t, err) + require.Equal(t, http.StatusOK, resp.StatusCode) + require.Empty(t, body) + } + + tc.verifyBucket(t, bkt, tc.files) + }) + } +} + +func setUpGet(bkt *bucket.ClientMock, pth string, content []byte, err error) { + bkt.On("Get", mock.Anything, pth).Return(func(_ context.Context, _ string) (io.ReadCloser, error) { + return io.NopCloser(bytes.NewReader(content)), err + }) +} + +// Test MultitenantCompactor.FinishBlockUpload +func TestMultitenantCompactor_FinishBlockUpload(t *testing.T) { + const tenantID = "test" + const blockID = "01G3FZ0JWJYJC0ZM6Y9778P6KD" + uploadingMetaPath := path.Join(tenantID, blockID, uploadingMetaFilename) + metaPath := path.Join(tenantID, blockID, block.MetaFilename) + injectedError := fmt.Errorf("injected error") + validMeta := block.Meta{ + BlockMeta: tsdb.BlockMeta{ + Version: block.TSDBVersion1, + ULID: ulid.MustParse(blockID), + }, + Thanos: block.ThanosMeta{ + Labels: map[string]string{ + mimir_tsdb.CompactorShardIDExternalLabel: "1_of_3", + }, + Files: []block.File{ + { + RelPath: "index", + SizeBytes: 1, + }, + { + RelPath: "chunks/000001", + SizeBytes: 2, + }, + }, + }, + } + + validSetup := func(t *testing.T, bkt objstore.Bucket) { + err := marshalAndUploadToBucket(context.Background(), bkt, uploadingMetaPath, validMeta) + require.NoError(t, err) + for _, file := range validMeta.Thanos.Files { + content := bytes.NewReader(make([]byte, file.SizeBytes)) + err = bkt.Upload(context.Background(), path.Join(tenantID, blockID, file.RelPath), content) + require.NoError(t, err) + } + } + + testCases := []struct { + name string + tenantID string + blockID string + setUpBucket func(*testing.T, objstore.Bucket) + errorInjector func(op bucket.Operation, name string) error + disableBlockUpload bool + enableValidation bool // should only be set to true for tests that fail before validation is started + maxConcurrency int + setConcurrency int64 + expBadRequest string + expConflict string + expNotFound string + expTooManyRequests bool + expInternalServerError bool + }{ + { + name: "without tenant ID", + blockID: blockID, + expBadRequest: "invalid tenant ID", + }, + { + name: "without block ID", + tenantID: tenantID, + expBadRequest: "invalid block ID", + }, + { + name: "invalid block ID", + tenantID: tenantID, + blockID: "1234", + expBadRequest: "invalid block ID", + }, + { + name: "block upload disabled", + tenantID: tenantID, + blockID: blockID, + disableBlockUpload: true, + expBadRequest: "block upload is disabled", + }, + { + name: "complete block already exists", + tenantID: tenantID, + blockID: blockID, + setUpBucket: func(t *testing.T, bkt objstore.Bucket) { + err := marshalAndUploadToBucket(context.Background(), bkt, metaPath, validMeta) + require.NoError(t, err) + }, + expConflict: "block already exists", + }, + { + name: "checking for complete block fails", + tenantID: tenantID, + blockID: blockID, + errorInjector: bucket.InjectErrorOn(bucket.OpExists, metaPath, injectedError), + expInternalServerError: true, + }, + { + name: "missing in-flight meta file", + tenantID: tenantID, + blockID: blockID, + expNotFound: "block upload not started", + }, + { + name: "downloading in-flight meta file fails", + tenantID: tenantID, + blockID: blockID, + setUpBucket: func(t *testing.T, bkt objstore.Bucket) { + err := marshalAndUploadToBucket(context.Background(), bkt, uploadingMetaPath, validMeta) + require.NoError(t, err) + }, + errorInjector: bucket.InjectErrorOn(bucket.OpGet, uploadingMetaPath, injectedError), + expInternalServerError: true, + }, + { + name: "corrupt in-flight meta file", + tenantID: tenantID, + blockID: blockID, + setUpBucket: func(t *testing.T, bkt objstore.Bucket) { + err := bkt.Upload(context.Background(), uploadingMetaPath, bytes.NewReader([]byte("{"))) + require.NoError(t, err) + }, + expInternalServerError: true, + }, + { + name: "uploading meta file fails", + tenantID: tenantID, + blockID: blockID, + setUpBucket: validSetup, + errorInjector: bucket.InjectErrorOn(bucket.OpUpload, metaPath, injectedError), + expInternalServerError: true, + }, + { + name: "too many concurrent validations", + tenantID: tenantID, + blockID: blockID, + setUpBucket: validSetup, + enableValidation: true, + maxConcurrency: 2, + setConcurrency: 2, + expTooManyRequests: true, + }, + } + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + bkt := objstore.NewInMemBucket() + injectedBkt := bucket.ErrorInjectedBucketClient{ + Bucket: bkt, + Injector: tc.errorInjector, + } + if tc.setUpBucket != nil { + tc.setUpBucket(t, bkt) + } + + cfgProvider := newMockConfigProvider() + cfgProvider.blockUploadEnabled[tc.tenantID] = !tc.disableBlockUpload + cfgProvider.blockUploadValidationEnabled[tc.tenantID] = tc.enableValidation + c := &MultitenantCompactor{ + logger: log.NewNopLogger(), + bucketClient: &injectedBkt, + cfgProvider: cfgProvider, + } + c.compactorCfg.MaxBlockUploadValidationConcurrency = tc.maxConcurrency + if tc.setConcurrency > 0 { + c.blockUploadValidations.Add(tc.setConcurrency) + } + + c.compactorCfg.DataDir = t.TempDir() + + r := httptest.NewRequest(http.MethodPost, fmt.Sprintf("/api/v1/upload/block/%s/finish", tc.blockID), nil) + if tc.tenantID != "" { + r = r.WithContext(user.InjectOrgID(r.Context(), tenantID)) + } + if tc.blockID != "" { + r = mux.SetURLVars(r, map[string]string{"block": tc.blockID}) + } + w := httptest.NewRecorder() + c.FinishBlockUpload(w, r) + + resp := w.Result() + body, err := io.ReadAll(resp.Body) + require.NoError(t, err) + + switch { + case tc.expBadRequest != "": + assert.Equal(t, http.StatusBadRequest, resp.StatusCode) + assert.Equal(t, fmt.Sprintf("%s\n", tc.expBadRequest), string(body)) + case tc.expConflict != "": + assert.Equal(t, http.StatusConflict, resp.StatusCode) + assert.Equal(t, fmt.Sprintf("%s\n", tc.expConflict), string(body)) + case tc.expNotFound != "": + assert.Equal(t, http.StatusNotFound, resp.StatusCode) + assert.Equal(t, fmt.Sprintf("%s\n", tc.expNotFound), string(body)) + case tc.expInternalServerError: + assert.Equal(t, http.StatusInternalServerError, resp.StatusCode) + assert.Regexp(t, "internal server error \\(id [0-9a-f]{16}\\)\n", string(body)) + case tc.expTooManyRequests: + assert.Equal(t, http.StatusTooManyRequests, resp.StatusCode) + assert.Equal(t, "too many block upload validations in progress, limit is 2\n", string(body)) + default: + assert.Equal(t, http.StatusOK, resp.StatusCode) + assert.Empty(t, string(body)) + exists, err := bkt.Exists(context.Background(), path.Join(tc.blockID, block.MetaFilename)) + require.NoError(t, err) + require.True(t, exists) + } + }) + } +} + +func TestMultitenantCompactor_ValidateAndComplete(t *testing.T) { + const tenantID = "test" + const blockID = "01G3FZ0JWJYJC0ZM6Y9778P6KD" + injectedError := fmt.Errorf("injected error") + + uploadingMetaPath := path.Join(tenantID, blockID, uploadingMetaFilename) + validationPath := path.Join(tenantID, blockID, validationFilename) + metaPath := path.Join(tenantID, blockID, block.MetaFilename) + + validationSucceeds := func(_ context.Context) error { return nil } + + testCases := []struct { + name string + errorInjector func(op bucket.Operation, name string) error + validation func(context.Context) error + expectValidationFile bool + expectErrorInValidationFile bool + expectTempUploadingMeta bool + expectMeta bool + }{ + { + name: "validation fails", + validation: func(_ context.Context) error { return injectedError }, + expectValidationFile: true, + expectErrorInValidationFile: true, + expectTempUploadingMeta: true, + expectMeta: false, + }, + { + name: "validation fails, uploading error fails", + errorInjector: bucket.InjectErrorOn(bucket.OpUpload, validationPath, injectedError), + validation: func(_ context.Context) error { return injectedError }, + expectValidationFile: true, + expectErrorInValidationFile: false, + expectTempUploadingMeta: true, + expectMeta: false, + }, + { + name: "uploading meta file fails", + errorInjector: bucket.InjectErrorOn(bucket.OpUpload, metaPath, injectedError), + validation: validationSucceeds, + expectValidationFile: true, + expectErrorInValidationFile: true, + expectTempUploadingMeta: true, + expectMeta: false, + }, + { + name: "uploading meta file fails, uploading error fails", + errorInjector: func(op bucket.Operation, target string) error { + if op == bucket.OpUpload && (target == metaPath || target == validationPath) { + return injectedError + } + return nil + }, + validation: validationSucceeds, + expectValidationFile: true, + expectErrorInValidationFile: false, + expectTempUploadingMeta: true, + expectMeta: false, + }, + { + name: "removing in-flight meta file fails", + errorInjector: bucket.InjectErrorOn(bucket.OpDelete, uploadingMetaPath, injectedError), + validation: validationSucceeds, + expectValidationFile: false, + expectTempUploadingMeta: true, + expectMeta: true, + }, + { + name: "removing validation file fails", + errorInjector: bucket.InjectErrorOn(bucket.OpDelete, validationPath, injectedError), + validation: validationSucceeds, + expectValidationFile: true, + expectErrorInValidationFile: false, + expectTempUploadingMeta: false, + expectMeta: true, + }, + { + name: "valid request", + validation: validationSucceeds, + expectValidationFile: false, + expectTempUploadingMeta: false, + expectMeta: true, + }, + } + + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + bkt := objstore.NewInMemBucket() + var injectedBkt objstore.Bucket = bkt + if tc.errorInjector != nil { + injectedBkt = &bucket.ErrorInjectedBucketClient{ + Bucket: bkt, + Injector: tc.errorInjector, + } + } + cfgProvider := newMockConfigProvider() + c := &MultitenantCompactor{ + logger: log.NewNopLogger(), + bucketClient: injectedBkt, + cfgProvider: cfgProvider, + blockUploadBlocks: promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{tenantID}), + blockUploadBytes: promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{tenantID}), + blockUploadFiles: promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{tenantID}), + } + userBkt := bucket.NewUserBucketClient(tenantID, injectedBkt, cfgProvider) + + meta := block.Meta{} + marshalAndUploadJSON(t, bkt, uploadingMetaPath, meta) + v := validationFile{} + marshalAndUploadJSON(t, bkt, validationPath, v) + + c.validateAndCompleteBlockUpload(log.NewNopLogger(), tenantID, userBkt, ulid.MustParse(blockID), &meta, tc.validation) + + tempUploadingMetaExists, err := bkt.Exists(context.Background(), uploadingMetaPath) + require.NoError(t, err) + require.Equal(t, tempUploadingMetaExists, tc.expectTempUploadingMeta) + + metaExists, err := bkt.Exists(context.Background(), metaPath) + require.NoError(t, err) + require.Equal(t, metaExists, tc.expectMeta) + + if !tc.expectValidationFile { + exists, err := bkt.Exists(context.Background(), validationPath) + require.NoError(t, err) + require.False(t, exists) + return + } + + r, err := bkt.Get(context.Background(), validationPath) + require.NoError(t, err) + decoder := json.NewDecoder(r) + err = decoder.Decode(&v) + require.NoError(t, err) + + if tc.expectErrorInValidationFile { + require.NotEmpty(t, v.Error) + } else { + require.Empty(t, v.Error) + } + }) + } +} + +func TestMultitenantCompactor_ValidateBlock(t *testing.T) { + const tenantID = "test" + ctx := context.Background() + tmpDir := t.TempDir() + bkt := objstore.NewInMemBucket() + + type Missing uint8 + const ( + MissingMeta Missing = 1 << iota + MissingIndex + MissingChunks + ) + + validLabels := func() []labels.Labels { + return []labels.Labels{ + labels.FromStrings("a", "1"), + labels.FromStrings("b", "2"), + labels.FromStrings("c", "3"), + } + } + + testCases := []struct { + name string + lbls func() []labels.Labels + metaInject func(meta *block.Meta) + indexInject func(fname string) + chunkInject func(fname string) + populateFileList bool + maximumBlockSize int64 + verifyChunks bool + missing Missing + expectError bool + expectedMsg string + }{ + { + name: "valid block", + lbls: validLabels, + verifyChunks: true, + expectError: false, + populateFileList: true, + }, + { + name: "maximum block size exceeded", + lbls: validLabels, + populateFileList: true, + maximumBlockSize: 1, + expectError: true, + expectedMsg: fmt.Sprintf(maxBlockUploadSizeBytesFormat, 1), + }, + { + name: "missing meta file", + lbls: validLabels, + missing: MissingMeta, + expectError: true, + expectedMsg: "failed renaming while preparing block for validation", + }, + { + name: "missing index file", + lbls: validLabels, + missing: MissingIndex, + expectError: true, + expectedMsg: "error validating block: open index file:", + }, + { + name: "missing chunks file", + lbls: validLabels, + populateFileList: true, + missing: MissingChunks, + expectError: true, + expectedMsg: "failed to stat chunks/", + }, + { + name: "file size mismatch", + lbls: validLabels, + metaInject: func(meta *block.Meta) { + require.Greater(t, len(meta.Thanos.Files), 0) + meta.Thanos.Files[0].SizeBytes += 10 + }, + populateFileList: true, + expectError: true, + expectedMsg: "file size mismatch", + }, + { + name: "empty index file", + lbls: validLabels, + indexInject: func(fname string) { + require.NoError(t, os.Truncate(fname, 0)) + }, + expectError: true, + expectedMsg: "error validating block: open index file: mmap, size 0: invalid argument", + }, + { + name: "index file invalid magic number", + lbls: validLabels, + indexInject: func(fname string) { + flipByteAt(t, fname, 0) // guaranteed to be a magic number byte + }, + expectError: true, + expectedMsg: "error validating block: open index file: invalid magic number", + }, + { + name: "out of order labels", + lbls: func() []labels.Labels { + b := labels.NewScratchBuilder(2) + b.Add("d", "4") + b.Add("a", "1") + oooLabels := []labels.Labels{ + b.Labels(), // Haven't called Sort(), so they will be out of order. + labels.FromStrings("b", "2"), + labels.FromStrings("c", "3"), + } + return oooLabels + }, + expectError: true, + expectedMsg: "error validating block: index contains 1 postings with out of order labels", + }, + { + name: "segment file invalid magic number", + lbls: validLabels, + chunkInject: func(fname string) { + flipByteAt(t, fname, 0) // guaranteed to be a magic number byte + }, + verifyChunks: true, + expectError: true, + expectedMsg: "invalid magic number", + }, + { + name: "segment file invalid checksum", + lbls: validLabels, + chunkInject: func(fname string) { + flipByteAt(t, fname, 12) // guaranteed to be a data byte + }, + populateFileList: true, + verifyChunks: true, + expectError: true, + expectedMsg: "checksum mismatch", + }, + { + name: "empty segment file", + lbls: validLabels, + chunkInject: func(fname string) { + require.NoError(t, os.Truncate(fname, 0)) + }, + verifyChunks: true, + expectError: true, + expectedMsg: "size 0: invalid argument", + }, + } + + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + // create a test block + now := time.Now() + blockID, err := block.CreateBlock(ctx, tmpDir, tc.lbls(), 300, now.Add(-2*time.Hour).UnixMilli(), now.UnixMilli(), labels.EmptyLabels()) + require.NoError(t, err) + testDir := filepath.Join(tmpDir, blockID.String()) + meta, err := block.ReadMetaFromDir(testDir) + require.NoError(t, err) + if tc.populateFileList { + stats, err := block.GatherFileStats(testDir) + require.NoError(t, err) + meta.Thanos.Files = stats + } + + // create a compactor + cfgProvider := newMockConfigProvider() + cfgProvider.blockUploadValidationEnabled[tenantID] = true + cfgProvider.verifyChunks[tenantID] = tc.verifyChunks + cfgProvider.blockUploadMaxBlockSizeBytes[tenantID] = tc.maximumBlockSize + c := &MultitenantCompactor{ + logger: log.NewNopLogger(), + bucketClient: bkt, + cfgProvider: cfgProvider, + } + + // upload the block + require.NoError(t, block.Upload(ctx, log.NewNopLogger(), bkt, testDir, nil)) + // remove meta.json as we will be uploading a new one with the uploading meta name + require.NoError(t, bkt.Delete(ctx, path.Join(blockID.String(), block.MetaFilename))) + + // handle meta file + if tc.metaInject != nil { + tc.metaInject(meta) + } + var metaBody bytes.Buffer + require.NoError(t, meta.Write(&metaBody)) + + // replace index file + if tc.indexInject != nil { + indexFile := filepath.Join(testDir, block.IndexFilename) + indexObject := path.Join(blockID.String(), block.IndexFilename) + require.NoError(t, bkt.Delete(ctx, indexObject)) + tc.indexInject(indexFile) + uploadLocalFileToBucket(ctx, t, bkt, indexFile, indexObject) + } + + // replace segment file + if tc.chunkInject != nil { + segmentFile := filepath.Join(testDir, block.ChunksDirname, "000001") + segmentObject := path.Join(blockID.String(), block.ChunksDirname, "000001") + require.NoError(t, bkt.Delete(ctx, segmentObject)) + tc.chunkInject(segmentFile) + uploadLocalFileToBucket(ctx, t, bkt, segmentFile, segmentObject) + } + + // delete any files that should be missing + if tc.missing&MissingIndex != 0 { + require.NoError(t, bkt.Delete(ctx, path.Join(blockID.String(), block.IndexFilename))) + } + + if tc.missing&MissingChunks != 0 { + chunkDir := path.Join(blockID.String(), block.ChunksDirname) + err := bkt.Iter(ctx, chunkDir, func(name string) error { + require.NoError(t, bkt.Delete(ctx, name)) + return nil + }) + require.NoError(t, err) + } + + // only upload renamed meta file if it is not meant to be missing + if tc.missing&MissingMeta == 0 { + // rename to uploading meta file as that is what validateBlock expects + require.NoError(t, bkt.Upload(ctx, path.Join(blockID.String(), uploadingMetaFilename), &metaBody)) + } + + // validate the block + err = c.validateBlock(ctx, c.logger, blockID, meta, bkt, tenantID) + if tc.expectError { + require.Error(t, err) + require.Contains(t, err.Error(), tc.expectedMsg) + } else { + require.NoError(t, err) + } + }) + } +} + +func TestMultitenantCompactor_PeriodicValidationUpdater(t *testing.T) { + const tenantID = "test" + const blockID = "01G3FZ0JWJYJC0ZM6Y9778P6KD" + injectedError := fmt.Errorf("injected error") + validationPath := path.Join(tenantID, blockID, validationFilename) + + heartbeatInterval := 50 * time.Millisecond + + validationExists := func(t *testing.T, bkt objstore.Bucket) bool { + exists, err := bkt.Exists(context.Background(), validationPath) + require.NoError(t, err) + return exists + } + + testCases := []struct { + name string + errorInjector func(op bucket.Operation, name string) error + cancelContext bool + assertions func(t *testing.T, ctx context.Context, bkt objstore.Bucket) + }{ + { + name: "updating validation file fails", + errorInjector: bucket.InjectErrorOn(bucket.OpUpload, validationPath, injectedError), + assertions: func(t *testing.T, ctx context.Context, bkt objstore.Bucket) { + <-ctx.Done() + require.True(t, errors.Is(context.Canceled, ctx.Err())) + require.False(t, validationExists(t, bkt)) + }, + }, + { + name: "updating validation file succeeds", + assertions: func(t *testing.T, ctx context.Context, bkt objstore.Bucket) { + test.Poll(t, heartbeatInterval*2, true, func() interface{} { + return validationExists(t, bkt) + }) + + v := validationFile{} + r, err := bkt.Get(context.Background(), validationPath) + require.NoError(t, err) + decoder := json.NewDecoder(r) + err = decoder.Decode(&v) + require.NoError(t, err) + require.NotEqual(t, 0, v.LastUpdate) + require.Empty(t, v.Error) + }, + }, + { + name: "context cancelled before update", + cancelContext: true, + assertions: func(t *testing.T, ctx context.Context, bkt objstore.Bucket) { + require.False(t, validationExists(t, bkt)) + }, + }, + } + + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + bkt := objstore.NewInMemBucket() + var injectedBkt objstore.Bucket = bkt + if tc.errorInjector != nil { + injectedBkt = &bucket.ErrorInjectedBucketClient{ + Bucket: bkt, + Injector: tc.errorInjector, + } + } + + cfgProvider := newMockConfigProvider() + c := &MultitenantCompactor{ + logger: log.NewNopLogger(), + bucketClient: injectedBkt, + cfgProvider: cfgProvider, + } + userBkt := bucket.NewUserBucketClient(tenantID, injectedBkt, cfgProvider) + ctx, cancel := context.WithCancel(context.Background()) + + heartbeatInterval := heartbeatInterval + if tc.cancelContext { + cancel() + heartbeatInterval = 1 * time.Hour // to avoid racing a heartbeat + } + + var wg sync.WaitGroup + wg.Add(1) + go func() { + defer wg.Done() + c.periodicValidationUpdater(ctx, log.NewNopLogger(), ulid.MustParse(blockID), userBkt, cancel, heartbeatInterval) + }() + + if !tc.cancelContext { + time.Sleep(heartbeatInterval) + } + + tc.assertions(t, ctx, bkt) + + cancel() + wg.Wait() + }) + } +} + +func TestMultitenantCompactor_GetBlockUploadStateHandler(t *testing.T) { + const ( + tenantID = "tenant" + blockID = "01G8X9GA8R6N8F75FW1J18G83N" + ) + + type testcase struct { + setupBucket func(t *testing.T, bkt objstore.Bucket) + disableBlockUpload bool + expectedStatusCode int + expectedBody string + } + + for name, tc := range map[string]testcase{ + "block doesn't exist": { + expectedStatusCode: http.StatusNotFound, + expectedBody: "block doesn't exist", + }, + + "complete block": { + setupBucket: func(t *testing.T, bkt objstore.Bucket) { + marshalAndUploadJSON(t, bkt, path.Join(tenantID, blockID, block.MetaFilename), block.Meta{}) + }, + expectedStatusCode: http.StatusOK, + expectedBody: `{"result":"complete"}`, + }, + + "upload in progress": { + setupBucket: func(t *testing.T, bkt objstore.Bucket) { + marshalAndUploadJSON(t, bkt, path.Join(tenantID, blockID, uploadingMetaFilename), block.Meta{}) + }, + expectedStatusCode: http.StatusOK, + expectedBody: `{"result":"uploading"}`, + }, + + "validating": { + setupBucket: func(t *testing.T, bkt objstore.Bucket) { + marshalAndUploadJSON(t, bkt, path.Join(tenantID, blockID, uploadingMetaFilename), block.Meta{}) + marshalAndUploadJSON(t, bkt, path.Join(tenantID, blockID, validationFilename), validationFile{LastUpdate: time.Now().UnixMilli()}) + }, + expectedStatusCode: http.StatusOK, + expectedBody: `{"result":"validating"}`, + }, + + "validation failed": { + setupBucket: func(t *testing.T, bkt objstore.Bucket) { + marshalAndUploadJSON(t, bkt, path.Join(tenantID, blockID, uploadingMetaFilename), block.Meta{}) + marshalAndUploadJSON(t, bkt, path.Join(tenantID, blockID, validationFilename), validationFile{LastUpdate: time.Now().UnixMilli(), Error: "error during validation"}) + }, + expectedStatusCode: http.StatusOK, + expectedBody: `{"result":"failed","error":"error during validation"}`, + }, + + "stale validation file": { + setupBucket: func(t *testing.T, bkt objstore.Bucket) { + marshalAndUploadJSON(t, bkt, path.Join(tenantID, blockID, uploadingMetaFilename), block.Meta{}) + marshalAndUploadJSON(t, bkt, path.Join(tenantID, blockID, validationFilename), validationFile{LastUpdate: time.Now().Add(-10 * time.Minute).UnixMilli()}) + }, + expectedStatusCode: http.StatusOK, + expectedBody: `{"result":"uploading"}`, + }, + } { + t.Run(name, func(t *testing.T) { + bkt := objstore.NewInMemBucket() + if tc.setupBucket != nil { + tc.setupBucket(t, bkt) + } + + cfgProvider := newMockConfigProvider() + cfgProvider.blockUploadEnabled[tenantID] = !tc.disableBlockUpload + + c := &MultitenantCompactor{ + logger: log.NewNopLogger(), + bucketClient: bkt, + cfgProvider: cfgProvider, + } + + r := httptest.NewRequest(http.MethodGet, fmt.Sprintf("/api/v1/upload/block/%s/check", blockID), nil) + urlVars := map[string]string{"block": blockID} + r = mux.SetURLVars(r, urlVars) + r = r.WithContext(user.InjectOrgID(r.Context(), tenantID)) + + w := httptest.NewRecorder() + c.GetBlockUploadStateHandler(w, r) + resp := w.Result() + + body, err := io.ReadAll(resp.Body) + + require.NoError(t, err) + require.Equal(t, tc.expectedStatusCode, resp.StatusCode) + require.Equal(t, tc.expectedBody, strings.TrimSpace(string(body))) + }) + } +} + +func TestMultitenantCompactor_ValidateMaximumBlockSize(t *testing.T) { + const userID = "user" + + type testCase struct { + maximumBlockSize int64 + fileSizes []int64 + expectErr bool + } + + for name, tc := range map[string]testCase{ + "no limit": { + maximumBlockSize: 0, + fileSizes: []int64{math.MaxInt64}, + expectErr: false, + }, + "under limit": { + maximumBlockSize: 4, + fileSizes: []int64{1, 2}, + expectErr: false, + }, + "under limit - zero size file included": { + maximumBlockSize: 2, + fileSizes: []int64{1, 0}, + expectErr: false, + }, + "under limit - negative size file included": { + maximumBlockSize: 2, + fileSizes: []int64{2, -1}, + expectErr: true, + }, + "exact limit": { + maximumBlockSize: 3, + fileSizes: []int64{1, 2}, + expectErr: false, + }, + "over limit": { + maximumBlockSize: 1, + fileSizes: []int64{1, 1}, + expectErr: true, + }, + "overflow": { + maximumBlockSize: math.MaxInt64, + fileSizes: []int64{math.MaxInt64, math.MaxInt64, math.MaxInt64}, + expectErr: true, + }, + } { + t.Run(name, func(t *testing.T) { + files := make([]block.File, len(tc.fileSizes)) + for i, size := range tc.fileSizes { + files[i] = block.File{SizeBytes: size} + } + + cfgProvider := newMockConfigProvider() + cfgProvider.blockUploadMaxBlockSizeBytes[userID] = tc.maximumBlockSize + c := &MultitenantCompactor{ + logger: log.NewNopLogger(), + cfgProvider: cfgProvider, + } + + err := c.validateMaximumBlockSize(c.logger, files, userID) + if tc.expectErr { + require.Error(t, err) + } else { + require.NoError(t, err) + } + }) + } +} + +func TestMultitenantCompactor_MarkBlockComplete(t *testing.T) { + const tenantID = "test" + const blockID = "01G3FZ0JWJYJC0ZM6Y9778P6KD" + injectedError := fmt.Errorf("injected error") + + uploadingMetaPath := path.Join(tenantID, blockID, uploadingMetaFilename) + metaPath := path.Join(tenantID, blockID, block.MetaFilename) + testCases := []struct { + name string + errorInjector func(op bucket.Operation, name string) error + expectSuccess bool + }{ + { + name: "marking block complete succeeds", + expectSuccess: true, + }, + { + name: "uploading meta file fails", + errorInjector: bucket.InjectErrorOn(bucket.OpUpload, metaPath, injectedError), + }, + { + name: "deleting uploading meta file fails", + errorInjector: bucket.InjectErrorOn(bucket.OpDelete, uploadingMetaPath, injectedError), + expectSuccess: true, + }, + } + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + bkt := objstore.NewInMemBucket() + var injectedBkt objstore.Bucket = bkt + if tc.errorInjector != nil { + injectedBkt = &bucket.ErrorInjectedBucketClient{ + Bucket: bkt, + Injector: tc.errorInjector, + } + } + cfgProvider := newMockConfigProvider() + c := &MultitenantCompactor{ + logger: log.NewNopLogger(), + bucketClient: injectedBkt, + cfgProvider: cfgProvider, + blockUploadBlocks: promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{tenantID}), + blockUploadBytes: promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{tenantID}), + blockUploadFiles: promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{tenantID}), + } + userBkt := bucket.NewUserBucketClient(tenantID, injectedBkt, cfgProvider) + + meta := block.Meta{ + Thanos: block.ThanosMeta{ + Files: []block.File{ + { + RelPath: "chunks/000001", + SizeBytes: 42, + }, + { + RelPath: "index", + SizeBytes: 17, + }, + { + RelPath: "meta.json", + }, + }, + }, + } + marshalAndUploadJSON(t, bkt, uploadingMetaPath, meta) + + ctx := context.Background() + err := c.markBlockComplete(ctx, log.NewNopLogger(), tenantID, userBkt, ulid.MustParse(blockID), &meta) + if tc.expectSuccess { + require.NoError(t, err) + assert.Equal(t, 1.0, promtest.ToFloat64(c.blockUploadBlocks.WithLabelValues(tenantID))) + assert.Equal(t, 59.0, promtest.ToFloat64(c.blockUploadBytes.WithLabelValues(tenantID))) + assert.Equal(t, 3.0, promtest.ToFloat64(c.blockUploadFiles.WithLabelValues(tenantID))) + } else { + require.Error(t, err) + assert.Equal(t, 0.0, promtest.ToFloat64(c.blockUploadBlocks.WithLabelValues(tenantID))) + assert.Equal(t, 0.0, promtest.ToFloat64(c.blockUploadBytes.WithLabelValues(tenantID))) + assert.Equal(t, 0.0, promtest.ToFloat64(c.blockUploadFiles.WithLabelValues(tenantID))) + } + }) + } +} + +// marshalAndUploadJSON is a test helper for uploading a meta file to a certain path in a bucket. +func marshalAndUploadJSON(t *testing.T, bkt objstore.Bucket, pth string, val interface{}) { + t.Helper() + err := marshalAndUploadToBucket(context.Background(), bkt, pth, val) + require.NoError(t, err) +} + +func uploadLocalFileToBucket(ctx context.Context, t *testing.T, bkt objstore.Bucket, src, dst string) { + t.Helper() + fd, err := os.Open(src) + require.NoError(t, err) + defer func(fd *os.File) { + err := fd.Close() + require.NoError(t, err) + }(fd) + require.NoError(t, bkt.Upload(ctx, dst, fd)) +} + +// flipByteAt flips a byte at a given offset in a file. +func flipByteAt(t *testing.T, fname string, offset int64) { + fd, err := os.OpenFile(fname, os.O_RDWR, 0o644) + require.NoError(t, err) + defer func(fd *os.File) { + err := fd.Close() + require.NoError(t, err) + }(fd) + var b [1]byte + _, err = fd.ReadAt(b[:], offset) + require.NoError(t, err) + // alter the byte + b[0] = 0xff - b[0] + _, err = fd.WriteAt(b[:], offset) + require.NoError(t, err) +} + +func TestHexTimeNowNano(t *testing.T) { + v := hexTimeNowNano() + require.Len(t, v, 16, "Should have exactly 16 characters") + + require.NotEqual(t, strings.Repeat("0", 16), v, "Should not be all zeros") + time.Sleep(time.Nanosecond) + require.NotEqual(t, v, hexTimeNowNano(), "Should generate a different one.") +} diff --git a/pkg/compactor/blocks_cleaner.go b/pkg/compactor/blocks_cleaner.go new file mode 100644 index 0000000000..806a5e4170 --- /dev/null +++ b/pkg/compactor/blocks_cleaner.go @@ -0,0 +1,639 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/grafana/mimir/blob/main/pkg/compactor/blocks_cleaner.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Cortex Authors. + +package compactor + +import ( + "context" + "fmt" + "strconv" + "strings" + "sync" + "time" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/grafana/dskit/concurrency" + "github.com/grafana/dskit/services" + "github.com/oklog/ulid" + "github.com/pkg/errors" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" + "github.com/thanos-io/objstore" + + "github.com/grafana/mimir/pkg/storage/bucket" + mimir_tsdb "github.com/grafana/mimir/pkg/storage/tsdb" + "github.com/grafana/mimir/pkg/storage/tsdb/block" + "github.com/grafana/mimir/pkg/storage/tsdb/bucketindex" + "github.com/grafana/mimir/pkg/util" + util_log "github.com/grafana/mimir/pkg/util/log" + "github.com/grafana/mimir/pkg/util/validation" +) + +const ( + defaultDeleteBlocksConcurrency = 16 +) + +type BlocksCleanerConfig struct { + DeletionDelay time.Duration + CleanupInterval time.Duration + CleanupConcurrency int + TenantCleanupDelay time.Duration // Delay before removing tenant deletion mark and "debug". + DeleteBlocksConcurrency int + NoBlocksFileCleanupEnabled bool +} + +type BlocksCleaner struct { + services.Service + + cfg BlocksCleanerConfig + cfgProvider ConfigProvider + logger log.Logger + bucketClient objstore.Bucket + usersScanner *mimir_tsdb.UsersScanner + ownUser func(userID string) (bool, error) + singleFlight *concurrency.LimitedConcurrencySingleFlight + + // Keep track of the last owned users. + lastOwnedUsers []string + + // Metrics. + runsStarted prometheus.Counter + runsCompleted prometheus.Counter + runsFailed prometheus.Counter + runsLastSuccess prometheus.Gauge + blocksCleanedTotal prometheus.Counter + blocksFailedTotal prometheus.Counter + blocksMarkedForDeletion prometheus.Counter + partialBlocksMarkedForDeletion prometheus.Counter + tenantBlocks *prometheus.GaugeVec + tenantMarkedBlocks *prometheus.GaugeVec + tenantPartialBlocks *prometheus.GaugeVec + tenantBucketIndexLastUpdate *prometheus.GaugeVec +} + +func NewBlocksCleaner(cfg BlocksCleanerConfig, bucketClient objstore.Bucket, ownUser func(userID string) (bool, error), cfgProvider ConfigProvider, logger log.Logger, reg prometheus.Registerer) *BlocksCleaner { + c := &BlocksCleaner{ + cfg: cfg, + bucketClient: bucketClient, + usersScanner: mimir_tsdb.NewUsersScanner(bucketClient, ownUser, logger), + ownUser: ownUser, + cfgProvider: cfgProvider, + singleFlight: concurrency.NewLimitedConcurrencySingleFlight(cfg.CleanupConcurrency), + logger: log.With(logger, "component", "cleaner"), + runsStarted: promauto.With(reg).NewCounter(prometheus.CounterOpts{ + Name: "cortex_compactor_block_cleanup_started_total", + Help: "Total number of blocks cleanup runs started.", + }), + runsCompleted: promauto.With(reg).NewCounter(prometheus.CounterOpts{ + Name: "cortex_compactor_block_cleanup_completed_total", + Help: "Total number of blocks cleanup runs successfully completed.", + }), + runsFailed: promauto.With(reg).NewCounter(prometheus.CounterOpts{ + Name: "cortex_compactor_block_cleanup_failed_total", + Help: "Total number of blocks cleanup runs failed.", + }), + runsLastSuccess: promauto.With(reg).NewGauge(prometheus.GaugeOpts{ + Name: "cortex_compactor_block_cleanup_last_successful_run_timestamp_seconds", + Help: "Unix timestamp of the last successful blocks cleanup run.", + }), + blocksCleanedTotal: promauto.With(reg).NewCounter(prometheus.CounterOpts{ + Name: "cortex_compactor_blocks_cleaned_total", + Help: "Total number of blocks deleted.", + }), + blocksFailedTotal: promauto.With(reg).NewCounter(prometheus.CounterOpts{ + Name: "cortex_compactor_block_cleanup_failures_total", + Help: "Total number of blocks failed to be deleted.", + }), + blocksMarkedForDeletion: promauto.With(reg).NewCounter(prometheus.CounterOpts{ + Name: blocksMarkedForDeletionName, + Help: blocksMarkedForDeletionHelp, + ConstLabels: prometheus.Labels{"reason": "retention"}, + }), + partialBlocksMarkedForDeletion: promauto.With(reg).NewCounter(prometheus.CounterOpts{ + Name: blocksMarkedForDeletionName, + Help: blocksMarkedForDeletionHelp, + ConstLabels: prometheus.Labels{"reason": "partial"}, + }), + + // The following metrics don't have the "cortex_compactor" prefix because not strictly related to + // the compactor. They're just tracked by the compactor because it's the most logical place where these + // metrics can be tracked. + tenantBlocks: promauto.With(reg).NewGaugeVec(prometheus.GaugeOpts{ + Name: "cortex_bucket_blocks_count", + Help: "Total number of blocks in the bucket. Includes blocks marked for deletion, but not partial blocks.", + }, []string{"user"}), + tenantMarkedBlocks: promauto.With(reg).NewGaugeVec(prometheus.GaugeOpts{ + Name: "cortex_bucket_blocks_marked_for_deletion_count", + Help: "Total number of blocks marked for deletion in the bucket.", + }, []string{"user"}), + tenantPartialBlocks: promauto.With(reg).NewGaugeVec(prometheus.GaugeOpts{ + Name: "cortex_bucket_blocks_partials_count", + Help: "Total number of partial blocks.", + }, []string{"user"}), + tenantBucketIndexLastUpdate: promauto.With(reg).NewGaugeVec(prometheus.GaugeOpts{ + Name: "cortex_bucket_index_last_successful_update_timestamp_seconds", + Help: "Timestamp of the last successful update of a tenant's bucket index.", + }, []string{"user"}), + } + + c.Service = services.NewTimerService(cfg.CleanupInterval, c.starting, c.ticker, c.stopping) + + return c +} + +func (c *BlocksCleaner) stopping(error) error { + c.singleFlight.Wait() + return nil +} + +func (c *BlocksCleaner) starting(ctx context.Context) error { + // Run an initial cleanup in starting state. (Note that compactor no longer waits + // for blocks cleaner to finish starting before it starts compactions.) + c.runCleanup(ctx, false) + + return nil +} + +func (c *BlocksCleaner) ticker(ctx context.Context) error { + c.runCleanup(ctx, true) + + return nil +} + +func (c *BlocksCleaner) runCleanup(ctx context.Context, async bool) { + // Wrap logger with some unique ID so if runCleanUp does run in parallel with itself, we can + // at least differentiate the logs in this function for each run. + logger := log.With(c.logger, + "run_id", strconv.FormatInt(time.Now().Unix(), 10), + "task", "clean_up_users", + ) + + c.instrumentStartedCleanupRun(logger) + + allUsers, isDeleted, err := c.refreshOwnedUsers(ctx) + if err != nil { + c.instrumentFinishedCleanupRun(err, logger) + return + } + + doCleanup := func() { + err := c.cleanUsers(ctx, allUsers, isDeleted, logger) + c.instrumentFinishedCleanupRun(err, logger) + } + + if async { + go doCleanup() + } else { + doCleanup() + } +} + +func (c *BlocksCleaner) instrumentStartedCleanupRun(logger log.Logger) { + level.Info(logger).Log("msg", "started blocks cleanup and maintenance") + c.runsStarted.Inc() +} + +func (c *BlocksCleaner) instrumentFinishedCleanupRun(err error, logger log.Logger) { + if err == nil { + level.Info(logger).Log("msg", "successfully completed blocks cleanup and maintenance") + c.runsCompleted.Inc() + c.runsLastSuccess.SetToCurrentTime() + } else if errors.Is(err, context.Canceled) { + level.Info(logger).Log("msg", "canceled blocks cleanup and maintenance", "err", err) + return + } else { + level.Error(logger).Log("msg", "failed to run blocks cleanup and maintenance", "err", err.Error()) + c.runsFailed.Inc() + } +} + +// refreshOwnedUsers is not required to be concurrency safe, but a single instance of this function +// could run concurrently with the cleanup job for any tenant. +func (c *BlocksCleaner) refreshOwnedUsers(ctx context.Context) ([]string, map[string]bool, error) { + users, deleted, err := c.usersScanner.ScanUsers(ctx) + if err != nil { + return nil, nil, errors.Wrap(err, "failed to discover users from bucket") + } + + isActive := util.StringsMap(users) + isDeleted := util.StringsMap(deleted) + allUsers := append(users, deleted...) + + // Delete per-tenant metrics for all tenants not belonging anymore to this shard. + // Such tenants have been moved to a different shard, so their updated metrics will + // be exported by the new shard. + for _, userID := range c.lastOwnedUsers { + if !isActive[userID] && !isDeleted[userID] { + c.tenantBlocks.DeleteLabelValues(userID) + c.tenantMarkedBlocks.DeleteLabelValues(userID) + c.tenantPartialBlocks.DeleteLabelValues(userID) + c.tenantBucketIndexLastUpdate.DeleteLabelValues(userID) + } + } + c.lastOwnedUsers = allUsers + return allUsers, isDeleted, nil +} + +// cleanUsers must be concurrency-safe because some invocations may take longer and overlap with the next periodic invocation. +func (c *BlocksCleaner) cleanUsers(ctx context.Context, allUsers []string, isDeleted map[string]bool, logger log.Logger) error { + return c.singleFlight.ForEachNotInFlight(ctx, allUsers, func(ctx context.Context, userID string) error { + own, err := c.ownUser(userID) + if err != nil || !own { + // This returns error only if err != nil. ForEachUser keeps working for other users. + return errors.Wrap(err, "check own user") + } + + userLogger := util_log.WithUserID(userID, logger) + if isDeleted[userID] { + return errors.Wrapf(c.deleteUserMarkedForDeletion(ctx, userID, userLogger), "failed to delete user marked for deletion: %s", userID) + } + return errors.Wrapf(c.cleanUser(ctx, userID, userLogger), "failed to delete blocks for user: %s", userID) + }) +} + +// deleteRemainingData removes any additional files that may remain when a user has no blocks. Should only +// be called when there no more blocks remaining. +func (c *BlocksCleaner) deleteRemainingData(ctx context.Context, userBucket objstore.Bucket, userID string, userLogger log.Logger) error { + // Delete bucket index + if err := bucketindex.DeleteIndex(ctx, c.bucketClient, userID, c.cfgProvider); err != nil { + return errors.Wrap(err, "failed to delete bucket index file") + } + level.Info(userLogger).Log("msg", "deleted bucket index for tenant with no blocks remaining") + + // Delete markers folder + if deleted, err := bucket.DeletePrefix(ctx, userBucket, block.MarkersPathname, userLogger); err != nil { + return errors.Wrap(err, "failed to delete marker files") + } else if deleted > 0 { + level.Info(userLogger).Log("msg", "deleted marker files for tenant with no blocks remaining", "count", deleted) + } + + // Delete debug folder + if deleted, err := bucket.DeletePrefix(ctx, userBucket, block.DebugMetas, userLogger); err != nil { + return errors.Wrap(err, "failed to delete "+block.DebugMetas) + } else if deleted > 0 { + level.Info(userLogger).Log("msg", "deleted files under "+block.DebugMetas+" for tenant with no blocks remaining", "count", deleted) + } + + return nil +} + +// deleteUserMarkedForDeletion removes blocks and remaining data for tenant marked for deletion. +func (c *BlocksCleaner) deleteUserMarkedForDeletion(ctx context.Context, userID string, userLogger log.Logger) error { + userBucket := bucket.NewUserBucketClient(userID, c.bucketClient, c.cfgProvider) + + level.Info(userLogger).Log("msg", "deleting blocks for tenant marked for deletion") + + // We immediately delete the bucket index, to signal to its consumers that + // the tenant has "no blocks" in the storage. + if err := bucketindex.DeleteIndex(ctx, c.bucketClient, userID, c.cfgProvider); err != nil { + return err + } + c.tenantBucketIndexLastUpdate.DeleteLabelValues(userID) + + var deletedBlocks, failed int + err := userBucket.Iter(ctx, "", func(name string) error { + if err := ctx.Err(); err != nil { + return err + } + + id, ok := block.IsBlockDir(name) + if !ok { + return nil + } + + err := block.Delete(ctx, userLogger, userBucket, id) + if err != nil { + failed++ + c.blocksFailedTotal.Inc() + level.Warn(userLogger).Log("msg", "failed to delete block", "block", id, "err", err) + return nil // Continue with other blocks. + } + + deletedBlocks++ + c.blocksCleanedTotal.Inc() + level.Info(userLogger).Log("msg", "deleted block", "block", id) + return nil + }) + if err != nil { + return err + } + + if failed > 0 { + // The number of blocks left in the storage is equal to the number of blocks we failed + // to delete. We also consider them all marked for deletion given the next run will try + // to delete them again. + c.tenantBlocks.WithLabelValues(userID).Set(float64(failed)) + c.tenantMarkedBlocks.WithLabelValues(userID).Set(float64(failed)) + c.tenantPartialBlocks.WithLabelValues(userID).Set(0) + + return errors.Errorf("failed to delete %d blocks", failed) + } + + // Given all blocks have been deleted, we can also remove the metrics. + c.tenantBlocks.DeleteLabelValues(userID) + c.tenantMarkedBlocks.DeleteLabelValues(userID) + c.tenantPartialBlocks.DeleteLabelValues(userID) + + if deletedBlocks > 0 { + level.Info(userLogger).Log("msg", "deleted blocks for tenant marked for deletion", "deletedBlocks", deletedBlocks) + } + + mark, err := mimir_tsdb.ReadTenantDeletionMark(ctx, c.bucketClient, userID) + if err != nil { + return errors.Wrap(err, "failed to read tenant deletion mark") + } + if mark == nil { + return fmt.Errorf("cannot find tenant deletion mark anymore") + } + + // If we have just deleted some blocks, update "finished" time. Also update "finished" time if it wasn't set yet, but there are no blocks. + // Note: this UPDATES the tenant deletion mark. Components that use caching bucket will NOT SEE this update, + // but that is fine -- they only check whether tenant deletion marker exists or not. + if deletedBlocks > 0 || mark.FinishedTime == 0 { + level.Info(userLogger).Log("msg", "updating finished time in tenant deletion mark") + mark.FinishedTime = time.Now().Unix() + return errors.Wrap(mimir_tsdb.WriteTenantDeletionMark(ctx, c.bucketClient, userID, c.cfgProvider, mark), "failed to update tenant deletion mark") + } + + if time.Since(time.Unix(mark.FinishedTime, 0)) < c.cfg.TenantCleanupDelay { + return nil + } + + level.Info(userLogger).Log("msg", "cleaning up remaining blocks data for tenant marked for deletion") + + // Let's do final cleanup of tenant. + if deleted, err := bucket.DeletePrefix(ctx, userBucket, block.DebugMetas, userLogger); err != nil { + return errors.Wrap(err, "failed to delete "+block.DebugMetas) + } else if deleted > 0 { + level.Info(userLogger).Log("msg", "deleted files under "+block.DebugMetas+" for tenant marked for deletion", "count", deleted) + } + + // Tenant deletion mark file is inside Markers as well. + if deleted, err := bucket.DeletePrefix(ctx, userBucket, block.MarkersPathname, userLogger); err != nil { + return errors.Wrap(err, "failed to delete marker files") + } else if deleted > 0 { + level.Info(userLogger).Log("msg", "deleted marker files for tenant marked for deletion", "count", deleted) + } + + return nil +} + +func (c *BlocksCleaner) cleanUser(ctx context.Context, userID string, userLogger log.Logger) (returnErr error) { + userBucket := bucket.NewUserBucketClient(userID, c.bucketClient, c.cfgProvider) + startTime := time.Now() + + level.Info(userLogger).Log("msg", "started blocks cleanup and maintenance") + defer func() { + if returnErr != nil { + level.Warn(userLogger).Log("msg", "failed blocks cleanup and maintenance", "err", returnErr, "duration", time.Since(startTime)) + } else { + level.Info(userLogger).Log("msg", "completed blocks cleanup and maintenance", "duration", time.Since(startTime)) + } + }() + + // Read the bucket index. + idx, err := bucketindex.ReadIndex(ctx, c.bucketClient, userID, c.cfgProvider, userLogger) + if errors.Is(err, bucketindex.ErrIndexCorrupted) { + level.Warn(userLogger).Log("msg", "found a corrupted bucket index, recreating it") + } else if err != nil && !errors.Is(err, bucketindex.ErrIndexNotFound) { + return err + } + + level.Info(userLogger).Log("msg", "fetched existing bucket index") + + // Mark blocks for future deletion based on the retention period for the user. + // Note doing this before UpdateIndex, so it reads in the deletion marks. + // The trade-off being that retention is not applied if the index has to be + // built, but this is rare. + if idx != nil { + // We do not want to stop the remaining work in the cleaner if an + // error occurs here. Errors are logged in the function. + retention := c.cfgProvider.CompactorBlocksRetentionPeriod(userID) + c.applyUserRetentionPeriod(ctx, idx, retention, userBucket, userLogger) + } + + // Generate an updated in-memory version of the bucket index. + w := bucketindex.NewUpdater(c.bucketClient, userID, c.cfgProvider, userLogger) + idx, partials, err := w.UpdateIndex(ctx, idx) + if err != nil { + return err + } + + c.deleteBlocksMarkedForDeletion(ctx, idx, userBucket, userLogger) + + // Partial blocks with a deletion mark can be cleaned up. This is a best effort, so we don't return + // error if the cleanup of partial blocks fail. + if len(partials) > 0 { + var partialDeletionCutoffTime time.Time // zero value, disabled. + if delay, valid := c.cfgProvider.CompactorPartialBlockDeletionDelay(userID); delay > 0 { + // enable cleanup of partial blocks without deletion marker + partialDeletionCutoffTime = time.Now().Add(-delay) + } else if !valid { + level.Warn(userLogger).Log("msg", "partial blocks deletion has been disabled for tenant because the delay has been set lower than the minimum value allowed", "minimum", validation.MinCompactorPartialBlockDeletionDelay) + } + + c.cleanUserPartialBlocks(ctx, partials, idx, partialDeletionCutoffTime, userBucket, userLogger) + level.Info(userLogger).Log("msg", "cleaned up partial blocks", "partials", len(partials)) + } + + // If there are no more blocks, clean up any remaining files + // Otherwise upload the updated index to the storage. + if c.cfg.NoBlocksFileCleanupEnabled && len(idx.Blocks) == 0 { + if err := c.deleteRemainingData(ctx, userBucket, userID, userLogger); err != nil { + return err + } + } else { + if err := bucketindex.WriteIndex(ctx, c.bucketClient, userID, c.cfgProvider, idx); err != nil { + return err + } + } + + c.tenantBlocks.WithLabelValues(userID).Set(float64(len(idx.Blocks))) + c.tenantMarkedBlocks.WithLabelValues(userID).Set(float64(len(idx.BlockDeletionMarks))) + c.tenantPartialBlocks.WithLabelValues(userID).Set(float64(len(partials))) + c.tenantBucketIndexLastUpdate.WithLabelValues(userID).SetToCurrentTime() + + return nil +} + +// Concurrently deletes blocks marked for deletion, and removes blocks from index. +func (c *BlocksCleaner) deleteBlocksMarkedForDeletion(ctx context.Context, idx *bucketindex.Index, userBucket objstore.Bucket, userLogger log.Logger) { + blocksToDelete := make([]ulid.ULID, 0, len(idx.BlockDeletionMarks)) + + // Collect blocks marked for deletion into buffered channel. + for _, mark := range idx.BlockDeletionMarks { + if time.Since(mark.GetDeletionTime()).Seconds() <= c.cfg.DeletionDelay.Seconds() { + continue + } + blocksToDelete = append(blocksToDelete, mark.ID) + } + + var mu sync.Mutex + + // We don't want to return errors from our function, as that would stop ForEach loop early. + _ = concurrency.ForEachJob(ctx, len(blocksToDelete), c.cfg.DeleteBlocksConcurrency, func(ctx context.Context, jobIdx int) error { + blockID := blocksToDelete[jobIdx] + + if err := block.Delete(ctx, userLogger, userBucket, blockID); err != nil { + c.blocksFailedTotal.Inc() + level.Warn(userLogger).Log("msg", "failed to delete block marked for deletion", "block", blockID, "err", err) + return nil + } + + // Remove the block from the bucket index too. + mu.Lock() + idx.RemoveBlock(blockID) + mu.Unlock() + + c.blocksCleanedTotal.Inc() + level.Info(userLogger).Log("msg", "deleted block marked for deletion", "block", blockID) + return nil + }) +} + +// cleanUserPartialBlocks deletes partial blocks which are safe to be deleted. The provided index is updated accordingly. +// partialDeletionCutoffTime, if not zero, is used to find blocks without deletion marker that were last modified before this time. Such blocks will be marked for deletion. +func (c *BlocksCleaner) cleanUserPartialBlocks(ctx context.Context, partials map[ulid.ULID]error, idx *bucketindex.Index, partialDeletionCutoffTime time.Time, userBucket objstore.InstrumentedBucket, userLogger log.Logger) { + // Collect all blocks with missing meta.json into buffered channel. + blocks := make([]ulid.ULID, 0, len(partials)) + + for blockID, blockErr := range partials { + // We can safely delete only blocks which are partial because the meta.json is missing. + if !errors.Is(blockErr, bucketindex.ErrBlockMetaNotFound) { + continue + } + blocks = append(blocks, blockID) + } + + var mu sync.Mutex + var partialBlocksWithoutDeletionMarker []ulid.ULID + + // We don't want to return errors from our function, as that would stop ForEach loop early. + _ = concurrency.ForEachJob(ctx, len(blocks), c.cfg.DeleteBlocksConcurrency, func(ctx context.Context, jobIdx int) error { + blockID := blocks[jobIdx] + + // We can safely delete only partial blocks with a deletion mark. + err := block.ReadMarker(ctx, userLogger, userBucket, blockID.String(), &block.DeletionMark{}) + if errors.Is(err, block.ErrorMarkerNotFound) { + mu.Lock() + partialBlocksWithoutDeletionMarker = append(partialBlocksWithoutDeletionMarker, blockID) + mu.Unlock() + return nil + } + if err != nil { + level.Warn(userLogger).Log("msg", "error reading partial block deletion mark", "block", blockID, "err", err) + return nil + } + + // Hard-delete partial blocks having a deletion mark, even if the deletion threshold has not + // been reached yet. + if err := block.Delete(ctx, userLogger, userBucket, blockID); err != nil { + c.blocksFailedTotal.Inc() + level.Warn(userLogger).Log("msg", "error deleting partial block marked for deletion", "block", blockID, "err", err) + return nil + } + + // Remove the block from the bucket index too. + mu.Lock() + idx.RemoveBlock(blockID) + delete(partials, blockID) + mu.Unlock() + + c.blocksCleanedTotal.Inc() + level.Info(userLogger).Log("msg", "deleted partial block marked for deletion", "block", blockID) + return nil + }) + + // Check if partial blocks are older than delay period, and mark for deletion + if !partialDeletionCutoffTime.IsZero() { + for _, blockID := range partialBlocksWithoutDeletionMarker { + lastModified, err := stalePartialBlockLastModifiedTime(ctx, blockID, userBucket, partialDeletionCutoffTime) + if err != nil { + level.Warn(userLogger).Log("msg", "failed while determining if partial block should be marked for deletion", "block", blockID, "err", err) + continue + } + if !lastModified.IsZero() { + level.Info(userLogger).Log("msg", "stale partial block found: marking block for deletion", "block", blockID, "last modified", lastModified) + if err := block.MarkForDeletion(ctx, userLogger, userBucket, blockID, "stale partial block", c.partialBlocksMarkedForDeletion); err != nil { + level.Warn(userLogger).Log("msg", "failed to mark partial block for deletion", "block", blockID, "err", err) + } + } + } + } +} + +// applyUserRetentionPeriod marks blocks for deletion which have aged past the retention period. +func (c *BlocksCleaner) applyUserRetentionPeriod(ctx context.Context, idx *bucketindex.Index, retention time.Duration, userBucket objstore.Bucket, userLogger log.Logger) { + // The retention period of zero is a special value indicating to never delete. + if retention <= 0 { + return + } + + blocks := listBlocksOutsideRetentionPeriod(idx, time.Now().Add(-retention)) + + // Attempt to mark all blocks. It is not critical if a marking fails, as + // the cleaner will retry applying the retention in its next cycle. + for _, b := range blocks { + level.Info(userLogger).Log("msg", "applied retention: marking block for deletion", "block", b.ID, "maxTime", b.MaxTime) + if err := block.MarkForDeletion(ctx, userLogger, userBucket, b.ID, fmt.Sprintf("block exceeding retention of %v", retention), c.blocksMarkedForDeletion); err != nil { + level.Warn(userLogger).Log("msg", "failed to mark block for deletion", "block", b.ID, "err", err) + } + } + level.Info(userLogger).Log("msg", "marked blocks for deletion", "num_blocks", len(blocks), "retention", retention.String()) +} + +// listBlocksOutsideRetentionPeriod determines the blocks which have aged past +// the specified retention period, and are not already marked for deletion. +func listBlocksOutsideRetentionPeriod(idx *bucketindex.Index, threshold time.Time) (result bucketindex.Blocks) { + // Whilst re-marking a block is not harmful, it is wasteful and generates + // a warning log message. Use the block deletion marks already in-memory + // to prevent marking blocks already marked for deletion. + marked := make(map[ulid.ULID]struct{}, len(idx.BlockDeletionMarks)) + for _, d := range idx.BlockDeletionMarks { + marked[d.ID] = struct{}{} + } + + for _, b := range idx.Blocks { + maxTime := time.Unix(b.MaxTime/1000, 0) + if maxTime.Before(threshold) { + if _, isMarked := marked[b.ID]; !isMarked { + result = append(result, b) + } + } + } + + return +} + +var errStopIter = errors.New("stop iteration") + +// stalePartialBlockLastModifiedTime returns the most recent last modified time of a stale partial block, or the zero value of time.Time if the provided block wasn't a stale partial block +func stalePartialBlockLastModifiedTime(ctx context.Context, blockID ulid.ULID, userBucket objstore.InstrumentedBucket, partialDeletionCutoffTime time.Time) (time.Time, error) { + var lastModified time.Time + err := userBucket.WithExpectedErrs(func(err error) bool { + return errors.Is(err, errStopIter) // sentinel error + }).Iter(ctx, blockID.String(), func(name string) error { + if strings.HasSuffix(name, objstore.DirDelim) { + return nil + } + attrib, err := userBucket.Attributes(ctx, name) + if err != nil { + return errors.Wrapf(err, "failed to get attributes for %s", name) + } + if attrib.LastModified.After(partialDeletionCutoffTime) { + return errStopIter + } + if attrib.LastModified.After(lastModified) { + lastModified = attrib.LastModified + } + return nil + }, objstore.WithRecursiveIter) + + if errors.Is(err, errStopIter) { + return time.Time{}, nil + } + return lastModified, err +} diff --git a/pkg/compactor/blocks_cleaner_test.go b/pkg/compactor/blocks_cleaner_test.go new file mode 100644 index 0000000000..728edd5558 --- /dev/null +++ b/pkg/compactor/blocks_cleaner_test.go @@ -0,0 +1,1121 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/grafana/mimir/blob/main/pkg/compactor/blocks_cleaner_test.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Cortex Authors. + +package compactor + +import ( + "context" + "crypto/rand" + "errors" + "fmt" + "os" + "path" + "path/filepath" + "strings" + "testing" + "time" + + "github.com/go-kit/log" + "github.com/grafana/dskit/concurrency" + "github.com/grafana/dskit/services" + "github.com/oklog/ulid" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/testutil" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/thanos-io/objstore" + + "github.com/grafana/mimir/pkg/storage/bucket" + "github.com/grafana/mimir/pkg/storage/tsdb" + "github.com/grafana/mimir/pkg/storage/tsdb/block" + "github.com/grafana/mimir/pkg/storage/tsdb/bucketindex" + mimir_testutil "github.com/grafana/mimir/pkg/storage/tsdb/testutil" + "github.com/grafana/mimir/pkg/util" + "github.com/grafana/mimir/pkg/util/test" +) + +type testBlocksCleanerOptions struct { + concurrency int + tenantDeletionDelay time.Duration + user4FilesExist bool // User 4 has "FinishedTime" in tenant deletion marker set to "1h" ago. +} + +func (o testBlocksCleanerOptions) String() string { + return fmt.Sprintf("concurrency=%d, tenant deletion delay=%v", + o.concurrency, o.tenantDeletionDelay) +} + +func TestBlocksCleaner(t *testing.T) { + for _, options := range []testBlocksCleanerOptions{ + {concurrency: 1, tenantDeletionDelay: 0, user4FilesExist: false}, + {concurrency: 1, tenantDeletionDelay: 2 * time.Hour, user4FilesExist: true}, + {concurrency: 2}, + {concurrency: 10}, + } { + options := options + + t.Run(options.String(), func(t *testing.T) { + t.Parallel() + testBlocksCleanerWithOptions(t, options) + }) + } +} + +func testBlocksCleanerWithOptions(t *testing.T, options testBlocksCleanerOptions) { + bucketClient, _ := mimir_testutil.PrepareFilesystemBucket(t) + bucketClient = block.BucketWithGlobalMarkers(bucketClient) + + // Create blocks. + ctx := context.Background() + now := time.Now() + deletionDelay := 12 * time.Hour + block1 := createTSDBBlock(t, bucketClient, "user-1", 10, 20, 2, nil) + block2 := createTSDBBlock(t, bucketClient, "user-1", 20, 30, 2, nil) + block3 := createTSDBBlock(t, bucketClient, "user-1", 30, 40, 2, nil) + block4 := ulid.MustNew(4, rand.Reader) + block5 := ulid.MustNew(5, rand.Reader) + block6 := createTSDBBlock(t, bucketClient, "user-1", 40, 50, 2, nil) + block7 := createTSDBBlock(t, bucketClient, "user-2", 10, 20, 2, nil) + block8 := createTSDBBlock(t, bucketClient, "user-2", 40, 50, 2, nil) + createDeletionMark(t, bucketClient, "user-1", block2, now.Add(-deletionDelay).Add(time.Hour)) // Block hasn't reached the deletion threshold yet. + createDeletionMark(t, bucketClient, "user-1", block3, now.Add(-deletionDelay).Add(-time.Hour)) // Block reached the deletion threshold. + createDeletionMark(t, bucketClient, "user-1", block4, now.Add(-deletionDelay).Add(time.Hour)) // Partial block hasn't reached the deletion threshold yet. + createDeletionMark(t, bucketClient, "user-1", block5, now.Add(-deletionDelay).Add(-time.Hour)) // Partial block reached the deletion threshold. + require.NoError(t, bucketClient.Delete(ctx, path.Join("user-1", block6.String(), block.MetaFilename))) // Partial block without deletion mark. + createDeletionMark(t, bucketClient, "user-2", block7, now.Add(-deletionDelay).Add(-time.Hour)) // Block reached the deletion threshold. + + // Blocks for user-3, marked for deletion. + require.NoError(t, tsdb.WriteTenantDeletionMark(context.Background(), bucketClient, "user-3", nil, tsdb.NewTenantDeletionMark(time.Now()))) + block9 := createTSDBBlock(t, bucketClient, "user-3", 10, 30, 2, nil) + block10 := createTSDBBlock(t, bucketClient, "user-3", 30, 50, 2, nil) + + // User-4 with no more blocks, but couple of mark and debug files. Should be fully deleted. + user4Mark := tsdb.NewTenantDeletionMark(time.Now()) + user4Mark.FinishedTime = time.Now().Unix() - 60 // Set to check final user cleanup. + require.NoError(t, tsdb.WriteTenantDeletionMark(context.Background(), bucketClient, "user-4", nil, user4Mark)) + user4DebugMetaFile := path.Join("user-4", block.DebugMetas, "meta.json") + require.NoError(t, bucketClient.Upload(context.Background(), user4DebugMetaFile, strings.NewReader("some random content here"))) + + cfg := BlocksCleanerConfig{ + DeletionDelay: deletionDelay, + CleanupInterval: time.Minute, + CleanupConcurrency: options.concurrency, + TenantCleanupDelay: options.tenantDeletionDelay, + DeleteBlocksConcurrency: 1, + } + + reg := prometheus.NewPedanticRegistry() + logger := log.NewNopLogger() + cfgProvider := newMockConfigProvider() + + cleaner := NewBlocksCleaner(cfg, bucketClient, tsdb.AllUsers, cfgProvider, logger, reg) + require.NoError(t, services.StartAndAwaitRunning(ctx, cleaner)) + defer services.StopAndAwaitTerminated(ctx, cleaner) //nolint:errcheck + + for _, tc := range []struct { + path string + expectedExists bool + }{ + // Check the storage to ensure only the block which has reached the deletion threshold + // has been effectively deleted. + {path: path.Join("user-1", block1.String(), block.MetaFilename), expectedExists: true}, + {path: path.Join("user-1", block3.String(), block.MetaFilename), expectedExists: false}, + {path: path.Join("user-2", block7.String(), block.MetaFilename), expectedExists: false}, + {path: path.Join("user-2", block8.String(), block.MetaFilename), expectedExists: true}, + // Should not delete a block with deletion mark who hasn't reached the deletion threshold yet. + {path: path.Join("user-1", block2.String(), block.MetaFilename), expectedExists: true}, + {path: path.Join("user-1", block.DeletionMarkFilepath(block2)), expectedExists: true}, + // Should delete a partial block with deletion mark who hasn't reached the deletion threshold yet. + {path: path.Join("user-1", block4.String(), block.DeletionMarkFilename), expectedExists: false}, + {path: path.Join("user-1", block.DeletionMarkFilepath(block4)), expectedExists: false}, + // Should delete a partial block with deletion mark who has reached the deletion threshold. + {path: path.Join("user-1", block5.String(), block.DeletionMarkFilename), expectedExists: false}, + {path: path.Join("user-1", block.DeletionMarkFilepath(block5)), expectedExists: false}, + // Should not delete a partial block without deletion mark. + {path: path.Join("user-1", block6.String(), "index"), expectedExists: true}, + // Should completely delete blocks for user-3, marked for deletion + {path: path.Join("user-3", block9.String(), block.MetaFilename), expectedExists: false}, + {path: path.Join("user-3", block9.String(), "index"), expectedExists: false}, + {path: path.Join("user-3", block10.String(), block.MetaFilename), expectedExists: false}, + {path: path.Join("user-3", block10.String(), "index"), expectedExists: false}, + // Tenant deletion mark is not removed. + {path: path.Join("user-3", tsdb.TenantDeletionMarkPath), expectedExists: true}, + // User-4 is removed fully. + {path: path.Join("user-4", tsdb.TenantDeletionMarkPath), expectedExists: options.user4FilesExist}, + {path: path.Join("user-4", block.DebugMetas, "meta.json"), expectedExists: options.user4FilesExist}, + } { + exists, err := bucketClient.Exists(ctx, tc.path) + require.NoError(t, err) + assert.Equal(t, tc.expectedExists, exists, tc.path) + } + + assert.Equal(t, float64(1), testutil.ToFloat64(cleaner.runsStarted)) + assert.Equal(t, float64(1), testutil.ToFloat64(cleaner.runsCompleted)) + assert.Equal(t, float64(0), testutil.ToFloat64(cleaner.runsFailed)) + assert.Equal(t, float64(6), testutil.ToFloat64(cleaner.blocksCleanedTotal)) + assert.Equal(t, float64(0), testutil.ToFloat64(cleaner.blocksFailedTotal)) + + // Check the updated bucket index. + for _, tc := range []struct { + userID string + expectedIndex bool + expectedBlocks []ulid.ULID + expectedMarks []ulid.ULID + }{ + { + userID: "user-1", + expectedIndex: true, + expectedBlocks: []ulid.ULID{block1, block2 /* deleted: block3, block4, block5, partial: block6 */}, + expectedMarks: []ulid.ULID{block2}, + }, { + userID: "user-2", + expectedIndex: true, + expectedBlocks: []ulid.ULID{block8}, + expectedMarks: []ulid.ULID{}, + }, { + userID: "user-3", + expectedIndex: false, + }, + } { + idx, err := bucketindex.ReadIndex(ctx, bucketClient, tc.userID, nil, logger) + if !tc.expectedIndex { + assert.Equal(t, bucketindex.ErrIndexNotFound, err) + continue + } + + require.NoError(t, err) + assert.ElementsMatch(t, tc.expectedBlocks, idx.Blocks.GetULIDs()) + assert.ElementsMatch(t, tc.expectedMarks, idx.BlockDeletionMarks.GetULIDs()) + } + + assert.NoError(t, testutil.GatherAndCompare(reg, strings.NewReader(` + # HELP cortex_bucket_blocks_count Total number of blocks in the bucket. Includes blocks marked for deletion, but not partial blocks. + # TYPE cortex_bucket_blocks_count gauge + cortex_bucket_blocks_count{user="user-1"} 2 + cortex_bucket_blocks_count{user="user-2"} 1 + # HELP cortex_bucket_blocks_marked_for_deletion_count Total number of blocks marked for deletion in the bucket. + # TYPE cortex_bucket_blocks_marked_for_deletion_count gauge + cortex_bucket_blocks_marked_for_deletion_count{user="user-1"} 1 + cortex_bucket_blocks_marked_for_deletion_count{user="user-2"} 0 + # HELP cortex_bucket_blocks_partials_count Total number of partial blocks. + # TYPE cortex_bucket_blocks_partials_count gauge + cortex_bucket_blocks_partials_count{user="user-1"} 2 + cortex_bucket_blocks_partials_count{user="user-2"} 0 + `), + "cortex_bucket_blocks_count", + "cortex_bucket_blocks_marked_for_deletion_count", + "cortex_bucket_blocks_partials_count", + )) +} + +func TestBlocksCleaner_ShouldContinueOnBlockDeletionFailure(t *testing.T) { + const userID = "user-1" + + bucketClient, _ := mimir_testutil.PrepareFilesystemBucket(t) + bucketClient = block.BucketWithGlobalMarkers(bucketClient) + + // Create blocks. + ctx := context.Background() + now := time.Now() + deletionDelay := 12 * time.Hour + block1 := createTSDBBlock(t, bucketClient, userID, 10, 20, 2, nil) + block2 := createTSDBBlock(t, bucketClient, userID, 20, 30, 2, nil) + block3 := createTSDBBlock(t, bucketClient, userID, 30, 40, 2, nil) + block4 := createTSDBBlock(t, bucketClient, userID, 40, 50, 2, nil) + createDeletionMark(t, bucketClient, userID, block2, now.Add(-deletionDelay).Add(-time.Hour)) + createDeletionMark(t, bucketClient, userID, block3, now.Add(-deletionDelay).Add(-time.Hour)) + createDeletionMark(t, bucketClient, userID, block4, now.Add(-deletionDelay).Add(-time.Hour)) + + // To emulate a failure deleting a block, we wrap the bucket client in a mocked one. + bucketClient = &mockBucketFailure{ + Bucket: bucketClient, + DeleteFailures: []string{path.Join(userID, block3.String(), block.MetaFilename)}, + } + + cfg := BlocksCleanerConfig{ + DeletionDelay: deletionDelay, + CleanupInterval: time.Minute, + CleanupConcurrency: 1, + DeleteBlocksConcurrency: 1, + } + + logger := log.NewNopLogger() + cfgProvider := newMockConfigProvider() + + cleaner := NewBlocksCleaner(cfg, bucketClient, tsdb.AllUsers, cfgProvider, logger, nil) + require.NoError(t, services.StartAndAwaitRunning(ctx, cleaner)) + defer services.StopAndAwaitTerminated(ctx, cleaner) //nolint:errcheck + + for _, tc := range []struct { + path string + expectedExists bool + }{ + {path: path.Join(userID, block1.String(), block.MetaFilename), expectedExists: true}, + {path: path.Join(userID, block2.String(), block.MetaFilename), expectedExists: false}, + {path: path.Join(userID, block3.String(), block.MetaFilename), expectedExists: true}, + {path: path.Join(userID, block4.String(), block.MetaFilename), expectedExists: false}, + } { + exists, err := bucketClient.Exists(ctx, tc.path) + require.NoError(t, err) + assert.Equal(t, tc.expectedExists, exists, tc.path) + } + + assert.Equal(t, float64(1), testutil.ToFloat64(cleaner.runsStarted)) + assert.Equal(t, float64(1), testutil.ToFloat64(cleaner.runsCompleted)) + assert.Equal(t, float64(0), testutil.ToFloat64(cleaner.runsFailed)) + assert.Equal(t, float64(2), testutil.ToFloat64(cleaner.blocksCleanedTotal)) + assert.Equal(t, float64(1), testutil.ToFloat64(cleaner.blocksFailedTotal)) + + // Check the updated bucket index. + idx, err := bucketindex.ReadIndex(ctx, bucketClient, userID, nil, logger) + require.NoError(t, err) + assert.ElementsMatch(t, []ulid.ULID{block1, block3}, idx.Blocks.GetULIDs()) + assert.ElementsMatch(t, []ulid.ULID{block3}, idx.BlockDeletionMarks.GetULIDs()) +} + +func TestBlocksCleaner_ShouldRebuildBucketIndexOnCorruptedOne(t *testing.T) { + const userID = "user-1" + + bucketClient, _ := mimir_testutil.PrepareFilesystemBucket(t) + bucketClient = block.BucketWithGlobalMarkers(bucketClient) + + // Create blocks. + ctx := context.Background() + now := time.Now() + deletionDelay := 12 * time.Hour + block1 := createTSDBBlock(t, bucketClient, userID, 10, 20, 2, nil) + block2 := createTSDBBlock(t, bucketClient, userID, 20, 30, 2, nil) + block3 := createTSDBBlock(t, bucketClient, userID, 30, 40, 2, nil) + createDeletionMark(t, bucketClient, userID, block2, now.Add(-deletionDelay).Add(-time.Hour)) + createDeletionMark(t, bucketClient, userID, block3, now.Add(-deletionDelay).Add(time.Hour)) + + // Write a corrupted bucket index. + require.NoError(t, bucketClient.Upload(ctx, path.Join(userID, bucketindex.IndexCompressedFilename), strings.NewReader("invalid!}"))) + + cfg := BlocksCleanerConfig{ + DeletionDelay: deletionDelay, + CleanupInterval: time.Minute, + CleanupConcurrency: 1, + DeleteBlocksConcurrency: 1, + } + + logger := log.NewNopLogger() + cfgProvider := newMockConfigProvider() + + cleaner := NewBlocksCleaner(cfg, bucketClient, tsdb.AllUsers, cfgProvider, logger, nil) + require.NoError(t, services.StartAndAwaitRunning(ctx, cleaner)) + defer services.StopAndAwaitTerminated(ctx, cleaner) //nolint:errcheck + + for _, tc := range []struct { + path string + expectedExists bool + }{ + {path: path.Join(userID, block1.String(), block.MetaFilename), expectedExists: true}, + {path: path.Join(userID, block2.String(), block.MetaFilename), expectedExists: false}, + {path: path.Join(userID, block3.String(), block.MetaFilename), expectedExists: true}, + } { + exists, err := bucketClient.Exists(ctx, tc.path) + require.NoError(t, err) + assert.Equal(t, tc.expectedExists, exists, tc.path) + } + + assert.Equal(t, float64(1), testutil.ToFloat64(cleaner.runsStarted)) + assert.Equal(t, float64(1), testutil.ToFloat64(cleaner.runsCompleted)) + assert.Equal(t, float64(0), testutil.ToFloat64(cleaner.runsFailed)) + assert.Equal(t, float64(1), testutil.ToFloat64(cleaner.blocksCleanedTotal)) + assert.Equal(t, float64(0), testutil.ToFloat64(cleaner.blocksFailedTotal)) + + // Check the updated bucket index. + idx, err := bucketindex.ReadIndex(ctx, bucketClient, userID, nil, logger) + require.NoError(t, err) + assert.ElementsMatch(t, []ulid.ULID{block1, block3}, idx.Blocks.GetULIDs()) + assert.ElementsMatch(t, []ulid.ULID{block3}, idx.BlockDeletionMarks.GetULIDs()) +} + +func TestBlocksCleaner_ShouldRemoveMetricsForTenantsNotBelongingAnymoreToTheShard(t *testing.T) { + bucketClient, _ := mimir_testutil.PrepareFilesystemBucket(t) + bucketClient = block.BucketWithGlobalMarkers(bucketClient) + + // Create blocks. + createTSDBBlock(t, bucketClient, "user-1", 10, 20, 2, nil) + createTSDBBlock(t, bucketClient, "user-1", 20, 30, 2, nil) + createTSDBBlock(t, bucketClient, "user-2", 30, 40, 2, nil) + + cfg := BlocksCleanerConfig{ + DeletionDelay: time.Hour, + CleanupInterval: time.Minute, + CleanupConcurrency: 1, + DeleteBlocksConcurrency: 1, + } + + ctx := context.Background() + logger := log.NewNopLogger() + reg := prometheus.NewPedanticRegistry() + cfgProvider := newMockConfigProvider() + + cleaner := NewBlocksCleaner(cfg, bucketClient, tsdb.AllUsers, cfgProvider, logger, reg) + require.NoError(t, cleaner.runCleanupWithErr(ctx)) + + assert.NoError(t, testutil.GatherAndCompare(reg, strings.NewReader(` + # HELP cortex_bucket_blocks_count Total number of blocks in the bucket. Includes blocks marked for deletion, but not partial blocks. + # TYPE cortex_bucket_blocks_count gauge + cortex_bucket_blocks_count{user="user-1"} 2 + cortex_bucket_blocks_count{user="user-2"} 1 + # HELP cortex_bucket_blocks_marked_for_deletion_count Total number of blocks marked for deletion in the bucket. + # TYPE cortex_bucket_blocks_marked_for_deletion_count gauge + cortex_bucket_blocks_marked_for_deletion_count{user="user-1"} 0 + cortex_bucket_blocks_marked_for_deletion_count{user="user-2"} 0 + # HELP cortex_bucket_blocks_partials_count Total number of partial blocks. + # TYPE cortex_bucket_blocks_partials_count gauge + cortex_bucket_blocks_partials_count{user="user-1"} 0 + cortex_bucket_blocks_partials_count{user="user-2"} 0 + `), + "cortex_bucket_blocks_count", + "cortex_bucket_blocks_marked_for_deletion_count", + "cortex_bucket_blocks_partials_count", + )) + + // Override the users scanner to reconfigure it to only return a subset of users. + cleaner.usersScanner = tsdb.NewUsersScanner(bucketClient, func(userID string) (bool, error) { return userID == "user-1", nil }, logger) + + // Create new blocks, to double check expected metrics have changed. + createTSDBBlock(t, bucketClient, "user-1", 40, 50, 2, nil) + createTSDBBlock(t, bucketClient, "user-2", 50, 60, 2, nil) + + require.NoError(t, cleaner.runCleanupWithErr(ctx)) + + assert.NoError(t, testutil.GatherAndCompare(reg, strings.NewReader(` + # HELP cortex_bucket_blocks_count Total number of blocks in the bucket. Includes blocks marked for deletion, but not partial blocks. + # TYPE cortex_bucket_blocks_count gauge + cortex_bucket_blocks_count{user="user-1"} 3 + # HELP cortex_bucket_blocks_marked_for_deletion_count Total number of blocks marked for deletion in the bucket. + # TYPE cortex_bucket_blocks_marked_for_deletion_count gauge + cortex_bucket_blocks_marked_for_deletion_count{user="user-1"} 0 + # HELP cortex_bucket_blocks_partials_count Total number of partial blocks. + # TYPE cortex_bucket_blocks_partials_count gauge + cortex_bucket_blocks_partials_count{user="user-1"} 0 + `), + "cortex_bucket_blocks_count", + "cortex_bucket_blocks_marked_for_deletion_count", + "cortex_bucket_blocks_partials_count", + )) +} + +func TestBlocksCleaner_ShouldNotCleanupUserThatDoesntBelongToShardAnymore(t *testing.T) { + bucketClient, _ := mimir_testutil.PrepareFilesystemBucket(t) + bucketClient = block.BucketWithGlobalMarkers(bucketClient) + + // Create blocks. + createTSDBBlock(t, bucketClient, "user-1", 10, 20, 2, nil) + createTSDBBlock(t, bucketClient, "user-2", 20, 30, 2, nil) + + cfg := BlocksCleanerConfig{ + DeletionDelay: time.Hour, + CleanupInterval: time.Minute, + CleanupConcurrency: 1, + DeleteBlocksConcurrency: 1, + } + + ctx := context.Background() + logger := log.NewNopLogger() + reg := prometheus.NewPedanticRegistry() + cfgProvider := newMockConfigProvider() + + // We will simulate change of "ownUser" by counting number of replies per user. First reply will be "true", + // all subsequent replies will be false. + + userSeen := map[string]bool{} + ownUser := func(user string) (bool, error) { + if userSeen[user] { + return false, nil + } + userSeen[user] = true + return true, nil + } + + cleaner := NewBlocksCleaner(cfg, bucketClient, ownUser, cfgProvider, logger, reg) + require.NoError(t, cleaner.runCleanupWithErr(ctx)) + + // Verify that we have seen the users + require.ElementsMatch(t, []string{"user-1", "user-2"}, cleaner.lastOwnedUsers) + + // But there are no metrics for any user, because we did not in fact clean them. + assert.NoError(t, testutil.GatherAndCompare(reg, strings.NewReader(` + # HELP cortex_bucket_blocks_count Total number of blocks in the bucket. Includes blocks marked for deletion, but not partial blocks. + # TYPE cortex_bucket_blocks_count gauge + `), + "cortex_bucket_blocks_count", + )) + + // Running cleanUsers again will see that users are no longer owned. + require.NoError(t, cleaner.runCleanupWithErr(ctx)) + require.ElementsMatch(t, []string{}, cleaner.lastOwnedUsers) +} + +func TestBlocksCleaner_ListBlocksOutsideRetentionPeriod(t *testing.T) { + bucketClient, _ := mimir_testutil.PrepareFilesystemBucket(t) + bucketClient = block.BucketWithGlobalMarkers(bucketClient) + ctx := context.Background() + logger := log.NewNopLogger() + + id1 := createTSDBBlock(t, bucketClient, "user-1", 5000, 6000, 2, nil) + id2 := createTSDBBlock(t, bucketClient, "user-1", 6000, 7000, 2, nil) + id3 := createTSDBBlock(t, bucketClient, "user-1", 7000, 8000, 2, nil) + + w := bucketindex.NewUpdater(bucketClient, "user-1", nil, logger) + idx, _, err := w.UpdateIndex(ctx, nil) + require.NoError(t, err) + + assert.ElementsMatch(t, []ulid.ULID{id1, id2, id3}, idx.Blocks.GetULIDs()) + + // Excessive retention period (wrapping epoch) + result := listBlocksOutsideRetentionPeriod(idx, time.Unix(10, 0).Add(-time.Hour)) + assert.ElementsMatch(t, []ulid.ULID{}, result.GetULIDs()) + + // Normal operation - varying retention period. + result = listBlocksOutsideRetentionPeriod(idx, time.Unix(6, 0)) + assert.ElementsMatch(t, []ulid.ULID{}, result.GetULIDs()) + + result = listBlocksOutsideRetentionPeriod(idx, time.Unix(7, 0)) + assert.ElementsMatch(t, []ulid.ULID{id1}, result.GetULIDs()) + + result = listBlocksOutsideRetentionPeriod(idx, time.Unix(8, 0)) + assert.ElementsMatch(t, []ulid.ULID{id1, id2}, result.GetULIDs()) + + result = listBlocksOutsideRetentionPeriod(idx, time.Unix(9, 0)) + assert.ElementsMatch(t, []ulid.ULID{id1, id2, id3}, result.GetULIDs()) + + // Avoiding redundant marking - blocks already marked for deletion. + + mark1 := &bucketindex.BlockDeletionMark{ID: id1} + mark2 := &bucketindex.BlockDeletionMark{ID: id2} + + idx.BlockDeletionMarks = bucketindex.BlockDeletionMarks{mark1} + + result = listBlocksOutsideRetentionPeriod(idx, time.Unix(7, 0)) + assert.ElementsMatch(t, []ulid.ULID{}, result.GetULIDs()) + + result = listBlocksOutsideRetentionPeriod(idx, time.Unix(8, 0)) + assert.ElementsMatch(t, []ulid.ULID{id2}, result.GetULIDs()) + + idx.BlockDeletionMarks = bucketindex.BlockDeletionMarks{mark1, mark2} + + result = listBlocksOutsideRetentionPeriod(idx, time.Unix(7, 0)) + assert.ElementsMatch(t, []ulid.ULID{}, result.GetULIDs()) + + result = listBlocksOutsideRetentionPeriod(idx, time.Unix(8, 0)) + assert.ElementsMatch(t, []ulid.ULID{}, result.GetULIDs()) + + result = listBlocksOutsideRetentionPeriod(idx, time.Unix(9, 0)) + assert.ElementsMatch(t, []ulid.ULID{id3}, result.GetULIDs()) +} + +func TestBlocksCleaner_ShouldRemoveBlocksOutsideRetentionPeriod(t *testing.T) { + bucketClient, _ := mimir_testutil.PrepareFilesystemBucket(t) + bucketClient = block.BucketWithGlobalMarkers(bucketClient) + + ts := func(hours int) int64 { + return time.Now().Add(time.Duration(hours)*time.Hour).Unix() * 1000 + } + + block1 := createTSDBBlock(t, bucketClient, "user-1", ts(-10), ts(-8), 2, nil) + block2 := createTSDBBlock(t, bucketClient, "user-1", ts(-8), ts(-6), 2, nil) + block3 := createTSDBBlock(t, bucketClient, "user-2", ts(-10), ts(-8), 2, nil) + block4 := createTSDBBlock(t, bucketClient, "user-2", ts(-8), ts(-6), 2, nil) + + cfg := BlocksCleanerConfig{ + DeletionDelay: time.Hour, + CleanupInterval: time.Minute, + CleanupConcurrency: 1, + DeleteBlocksConcurrency: 1, + } + + ctx := context.Background() + logger := test.NewTestingLogger(t) + reg := prometheus.NewPedanticRegistry() + cfgProvider := newMockConfigProvider() + + cleaner := NewBlocksCleaner(cfg, bucketClient, tsdb.AllUsers, cfgProvider, logger, reg) + + assertBlockExists := func(user string, blockID ulid.ULID, expectExists bool) { + exists, err := bucketClient.Exists(ctx, path.Join(user, blockID.String(), block.MetaFilename)) + require.NoError(t, err) + assert.Equal(t, expectExists, exists) + } + + // Existing behaviour - retention period disabled. + { + cfgProvider.userRetentionPeriods["user-1"] = 0 + cfgProvider.userRetentionPeriods["user-2"] = 0 + + require.NoError(t, cleaner.runCleanupWithErr(ctx)) + assertBlockExists("user-1", block1, true) + assertBlockExists("user-1", block2, true) + assertBlockExists("user-2", block3, true) + assertBlockExists("user-2", block4, true) + + assert.NoError(t, testutil.GatherAndCompare(reg, strings.NewReader(` + # HELP cortex_bucket_blocks_count Total number of blocks in the bucket. Includes blocks marked for deletion, but not partial blocks. + # TYPE cortex_bucket_blocks_count gauge + cortex_bucket_blocks_count{user="user-1"} 2 + cortex_bucket_blocks_count{user="user-2"} 2 + # HELP cortex_bucket_blocks_marked_for_deletion_count Total number of blocks marked for deletion in the bucket. + # TYPE cortex_bucket_blocks_marked_for_deletion_count gauge + cortex_bucket_blocks_marked_for_deletion_count{user="user-1"} 0 + cortex_bucket_blocks_marked_for_deletion_count{user="user-2"} 0 + # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. + # TYPE cortex_compactor_blocks_marked_for_deletion_total counter + cortex_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 + cortex_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 + `), + "cortex_bucket_blocks_count", + "cortex_bucket_blocks_marked_for_deletion_count", + "cortex_compactor_blocks_marked_for_deletion_total", + )) + } + + // Retention enabled only for a single user, but does nothing. + { + cfgProvider.userRetentionPeriods["user-1"] = 9 * time.Hour + + require.NoError(t, cleaner.runCleanupWithErr(ctx)) + assertBlockExists("user-1", block1, true) + assertBlockExists("user-1", block2, true) + assertBlockExists("user-2", block3, true) + assertBlockExists("user-2", block4, true) + } + + // Retention enabled only for a single user, marking a single block. + // Note the block won't be deleted yet due to deletion delay. + { + cfgProvider.userRetentionPeriods["user-1"] = 7 * time.Hour + + require.NoError(t, cleaner.runCleanupWithErr(ctx)) + assertBlockExists("user-1", block1, true) + assertBlockExists("user-1", block2, true) + assertBlockExists("user-2", block3, true) + assertBlockExists("user-2", block4, true) + + assert.NoError(t, testutil.GatherAndCompare(reg, strings.NewReader(` + # HELP cortex_bucket_blocks_count Total number of blocks in the bucket. Includes blocks marked for deletion, but not partial blocks. + # TYPE cortex_bucket_blocks_count gauge + cortex_bucket_blocks_count{user="user-1"} 2 + cortex_bucket_blocks_count{user="user-2"} 2 + # HELP cortex_bucket_blocks_marked_for_deletion_count Total number of blocks marked for deletion in the bucket. + # TYPE cortex_bucket_blocks_marked_for_deletion_count gauge + cortex_bucket_blocks_marked_for_deletion_count{user="user-1"} 1 + cortex_bucket_blocks_marked_for_deletion_count{user="user-2"} 0 + # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. + # TYPE cortex_compactor_blocks_marked_for_deletion_total counter + cortex_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 + cortex_compactor_blocks_marked_for_deletion_total{reason="retention"} 1 + `), + "cortex_bucket_blocks_count", + "cortex_bucket_blocks_marked_for_deletion_count", + "cortex_compactor_blocks_marked_for_deletion_total", + )) + } + + // Marking the block again, before the deletion occurs, should not cause an error. + { + require.NoError(t, cleaner.runCleanupWithErr(ctx)) + assertBlockExists("user-1", block1, true) + assertBlockExists("user-1", block2, true) + assertBlockExists("user-2", block3, true) + assertBlockExists("user-2", block4, true) + } + + // Reduce the deletion delay. Now the block will be deleted. + { + cleaner.cfg.DeletionDelay = 0 + + require.NoError(t, cleaner.runCleanupWithErr(ctx)) + assertBlockExists("user-1", block1, false) + assertBlockExists("user-1", block2, true) + assertBlockExists("user-2", block3, true) + assertBlockExists("user-2", block4, true) + + assert.NoError(t, testutil.GatherAndCompare(reg, strings.NewReader(` + # HELP cortex_bucket_blocks_count Total number of blocks in the bucket. Includes blocks marked for deletion, but not partial blocks. + # TYPE cortex_bucket_blocks_count gauge + cortex_bucket_blocks_count{user="user-1"} 1 + cortex_bucket_blocks_count{user="user-2"} 2 + # HELP cortex_bucket_blocks_marked_for_deletion_count Total number of blocks marked for deletion in the bucket. + # TYPE cortex_bucket_blocks_marked_for_deletion_count gauge + cortex_bucket_blocks_marked_for_deletion_count{user="user-1"} 0 + cortex_bucket_blocks_marked_for_deletion_count{user="user-2"} 0 + # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. + # TYPE cortex_compactor_blocks_marked_for_deletion_total counter + cortex_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 + cortex_compactor_blocks_marked_for_deletion_total{reason="retention"} 1 + `), + "cortex_bucket_blocks_count", + "cortex_bucket_blocks_marked_for_deletion_count", + "cortex_compactor_blocks_marked_for_deletion_total", + )) + } + + // Retention enabled for other user; test deleting multiple blocks. + { + cfgProvider.userRetentionPeriods["user-2"] = 5 * time.Hour + + require.NoError(t, cleaner.runCleanupWithErr(ctx)) + assertBlockExists("user-1", block1, false) + assertBlockExists("user-1", block2, true) + assertBlockExists("user-2", block3, false) + assertBlockExists("user-2", block4, false) + + assert.NoError(t, testutil.GatherAndCompare(reg, strings.NewReader(` + # HELP cortex_bucket_blocks_count Total number of blocks in the bucket. Includes blocks marked for deletion, but not partial blocks. + # TYPE cortex_bucket_blocks_count gauge + cortex_bucket_blocks_count{user="user-1"} 1 + cortex_bucket_blocks_count{user="user-2"} 0 + # HELP cortex_bucket_blocks_marked_for_deletion_count Total number of blocks marked for deletion in the bucket. + # TYPE cortex_bucket_blocks_marked_for_deletion_count gauge + cortex_bucket_blocks_marked_for_deletion_count{user="user-1"} 0 + cortex_bucket_blocks_marked_for_deletion_count{user="user-2"} 0 + # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. + # TYPE cortex_compactor_blocks_marked_for_deletion_total counter + cortex_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 + cortex_compactor_blocks_marked_for_deletion_total{reason="retention"} 3 + `), + "cortex_bucket_blocks_count", + "cortex_bucket_blocks_marked_for_deletion_count", + "cortex_compactor_blocks_marked_for_deletion_total", + )) + } +} + +func checkBlock(t *testing.T, user string, bucketClient objstore.Bucket, blockID ulid.ULID, metaJSONExists bool, markedForDeletion bool) { + exists, err := bucketClient.Exists(context.Background(), path.Join(user, blockID.String(), block.MetaFilename)) + require.NoError(t, err) + require.Equal(t, metaJSONExists, exists) + + exists, err = bucketClient.Exists(context.Background(), path.Join(user, blockID.String(), block.DeletionMarkFilename)) + require.NoError(t, err) + require.Equal(t, markedForDeletion, exists) +} + +func TestBlocksCleaner_ShouldCleanUpFilesWhenNoMoreBlocksRemain(t *testing.T) { + bucketClient, _ := mimir_testutil.PrepareFilesystemBucket(t) + bucketClient = block.BucketWithGlobalMarkers(bucketClient) + + const userID = "user-1" + ctx := context.Background() + now := time.Now() + deletionDelay := 12 * time.Hour + + // Create two blocks and mark them for deletion at a time before the deletionDelay + block1 := createTSDBBlock(t, bucketClient, userID, 10, 20, 2, nil) + block2 := createTSDBBlock(t, bucketClient, userID, 20, 30, 2, nil) + + createDeletionMark(t, bucketClient, userID, block1, now.Add(-deletionDelay).Add(-time.Hour)) + createDeletionMark(t, bucketClient, userID, block2, now.Add(-deletionDelay).Add(-time.Hour)) + + checkBlock(t, "user-1", bucketClient, block1, true, true) + checkBlock(t, "user-1", bucketClient, block2, true, true) + + // Create a deletion mark within the deletionDelay period that won't correspond to any block + randomULID := ulid.MustNew(ulid.Now(), rand.Reader) + createDeletionMark(t, bucketClient, userID, randomULID, now.Add(-deletionDelay).Add(time.Hour)) + blockDeletionMarkFile := path.Join(userID, block.DeletionMarkFilepath(randomULID)) + exists, err := bucketClient.Exists(ctx, blockDeletionMarkFile) + require.NoError(t, err) + assert.True(t, exists) + + // Create a debug file that wouldn't otherwise be deleted by the cleaner + debugMetaFile := path.Join(userID, block.DebugMetas, "meta.json") + require.NoError(t, bucketClient.Upload(context.Background(), debugMetaFile, strings.NewReader("random content"))) + + cfg := BlocksCleanerConfig{ + DeletionDelay: deletionDelay, + CleanupInterval: time.Minute, + CleanupConcurrency: 1, + DeleteBlocksConcurrency: 1, + NoBlocksFileCleanupEnabled: true, + } + + logger := test.NewTestingLogger(t) + reg := prometheus.NewPedanticRegistry() + cfgProvider := newMockConfigProvider() + + cleaner := NewBlocksCleaner(cfg, bucketClient, tsdb.AllUsers, cfgProvider, logger, reg) + require.NoError(t, cleaner.runCleanupWithErr(ctx)) + + // Check bucket index, markers and debug files have been deleted. + exists, err = bucketClient.Exists(ctx, blockDeletionMarkFile) + require.NoError(t, err) + assert.False(t, exists) + + exists, err = bucketClient.Exists(ctx, debugMetaFile) + require.NoError(t, err) + assert.False(t, exists) + + _, err = bucketindex.ReadIndex(ctx, bucketClient, userID, nil, logger) + require.ErrorIs(t, err, bucketindex.ErrIndexNotFound) +} + +func TestBlocksCleaner_ShouldRemovePartialBlocksOutsideDelayPeriod(t *testing.T) { + bucketClient, _ := mimir_testutil.PrepareFilesystemBucket(t) + bucketClient = block.BucketWithGlobalMarkers(bucketClient) + + ts := func(hours int) int64 { + return time.Now().Add(time.Duration(hours)*time.Hour).Unix() * 1000 + } + + block1 := createTSDBBlock(t, bucketClient, "user-1", ts(-10), ts(-8), 2, nil) + block2 := createTSDBBlock(t, bucketClient, "user-1", ts(-8), ts(-6), 2, nil) + + cfg := BlocksCleanerConfig{ + DeletionDelay: time.Hour, + CleanupInterval: time.Minute, + CleanupConcurrency: 1, + DeleteBlocksConcurrency: 1, + } + + ctx := context.Background() + logger := test.NewTestingLogger(t) + reg := prometheus.NewPedanticRegistry() + cfgProvider := newMockConfigProvider() + + cleaner := NewBlocksCleaner(cfg, bucketClient, tsdb.AllUsers, cfgProvider, logger, reg) + + makeBlockPartial := func(user string, blockID ulid.ULID) { + err := bucketClient.Delete(ctx, path.Join(user, blockID.String(), block.MetaFilename)) + require.NoError(t, err) + } + + checkBlock(t, "user-1", bucketClient, block1, true, false) + checkBlock(t, "user-1", bucketClient, block2, true, false) + makeBlockPartial("user-1", block1) + checkBlock(t, "user-1", bucketClient, block1, false, false) + checkBlock(t, "user-1", bucketClient, block2, true, false) + + require.NoError(t, cleaner.cleanUser(ctx, "user-1", logger)) + + // check that no blocks were marked for deletion, because deletion delay is set to 0. + checkBlock(t, "user-1", bucketClient, block1, false, false) + checkBlock(t, "user-1", bucketClient, block2, true, false) + + // Test that partial block does get marked for deletion + // The delay time must be very short since these temporary files were just created + cfgProvider.userPartialBlockDelay["user-1"] = 1 * time.Nanosecond + + require.NoError(t, cleaner.cleanUser(ctx, "user-1", logger)) + + // check that first block was marked for deletion (partial block updated far in the past), but not the second one, because it's not partial. + checkBlock(t, "user-1", bucketClient, block1, false, true) + checkBlock(t, "user-1", bucketClient, block2, true, false) + + require.NoError(t, testutil.GatherAndCompare(reg, strings.NewReader(` + # HELP cortex_bucket_blocks_count Total number of blocks in the bucket. Includes blocks marked for deletion, but not partial blocks. + # TYPE cortex_bucket_blocks_count gauge + cortex_bucket_blocks_count{user="user-1"} 1 + # HELP cortex_bucket_blocks_marked_for_deletion_count Total number of blocks marked for deletion in the bucket. + # TYPE cortex_bucket_blocks_marked_for_deletion_count gauge + cortex_bucket_blocks_marked_for_deletion_count{user="user-1"} 0 + # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. + # TYPE cortex_compactor_blocks_marked_for_deletion_total counter + cortex_compactor_blocks_marked_for_deletion_total{reason="partial"} 1 + cortex_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 + `), + "cortex_bucket_blocks_count", + "cortex_bucket_blocks_marked_for_deletion_count", + "cortex_compactor_blocks_marked_for_deletion_total", + )) +} + +func TestBlocksCleaner_ShouldNotRemovePartialBlocksInsideDelayPeriod(t *testing.T) { + bucketClient, _ := mimir_testutil.PrepareFilesystemBucket(t) + bucketClient = block.BucketWithGlobalMarkers(bucketClient) + + ts := func(hours int) int64 { + return time.Now().Add(time.Duration(hours)*time.Hour).Unix() * 1000 + } + + block1 := createTSDBBlock(t, bucketClient, "user-1", ts(-10), ts(-8), 2, nil) + block2 := createTSDBBlock(t, bucketClient, "user-2", ts(-8), ts(-6), 2, nil) + + cfg := BlocksCleanerConfig{ + DeletionDelay: time.Hour, + CleanupInterval: time.Minute, + CleanupConcurrency: 1, + DeleteBlocksConcurrency: 1, + } + + ctx := context.Background() + logger := test.NewTestingLogger(t) + reg := prometheus.NewPedanticRegistry() + cfgProvider := newMockConfigProvider() + + cleaner := NewBlocksCleaner(cfg, bucketClient, tsdb.AllUsers, cfgProvider, logger, reg) + + makeBlockPartial := func(user string, blockID ulid.ULID) { + err := bucketClient.Delete(ctx, path.Join(user, blockID.String(), block.MetaFilename)) + require.NoError(t, err) + } + + corruptMeta := func(user string, blockID ulid.ULID) { + err := bucketClient.Upload(ctx, path.Join(user, blockID.String(), block.MetaFilename), strings.NewReader("corrupted file contents")) + require.NoError(t, err) + } + + checkBlock(t, "user-1", bucketClient, block1, true, false) + checkBlock(t, "user-2", bucketClient, block2, true, false) + + makeBlockPartial("user-1", block1) + corruptMeta("user-2", block2) + + checkBlock(t, "user-1", bucketClient, block1, false, false) + checkBlock(t, "user-2", bucketClient, block2, true, false) + + // Set partial block delay such that block will not be marked for deletion + // The comparison is based on inode modification time, so anything more than very recent (< 1 second) won't be + // out of range + cfgProvider.userPartialBlockDelay["user-1"] = 1 * time.Hour + cfgProvider.userPartialBlockDelay["user-2"] = 1 * time.Nanosecond + + require.NoError(t, cleaner.cleanUser(ctx, "user-1", logger)) + checkBlock(t, "user-1", bucketClient, block1, false, false) // This block was updated too recently, so we don't mark it for deletion just yet. + checkBlock(t, "user-2", bucketClient, block2, true, false) // No change for user-2. + + require.NoError(t, cleaner.cleanUser(ctx, "user-2", logger)) + checkBlock(t, "user-1", bucketClient, block1, false, false) // No change for user-1 + checkBlock(t, "user-2", bucketClient, block2, true, false) // Block with corrupted meta is NOT marked for deletion. + + // The cortex_compactor_blocks_marked_for_deletion_total{reason="partial"} counter should be zero since for user-1 + // the time since modification is shorter than the delay, and for user-2, the metadata is corrupted but the file + // is still present in the bucket so the block is not partial + require.NoError(t, testutil.GatherAndCompare(reg, strings.NewReader(` + # HELP cortex_bucket_blocks_count Total number of blocks in the bucket. Includes blocks marked for deletion, but not partial blocks. + # TYPE cortex_bucket_blocks_count gauge + cortex_bucket_blocks_count{user="user-1"} 0 + cortex_bucket_blocks_count{user="user-2"} 0 + # HELP cortex_bucket_blocks_marked_for_deletion_count Total number of blocks marked for deletion in the bucket. + # TYPE cortex_bucket_blocks_marked_for_deletion_count gauge + cortex_bucket_blocks_marked_for_deletion_count{user="user-1"} 0 + cortex_bucket_blocks_marked_for_deletion_count{user="user-2"} 0 + # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. + # TYPE cortex_compactor_blocks_marked_for_deletion_total counter + cortex_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 + cortex_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 + `), + "cortex_bucket_blocks_count", + "cortex_bucket_blocks_marked_for_deletion_count", + "cortex_compactor_blocks_marked_for_deletion_total", + )) +} + +func TestBlocksCleaner_ShouldNotRemovePartialBlocksIfConfiguredDelayIsInvalid(t *testing.T) { + ctx := context.Background() + reg := prometheus.NewPedanticRegistry() + logs := &concurrency.SyncBuffer{} + logger := log.NewLogfmtLogger(logs) + + bucketClient, _ := mimir_testutil.PrepareFilesystemBucket(t) + bucketClient = block.BucketWithGlobalMarkers(bucketClient) + + ts := func(hours int) int64 { + return time.Now().Add(time.Duration(hours)*time.Hour).Unix() * 1000 + } + + // Create a partial block. + block1 := createTSDBBlock(t, bucketClient, "user-1", ts(-10), ts(-8), 2, nil) + err := bucketClient.Delete(ctx, path.Join("user-1", block1.String(), block.MetaFilename)) + require.NoError(t, err) + + cfg := BlocksCleanerConfig{ + DeletionDelay: time.Hour, + CleanupInterval: time.Minute, + CleanupConcurrency: 1, + DeleteBlocksConcurrency: 1, + } + + // Configure an invalid delay. + cfgProvider := newMockConfigProvider() + cfgProvider.userPartialBlockDelay["user-1"] = 0 + cfgProvider.userPartialBlockDelayInvalid["user-1"] = true + + // Pre-condition check: block should be partial and not being marked for deletion. + checkBlock(t, "user-1", bucketClient, block1, false, false) + + // Run the cleanup. + cleaner := NewBlocksCleaner(cfg, bucketClient, tsdb.AllUsers, cfgProvider, logger, reg) + require.NoError(t, cleaner.cleanUser(ctx, "user-1", logger)) + + // Ensure the block has NOT been marked for deletion. + checkBlock(t, "user-1", bucketClient, block1, false, false) + assert.Contains(t, logs.String(), "partial blocks deletion has been disabled for tenant because the delay has been set lower than the minimum value allowed") + + require.NoError(t, testutil.GatherAndCompare(reg, strings.NewReader(` + # HELP cortex_bucket_blocks_count Total number of blocks in the bucket. Includes blocks marked for deletion, but not partial blocks. + # TYPE cortex_bucket_blocks_count gauge + cortex_bucket_blocks_count{user="user-1"} 0 + # HELP cortex_bucket_blocks_marked_for_deletion_count Total number of blocks marked for deletion in the bucket. + # TYPE cortex_bucket_blocks_marked_for_deletion_count gauge + cortex_bucket_blocks_marked_for_deletion_count{user="user-1"} 0 + # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. + # TYPE cortex_compactor_blocks_marked_for_deletion_total counter + cortex_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 + cortex_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 + `), + "cortex_bucket_blocks_count", + "cortex_bucket_blocks_marked_for_deletion_count", + "cortex_compactor_blocks_marked_for_deletion_total", + )) +} + +func TestStalePartialBlockLastModifiedTime(t *testing.T) { + b, dir := mimir_testutil.PrepareFilesystemBucket(t) + + const tenant = "user" + + objectTime := time.Now().Add(-1 * time.Hour).Truncate(time.Second) // ignore milliseconds, as not all filesystems store them. + blockID := createTSDBBlock(t, b, tenant, objectTime.UnixMilli(), time.Now().UnixMilli(), 2, nil) + for _, f := range []string{"meta.json", "index", "chunks/000001", "tombstones"} { + require.NoError(t, os.Chtimes(filepath.Join(dir, tenant, blockID.String(), filepath.FromSlash(f)), objectTime, objectTime)) + } + + userBucket := bucket.NewUserBucketClient(tenant, b, nil) + + emptyBlockID := ulid.ULID{} + require.NotEqual(t, blockID, emptyBlockID) + empty := true + err := userBucket.Iter(context.Background(), emptyBlockID.String(), func(_ string) error { + empty = false + return nil + }) + require.NoError(t, err) + require.True(t, empty) + + testCases := []struct { + name string + blockID ulid.ULID + cutoff time.Time + expectedLastModified time.Time + }{ + {name: "no objects", blockID: emptyBlockID, cutoff: objectTime, expectedLastModified: time.Time{}}, + {name: "objects newer than delay cutoff", blockID: blockID, cutoff: objectTime.Add(-1 * time.Second), expectedLastModified: time.Time{}}, + {name: "objects equal to delay cutoff", blockID: blockID, cutoff: objectTime, expectedLastModified: objectTime}, + {name: "objects older than delay cutoff", blockID: blockID, cutoff: objectTime.Add(1 * time.Second), expectedLastModified: objectTime}, + } + + for _, tc := range testCases { + t.Run(tc.name, func(t *testing.T) { + lastModified, err := stalePartialBlockLastModifiedTime(context.Background(), tc.blockID, userBucket, tc.cutoff) + require.NoError(t, err) + require.Equal(t, tc.expectedLastModified, lastModified) + }) + } +} + +type mockBucketFailure struct { + objstore.Bucket + + DeleteFailures []string +} + +func (m *mockBucketFailure) Delete(ctx context.Context, name string) error { + if util.StringsContain(m.DeleteFailures, name) { + return errors.New("mocked delete failure") + } + return m.Bucket.Delete(ctx, name) +} + +type mockConfigProvider struct { + userRetentionPeriods map[string]time.Duration + splitAndMergeShards map[string]int + instancesShardSize map[string]int + splitGroups map[string]int + blockUploadEnabled map[string]bool + blockUploadValidationEnabled map[string]bool + blockUploadMaxBlockSizeBytes map[string]int64 + userPartialBlockDelay map[string]time.Duration + userPartialBlockDelayInvalid map[string]bool + verifyChunks map[string]bool +} + +func newMockConfigProvider() *mockConfigProvider { + return &mockConfigProvider{ + userRetentionPeriods: make(map[string]time.Duration), + splitAndMergeShards: make(map[string]int), + splitGroups: make(map[string]int), + blockUploadEnabled: make(map[string]bool), + blockUploadValidationEnabled: make(map[string]bool), + blockUploadMaxBlockSizeBytes: make(map[string]int64), + userPartialBlockDelay: make(map[string]time.Duration), + userPartialBlockDelayInvalid: make(map[string]bool), + verifyChunks: make(map[string]bool), + } +} + +func (m *mockConfigProvider) CompactorBlocksRetentionPeriod(user string) time.Duration { + if result, ok := m.userRetentionPeriods[user]; ok { + return result + } + return 0 +} + +func (m *mockConfigProvider) CompactorSplitAndMergeShards(user string) int { + if result, ok := m.splitAndMergeShards[user]; ok { + return result + } + return 0 +} + +func (m *mockConfigProvider) CompactorSplitGroups(user string) int { + if result, ok := m.splitGroups[user]; ok { + return result + } + return 0 +} + +func (m *mockConfigProvider) CompactorTenantShardSize(user string) int { + if result, ok := m.instancesShardSize[user]; ok { + return result + } + return 0 +} + +func (m *mockConfigProvider) CompactorBlockUploadEnabled(tenantID string) bool { + return m.blockUploadEnabled[tenantID] +} + +func (m *mockConfigProvider) CompactorBlockUploadValidationEnabled(tenantID string) bool { + return m.blockUploadValidationEnabled[tenantID] +} + +func (m *mockConfigProvider) CompactorPartialBlockDeletionDelay(user string) (time.Duration, bool) { + return m.userPartialBlockDelay[user], !m.userPartialBlockDelayInvalid[user] +} + +func (m *mockConfigProvider) CompactorBlockUploadVerifyChunks(tenantID string) bool { + return m.verifyChunks[tenantID] +} + +func (m *mockConfigProvider) CompactorBlockUploadMaxBlockSizeBytes(user string) int64 { + return m.blockUploadMaxBlockSizeBytes[user] +} + +func (m *mockConfigProvider) S3SSEType(string) string { + return "" +} + +func (m *mockConfigProvider) S3SSEKMSKeyID(string) string { + return "" +} + +func (m *mockConfigProvider) S3SSEKMSEncryptionContext(string) string { + return "" +} + +func (c *BlocksCleaner) runCleanupWithErr(ctx context.Context) error { + allUsers, isDeleted, err := c.refreshOwnedUsers(ctx) + if err != nil { + return err + } + + return c.cleanUsers(ctx, allUsers, isDeleted, log.NewNopLogger()) +} diff --git a/pkg/compactor/bucket_compactor.go b/pkg/compactor/bucket_compactor.go new file mode 100644 index 0000000000..d4d28f8e5c --- /dev/null +++ b/pkg/compactor/bucket_compactor.go @@ -0,0 +1,1032 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/grafana/mimir/blob/main/pkg/compactor/bucket_compactor.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Cortex Authors. + +package compactor + +import ( + "context" + "fmt" + "os" + "path" + "path/filepath" + "sync" + "time" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/grafana/dskit/concurrency" + "github.com/grafana/dskit/multierror" + "github.com/grafana/dskit/runutil" + "github.com/oklog/ulid" + "github.com/pkg/errors" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/tsdb" + "github.com/thanos-io/objstore" + "go.uber.org/atomic" + + "github.com/grafana/mimir/pkg/storage/sharding" + mimir_tsdb "github.com/grafana/mimir/pkg/storage/tsdb" + "github.com/grafana/mimir/pkg/storage/tsdb/block" +) + +type DeduplicateFilter interface { + block.MetadataFilter + + // DuplicateIDs returns IDs of duplicate blocks generated by last call to Filter method. + DuplicateIDs() []ulid.ULID +} + +// Syncer synchronizes block metas from a bucket into a local directory. +// It sorts them into compaction groups based on equal label sets. +type Syncer struct { + logger log.Logger + bkt objstore.Bucket + fetcher *block.MetaFetcher + mtx sync.Mutex + blocks map[ulid.ULID]*block.Meta + metrics *syncerMetrics + deduplicateBlocksFilter DeduplicateFilter +} + +type syncerMetrics struct { + garbageCollections prometheus.Counter + garbageCollectionFailures prometheus.Counter + garbageCollectionDuration prometheus.Histogram + blocksMarkedForDeletion prometheus.Counter +} + +func newSyncerMetrics(reg prometheus.Registerer, blocksMarkedForDeletion prometheus.Counter) *syncerMetrics { + var m syncerMetrics + + m.garbageCollections = promauto.With(reg).NewCounter(prometheus.CounterOpts{ + Name: "thanos_compact_garbage_collection_total", + Help: "Total number of garbage collection operations.", + }) + m.garbageCollectionFailures = promauto.With(reg).NewCounter(prometheus.CounterOpts{ + Name: "thanos_compact_garbage_collection_failures_total", + Help: "Total number of failed garbage collection operations.", + }) + m.garbageCollectionDuration = promauto.With(reg).NewHistogram(prometheus.HistogramOpts{ + Name: "thanos_compact_garbage_collection_duration_seconds", + Help: "Time it took to perform garbage collection iteration.", + Buckets: []float64{0.01, 0.1, 0.3, 0.6, 1, 3, 6, 9, 20, 30, 60, 90, 120, 240, 360, 720}, + }) + + m.blocksMarkedForDeletion = blocksMarkedForDeletion + + return &m +} + +// NewMetaSyncer returns a new Syncer for the given Bucket and directory. +// Blocks must be at least as old as the sync delay for being considered. +func NewMetaSyncer(logger log.Logger, reg prometheus.Registerer, bkt objstore.Bucket, fetcher *block.MetaFetcher, deduplicateBlocksFilter DeduplicateFilter, blocksMarkedForDeletion prometheus.Counter) (*Syncer, error) { + if logger == nil { + logger = log.NewNopLogger() + } + return &Syncer{ + logger: logger, + bkt: bkt, + fetcher: fetcher, + blocks: map[ulid.ULID]*block.Meta{}, + metrics: newSyncerMetrics(reg, blocksMarkedForDeletion), + deduplicateBlocksFilter: deduplicateBlocksFilter, + }, nil +} + +// SyncMetas synchronizes local state of block metas with what we have in the bucket. +func (s *Syncer) SyncMetas(ctx context.Context) error { + s.mtx.Lock() + defer s.mtx.Unlock() + + // While fetching blocks, we filter out blocks that were marked for deletion. + // No deletion delay is used -- all blocks with deletion marker are ignored, and not considered for compaction. + metas, _, err := s.fetcher.FetchWithoutMarkedForDeletion(ctx) + if err != nil { + return err + } + s.blocks = metas + return nil +} + +// Metas returns loaded metadata blocks since last sync. +func (s *Syncer) Metas() map[ulid.ULID]*block.Meta { + s.mtx.Lock() + defer s.mtx.Unlock() + + return s.blocks +} + +// GarbageCollect marks blocks for deletion from bucket if their data is available as part of a +// block with a higher compaction level. +// Call to SyncMetas function is required to populate duplicateIDs in duplicateBlocksFilter. +func (s *Syncer) GarbageCollect(ctx context.Context) error { + s.mtx.Lock() + defer s.mtx.Unlock() + + begin := time.Now() + + // The deduplication filter is applied after all blocks marked for deletion have been excluded + // (with no deletion delay), so we expect that all duplicated blocks have not been marked for + // deletion yet. Even in the remote case these blocks have already been marked for deletion, + // the block.MarkForDeletion() call will correctly handle it. + duplicateIDs := s.deduplicateBlocksFilter.DuplicateIDs() + + for _, id := range duplicateIDs { + if ctx.Err() != nil { + return ctx.Err() + } + + // Spawn a new context so we always mark a block for deletion in full on shutdown. + delCtx, cancel := context.WithTimeout(context.Background(), 5*time.Minute) + + level.Info(s.logger).Log("msg", "marking outdated block for deletion", "block", id) + err := block.MarkForDeletion(delCtx, s.logger, s.bkt, id, "outdated block", s.metrics.blocksMarkedForDeletion) + cancel() + if err != nil { + s.metrics.garbageCollectionFailures.Inc() + return errors.Wrapf(err, "mark block %s for deletion", id) + } + + // Immediately update our in-memory state so no further call to SyncMetas is needed + // after running garbage collection. + delete(s.blocks, id) + } + s.metrics.garbageCollections.Inc() + s.metrics.garbageCollectionDuration.Observe(time.Since(begin).Seconds()) + return nil +} + +// Grouper is responsible to group all known blocks into compaction Job which are safe to be +// compacted concurrently. +type Grouper interface { + // Groups returns the compaction jobs for all blocks currently known to the syncer. + // It creates all jobs from the scratch on every call. + Groups(blocks map[ulid.ULID]*block.Meta) (res []*Job, err error) +} + +// DefaultGroupKey returns a unique identifier for the group the block belongs to, based on +// the DefaultGrouper logic. It considers the downsampling resolution and the block's labels. +func DefaultGroupKey(meta block.ThanosMeta) string { + return defaultGroupKey(meta.Downsample.Resolution, labels.FromMap(meta.Labels)) +} + +func defaultGroupKey(res int64, lbls labels.Labels) string { + return fmt.Sprintf("%d@%v", res, labels.StableHash(lbls)) +} + +func minTime(metas []*block.Meta) time.Time { + if len(metas) == 0 { + return time.Time{} + } + + minT := metas[0].MinTime + for _, meta := range metas { + if meta.MinTime < minT { + minT = meta.MinTime + } + } + + return time.Unix(0, minT*int64(time.Millisecond)).UTC() +} + +func maxTime(metas []*block.Meta) time.Time { + if len(metas) == 0 { + return time.Time{} + } + + maxT := metas[0].MaxTime + for _, meta := range metas { + if meta.MaxTime > maxT { + maxT = meta.MaxTime + } + } + + return time.Unix(0, maxT*int64(time.Millisecond)).UTC() +} + +// Planner returns blocks to compact. +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 []*block.Meta) ([]*block.Meta, error) +} + +// Compactor provides compaction against an underlying storage of time series data. +// This is similar to tsdb.Compactor just without Plan method. +// TODO(bwplotka): Split the Planner from Compactor on upstream as well, so we can import it. +type Compactor interface { + // Write persists a Block into a directory. + // No Block is written when resulting Block has 0 samples, and returns empty ulid.ULID{}. + Write(dest string, b tsdb.BlockReader, mint, maxt int64, parent *tsdb.BlockMeta) (ulid.ULID, error) + + // Compact runs compaction against the provided directories. Must + // only be called concurrently with results of Plan(). + // Can optionally pass a list of already open blocks, + // to avoid having to reopen them. + // When resulting Block has 0 samples + // * No block is written. + // * The source dirs are marked Deletable. + // * Returns empty ulid.ULID{}. + Compact(dest string, dirs []string, open []*tsdb.Block) (ulid.ULID, error) + + // CompactWithSplitting merges and splits the source blocks into shardCount number of compacted blocks, + // and returns slice of block IDs. Position of returned block ID in the result slice corresponds to the shard index. + // If given compacted block has no series, corresponding block ID will be zero ULID value. + CompactWithSplitting(dest string, dirs []string, open []*tsdb.Block, shardCount uint64) (result []ulid.ULID, _ error) +} + +// runCompactionJob plans and runs a single compaction against the provided job. The compacted result +// is uploaded into the bucket the blocks were retrieved from. +func (c *BucketCompactor) runCompactionJob(ctx context.Context, job *Job) (shouldRerun bool, compIDs []ulid.ULID, rerr error) { + jobBeginTime := time.Now() + + jobLogger := log.With(c.logger, "groupKey", job.Key()) + subDir := filepath.Join(c.compactDir, job.Key()) + + defer func() { + elapsed := time.Since(jobBeginTime) + + if rerr == nil { + level.Info(jobLogger).Log("msg", "compaction job succeeded", "duration", elapsed, "duration_ms", elapsed.Milliseconds()) + } else { + level.Error(jobLogger).Log("msg", "compaction job failed", "duration", elapsed, "duration_ms", elapsed.Milliseconds(), "err", rerr) + } + + if err := os.RemoveAll(subDir); err != nil { + level.Error(jobLogger).Log("msg", "failed to remove compaction group work directory", "path", subDir, "err", err) + } + }() + + if err := os.MkdirAll(subDir, 0o750); err != nil { + return false, nil, errors.Wrap(err, "create compaction job dir") + } + + toCompact, err := c.planner.Plan(ctx, job.metasByMinTime) + if err != nil { + return false, nil, errors.Wrap(err, "plan compaction") + } + if len(toCompact) == 0 { + // Nothing to do. + return false, nil, nil + } + + // The planner returned some blocks to compact, so we can enrich the logger + // with the min/max time between all blocks to compact. + toCompactMinTime := minTime(toCompact) + toCompactMaxTime := maxTime(toCompact) + jobLogger = log.With(jobLogger, "minTime", toCompactMinTime.String(), "maxTime", toCompactMaxTime.String()) + + level.Info(jobLogger).Log("msg", "compaction available and planned; downloading blocks", "blocks", len(toCompact), "plan", fmt.Sprintf("%v", toCompact)) + + // Once we have a plan we need to download the actual data. + downloadBegin := time.Now() + + err = concurrency.ForEachJob(ctx, len(toCompact), c.blockSyncConcurrency, func(ctx context.Context, idx int) error { + meta := toCompact[idx] + + // Must be the same as in blocksToCompactDirs. + bdir := filepath.Join(subDir, meta.ULID.String()) + + if err := block.Download(ctx, jobLogger, c.bkt, meta.ULID, bdir); err != nil { + return errors.Wrapf(err, "download block %s", meta.ULID) + } + + // Ensure all source blocks are valid. + stats, err := block.GatherBlockHealthStats(jobLogger, bdir, meta.MinTime, meta.MaxTime, false) + if err != nil { + return errors.Wrapf(err, "gather index issues for block %s", bdir) + } + + if err := stats.CriticalErr(); err != nil { + return 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(); err != nil { + return errors.Wrapf(err, "block id %s", meta.ULID) + } + return nil + }) + if err != nil { + return false, nil, err + } + + blocksToCompactDirs := make([]string, len(toCompact)) + for ix, meta := range toCompact { + blocksToCompactDirs[ix] = filepath.Join(subDir, meta.ULID.String()) + } + + elapsed := time.Since(downloadBegin) + level.Info(jobLogger).Log("msg", "downloaded and verified blocks; compacting blocks", "blocks", len(blocksToCompactDirs), "plan", fmt.Sprintf("%v", blocksToCompactDirs), "duration", elapsed, "duration_ms", elapsed.Milliseconds()) + + compactionBegin := time.Now() + + if job.UseSplitting() { + compIDs, err = c.comp.CompactWithSplitting(subDir, blocksToCompactDirs, nil, uint64(job.SplittingShards())) + } else { + var compID ulid.ULID + compID, err = c.comp.Compact(subDir, blocksToCompactDirs, nil) + compIDs = append(compIDs, compID) + } + if err != nil { + return false, nil, errors.Wrapf(err, "compact blocks %v", blocksToCompactDirs) + } + + if !hasNonZeroULIDs(compIDs) { + // Prometheus compactor found that the compacted block would have no samples. + level.Info(jobLogger).Log("msg", "compacted block would have no samples, deleting source blocks", "blocks", fmt.Sprintf("%v", blocksToCompactDirs)) + for _, meta := range toCompact { + if meta.Stats.NumSamples == 0 { + if err := deleteBlock(c.bkt, meta.ULID, filepath.Join(subDir, meta.ULID.String()), jobLogger, c.metrics.blocksMarkedForDeletion); err != nil { + level.Warn(jobLogger).Log("msg", "failed to mark for deletion an empty block found during compaction", "block", meta.ULID, "err", err) + } + } + } + // Even though this block was empty, there may be more work to do. + return true, nil, nil + } + + elapsed = time.Since(compactionBegin) + level.Info(jobLogger).Log("msg", "compacted blocks", "new", fmt.Sprintf("%v", compIDs), "blocks", fmt.Sprintf("%v", blocksToCompactDirs), "duration", elapsed, "duration_ms", elapsed.Milliseconds()) + + uploadBegin := time.Now() + uploadedBlocks := atomic.NewInt64(0) + + if err = verifyCompactedBlocksTimeRanges(compIDs, toCompactMinTime.UnixMilli(), toCompactMaxTime.UnixMilli(), subDir); err != nil { + level.Warn(jobLogger).Log("msg", "compacted blocks verification failed", "err", err) + c.metrics.compactionBlocksVerificationFailed.Inc() + } + + blocksToUpload := convertCompactionResultToForEachJobs(compIDs, job.UseSplitting(), jobLogger) + err = concurrency.ForEachJob(ctx, len(blocksToUpload), c.blockSyncConcurrency, func(ctx context.Context, idx int) error { + blockToUpload := blocksToUpload[idx] + + uploadedBlocks.Inc() + + bdir := filepath.Join(subDir, blockToUpload.ulid.String()) + + // When splitting is enabled, we need to inject the shard ID as external label. + newLabels := job.Labels().Map() + if job.UseSplitting() { + newLabels[mimir_tsdb.CompactorShardIDExternalLabel] = sharding.FormatShardIDLabelValue(uint64(blockToUpload.shardIndex), uint64(job.SplittingShards())) + } + + newMeta, err := block.InjectThanosMeta(jobLogger, bdir, block.ThanosMeta{ + Labels: newLabels, + Downsample: block.ThanosDownsample{Resolution: job.Resolution()}, + Source: block.CompactorSource, + SegmentFiles: block.GetSegmentFiles(bdir), + }, nil) + if err != nil { + return errors.Wrapf(err, "failed to finalize the block %s", bdir) + } + + if err = os.Remove(filepath.Join(bdir, "tombstones")); err != nil { + return errors.Wrap(err, "remove tombstones") + } + + // Ensure the compacted block is valid. + if err := block.VerifyBlock(jobLogger, bdir, newMeta.MinTime, newMeta.MaxTime, false); err != nil { + return errors.Wrapf(err, "invalid result block %s", bdir) + } + + begin := time.Now() + if err := block.Upload(ctx, jobLogger, c.bkt, bdir, nil); err != nil { + return errors.Wrapf(err, "upload of %s failed", blockToUpload.ulid) + } + + elapsed := time.Since(begin) + level.Info(jobLogger).Log("msg", "uploaded block", "result_block", blockToUpload.ulid, "duration", elapsed, "duration_ms", elapsed.Milliseconds(), "external_labels", labels.FromMap(newLabels)) + return nil + }) + if err != nil { + return false, nil, err + } + + elapsed = time.Since(uploadBegin) + level.Info(jobLogger).Log("msg", "uploaded all blocks", "blocks", uploadedBlocks, "duration", elapsed, "duration_ms", elapsed.Milliseconds()) + + // Mark for deletion the blocks we just compacted from the job and bucket so they do not get included + // into the next planning cycle. + // Eventually the block we just uploaded should get synced into the job again (including sync-delay). + for _, meta := range toCompact { + if err := deleteBlock(c.bkt, meta.ULID, filepath.Join(subDir, meta.ULID.String()), jobLogger, c.metrics.blocksMarkedForDeletion); err != nil { + return false, nil, errors.Wrapf(err, "mark old block for deletion from bucket") + } + } + + return true, compIDs, nil +} + +// verifyCompactedBlocksTimeRanges does a full run over the compacted blocks +// and verifies that they satisfy the min/maxTime from the source blocks +func verifyCompactedBlocksTimeRanges(compIDs []ulid.ULID, sourceBlocksMinTime, sourceBlocksMaxTime int64, subDir string) error { + sourceBlocksMinTimeFound := false + sourceBlocksMaxTimeFound := false + + for _, compID := range compIDs { + // Skip empty block + if compID == (ulid.ULID{}) { + continue + } + + bdir := filepath.Join(subDir, compID.String()) + meta, err := block.ReadMetaFromDir(bdir) + if err != nil { + return errors.Wrapf(err, "failed to read meta.json from %s during block time range verification", bdir) + } + + // Ensure compacted block min/maxTime within source blocks min/maxTime + if meta.MinTime < sourceBlocksMinTime { + return fmt.Errorf("invalid minTime for block %s, compacted block minTime %d is before source minTime %d", compID.String(), meta.MinTime, sourceBlocksMinTime) + } + + if meta.MaxTime > sourceBlocksMaxTime { + return fmt.Errorf("invalid maxTime for block %s, compacted block maxTime %d is after source maxTime %d", compID.String(), meta.MaxTime, sourceBlocksMaxTime) + } + + if meta.MinTime == sourceBlocksMinTime { + sourceBlocksMinTimeFound = true + } + + if meta.MaxTime == sourceBlocksMaxTime { + sourceBlocksMaxTimeFound = true + } + } + + // Check that the minTime and maxTime from the source blocks + // are found at least once in the compacted blocks + if !sourceBlocksMinTimeFound || !sourceBlocksMaxTimeFound { + return fmt.Errorf("compacted block(s) do not contain minTime %d and maxTime %d from the source blocks", sourceBlocksMinTime, sourceBlocksMaxTime) + } + + return nil +} + +// convertCompactionResultToForEachJobs filters out empty ULIDs. +// When handling result of split compactions, shard index is index in the slice returned by compaction. +func convertCompactionResultToForEachJobs(compactedBlocks []ulid.ULID, splitJob bool, jobLogger log.Logger) []ulidWithShardIndex { + result := make([]ulidWithShardIndex, 0, len(compactedBlocks)) + + for ix, id := range compactedBlocks { + // Skip if it's an empty block. + if id == (ulid.ULID{}) { + if splitJob { + level.Info(jobLogger).Log("msg", "compaction produced an empty block", "shard_id", sharding.FormatShardIDLabelValue(uint64(ix), uint64(len(compactedBlocks)))) + } else { + level.Info(jobLogger).Log("msg", "compaction produced an empty block") + } + continue + } + + result = append(result, ulidWithShardIndex{shardIndex: ix, ulid: id}) + } + return result +} + +type ulidWithShardIndex struct { + ulid ulid.ULID + shardIndex int +} + +// Issue347Error is a type wrapper for errors that should invoke repair process for broken block. +type Issue347Error struct { + err error + + id ulid.ULID +} + +func issue347Error(err error, brokenBlock ulid.ULID) Issue347Error { + return Issue347Error{err: err, id: brokenBlock} +} + +func (e Issue347Error) Error() string { + return e.err.Error() +} + +// IsIssue347Error returns true if the base error is a Issue347Error. +func IsIssue347Error(err error) bool { + _, ok := errors.Cause(err).(Issue347Error) + return ok +} + +// OutOfOrderChunkError is a type wrapper for OOO chunk error from validating block index. +type OutOfOrderChunksError struct { + err error + id ulid.ULID +} + +func (e OutOfOrderChunksError) Error() string { + return e.err.Error() +} + +func outOfOrderChunkError(err error, brokenBlock ulid.ULID) OutOfOrderChunksError { + return OutOfOrderChunksError{err: err, id: brokenBlock} +} + +// IsOutOfOrderChunk returns true if the base error is a OutOfOrderChunkError. +func IsOutOfOrderChunkError(err error) bool { + _, ok := errors.Cause(err).(OutOfOrderChunksError) + return ok +} + +// RepairIssue347 repairs the https://github.com/prometheus/tsdb/issues/347 issue when having issue347Error. +func RepairIssue347(ctx context.Context, logger log.Logger, bkt objstore.Bucket, blocksMarkedForDeletion prometheus.Counter, issue347Err error) error { + ie, ok := errors.Cause(issue347Err).(Issue347Error) + if !ok { + return errors.Errorf("Given error is not an issue347 error: %v", issue347Err) + } + + level.Info(logger).Log("msg", "Repairing block broken by https://github.com/prometheus/tsdb/issues/347", "id", ie.id, "err", issue347Err) + + tmpdir, err := os.MkdirTemp("", fmt.Sprintf("repair-issue-347-id-%s-", ie.id)) + if err != nil { + return err + } + + defer func() { + if err := os.RemoveAll(tmpdir); err != nil { + level.Warn(logger).Log("msg", "failed to remote tmpdir", "err", err, "tmpdir", tmpdir) + } + }() + + bdir := filepath.Join(tmpdir, ie.id.String()) + if err := block.Download(ctx, logger, bkt, ie.id, bdir); err != nil { + return errors.Wrapf(err, "download block %s", ie.id) + } + + meta, err := block.ReadMetaFromDir(bdir) + if err != nil { + return errors.Wrapf(err, "read meta from %s", bdir) + } + + resid, err := block.Repair(logger, tmpdir, ie.id, block.CompactorRepairSource, block.IgnoreIssue347OutsideChunk) + if err != nil { + return errors.Wrapf(err, "repair failed for block %s", ie.id) + } + + // Verify repaired id before uploading it. + if err := block.VerifyBlock(logger, filepath.Join(tmpdir, resid.String()), meta.MinTime, meta.MaxTime, false); err != nil { + return errors.Wrapf(err, "repaired block is invalid %s", resid) + } + + level.Info(logger).Log("msg", "uploading repaired block", "newID", resid) + if err = block.Upload(ctx, logger, bkt, filepath.Join(tmpdir, resid.String()), nil); err != nil { + return errors.Wrapf(err, "upload of %s failed", resid) + } + + level.Info(logger).Log("msg", "deleting broken block", "id", ie.id) + + // Spawn a new context so we always mark a block for deletion in full on shutdown. + delCtx, cancel := context.WithTimeout(context.Background(), 5*time.Minute) + defer cancel() + + // TODO(bplotka): Issue with this will introduce overlap that will halt compactor. Automate that (fix duplicate overlaps caused by this). + if err := block.MarkForDeletion(delCtx, logger, bkt, ie.id, "source of repaired block", blocksMarkedForDeletion); err != nil { + return errors.Wrapf(err, "marking old block %s for deletion has failed", ie.id) + } + return nil +} + +func deleteBlock(bkt objstore.Bucket, id ulid.ULID, bdir string, logger log.Logger, blocksMarkedForDeletion prometheus.Counter) error { + if err := os.RemoveAll(bdir); err != nil { + return errors.Wrapf(err, "remove old block dir %s", id) + } + + // Spawn a new context so we always mark a block for deletion in full on shutdown. + delCtx, cancel := context.WithTimeout(context.Background(), 5*time.Minute) + defer cancel() + level.Info(logger).Log("msg", "marking compacted block for deletion", "old_block", id) + if err := block.MarkForDeletion(delCtx, logger, bkt, id, "source of compacted block", blocksMarkedForDeletion); err != nil { + return errors.Wrapf(err, "mark block %s for deletion from bucket", id) + } + return nil +} + +// BucketCompactorMetrics holds the metrics tracked by BucketCompactor. +type BucketCompactorMetrics struct { + groupCompactionRunsStarted prometheus.Counter + groupCompactionRunsCompleted prometheus.Counter + groupCompactionRunsFailed prometheus.Counter + groupCompactions prometheus.Counter + compactionBlocksVerificationFailed prometheus.Counter + blocksMarkedForDeletion prometheus.Counter + blocksMarkedForNoCompact prometheus.Counter + blocksMaxTimeDelta prometheus.Histogram +} + +// NewBucketCompactorMetrics makes a new BucketCompactorMetrics. +func NewBucketCompactorMetrics(blocksMarkedForDeletion prometheus.Counter, reg prometheus.Registerer) *BucketCompactorMetrics { + return &BucketCompactorMetrics{ + groupCompactionRunsStarted: promauto.With(reg).NewCounter(prometheus.CounterOpts{ + Name: "cortex_compactor_group_compaction_runs_started_total", + Help: "Total number of group compaction attempts.", + }), + groupCompactionRunsCompleted: promauto.With(reg).NewCounter(prometheus.CounterOpts{ + Name: "cortex_compactor_group_compaction_runs_completed_total", + Help: "Total number of group completed compaction runs. This also includes compactor group runs that resulted with no compaction.", + }), + groupCompactionRunsFailed: promauto.With(reg).NewCounter(prometheus.CounterOpts{ + Name: "cortex_compactor_group_compactions_failures_total", + Help: "Total number of failed group compactions.", + }), + groupCompactions: promauto.With(reg).NewCounter(prometheus.CounterOpts{ + Name: "cortex_compactor_group_compactions_total", + Help: "Total number of group compaction attempts that resulted in new block(s).", + }), + compactionBlocksVerificationFailed: promauto.With(reg).NewCounter(prometheus.CounterOpts{ + Name: "cortex_compactor_blocks_verification_failures_total", + Help: "Total number of failures when verifying min/max time ranges of compacted blocks.", + }), + blocksMarkedForDeletion: blocksMarkedForDeletion, + blocksMarkedForNoCompact: promauto.With(reg).NewCounter(prometheus.CounterOpts{ + Name: "cortex_compactor_blocks_marked_for_no_compaction_total", + Help: "Total number of blocks that were marked for no-compaction.", + ConstLabels: prometheus.Labels{"reason": block.OutOfOrderChunksNoCompactReason}, + }), + blocksMaxTimeDelta: promauto.With(reg).NewHistogram(prometheus.HistogramOpts{ + Name: "cortex_compactor_block_max_time_delta_seconds", + Help: "Difference between now and the max time of a block being compacted in seconds.", + Buckets: prometheus.LinearBuckets(86400, 43200, 8), // 1 to 5 days, in 12 hour intervals + }), + } +} + +type ownCompactionJobFunc func(job *Job) (bool, error) + +// ownAllJobs is a ownCompactionJobFunc that always return true. +var ownAllJobs = func(job *Job) (bool, error) { + return true, nil +} + +// BucketCompactor compacts blocks in a bucket. +type BucketCompactor struct { + logger log.Logger + sy *Syncer + grouper Grouper + comp Compactor + planner Planner + compactDir string + bkt objstore.Bucket + concurrency int + skipBlocksWithOutOfOrderChunks bool + ownJob ownCompactionJobFunc + sortJobs JobsOrderFunc + waitPeriod time.Duration + blockSyncConcurrency int + metrics *BucketCompactorMetrics +} + +// NewBucketCompactor creates a new bucket compactor. +func NewBucketCompactor( + logger log.Logger, + sy *Syncer, + grouper Grouper, + planner Planner, + comp Compactor, + compactDir string, + bkt objstore.Bucket, + concurrency int, + skipBlocksWithOutOfOrderChunks bool, + ownJob ownCompactionJobFunc, + sortJobs JobsOrderFunc, + waitPeriod time.Duration, + blockSyncConcurrency int, + metrics *BucketCompactorMetrics, +) (*BucketCompactor, error) { + if concurrency <= 0 { + return nil, errors.Errorf("invalid concurrency level (%d), concurrency level must be > 0", concurrency) + } + return &BucketCompactor{ + logger: logger, + sy: sy, + grouper: grouper, + planner: planner, + comp: comp, + compactDir: compactDir, + bkt: bkt, + concurrency: concurrency, + skipBlocksWithOutOfOrderChunks: skipBlocksWithOutOfOrderChunks, + ownJob: ownJob, + sortJobs: sortJobs, + waitPeriod: waitPeriod, + blockSyncConcurrency: blockSyncConcurrency, + metrics: metrics, + }, nil +} + +// Compact runs compaction over bucket. +// If maxCompactionTime is positive then after this time no more new compactions are started. +func (c *BucketCompactor) Compact(ctx context.Context, maxCompactionTime time.Duration) (rerr error) { + defer func() { + // Do not remove the compactDir if an error has occurred + // because potentially on the next run we would not have to download + // everything again. + if rerr != nil { + return + } + if err := os.RemoveAll(c.compactDir); err != nil { + level.Error(c.logger).Log("msg", "failed to remove compaction work directory", "path", c.compactDir, "err", err) + } + }() + + var maxCompactionTimeChan <-chan time.Time + if maxCompactionTime > 0 { + maxCompactionTimeChan = time.After(maxCompactionTime) + } + + // Loop over bucket and compact until there's no work left. + for { + var ( + wg sync.WaitGroup + workCtx, workCtxCancel = context.WithCancel(ctx) + jobChan = make(chan *Job) + errChan = make(chan error, c.concurrency) + finishedAllJobs = true + mtx sync.Mutex + ) + defer workCtxCancel() + + // Set up workers who will compact the jobs when the jobs are ready. + // They will compact available jobs until they encounter an error, after which they will stop. + for i := 0; i < c.concurrency; i++ { + wg.Add(1) + go func() { + defer wg.Done() + for g := range jobChan { + // Ensure the job is still owned by the current compactor instance. + // If not, we shouldn't run it because another compactor instance may already + // process it (or will do it soon). + if ok, err := c.ownJob(g); err != nil { + level.Info(c.logger).Log("msg", "skipped compaction because unable to check whether the job is owned by the compactor instance", "groupKey", g.Key(), "err", err) + continue + } else if !ok { + level.Info(c.logger).Log("msg", "skipped compaction because job is not owned by the compactor instance anymore", "groupKey", g.Key()) + continue + } + + c.metrics.groupCompactionRunsStarted.Inc() + + shouldRerunJob, compactedBlockIDs, err := c.runCompactionJob(workCtx, g) + if err == nil { + c.metrics.groupCompactionRunsCompleted.Inc() + if hasNonZeroULIDs(compactedBlockIDs) { + c.metrics.groupCompactions.Inc() + } + + if shouldRerunJob { + mtx.Lock() + finishedAllJobs = false + mtx.Unlock() + } + continue + } + + // At this point the compaction has failed. + c.metrics.groupCompactionRunsFailed.Inc() + + if IsIssue347Error(err) { + if err := RepairIssue347(workCtx, c.logger, c.bkt, c.sy.metrics.blocksMarkedForDeletion, err); err == nil { + mtx.Lock() + finishedAllJobs = 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, + block.OutOfOrderChunksNoCompactReason, + "OutofOrderChunk: marking block with out-of-order series/chunks to as no compact to unblock compaction", c.metrics.blocksMarkedForNoCompact); err == nil { + mtx.Lock() + finishedAllJobs = false + mtx.Unlock() + continue + } + } + errChan <- errors.Wrapf(err, "group %s", g.Key()) + return + } + }() + } + + level.Info(c.logger).Log("msg", "start sync of metas") + if err := c.sy.SyncMetas(ctx); err != nil { + return errors.Wrap(err, "sync") + } + + level.Info(c.logger).Log("msg", "start of GC") + // Blocks that were compacted are garbage collected after each Compaction. + // However if compactor crashes we need to resolve those on startup. + if err := c.sy.GarbageCollect(ctx); err != nil { + return errors.Wrap(err, "blocks garbage collect") + } + + jobs, err := c.grouper.Groups(c.sy.Metas()) + if err != nil { + return errors.Wrap(err, "build compaction jobs") + } + + // There is another check just before we start processing the job, but we can avoid sending it + // to the goroutine in the first place. + jobs, err = c.filterOwnJobs(jobs) + if err != nil { + return err + } + + // Record the difference between now and the max time for a block being compacted. This + // is used to detect compactors not being able to keep up with the rate of blocks being + // created. The idea is that most blocks should be for within 24h or 48h. + now := time.Now() + for _, delta := range c.blockMaxTimeDeltas(now, jobs) { + c.metrics.blocksMaxTimeDelta.Observe(delta) + } + + // Skip jobs for which the wait period hasn't been honored yet. + jobs = c.filterJobsByWaitPeriod(ctx, jobs) + + // Sort jobs based on the configured ordering algorithm. + jobs = c.sortJobs(jobs) + + ignoreDirs := []string{} + for _, gr := range jobs { + for _, grID := range gr.IDs() { + ignoreDirs = append(ignoreDirs, filepath.Join(gr.Key(), grID.String())) + } + } + + if err := runutil.DeleteAll(c.compactDir, ignoreDirs...); err != nil { + level.Warn(c.logger).Log("msg", "failed deleting non-compaction job directories/files, some disk space usage might have leaked. Continuing", "err", err, "dir", c.compactDir) + } + + level.Info(c.logger).Log("msg", "start of compactions") + + maxCompactionTimeReached := false + // Send all jobs found during this pass to the compaction workers. + var jobErrs multierror.MultiError + jobLoop: + for _, g := range jobs { + select { + case jobErr := <-errChan: + jobErrs.Add(jobErr) + break jobLoop + case jobChan <- g: + case <-maxCompactionTimeChan: + maxCompactionTimeReached = true + level.Info(c.logger).Log("msg", "max compaction time reached, no more compactions will be started") + break jobLoop + } + } + close(jobChan) + wg.Wait() + + // Collect any other error reported by the workers, or any error reported + // while we were waiting for the last batch of jobs to run the compaction. + close(errChan) + for jobErr := range errChan { + jobErrs.Add(jobErr) + } + + workCtxCancel() + if len(jobErrs) > 0 { + return jobErrs.Err() + } + + if maxCompactionTimeReached || finishedAllJobs { + break + } + } + level.Info(c.logger).Log("msg", "compaction iterations done") + return nil +} + +// blockMaxTimeDeltas returns a slice of the difference between now and the MaxTime of each +// block that will be compacted as part of the provided jobs, in seconds. +func (c *BucketCompactor) blockMaxTimeDeltas(now time.Time, jobs []*Job) []float64 { + var out []float64 + + for _, j := range jobs { + for _, m := range j.Metas() { + out = append(out, now.Sub(time.UnixMilli(m.MaxTime)).Seconds()) + } + } + + return out +} + +func (c *BucketCompactor) filterOwnJobs(jobs []*Job) ([]*Job, error) { + for ix := 0; ix < len(jobs); { + // Skip any job which doesn't belong to this compactor instance. + if ok, err := c.ownJob(jobs[ix]); err != nil { + return nil, errors.Wrap(err, "ownJob") + } else if !ok { + jobs = append(jobs[:ix], jobs[ix+1:]...) + } else { + ix++ + } + } + return jobs, nil +} + +// filterJobsByWaitPeriod filters out jobs for which the configured wait period hasn't been honored yet. +func (c *BucketCompactor) filterJobsByWaitPeriod(ctx context.Context, jobs []*Job) []*Job { + for i := 0; i < len(jobs); { + if elapsed, notElapsedBlock, err := jobWaitPeriodElapsed(ctx, jobs[i], c.waitPeriod, c.bkt); err != nil { + level.Warn(c.logger).Log("msg", "not enforcing compaction wait period because the check if compaction job contains recently uploaded blocks has failed", "groupKey", jobs[i].Key(), "err", err) + + // Keep the job. + i++ + } else if !elapsed { + level.Info(c.logger).Log("msg", "skipping compaction job because blocks in this job were uploaded too recently (within wait period)", "groupKey", jobs[i].Key(), "waitPeriodNotElapsedFor", notElapsedBlock.String()) + jobs = append(jobs[:i], jobs[i+1:]...) + } else { + i++ + } + } + + return jobs +} + +var _ block.MetadataFilter = &NoCompactionMarkFilter{} + +// NoCompactionMarkFilter is a block.Fetcher filter that finds all blocks with no-compact marker files, and optionally +// removes them from synced metas. +type NoCompactionMarkFilter struct { + bkt objstore.InstrumentedBucketReader + noCompactMarkedMap map[ulid.ULID]struct{} + removeNoCompactBlocks bool +} + +// NewNoCompactionMarkFilter creates NoCompactionMarkFilter. +func NewNoCompactionMarkFilter(bkt objstore.InstrumentedBucketReader, removeNoCompactBlocks bool) *NoCompactionMarkFilter { + return &NoCompactionMarkFilter{ + bkt: bkt, + removeNoCompactBlocks: removeNoCompactBlocks, + } +} + +// NoCompactMarkedBlocks returns block ids that were marked for no compaction. +// It is safe to call this method only after Filter has finished, and it is also safe to manipulate the map between calls to Filter. +func (f *NoCompactionMarkFilter) NoCompactMarkedBlocks() map[ulid.ULID]struct{} { + return f.noCompactMarkedMap +} + +// Filter finds blocks that should not be compacted, and fills f.noCompactMarkedMap. If f.removeNoCompactBlocks is true, +// blocks are also removed from metas. (Thanos version of the filter doesn't do removal). +func (f *NoCompactionMarkFilter) Filter(ctx context.Context, metas map[ulid.ULID]*block.Meta, synced block.GaugeVec) error { + noCompactMarkedMap := make(map[ulid.ULID]struct{}) + + // Find all no-compact markers in the storage. + err := f.bkt.Iter(ctx, block.MarkersPathname+"/", func(name string) error { + if err := ctx.Err(); err != nil { + return err + } + + if blockID, ok := block.IsNoCompactMarkFilename(path.Base(name)); ok { + _, exists := metas[blockID] + if exists { + noCompactMarkedMap[blockID] = struct{}{} + synced.WithLabelValues(block.MarkedForNoCompactionMeta).Inc() + + if f.removeNoCompactBlocks { + delete(metas, blockID) + } + } + + } + return nil + }) + if err != nil { + return errors.Wrap(err, "list block no-compact marks") + } + + f.noCompactMarkedMap = noCompactMarkedMap + return nil +} + +func hasNonZeroULIDs(ids []ulid.ULID) bool { + for _, id := range ids { + if id != (ulid.ULID{}) { + return true + } + } + + return false +} diff --git a/pkg/compactor/bucket_compactor_e2e_test.go b/pkg/compactor/bucket_compactor_e2e_test.go new file mode 100644 index 0000000000..9c2018722a --- /dev/null +++ b/pkg/compactor/bucket_compactor_e2e_test.go @@ -0,0 +1,848 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/grafana/mimir/blob/main/pkg/compactor/bucket_compactor_e2e_test.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Cortex Authors. + +package compactor + +import ( + "bytes" + "context" + "encoding/json" + "fmt" + "math/rand" + "os" + "path" + "path/filepath" + "runtime" + "sort" + "strings" + "testing" + "time" + + "github.com/go-kit/log" + "github.com/grafana/dskit/runutil" + "github.com/oklog/ulid" + "github.com/pkg/errors" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" + promtest "github.com/prometheus/client_golang/prometheus/testutil" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/tsdb" + "github.com/prometheus/prometheus/tsdb/chunkenc" + "github.com/prometheus/prometheus/tsdb/chunks" + "github.com/prometheus/prometheus/tsdb/index" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/thanos-io/objstore" + "github.com/thanos-io/objstore/providers/filesystem" + "golang.org/x/exp/slices" + "golang.org/x/sync/errgroup" + + "github.com/grafana/mimir/pkg/storage/tsdb/block" +) + +func TestSyncer_GarbageCollect_e2e(t *testing.T) { + foreachStore(t, func(t *testing.T, bkt objstore.Bucket) { + // Use bucket with global markers to make sure that our custom filters work correctly. + bkt = block.BucketWithGlobalMarkers(bkt) + + ctx, cancel := context.WithTimeout(context.Background(), 120*time.Second) + defer cancel() + + // Generate 10 source block metas and construct higher level blocks + // that are higher compactions of them. + var metas []*block.Meta + var ids []ulid.ULID + + for i := 0; i < 10; i++ { + var m block.Meta + + m.Version = 1 + m.ULID = ulid.MustNew(uint64(i), nil) + m.Compaction.Sources = []ulid.ULID{m.ULID} + m.Compaction.Level = 1 + m.MinTime = 0 + m.MaxTime = 2 * time.Hour.Milliseconds() + + ids = append(ids, m.ULID) + metas = append(metas, &m) + } + + var m1 block.Meta + m1.Version = 1 + m1.ULID = ulid.MustNew(100, nil) + m1.Compaction.Level = 2 + m1.Compaction.Sources = ids[:4] + m1.Thanos.Downsample.Resolution = 0 + + var m2 block.Meta + m2.Version = 1 + m2.ULID = ulid.MustNew(200, nil) + m2.Compaction.Level = 2 + m2.Compaction.Sources = ids[4:8] // last two source IDs is not part of a level 2 block. + m2.Thanos.Downsample.Resolution = 0 + + var m3 block.Meta + m3.Version = 1 + m3.ULID = ulid.MustNew(300, nil) + m3.Compaction.Level = 3 + m3.Compaction.Sources = ids[:9] // last source ID is not part of level 3 block. + m3.Thanos.Downsample.Resolution = 0 + m3.MinTime = 0 + m3.MaxTime = 2 * time.Hour.Milliseconds() + + var m4 block.Meta + m4.Version = 1 + m4.ULID = ulid.MustNew(400, nil) + m4.Compaction.Level = 2 + m4.Compaction.Sources = ids[9:] // covers the last block but is a different resolution. Must not trigger deletion. + m4.Thanos.Downsample.Resolution = 1000 + m4.MinTime = 0 + m4.MaxTime = 2 * time.Hour.Milliseconds() + + var m5 block.Meta + m5.Version = 1 + m5.ULID = ulid.MustNew(500, nil) + m5.Compaction.Level = 2 + m5.Compaction.Sources = ids[8:9] // built from block 8, but different resolution. Block 8 is already included in m3, can be deleted. + m5.Thanos.Downsample.Resolution = 1000 + m5.MinTime = 0 + m5.MaxTime = 2 * time.Hour.Milliseconds() + + // Create all blocks in the bucket. + for _, m := range append(metas, &m1, &m2, &m3, &m4, &m5) { + fmt.Println("create", m.ULID) + var buf bytes.Buffer + require.NoError(t, json.NewEncoder(&buf).Encode(&m)) + require.NoError(t, bkt.Upload(ctx, path.Join(m.ULID.String(), block.MetaFilename), &buf)) + } + + duplicateBlocksFilter := NewShardAwareDeduplicateFilter() + metaFetcher, err := block.NewMetaFetcher(nil, 32, objstore.WithNoopInstr(bkt), "", nil, []block.MetadataFilter{ + duplicateBlocksFilter, + }) + require.NoError(t, err) + + blocksMarkedForDeletion := promauto.With(nil).NewCounter(prometheus.CounterOpts{}) + sy, err := NewMetaSyncer(nil, nil, bkt, metaFetcher, duplicateBlocksFilter, blocksMarkedForDeletion) + require.NoError(t, err) + + // Do one initial synchronization with the bucket. + require.NoError(t, sy.SyncMetas(ctx)) + require.NoError(t, sy.GarbageCollect(ctx)) + + var rem []ulid.ULID + err = bkt.Iter(ctx, "", func(n string) error { + id, ok := block.IsBlockDir(n) + if !ok { + return nil + } + deletionMarkFile := path.Join(id.String(), block.DeletionMarkFilename) + + exists, err := bkt.Exists(ctx, deletionMarkFile) + if err != nil { + return err + } + if !exists { + rem = append(rem, id) + } + return nil + }) + require.NoError(t, err) + + sort.Slice(rem, func(i, j int) bool { + return rem[i].Compare(rem[j]) < 0 + }) + + // Only the level 3 block, the last source block in both resolutions should be left. + assert.Equal(t, []ulid.ULID{metas[9].ULID, m3.ULID, m4.ULID, m5.ULID}, rem) + + // After another sync the changes should also be reflected in the local groups. + require.NoError(t, sy.SyncMetas(ctx)) + require.NoError(t, sy.GarbageCollect(ctx)) + + // Only the level 3 block, the last source block in both resolutions should be left. + grouper := NewSplitAndMergeGrouper("user-1", []int64{2 * time.Hour.Milliseconds()}, 0, 0, log.NewNopLogger()) + groups, err := grouper.Groups(sy.Metas()) + require.NoError(t, err) + + assert.Equal(t, "0@17241709254077376921-merge--0-7200000", groups[0].Key()) + assert.Equal(t, []ulid.ULID{metas[9].ULID, m3.ULID}, groups[0].IDs()) + + assert.Equal(t, "1000@17241709254077376921-merge--0-7200000", groups[1].Key()) + assert.Equal(t, []ulid.ULID{m4.ULID, m5.ULID}, groups[1].IDs()) + }) +} + +func TestGroupCompactE2E(t *testing.T) { + foreachStore(t, func(t *testing.T, bkt objstore.Bucket) { + // Use bucket with global markers to make sure that our custom filters work correctly. + bkt = block.BucketWithGlobalMarkers(bkt) + + ctx, cancel := context.WithTimeout(context.Background(), 120*time.Second) + defer cancel() + + // Create fresh, empty directory for actual test. + dir := t.TempDir() + + // Start dir checker... we make sure that "dir" only contains group subdirectories during compaction, + // and not any block directories. Dir checker stops when context is canceled, or on first error, + // in which case error is logger and test is failed. (We cannot use Fatal or FailNow from a goroutine). + go func() { + for ctx.Err() == nil { + fs, err := os.ReadDir(dir) + if err != nil && !os.IsNotExist(err) { + t.Log("error while listing directory", dir) + t.Fail() + return + } + + for _, fi := range fs { + // Suffix used by Prometheus LeveledCompactor when doing compaction. + toCheck := strings.TrimSuffix(fi.Name(), ".tmp-for-creation") + + _, err := ulid.Parse(toCheck) + if err == nil { + t.Log("found block directory in main compaction directory", fi.Name()) + t.Fail() + return + } + } + + select { + case <-time.After(100 * time.Millisecond): + continue + case <-ctx.Done(): + return + } + } + }() + + logger := log.NewLogfmtLogger(os.Stderr) + + reg := prometheus.NewRegistry() + + duplicateBlocksFilter := NewShardAwareDeduplicateFilter() + noCompactMarkerFilter := NewNoCompactionMarkFilter(objstore.WithNoopInstr(bkt), true) + metaFetcher, err := block.NewMetaFetcher(nil, 32, objstore.WithNoopInstr(bkt), "", nil, []block.MetadataFilter{ + duplicateBlocksFilter, + noCompactMarkerFilter, + }) + require.NoError(t, err) + + blocksMarkedForDeletion := promauto.With(nil).NewCounter(prometheus.CounterOpts{}) + sy, err := NewMetaSyncer(nil, nil, bkt, metaFetcher, duplicateBlocksFilter, blocksMarkedForDeletion) + require.NoError(t, err) + + comp, err := tsdb.NewLeveledCompactor(ctx, reg, logger, []int64{1000, 3000}, nil, nil, true) + require.NoError(t, err) + + planner := NewSplitAndMergePlanner([]int64{1000, 3000}) + grouper := NewSplitAndMergeGrouper("user-1", []int64{1000, 3000}, 0, 0, logger) + metrics := NewBucketCompactorMetrics(blocksMarkedForDeletion, prometheus.NewPedanticRegistry()) + bComp, err := NewBucketCompactor(logger, sy, grouper, planner, comp, dir, bkt, 2, true, ownAllJobs, sortJobsByNewestBlocksFirst, 0, 4, metrics) + require.NoError(t, err) + + // Compaction on empty should not fail. + require.NoError(t, bComp.Compact(ctx, 0), 0) + assert.Equal(t, 0.0, promtest.ToFloat64(sy.metrics.blocksMarkedForDeletion)) + assert.Equal(t, 0.0, promtest.ToFloat64(sy.metrics.garbageCollectionFailures)) + assert.Equal(t, 0.0, promtest.ToFloat64(metrics.blocksMarkedForNoCompact)) + assert.Equal(t, 0.0, promtest.ToFloat64(metrics.groupCompactions)) + assert.Equal(t, 0.0, promtest.ToFloat64(metrics.groupCompactionRunsStarted)) + assert.Equal(t, 0.0, promtest.ToFloat64(metrics.groupCompactionRunsCompleted)) + assert.Equal(t, 0.0, promtest.ToFloat64(metrics.groupCompactionRunsFailed)) + + _, err = os.Stat(dir) + assert.True(t, os.IsNotExist(err), "dir %s should be remove after compaction.", dir) + + // Test label name with slash, regression: https://github.com/thanos-io/thanos/issues/1661. + extLabels := labels.FromStrings("e1", "1/weird") + extLabels2 := labels.FromStrings("e1", "1") + metas := createAndUpload(t, bkt, []blockgenSpec{ + { + numSamples: 100, mint: 500, maxt: 1000, extLset: extLabels, res: 124, + series: []labels.Labels{ + labels.FromStrings("a", "1"), + labels.FromStrings("a", "2", "b", "2"), + labels.FromStrings("a", "3"), + labels.FromStrings("a", "4"), + }, + }, + { + numSamples: 100, mint: 2000, maxt: 3000, extLset: extLabels, res: 124, + series: []labels.Labels{ + labels.FromStrings("a", "3"), + labels.FromStrings("a", "4"), + labels.FromStrings("a", "5"), + labels.FromStrings("a", "6"), + }, + }, + // Mix order to make sure compactor is able to deduct min time / max time. + // Currently TSDB does not produces empty blocks (see: https://github.com/prometheus/tsdb/pull/374). However before v2.7.0 it was + // so we still want to mimick this case as close as possible. + { + mint: 1000, maxt: 2000, extLset: extLabels, res: 124, + // Empty block. + }, + // Due to TSDB compaction delay (not compacting fresh block), we need one more block to be pushed to trigger compaction. + { + numSamples: 100, mint: 3000, maxt: 4000, extLset: extLabels, res: 124, + series: []labels.Labels{ + labels.FromStrings("a", "7"), + }, + }, + // Extra block for "distraction" for different resolution and one for different labels. + { + numSamples: 100, mint: 5000, maxt: 6000, extLset: labels.FromStrings("e1", "2"), res: 124, + series: []labels.Labels{ + labels.FromStrings("a", "7"), + }, + }, + // Extra block for "distraction" for different resolution and one for different labels. + { + numSamples: 100, mint: 4000, maxt: 5000, extLset: extLabels, res: 0, + series: []labels.Labels{ + labels.FromStrings("a", "7"), + }, + }, + // Second group (extLabels2). + { + numSamples: 100, mint: 2000, maxt: 3000, extLset: extLabels2, res: 124, + series: []labels.Labels{ + labels.FromStrings("a", "3"), + labels.FromStrings("a", "4"), + labels.FromStrings("a", "6"), + }, + }, + { + numSamples: 100, mint: 0, maxt: 1000, extLset: extLabels2, res: 124, + series: []labels.Labels{ + labels.FromStrings("a", "1"), + labels.FromStrings("a", "2", "b", "2"), + labels.FromStrings("a", "3"), + labels.FromStrings("a", "4"), + }, + }, + // Due to TSDB compaction delay (not compacting fresh block), we need one more block to be pushed to trigger compaction. + { + numSamples: 100, mint: 3000, maxt: 4000, extLset: extLabels2, res: 124, + series: []labels.Labels{ + labels.FromStrings("a", "7"), + }, + }, + }, []blockgenSpec{ + { + numSamples: 100, mint: 0, maxt: 499, extLset: extLabels, res: 124, + series: []labels.Labels{ + labels.FromStrings("a", "1"), + labels.FromStrings("a", "1", "b", "2"), + labels.FromStrings("a", "3"), + labels.FromStrings("a", "4"), + }, + }, + }) + + require.NoError(t, bComp.Compact(ctx, 0), 0) + assert.Equal(t, 5.0, promtest.ToFloat64(sy.metrics.blocksMarkedForDeletion)) + assert.Equal(t, 1.0, promtest.ToFloat64(metrics.blocksMarkedForNoCompact)) + assert.Equal(t, 0.0, promtest.ToFloat64(sy.metrics.garbageCollectionFailures)) + assert.Equal(t, 2.0, promtest.ToFloat64(metrics.groupCompactions)) + assert.Equal(t, 3.0, promtest.ToFloat64(metrics.groupCompactionRunsStarted)) + assert.Equal(t, 2.0, promtest.ToFloat64(metrics.groupCompactionRunsCompleted)) + assert.Equal(t, 1.0, promtest.ToFloat64(metrics.groupCompactionRunsFailed)) + + _, err = os.Stat(dir) + assert.True(t, os.IsNotExist(err), "dir %s should be remove after compaction.", dir) + + // Check object storage. All blocks that were included in new compacted one should be removed. New compacted ones + // are present and looks as expected. + nonCompactedExpected := map[ulid.ULID]bool{ + metas[3].ULID: false, + metas[4].ULID: false, + metas[5].ULID: false, + metas[8].ULID: false, + metas[9].ULID: false, + } + others := map[string]block.Meta{} + require.NoError(t, bkt.Iter(ctx, "", func(n string) error { + id, ok := block.IsBlockDir(n) + if !ok { + return nil + } + + if _, ok := nonCompactedExpected[id]; ok { + nonCompactedExpected[id] = true + return nil + } + + meta, err := block.DownloadMeta(ctx, logger, bkt, id) + if err != nil { + return err + } + + others[DefaultGroupKey(meta.Thanos)] = meta + return nil + })) + + for id, found := range nonCompactedExpected { + assert.True(t, found, "not found expected block %s", id.String()) + } + + // We expect two compacted blocks only outside of what we expected in `nonCompactedExpected`. + assert.Equal(t, 2, len(others)) + { + meta, ok := others[defaultGroupKey(124, extLabels)] + assert.True(t, ok, "meta not found") + + assert.Equal(t, int64(500), meta.MinTime) + assert.Equal(t, int64(3000), meta.MaxTime) + assert.Equal(t, uint64(6), meta.Stats.NumSeries) + assert.Equal(t, uint64(2*4*100), meta.Stats.NumSamples) // Only 2 times 4*100 because one block was empty. + assert.Equal(t, 2, meta.Compaction.Level) + assert.Equal(t, []ulid.ULID{metas[0].ULID, metas[1].ULID, metas[2].ULID}, meta.Compaction.Sources) + + // Check thanos meta. + assert.True(t, labels.Equal(extLabels, labels.FromMap(meta.Thanos.Labels)), "ext labels does not match") + assert.Equal(t, int64(124), meta.Thanos.Downsample.Resolution) + assert.True(t, len(meta.Thanos.SegmentFiles) > 0, "compacted blocks have segment files set") + } + { + meta, ok := others[defaultGroupKey(124, extLabels2)] + assert.True(t, ok, "meta not found") + + assert.Equal(t, int64(0), meta.MinTime) + assert.Equal(t, int64(3000), meta.MaxTime) + assert.Equal(t, uint64(5), meta.Stats.NumSeries) + assert.Equal(t, uint64(2*4*100-100), meta.Stats.NumSamples) + assert.Equal(t, 2, meta.Compaction.Level) + assert.Equal(t, []ulid.ULID{metas[6].ULID, metas[7].ULID}, meta.Compaction.Sources) + + // Check thanos meta. + assert.True(t, labels.Equal(extLabels2, labels.FromMap(meta.Thanos.Labels)), "ext labels does not match") + assert.Equal(t, int64(124), meta.Thanos.Downsample.Resolution) + assert.True(t, len(meta.Thanos.SegmentFiles) > 0, "compacted blocks have segment files set") + } + }) +} + +type blockgenSpec struct { + mint, maxt int64 + series []labels.Labels + numSamples int + extLset labels.Labels + res int64 +} + +func createAndUpload(t testing.TB, bkt objstore.Bucket, blocks []blockgenSpec, blocksWithOutOfOrderChunks []blockgenSpec) (metas []*block.Meta) { + prepareDir := t.TempDir() + + ctx, cancel := context.WithTimeout(context.Background(), 20*time.Second) + defer cancel() + + for _, b := range blocks { + id, meta := createBlock(ctx, t, prepareDir, b) + metas = append(metas, meta) + require.NoError(t, block.Upload(ctx, log.NewNopLogger(), bkt, filepath.Join(prepareDir, id.String()), nil)) + } + for _, b := range blocksWithOutOfOrderChunks { + id, meta := createBlock(ctx, t, prepareDir, b) + + err := putOutOfOrderIndex(filepath.Join(prepareDir, id.String()), b.mint, b.maxt) + require.NoError(t, err) + + metas = append(metas, meta) + require.NoError(t, block.Upload(ctx, log.NewNopLogger(), bkt, filepath.Join(prepareDir, id.String()), nil)) + } + + return metas +} + +func createBlock(ctx context.Context, t testing.TB, prepareDir string, b blockgenSpec) (id ulid.ULID, meta *block.Meta) { + var err error + if b.numSamples == 0 { + id, err = createEmptyBlock(prepareDir, b.mint, b.maxt, b.extLset, b.res) + } else { + id, err = createBlockWithOptions(ctx, prepareDir, b.series, b.numSamples, b.mint, b.maxt, b.extLset, b.res, false) + } + require.NoError(t, err) + + meta, err = block.ReadMetaFromDir(filepath.Join(prepareDir, id.String())) + require.NoError(t, err) + return +} + +// Regression test for Thanos issue #2459. +func TestGarbageCollectDoesntCreateEmptyBlocksWithDeletionMarksOnly(t *testing.T) { + logger := log.NewLogfmtLogger(os.Stderr) + + foreachStore(t, func(t *testing.T, bkt objstore.Bucket) { + // Use bucket with global markers to make sure that our custom filters work correctly. + bkt = block.BucketWithGlobalMarkers(bkt) + + ctx, cancel := context.WithTimeout(context.Background(), 120*time.Second) + defer cancel() + + // Generate two blocks, and then another block that covers both of them. + var metas []*block.Meta + var ids []ulid.ULID + + for i := 0; i < 2; i++ { + var m block.Meta + + m.Version = 1 + m.ULID = ulid.MustNew(uint64(i), nil) + m.Compaction.Sources = []ulid.ULID{m.ULID} + m.Compaction.Level = 1 + + ids = append(ids, m.ULID) + metas = append(metas, &m) + } + + var m1 block.Meta + m1.Version = 1 + m1.ULID = ulid.MustNew(100, nil) + m1.Compaction.Level = 2 + m1.Compaction.Sources = ids + m1.Thanos.Downsample.Resolution = 0 + + // Create all blocks in the bucket. + for _, m := range append(metas, &m1) { + fmt.Println("create", m.ULID) + var buf bytes.Buffer + require.NoError(t, json.NewEncoder(&buf).Encode(&m)) + require.NoError(t, bkt.Upload(ctx, path.Join(m.ULID.String(), block.MetaFilename), &buf)) + } + + blocksMarkedForDeletion := promauto.With(nil).NewCounter(prometheus.CounterOpts{}) + + duplicateBlocksFilter := NewShardAwareDeduplicateFilter() + metaFetcher, err := block.NewMetaFetcher(nil, 32, objstore.WithNoopInstr(bkt), "", nil, []block.MetadataFilter{ + duplicateBlocksFilter, + }) + require.NoError(t, err) + + sy, err := NewMetaSyncer(nil, nil, bkt, metaFetcher, duplicateBlocksFilter, blocksMarkedForDeletion) + require.NoError(t, err) + + // Do one initial synchronization with the bucket. + require.NoError(t, sy.SyncMetas(ctx)) + require.NoError(t, sy.GarbageCollect(ctx)) + + rem, err := listBlocksMarkedForDeletion(ctx, bkt) + require.NoError(t, err) + + sort.Slice(rem, func(i, j int) bool { + return rem[i].Compare(rem[j]) < 0 + }) + + assert.Equal(t, ids, rem) + + // Delete source blocks. + for _, id := range ids { + require.NoError(t, block.Delete(ctx, logger, bkt, id)) + } + + // After another garbage-collect, we should not find new blocks that are deleted with new deletion mark files. + require.NoError(t, sy.SyncMetas(ctx)) + require.NoError(t, sy.GarbageCollect(ctx)) + + rem, err = listBlocksMarkedForDeletion(ctx, bkt) + require.NoError(t, err) + assert.Equal(t, 0, len(rem)) + }) +} + +func listBlocksMarkedForDeletion(ctx context.Context, bkt objstore.Bucket) ([]ulid.ULID, error) { + var rem []ulid.ULID + err := bkt.Iter(ctx, "", func(n string) error { + id, ok := block.IsBlockDir(n) + if !ok { + return nil + } + deletionMarkFile := path.Join(id.String(), block.DeletionMarkFilename) + + exists, err := bkt.Exists(ctx, deletionMarkFile) + if err != nil { + return err + } + if exists { + rem = append(rem, id) + } + return nil + }) + return rem, err +} + +func foreachStore(t *testing.T, testFn func(t *testing.T, bkt objstore.Bucket)) { + t.Parallel() + + // Mandatory Inmem. Not parallel, to detect problem early. + if ok := t.Run("inmem", func(t *testing.T) { + testFn(t, objstore.NewInMemBucket()) + }); !ok { + return + } + + // Mandatory Filesystem. + t.Run("filesystem", func(t *testing.T) { + t.Parallel() + + dir := t.TempDir() + + b, err := filesystem.NewBucket(dir) + require.NoError(t, err) + testFn(t, b) + }) +} + +// createEmptyBlock produces empty block like it was the case before fix: https://github.com/prometheus/tsdb/pull/374. +// (Prometheus pre v2.7.0). +func createEmptyBlock(dir string, mint, maxt int64, extLset labels.Labels, resolution int64) (ulid.ULID, error) { + entropy := rand.New(rand.NewSource(time.Now().UnixNano())) + uid := ulid.MustNew(ulid.Now(), entropy) + + if err := os.Mkdir(path.Join(dir, uid.String()), os.ModePerm); err != nil { + return ulid.ULID{}, errors.Wrap(err, "close index") + } + + if err := os.Mkdir(path.Join(dir, uid.String(), "chunks"), os.ModePerm); err != nil { + return ulid.ULID{}, errors.Wrap(err, "close index") + } + + w, err := index.NewWriter(context.Background(), path.Join(dir, uid.String(), "index")) + if err != nil { + return ulid.ULID{}, errors.Wrap(err, "new index") + } + + if err := w.Close(); err != nil { + return ulid.ULID{}, errors.Wrap(err, "close index") + } + + m := tsdb.BlockMeta{ + Version: 1, + ULID: uid, + MinTime: mint, + MaxTime: maxt, + Compaction: tsdb.BlockMetaCompaction{ + Level: 1, + Sources: []ulid.ULID{uid}, + }, + } + b, err := json.Marshal(&m) + if err != nil { + return ulid.ULID{}, err + } + + if err := os.WriteFile(path.Join(dir, uid.String(), "meta.json"), b, os.ModePerm); err != nil { + return ulid.ULID{}, errors.Wrap(err, "saving meta.json") + } + + if _, err = block.InjectThanosMeta(log.NewNopLogger(), filepath.Join(dir, uid.String()), block.ThanosMeta{ + Labels: extLset.Map(), + Downsample: block.ThanosDownsample{Resolution: resolution}, + Source: block.TestSource, + }, nil); err != nil { + return ulid.ULID{}, errors.Wrap(err, "finalize block") + } + + return uid, nil +} + +func createBlockWithOptions( + ctx context.Context, + dir string, + series []labels.Labels, + numSamples int, + mint, maxt int64, + extLset labels.Labels, + resolution int64, + tombstones bool, +) (id ulid.ULID, err error) { + headOpts := tsdb.DefaultHeadOptions() + headOpts.ChunkDirRoot = filepath.Join(dir, "chunks") + headOpts.ChunkRange = 10000000000 + h, err := tsdb.NewHead(nil, nil, nil, nil, headOpts, nil) + if err != nil { + return id, errors.Wrap(err, "create head block") + } + defer func() { + runutil.CloseWithErrCapture(&err, h, "TSDB Head") + if e := os.RemoveAll(headOpts.ChunkDirRoot); e != nil { + err = errors.Wrap(e, "delete chunks dir") + } + }() + + var g errgroup.Group + timeStepSize := (maxt - mint) / int64(numSamples+1) + batchSize := len(series) / runtime.GOMAXPROCS(0) + + for len(series) > 0 { + l := batchSize + if len(series) < 1000 { + l = len(series) + } + batch := series[:l] + series = series[l:] + + g.Go(func() error { + t := mint + + for i := 0; i < numSamples; i++ { + app := h.Appender(ctx) + + for _, lset := range batch { + _, err := app.Append(0, lset, t, rand.Float64()) + if err != nil { + if rerr := app.Rollback(); rerr != nil { + err = errors.Wrapf(err, "rollback failed: %v", rerr) + } + + return errors.Wrap(err, "add sample") + } + } + if err := app.Commit(); err != nil { + return errors.Wrap(err, "commit") + } + t += timeStepSize + } + return nil + }) + } + if err := g.Wait(); err != nil { + return id, err + } + c, err := tsdb.NewLeveledCompactor(ctx, nil, log.NewNopLogger(), []int64{maxt - mint}, nil, nil, true) + if err != nil { + return id, errors.Wrap(err, "create compactor") + } + + id, err = c.Write(dir, h, mint, maxt, nil) + if err != nil { + return id, errors.Wrap(err, "write block") + } + + if id.Compare(ulid.ULID{}) == 0 { + return id, errors.Errorf("nothing to write, asked for %d samples", numSamples) + } + + blockDir := filepath.Join(dir, id.String()) + + if _, err = block.InjectThanosMeta(log.NewNopLogger(), blockDir, block.ThanosMeta{ + Labels: extLset.Map(), + Downsample: block.ThanosDownsample{Resolution: resolution}, + Source: block.TestSource, + Files: []block.File{}, + }, nil); err != nil { + return id, errors.Wrap(err, "finalize block") + } + + if !tombstones { + if err = os.Remove(filepath.Join(dir, id.String(), "tombstones")); err != nil { + return id, errors.Wrap(err, "remove tombstones") + } + } + + return id, nil +} + +var indexFilename = "index" + +type indexWriterSeries struct { + labels labels.Labels + chunks []chunks.Meta // series file offset of chunks +} + +type indexWriterSeriesSlice []*indexWriterSeries + +// putOutOfOrderIndex updates the index in blockDir with an index containing an out-of-order chunk +// copied from https://github.com/prometheus/prometheus/blob/b1ed4a0a663d0c62526312311c7529471abbc565/tsdb/index/index_test.go#L346 +func putOutOfOrderIndex(blockDir string, minTime int64, maxTime int64) error { + if minTime >= maxTime || minTime+4 >= maxTime { + return fmt.Errorf("minTime must be at least 4 less than maxTime to not create overlapping chunks") + } + + lbls := []labels.Labels{ + labels.FromStrings("lbl1", "1"), + } + + // Sort labels as the index writer expects series in sorted order. + sort.Sort(labels.Slice(lbls)) + + symbols := map[string]struct{}{} + for _, lset := range lbls { + lset.Range(func(l labels.Label) { + symbols[l.Name] = struct{}{} + symbols[l.Value] = struct{}{} + }) + } + + var input indexWriterSeriesSlice + + // Generate ChunkMetas for every label set. + for _, lset := range lbls { + var metas []chunks.Meta + // only need two chunks that are out-of-order + chk1 := chunks.Meta{ + MinTime: maxTime - 2, + MaxTime: maxTime - 1, + Ref: chunks.ChunkRef(rand.Uint64()), + Chunk: chunkenc.NewXORChunk(), + } + metas = append(metas, chk1) + chk2 := chunks.Meta{ + MinTime: minTime + 1, + MaxTime: minTime + 2, + Ref: chunks.ChunkRef(rand.Uint64()), + Chunk: chunkenc.NewXORChunk(), + } + metas = append(metas, chk2) + + input = append(input, &indexWriterSeries{ + labels: lset, + chunks: metas, + }) + } + + iw, err := index.NewWriter(context.Background(), filepath.Join(blockDir, indexFilename)) + if err != nil { + return err + } + + syms := []string{} + for s := range symbols { + syms = append(syms, s) + } + slices.Sort(syms) + for _, s := range syms { + if err := iw.AddSymbol(s); err != nil { + return err + } + } + + // Population procedure as done by compaction. + var ( + postings = index.NewMemPostings() + values = map[string]map[string]struct{}{} + ) + + for i, s := range input { + if err := iw.AddSeries(storage.SeriesRef(i), s.labels, s.chunks...); err != nil { + return err + } + + s.labels.Range(func(l labels.Label) { + valset, ok := values[l.Name] + if !ok { + valset = map[string]struct{}{} + values[l.Name] = valset + } + valset[l.Value] = struct{}{} + }) + postings.Add(storage.SeriesRef(i), s.labels) + } + + return iw.Close() +} diff --git a/pkg/compactor/bucket_compactor_test.go b/pkg/compactor/bucket_compactor_test.go new file mode 100644 index 0000000000..cadce93354 --- /dev/null +++ b/pkg/compactor/bucket_compactor_test.go @@ -0,0 +1,355 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/grafana/mimir/blob/main/pkg/compactor/bucket_compactor_test.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Cortex Authors. + +package compactor + +import ( + "context" + "fmt" + "strings" + "testing" + "time" + + "github.com/go-kit/log" + "github.com/oklog/ulid" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" + "github.com/prometheus/client_golang/prometheus/testutil" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/tsdb" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/thanos-io/objstore" + + "github.com/grafana/mimir/pkg/storage/tsdb/block" + "github.com/grafana/mimir/pkg/util/extprom" +) + +func TestGroupKey(t *testing.T) { + for _, tcase := range []struct { + input block.ThanosMeta + expected string + }{ + { + input: block.ThanosMeta{}, + expected: "0@17241709254077376921", + }, + { + input: block.ThanosMeta{ + Labels: map[string]string{}, + Downsample: block.ThanosDownsample{Resolution: 0}, + }, + expected: "0@17241709254077376921", + }, + { + input: block.ThanosMeta{ + Labels: map[string]string{"foo": "bar", "foo1": "bar2"}, + Downsample: block.ThanosDownsample{Resolution: 0}, + }, + expected: "0@2124638872457683483", + }, + { + input: block.ThanosMeta{ + Labels: map[string]string{`foo/some..thing/some.thing/../`: `a_b_c/bar-something-a\metric/a\x`}, + Downsample: block.ThanosDownsample{Resolution: 0}, + }, + expected: "0@16590761456214576373", + }, + } { + if ok := t.Run("", func(t *testing.T) { + assert.Equal(t, tcase.expected, DefaultGroupKey(tcase.input)) + }); !ok { + return + } + } +} + +func TestGroupMaxMinTime(t *testing.T) { + g := &Job{ + metasByMinTime: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{MinTime: 0, MaxTime: 10}}, + {BlockMeta: tsdb.BlockMeta{MinTime: 1, MaxTime: 20}}, + {BlockMeta: tsdb.BlockMeta{MinTime: 2, MaxTime: 30}}, + }, + } + + assert.Equal(t, int64(0), g.MinTime()) + assert.Equal(t, int64(30), g.MaxTime()) +} + +func TestBucketCompactor_FilterOwnJobs(t *testing.T) { + jobsFn := func() []*Job { + return []*Job{ + NewJob("user", "key1", labels.EmptyLabels(), 0, false, 0, ""), + NewJob("user", "key2", labels.EmptyLabels(), 0, false, 0, ""), + NewJob("user", "key3", labels.EmptyLabels(), 0, false, 0, ""), + NewJob("user", "key4", labels.EmptyLabels(), 0, false, 0, ""), + } + } + + tests := map[string]struct { + ownJob ownCompactionJobFunc + expectedJobs int + }{ + "should return all planned jobs if the compactor instance owns all of them": { + ownJob: func(job *Job) (bool, error) { + return true, nil + }, + expectedJobs: 4, + }, + "should return no jobs if the compactor instance owns none of them": { + ownJob: func(job *Job) (bool, error) { + return false, nil + }, + expectedJobs: 0, + }, + "should return some jobs if the compactor instance owns some of them": { + ownJob: func() ownCompactionJobFunc { + count := 0 + return func(job *Job) (bool, error) { + count++ + return count%2 == 0, nil + } + }(), + expectedJobs: 2, + }, + } + + m := NewBucketCompactorMetrics(promauto.With(nil).NewCounter(prometheus.CounterOpts{}), nil) + for testName, testCase := range tests { + t.Run(testName, func(t *testing.T) { + bc, err := NewBucketCompactor(log.NewNopLogger(), nil, nil, nil, nil, "", nil, 2, false, testCase.ownJob, nil, 0, 4, m) + require.NoError(t, err) + + res, err := bc.filterOwnJobs(jobsFn()) + + require.NoError(t, err) + assert.Len(t, res, testCase.expectedJobs) + }) + } +} + +func TestBlockMaxTimeDeltas(t *testing.T) { + j1 := NewJob("user", "key1", labels.EmptyLabels(), 0, false, 0, "") + require.NoError(t, j1.AppendMeta(&block.Meta{ + BlockMeta: tsdb.BlockMeta{ + MinTime: 1500002700159, + MaxTime: 1500002800159, + }, + })) + + j2 := NewJob("user", "key2", labels.EmptyLabels(), 0, false, 0, "") + require.NoError(t, j2.AppendMeta(&block.Meta{ + BlockMeta: tsdb.BlockMeta{ + MinTime: 1500002600159, + MaxTime: 1500002700159, + }, + })) + require.NoError(t, j2.AppendMeta(&block.Meta{ + BlockMeta: tsdb.BlockMeta{ + MinTime: 1500002700159, + MaxTime: 1500002800159, + }, + })) + + metrics := NewBucketCompactorMetrics(promauto.With(nil).NewCounter(prometheus.CounterOpts{}), nil) + now := time.UnixMilli(1500002900159) + bc, err := NewBucketCompactor(log.NewNopLogger(), nil, nil, nil, nil, "", nil, 2, false, nil, nil, 0, 4, metrics) + require.NoError(t, err) + + deltas := bc.blockMaxTimeDeltas(now, []*Job{j1, j2}) + assert.Equal(t, []float64{100, 200, 100}, deltas) +} + +func TestNoCompactionMarkFilter(t *testing.T) { + ctx := context.Background() + // Use bucket with global markers to make sure that our custom filters work correctly. + bkt := block.BucketWithGlobalMarkers(objstore.NewInMemBucket()) + + block1 := ulid.MustParse("01DTVP434PA9VFXSW2JK000001") // No mark file. + block2 := ulid.MustParse("01DTVP434PA9VFXSW2JK000002") // Marked for no-compaction + block3 := ulid.MustParse("01DTVP434PA9VFXSW2JK000003") // Has wrong version of marker file. + block4 := ulid.MustParse("01DTVP434PA9VFXSW2JK000004") // Has invalid marker file. + block5 := ulid.MustParse("01DTVP434PA9VFXSW2JK000005") // No mark file. + + for name, testFn := range map[string]func(t *testing.T, synced block.GaugeVec){ + "filter with no deletion of blocks marked for no-compaction": func(t *testing.T, synced block.GaugeVec) { + metas := map[ulid.ULID]*block.Meta{ + block1: blockMeta(block1.String(), 100, 200, nil), + block2: blockMeta(block2.String(), 200, 300, nil), // Has no-compaction marker. + block4: blockMeta(block4.String(), 400, 500, nil), // Invalid marker is still a marker, and block will be in NoCompactMarkedBlocks. + block5: blockMeta(block5.String(), 500, 600, nil), + } + + f := NewNoCompactionMarkFilter(objstore.WithNoopInstr(bkt), false) + require.NoError(t, f.Filter(ctx, metas, synced)) + + require.Contains(t, metas, block1) + require.Contains(t, metas, block2) + require.Contains(t, metas, block4) + require.Contains(t, metas, block5) + + require.Len(t, f.NoCompactMarkedBlocks(), 2) + require.Contains(t, f.NoCompactMarkedBlocks(), block2, block4) + + assert.Equal(t, 2.0, testutil.ToFloat64(synced.WithLabelValues(block.MarkedForNoCompactionMeta))) + }, + "filter with deletion enabled": func(t *testing.T, synced block.GaugeVec) { + metas := map[ulid.ULID]*block.Meta{ + block1: blockMeta(block1.String(), 100, 200, nil), + block2: blockMeta(block2.String(), 300, 300, nil), // Has no-compaction marker. + block4: blockMeta(block4.String(), 400, 500, nil), // Marker with invalid syntax is ignored. + block5: blockMeta(block5.String(), 500, 600, nil), + } + + f := NewNoCompactionMarkFilter(objstore.WithNoopInstr(bkt), true) + require.NoError(t, f.Filter(ctx, metas, synced)) + + require.Contains(t, metas, block1) + require.NotContains(t, metas, block2) // block2 was removed from metas. + require.NotContains(t, metas, block4) // block4 has invalid marker, but we don't check for marker content. + require.Contains(t, metas, block5) + + require.Len(t, f.NoCompactMarkedBlocks(), 2) + require.Contains(t, f.NoCompactMarkedBlocks(), block2) + require.Contains(t, f.NoCompactMarkedBlocks(), block4) + + assert.Equal(t, 2.0, testutil.ToFloat64(synced.WithLabelValues(block.MarkedForNoCompactionMeta))) + }, + "filter with deletion enabled, but canceled context": func(t *testing.T, synced block.GaugeVec) { + metas := map[ulid.ULID]*block.Meta{ + block1: blockMeta(block1.String(), 100, 200, nil), + block2: blockMeta(block2.String(), 200, 300, nil), + block3: blockMeta(block3.String(), 300, 400, nil), + block4: blockMeta(block4.String(), 400, 500, nil), + block5: blockMeta(block5.String(), 500, 600, nil), + } + + canceledCtx, cancel := context.WithCancel(context.Background()) + cancel() + + f := NewNoCompactionMarkFilter(objstore.WithNoopInstr(bkt), true) + require.Error(t, f.Filter(canceledCtx, metas, synced)) + + require.Contains(t, metas, block1) + require.Contains(t, metas, block2) + require.Contains(t, metas, block3) + require.Contains(t, metas, block4) + require.Contains(t, metas, block5) + + require.Empty(t, f.NoCompactMarkedBlocks()) + assert.Equal(t, 0.0, testutil.ToFloat64(synced.WithLabelValues(block.MarkedForNoCompactionMeta))) + }, + "filtering block with wrong marker version": func(t *testing.T, synced block.GaugeVec) { + metas := map[ulid.ULID]*block.Meta{ + block3: blockMeta(block3.String(), 300, 300, nil), // Has compaction marker with invalid version, but Filter doesn't check for that. + } + + f := NewNoCompactionMarkFilter(objstore.WithNoopInstr(bkt), true) + err := f.Filter(ctx, metas, synced) + require.NoError(t, err) + require.Empty(t, metas) + + assert.Equal(t, 1.0, testutil.ToFloat64(synced.WithLabelValues(block.MarkedForNoCompactionMeta))) + }, + } { + t.Run(name, func(t *testing.T) { + // Block 2 is marked for no-compaction. + require.NoError(t, block.MarkForNoCompact(ctx, log.NewNopLogger(), bkt, block2, block.OutOfOrderChunksNoCompactReason, "details...", promauto.With(nil).NewCounter(prometheus.CounterOpts{}))) + // Block 3 has marker with invalid version + require.NoError(t, bkt.Upload(ctx, block3.String()+"/no-compact-mark.json", strings.NewReader(`{"id":"`+block3.String()+`","version":100,"details":"details","no_compact_time":1637757932,"reason":"reason"}`))) + // Block 4 has marker with invalid JSON syntax + require.NoError(t, bkt.Upload(ctx, block4.String()+"/no-compact-mark.json", strings.NewReader(`invalid json`))) + + synced := extprom.NewTxGaugeVec(nil, prometheus.GaugeOpts{Name: "synced", Help: "Number of block metadata synced"}, + []string{"state"}, []string{block.MarkedForNoCompactionMeta}, + ) + + testFn(t, synced) + }) + } +} + +func TestConvertCompactionResultToForEachJobs(t *testing.T) { + ulid1 := ulid.MustNew(1, nil) + ulid2 := ulid.MustNew(2, nil) + + res := convertCompactionResultToForEachJobs([]ulid.ULID{{}, ulid1, {}, ulid2, {}}, true, log.NewNopLogger()) + require.Len(t, res, 2) + require.Equal(t, ulidWithShardIndex{ulid: ulid1, shardIndex: 1}, res[0]) + require.Equal(t, ulidWithShardIndex{ulid: ulid2, shardIndex: 3}, res[1]) +} + +func TestCompactedBlocksTimeRangeVerification(t *testing.T) { + const ( + sourceMinTime = 1000 + sourceMaxTime = 2500 + ) + + tests := map[string]struct { + compactedBlockMinTime int64 + compactedBlockMaxTime int64 + shouldErr bool + expectedErrMsg string + }{ + "should pass with minTime and maxTime matching the source blocks": { + compactedBlockMinTime: sourceMinTime, + compactedBlockMaxTime: sourceMaxTime, + shouldErr: false, + }, + "should fail with compacted block minTime < source minTime": { + compactedBlockMinTime: sourceMinTime - 500, + compactedBlockMaxTime: sourceMaxTime, + shouldErr: true, + expectedErrMsg: fmt.Sprintf("compacted block minTime %d is before source minTime %d", sourceMinTime-500, sourceMinTime), + }, + "should fail with compacted block maxTime > source maxTime": { + compactedBlockMinTime: sourceMinTime, + compactedBlockMaxTime: sourceMaxTime + 500, + shouldErr: true, + expectedErrMsg: fmt.Sprintf("compacted block maxTime %d is after source maxTime %d", sourceMaxTime+500, sourceMaxTime), + }, + "should fail due to minTime and maxTime not found": { + compactedBlockMinTime: sourceMinTime + 250, + compactedBlockMaxTime: sourceMaxTime - 250, + shouldErr: true, + expectedErrMsg: fmt.Sprintf("compacted block(s) do not contain minTime %d and maxTime %d from the source blocks", sourceMinTime, sourceMaxTime), + }, + } + + for testName, testData := range tests { + testData := testData // Prevent loop variable being captured by func literal + t.Run(testName, func(t *testing.T) { + t.Parallel() + + tempDir := t.TempDir() + + compactedBlock1, err := block.CreateBlock( + context.Background(), tempDir, + []labels.Labels{ + labels.FromStrings("test", "foo", "a", "1"), + labels.FromStrings("test", "foo", "a", "2"), + labels.FromStrings("test", "foo", "a", "3"), + }, 10, testData.compactedBlockMinTime, testData.compactedBlockMinTime+500, labels.EmptyLabels()) + require.NoError(t, err) + + compactedBlock2, err := block.CreateBlock( + context.Background(), tempDir, + []labels.Labels{ + labels.FromStrings("test", "foo", "a", "1"), + labels.FromStrings("test", "foo", "a", "2"), + labels.FromStrings("test", "foo", "a", "3"), + }, 10, testData.compactedBlockMaxTime-500, testData.compactedBlockMaxTime, labels.EmptyLabels()) + require.NoError(t, err) + + err = verifyCompactedBlocksTimeRanges([]ulid.ULID{compactedBlock1, compactedBlock2}, sourceMinTime, sourceMaxTime, tempDir) + if testData.shouldErr { + require.ErrorContains(t, err, testData.expectedErrMsg) + } else { + require.NoError(t, err) + } + }) + } +} diff --git a/pkg/compactor/compactor.go b/pkg/compactor/compactor.go new file mode 100644 index 0000000000..339c4e6316 --- /dev/null +++ b/pkg/compactor/compactor.go @@ -0,0 +1,928 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/grafana/mimir/blob/main/pkg/compactor/compactor.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Cortex Authors. +package compactor + +import ( + "context" + "flag" + "fmt" + "hash/fnv" + "math/rand" + "os" + "path" + "path/filepath" + "strings" + "time" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/grafana/dskit/backoff" + "github.com/grafana/dskit/flagext" + "github.com/grafana/dskit/kv" + "github.com/grafana/dskit/ring" + "github.com/grafana/dskit/services" + "github.com/pkg/errors" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" + "github.com/thanos-io/objstore" + "go.uber.org/atomic" + + "github.com/grafana/mimir/pkg/storage/bucket" + mimir_tsdb "github.com/grafana/mimir/pkg/storage/tsdb" + "github.com/grafana/mimir/pkg/storage/tsdb/block" + "github.com/grafana/mimir/pkg/util" + util_log "github.com/grafana/mimir/pkg/util/log" +) + +const ( + // ringKey is the key under which we store the compactors ring in the KVStore. + ringKey = "compactor" + + // ringAutoForgetUnhealthyPeriods is how many consecutive timeout periods an unhealthy instance + // in the ring will be automatically removed after. + ringAutoForgetUnhealthyPeriods = 10 +) + +const ( + blocksMarkedForDeletionName = "cortex_compactor_blocks_marked_for_deletion_total" + blocksMarkedForDeletionHelp = "Total number of blocks marked for deletion in compactor." +) + +var ( + errInvalidBlockRanges = "compactor block range periods should be divisible by the previous one, but %s is not divisible by %s" + errInvalidCompactionOrder = fmt.Errorf("unsupported compaction order (supported values: %s)", strings.Join(CompactionOrders, ", ")) + errInvalidMaxOpeningBlocksConcurrency = fmt.Errorf("invalid max-opening-blocks-concurrency value, must be positive") + errInvalidMaxClosingBlocksConcurrency = fmt.Errorf("invalid max-closing-blocks-concurrency value, must be positive") + errInvalidSymbolFlushersConcurrency = fmt.Errorf("invalid symbols-flushers-concurrency value, must be positive") + errInvalidMaxBlockUploadValidationConcurrency = fmt.Errorf("invalid max-block-upload-validation-concurrency value, can't be negative") + RingOp = ring.NewOp([]ring.InstanceState{ring.ACTIVE}, nil) +) + +// BlocksGrouperFactory builds and returns the grouper to use to compact a tenant's blocks. +type BlocksGrouperFactory func( + ctx context.Context, + cfg Config, + cfgProvider ConfigProvider, + userID string, + logger log.Logger, + reg prometheus.Registerer, +) Grouper + +// BlocksCompactorFactory builds and returns the compactor and planner to use to compact a tenant's blocks. +type BlocksCompactorFactory func( + ctx context.Context, + cfg Config, + logger log.Logger, + reg prometheus.Registerer, +) (Compactor, Planner, error) + +// Config holds the MultitenantCompactor config. +type Config struct { + BlockRanges mimir_tsdb.DurationList `yaml:"block_ranges" category:"advanced"` + BlockSyncConcurrency int `yaml:"block_sync_concurrency" category:"advanced"` + MetaSyncConcurrency int `yaml:"meta_sync_concurrency" category:"advanced"` + DataDir string `yaml:"data_dir"` + CompactionInterval time.Duration `yaml:"compaction_interval" category:"advanced"` + CompactionRetries int `yaml:"compaction_retries" category:"advanced"` + CompactionConcurrency int `yaml:"compaction_concurrency" category:"advanced"` + CompactionWaitPeriod time.Duration `yaml:"first_level_compaction_wait_period"` + CleanupInterval time.Duration `yaml:"cleanup_interval" category:"advanced"` + CleanupConcurrency int `yaml:"cleanup_concurrency" category:"advanced"` + DeletionDelay time.Duration `yaml:"deletion_delay" category:"advanced"` + TenantCleanupDelay time.Duration `yaml:"tenant_cleanup_delay" category:"advanced"` + MaxCompactionTime time.Duration `yaml:"max_compaction_time" category:"advanced"` + NoBlocksFileCleanupEnabled bool `yaml:"no_blocks_file_cleanup_enabled" category:"experimental"` + + // Compactor concurrency options + MaxOpeningBlocksConcurrency int `yaml:"max_opening_blocks_concurrency" category:"advanced"` // Number of goroutines opening blocks before compaction. + MaxClosingBlocksConcurrency int `yaml:"max_closing_blocks_concurrency" category:"advanced"` // Max number of blocks that can be closed concurrently during split compaction. Note that closing of newly compacted block uses a lot of memory for writing index. + SymbolsFlushersConcurrency int `yaml:"symbols_flushers_concurrency" category:"advanced"` // Number of symbols flushers used when doing split compaction. + MaxBlockUploadValidationConcurrency int `yaml:"max_block_upload_validation_concurrency" category:"advanced"` // Max number of uploaded blocks that can be validated concurrently. + + EnabledTenants flagext.StringSliceCSV `yaml:"enabled_tenants" category:"advanced"` + DisabledTenants flagext.StringSliceCSV `yaml:"disabled_tenants" category:"advanced"` + + // Compactors sharding. + ShardingRing RingConfig `yaml:"sharding_ring"` + + CompactionJobsOrder string `yaml:"compaction_jobs_order" category:"advanced"` + + // No need to add options to customize the retry backoff, + // given the defaults should be fine, but allow to override + // it in tests. + retryMinBackoff time.Duration `yaml:"-"` + retryMaxBackoff time.Duration `yaml:"-"` + + // Allow downstream projects to customise the blocks compactor. + BlocksGrouperFactory BlocksGrouperFactory `yaml:"-"` + BlocksCompactorFactory BlocksCompactorFactory `yaml:"-"` +} + +// RegisterFlags registers the MultitenantCompactor flags. +func (cfg *Config) RegisterFlags(f *flag.FlagSet, logger log.Logger) { + cfg.ShardingRing.RegisterFlags(f, logger) + + cfg.BlockRanges = mimir_tsdb.DurationList{2 * time.Hour, 12 * time.Hour, 24 * time.Hour} + cfg.retryMinBackoff = 10 * time.Second + cfg.retryMaxBackoff = time.Minute + + f.Var(&cfg.BlockRanges, "compactor.block-ranges", "List of compaction time ranges.") + f.IntVar(&cfg.BlockSyncConcurrency, "compactor.block-sync-concurrency", 8, "Number of Go routines to use when downloading blocks for compaction and uploading resulting blocks.") + f.IntVar(&cfg.MetaSyncConcurrency, "compactor.meta-sync-concurrency", 20, "Number of Go routines to use when syncing block meta files from the long term storage.") + f.StringVar(&cfg.DataDir, "compactor.data-dir", "./data-compactor/", "Directory to temporarily store blocks during compaction. This directory is not required to be persisted between restarts.") + f.DurationVar(&cfg.CompactionInterval, "compactor.compaction-interval", time.Hour, "The frequency at which the compaction runs") + f.DurationVar(&cfg.MaxCompactionTime, "compactor.max-compaction-time", time.Hour, "Max time for starting compactions for a single tenant. After this time no new compactions for the tenant are started before next compaction cycle. This can help in multi-tenant environments to avoid single tenant using all compaction time, but also in single-tenant environments to force new discovery of blocks more often. 0 = disabled.") + f.IntVar(&cfg.CompactionRetries, "compactor.compaction-retries", 3, "How many times to retry a failed compaction within a single compaction run.") + f.IntVar(&cfg.CompactionConcurrency, "compactor.compaction-concurrency", 1, "Max number of concurrent compactions running.") + f.DurationVar(&cfg.CompactionWaitPeriod, "compactor.first-level-compaction-wait-period", 25*time.Minute, "How long the compactor waits before compacting first-level blocks that are uploaded by the ingesters. This configuration option allows for the reduction of cases where the compactor begins to compact blocks before all ingesters have uploaded their blocks to the storage.") + f.DurationVar(&cfg.CleanupInterval, "compactor.cleanup-interval", 15*time.Minute, "How frequently compactor should run blocks cleanup and maintenance, as well as update the bucket index.") + f.IntVar(&cfg.CleanupConcurrency, "compactor.cleanup-concurrency", 20, "Max number of tenants for which blocks cleanup and maintenance should run concurrently.") + f.StringVar(&cfg.CompactionJobsOrder, "compactor.compaction-jobs-order", CompactionOrderOldestFirst, fmt.Sprintf("The sorting to use when deciding which compaction jobs should run first for a given tenant. Supported values are: %s.", strings.Join(CompactionOrders, ", "))) + f.DurationVar(&cfg.DeletionDelay, "compactor.deletion-delay", 12*time.Hour, "Time before a block marked for deletion is deleted from bucket. "+ + "If not 0, blocks will be marked for deletion and compactor component will permanently delete blocks marked for deletion from the bucket. "+ + "If 0, blocks will be deleted straight away. Note that deleting blocks immediately can cause query failures.") + f.DurationVar(&cfg.TenantCleanupDelay, "compactor.tenant-cleanup-delay", 6*time.Hour, "For tenants marked for deletion, this is time between deleting of last block, and doing final cleanup (marker files, debug files) of the tenant.") + f.BoolVar(&cfg.NoBlocksFileCleanupEnabled, "compactor.no-blocks-file-cleanup-enabled", false, "If enabled, will delete the bucket-index, markers and debug files in the tenant bucket when there are no blocks left in the index.") + // compactor concurrency options + f.IntVar(&cfg.MaxOpeningBlocksConcurrency, "compactor.max-opening-blocks-concurrency", 1, "Number of goroutines opening blocks before compaction.") + f.IntVar(&cfg.MaxClosingBlocksConcurrency, "compactor.max-closing-blocks-concurrency", 1, "Max number of blocks that can be closed concurrently during split compaction. Note that closing of newly compacted block uses a lot of memory for writing index.") + f.IntVar(&cfg.SymbolsFlushersConcurrency, "compactor.symbols-flushers-concurrency", 1, "Number of symbols flushers used when doing split compaction.") + f.IntVar(&cfg.MaxBlockUploadValidationConcurrency, "compactor.max-block-upload-validation-concurrency", 1, "Max number of uploaded blocks that can be validated concurrently. 0 = no limit.") + + f.Var(&cfg.EnabledTenants, "compactor.enabled-tenants", "Comma separated list of tenants that can be compacted. If specified, only these tenants will be compacted by compactor, otherwise all tenants can be compacted. Subject to sharding.") + f.Var(&cfg.DisabledTenants, "compactor.disabled-tenants", "Comma separated list of tenants that cannot be compacted by this compactor. If specified, and compactor would normally pick given tenant for compaction (via -compactor.enabled-tenants or sharding), it will be ignored instead.") +} + +func (cfg *Config) Validate() error { + // Each block range period should be divisible by the previous one. + for i := 1; i < len(cfg.BlockRanges); i++ { + if cfg.BlockRanges[i]%cfg.BlockRanges[i-1] != 0 { + return errors.Errorf(errInvalidBlockRanges, cfg.BlockRanges[i].String(), cfg.BlockRanges[i-1].String()) + } + } + + if cfg.MaxOpeningBlocksConcurrency < 1 { + return errInvalidMaxOpeningBlocksConcurrency + } + if cfg.MaxClosingBlocksConcurrency < 1 { + return errInvalidMaxClosingBlocksConcurrency + } + if cfg.SymbolsFlushersConcurrency < 1 { + return errInvalidSymbolFlushersConcurrency + } + if cfg.MaxBlockUploadValidationConcurrency < 0 { + return errInvalidMaxBlockUploadValidationConcurrency + } + if !util.StringsContain(CompactionOrders, cfg.CompactionJobsOrder) { + return errInvalidCompactionOrder + } + + return nil +} + +// ConfigProvider defines the per-tenant config provider for the MultitenantCompactor. +type ConfigProvider interface { + bucket.TenantConfigProvider + + // CompactorBlocksRetentionPeriod returns the retention period for a given user. + CompactorBlocksRetentionPeriod(user string) time.Duration + + // CompactorSplitAndMergeShards returns the number of shards to use when splitting blocks. + CompactorSplitAndMergeShards(userID string) int + + // CompactorSplitGroups returns the number of groups that blocks used for splitting should + // be grouped into. Different groups are then split by different jobs. + CompactorSplitGroups(userID string) int + + // CompactorTenantShardSize returns number of compactors that this user can use. 0 = all compactors. + CompactorTenantShardSize(userID string) int + + // CompactorPartialBlockDeletionDelay returns the partial block delay time period for a given user, + // and whether the configured value was valid. If the value wasn't valid, the returned delay is the default one + // and the caller is responsible to warn the Mimir operator about it. + CompactorPartialBlockDeletionDelay(userID string) (delay time.Duration, valid bool) + + // CompactorBlockUploadEnabled returns whether block upload is enabled for a given tenant. + CompactorBlockUploadEnabled(tenantID string) bool + + // CompactorBlockUploadValidationEnabled returns whether block upload validation is enabled for a given tenant. + CompactorBlockUploadValidationEnabled(tenantID string) bool + + // CompactorBlockUploadVerifyChunks returns whether chunk verification is enabled for a given tenant. + CompactorBlockUploadVerifyChunks(tenantID string) bool + + // CompactorBlockUploadMaxBlockSizeBytes returns the maximum size in bytes of a block that is allowed to be uploaded or validated for a given user. + CompactorBlockUploadMaxBlockSizeBytes(userID string) int64 +} + +// MultitenantCompactor is a multi-tenant TSDB blocks compactor based on Thanos. +type MultitenantCompactor struct { + services.Service + + compactorCfg Config + storageCfg mimir_tsdb.BlocksStorageConfig + cfgProvider ConfigProvider + logger log.Logger + parentLogger log.Logger + registerer prometheus.Registerer + + // Functions that creates bucket client, grouper, planner and compactor using the context. + // Useful for injecting mock objects from tests. + bucketClientFactory func(ctx context.Context) (objstore.Bucket, error) + blocksGrouperFactory BlocksGrouperFactory + blocksCompactorFactory BlocksCompactorFactory + + // Blocks cleaner is responsible to hard delete blocks marked for deletion. + blocksCleaner *BlocksCleaner + + // Underlying compactor and planner used to compact TSDB blocks. + blocksCompactor Compactor + blocksPlanner Planner + + // Client used to run operations on the bucket storing blocks. + bucketClient objstore.Bucket + + // Ring used for sharding compactions. + ringLifecycler *ring.BasicLifecycler + ring *ring.Ring + ringSubservices *services.Manager + ringSubservicesWatcher *services.FailureWatcher + + shardingStrategy shardingStrategy + jobsOrder JobsOrderFunc + + // Metrics. + compactionRunsStarted prometheus.Counter + compactionRunsCompleted prometheus.Counter + compactionRunsErred prometheus.Counter + compactionRunsShutdown prometheus.Counter + compactionRunsLastSuccess prometheus.Gauge + compactionRunDiscoveredTenants prometheus.Gauge + compactionRunSkippedTenants prometheus.Gauge + compactionRunSucceededTenants prometheus.Gauge + compactionRunFailedTenants prometheus.Gauge + compactionRunInterval prometheus.Gauge + blocksMarkedForDeletion prometheus.Counter + + // Metrics shared across all BucketCompactor instances. + bucketCompactorMetrics *BucketCompactorMetrics + + // TSDB syncer metrics + syncerMetrics *aggregatedSyncerMetrics + + // Block upload metrics + blockUploadBlocks *prometheus.GaugeVec + blockUploadBytes *prometheus.GaugeVec + blockUploadFiles *prometheus.GaugeVec + blockUploadValidations atomic.Int64 +} + +// NewMultitenantCompactor makes a new MultitenantCompactor. +func NewMultitenantCompactor(compactorCfg Config, storageCfg mimir_tsdb.BlocksStorageConfig, cfgProvider ConfigProvider, logger log.Logger, registerer prometheus.Registerer) (*MultitenantCompactor, error) { + bucketClientFactory := func(ctx context.Context) (objstore.Bucket, error) { + return bucket.NewClient(ctx, storageCfg.Bucket, "compactor", logger, registerer) + } + + // Configure the compactor and grouper factories only if they weren't already set by a downstream project. + if compactorCfg.BlocksGrouperFactory == nil || compactorCfg.BlocksCompactorFactory == nil { + configureSplitAndMergeCompactor(&compactorCfg) + } + + blocksGrouperFactory := compactorCfg.BlocksGrouperFactory + blocksCompactorFactory := compactorCfg.BlocksCompactorFactory + + mimirCompactor, err := newMultitenantCompactor(compactorCfg, storageCfg, cfgProvider, logger, registerer, bucketClientFactory, blocksGrouperFactory, blocksCompactorFactory) + if err != nil { + return nil, errors.Wrap(err, "failed to create blocks compactor") + } + + return mimirCompactor, nil +} + +func newMultitenantCompactor( + compactorCfg Config, + storageCfg mimir_tsdb.BlocksStorageConfig, + cfgProvider ConfigProvider, + logger log.Logger, + registerer prometheus.Registerer, + bucketClientFactory func(ctx context.Context) (objstore.Bucket, error), + blocksGrouperFactory BlocksGrouperFactory, + blocksCompactorFactory BlocksCompactorFactory, +) (*MultitenantCompactor, error) { + c := &MultitenantCompactor{ + compactorCfg: compactorCfg, + storageCfg: storageCfg, + cfgProvider: cfgProvider, + parentLogger: logger, + logger: log.With(logger, "component", "compactor"), + registerer: registerer, + syncerMetrics: newAggregatedSyncerMetrics(registerer), + bucketClientFactory: bucketClientFactory, + blocksGrouperFactory: blocksGrouperFactory, + blocksCompactorFactory: blocksCompactorFactory, + + compactionRunsStarted: promauto.With(registerer).NewCounter(prometheus.CounterOpts{ + Name: "cortex_compactor_runs_started_total", + Help: "Total number of compaction runs started.", + }), + compactionRunsCompleted: promauto.With(registerer).NewCounter(prometheus.CounterOpts{ + Name: "cortex_compactor_runs_completed_total", + Help: "Total number of compaction runs successfully completed.", + }), + compactionRunsErred: promauto.With(registerer).NewCounter(prometheus.CounterOpts{ + Name: "cortex_compactor_runs_failed_total", + Help: "Total number of compaction runs failed.", + ConstLabels: map[string]string{"reason": "error"}, + }), + compactionRunsShutdown: promauto.With(registerer).NewCounter(prometheus.CounterOpts{ + Name: "cortex_compactor_runs_failed_total", + Help: "Total number of compaction runs failed.", + ConstLabels: map[string]string{"reason": "shutdown"}, + }), + compactionRunsLastSuccess: promauto.With(registerer).NewGauge(prometheus.GaugeOpts{ + Name: "cortex_compactor_last_successful_run_timestamp_seconds", + Help: "Unix timestamp of the last successful compaction run.", + }), + compactionRunDiscoveredTenants: promauto.With(registerer).NewGauge(prometheus.GaugeOpts{ + Name: "cortex_compactor_tenants_discovered", + Help: "Number of tenants discovered during the current compaction run. Reset to 0 when compactor is idle.", + }), + compactionRunSkippedTenants: promauto.With(registerer).NewGauge(prometheus.GaugeOpts{ + Name: "cortex_compactor_tenants_skipped", + Help: "Number of tenants skipped during the current compaction run. Reset to 0 when compactor is idle.", + }), + compactionRunSucceededTenants: promauto.With(registerer).NewGauge(prometheus.GaugeOpts{ + Name: "cortex_compactor_tenants_processing_succeeded", + Help: "Number of tenants successfully processed during the current compaction run. Reset to 0 when compactor is idle.", + }), + compactionRunFailedTenants: promauto.With(registerer).NewGauge(prometheus.GaugeOpts{ + Name: "cortex_compactor_tenants_processing_failed", + Help: "Number of tenants failed processing during the current compaction run. Reset to 0 when compactor is idle.", + }), + compactionRunInterval: promauto.With(registerer).NewGauge(prometheus.GaugeOpts{ + Name: "cortex_compactor_compaction_interval_seconds", + Help: "The configured interval on which compaction is run in seconds. Useful when compared to the last successful run metric to accurately detect multiple failed compaction runs.", + }), + blocksMarkedForDeletion: promauto.With(registerer).NewCounter(prometheus.CounterOpts{ + Name: blocksMarkedForDeletionName, + Help: blocksMarkedForDeletionHelp, + ConstLabels: prometheus.Labels{"reason": "compaction"}, + }), + blockUploadBlocks: promauto.With(registerer).NewGaugeVec(prometheus.GaugeOpts{ + Name: "cortex_block_upload_api_blocks_total", + Help: "Total number of blocks successfully uploaded and validated using the block upload API.", + }, []string{"user"}), + blockUploadBytes: promauto.With(registerer).NewGaugeVec(prometheus.GaugeOpts{ + Name: "cortex_block_upload_api_bytes_total", + Help: "Total number of bytes from successfully uploaded and validated blocks using block upload API.", + }, []string{"user"}), + blockUploadFiles: promauto.With(registerer).NewGaugeVec(prometheus.GaugeOpts{ + Name: "cortex_block_upload_api_files_total", + Help: "Total number of files from successfully uploaded and validated blocks using block upload API.", + }, []string{"user"}), + } + + promauto.With(registerer).NewGaugeFunc(prometheus.GaugeOpts{ + Name: "cortex_block_upload_validations_in_progress", + Help: "Number of block upload validations currently running.", + }, func() float64 { + return float64(c.blockUploadValidations.Load()) + }) + + c.bucketCompactorMetrics = NewBucketCompactorMetrics(c.blocksMarkedForDeletion, registerer) + + if len(compactorCfg.EnabledTenants) > 0 { + level.Info(c.logger).Log("msg", "compactor using enabled users", "enabled", strings.Join(compactorCfg.EnabledTenants, ", ")) + } + if len(compactorCfg.DisabledTenants) > 0 { + level.Info(c.logger).Log("msg", "compactor using disabled users", "disabled", strings.Join(compactorCfg.DisabledTenants, ", ")) + } + + c.jobsOrder = GetJobsOrderFunction(compactorCfg.CompactionJobsOrder) + if c.jobsOrder == nil { + return nil, errInvalidCompactionOrder + } + + c.Service = services.NewBasicService(c.starting, c.running, c.stopping) + + // The last successful compaction run metric is exposed as seconds since epoch, so we need to use seconds for this metric. + c.compactionRunInterval.Set(c.compactorCfg.CompactionInterval.Seconds()) + + return c, nil +} + +// Start the compactor. +func (c *MultitenantCompactor) starting(ctx context.Context) error { + var err error + + // Create bucket client. + c.bucketClient, err = c.bucketClientFactory(ctx) + if err != nil { + return errors.Wrap(err, "failed to create bucket client") + } + + // Create blocks compactor dependencies. + c.blocksCompactor, c.blocksPlanner, err = c.blocksCompactorFactory(ctx, c.compactorCfg, c.logger, c.registerer) + if err != nil { + return errors.Wrap(err, "failed to initialize compactor dependencies") + } + + // Wrap the bucket client to write block deletion marks in the global location too. + c.bucketClient = block.BucketWithGlobalMarkers(c.bucketClient) + + // Initialize the compactors ring if sharding is enabled. + c.ring, c.ringLifecycler, err = newRingAndLifecycler(c.compactorCfg.ShardingRing, c.logger, c.registerer) + if err != nil { + return err + } + + c.ringSubservices, err = services.NewManager(c.ringLifecycler, c.ring) + if err != nil { + return errors.Wrap(err, "unable to create compactor ring dependencies") + } + + c.ringSubservicesWatcher = services.NewFailureWatcher() + c.ringSubservicesWatcher.WatchManager(c.ringSubservices) + if err = c.ringSubservices.StartAsync(ctx); err != nil { + return errors.Wrap(err, "unable to start compactor ring dependencies") + } + + ctxTimeout, cancel := context.WithTimeout(ctx, c.compactorCfg.ShardingRing.WaitActiveInstanceTimeout) + defer cancel() + if err = c.ringSubservices.AwaitHealthy(ctxTimeout); err != nil { + return errors.Wrap(err, "unable to start compactor ring dependencies") + } + + // If sharding is enabled we should wait until this instance is ACTIVE within the ring. This + // MUST be done before starting any other component depending on the users scanner, because + // the users scanner depends on the ring (to check whether a user belongs to this shard or not). + level.Info(c.logger).Log("msg", "waiting until compactor is ACTIVE in the ring") + if err = ring.WaitInstanceState(ctxTimeout, c.ring, c.ringLifecycler.GetInstanceID(), ring.ACTIVE); err != nil { + return errors.Wrap(err, "compactor failed to become ACTIVE in the ring") + } + + level.Info(c.logger).Log("msg", "compactor is ACTIVE in the ring") + + // In the event of a cluster cold start or scale up of 2+ compactor instances at the same + // time, we may end up in a situation where each new compactor instance starts at a slightly + // different time and thus each one starts with a different state of the ring. It's better + // to just wait a short time for ring stability. + if c.compactorCfg.ShardingRing.WaitStabilityMinDuration > 0 { + minWaiting := c.compactorCfg.ShardingRing.WaitStabilityMinDuration + maxWaiting := c.compactorCfg.ShardingRing.WaitStabilityMaxDuration + + level.Info(c.logger).Log("msg", "waiting until compactor ring topology is stable", "min_waiting", minWaiting.String(), "max_waiting", maxWaiting.String()) + if err := ring.WaitRingStability(ctx, c.ring, RingOp, minWaiting, maxWaiting); err != nil { + level.Warn(c.logger).Log("msg", "compactor ring topology is not stable after the max waiting time, proceeding anyway") + } else { + level.Info(c.logger).Log("msg", "compactor ring topology is stable") + } + } + + allowedTenants := util.NewAllowedTenants(c.compactorCfg.EnabledTenants, c.compactorCfg.DisabledTenants) + c.shardingStrategy = newSplitAndMergeShardingStrategy(allowedTenants, c.ring, c.ringLifecycler, c.cfgProvider) + + // Create the blocks cleaner (service). + c.blocksCleaner = NewBlocksCleaner(BlocksCleanerConfig{ + DeletionDelay: c.compactorCfg.DeletionDelay, + CleanupInterval: util.DurationWithJitter(c.compactorCfg.CleanupInterval, 0.1), + CleanupConcurrency: c.compactorCfg.CleanupConcurrency, + TenantCleanupDelay: c.compactorCfg.TenantCleanupDelay, + DeleteBlocksConcurrency: defaultDeleteBlocksConcurrency, + NoBlocksFileCleanupEnabled: c.compactorCfg.NoBlocksFileCleanupEnabled, + }, c.bucketClient, c.shardingStrategy.blocksCleanerOwnUser, c.cfgProvider, c.parentLogger, c.registerer) + + // Start blocks cleaner asynchronously, don't wait until initial cleanup is finished. + if err := c.blocksCleaner.StartAsync(ctx); err != nil { + c.ringSubservices.StopAsync() + return errors.Wrap(err, "failed to start the blocks cleaner") + } + + return nil +} + +func newRingAndLifecycler(cfg RingConfig, logger log.Logger, reg prometheus.Registerer) (*ring.Ring, *ring.BasicLifecycler, error) { + reg = prometheus.WrapRegistererWithPrefix("cortex_", reg) + kvStore, err := kv.NewClient(cfg.Common.KVStore, ring.GetCodec(), kv.RegistererWithKVName(reg, "compactor-lifecycler"), logger) + if err != nil { + return nil, nil, errors.Wrap(err, "failed to initialize compactors' KV store") + } + + lifecyclerCfg, err := cfg.ToBasicLifecyclerConfig(logger) + if err != nil { + return nil, nil, errors.Wrap(err, "failed to build compactors' lifecycler config") + } + + var delegate ring.BasicLifecyclerDelegate + delegate = ring.NewInstanceRegisterDelegate(ring.ACTIVE, lifecyclerCfg.NumTokens) + delegate = ring.NewLeaveOnStoppingDelegate(delegate, logger) + delegate = ring.NewAutoForgetDelegate(ringAutoForgetUnhealthyPeriods*lifecyclerCfg.HeartbeatTimeout, delegate, logger) + + compactorsLifecycler, err := ring.NewBasicLifecycler(lifecyclerCfg, "compactor", ringKey, kvStore, delegate, logger, reg) + if err != nil { + return nil, nil, errors.Wrap(err, "failed to initialize compactors' lifecycler") + } + + compactorsRing, err := ring.New(cfg.toRingConfig(), "compactor", ringKey, logger, reg) + if err != nil { + return nil, nil, errors.Wrap(err, "failed to initialize compactors' ring client") + } + + return compactorsRing, compactorsLifecycler, nil +} + +func (c *MultitenantCompactor) stopping(_ error) error { + ctx := context.Background() + + services.StopAndAwaitTerminated(ctx, c.blocksCleaner) //nolint:errcheck + if c.ringSubservices != nil { + return services.StopManagerAndAwaitStopped(ctx, c.ringSubservices) + } + return nil +} + +func (c *MultitenantCompactor) running(ctx context.Context) error { + // Run an initial compaction before starting the interval. + c.compactUsers(ctx) + + ticker := time.NewTicker(util.DurationWithJitter(c.compactorCfg.CompactionInterval, 0.05)) + defer ticker.Stop() + + for { + select { + case <-ticker.C: + c.compactUsers(ctx) + case <-ctx.Done(): + return nil + case err := <-c.ringSubservicesWatcher.Chan(): + return errors.Wrap(err, "compactor subservice failed") + } + } +} + +func (c *MultitenantCompactor) compactUsers(ctx context.Context) { + succeeded := false + compactionErrorCount := 0 + + c.compactionRunsStarted.Inc() + + defer func() { + if succeeded && compactionErrorCount == 0 { + c.compactionRunsCompleted.Inc() + c.compactionRunsLastSuccess.SetToCurrentTime() + } else if compactionErrorCount == 0 { + c.compactionRunsShutdown.Inc() + } else { + c.compactionRunsErred.Inc() + } + + // Reset progress metrics once done. + c.compactionRunDiscoveredTenants.Set(0) + c.compactionRunSkippedTenants.Set(0) + c.compactionRunSucceededTenants.Set(0) + c.compactionRunFailedTenants.Set(0) + }() + + level.Info(c.logger).Log("msg", "discovering users from bucket") + users, err := c.discoverUsersWithRetries(ctx) + if err != nil { + if !errors.Is(err, context.Canceled) { + compactionErrorCount++ + level.Error(c.logger).Log("msg", "failed to discover users from bucket", "err", err) + } + return + } + + level.Info(c.logger).Log("msg", "discovered users from bucket", "users", len(users)) + c.compactionRunDiscoveredTenants.Set(float64(len(users))) + + // When starting multiple compactor replicas nearly at the same time, running in a cluster with + // a large number of tenants, we may end up in a situation where the 1st user is compacted by + // multiple replicas at the same time. Shuffling users helps reduce the likelihood this will happen. + rand.Shuffle(len(users), func(i, j int) { + users[i], users[j] = users[j], users[i] + }) + + // Keep track of users owned by this shard, so that we can delete the local files for all other users. + ownedUsers := map[string]struct{}{} + for _, userID := range users { + // Ensure the context has not been canceled (ie. compactor shutdown has been triggered). + if ctx.Err() != nil { + level.Info(c.logger).Log("msg", "interrupting compaction of user blocks", "err", err) + return + } + + // Ensure the user ID belongs to our shard. + if owned, err := c.shardingStrategy.compactorOwnUser(userID); err != nil { + c.compactionRunSkippedTenants.Inc() + level.Warn(c.logger).Log("msg", "unable to check if user is owned by this shard", "user", userID, "err", err) + continue + } else if !owned { + c.compactionRunSkippedTenants.Inc() + level.Debug(c.logger).Log("msg", "skipping user because it is not owned by this shard", "user", userID) + continue + } + + ownedUsers[userID] = struct{}{} + + if markedForDeletion, err := mimir_tsdb.TenantDeletionMarkExists(ctx, c.bucketClient, userID); err != nil { + c.compactionRunSkippedTenants.Inc() + level.Warn(c.logger).Log("msg", "unable to check if user is marked for deletion", "user", userID, "err", err) + continue + } else if markedForDeletion { + c.compactionRunSkippedTenants.Inc() + level.Debug(c.logger).Log("msg", "skipping user because it is marked for deletion", "user", userID) + continue + } + + level.Info(c.logger).Log("msg", "starting compaction of user blocks", "user", userID) + + if err = c.compactUserWithRetries(ctx, userID); err != nil { + switch { + case errors.Is(err, context.Canceled): + // We don't want to count shutdowns as failed compactions because we will pick up with the rest of the compaction after the restart. + level.Info(c.logger).Log("msg", "compaction for user was interrupted by a shutdown", "user", userID) + return + default: + c.compactionRunFailedTenants.Inc() + compactionErrorCount++ + level.Error(c.logger).Log("msg", "failed to compact user blocks", "user", userID, "err", err) + } + continue + } + + c.compactionRunSucceededTenants.Inc() + level.Info(c.logger).Log("msg", "successfully compacted user blocks", "user", userID) + } + + // Delete local files for unowned tenants, if there are any. This cleans up + // leftover local files for tenants that belong to different compactors now, + // or have been deleted completely. + for userID := range c.listTenantsWithMetaSyncDirectories() { + if _, owned := ownedUsers[userID]; owned { + continue + } + + dir := c.metaSyncDirForUser(userID) + s, err := os.Stat(dir) + if err != nil { + if !os.IsNotExist(err) { + level.Warn(c.logger).Log("msg", "failed to stat local directory with user data", "dir", dir, "err", err) + } + continue + } + + if s.IsDir() { + err := os.RemoveAll(dir) + if err == nil { + level.Info(c.logger).Log("msg", "deleted directory for user not owned by this shard", "dir", dir) + } else { + level.Warn(c.logger).Log("msg", "failed to delete directory for user not owned by this shard", "dir", dir, "err", err) + } + } + } + + succeeded = true +} + +func (c *MultitenantCompactor) compactUserWithRetries(ctx context.Context, userID string) error { + var lastErr error + + retries := backoff.New(ctx, backoff.Config{ + MinBackoff: c.compactorCfg.retryMinBackoff, + MaxBackoff: c.compactorCfg.retryMaxBackoff, + MaxRetries: c.compactorCfg.CompactionRetries, + }) + + for retries.Ongoing() { + lastErr = c.compactUser(ctx, userID) + if lastErr == nil { + return nil + } + + retries.Wait() + } + + return lastErr +} + +func (c *MultitenantCompactor) compactUser(ctx context.Context, userID string) error { + userBucket := bucket.NewUserBucketClient(userID, c.bucketClient, c.cfgProvider) + reg := prometheus.NewRegistry() + defer c.syncerMetrics.gatherThanosSyncerMetrics(reg) + + userLogger := util_log.WithUserID(userID, c.logger) + + // Filters out duplicate blocks that can be formed from two or more overlapping + // blocks that fully submatches the source blocks of the older blocks. + deduplicateBlocksFilter := NewShardAwareDeduplicateFilter() + + // List of filters to apply (order matters). + fetcherFilters := []block.MetadataFilter{ + // Remove the ingester ID because we don't shard blocks anymore, while still + // honoring the shard ID if sharding was done in the past. + // Remove TenantID external label to make sure that we compact blocks with and without the label + // together. + NewLabelRemoverFilter([]string{ + mimir_tsdb.DeprecatedTenantIDExternalLabel, + mimir_tsdb.DeprecatedIngesterIDExternalLabel, + }), + deduplicateBlocksFilter, + // removes blocks that should not be compacted due to being marked so. + NewNoCompactionMarkFilter(userBucket, true), + } + + fetcher, err := block.NewMetaFetcher( + userLogger, + c.compactorCfg.MetaSyncConcurrency, + userBucket, + c.metaSyncDirForUser(userID), + reg, + fetcherFilters, + ) + if err != nil { + return err + } + + syncer, err := NewMetaSyncer( + userLogger, + reg, + userBucket, + fetcher, + deduplicateBlocksFilter, + c.blocksMarkedForDeletion, + ) + if err != nil { + return errors.Wrap(err, "failed to create syncer") + } + + compactor, err := NewBucketCompactor( + userLogger, + syncer, + c.blocksGrouperFactory(ctx, c.compactorCfg, c.cfgProvider, userID, userLogger, reg), + c.blocksPlanner, + c.blocksCompactor, + path.Join(c.compactorCfg.DataDir, "compact"), + userBucket, + c.compactorCfg.CompactionConcurrency, + true, // Skip blocks with out of order chunks, and mark them for no-compaction. + c.shardingStrategy.ownJob, + c.jobsOrder, + c.compactorCfg.CompactionWaitPeriod, + c.compactorCfg.BlockSyncConcurrency, + c.bucketCompactorMetrics, + ) + if err != nil { + return errors.Wrap(err, "failed to create bucket compactor") + } + + if err := compactor.Compact(ctx, c.compactorCfg.MaxCompactionTime); err != nil { + return errors.Wrap(err, "compaction") + } + + return nil +} + +func (c *MultitenantCompactor) discoverUsersWithRetries(ctx context.Context) ([]string, error) { + var lastErr error + + retries := backoff.New(ctx, backoff.Config{ + MinBackoff: c.compactorCfg.retryMinBackoff, + MaxBackoff: c.compactorCfg.retryMaxBackoff, + MaxRetries: c.compactorCfg.CompactionRetries, + }) + + for retries.Ongoing() { + var users []string + + users, lastErr = c.discoverUsers(ctx) + if lastErr == nil { + return users, nil + } + + retries.Wait() + } + + return nil, lastErr +} + +func (c *MultitenantCompactor) discoverUsers(ctx context.Context) ([]string, error) { + return mimir_tsdb.ListUsers(ctx, c.bucketClient) +} + +// shardingStrategy describes whether compactor "owns" given user or job. +type shardingStrategy interface { + compactorOwnUser(userID string) (bool, error) + // blocksCleanerOwnUser must be concurrency-safe + blocksCleanerOwnUser(userID string) (bool, error) + ownJob(job *Job) (bool, error) +} + +// splitAndMergeShardingStrategy is used by split-and-merge compactor when configured with sharding. +// All compactors from user's shard own the user for compaction purposes, and plan jobs. +// Each job is only owned and executed by single compactor. +// Only one of compactors from user's shard will do cleanup. +type splitAndMergeShardingStrategy struct { + allowedTenants *util.AllowedTenants + ring *ring.Ring + ringLifecycler *ring.BasicLifecycler + configProvider ConfigProvider +} + +func newSplitAndMergeShardingStrategy(allowedTenants *util.AllowedTenants, ring *ring.Ring, ringLifecycler *ring.BasicLifecycler, configProvider ConfigProvider) *splitAndMergeShardingStrategy { + return &splitAndMergeShardingStrategy{ + allowedTenants: allowedTenants, + ring: ring, + ringLifecycler: ringLifecycler, + configProvider: configProvider, + } +} + +// Only single instance in the subring can run blocks cleaner for given user. blocksCleanerOwnUser is concurrency-safe. +func (s *splitAndMergeShardingStrategy) blocksCleanerOwnUser(userID string) (bool, error) { + if !s.allowedTenants.IsAllowed(userID) { + return false, nil + } + + r := s.ring.ShuffleShard(userID, s.configProvider.CompactorTenantShardSize(userID)) + + return instanceOwnsTokenInRing(r, s.ringLifecycler.GetInstanceAddr(), userID) +} + +// ALL compactors should plan jobs for all users. +func (s *splitAndMergeShardingStrategy) compactorOwnUser(userID string) (bool, error) { + if !s.allowedTenants.IsAllowed(userID) { + return false, nil + } + + r := s.ring.ShuffleShard(userID, s.configProvider.CompactorTenantShardSize(userID)) + + return r.HasInstance(s.ringLifecycler.GetInstanceID()), nil +} + +// Only single compactor should execute the job. +func (s *splitAndMergeShardingStrategy) ownJob(job *Job) (bool, error) { + ok, err := s.compactorOwnUser(job.UserID()) + if err != nil || !ok { + return ok, err + } + + r := s.ring.ShuffleShard(job.UserID(), s.configProvider.CompactorTenantShardSize(job.UserID())) + + return instanceOwnsTokenInRing(r, s.ringLifecycler.GetInstanceAddr(), job.ShardingKey()) +} + +func instanceOwnsTokenInRing(r ring.ReadRing, instanceAddr string, key string) (bool, error) { + // Hash the key. + hasher := fnv.New32a() + _, _ = hasher.Write([]byte(key)) + hash := hasher.Sum32() + + // Check whether this compactor instance owns the token. + rs, err := r.Get(hash, RingOp, nil, nil, nil) + if err != nil { + return false, err + } + + if len(rs.Instances) != 1 { + return false, fmt.Errorf("unexpected number of compactors in the shard (expected 1, got %d)", len(rs.Instances)) + } + + return rs.Instances[0].Addr == instanceAddr, nil +} + +const compactorMetaPrefix = "compactor-meta-" + +// metaSyncDirForUser returns directory to store cached meta files. +// The fetcher stores cached metas in the "meta-syncer/" sub directory, +// but we prefix it with "compactor-meta-" in order to guarantee no clashing with +// the directory used by the Thanos Syncer, whatever is the user ID. +func (c *MultitenantCompactor) metaSyncDirForUser(userID string) string { + return filepath.Join(c.compactorCfg.DataDir, compactorMetaPrefix+userID) +} + +// This function returns tenants with meta sync directories found on local disk. On error, it returns nil map. +func (c *MultitenantCompactor) listTenantsWithMetaSyncDirectories() map[string]struct{} { + result := map[string]struct{}{} + + files, err := os.ReadDir(c.compactorCfg.DataDir) + if err != nil { + return nil + } + + for _, f := range files { + if !f.IsDir() { + continue + } + + if !strings.HasPrefix(f.Name(), compactorMetaPrefix) { + continue + } + + result[f.Name()[len(compactorMetaPrefix):]] = struct{}{} + } + + return result +} diff --git a/pkg/compactor/compactor_http.go b/pkg/compactor/compactor_http.go new file mode 100644 index 0000000000..227d0894c3 --- /dev/null +++ b/pkg/compactor/compactor_http.go @@ -0,0 +1,46 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/grafana/mimir/blob/main/pkg/compactor/compactor_http.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Cortex Authors. + +package compactor + +import ( + _ "embed" // Used to embed html template + "html/template" + "net/http" + + "github.com/go-kit/log/level" + "github.com/grafana/dskit/services" + + util_log "github.com/grafana/mimir/pkg/util/log" +) + +var ( + //go:embed status.gohtml + statusPageHTML string + statusPageTemplate = template.Must(template.New("main").Parse(statusPageHTML)) +) + +type statusPageContents struct { + Message string +} + +func writeMessage(w http.ResponseWriter, message string) { + w.WriteHeader(http.StatusOK) + err := statusPageTemplate.Execute(w, statusPageContents{Message: message}) + if err != nil { + level.Error(util_log.Logger).Log("msg", "unable to serve compactor ring page", "err", err) + } +} + +func (c *MultitenantCompactor) RingHandler(w http.ResponseWriter, req *http.Request) { + if c.State() != services.Running { + // we cannot read the ring before MultitenantCompactor is in Running state, + // because that would lead to race condition. + writeMessage(w, "Compactor is not running yet.") + return + } + + c.ring.ServeHTTP(w, req) +} diff --git a/pkg/compactor/compactor_ring.go b/pkg/compactor/compactor_ring.go new file mode 100644 index 0000000000..b2872d6409 --- /dev/null +++ b/pkg/compactor/compactor_ring.go @@ -0,0 +1,81 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/grafana/mimir/blob/main/pkg/compactor/compactor_ring.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Cortex Authors. + +package compactor + +import ( + "flag" + "fmt" + "time" + + "github.com/go-kit/log" + "github.com/grafana/dskit/ring" + + "github.com/grafana/mimir/pkg/util" +) + +const ( + // ringNumTokens is how many tokens each compactor should have in the ring. We use a + // safe default instead of exposing to config option to the user in order to simplify + // the config. + ringNumTokens = 512 +) + +// RingConfig masks the ring lifecycler config which contains +// many options not really required by the compactors ring. This config +// is used to strip down the config to the minimum, and avoid confusion +// to the user. +type RingConfig struct { + Common util.CommonRingConfig `yaml:",inline"` + + // Wait ring stability. + WaitStabilityMinDuration time.Duration `yaml:"wait_stability_min_duration" category:"advanced"` + WaitStabilityMaxDuration time.Duration `yaml:"wait_stability_max_duration" category:"advanced"` + + WaitActiveInstanceTimeout time.Duration `yaml:"wait_active_instance_timeout" category:"advanced"` + + ObservePeriod time.Duration `yaml:"-"` +} + +// RegisterFlags adds the flags required to config this to the given FlagSet +func (cfg *RingConfig) RegisterFlags(f *flag.FlagSet, logger log.Logger) { + const flagNamePrefix = "compactor.ring." + const kvStorePrefix = "collectors/" + const componentPlural = "compactors" + cfg.Common.RegisterFlags(flagNamePrefix, kvStorePrefix, componentPlural, f, logger) + + // Wait stability flags. + f.DurationVar(&cfg.WaitStabilityMinDuration, flagNamePrefix+"wait-stability-min-duration", 0, "Minimum time to wait for ring stability at startup. 0 to disable.") + f.DurationVar(&cfg.WaitStabilityMaxDuration, flagNamePrefix+"wait-stability-max-duration", 5*time.Minute, "Maximum time to wait for ring stability at startup. If the compactor ring keeps changing after this period of time, the compactor will start anyway.") + + // Timeout durations + f.DurationVar(&cfg.WaitActiveInstanceTimeout, flagNamePrefix+"wait-active-instance-timeout", 10*time.Minute, "Timeout for waiting on compactor to become ACTIVE in the ring.") +} + +func (cfg *RingConfig) ToBasicLifecyclerConfig(logger log.Logger) (ring.BasicLifecyclerConfig, error) { + instanceAddr, err := ring.GetInstanceAddr(cfg.Common.InstanceAddr, cfg.Common.InstanceInterfaceNames, logger, cfg.Common.EnableIPv6) + if err != nil { + return ring.BasicLifecyclerConfig{}, err + } + + instancePort := ring.GetInstancePort(cfg.Common.InstancePort, cfg.Common.ListenPort) + + return ring.BasicLifecyclerConfig{ + ID: cfg.Common.InstanceID, + Addr: fmt.Sprintf("%s:%d", instanceAddr, instancePort), + HeartbeatPeriod: cfg.Common.HeartbeatPeriod, + HeartbeatTimeout: cfg.Common.HeartbeatTimeout, + TokensObservePeriod: cfg.ObservePeriod, + NumTokens: ringNumTokens, + KeepInstanceInTheRingOnShutdown: false, + }, nil +} + +func (cfg *RingConfig) toRingConfig() ring.Config { + rc := cfg.Common.ToRingConfig() + rc.ReplicationFactor = 1 + + return rc +} diff --git a/pkg/compactor/compactor_test.go b/pkg/compactor/compactor_test.go new file mode 100644 index 0000000000..f755040c42 --- /dev/null +++ b/pkg/compactor/compactor_test.go @@ -0,0 +1,2298 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/grafana/mimir/blob/main/pkg/compactor/compactor_test.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Cortex Authors. + +package compactor + +import ( + "bytes" + "context" + "crypto/rand" + "encoding/json" + "flag" + "fmt" + "os" + "path" + "path/filepath" + "strconv" + "strings" + "testing" + "time" + + "github.com/go-kit/log" + "github.com/grafana/dskit/concurrency" + "github.com/grafana/dskit/flagext" + "github.com/grafana/dskit/kv/consul" + "github.com/grafana/dskit/ring" + "github.com/grafana/dskit/services" + "github.com/grafana/dskit/test" + "github.com/grafana/regexp" + "github.com/oklog/ulid" + "github.com/pkg/errors" + "github.com/prometheus/client_golang/prometheus" + prom_testutil "github.com/prometheus/client_golang/prometheus/testutil" + "github.com/prometheus/prometheus/model/histogram" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/tsdb" + "github.com/prometheus/prometheus/tsdb/chunkenc" + "github.com/prometheus/prometheus/tsdb/chunks" + "github.com/prometheus/prometheus/tsdb/tsdbutil" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/mock" + "github.com/stretchr/testify/require" + "github.com/thanos-io/objstore" + "gopkg.in/yaml.v3" + + "github.com/grafana/mimir/pkg/storage/bucket" + "github.com/grafana/mimir/pkg/storage/bucket/filesystem" + mimir_tsdb "github.com/grafana/mimir/pkg/storage/tsdb" + "github.com/grafana/mimir/pkg/storage/tsdb/block" + "github.com/grafana/mimir/pkg/util/validation" +) + +func TestConfig_ShouldSupportYamlConfig(t *testing.T) { + yamlCfg := ` +block_ranges: [2h, 48h] +block_sync_concurrency: 123 +data_dir: /tmp +compaction_interval: 15m +compaction_retries: 123 +` + + cfg := Config{} + flagext.DefaultValues(&cfg) + assert.NoError(t, yaml.Unmarshal([]byte(yamlCfg), &cfg)) + assert.Equal(t, mimir_tsdb.DurationList{2 * time.Hour, 48 * time.Hour}, cfg.BlockRanges) + assert.Equal(t, 123, cfg.BlockSyncConcurrency) + assert.Equal(t, "/tmp", cfg.DataDir) + assert.Equal(t, 15*time.Minute, cfg.CompactionInterval) + assert.Equal(t, 123, cfg.CompactionRetries) +} + +func TestConfig_ShouldSupportCliFlags(t *testing.T) { + fs := flag.NewFlagSet("", flag.PanicOnError) + cfg := Config{} + cfg.RegisterFlags(fs, log.NewNopLogger()) + require.NoError(t, fs.Parse([]string{ + "-compactor.block-ranges=2h,48h", + "-compactor.block-sync-concurrency=123", + "-compactor.data-dir=/tmp", + "-compactor.compaction-interval=15m", + "-compactor.compaction-retries=123", + })) + + assert.Equal(t, mimir_tsdb.DurationList{2 * time.Hour, 48 * time.Hour}, cfg.BlockRanges) + assert.Equal(t, 123, cfg.BlockSyncConcurrency) + assert.Equal(t, "/tmp", cfg.DataDir) + assert.Equal(t, 15*time.Minute, cfg.CompactionInterval) + assert.Equal(t, 123, cfg.CompactionRetries) +} + +func TestConfig_Validate(t *testing.T) { + tests := map[string]struct { + setup func(cfg *Config) + expected string + }{ + "should pass with the default config": { + setup: func(cfg *Config) {}, + expected: "", + }, + "should pass with only 1 block range period": { + setup: func(cfg *Config) { + cfg.BlockRanges = mimir_tsdb.DurationList{time.Hour} + }, + expected: "", + }, + "should fail with non divisible block range periods": { + setup: func(cfg *Config) { + cfg.BlockRanges = mimir_tsdb.DurationList{2 * time.Hour, 12 * time.Hour, 24 * time.Hour, 30 * time.Hour} + }, + expected: errors.Errorf(errInvalidBlockRanges, 30*time.Hour, 24*time.Hour).Error(), + }, + "should fail on unknown compaction jobs order": { + setup: func(cfg *Config) { + cfg.CompactionJobsOrder = "everything-is-important" + }, + expected: errInvalidCompactionOrder.Error(), + }, + "should fail on invalid value of max-opening-blocks-concurrency": { + setup: func(cfg *Config) { cfg.MaxOpeningBlocksConcurrency = 0 }, + expected: errInvalidMaxOpeningBlocksConcurrency.Error(), + }, + "should fail on invalid value of max-closing-blocks-concurrency": { + setup: func(cfg *Config) { cfg.MaxClosingBlocksConcurrency = 0 }, + expected: errInvalidMaxClosingBlocksConcurrency.Error(), + }, + "should fail on invalid value of symbols-flushers-concurrency": { + setup: func(cfg *Config) { cfg.SymbolsFlushersConcurrency = 0 }, + expected: errInvalidSymbolFlushersConcurrency.Error(), + }, + } + + for testName, testData := range tests { + t.Run(testName, func(t *testing.T) { + cfg := &Config{} + flagext.DefaultValues(cfg) + testData.setup(cfg) + + if actualErr := cfg.Validate(); testData.expected != "" { + assert.EqualError(t, actualErr, testData.expected) + } else { + assert.NoError(t, actualErr) + } + }) + } +} + +func TestMultitenantCompactor_ShouldDoNothingOnNoUserBlocks(t *testing.T) { + t.Parallel() + + // No user blocks stored in the bucket. + bucketClient := &bucket.ClientMock{} + bucketClient.MockIter("", []string{}, nil) + cfg := prepareConfig(t) + c, _, _, logs, registry := prepare(t, cfg, bucketClient) + require.NoError(t, services.StartAndAwaitRunning(context.Background(), c)) + + // Compactor doesn't wait for blocks cleaner to finish, but our test checks for cleaner metrics. + require.NoError(t, c.blocksCleaner.AwaitRunning(context.Background())) + + // Wait until a run has completed. + test.Poll(t, time.Second, 1.0, func() interface{} { + return prom_testutil.ToFloat64(c.compactionRunsCompleted) + }) + + require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) + + assert.Equal(t, prom_testutil.ToFloat64(c.compactionRunInterval), cfg.CompactionInterval.Seconds()) + + assert.Equal(t, []string{ + `level=info component=compactor msg="waiting until compactor is ACTIVE in the ring"`, + `level=info component=compactor msg="compactor is ACTIVE in the ring"`, + `level=info component=compactor msg="discovering users from bucket"`, + `level=info component=compactor msg="discovered users from bucket" users=0`, + }, removeIgnoredLogs(strings.Split(strings.TrimSpace(logs.String()), "\n"))) + + assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` + # TYPE cortex_compactor_runs_started_total counter + # HELP cortex_compactor_runs_started_total Total number of compaction runs started. + cortex_compactor_runs_started_total 1 + + # TYPE cortex_compactor_runs_completed_total counter + # HELP cortex_compactor_runs_completed_total Total number of compaction runs successfully completed. + cortex_compactor_runs_completed_total 1 + + # TYPE cortex_compactor_runs_failed_total counter + # HELP cortex_compactor_runs_failed_total Total number of compaction runs failed. + cortex_compactor_runs_failed_total{reason="error"} 0 + cortex_compactor_runs_failed_total{reason="shutdown"} 0 + + # HELP cortex_compactor_garbage_collection_duration_seconds Time it took to perform garbage collection iteration. + # TYPE cortex_compactor_garbage_collection_duration_seconds histogram + cortex_compactor_garbage_collection_duration_seconds_bucket{le="+Inf"} 0 + cortex_compactor_garbage_collection_duration_seconds_sum 0 + cortex_compactor_garbage_collection_duration_seconds_count 0 + + # HELP cortex_compactor_garbage_collection_failures_total Total number of failed garbage collection operations. + # TYPE cortex_compactor_garbage_collection_failures_total counter + cortex_compactor_garbage_collection_failures_total 0 + + # HELP cortex_compactor_garbage_collection_total Total number of garbage collection operations. + # TYPE cortex_compactor_garbage_collection_total counter + cortex_compactor_garbage_collection_total 0 + + # HELP cortex_compactor_meta_sync_duration_seconds Duration of the blocks metadata synchronization in seconds. + # TYPE cortex_compactor_meta_sync_duration_seconds histogram + cortex_compactor_meta_sync_duration_seconds_bucket{le="+Inf"} 0 + cortex_compactor_meta_sync_duration_seconds_sum 0 + cortex_compactor_meta_sync_duration_seconds_count 0 + + # HELP cortex_compactor_meta_sync_failures_total Total blocks metadata synchronization failures. + # TYPE cortex_compactor_meta_sync_failures_total counter + cortex_compactor_meta_sync_failures_total 0 + + # HELP cortex_compactor_meta_syncs_total Total blocks metadata synchronization attempts. + # TYPE cortex_compactor_meta_syncs_total counter + cortex_compactor_meta_syncs_total 0 + + # HELP cortex_compactor_group_compaction_runs_completed_total Total number of group completed compaction runs. This also includes compactor group runs that resulted with no compaction. + # TYPE cortex_compactor_group_compaction_runs_completed_total counter + cortex_compactor_group_compaction_runs_completed_total 0 + + # HELP cortex_compactor_group_compaction_runs_started_total Total number of group compaction attempts. + # TYPE cortex_compactor_group_compaction_runs_started_total counter + cortex_compactor_group_compaction_runs_started_total 0 + + # HELP cortex_compactor_group_compactions_failures_total Total number of failed group compactions. + # TYPE cortex_compactor_group_compactions_failures_total counter + cortex_compactor_group_compactions_failures_total 0 + + # HELP cortex_compactor_group_compactions_total Total number of group compaction attempts that resulted in new block(s). + # TYPE cortex_compactor_group_compactions_total counter + cortex_compactor_group_compactions_total 0 + + # TYPE cortex_compactor_block_cleanup_failures_total counter + # HELP cortex_compactor_block_cleanup_failures_total Total number of blocks failed to be deleted. + cortex_compactor_block_cleanup_failures_total 0 + + # HELP cortex_compactor_blocks_cleaned_total Total number of blocks deleted. + # TYPE cortex_compactor_blocks_cleaned_total counter + cortex_compactor_blocks_cleaned_total 0 + + # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. + # TYPE cortex_compactor_blocks_marked_for_deletion_total counter + cortex_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 + cortex_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 + cortex_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 + + # TYPE cortex_compactor_block_cleanup_started_total counter + # HELP cortex_compactor_block_cleanup_started_total Total number of blocks cleanup runs started. + cortex_compactor_block_cleanup_started_total 1 + + # TYPE cortex_compactor_block_cleanup_completed_total counter + # HELP cortex_compactor_block_cleanup_completed_total Total number of blocks cleanup runs successfully completed. + cortex_compactor_block_cleanup_completed_total 1 + + # TYPE cortex_compactor_block_cleanup_failed_total counter + # HELP cortex_compactor_block_cleanup_failed_total Total number of blocks cleanup runs failed. + cortex_compactor_block_cleanup_failed_total 0 + `), + "cortex_compactor_runs_started_total", + "cortex_compactor_runs_completed_total", + "cortex_compactor_runs_failed_total", + "cortex_compactor_garbage_collection_duration_seconds", + "cortex_compactor_garbage_collection_failures_total", + "cortex_compactor_garbage_collection_total", + "cortex_compactor_meta_sync_duration_seconds", + "cortex_compactor_meta_sync_failures_total", + "cortex_compactor_meta_syncs_total", + "cortex_compactor_group_compaction_runs_completed_total", + "cortex_compactor_group_compaction_runs_started_total", + "cortex_compactor_group_compactions_failures_total", + "cortex_compactor_group_compactions_total", + "cortex_compactor_block_cleanup_failures_total", + "cortex_compactor_blocks_cleaned_total", + "cortex_compactor_blocks_marked_for_deletion_total", + "cortex_compactor_block_cleanup_started_total", + "cortex_compactor_block_cleanup_completed_total", + "cortex_compactor_block_cleanup_failed_total", + )) +} + +func TestMultitenantCompactor_ShouldRetryCompactionOnFailureWhileDiscoveringUsersFromBucket(t *testing.T) { + t.Parallel() + + // Fail to iterate over the bucket while discovering users. + bucketClient := &bucket.ClientMock{} + bucketClient.MockIter("", nil, errors.New("failed to iterate the bucket")) + + c, _, _, logs, registry := prepare(t, prepareConfig(t), bucketClient) + require.NoError(t, services.StartAndAwaitRunning(context.Background(), c)) + + // Compactor doesn't wait for blocks cleaner to finish, but our test checks for cleaner metrics. + require.NoError(t, c.blocksCleaner.AwaitRunning(context.Background())) + t.Cleanup(func() { + t.Log(logs.String()) + }) + + // Wait until all retry attempts have completed. + test.Poll(t, time.Second, 1.0, func() interface{} { + return prom_testutil.ToFloat64(c.compactionRunsErred) + }) + + require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) + + // Ensure the bucket iteration has been retried the configured number of times. + bucketClient.AssertNumberOfCalls(t, "Iter", 1+3) + + assert.Equal(t, []string{ + `level=info component=compactor msg="waiting until compactor is ACTIVE in the ring"`, + `level=info component=compactor msg="compactor is ACTIVE in the ring"`, + `level=info component=compactor msg="discovering users from bucket"`, + `level=error component=compactor msg="failed to discover users from bucket" err="failed to iterate the bucket"`, + }, removeIgnoredLogs(strings.Split(strings.TrimSpace(logs.String()), "\n"))) + + assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` + # TYPE cortex_compactor_runs_started_total counter + # HELP cortex_compactor_runs_started_total Total number of compaction runs started. + cortex_compactor_runs_started_total 1 + + # TYPE cortex_compactor_runs_completed_total counter + # HELP cortex_compactor_runs_completed_total Total number of compaction runs successfully completed. + cortex_compactor_runs_completed_total 0 + + # TYPE cortex_compactor_runs_failed_total counter + # HELP cortex_compactor_runs_failed_total Total number of compaction runs failed. + cortex_compactor_runs_failed_total{reason="error"} 1 + cortex_compactor_runs_failed_total{reason="shutdown"} 0 + + # HELP cortex_compactor_garbage_collection_duration_seconds Time it took to perform garbage collection iteration. + # TYPE cortex_compactor_garbage_collection_duration_seconds histogram + cortex_compactor_garbage_collection_duration_seconds_bucket{le="+Inf"} 0 + cortex_compactor_garbage_collection_duration_seconds_sum 0 + cortex_compactor_garbage_collection_duration_seconds_count 0 + + # HELP cortex_compactor_garbage_collection_failures_total Total number of failed garbage collection operations. + # TYPE cortex_compactor_garbage_collection_failures_total counter + cortex_compactor_garbage_collection_failures_total 0 + + # HELP cortex_compactor_garbage_collection_total Total number of garbage collection operations. + # TYPE cortex_compactor_garbage_collection_total counter + cortex_compactor_garbage_collection_total 0 + + # HELP cortex_compactor_meta_sync_duration_seconds Duration of the blocks metadata synchronization in seconds. + # TYPE cortex_compactor_meta_sync_duration_seconds histogram + cortex_compactor_meta_sync_duration_seconds_bucket{le="+Inf"} 0 + cortex_compactor_meta_sync_duration_seconds_sum 0 + cortex_compactor_meta_sync_duration_seconds_count 0 + + # HELP cortex_compactor_meta_sync_failures_total Total blocks metadata synchronization failures. + # TYPE cortex_compactor_meta_sync_failures_total counter + cortex_compactor_meta_sync_failures_total 0 + + # HELP cortex_compactor_meta_syncs_total Total blocks metadata synchronization attempts. + # TYPE cortex_compactor_meta_syncs_total counter + cortex_compactor_meta_syncs_total 0 + + # HELP cortex_compactor_group_compaction_runs_completed_total Total number of group completed compaction runs. This also includes compactor group runs that resulted with no compaction. + # TYPE cortex_compactor_group_compaction_runs_completed_total counter + cortex_compactor_group_compaction_runs_completed_total 0 + + # HELP cortex_compactor_group_compaction_runs_started_total Total number of group compaction attempts. + # TYPE cortex_compactor_group_compaction_runs_started_total counter + cortex_compactor_group_compaction_runs_started_total 0 + + # HELP cortex_compactor_group_compactions_failures_total Total number of failed group compactions. + # TYPE cortex_compactor_group_compactions_failures_total counter + cortex_compactor_group_compactions_failures_total 0 + + # HELP cortex_compactor_group_compactions_total Total number of group compaction attempts that resulted in new block(s). + # TYPE cortex_compactor_group_compactions_total counter + cortex_compactor_group_compactions_total 0 + + # TYPE cortex_compactor_block_cleanup_failures_total counter + # HELP cortex_compactor_block_cleanup_failures_total Total number of blocks failed to be deleted. + cortex_compactor_block_cleanup_failures_total 0 + + # HELP cortex_compactor_blocks_cleaned_total Total number of blocks deleted. + # TYPE cortex_compactor_blocks_cleaned_total counter + cortex_compactor_blocks_cleaned_total 0 + + # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. + # TYPE cortex_compactor_blocks_marked_for_deletion_total counter + cortex_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 + cortex_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 + cortex_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 + + # TYPE cortex_compactor_block_cleanup_started_total counter + # HELP cortex_compactor_block_cleanup_started_total Total number of blocks cleanup runs started. + cortex_compactor_block_cleanup_started_total 1 + + # TYPE cortex_compactor_block_cleanup_completed_total counter + # HELP cortex_compactor_block_cleanup_completed_total Total number of blocks cleanup runs successfully completed. + cortex_compactor_block_cleanup_completed_total 0 + + # TYPE cortex_compactor_block_cleanup_failed_total counter + # HELP cortex_compactor_block_cleanup_failed_total Total number of blocks cleanup runs failed. + cortex_compactor_block_cleanup_failed_total 1 + `), + "cortex_compactor_runs_started_total", + "cortex_compactor_runs_completed_total", + "cortex_compactor_runs_failed_total", + "cortex_compactor_garbage_collection_duration_seconds", + "cortex_compactor_garbage_collection_failures_total", + "cortex_compactor_garbage_collection_total", + "cortex_compactor_meta_sync_duration_seconds", + "cortex_compactor_meta_sync_failures_total", + "cortex_compactor_meta_syncs_total", + "cortex_compactor_group_compaction_runs_completed_total", + "cortex_compactor_group_compaction_runs_started_total", + "cortex_compactor_group_compactions_failures_total", + "cortex_compactor_group_compactions_total", + "cortex_compactor_block_cleanup_failures_total", + "cortex_compactor_blocks_cleaned_total", + "cortex_compactor_blocks_marked_for_deletion_total", + "cortex_compactor_block_cleanup_started_total", + "cortex_compactor_block_cleanup_completed_total", + "cortex_compactor_block_cleanup_failed_total", + )) +} + +func TestMultitenantCompactor_ShouldIncrementCompactionErrorIfFailedToCompactASingleTenant(t *testing.T) { + t.Parallel() + + userID := "test-user" + bucketClient := &bucket.ClientMock{} + bucketClient.MockIter("", []string{userID}, nil) + bucketClient.MockIter(userID+"/", []string{userID + "/01DTVP434PA9VFXSW2JKB3392D", userID + "/01DTW0ZCPDDNV4BV83Q2SV4QAZ"}, nil) + bucketClient.MockIter(userID+"/markers/", nil, nil) + bucketClient.MockExists(path.Join(userID, mimir_tsdb.TenantDeletionMarkPath), false, nil) + bucketClient.MockGet(userID+"/01DTVP434PA9VFXSW2JKB3392D/meta.json", mockBlockMetaJSON("01DTVP434PA9VFXSW2JKB3392D"), nil) + bucketClient.MockGet(userID+"/01DTVP434PA9VFXSW2JKB3392D/deletion-mark.json", "", nil) + bucketClient.MockGet(userID+"/01DTVP434PA9VFXSW2JKB3392D/no-compact-mark.json", "", nil) + bucketClient.MockGet(userID+"/01DTW0ZCPDDNV4BV83Q2SV4QAZ/meta.json", mockBlockMetaJSON("01DTW0ZCPDDNV4BV83Q2SV4QAZ"), nil) + bucketClient.MockGet(userID+"/01DTW0ZCPDDNV4BV83Q2SV4QAZ/deletion-mark.json", "", nil) + bucketClient.MockGet(userID+"/01DTW0ZCPDDNV4BV83Q2SV4QAZ/no-compact-mark.json", "", nil) + bucketClient.MockGet(userID+"/bucket-index.json.gz", "", nil) + bucketClient.MockUpload(userID+"/bucket-index.json.gz", nil) + + c, _, tsdbPlannerMock, _, registry := prepare(t, prepareConfig(t), bucketClient) + tsdbPlannerMock.On("Plan", mock.Anything, mock.Anything).Return([]*block.Meta{}, errors.New("Failed to plan")) + require.NoError(t, services.StartAndAwaitRunning(context.Background(), c)) + + // Wait until all retry attempts have completed. + test.Poll(t, time.Minute, 1.0, func() interface{} { + return prom_testutil.ToFloat64(c.compactionRunsErred) + }) + + require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) + + assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` + # TYPE cortex_compactor_runs_started_total counter + # HELP cortex_compactor_runs_started_total Total number of compaction runs started. + cortex_compactor_runs_started_total 1 + + # TYPE cortex_compactor_runs_completed_total counter + # HELP cortex_compactor_runs_completed_total Total number of compaction runs successfully completed. + cortex_compactor_runs_completed_total 0 + + # TYPE cortex_compactor_runs_failed_total counter + # HELP cortex_compactor_runs_failed_total Total number of compaction runs failed. + cortex_compactor_runs_failed_total{reason="error"} 1 + cortex_compactor_runs_failed_total{reason="shutdown"} 0 + `), + "cortex_compactor_runs_started_total", + "cortex_compactor_runs_completed_total", + "cortex_compactor_runs_failed_total", + )) +} + +func TestMultitenantCompactor_ShouldIncrementCompactionShutdownIfTheContextIsCancelled(t *testing.T) { + t.Parallel() + + userID := "test-user" + bucketClient := &bucket.ClientMock{} + bucketClient.MockIter("", []string{userID}, nil) + bucketClient.MockIter(userID+"/", []string{userID + "/01DTVP434PA9VFXSW2JKB3392D", userID + "/01DTW0ZCPDDNV4BV83Q2SV4QAZ"}, nil) + bucketClient.MockIter(userID+"/markers/", nil, nil) + bucketClient.MockExists(path.Join(userID, mimir_tsdb.TenantDeletionMarkPath), false, nil) + bucketClient.MockGet(userID+"/01DTVP434PA9VFXSW2JKB3392D/meta.json", mockBlockMetaJSON("01DTVP434PA9VFXSW2JKB3392D"), nil) + bucketClient.MockGet(userID+"/01DTVP434PA9VFXSW2JKB3392D/deletion-mark.json", "", nil) + bucketClient.MockGet(userID+"/01DTVP434PA9VFXSW2JKB3392D/no-compact-mark.json", "", nil) + bucketClient.MockGet(userID+"/01DTW0ZCPDDNV4BV83Q2SV4QAZ/meta.json", mockBlockMetaJSON("01DTW0ZCPDDNV4BV83Q2SV4QAZ"), nil) + bucketClient.MockGet(userID+"/01DTW0ZCPDDNV4BV83Q2SV4QAZ/deletion-mark.json", "", nil) + bucketClient.MockGet(userID+"/01DTW0ZCPDDNV4BV83Q2SV4QAZ/no-compact-mark.json", "", nil) + bucketClient.MockGet(userID+"/bucket-index.json.gz", "", nil) + bucketClient.MockUpload(userID+"/bucket-index.json.gz", nil) + + c, _, tsdbPlannerMock, logs, registry := prepare(t, prepareConfig(t), bucketClient) + t.Cleanup(func() { + t.Log(logs.String()) + }) + // Mock the planner as if a shutdown was triggered and the service was terminated. + tsdbPlannerMock.On("Plan", mock.Anything, mock.Anything).Return([]*block.Meta{}, context.Canceled) + require.NoError(t, services.StartAndAwaitRunning(context.Background(), c)) + + // Wait until the error is recorded. + test.Poll(t, time.Second, 1.0, func() interface{} { + return prom_testutil.ToFloat64(c.compactionRunsShutdown) + }) + + require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) + + assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` + # TYPE cortex_compactor_runs_started_total counter + # HELP cortex_compactor_runs_started_total Total number of compaction runs started. + cortex_compactor_runs_started_total 1 + + # TYPE cortex_compactor_runs_completed_total counter + # HELP cortex_compactor_runs_completed_total Total number of compaction runs successfully completed. + cortex_compactor_runs_completed_total 0 + + # TYPE cortex_compactor_runs_failed_total counter + # HELP cortex_compactor_runs_failed_total Total number of compaction runs failed. + cortex_compactor_runs_failed_total{reason="error"} 0 + cortex_compactor_runs_failed_total{reason="shutdown"} 1 + `), + "cortex_compactor_runs_started_total", + "cortex_compactor_runs_completed_total", + "cortex_compactor_runs_failed_total", + )) +} + +func TestMultitenantCompactor_ShouldIterateOverUsersAndRunCompaction(t *testing.T) { + t.Parallel() + + // Mock the bucket to contain two users, each one with two blocks (to make sure that grouper doesn't skip them). + bucketClient := &bucket.ClientMock{} + bucketClient.MockIter("", []string{"user-1", "user-2"}, nil) + bucketClient.MockExists(path.Join("user-1", mimir_tsdb.TenantDeletionMarkPath), false, nil) + bucketClient.MockExists(path.Join("user-2", mimir_tsdb.TenantDeletionMarkPath), false, nil) + bucketClient.MockIter("user-1/", []string{"user-1/01DTVP434PA9VFXSW2JKB3392D", "user-1/01FS51A7GQ1RQWV35DBVYQM4KF"}, nil) + bucketClient.MockIter("user-2/", []string{"user-2/01DTW0ZCPDDNV4BV83Q2SV4QAZ", "user-2/01FRSF035J26D6CGX7STCSD1KG"}, nil) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", mockBlockMetaJSON("01DTVP434PA9VFXSW2JKB3392D"), nil) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/deletion-mark.json", "", nil) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/no-compact-mark.json", "", nil) + bucketClient.MockGet("user-1/01FS51A7GQ1RQWV35DBVYQM4KF/meta.json", mockBlockMetaJSON("01FS51A7GQ1RQWV35DBVYQM4KF"), nil) + bucketClient.MockGet("user-1/01FS51A7GQ1RQWV35DBVYQM4KF/deletion-mark.json", "", nil) + bucketClient.MockGet("user-1/01FS51A7GQ1RQWV35DBVYQM4KF/no-compact-mark.json", "", nil) + + bucketClient.MockGet("user-2/01DTW0ZCPDDNV4BV83Q2SV4QAZ/meta.json", mockBlockMetaJSON("01DTW0ZCPDDNV4BV83Q2SV4QAZ"), nil) + bucketClient.MockGet("user-2/01DTW0ZCPDDNV4BV83Q2SV4QAZ/deletion-mark.json", "", nil) + bucketClient.MockGet("user-2/01DTW0ZCPDDNV4BV83Q2SV4QAZ/no-compact-mark.json", "", nil) + bucketClient.MockGet("user-2/01FRSF035J26D6CGX7STCSD1KG/meta.json", mockBlockMetaJSON("01FRSF035J26D6CGX7STCSD1KG"), nil) + bucketClient.MockGet("user-2/01FRSF035J26D6CGX7STCSD1KG/deletion-mark.json", "", nil) + bucketClient.MockGet("user-2/01FRSF035J26D6CGX7STCSD1KG/no-compact-mark.json", "", nil) + bucketClient.MockGet("user-1/bucket-index.json.gz", "", nil) + bucketClient.MockGet("user-2/bucket-index.json.gz", "", nil) + bucketClient.MockIter("user-1/markers/", nil, nil) + bucketClient.MockIter("user-2/markers/", nil, nil) + bucketClient.MockUpload("user-1/bucket-index.json.gz", nil) + bucketClient.MockUpload("user-2/bucket-index.json.gz", nil) + + c, _, tsdbPlanner, logs, registry := prepare(t, prepareConfig(t), bucketClient) + + // Mock the planner as if there's no compaction to do, + // in order to simplify tests (all in all, we just want to + // test our logic and not TSDB compactor which we expect to + // be already tested). + tsdbPlanner.On("Plan", mock.Anything, mock.Anything).Return([]*block.Meta{}, nil) + + require.NoError(t, services.StartAndAwaitRunning(context.Background(), c)) + + // Compactor doesn't wait for blocks cleaner to finish, but our test checks for cleaner metrics. + require.NoError(t, c.blocksCleaner.AwaitRunning(context.Background())) + + // Wait until a run has completed. + test.Poll(t, time.Second, 1.0, func() interface{} { + return prom_testutil.ToFloat64(c.compactionRunsCompleted) + }) + + require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) + + // Ensure a plan has been executed for the blocks of each user. + tsdbPlanner.AssertNumberOfCalls(t, "Plan", 2) + + assert.ElementsMatch(t, []string{ + `level=info component=compactor msg="waiting until compactor is ACTIVE in the ring"`, + `level=info component=compactor msg="compactor is ACTIVE in the ring"`, + `level=info component=compactor msg="discovering users from bucket"`, + `level=info component=compactor msg="discovered users from bucket" users=2`, + `level=info component=compactor msg="starting compaction of user blocks" user=user-1`, + `level=info component=compactor user=user-1 msg="start sync of metas"`, + `level=info component=compactor user=user-1 msg="start of GC"`, + `level=debug component=compactor user=user-1 msg="grouper found a compactable blocks group" groupKey=0@17241709254077376921-merge--1574776800000-1574784000000 job="stage: merge, range start: 1574776800000, range end: 1574784000000, shard: , blocks: 01DTVP434PA9VFXSW2JKB3392D (min time: 2019-11-26 14:00:00 +0000 UTC, max time: 2019-11-26 16:00:00 +0000 UTC),01FS51A7GQ1RQWV35DBVYQM4KF (min time: 2019-11-26 14:00:00 +0000 UTC, max time: 2019-11-26 16:00:00 +0000 UTC)"`, + `level=info component=compactor user=user-1 msg="start of compactions"`, + `level=info component=compactor user=user-1 groupKey=0@17241709254077376921-merge--1574776800000-1574784000000 msg="compaction job succeeded"`, + `level=info component=compactor user=user-1 msg="compaction iterations done"`, + `level=info component=compactor msg="successfully compacted user blocks" user=user-1`, + `level=info component=compactor msg="starting compaction of user blocks" user=user-2`, + `level=info component=compactor user=user-2 msg="start sync of metas"`, + `level=info component=compactor user=user-2 msg="start of GC"`, + `level=debug component=compactor user=user-2 msg="grouper found a compactable blocks group" groupKey=0@17241709254077376921-merge--1574776800000-1574784000000 job="stage: merge, range start: 1574776800000, range end: 1574784000000, shard: , blocks: 01DTW0ZCPDDNV4BV83Q2SV4QAZ (min time: 2019-11-26 14:00:00 +0000 UTC, max time: 2019-11-26 16:00:00 +0000 UTC),01FRSF035J26D6CGX7STCSD1KG (min time: 2019-11-26 14:00:00 +0000 UTC, max time: 2019-11-26 16:00:00 +0000 UTC)"`, + `level=info component=compactor user=user-2 msg="start of compactions"`, + `level=info component=compactor user=user-2 groupKey=0@17241709254077376921-merge--1574776800000-1574784000000 msg="compaction job succeeded"`, + `level=info component=compactor user=user-2 msg="compaction iterations done"`, + `level=info component=compactor msg="successfully compacted user blocks" user=user-2`, + }, removeIgnoredLogs(strings.Split(strings.TrimSpace(logs.String()), "\n"))) + + // Instead of testing for shipper metrics, we only check our metrics here. + // Real shipper metrics are too variable to embed into a test. + testedMetrics := []string{ + "cortex_compactor_runs_started_total", "cortex_compactor_runs_completed_total", "cortex_compactor_runs_failed_total", + "cortex_compactor_blocks_cleaned_total", "cortex_compactor_block_cleanup_failures_total", "cortex_compactor_blocks_marked_for_deletion_total", + "cortex_compactor_block_cleanup_started_total", "cortex_compactor_block_cleanup_completed_total", "cortex_compactor_block_cleanup_failed_total", + "cortex_compactor_group_compaction_runs_completed_total", "cortex_compactor_group_compaction_runs_started_total", + "cortex_compactor_group_compactions_failures_total", "cortex_compactor_group_compactions_total", + } + + assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` + # TYPE cortex_compactor_runs_started_total counter + # HELP cortex_compactor_runs_started_total Total number of compaction runs started. + cortex_compactor_runs_started_total 1 + + # TYPE cortex_compactor_runs_completed_total counter + # HELP cortex_compactor_runs_completed_total Total number of compaction runs successfully completed. + cortex_compactor_runs_completed_total 1 + + # TYPE cortex_compactor_runs_failed_total counter + # HELP cortex_compactor_runs_failed_total Total number of compaction runs failed. + cortex_compactor_runs_failed_total{reason="error"} 0 + cortex_compactor_runs_failed_total{reason="shutdown"} 0 + + # HELP cortex_compactor_group_compaction_runs_completed_total Total number of group completed compaction runs. This also includes compactor group runs that resulted with no compaction. + # TYPE cortex_compactor_group_compaction_runs_completed_total counter + cortex_compactor_group_compaction_runs_completed_total 2 + + # HELP cortex_compactor_group_compaction_runs_started_total Total number of group compaction attempts. + # TYPE cortex_compactor_group_compaction_runs_started_total counter + cortex_compactor_group_compaction_runs_started_total 2 + + # HELP cortex_compactor_group_compactions_failures_total Total number of failed group compactions. + # TYPE cortex_compactor_group_compactions_failures_total counter + cortex_compactor_group_compactions_failures_total 0 + + # HELP cortex_compactor_group_compactions_total Total number of group compaction attempts that resulted in new block(s). + # TYPE cortex_compactor_group_compactions_total counter + cortex_compactor_group_compactions_total 0 + + # TYPE cortex_compactor_block_cleanup_failures_total counter + # HELP cortex_compactor_block_cleanup_failures_total Total number of blocks failed to be deleted. + cortex_compactor_block_cleanup_failures_total 0 + + # HELP cortex_compactor_blocks_cleaned_total Total number of blocks deleted. + # TYPE cortex_compactor_blocks_cleaned_total counter + cortex_compactor_blocks_cleaned_total 0 + + # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. + # TYPE cortex_compactor_blocks_marked_for_deletion_total counter + cortex_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 + cortex_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 + cortex_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 + + # TYPE cortex_compactor_block_cleanup_started_total counter + # HELP cortex_compactor_block_cleanup_started_total Total number of blocks cleanup runs started. + cortex_compactor_block_cleanup_started_total 1 + + # TYPE cortex_compactor_block_cleanup_completed_total counter + # HELP cortex_compactor_block_cleanup_completed_total Total number of blocks cleanup runs successfully completed. + cortex_compactor_block_cleanup_completed_total 1 + + # TYPE cortex_compactor_block_cleanup_failed_total counter + # HELP cortex_compactor_block_cleanup_failed_total Total number of blocks cleanup runs failed. + cortex_compactor_block_cleanup_failed_total 0 + `), testedMetrics...)) +} + +func TestMultitenantCompactor_ShouldStopCompactingTenantOnReachingMaxCompactionTime(t *testing.T) { + t.Parallel() + + // By using blocks with different labels, we get two compaction jobs. Only one of these jobs will be started, + // and since its planning will take longer than maxCompactionTime, we stop compactions early. + bucketClient := &bucket.ClientMock{} + bucketClient.MockIter("", []string{"user-1"}, nil) + bucketClient.MockExists(path.Join("user-1", mimir_tsdb.TenantDeletionMarkPath), false, nil) + bucketClient.MockIter("user-1/", []string{"user-1/01DTVP434PA9VFXSW2JKB3392D", "user-1/01FN3VCQV5X342W2ZKMQQXAZRX", "user-1/01FS51A7GQ1RQWV35DBVYQM4KF", "user-1/01FRQGQB7RWQ2TS0VWA82QTPXE"}, nil) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", mockBlockMetaJSONWithTimeRangeAndLabels("01DTVP434PA9VFXSW2JKB3392D", 1574776800000, 1574784000000, map[string]string{"A": "B"}), nil) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/deletion-mark.json", "", nil) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/no-compact-mark.json", "", nil) + bucketClient.MockGet("user-1/01FS51A7GQ1RQWV35DBVYQM4KF/meta.json", mockBlockMetaJSONWithTimeRangeAndLabels("01FS51A7GQ1RQWV35DBVYQM4KF", 1574776800000, 1574784000000, map[string]string{"A": "B"}), nil) + bucketClient.MockGet("user-1/01FS51A7GQ1RQWV35DBVYQM4KF/deletion-mark.json", "", nil) + bucketClient.MockGet("user-1/01FS51A7GQ1RQWV35DBVYQM4KF/no-compact-mark.json", "", nil) + bucketClient.MockGet("user-1/01FN3VCQV5X342W2ZKMQQXAZRX/meta.json", mockBlockMetaJSONWithTimeRangeAndLabels("01FN3VCQV5X342W2ZKMQQXAZRX", 1574776800000, 1574784000000, map[string]string{"C": "D"}), nil) + bucketClient.MockGet("user-1/01FN3VCQV5X342W2ZKMQQXAZRX/deletion-mark.json", "", nil) + bucketClient.MockGet("user-1/01FN3VCQV5X342W2ZKMQQXAZRX/no-compact-mark.json", "", nil) + bucketClient.MockGet("user-1/01FRQGQB7RWQ2TS0VWA82QTPXE/meta.json", mockBlockMetaJSONWithTimeRangeAndLabels("01FRQGQB7RWQ2TS0VWA82QTPXE", 1574776800000, 1574784000000, map[string]string{"C": "D"}), nil) + bucketClient.MockGet("user-1/01FRQGQB7RWQ2TS0VWA82QTPXE/deletion-mark.json", "", nil) + bucketClient.MockGet("user-1/01FRQGQB7RWQ2TS0VWA82QTPXE/no-compact-mark.json", "", nil) + bucketClient.MockGet("user-1/bucket-index.json.gz", "", nil) + bucketClient.MockIter("user-1/markers/", nil, nil) + bucketClient.MockUpload("user-1/bucket-index.json.gz", nil) + + cfg := prepareConfig(t) + cfg.MaxCompactionTime = 500 * time.Millisecond // Enough time to start one compaction. We will make it last longer than this. + cfg.CompactionConcurrency = 1 + + c, _, tsdbPlanner, logs, _ := prepare(t, cfg, bucketClient) + + // Planner is called at the beginning of each job. We make it return no work, but only after delay. + plannerDelay := 2 * cfg.MaxCompactionTime + tsdbPlanner.On("Plan", mock.Anything, mock.Anything).After(plannerDelay).Return([]*block.Meta{}, nil) + + require.NoError(t, services.StartAndAwaitRunning(context.Background(), c)) + + // Compactor doesn't wait for blocks cleaner to finish, but our test checks for cleaner metrics. + require.NoError(t, c.blocksCleaner.AwaitRunning(context.Background())) + + // Wait until a run has completed. Since planner takes "2*cfg.MaxCompactionTime", we wait for twice as long. + test.Poll(t, 2*plannerDelay, 1.0, func() interface{} { + return prom_testutil.ToFloat64(c.compactionRunsCompleted) + }) + + require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) + + // Ensure a plan has been called only once. + tsdbPlanner.AssertNumberOfCalls(t, "Plan", 1) + + assert.Equal(t, []string{ + `level=info component=compactor msg="waiting until compactor is ACTIVE in the ring"`, + `level=info component=compactor msg="compactor is ACTIVE in the ring"`, + `level=info component=compactor msg="discovering users from bucket"`, + `level=info component=compactor msg="discovered users from bucket" users=1`, + `level=info component=compactor msg="starting compaction of user blocks" user=user-1`, + `level=info component=compactor user=user-1 msg="start sync of metas"`, + `level=info component=compactor user=user-1 msg="start of GC"`, + `level=debug component=compactor user=user-1 msg="grouper found a compactable blocks group" groupKey=0@12695595599644216241-merge--1574776800000-1574784000000 job="stage: merge, range start: 1574776800000, range end: 1574784000000, shard: , blocks: 01FN3VCQV5X342W2ZKMQQXAZRX (min time: 2019-11-26 14:00:00 +0000 UTC, max time: 2019-11-26 16:00:00 +0000 UTC),01FRQGQB7RWQ2TS0VWA82QTPXE (min time: 2019-11-26 14:00:00 +0000 UTC, max time: 2019-11-26 16:00:00 +0000 UTC)"`, + `level=debug component=compactor user=user-1 msg="grouper found a compactable blocks group" groupKey=0@414047632870839233-merge--1574776800000-1574784000000 job="stage: merge, range start: 1574776800000, range end: 1574784000000, shard: , blocks: 01DTVP434PA9VFXSW2JKB3392D (min time: 2019-11-26 14:00:00 +0000 UTC, max time: 2019-11-26 16:00:00 +0000 UTC),01FS51A7GQ1RQWV35DBVYQM4KF (min time: 2019-11-26 14:00:00 +0000 UTC, max time: 2019-11-26 16:00:00 +0000 UTC)"`, + `level=info component=compactor user=user-1 msg="start of compactions"`, + `level=info component=compactor user=user-1 msg="max compaction time reached, no more compactions will be started"`, + `level=info component=compactor user=user-1 groupKey=0@12695595599644216241-merge--1574776800000-1574784000000 msg="compaction job succeeded"`, + `level=info component=compactor user=user-1 msg="compaction iterations done"`, + `level=info component=compactor msg="successfully compacted user blocks" user=user-1`, + }, removeIgnoredLogs(strings.Split(strings.TrimSpace(logs.String()), "\n"))) +} + +func TestMultitenantCompactor_ShouldNotCompactBlocksMarkedForDeletion(t *testing.T) { + t.Parallel() + + cfg := prepareConfig(t) + cfg.DeletionDelay = 10 * time.Minute // Delete block after 10 minutes + + // Mock the bucket to contain two users, each one with one block. + bucketClient := &bucket.ClientMock{} + bucketClient.MockIter("", []string{"user-1"}, nil) + bucketClient.MockIter("user-1/", []string{"user-1/01DTVP434PA9VFXSW2JKB3392D", "user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ"}, nil) + bucketClient.MockExists(path.Join("user-1", mimir_tsdb.TenantDeletionMarkPath), false, nil) + + // Block that has just been marked for deletion. It will not be deleted just yet, and it also will not be compacted. + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", mockBlockMetaJSON("01DTVP434PA9VFXSW2JKB3392D"), nil) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/deletion-mark.json", mockDeletionMarkJSON("01DTVP434PA9VFXSW2JKB3392D", time.Now()), nil) + bucketClient.MockGet("user-1/markers/01DTVP434PA9VFXSW2JKB3392D-deletion-mark.json", mockDeletionMarkJSON("01DTVP434PA9VFXSW2JKB3392D", time.Now()), nil) + + // This block will be deleted by cleaner. + bucketClient.MockGet("user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ/meta.json", mockBlockMetaJSON("01DTW0ZCPDDNV4BV83Q2SV4QAZ"), nil) + bucketClient.MockGet("user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ/deletion-mark.json", mockDeletionMarkJSON("01DTW0ZCPDDNV4BV83Q2SV4QAZ", time.Now().Add(-cfg.DeletionDelay)), nil) + bucketClient.MockGet("user-1/markers/01DTW0ZCPDDNV4BV83Q2SV4QAZ-deletion-mark.json", mockDeletionMarkJSON("01DTW0ZCPDDNV4BV83Q2SV4QAZ", time.Now().Add(-cfg.DeletionDelay)), nil) + + bucketClient.MockIter("user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ", []string{ + "user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ/meta.json", + "user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ/deletion-mark.json", + }, nil) + + bucketClient.MockIter("user-1/markers/", []string{ + "user-1/markers/01DTVP434PA9VFXSW2JKB3392D-deletion-mark.json", + "user-1/markers/01DTW0ZCPDDNV4BV83Q2SV4QAZ-deletion-mark.json", + }, nil) + + bucketClient.MockDelete("user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ/meta.json", nil) + bucketClient.MockDelete("user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ/deletion-mark.json", nil) + bucketClient.MockDelete("user-1/markers/01DTW0ZCPDDNV4BV83Q2SV4QAZ-deletion-mark.json", nil) + bucketClient.MockDelete("user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ", nil) + bucketClient.MockGet("user-1/bucket-index.json.gz", "", nil) + bucketClient.MockUpload("user-1/bucket-index.json.gz", nil) + + c, _, tsdbPlanner, logs, registry := prepare(t, cfg, bucketClient) + + require.NoError(t, services.StartAndAwaitRunning(context.Background(), c)) + + // Compactor doesn't wait for blocks cleaner to finish, but our test checks for cleaner metrics. + require.NoError(t, c.blocksCleaner.AwaitRunning(context.Background())) + + // Wait until a run has completed. + test.Poll(t, time.Second, 1.0, func() interface{} { + return prom_testutil.ToFloat64(c.compactionRunsCompleted) + }) + + require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) + + // Since both blocks are marked for deletion, none of them are going to be compacted. + tsdbPlanner.AssertNumberOfCalls(t, "Plan", 0) + + assert.ElementsMatch(t, []string{ + `level=info component=compactor msg="waiting until compactor is ACTIVE in the ring"`, + `level=info component=compactor msg="compactor is ACTIVE in the ring"`, + `level=info component=compactor msg="discovering users from bucket"`, + `level=info component=compactor msg="discovered users from bucket" users=1`, + `level=info component=compactor msg="starting compaction of user blocks" user=user-1`, + `level=info component=compactor user=user-1 msg="start sync of metas"`, + `level=info component=compactor user=user-1 msg="start of GC"`, + `level=info component=compactor user=user-1 msg="start of compactions"`, + `level=info component=compactor user=user-1 msg="compaction iterations done"`, + `level=info component=compactor msg="successfully compacted user blocks" user=user-1`, + }, removeIgnoredLogs(strings.Split(strings.TrimSpace(logs.String()), "\n"))) + + // Instead of testing for shipper metrics, we only check our metrics here. + // Real shipper metrics are too variable to embed into a test. + testedMetrics := []string{ + "cortex_compactor_runs_started_total", "cortex_compactor_runs_completed_total", "cortex_compactor_runs_failed_total", + "cortex_compactor_blocks_cleaned_total", "cortex_compactor_block_cleanup_failures_total", "cortex_compactor_blocks_marked_for_deletion_total", + "cortex_compactor_block_cleanup_started_total", "cortex_compactor_block_cleanup_completed_total", "cortex_compactor_block_cleanup_failed_total", + } + assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` + # TYPE cortex_compactor_runs_started_total counter + # HELP cortex_compactor_runs_started_total Total number of compaction runs started. + cortex_compactor_runs_started_total 1 + + # TYPE cortex_compactor_runs_completed_total counter + # HELP cortex_compactor_runs_completed_total Total number of compaction runs successfully completed. + cortex_compactor_runs_completed_total 1 + + # TYPE cortex_compactor_runs_failed_total counter + # HELP cortex_compactor_runs_failed_total Total number of compaction runs failed. + cortex_compactor_runs_failed_total{reason="error"} 0 + cortex_compactor_runs_failed_total{reason="shutdown"} 0 + + # TYPE cortex_compactor_block_cleanup_failures_total counter + # HELP cortex_compactor_block_cleanup_failures_total Total number of blocks failed to be deleted. + cortex_compactor_block_cleanup_failures_total 0 + + # HELP cortex_compactor_blocks_cleaned_total Total number of blocks deleted. + # TYPE cortex_compactor_blocks_cleaned_total counter + cortex_compactor_blocks_cleaned_total 1 + + # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. + # TYPE cortex_compactor_blocks_marked_for_deletion_total counter + cortex_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 + cortex_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 + cortex_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 + + # TYPE cortex_compactor_block_cleanup_started_total counter + # HELP cortex_compactor_block_cleanup_started_total Total number of blocks cleanup runs started. + cortex_compactor_block_cleanup_started_total 1 + + # TYPE cortex_compactor_block_cleanup_completed_total counter + # HELP cortex_compactor_block_cleanup_completed_total Total number of blocks cleanup runs successfully completed. + cortex_compactor_block_cleanup_completed_total 1 + + # TYPE cortex_compactor_block_cleanup_failed_total counter + # HELP cortex_compactor_block_cleanup_failed_total Total number of blocks cleanup runs failed. + cortex_compactor_block_cleanup_failed_total 0 + `), testedMetrics...)) +} + +func TestMultitenantCompactor_ShouldNotCompactBlocksMarkedForNoCompaction(t *testing.T) { + t.Parallel() + + cfg := prepareConfig(t) + cfg.DeletionDelay = 10 * time.Minute // Delete block after 10 minutes + + // Mock the bucket to contain one user with a block marked for no-compaction. + bucketClient := &bucket.ClientMock{} + bucketClient.MockIter("", []string{"user-1"}, nil) + bucketClient.MockIter("user-1/", []string{"user-1/01DTVP434PA9VFXSW2JKB3392D"}, nil) + bucketClient.MockExists(path.Join("user-1", mimir_tsdb.TenantDeletionMarkPath), false, nil) + + // Block that is marked for no compaction. It will be ignored. + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", mockBlockMetaJSON("01DTVP434PA9VFXSW2JKB3392D"), nil) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/deletion-mark.json", "", nil) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/no-compact-mark.json", `{"id":"01DTVP434PA9VFXSW2JKB3392D","version":1,"details":"details","no_compact_time":1637757932,"reason":"reason"}`, nil) + + bucketClient.MockIter("user-1/markers/", []string{"user-1/markers/01DTVP434PA9VFXSW2JKB3392D-no-compact-mark.json"}, nil) + + bucketClient.MockGet("user-1/bucket-index.json.gz", "", nil) + bucketClient.MockUpload("user-1/bucket-index.json.gz", nil) + + c, _, tsdbPlanner, logs, _ := prepare(t, cfg, bucketClient) + + require.NoError(t, services.StartAndAwaitRunning(context.Background(), c)) + + // Compactor doesn't wait for blocks cleaner to finish, but our test checks for cleaner metrics. + require.NoError(t, c.blocksCleaner.AwaitRunning(context.Background())) + + // Wait until a run has completed. + test.Poll(t, time.Second, 1.0, func() interface{} { + return prom_testutil.ToFloat64(c.compactionRunsCompleted) + }) + + require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) + + // Since block is not compacted, there will be no planning done. + tsdbPlanner.AssertNumberOfCalls(t, "Plan", 0) + + assert.ElementsMatch(t, []string{ + `level=info component=compactor msg="waiting until compactor is ACTIVE in the ring"`, + `level=info component=compactor msg="compactor is ACTIVE in the ring"`, + `level=info component=compactor msg="discovering users from bucket"`, + `level=info component=compactor msg="discovered users from bucket" users=1`, + `level=info component=compactor msg="starting compaction of user blocks" user=user-1`, + `level=info component=compactor user=user-1 msg="start sync of metas"`, + `level=info component=compactor user=user-1 msg="start of GC"`, + `level=info component=compactor user=user-1 msg="start of compactions"`, + `level=info component=compactor user=user-1 msg="compaction iterations done"`, + `level=info component=compactor msg="successfully compacted user blocks" user=user-1`, + }, removeIgnoredLogs(strings.Split(strings.TrimSpace(logs.String()), "\n"))) +} + +func TestMultitenantCompactor_ShouldNotCompactBlocksForUsersMarkedForDeletion(t *testing.T) { + t.Parallel() + + cfg := prepareConfig(t) + cfg.DeletionDelay = 10 * time.Minute // Delete block after 10 minutes + cfg.TenantCleanupDelay = 10 * time.Minute // To make sure it's not 0. + + // Mock the bucket to contain two users, each one with one block. + bucketClient := &bucket.ClientMock{} + bucketClient.MockIter("", []string{"user-1"}, nil) + bucketClient.MockIter("user-1/", []string{"user-1/01DTVP434PA9VFXSW2JKB3392D"}, nil) + bucketClient.MockGet(path.Join("user-1", mimir_tsdb.TenantDeletionMarkPath), `{"deletion_time": 1}`, nil) + bucketClient.MockUpload(path.Join("user-1", mimir_tsdb.TenantDeletionMarkPath), nil) + + bucketClient.MockIter("user-1/01DTVP434PA9VFXSW2JKB3392D", []string{"user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", "user-1/01DTVP434PA9VFXSW2JKB3392D/index"}, nil) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", mockBlockMetaJSON("01DTVP434PA9VFXSW2JKB3392D"), nil) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/index", "some index content", nil) + bucketClient.MockExists("user-1/01DTVP434PA9VFXSW2JKB3392D/deletion-mark.json", false, nil) + bucketClient.MockExists("user-1/markers/01DTVP434PA9VFXSW2JKB3392D-deletion-mark.json", false, nil) + + bucketClient.MockDelete("user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", nil) + bucketClient.MockDelete("user-1/01DTVP434PA9VFXSW2JKB3392D/index", nil) + bucketClient.MockDelete("user-1/bucket-index.json.gz", nil) + + c, _, tsdbPlanner, logs, registry := prepare(t, cfg, bucketClient) + + // Mock the planner as if there's no compaction to do, + // in order to simplify tests (all in all, we just want to + // test our logic and not TSDB compactor which we expect to + // be already tested). + tsdbPlanner.On("Plan", mock.Anything, mock.Anything).Return([]*block.Meta{}, nil) + + require.NoError(t, services.StartAndAwaitRunning(context.Background(), c)) + + // Compactor doesn't wait for blocks cleaner to finish, but our test checks for cleaner metrics. + require.NoError(t, c.blocksCleaner.AwaitRunning(context.Background())) + + // Wait until a run has completed. + test.Poll(t, time.Second, 1.0, func() interface{} { + return prom_testutil.ToFloat64(c.compactionRunsCompleted) + }) + + require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) + + // No user is compacted, single user we have is marked for deletion. + tsdbPlanner.AssertNumberOfCalls(t, "Plan", 0) + + assert.ElementsMatch(t, []string{ + `level=info component=compactor msg="waiting until compactor is ACTIVE in the ring"`, + `level=info component=compactor msg="compactor is ACTIVE in the ring"`, + `level=info component=compactor msg="discovering users from bucket"`, + `level=info component=compactor msg="discovered users from bucket" users=1`, + `level=debug component=compactor msg="skipping user because it is marked for deletion" user=user-1`, + }, removeIgnoredLogs(strings.Split(strings.TrimSpace(logs.String()), "\n"))) + + // Instead of testing for shipper metrics, we only check our metrics here. + // Real shipper metrics are too variable to embed into a test. + testedMetrics := []string{ + "cortex_compactor_runs_started_total", "cortex_compactor_runs_completed_total", "cortex_compactor_runs_failed_total", + "cortex_compactor_blocks_cleaned_total", "cortex_compactor_block_cleanup_failures_total", "cortex_compactor_blocks_marked_for_deletion_total", + "cortex_compactor_block_cleanup_started_total", "cortex_compactor_block_cleanup_completed_total", "cortex_compactor_block_cleanup_failed_total", + "cortex_bucket_blocks_count", "cortex_bucket_blocks_marked_for_deletion_count", "cortex_bucket_index_last_successful_update_timestamp_seconds", + } + assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` + # TYPE cortex_compactor_runs_started_total counter + # HELP cortex_compactor_runs_started_total Total number of compaction runs started. + cortex_compactor_runs_started_total 1 + + # TYPE cortex_compactor_runs_completed_total counter + # HELP cortex_compactor_runs_completed_total Total number of compaction runs successfully completed. + cortex_compactor_runs_completed_total 1 + + # TYPE cortex_compactor_runs_failed_total counter + # HELP cortex_compactor_runs_failed_total Total number of compaction runs failed. + cortex_compactor_runs_failed_total{reason="error"} 0 + cortex_compactor_runs_failed_total{reason="shutdown"} 0 + + # TYPE cortex_compactor_block_cleanup_failures_total counter + # HELP cortex_compactor_block_cleanup_failures_total Total number of blocks failed to be deleted. + cortex_compactor_block_cleanup_failures_total 0 + + # HELP cortex_compactor_blocks_cleaned_total Total number of blocks deleted. + # TYPE cortex_compactor_blocks_cleaned_total counter + cortex_compactor_blocks_cleaned_total 1 + + # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. + # TYPE cortex_compactor_blocks_marked_for_deletion_total counter + cortex_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 + cortex_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 + cortex_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 + + # TYPE cortex_compactor_block_cleanup_started_total counter + # HELP cortex_compactor_block_cleanup_started_total Total number of blocks cleanup runs started. + cortex_compactor_block_cleanup_started_total 1 + + # TYPE cortex_compactor_block_cleanup_completed_total counter + # HELP cortex_compactor_block_cleanup_completed_total Total number of blocks cleanup runs successfully completed. + cortex_compactor_block_cleanup_completed_total 1 + + # TYPE cortex_compactor_block_cleanup_failed_total counter + # HELP cortex_compactor_block_cleanup_failed_total Total number of blocks cleanup runs failed. + cortex_compactor_block_cleanup_failed_total 0 + `), testedMetrics...)) +} + +func TestMultitenantCompactor_ShouldCompactAllUsersOnShardingEnabledButOnlyOneInstanceRunning(t *testing.T) { + t.Parallel() + + // Mock the bucket to contain two users, each one with one block. + bucketClient := &bucket.ClientMock{} + bucketClient.MockIter("", []string{"user-1", "user-2"}, nil) + bucketClient.MockExists(path.Join("user-1", mimir_tsdb.TenantDeletionMarkPath), false, nil) + bucketClient.MockExists(path.Join("user-2", mimir_tsdb.TenantDeletionMarkPath), false, nil) + bucketClient.MockIter("user-1/", []string{"user-1/01DTVP434PA9VFXSW2JKB3392D", "user-1/01FSTQ95C8FS0ZAGTQS2EF1NEG"}, nil) + bucketClient.MockIter("user-2/", []string{"user-2/01DTW0ZCPDDNV4BV83Q2SV4QAZ", "user-2/01FSV54G6QFQH1G9QE93G3B9TB"}, nil) + bucketClient.MockIter("user-1/markers/", nil, nil) + bucketClient.MockIter("user-2/markers/", nil, nil) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", mockBlockMetaJSON("01DTVP434PA9VFXSW2JKB3392D"), nil) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/deletion-mark.json", "", nil) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/no-compact-mark.json", "", nil) + bucketClient.MockGet("user-1/01FSTQ95C8FS0ZAGTQS2EF1NEG/meta.json", mockBlockMetaJSON("01FSTQ95C8FS0ZAGTQS2EF1NEG"), nil) + bucketClient.MockGet("user-1/01FSTQ95C8FS0ZAGTQS2EF1NEG/deletion-mark.json", "", nil) + bucketClient.MockGet("user-1/01FSTQ95C8FS0ZAGTQS2EF1NEG/no-compact-mark.json", "", nil) + bucketClient.MockGet("user-2/01DTW0ZCPDDNV4BV83Q2SV4QAZ/meta.json", mockBlockMetaJSON("01DTW0ZCPDDNV4BV83Q2SV4QAZ"), nil) + bucketClient.MockGet("user-2/01DTW0ZCPDDNV4BV83Q2SV4QAZ/deletion-mark.json", "", nil) + bucketClient.MockGet("user-2/01DTW0ZCPDDNV4BV83Q2SV4QAZ/no-compact-mark.json", "", nil) + bucketClient.MockGet("user-2/01FSV54G6QFQH1G9QE93G3B9TB/meta.json", mockBlockMetaJSON("01FSV54G6QFQH1G9QE93G3B9TB"), nil) + bucketClient.MockGet("user-2/01FSV54G6QFQH1G9QE93G3B9TB/deletion-mark.json", "", nil) + bucketClient.MockGet("user-2/01FSV54G6QFQH1G9QE93G3B9TB/no-compact-mark.json", "", nil) + bucketClient.MockGet("user-1/bucket-index.json.gz", "", nil) + bucketClient.MockGet("user-2/bucket-index.json.gz", "", nil) + bucketClient.MockUpload("user-1/bucket-index.json.gz", nil) + bucketClient.MockUpload("user-2/bucket-index.json.gz", nil) + + ringStore, closer := consul.NewInMemoryClient(ring.GetCodec(), log.NewNopLogger(), nil) + t.Cleanup(func() { assert.NoError(t, closer.Close()) }) + + cfg := prepareConfig(t) + cfg.ShardingRing.Common.InstanceID = "compactor-1" + cfg.ShardingRing.Common.InstanceAddr = "1.2.3.4" + cfg.ShardingRing.Common.KVStore.Mock = ringStore + c, _, tsdbPlanner, logs, registry := prepare(t, cfg, bucketClient) + + // Mock the planner as if there's no compaction to do, + // in order to simplify tests (all in all, we just want to + // test our logic and not TSDB compactor which we expect to + // be already tested). + tsdbPlanner.On("Plan", mock.Anything, mock.Anything).Return([]*block.Meta{}, nil) + + require.NoError(t, services.StartAndAwaitRunning(context.Background(), c)) + + // Compactor doesn't wait for blocks cleaner to finish, but our test checks for cleaner metrics. + require.NoError(t, c.blocksCleaner.AwaitRunning(context.Background())) + + // Wait until a run has completed. + test.Poll(t, 5*time.Second, 1.0, func() interface{} { + return prom_testutil.ToFloat64(c.compactionRunsCompleted) + }) + + require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) + + // Ensure a plan has been executed for the blocks of each user. + tsdbPlanner.AssertNumberOfCalls(t, "Plan", 2) + + assert.ElementsMatch(t, []string{ + `level=info component=compactor msg="waiting until compactor is ACTIVE in the ring"`, + `level=info component=compactor msg="compactor is ACTIVE in the ring"`, + `level=info component=compactor msg="discovering users from bucket"`, + `level=info component=compactor msg="discovered users from bucket" users=2`, + `level=info component=compactor msg="starting compaction of user blocks" user=user-1`, + `level=info component=compactor user=user-1 msg="start sync of metas"`, + `level=info component=compactor user=user-1 msg="start of GC"`, + `level=debug component=compactor user=user-1 msg="grouper found a compactable blocks group" groupKey=0@17241709254077376921-merge--1574776800000-1574784000000 job="stage: merge, range start: 1574776800000, range end: 1574784000000, shard: , blocks: 01DTVP434PA9VFXSW2JKB3392D (min time: 2019-11-26 14:00:00 +0000 UTC, max time: 2019-11-26 16:00:00 +0000 UTC),01FSTQ95C8FS0ZAGTQS2EF1NEG (min time: 2019-11-26 14:00:00 +0000 UTC, max time: 2019-11-26 16:00:00 +0000 UTC)"`, + `level=info component=compactor user=user-1 msg="start of compactions"`, + `level=info component=compactor user=user-1 groupKey=0@17241709254077376921-merge--1574776800000-1574784000000 msg="compaction job succeeded"`, + `level=info component=compactor user=user-1 msg="compaction iterations done"`, + `level=info component=compactor msg="successfully compacted user blocks" user=user-1`, + `level=info component=compactor msg="starting compaction of user blocks" user=user-2`, + `level=info component=compactor user=user-2 msg="start sync of metas"`, + `level=info component=compactor user=user-2 msg="start of GC"`, + `level=debug component=compactor user=user-2 msg="grouper found a compactable blocks group" groupKey=0@17241709254077376921-merge--1574776800000-1574784000000 job="stage: merge, range start: 1574776800000, range end: 1574784000000, shard: , blocks: 01DTW0ZCPDDNV4BV83Q2SV4QAZ (min time: 2019-11-26 14:00:00 +0000 UTC, max time: 2019-11-26 16:00:00 +0000 UTC),01FSV54G6QFQH1G9QE93G3B9TB (min time: 2019-11-26 14:00:00 +0000 UTC, max time: 2019-11-26 16:00:00 +0000 UTC)"`, + `level=info component=compactor user=user-2 msg="start of compactions"`, + `level=info component=compactor user=user-2 groupKey=0@17241709254077376921-merge--1574776800000-1574784000000 msg="compaction job succeeded"`, + `level=info component=compactor user=user-2 msg="compaction iterations done"`, + `level=info component=compactor msg="successfully compacted user blocks" user=user-2`, + }, removeIgnoredLogs(strings.Split(strings.TrimSpace(logs.String()), "\n"))) + + assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` + # TYPE cortex_compactor_runs_started_total counter + # HELP cortex_compactor_runs_started_total Total number of compaction runs started. + cortex_compactor_runs_started_total 1 + + # TYPE cortex_compactor_runs_completed_total counter + # HELP cortex_compactor_runs_completed_total Total number of compaction runs successfully completed. + cortex_compactor_runs_completed_total 1 + + # TYPE cortex_compactor_runs_failed_total counter + # HELP cortex_compactor_runs_failed_total Total number of compaction runs failed. + cortex_compactor_runs_failed_total{reason="error"} 0 + cortex_compactor_runs_failed_total{reason="shutdown"} 0 + + # HELP cortex_compactor_group_compaction_runs_completed_total Total number of group completed compaction runs. This also includes compactor group runs that resulted with no compaction. + # TYPE cortex_compactor_group_compaction_runs_completed_total counter + cortex_compactor_group_compaction_runs_completed_total 2 + + # HELP cortex_compactor_group_compaction_runs_started_total Total number of group compaction attempts. + # TYPE cortex_compactor_group_compaction_runs_started_total counter + cortex_compactor_group_compaction_runs_started_total 2 + + # HELP cortex_compactor_group_compactions_failures_total Total number of failed group compactions. + # TYPE cortex_compactor_group_compactions_failures_total counter + cortex_compactor_group_compactions_failures_total 0 + + # HELP cortex_compactor_group_compactions_total Total number of group compaction attempts that resulted in new block(s). + # TYPE cortex_compactor_group_compactions_total counter + cortex_compactor_group_compactions_total 0 + + # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. + # TYPE cortex_compactor_blocks_marked_for_deletion_total counter + cortex_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 + cortex_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 + cortex_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 + `), + "cortex_compactor_runs_started_total", + "cortex_compactor_runs_completed_total", + "cortex_compactor_runs_failed_total", + "cortex_compactor_group_compaction_runs_completed_total", + "cortex_compactor_group_compaction_runs_started_total", + "cortex_compactor_group_compactions_failures_total", + "cortex_compactor_group_compactions_total", + "cortex_compactor_blocks_marked_for_deletion_total", + )) +} + +func TestMultitenantCompactor_ShouldCompactOnlyUsersOwnedByTheInstanceOnShardingEnabledAndMultipleInstancesRunning(t *testing.T) { + t.Parallel() + + numUsers := 100 + + // Setup user IDs + userIDs := make([]string, 0, numUsers) + for i := 1; i <= numUsers; i++ { + userIDs = append(userIDs, fmt.Sprintf("user-%d", i)) + } + + // Mock the bucket to contain all users, each one with one block. + bucketClient := &bucket.ClientMock{} + bucketClient.MockIter("", userIDs, nil) + for _, userID := range userIDs { + bucketClient.MockIter(userID+"/", []string{userID + "/01DTVP434PA9VFXSW2JKB3392D"}, nil) + bucketClient.MockIter(userID+"/markers/", nil, nil) + bucketClient.MockExists(path.Join(userID, mimir_tsdb.TenantDeletionMarkPath), false, nil) + bucketClient.MockGet(userID+"/01DTVP434PA9VFXSW2JKB3392D/meta.json", mockBlockMetaJSON("01DTVP434PA9VFXSW2JKB3392D"), nil) + bucketClient.MockGet(userID+"/01DTVP434PA9VFXSW2JKB3392D/deletion-mark.json", "", nil) + bucketClient.MockGet(userID+"/01DTVP434PA9VFXSW2JKB3392D/no-compact-mark.json", "", nil) + bucketClient.MockGet(userID+"/bucket-index.json.gz", "", nil) + bucketClient.MockUpload(userID+"/bucket-index.json.gz", nil) + } + + // Create a shared KV Store + kvstore, closer := consul.NewInMemoryClient(ring.GetCodec(), log.NewNopLogger(), nil) + t.Cleanup(func() { assert.NoError(t, closer.Close()) }) + + // Create two compactors + var compactors []*MultitenantCompactor + var logs []*concurrency.SyncBuffer + + for i := 1; i <= 2; i++ { + cfg := prepareConfig(t) + cfg.ShardingRing.Common.InstanceID = fmt.Sprintf("compactor-%d", i) + cfg.ShardingRing.Common.InstanceAddr = fmt.Sprintf("127.0.0.%d", i) + cfg.ShardingRing.WaitStabilityMinDuration = 3 * time.Second + cfg.ShardingRing.WaitStabilityMaxDuration = 10 * time.Second + cfg.ShardingRing.Common.KVStore.Mock = kvstore + + var limits validation.Limits + flagext.DefaultValues(&limits) + limits.CompactorTenantShardSize = 1 + overrides, err := validation.NewOverrides(limits, nil) + require.NoError(t, err) + + c, _, tsdbPlanner, l, _ := prepareWithConfigProvider(t, cfg, bucketClient, overrides) + defer services.StopAndAwaitTerminated(context.Background(), c) //nolint:errcheck + + compactors = append(compactors, c) + logs = append(logs, l) + + // Mock the planner as if there's no compaction to do, + // in order to simplify tests (all in all, we just want to + // test our logic and not TSDB compactor which we expect to + // be already tested). + tsdbPlanner.On("Plan", mock.Anything, mock.Anything).Return([]*block.Meta{}, nil) + } + + // Start all compactors + for _, c := range compactors { + require.NoError(t, services.StartAndAwaitRunning(context.Background(), c)) + } + + // Wait until a run has been completed on each compactor + test.Poll(t, 30*time.Second, true, func() interface{} { + for _, c := range compactors { + if prom_testutil.ToFloat64(c.compactionRunsCompleted) < 1.0 { + return false + } + } + return true + }) + + // Ensure that each user has been compacted by the correct instance + for _, userID := range userIDs { + _, l, err := findCompactorByUserID(compactors, logs, userID) + require.NoError(t, err) + assert.Contains(t, l.String(), fmt.Sprintf(`level=info component=compactor msg="successfully compacted user blocks" user=%s`, userID)) + } +} + +func TestMultitenantCompactor_ShouldFailWithInvalidTSDBCompactOutput(t *testing.T) { + const user = "user-1" + + // Two blocks with overlapping time range + sourceBlock1Spec := []*block.SeriesSpec{ + { + Labels: labels.FromStrings("case", "source_spec_1"), + Chunks: []chunks.Meta{ + must(tsdbutil.ChunkFromSamples([]tsdbutil.Sample{ + newSample(1000, 1000, nil, nil), + newSample(2000, 2000, nil, nil), + })), + }, + }, + } + + sourceBlock2Spec := []*block.SeriesSpec{ + { + Labels: labels.FromStrings("case", "source_spec_2"), + Chunks: []chunks.Meta{ + must(tsdbutil.ChunkFromSamples([]tsdbutil.Sample{ + newSample(1500, 1500, nil, nil), + newSample(2500, 2500, nil, nil), + })), + }, + }, + } + + // Block with sufficient time range so compaction job gets triggered + sourceBlock3Spec := []*block.SeriesSpec{ + { + Labels: labels.FromStrings("case", "source_spec_3"), + Chunks: []chunks.Meta{ + must(tsdbutil.ChunkFromSamples([]tsdbutil.Sample{ + newSample(0, 0, nil, nil), + newSample(2*time.Hour.Milliseconds()-1, 0, nil, nil), + })), + }, + }, + } + + // Compacted block not containing minTime/maxTime from source blocks + compactedBlockSpec := []*block.SeriesSpec{ + { + Labels: labels.FromStrings("case", "compacted_spec"), + Chunks: []chunks.Meta{ + must(tsdbutil.ChunkFromSamples([]tsdbutil.Sample{ + newSample(1250, 1250, nil, nil), + newSample(2250, 2250, nil, nil), + })), + }, + }, + } + + storageDir := t.TempDir() + + meta1, err := block.GenerateBlockFromSpec(user, filepath.Join(storageDir, user), sourceBlock1Spec) + require.NoError(t, err) + meta2, err := block.GenerateBlockFromSpec(user, filepath.Join(storageDir, user), sourceBlock2Spec) + require.NoError(t, err) + _, err = block.GenerateBlockFromSpec(user, filepath.Join(storageDir, user), sourceBlock3Spec) + require.NoError(t, err) + + bkt, err := filesystem.NewBucketClient(filesystem.Config{Directory: storageDir}) + require.NoError(t, err) + + cfg := prepareConfig(t) + cfg.CompactionRetries = 1 // No need to retry as we're testing for failure + c, tsdbCompactor, tsdbPlanner, logs, _ := prepare(t, cfg, bkt) + + tsdbPlanner.On("Plan", mock.Anything, mock.Anything).Return([]*block.Meta{meta1, meta2}, nil).Once() + tsdbPlanner.On("Plan", mock.Anything, mock.Anything).Return([]*block.Meta{}, nil).Once() + mockCall := tsdbCompactor.On("Compact", mock.Anything, mock.Anything, mock.Anything) + mockCall.RunFn = func(args mock.Arguments) { + dir := args.Get(0).(string) + + compactedMeta, err := block.GenerateBlockFromSpec(user, dir, compactedBlockSpec) + require.NoError(t, err) + f, err := os.OpenFile(filepath.Join(dir, compactedMeta.ULID.String(), "tombstones"), os.O_RDONLY|os.O_CREATE, 0o666) + require.NoError(t, err) + defer f.Close() + + mockCall.ReturnArguments = mock.Arguments{compactedMeta.ULID, nil} + } + + // Start the compactor + require.NoError(t, services.StartAndAwaitRunning(context.Background(), c)) + + // Compaction block verification should fail due to invalid output block + test.Poll(t, 5*time.Second, 1.0, func() interface{} { + return prom_testutil.ToFloat64(c.bucketCompactorMetrics.compactionBlocksVerificationFailed) + }) + + // Stop the compactor. + require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) + + // Check logs for compacted block verification failure + assert.Contains(t, logs.String(), "compacted block(s) do not contain minTime 1000 and maxTime 2501 from the source blocks") +} + +func TestMultitenantCompactor_ShouldSkipCompactionForJobsNoMoreOwnedAfterPlanning(t *testing.T) { + t.Parallel() + + // Mock the bucket to contain one user with two non-overlapping blocks (we expect two compaction jobs to be scheduled + // for the splitting stage). + bucketClient := &bucket.ClientMock{} + bucketClient.MockIter("", []string{"user-1"}, nil) + bucketClient.MockExists(path.Join("user-1", mimir_tsdb.TenantDeletionMarkPath), false, nil) + bucketClient.MockIter("user-1/", []string{"user-1/01DTVP434PA9VFXSW2JK000001", "user-1/01DTVP434PA9VFXSW2JK000002"}, nil) + bucketClient.MockIter("user-1/markers/", nil, nil) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JK000001/meta.json", mockBlockMetaJSONWithTimeRange("01DTVP434PA9VFXSW2JK000001", 1574776800000, 1574784000000), nil) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JK000001/deletion-mark.json", "", nil) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JK000001/no-compact-mark.json", "", nil) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JK000002/meta.json", mockBlockMetaJSONWithTimeRange("01DTVP434PA9VFXSW2JK000002", 1574863200000, 1574870400000), nil) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JK000002/deletion-mark.json", "", nil) + bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JK000002/no-compact-mark.json", "", nil) + bucketClient.MockGet("user-1/bucket-index.json.gz", "", nil) + bucketClient.MockUpload("user-1/bucket-index.json.gz", nil) + + ringStore, closer := consul.NewInMemoryClient(ring.GetCodec(), log.NewNopLogger(), nil) + t.Cleanup(func() { assert.NoError(t, closer.Close()) }) + + cfg := prepareConfig(t) + cfg.CompactionConcurrency = 1 + cfg.ShardingRing.Common.InstanceID = "compactor-1" + cfg.ShardingRing.Common.InstanceAddr = "1.2.3.4" + cfg.ShardingRing.Common.KVStore.Mock = ringStore + + limits := newMockConfigProvider() + limits.splitAndMergeShards = map[string]int{"user-1": 4} + limits.splitGroups = map[string]int{"user-1": 4} + + c, _, tsdbPlanner, logs, registry := prepareWithConfigProvider(t, cfg, bucketClient, limits) + + // Mock the planner as if there's no compaction to do, in order to simplify tests. + tsdbPlanner.On("Plan", mock.Anything, mock.Anything).Return([]*block.Meta{}, nil).Run(func(args mock.Arguments) { + // As soon as the first Plan() is called by the compactor, we do switch + // the instance to LEAVING state. This way, after this call, we expect the compactor + // to skip next compaction job because not owned anymore by this instance. + require.NoError(t, c.ringLifecycler.ChangeState(context.Background(), ring.LEAVING)) + + // Wait until the compactor ring client has updated. + test.Poll(t, time.Second, 0, func() interface{} { + set, _ := c.ring.GetAllHealthy(RingOp) + return len(set.Instances) + }) + }) + + require.NoError(t, services.StartAndAwaitRunning(context.Background(), c)) + + // Compactor doesn't wait for blocks cleaner to finish, but our test checks for cleaner metrics. + require.NoError(t, c.blocksCleaner.AwaitRunning(context.Background())) + + // Wait until a run has completed. + test.Poll(t, 5*time.Second, 1.0, func() interface{} { + return prom_testutil.ToFloat64(c.compactionRunsCompleted) + }) + + require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) + + // We expect only 1 compaction job has been expected, while the 2nd has been skipped. + tsdbPlanner.AssertNumberOfCalls(t, "Plan", 1) + + assert.ElementsMatch(t, []string{ + `level=info component=compactor msg="waiting until compactor is ACTIVE in the ring"`, + `level=info component=compactor msg="compactor is ACTIVE in the ring"`, + `level=info component=compactor msg="discovering users from bucket"`, + `level=info component=compactor msg="discovered users from bucket" users=1`, + `level=info component=compactor msg="starting compaction of user blocks" user=user-1`, + `level=info component=compactor user=user-1 msg="start sync of metas"`, + `level=info component=compactor user=user-1 msg="start of GC"`, + `level=info component=compactor user=user-1 msg="start of compactions"`, + `level=debug component=compactor user=user-1 msg="grouper found a compactable blocks group" groupKey=0@17241709254077376921-split-4_of_4-1574776800000-1574784000000 job="stage: split, range start: 1574776800000, range end: 1574784000000, shard: 4_of_4, blocks: 01DTVP434PA9VFXSW2JK000001 (min time: 2019-11-26 14:00:00 +0000 UTC, max time: 2019-11-26 16:00:00 +0000 UTC)"`, + `level=debug component=compactor user=user-1 msg="grouper found a compactable blocks group" groupKey=0@17241709254077376921-split-1_of_4-1574863200000-1574870400000 job="stage: split, range start: 1574863200000, range end: 1574870400000, shard: 1_of_4, blocks: 01DTVP434PA9VFXSW2JK000002 (min time: 2019-11-27 14:00:00 +0000 UTC, max time: 2019-11-27 16:00:00 +0000 UTC)"`, + // The ownership check is failing because, to keep this test simple, we've just switched + // the instance state to LEAVING and there are no other instances in the ring. + `level=info component=compactor user=user-1 groupKey=0@17241709254077376921-split-4_of_4-1574776800000-1574784000000 msg="compaction job succeeded"`, + `level=info component=compactor user=user-1 msg="skipped compaction because unable to check whether the job is owned by the compactor instance" groupKey=0@17241709254077376921-split-1_of_4-1574863200000-1574870400000 err="at least 1 live replicas required, could only find 0 - unhealthy instances: 1.2.3.4:0"`, + `level=info component=compactor user=user-1 msg="compaction iterations done"`, + `level=info component=compactor msg="successfully compacted user blocks" user=user-1`, + }, removeIgnoredLogs(strings.Split(strings.TrimSpace(logs.String()), "\n"))) + + assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` + # TYPE cortex_compactor_runs_started_total counter + # HELP cortex_compactor_runs_started_total Total number of compaction runs started. + cortex_compactor_runs_started_total 1 + + # TYPE cortex_compactor_runs_completed_total counter + # HELP cortex_compactor_runs_completed_total Total number of compaction runs successfully completed. + cortex_compactor_runs_completed_total 1 + + # TYPE cortex_compactor_runs_failed_total counter + # HELP cortex_compactor_runs_failed_total Total number of compaction runs failed. + cortex_compactor_runs_failed_total{reason="error"} 0 + cortex_compactor_runs_failed_total{reason="shutdown"} 0 + + # HELP cortex_compactor_group_compaction_runs_completed_total Total number of group completed compaction runs. This also includes compactor group runs that resulted with no compaction. + # TYPE cortex_compactor_group_compaction_runs_completed_total counter + cortex_compactor_group_compaction_runs_completed_total 1 + + # HELP cortex_compactor_group_compaction_runs_started_total Total number of group compaction attempts. + # TYPE cortex_compactor_group_compaction_runs_started_total counter + cortex_compactor_group_compaction_runs_started_total 1 + + # HELP cortex_compactor_group_compactions_failures_total Total number of failed group compactions. + # TYPE cortex_compactor_group_compactions_failures_total counter + cortex_compactor_group_compactions_failures_total 0 + + # HELP cortex_compactor_group_compactions_total Total number of group compaction attempts that resulted in new block(s). + # TYPE cortex_compactor_group_compactions_total counter + cortex_compactor_group_compactions_total 0 + + # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. + # TYPE cortex_compactor_blocks_marked_for_deletion_total counter + cortex_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 + cortex_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 + cortex_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 + `), + "cortex_compactor_runs_started_total", + "cortex_compactor_runs_completed_total", + "cortex_compactor_runs_failed_total", + "cortex_compactor_group_compaction_runs_completed_total", + "cortex_compactor_group_compaction_runs_started_total", + "cortex_compactor_group_compactions_failures_total", + "cortex_compactor_group_compactions_total", + "cortex_compactor_blocks_marked_for_deletion_total", + )) +} + +func TestMultitenantCompactor_ShouldSkipCompactionForJobsWithFirstLevelCompactionBlocksAndWaitPeriodNotElapsed(t *testing.T) { + t.Parallel() + + storageDir := t.TempDir() + bucketClient, err := filesystem.NewBucketClient(filesystem.Config{Directory: storageDir}) + require.NoError(t, err) + + // Mock two tenants, each with 2 overlapping blocks. + spec := []*block.SeriesSpec{{ + Labels: labels.FromStrings(labels.MetricName, "series_1"), + Chunks: []chunks.Meta{must(tsdbutil.ChunkFromSamples([]tsdbutil.Sample{ + newSample(1574776800000, 0, nil, nil), + newSample(1574783999999, 0, nil, nil), + }))}, + }} + + user1Meta1, err := block.GenerateBlockFromSpec("user-1", filepath.Join(storageDir, "user-1"), spec) + require.NoError(t, err) + user1Meta2, err := block.GenerateBlockFromSpec("user-1", filepath.Join(storageDir, "user-1"), spec) + require.NoError(t, err) + user2Meta1, err := block.GenerateBlockFromSpec("user-2", filepath.Join(storageDir, "user-2"), spec) + require.NoError(t, err) + user2Meta2, err := block.GenerateBlockFromSpec("user-2", filepath.Join(storageDir, "user-2"), spec) + require.NoError(t, err) + + // Mock the last modified timestamp returned for each of the block's meta.json. + const waitPeriod = 10 * time.Minute + bucketClient = &bucketWithMockedAttributes{ + Bucket: bucketClient, + customAttributes: map[string]objstore.ObjectAttributes{ + path.Join("user-1", user1Meta1.ULID.String(), block.MetaFilename): {LastModified: time.Now().Add(-20 * time.Minute)}, + path.Join("user-1", user1Meta2.ULID.String(), block.MetaFilename): {LastModified: time.Now().Add(-20 * time.Minute)}, + path.Join("user-2", user2Meta1.ULID.String(), block.MetaFilename): {LastModified: time.Now().Add(-20 * time.Minute)}, + path.Join("user-2", user2Meta2.ULID.String(), block.MetaFilename): {LastModified: time.Now().Add(-5 * time.Minute)}, + }, + } + + cfg := prepareConfig(t) + cfg.CompactionWaitPeriod = waitPeriod + c, _, tsdbPlanner, logs, registry := prepare(t, cfg, bucketClient) + + // Mock the planner as if there's no compaction to do, in order to simplify tests. + tsdbPlanner.On("Plan", mock.Anything, mock.Anything).Return([]*block.Meta{}, nil) + + require.NoError(t, services.StartAndAwaitRunning(context.Background(), c)) + + // Compactor doesn't wait for blocks cleaner to finish, but our test checks for cleaner metrics. + require.NoError(t, c.blocksCleaner.AwaitRunning(context.Background())) + + // Wait until a run has completed. + test.Poll(t, 5*time.Second, 1.0, func() interface{} { + return prom_testutil.ToFloat64(c.compactionRunsCompleted) + }) + + require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) + + // We expect only 1 compaction job has been expected, while the 2nd has been skipped. + tsdbPlanner.AssertNumberOfCalls(t, "Plan", 1) + + // Ensure the skipped compaction job is the expected one. + assert.Contains(t, strings.Split(strings.TrimSpace(logs.String()), "\n"), + fmt.Sprintf(`level=info component=compactor user=user-2 msg="skipping compaction job because blocks in this job were uploaded too recently (within wait period)" groupKey=0@17241709254077376921-merge--1574776800000-1574784000000 waitPeriodNotElapsedFor="%s (min time: 1574776800000, max time: 1574784000000)"`, user2Meta2.ULID.String())) + + assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` + # TYPE cortex_compactor_runs_started_total counter + # HELP cortex_compactor_runs_started_total Total number of compaction runs started. + cortex_compactor_runs_started_total 1 + + # TYPE cortex_compactor_runs_completed_total counter + # HELP cortex_compactor_runs_completed_total Total number of compaction runs successfully completed. + cortex_compactor_runs_completed_total 1 + + # TYPE cortex_compactor_runs_failed_total counter + # HELP cortex_compactor_runs_failed_total Total number of compaction runs failed. + cortex_compactor_runs_failed_total{reason="error"} 0 + cortex_compactor_runs_failed_total{reason="shutdown"} 0 + + # HELP cortex_compactor_group_compaction_runs_completed_total Total number of group completed compaction runs. This also includes compactor group runs that resulted with no compaction. + # TYPE cortex_compactor_group_compaction_runs_completed_total counter + cortex_compactor_group_compaction_runs_completed_total 1 + + # HELP cortex_compactor_group_compaction_runs_started_total Total number of group compaction attempts. + # TYPE cortex_compactor_group_compaction_runs_started_total counter + cortex_compactor_group_compaction_runs_started_total 1 + + # HELP cortex_compactor_group_compactions_failures_total Total number of failed group compactions. + # TYPE cortex_compactor_group_compactions_failures_total counter + cortex_compactor_group_compactions_failures_total 0 + + # HELP cortex_compactor_group_compactions_total Total number of group compaction attempts that resulted in new block(s). + # TYPE cortex_compactor_group_compactions_total counter + cortex_compactor_group_compactions_total 0 + + # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. + # TYPE cortex_compactor_blocks_marked_for_deletion_total counter + cortex_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 + cortex_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 + cortex_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 + `), + "cortex_compactor_runs_started_total", + "cortex_compactor_runs_completed_total", + "cortex_compactor_runs_failed_total", + "cortex_compactor_group_compaction_runs_completed_total", + "cortex_compactor_group_compaction_runs_started_total", + "cortex_compactor_group_compactions_failures_total", + "cortex_compactor_group_compactions_total", + "cortex_compactor_blocks_marked_for_deletion_total", + )) +} + +func createCustomTSDBBlock(t *testing.T, bkt objstore.Bucket, userID string, externalLabels map[string]string, appendFunc func(*tsdb.DB)) ulid.ULID { + // Create a temporary dir for TSDB. + tempDir := t.TempDir() + + // Create a temporary dir for the snapshot. + snapshotDir := t.TempDir() + + // Create a new TSDB. + db, err := tsdb.Open(tempDir, nil, nil, &tsdb.Options{ + MinBlockDuration: int64(2 * 60 * 60 * 1000), // 2h period + MaxBlockDuration: int64(2 * 60 * 60 * 1000), // 2h period + RetentionDuration: int64(15 * 86400 * 1000), // 15 days + }, nil) + require.NoError(t, err) + + db.DisableCompactions() + + appendFunc(db) + + require.NoError(t, db.Compact()) + require.NoError(t, db.Snapshot(snapshotDir, true)) + + // Look for the created block (we expect one). + entries, err := os.ReadDir(snapshotDir) + require.NoError(t, err) + require.Len(t, entries, 1) + require.True(t, entries[0].IsDir()) + + blockID, err := ulid.Parse(entries[0].Name()) + require.NoError(t, err) + + // Inject Thanos external labels to the block. + meta := block.ThanosMeta{ + Labels: externalLabels, + Source: "test", + } + _, err = block.InjectThanosMeta(log.NewNopLogger(), filepath.Join(snapshotDir, blockID.String()), meta, nil) + require.NoError(t, err) + + // Copy the block files to the bucket. + srcRoot := filepath.Join(snapshotDir, blockID.String()) + require.NoError(t, filepath.Walk(srcRoot, func(file string, info os.FileInfo, err error) error { + if err != nil { + return err + } + if info.IsDir() { + return nil + } + + // Read the file content in memory. + content, err := os.ReadFile(file) + if err != nil { + return err + } + + // Upload it to the bucket. + relPath, err := filepath.Rel(srcRoot, file) + if err != nil { + return err + } + + return bkt.Upload(context.Background(), path.Join(userID, blockID.String(), relPath), bytes.NewReader(content)) + })) + + return blockID +} + +func createTSDBBlock(t *testing.T, bkt objstore.Bucket, userID string, minT, maxT int64, numSeries int, externalLabels map[string]string) ulid.ULID { + return createCustomTSDBBlock(t, bkt, userID, externalLabels, func(db *tsdb.DB) { + appendSample := func(seriesID int, ts int64, value float64) { + lbls := labels.FromStrings("series_id", strconv.Itoa(seriesID)) + + app := db.Appender(context.Background()) + _, err := app.Append(0, lbls, ts, value) + require.NoError(t, err) + + err = app.Commit() + require.NoError(t, err) + } + + seriesID := 0 + + // Append a sample for each series, spreading it between minT and maxT-1 (both included). + // Since we append one more series below, here we create N-1 series. + if numSeries > 1 { + for ts := minT; ts < maxT; ts += (maxT - minT) / int64(numSeries-1) { + appendSample(seriesID, ts, float64(seriesID)) + seriesID++ + } + } + + // Guarantee a series with a sample at time maxT-1 + appendSample(seriesID, maxT-1, float64(seriesID)) + }) +} + +func createDeletionMark(t *testing.T, bkt objstore.Bucket, userID string, blockID ulid.ULID, deletionTime time.Time) { + content := mockDeletionMarkJSON(blockID.String(), deletionTime) + blockPath := path.Join(userID, blockID.String()) + markPath := path.Join(blockPath, block.DeletionMarkFilename) + + require.NoError(t, bkt.Upload(context.Background(), markPath, strings.NewReader(content))) +} + +func findCompactorByUserID(compactors []*MultitenantCompactor, logs []*concurrency.SyncBuffer, userID string) (*MultitenantCompactor, *concurrency.SyncBuffer, error) { + var compactor *MultitenantCompactor + var log *concurrency.SyncBuffer + + for i, c := range compactors { + owned, err := c.shardingStrategy.compactorOwnUser(userID) + if err != nil { + return nil, nil, err + } + + // Ensure the user is not owned by multiple compactors + if owned && compactor != nil { + return nil, nil, fmt.Errorf("user %s owned by multiple compactors", userID) + } + if owned { + compactor = c + log = logs[i] + } + } + + // Return an error if we've not been able to find it + if compactor == nil { + return nil, nil, fmt.Errorf("user %s not owned by any compactor", userID) + } + + return compactor, log, nil +} + +func removeIgnoredLogs(input []string) []string { + ignoredLogStringsMap := map[string]struct{}{ + // Since we moved to the component logger from the global logger for the ring in dskit these lines are now expected but are just ring setup information. + `level=info component=compactor msg="ring doesn't exist in KV store yet"`: {}, + `level=info component=compactor msg="not loading tokens from file, tokens file path is empty"`: {}, + `level=info component=compactor msg="tokens verification succeeded" ring=compactor`: {}, + `level=info component=compactor msg="waiting stable tokens" ring=compactor`: {}, + `level=info component=compactor msg="instance not found in ring, adding with no tokens" ring=compactor`: {}, + `level=debug component=compactor msg="JoinAfter expired" ring=compactor`: {}, + `level=info component=compactor msg="auto-joining cluster after timeout" ring=compactor`: {}, + `level=info component=compactor msg="lifecycler loop() exited gracefully" ring=compactor`: {}, + `level=info component=compactor msg="changing instance state from" old_state=ACTIVE new_state=LEAVING ring=compactor`: {}, + `level=error component=compactor msg="failed to set state to LEAVING" ring=compactor err="Changing instance state from LEAVING -> LEAVING is disallowed"`: {}, + `level=error component=compactor msg="failed to set state to LEAVING" ring=compactor err="Changing instance state from JOINING -> LEAVING is disallowed"`: {}, + `level=info component=compactor msg="unregistering instance from ring" ring=compactor`: {}, + `level=info component=compactor msg="instance removed from the ring" ring=compactor`: {}, + `level=info component=compactor msg="observing tokens before going ACTIVE" ring=compactor`: {}, + `level=info component=compactor msg="lifecycler entering final sleep before shutdown" final_sleep=0s`: {}, + `level=info component=compactor msg="ring lifecycler is shutting down" ring=compactor`: {}, + } + + out := make([]string, 0, len(input)) + + for i := 0; i < len(input); i++ { + log := input[i] + if strings.Contains(log, "block.MetaFetcher") || strings.Contains(log, "instance not found in the ring") { + continue + } + + if _, exists := ignoredLogStringsMap[log]; exists { + continue + } + + out = append(out, log) + } + + return out +} + +func prepareConfig(t *testing.T) Config { + compactorCfg := Config{} + flagext.DefaultValues(&compactorCfg) + + compactorCfg.retryMinBackoff = 0 + compactorCfg.retryMaxBackoff = 0 + + // Use settings that ensure things will be done concurrently, verifying ordering assumptions. + // Helps to expose bugs such as https://github.com/prometheus/prometheus/pull/10108 + compactorCfg.MaxOpeningBlocksConcurrency = 3 + compactorCfg.MaxClosingBlocksConcurrency = 3 + + // Do not wait for ring stability by default, in order to speed up tests. + compactorCfg.ShardingRing.WaitStabilityMinDuration = 0 + compactorCfg.ShardingRing.WaitStabilityMaxDuration = 0 + + // Set lower timeout for waiting on compactor to become ACTIVE in the ring for unit tests + compactorCfg.ShardingRing.WaitActiveInstanceTimeout = 5 * time.Second + + // Inject default KV store. Must be overridden if "real" sharding is required. + inmem, closer := consul.NewInMemoryClient(ring.GetCodec(), log.NewNopLogger(), nil) + t.Cleanup(func() { _ = closer.Close() }) + compactorCfg.ShardingRing.Common.KVStore.Mock = inmem + compactorCfg.ShardingRing.Common.InstanceAddr = "localhost" + + // The new default is 25m, but tests rely on the previous value of 0s + compactorCfg.CompactionWaitPeriod = 0 + + return compactorCfg +} + +func prepare(t *testing.T, compactorCfg Config, bucketClient objstore.Bucket) (*MultitenantCompactor, *tsdbCompactorMock, *tsdbPlannerMock, *concurrency.SyncBuffer, prometheus.Gatherer) { + var limits validation.Limits + flagext.DefaultValues(&limits) + overrides, err := validation.NewOverrides(limits, nil) + require.NoError(t, err) + + return prepareWithConfigProvider(t, compactorCfg, bucketClient, overrides) +} + +func prepareWithConfigProvider(t *testing.T, compactorCfg Config, bucketClient objstore.Bucket, limits ConfigProvider) (*MultitenantCompactor, *tsdbCompactorMock, *tsdbPlannerMock, *concurrency.SyncBuffer, prometheus.Gatherer) { + storageCfg := mimir_tsdb.BlocksStorageConfig{} + flagext.DefaultValues(&storageCfg) + + // Create a temporary directory for compactor data. + dataDir := t.TempDir() + + compactorCfg.DataDir = dataDir + + tsdbCompactor := &tsdbCompactorMock{} + tsdbPlanner := &tsdbPlannerMock{} + logs := &concurrency.SyncBuffer{} + logger := &componentLogger{component: "compactor", log: log.NewLogfmtLogger(logs)} + registry := prometheus.NewRegistry() + + bucketClientFactory := func(ctx context.Context) (objstore.Bucket, error) { + return bucketClient, nil + } + + blocksCompactorFactory := func(ctx context.Context, cfg Config, logger log.Logger, reg prometheus.Registerer) (Compactor, Planner, error) { + return tsdbCompactor, tsdbPlanner, nil + } + + c, err := newMultitenantCompactor(compactorCfg, storageCfg, limits, logger, registry, bucketClientFactory, splitAndMergeGrouperFactory, blocksCompactorFactory) + require.NoError(t, err) + + return c, tsdbCompactor, tsdbPlanner, logs, registry +} + +type componentLogger struct { + component string + log log.Logger +} + +func (c *componentLogger) Log(keyvals ...interface{}) error { + // Remove duration fields. + for ix := 0; ix+1 < len(keyvals); { + k := keyvals[ix] + + ks, ok := k.(string) + if !ok { + ix += 2 + continue + } + + if ks == "duration" || ks == "duration_ms" { + keyvals = append(keyvals[:ix], keyvals[ix+2:]...) + } else { + ix += 2 + } + } + + for ix := 0; ix+1 < len(keyvals); ix += 2 { + k := keyvals[ix] + v := keyvals[ix+1] + + ks, ok := k.(string) + if !ok { + continue + } + vs, ok := v.(string) + if !ok { + continue + } + if ks == "component" && vs == c.component { + return c.log.Log(keyvals...) + } + } + return nil +} + +type tsdbCompactorMock struct { + mock.Mock +} + +func (m *tsdbCompactorMock) Plan(dir string) ([]string, error) { + args := m.Called(dir) + return args.Get(0).([]string), args.Error(1) +} + +func (m *tsdbCompactorMock) Write(dest string, b tsdb.BlockReader, mint, maxt int64, parent *tsdb.BlockMeta) (ulid.ULID, error) { + args := m.Called(dest, b, mint, maxt, parent) + return args.Get(0).(ulid.ULID), args.Error(1) +} + +func (m *tsdbCompactorMock) Compact(dest string, dirs []string, open []*tsdb.Block) (ulid.ULID, error) { + args := m.Called(dest, dirs, open) + return args.Get(0).(ulid.ULID), args.Error(1) +} + +func (m *tsdbCompactorMock) CompactWithSplitting(dest string, dirs []string, open []*tsdb.Block, shardCount uint64) (result []ulid.ULID, _ error) { + args := m.Called(dest, dirs, open, shardCount) + return args.Get(0).([]ulid.ULID), args.Error(1) +} + +type tsdbPlannerMock struct { + mock.Mock +} + +func (m *tsdbPlannerMock) Plan(ctx context.Context, metasByMinTime []*block.Meta) ([]*block.Meta, error) { + args := m.Called(ctx, metasByMinTime) + return args.Get(0).([]*block.Meta), args.Error(1) +} + +func mockBlockMetaJSON(id string) string { + return mockBlockMetaJSONWithTimeRange(id, 1574776800000, 1574784000000) +} + +func mockBlockMetaJSONWithTimeRange(id string, mint, maxt int64) string { + return mockBlockMetaJSONWithTimeRangeAndLabels(id, mint, maxt, nil) +} + +func mockBlockMetaJSONWithTimeRangeAndLabels(id string, mint, maxt int64, lbls map[string]string) string { + content, err := json.Marshal(blockMeta(id, mint, maxt, lbls)) + if err != nil { + panic("failed to marshal mocked block meta") + } + return string(content) +} + +func blockMeta(id string, mint, maxt int64, lbls map[string]string) *block.Meta { + return &block.Meta{ + BlockMeta: tsdb.BlockMeta{ + Version: 1, + ULID: ulid.MustParse(id), + MinTime: mint, + MaxTime: maxt, + Compaction: tsdb.BlockMetaCompaction{ + Level: 1, + Sources: []ulid.ULID{ulid.MustParse(id)}, + }, + }, + Thanos: block.ThanosMeta{ + Labels: lbls, + }, + } +} + +func mockDeletionMarkJSON(id string, deletionTime time.Time) string { + meta := block.DeletionMark{ + Version: block.DeletionMarkVersion1, + ID: ulid.MustParse(id), + DeletionTime: deletionTime.Unix(), + } + + content, err := json.Marshal(meta) + if err != nil { + panic("failed to marshal mocked block meta") + } + + return string(content) +} + +func TestMultitenantCompactor_DeleteLocalSyncFiles(t *testing.T) { + numUsers := 10 + + // Setup user IDs + userIDs := make([]string, 0, numUsers) + for i := 1; i <= numUsers; i++ { + userIDs = append(userIDs, fmt.Sprintf("user-%d", i)) + } + + inmem := objstore.NewInMemBucket() + for _, userID := range userIDs { + id, err := ulid.New(ulid.Now(), rand.Reader) + require.NoError(t, err) + require.NoError(t, inmem.Upload(context.Background(), userID+"/"+id.String()+"/meta.json", strings.NewReader(mockBlockMetaJSON(id.String())))) + } + + // Create a shared KV Store + kvstore, closer := consul.NewInMemoryClient(ring.GetCodec(), log.NewNopLogger(), nil) + t.Cleanup(func() { assert.NoError(t, closer.Close()) }) + + // Create two compactors + var compactors []*MultitenantCompactor + + for i := 1; i <= 2; i++ { + cfg := prepareConfig(t) + cfg.CompactionInterval = 10 * time.Minute // We will only call compaction manually. + + cfg.ShardingRing.Common.InstanceID = fmt.Sprintf("compactor-%d", i) + cfg.ShardingRing.Common.InstanceAddr = fmt.Sprintf("127.0.0.%d", i) + cfg.ShardingRing.WaitStabilityMinDuration = 3 * time.Second + cfg.ShardingRing.WaitStabilityMaxDuration = 10 * time.Second + cfg.ShardingRing.Common.KVStore.Mock = kvstore + + // Each compactor will get its own temp dir for storing local files. + var limits validation.Limits + flagext.DefaultValues(&limits) + limits.CompactorTenantShardSize = 1 // Each tenant will belong to single compactor only. + overrides, err := validation.NewOverrides(limits, nil) + require.NoError(t, err) + + c, _, tsdbPlanner, _, _ := prepareWithConfigProvider(t, cfg, inmem, overrides) + t.Cleanup(func() { + require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) + }) + + compactors = append(compactors, c) + + // Mock the planner as if there's no compaction to do, + // in order to simplify tests (all in all, we just want to + // test our logic and not TSDB compactor which we expect to + // be already tested). + tsdbPlanner.On("Plan", mock.Anything, mock.Anything).Return([]*block.Meta{}, nil) + } + + require.Equal(t, 2, len(compactors)) + c1 := compactors[0] + c2 := compactors[1] + + // Start first compactor + require.NoError(t, services.StartAndAwaitRunning(context.Background(), c1)) + + // Wait until a run has been completed on first compactor. This happens as soon as compactor starts. + test.Poll(t, 10*time.Second, 1.0, func() interface{} { + return prom_testutil.ToFloat64(c1.compactionRunsCompleted) + }) + + require.NoError(t, os.Mkdir(c1.metaSyncDirForUser("new-user"), 0o600)) + + // Verify that first compactor has synced all the users, plus there is one extra we have just created. + require.Equal(t, numUsers+1, len(c1.listTenantsWithMetaSyncDirectories())) + + // Now start second compactor, and wait until it runs compaction. + require.NoError(t, services.StartAndAwaitRunning(context.Background(), c2)) + test.Poll(t, 10*time.Second, 1.0, func() interface{} { + return prom_testutil.ToFloat64(c2.compactionRunsCompleted) + }) + + // Let's check how many users second compactor has. + c2Users := len(c2.listTenantsWithMetaSyncDirectories()) + require.NotZero(t, c2Users) + + // Force new compaction cycle on first compactor. It will run the cleanup of un-owned users at the end of compaction cycle. + c1.compactUsers(context.Background()) + c1Users := len(c1.listTenantsWithMetaSyncDirectories()) + + // Now compactor 1 should have cleaned old sync files. + require.NotEqual(t, numUsers, c1Users) + require.Equal(t, numUsers, c1Users+c2Users) +} + +func TestMultitenantCompactor_ShouldFailCompactionOnTimeout(t *testing.T) { + t.Parallel() + + // Mock the bucket + bucketClient := &bucket.ClientMock{} + bucketClient.MockIter("", []string{}, nil) + + ringStore, closer := consul.NewInMemoryClient(ring.GetCodec(), log.NewNopLogger(), nil) + t.Cleanup(func() { assert.NoError(t, closer.Close()) }) + + cfg := prepareConfig(t) + cfg.ShardingRing.Common.InstanceID = "compactor-1" + cfg.ShardingRing.Common.InstanceAddr = "1.2.3.4" + cfg.ShardingRing.Common.KVStore.Mock = ringStore + + // Set ObservePeriod to longer than the timeout period to mock a timeout while waiting on ring to become ACTIVE + cfg.ShardingRing.ObservePeriod = time.Second * 10 + + c, _, _, _, _ := prepare(t, cfg, bucketClient) + + // Try to start the compactor with a bad consul kv-store. The + err := services.StartAndAwaitRunning(context.Background(), c) + + // Assert that the compactor timed out + require.ErrorIs(t, err, context.DeadlineExceeded) +} + +type ownUserReason int + +const ( + ownUserReasonBlocksCleaner ownUserReason = iota + ownUserReasonCompactor +) + +func TestOwnUser(t *testing.T) { + type testCase struct { + compactors int + enabledUsers []string + disabledUsers []string + compactorShards map[string]int + + check func(t *testing.T, comps []*MultitenantCompactor) + } + + const user1 = "user1" + const user2 = "another-user" + + testCases := map[string]testCase{ + "5 compactors, sharding enabled, no compactor shard size": { + compactors: 5, + compactorShards: nil, // no limits + + check: func(t *testing.T, comps []*MultitenantCompactor) { + require.Len(t, owningCompactors(t, comps, user1, ownUserReasonCompactor), 5) + require.Len(t, owningCompactors(t, comps, user1, ownUserReasonBlocksCleaner), 1) + + require.Len(t, owningCompactors(t, comps, user2, ownUserReasonCompactor), 5) + require.Len(t, owningCompactors(t, comps, user2, ownUserReasonBlocksCleaner), 1) + }, + }, + + "10 compactors, sharding enabled, with non-zero shard sizes": { + compactors: 10, + compactorShards: map[string]int{user1: 2, user2: 3}, + + check: func(t *testing.T, comps []*MultitenantCompactor) { + require.Len(t, owningCompactors(t, comps, user1, ownUserReasonCompactor), 2) + require.Len(t, owningCompactors(t, comps, user1, ownUserReasonBlocksCleaner), 1) + // Blocks cleanup is done by one of the compactors that "own" the user. + require.Subset(t, owningCompactors(t, comps, user1, ownUserReasonCompactor), owningCompactors(t, comps, user1, ownUserReasonBlocksCleaner)) + + require.Len(t, owningCompactors(t, comps, user2, ownUserReasonCompactor), 3) + require.Len(t, owningCompactors(t, comps, user2, ownUserReasonBlocksCleaner), 1) + // Blocks cleanup is done by one of the compactors that "own" the user. + require.Subset(t, owningCompactors(t, comps, user2, ownUserReasonCompactor), owningCompactors(t, comps, user2, ownUserReasonBlocksCleaner)) + }, + }, + + "10 compactors, sharding enabled, with zero shard size": { + compactors: 10, + compactorShards: map[string]int{user2: 0}, + + check: func(t *testing.T, comps []*MultitenantCompactor) { + require.Len(t, owningCompactors(t, comps, user2, ownUserReasonCompactor), 10) + require.Len(t, owningCompactors(t, comps, user2, ownUserReasonBlocksCleaner), 1) + }, + }, + } + + for name, tc := range testCases { + tc := tc + t.Run(name, func(t *testing.T) { + t.Parallel() + + kvStore, closer := consul.NewInMemoryClient(ring.GetCodec(), log.NewNopLogger(), nil) + t.Cleanup(func() { assert.NoError(t, closer.Close()) }) + + inmem := objstore.NewInMemBucket() + + compactors := []*MultitenantCompactor(nil) + + for i := 0; i < tc.compactors; i++ { + cfg := prepareConfig(t) + cfg.CompactionInterval = 10 * time.Minute // We will only call compaction manually. + + cfg.EnabledTenants = tc.enabledUsers + cfg.DisabledTenants = tc.disabledUsers + + cfg.ShardingRing.Common.InstanceID = fmt.Sprintf("compactor-%d", i) + cfg.ShardingRing.Common.InstanceAddr = fmt.Sprintf("127.0.0.%d", i) + // No need to wait. All compactors are started before we do any tests, and we wait for all of them + // to appear in all rings. + cfg.ShardingRing.WaitStabilityMinDuration = 0 + cfg.ShardingRing.WaitStabilityMaxDuration = 0 + cfg.ShardingRing.Common.KVStore.Mock = kvStore + + limits := newMockConfigProvider() + limits.instancesShardSize = tc.compactorShards + + c, _, _, _, _ := prepareWithConfigProvider(t, cfg, inmem, limits) + require.NoError(t, services.StartAndAwaitRunning(context.Background(), c)) + t.Cleanup(stopServiceFn(t, c)) + + compactors = append(compactors, c) + } + + // Make sure all compactors see all other compactors in the ring before running tests. + test.Poll(t, 2*time.Second, true, func() interface{} { + for _, c := range compactors { + rs, err := c.ring.GetAllHealthy(RingOp) + if err != nil { + return false + } + if len(rs.Instances) != len(compactors) { + return false + } + } + return true + }) + + tc.check(t, compactors) + }) + } +} + +func owningCompactors(t *testing.T, comps []*MultitenantCompactor, user string, reason ownUserReason) []string { + result := []string(nil) + for _, c := range comps { + var f func(string) (bool, error) + if reason == ownUserReasonCompactor { + f = c.shardingStrategy.compactorOwnUser + } else { + f = c.shardingStrategy.blocksCleanerOwnUser + } + ok, err := f(user) + require.NoError(t, err) + if ok { + // We set instance ID even when not using sharding. It makes output nicer, since + // calling method only wants to see some identifier. + result = append(result, c.compactorCfg.ShardingRing.Common.InstanceID) + } + } + return result +} + +func stopServiceFn(t *testing.T, serv services.Service) func() { + return func() { + require.NoError(t, services.StopAndAwaitTerminated(context.Background(), serv)) + } +} + +func TestMultitenantCompactor_OutOfOrderCompaction(t *testing.T) { + // Generate a single block with out of order chunks. + specs := []*block.SeriesSpec{ + { + Labels: labels.FromStrings("case", "out_of_order"), + Chunks: []chunks.Meta{ + must(tsdbutil.ChunkFromSamples([]tsdbutil.Sample{newSample(20, 20, nil, nil), newSample(21, 21, nil, nil)})), + must(tsdbutil.ChunkFromSamples([]tsdbutil.Sample{newSample(10, 10, nil, nil), newSample(11, 11, nil, nil)})), + // Extend block to cover 2h. + must(tsdbutil.ChunkFromSamples([]tsdbutil.Sample{newSample(0, 0, nil, nil), newSample(2*time.Hour.Milliseconds()-1, 0, nil, nil)})), + }, + }, + } + + const user = "user" + + storageDir := t.TempDir() + // We need two blocks to start compaction. + meta1, err := block.GenerateBlockFromSpec(user, filepath.Join(storageDir, user), specs) + require.NoError(t, err) + meta2, err := block.GenerateBlockFromSpec(user, filepath.Join(storageDir, user), specs) + require.NoError(t, err) + + bkt, err := filesystem.NewBucketClient(filesystem.Config{Directory: storageDir}) + require.NoError(t, err) + + cfg := prepareConfig(t) + c, _, tsdbPlanner, logs, registry := prepare(t, cfg, bkt) + + tsdbPlanner.On("Plan", mock.Anything, mock.Anything).Return([]*block.Meta{meta1, meta2}, nil) + + // Start the compactor + require.NoError(t, services.StartAndAwaitRunning(context.Background(), c)) + + // Wait until a compaction run has been completed. + test.Poll(t, 10*time.Second, 1.0, func() interface{} { + return prom_testutil.ToFloat64(c.compactionRunsCompleted) + }) + + // Stop the compactor. + require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) + + // Verify that compactor has found block with out of order chunks, and this block is now marked for no-compaction. + r := regexp.MustCompile("level=info component=compactor user=user msg=\"block has been marked for no compaction\" block=([0-9A-Z]+)") + matches := r.FindStringSubmatch(logs.String()) + require.Len(t, matches, 2) // Entire string match + single group match. + + skippedBlock := matches[1] + require.True(t, skippedBlock == meta1.ULID.String() || skippedBlock == meta2.ULID.String()) + + m := &block.NoCompactMark{} + require.NoError(t, block.ReadMarker(context.Background(), log.NewNopLogger(), objstore.WithNoopInstr(bkt), path.Join(user, skippedBlock), m)) + require.Equal(t, skippedBlock, m.ID.String()) + require.NotZero(t, m.NoCompactTime) + require.Equal(t, block.NoCompactReason(block.OutOfOrderChunksNoCompactReason), m.Reason) + + assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` + # HELP cortex_compactor_blocks_marked_for_no_compaction_total Total number of blocks that were marked for no-compaction. + # TYPE cortex_compactor_blocks_marked_for_no_compaction_total counter + cortex_compactor_blocks_marked_for_no_compaction_total{reason="block-index-out-of-order-chunk"} 1 + `), + "cortex_compactor_blocks_marked_for_no_compaction_total", + )) +} + +type sample struct { + t int64 + v float64 + h *histogram.Histogram + fh *histogram.FloatHistogram +} + +func newSample(t int64, v float64, h *histogram.Histogram, fh *histogram.FloatHistogram) tsdbutil.Sample { + return sample{t, v, h, fh} +} +func (s sample) T() int64 { return s.t } +func (s sample) F() float64 { return s.v } +func (s sample) H() *histogram.Histogram { return s.h } +func (s sample) FH() *histogram.FloatHistogram { return s.fh } + +func (s sample) Type() chunkenc.ValueType { + switch { + case s.h != nil: + return chunkenc.ValHistogram + case s.fh != nil: + return chunkenc.ValFloatHistogram + default: + return chunkenc.ValFloat + } +} + +type bucketWithMockedAttributes struct { + objstore.Bucket + + customAttributes map[string]objstore.ObjectAttributes +} + +func (b *bucketWithMockedAttributes) Attributes(ctx context.Context, name string) (objstore.ObjectAttributes, error) { + if attrs, ok := b.customAttributes[name]; ok { + return attrs, nil + } + + return b.Bucket.Attributes(ctx, name) +} + +func must[T any](v T, err error) T { + if err != nil { + panic(err) + } + return v +} diff --git a/pkg/compactor/job.go b/pkg/compactor/job.go new file mode 100644 index 0000000000..53d224a892 --- /dev/null +++ b/pkg/compactor/job.go @@ -0,0 +1,188 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/grafana/mimir/blob/main/pkg/compactor/job.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Cortex Authors. +package compactor + +import ( + "context" + "fmt" + "math" + "path" + "sort" + "time" + + "github.com/oklog/ulid" + "github.com/pkg/errors" + "github.com/prometheus/prometheus/model/labels" + "github.com/thanos-io/objstore" + + "github.com/grafana/mimir/pkg/storage/tsdb/block" +) + +// Job holds a compaction job, which consists of a group of blocks that should be compacted together. +// Not goroutine safe. +type Job struct { + userID string + key string + labels labels.Labels + resolution int64 + metasByMinTime []*block.Meta + useSplitting bool + shardingKey string + + // The number of shards to split compacted block into. Not used if splitting is disabled. + splitNumShards uint32 +} + +// NewJob returns a new compaction Job. +func NewJob(userID string, key string, lset labels.Labels, resolution int64, useSplitting bool, splitNumShards uint32, shardingKey string) *Job { + return &Job{ + userID: userID, + key: key, + labels: lset, + resolution: resolution, + useSplitting: useSplitting, + splitNumShards: splitNumShards, + shardingKey: shardingKey, + } +} + +// UserID returns the user/tenant to which this job belongs to. +func (job *Job) UserID() string { + return job.userID +} + +// Key returns an identifier for the job. +func (job *Job) Key() string { + return job.key +} + +// AppendMeta the block with the given meta to the job. +func (job *Job) AppendMeta(meta *block.Meta) error { + if !labels.Equal(job.labels, labels.FromMap(meta.Thanos.Labels)) { + return errors.New("block and group labels do not match") + } + if job.resolution != meta.Thanos.Downsample.Resolution { + return errors.New("block and group resolution do not match") + } + + job.metasByMinTime = append(job.metasByMinTime, meta) + sort.Slice(job.metasByMinTime, func(i, j int) bool { + return job.metasByMinTime[i].MinTime < job.metasByMinTime[j].MinTime + }) + return nil +} + +// IDs returns all sorted IDs of blocks in the job. +func (job *Job) IDs() (ids []ulid.ULID) { + for _, m := range job.metasByMinTime { + ids = append(ids, m.ULID) + } + sort.Slice(ids, func(i, j int) bool { + return ids[i].Compare(ids[j]) < 0 + }) + return ids +} + +// MinTime returns the min time across all job's blocks. +func (job *Job) MinTime() int64 { + if len(job.metasByMinTime) > 0 { + return job.metasByMinTime[0].MinTime + } + return math.MaxInt64 +} + +// MaxTime returns the max time across all job's blocks. +func (job *Job) MaxTime() int64 { + max := int64(math.MinInt64) + for _, m := range job.metasByMinTime { + if m.MaxTime > max { + max = m.MaxTime + } + } + return max +} + +// MinCompactionLevel returns the minimum compaction level across all source blocks +// in this job. +func (job *Job) MinCompactionLevel() int { + min := math.MaxInt + + for _, m := range job.metasByMinTime { + if m.Compaction.Level < min { + min = m.Compaction.Level + } + } + + return min +} + +// Metas returns the metadata for each block that is part of this job, ordered by the block's MinTime +func (job *Job) Metas() []*block.Meta { + out := make([]*block.Meta, len(job.metasByMinTime)) + copy(out, job.metasByMinTime) + return out +} + +// Labels returns the external labels for the output block(s) of this job. +func (job *Job) Labels() labels.Labels { + return job.labels +} + +// Resolution returns the common downsampling resolution of blocks in the job. +func (job *Job) Resolution() int64 { + return job.resolution +} + +// UseSplitting returns whether blocks should be split into multiple shards when compacted. +func (job *Job) UseSplitting() bool { + return job.useSplitting +} + +// SplittingShards returns the number of output shards to build if splitting is enabled. +func (job *Job) SplittingShards() uint32 { + return job.splitNumShards +} + +// ShardingKey returns the key used to shard this job across multiple instances. +func (job *Job) ShardingKey() string { + return job.shardingKey +} + +func (job *Job) String() string { + return fmt.Sprintf("%s (minTime: %d maxTime: %d)", job.Key(), job.MinTime(), job.MaxTime()) +} + +// jobWaitPeriodElapsed returns whether the 1st level compaction wait period has +// elapsed for the input job. If the wait period has not elapsed, then this function +// also returns the Meta of the first source block encountered for which the wait +// period has not elapsed yet. +func jobWaitPeriodElapsed(ctx context.Context, job *Job, waitPeriod time.Duration, userBucket objstore.Bucket) (bool, *block.Meta, error) { + if waitPeriod <= 0 { + return true, nil, nil + } + + if job.MinCompactionLevel() > 1 { + return true, nil, nil + } + + // Check if the job contains any source block uploaded more recently + // than "wait period" ago. + threshold := time.Now().Add(-waitPeriod) + + for _, meta := range job.Metas() { + metaPath := path.Join(meta.ULID.String(), block.MetaFilename) + + attrs, err := userBucket.Attributes(ctx, metaPath) + if err != nil { + return false, meta, errors.Wrapf(err, "unable to get object attributes for %s", metaPath) + } + + if attrs.LastModified.After(threshold) { + return false, meta, nil + } + } + + return true, nil, nil +} diff --git a/pkg/compactor/job_sorting.go b/pkg/compactor/job_sorting.go new file mode 100644 index 0000000000..738b067c64 --- /dev/null +++ b/pkg/compactor/job_sorting.go @@ -0,0 +1,99 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/grafana/mimir/blob/main/pkg/compactor/job_sorting.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Cortex Authors. + +package compactor + +import ( + "sort" +) + +const ( + CompactionOrderOldestFirst = "smallest-range-oldest-blocks-first" + CompactionOrderNewestFirst = "newest-blocks-first" +) + +var CompactionOrders = []string{CompactionOrderOldestFirst, CompactionOrderNewestFirst} + +type JobsOrderFunc func(jobs []*Job) []*Job + +// GetJobsOrderFunction returns jobs ordering function, or nil, if name doesn't refer to any function. +func GetJobsOrderFunction(name string) JobsOrderFunc { + switch name { + case CompactionOrderNewestFirst: + return sortJobsByNewestBlocksFirst + case CompactionOrderOldestFirst: + return sortJobsBySmallestRangeOldestBlocksFirst + default: + return nil + } +} + +// sortJobsBySmallestRangeOldestBlocksFirst returns input jobs sorted by smallest range, oldest min time first. +// The rationale of this sorting is that we may want to favor smaller ranges first (ie. to deduplicate samples +// sooner than later) and older ones are more likely to be "complete" (no missing block still to be uploaded). +// Split jobs are moved to the beginning of the output, because merge jobs are only generated if there are no split jobs in the +// same time range, so finishing split jobs first unblocks more jobs and gives opportunity to more compactors +// to work on them. +func sortJobsBySmallestRangeOldestBlocksFirst(jobs []*Job) []*Job { + sort.SliceStable(jobs, func(i, j int) bool { + // Move split jobs to the front. + if jobs[i].UseSplitting() && !jobs[j].UseSplitting() { + return true + } + + if !jobs[i].UseSplitting() && jobs[j].UseSplitting() { + return false + } + + checkLength := true + // Don't check length for splitting jobs. We want to the oldest split blocks to be first, no matter the length. + if jobs[i].UseSplitting() && jobs[j].UseSplitting() { + checkLength = false + } + + if checkLength { + iLength := jobs[i].MaxTime() - jobs[i].MinTime() + jLength := jobs[j].MaxTime() - jobs[j].MinTime() + + if iLength != jLength { + return iLength < jLength + } + } + + if jobs[i].MinTime() != jobs[j].MinTime() { + return jobs[i].MinTime() < jobs[j].MinTime() + } + + // Guarantee stable sort for tests. + return jobs[i].Key() < jobs[j].Key() + }) + + return jobs +} + +// sortJobsByNewestBlocksFirst returns input jobs sorted by most recent time ranges first +// (regardless of their compaction level). The rationale of this sorting is that in case the +// compactor is lagging behind, we compact up to the largest range (eg. 24h) the most recent +// blocks first and the move to older ones. Most recent blocks are the one more likely to be queried. +func sortJobsByNewestBlocksFirst(jobs []*Job) []*Job { + sort.SliceStable(jobs, func(i, j int) bool { + iMaxTime := jobs[i].MaxTime() + jMaxTime := jobs[j].MaxTime() + if iMaxTime != jMaxTime { + return iMaxTime > jMaxTime + } + + iLength := iMaxTime - jobs[i].MinTime() + jLength := jMaxTime - jobs[j].MinTime() + if iLength != jLength { + return iLength < jLength + } + + // Guarantee stable sort for tests. + return jobs[i].Key() < jobs[j].Key() + }) + + return jobs +} diff --git a/pkg/compactor/job_sorting_test.go b/pkg/compactor/job_sorting_test.go new file mode 100644 index 0000000000..33c9ea2259 --- /dev/null +++ b/pkg/compactor/job_sorting_test.go @@ -0,0 +1,146 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/grafana/mimir/blob/main/pkg/compactor/job_sorting_test.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Cortex Authors. + +package compactor + +import ( + "testing" + + "github.com/oklog/ulid" + "github.com/prometheus/prometheus/tsdb" + "github.com/stretchr/testify/assert" + + "github.com/grafana/mimir/pkg/storage/tsdb/block" +) + +func TestSortJobsBySmallestRangeOldestBlocksFirst(t *testing.T) { + block1 := ulid.MustNew(1, nil) + block2 := ulid.MustNew(2, nil) + block3 := ulid.MustNew(3, nil) + block4 := ulid.MustNew(4, nil) + block5 := ulid.MustNew(5, nil) + block6 := ulid.MustNew(6, nil) + + tests := map[string]struct { + input []*Job + expected []*Job + }{ + "should do nothing on empty input": { + input: nil, + expected: nil, + }, + "should sort jobs by smallest range, oldest blocks first": { + input: []*Job{ + {metasByMinTime: []*block.Meta{mockMetaWithMinMax(block5, 40, 60), mockMetaWithMinMax(block6, 40, 80)}}, + {metasByMinTime: []*block.Meta{mockMetaWithMinMax(block3, 10, 20), mockMetaWithMinMax(block4, 20, 30)}}, + {metasByMinTime: []*block.Meta{mockMetaWithMinMax(block1, 10, 20), mockMetaWithMinMax(block2, 10, 20)}}, + }, + expected: []*Job{ + {metasByMinTime: []*block.Meta{mockMetaWithMinMax(block1, 10, 20), mockMetaWithMinMax(block2, 10, 20)}}, + {metasByMinTime: []*block.Meta{mockMetaWithMinMax(block3, 10, 20), mockMetaWithMinMax(block4, 20, 30)}}, + {metasByMinTime: []*block.Meta{mockMetaWithMinMax(block5, 40, 60), mockMetaWithMinMax(block6, 40, 80)}}, + }, + }, + "split jobs are always sorted first": { + input: []*Job{ + {metasByMinTime: []*block.Meta{mockMetaWithMinMax(block5, 40, 60), mockMetaWithMinMax(block6, 40, 80)}}, + {metasByMinTime: []*block.Meta{mockMetaWithMinMax(block3, 10, 20), mockMetaWithMinMax(block4, 20, 30)}, useSplitting: false}, + {metasByMinTime: []*block.Meta{mockMetaWithMinMax(block3, 10, 20), mockMetaWithMinMax(block4, 20, 30)}, useSplitting: true}, + {metasByMinTime: []*block.Meta{mockMetaWithMinMax(block1, 10, 20), mockMetaWithMinMax(block2, 10, 20)}}, + {metasByMinTime: []*block.Meta{mockMetaWithMinMax(block4, 5, 50)}, useSplitting: true}, // Big splitting block. Should be sorted by minTime only. + }, + expected: []*Job{ + {metasByMinTime: []*block.Meta{mockMetaWithMinMax(block4, 5, 50)}, useSplitting: true}, + {metasByMinTime: []*block.Meta{mockMetaWithMinMax(block3, 10, 20), mockMetaWithMinMax(block4, 20, 30)}, useSplitting: true}, // Split job is first. + {metasByMinTime: []*block.Meta{mockMetaWithMinMax(block1, 10, 20), mockMetaWithMinMax(block2, 10, 20)}}, + {metasByMinTime: []*block.Meta{mockMetaWithMinMax(block3, 10, 20), mockMetaWithMinMax(block4, 20, 30)}, useSplitting: false}, + {metasByMinTime: []*block.Meta{mockMetaWithMinMax(block5, 40, 60), mockMetaWithMinMax(block6, 40, 80)}}, + }, + }, + } + + for testName, testData := range tests { + t.Run(testName, func(t *testing.T) { + assert.Equal(t, testData.expected, sortJobsBySmallestRangeOldestBlocksFirst(testData.input)) + }) + } +} + +func TestSortJobsByNewestBlocksFirst(t *testing.T) { + block1 := ulid.MustNew(1, nil) + block2 := ulid.MustNew(2, nil) + block3 := ulid.MustNew(3, nil) + block4 := ulid.MustNew(4, nil) + block5 := ulid.MustNew(5, nil) + block6 := ulid.MustNew(6, nil) + block7 := ulid.MustNew(7, nil) + + tests := map[string]struct { + input []*Job + expected []*Job + }{ + "should do nothing on empty input": { + input: nil, + expected: nil, + }, + "should sort jobs by newest blocks first": { + input: []*Job{ + {metasByMinTime: []*block.Meta{mockMetaWithMinMax(block1, 10, 20), mockMetaWithMinMax(block2, 10, 20)}}, + {metasByMinTime: []*block.Meta{mockMetaWithMinMax(block3, 10, 20), mockMetaWithMinMax(block4, 20, 30)}}, + {metasByMinTime: []*block.Meta{mockMetaWithMinMax(block5, 40, 60), mockMetaWithMinMax(block6, 40, 80)}}, + }, + expected: []*Job{ + {metasByMinTime: []*block.Meta{mockMetaWithMinMax(block5, 40, 60), mockMetaWithMinMax(block6, 40, 80)}}, + {metasByMinTime: []*block.Meta{mockMetaWithMinMax(block3, 10, 20), mockMetaWithMinMax(block4, 20, 30)}}, + {metasByMinTime: []*block.Meta{mockMetaWithMinMax(block1, 10, 20), mockMetaWithMinMax(block2, 10, 20)}}, + }, + }, + "should give precedence to smaller time ranges in case of multiple jobs with the same max time": { + input: []*Job{ + {metasByMinTime: []*block.Meta{mockMetaWithMinMax(block1, 10, 20), mockMetaWithMinMax(block2, 20, 30), mockMetaWithMinMax(block3, 30, 40)}}, + {metasByMinTime: []*block.Meta{mockMetaWithMinMax(block4, 30, 40), mockMetaWithMinMax(block5, 30, 40)}}, + }, + expected: []*Job{ + {metasByMinTime: []*block.Meta{mockMetaWithMinMax(block4, 30, 40), mockMetaWithMinMax(block5, 30, 40)}}, + {metasByMinTime: []*block.Meta{mockMetaWithMinMax(block1, 10, 20), mockMetaWithMinMax(block2, 20, 30), mockMetaWithMinMax(block3, 30, 40)}}, + }, + }, + "should give precedence to newest blocks over smaller time ranges": { + input: []*Job{ + {metasByMinTime: []*block.Meta{mockMetaWithMinMax(block1, 10, 20), mockMetaWithMinMax(block2, 20, 30), mockMetaWithMinMax(block3, 30, 40)}}, + {metasByMinTime: []*block.Meta{mockMetaWithMinMax(block6, 10, 20), mockMetaWithMinMax(block7, 10, 20)}}, + {metasByMinTime: []*block.Meta{mockMetaWithMinMax(block4, 10, 30), mockMetaWithMinMax(block5, 20, 30)}}, + }, + expected: []*Job{ + {metasByMinTime: []*block.Meta{mockMetaWithMinMax(block1, 10, 20), mockMetaWithMinMax(block2, 20, 30), mockMetaWithMinMax(block3, 30, 40)}}, + {metasByMinTime: []*block.Meta{mockMetaWithMinMax(block4, 10, 30), mockMetaWithMinMax(block5, 20, 30)}}, + {metasByMinTime: []*block.Meta{mockMetaWithMinMax(block6, 10, 20), mockMetaWithMinMax(block7, 10, 20)}}, + }, + }, + } + + for testName, testData := range tests { + t.Run(testName, func(t *testing.T) { + actual := sortJobsByNewestBlocksFirst(testData.input) + assert.Equal(t, testData.expected, actual) + + // Print for debugging. + t.Log("sorted jobs:") + for _, job := range actual { + t.Logf("- %s", job.String()) + } + }) + } +} + +func mockMetaWithMinMax(id ulid.ULID, minTime, maxTime int64) *block.Meta { + return &block.Meta{ + BlockMeta: tsdb.BlockMeta{ + ULID: id, + MinTime: minTime, + MaxTime: maxTime, + }, + } +} diff --git a/pkg/compactor/job_test.go b/pkg/compactor/job_test.go new file mode 100644 index 0000000000..35df7e65d0 --- /dev/null +++ b/pkg/compactor/job_test.go @@ -0,0 +1,135 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/grafana/mimir/blob/main/pkg/compactor/job_test.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Cortex Authors. + +package compactor + +import ( + "context" + "path" + "testing" + "time" + + "github.com/oklog/ulid" + "github.com/pkg/errors" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/tsdb" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/thanos-io/objstore" + + "github.com/grafana/mimir/pkg/storage/bucket" + "github.com/grafana/mimir/pkg/storage/tsdb/block" +) + +func TestJob_MinCompactionLevel(t *testing.T) { + job := NewJob("user-1", "group-1", labels.EmptyLabels(), 0, true, 2, "shard-1") + require.NoError(t, job.AppendMeta(&block.Meta{BlockMeta: tsdb.BlockMeta{ULID: ulid.MustNew(1, nil), Compaction: tsdb.BlockMetaCompaction{Level: 2}}})) + assert.Equal(t, 2, job.MinCompactionLevel()) + + require.NoError(t, job.AppendMeta(&block.Meta{BlockMeta: tsdb.BlockMeta{ULID: ulid.MustNew(2, nil), Compaction: tsdb.BlockMetaCompaction{Level: 3}}})) + assert.Equal(t, 2, job.MinCompactionLevel()) + + require.NoError(t, job.AppendMeta(&block.Meta{BlockMeta: tsdb.BlockMeta{ULID: ulid.MustNew(3, nil), Compaction: tsdb.BlockMetaCompaction{Level: 1}}})) + assert.Equal(t, 1, job.MinCompactionLevel()) +} + +func TestJobWaitPeriodElapsed(t *testing.T) { + type jobBlock struct { + meta *block.Meta + attrs objstore.ObjectAttributes + attrsErr error + } + + // Blocks with compaction level 1. + meta1 := &block.Meta{BlockMeta: tsdb.BlockMeta{ULID: ulid.MustNew(1, nil), Compaction: tsdb.BlockMetaCompaction{Level: 1}}} + meta2 := &block.Meta{BlockMeta: tsdb.BlockMeta{ULID: ulid.MustNew(2, nil), Compaction: tsdb.BlockMetaCompaction{Level: 1}}} + + // Blocks with compaction level 2. + meta3 := &block.Meta{BlockMeta: tsdb.BlockMeta{ULID: ulid.MustNew(3, nil), Compaction: tsdb.BlockMetaCompaction{Level: 2}}} + meta4 := &block.Meta{BlockMeta: tsdb.BlockMeta{ULID: ulid.MustNew(4, nil), Compaction: tsdb.BlockMetaCompaction{Level: 2}}} + + tests := map[string]struct { + waitPeriod time.Duration + jobBlocks []jobBlock + expectedElapsed bool + expectedMeta *block.Meta + expectedErr string + }{ + "wait period disabled": { + waitPeriod: 0, + jobBlocks: []jobBlock{ + {meta: meta1, attrs: objstore.ObjectAttributes{LastModified: time.Now().Add(-20 * time.Minute)}}, + {meta: meta2, attrs: objstore.ObjectAttributes{LastModified: time.Now().Add(-5 * time.Minute)}}, + }, + expectedElapsed: true, + expectedMeta: nil, + }, + "blocks uploaded since more than the wait period": { + waitPeriod: 10 * time.Minute, + jobBlocks: []jobBlock{ + {meta: meta1, attrs: objstore.ObjectAttributes{LastModified: time.Now().Add(-20 * time.Minute)}}, + {meta: meta2, attrs: objstore.ObjectAttributes{LastModified: time.Now().Add(-25 * time.Minute)}}, + }, + expectedElapsed: true, + expectedMeta: nil, + }, + "blocks uploaded since less than the wait period": { + waitPeriod: 10 * time.Minute, + jobBlocks: []jobBlock{ + {meta: meta1, attrs: objstore.ObjectAttributes{LastModified: time.Now().Add(-20 * time.Minute)}}, + {meta: meta2, attrs: objstore.ObjectAttributes{LastModified: time.Now().Add(-5 * time.Minute)}}, + }, + expectedElapsed: false, + expectedMeta: meta2, + }, + "blocks uploaded since less than the wait period but their compaction level is > 1": { + waitPeriod: 10 * time.Minute, + jobBlocks: []jobBlock{ + {meta: meta3, attrs: objstore.ObjectAttributes{LastModified: time.Now().Add(-4 * time.Minute)}}, + {meta: meta4, attrs: objstore.ObjectAttributes{LastModified: time.Now().Add(-5 * time.Minute)}}, + }, + expectedElapsed: true, + expectedMeta: nil, + }, + "an error occurred while checking the blocks upload timestamp": { + waitPeriod: 10 * time.Minute, + jobBlocks: []jobBlock{ + // This block has been uploaded since more than the wait period. + {meta: meta1, attrs: objstore.ObjectAttributes{LastModified: time.Now().Add(-20 * time.Minute)}}, + + // This block has been uploaded since less than the wait period, but we failed getting its attributes. + {meta: meta2, attrs: objstore.ObjectAttributes{LastModified: time.Now().Add(-5 * time.Minute)}, attrsErr: errors.New("mocked error")}, + }, + expectedErr: "mocked error", + expectedMeta: meta2, + }, + } + + for testName, testData := range tests { + t.Run(testName, func(t *testing.T) { + job := NewJob("user-1", "group-1", labels.EmptyLabels(), 0, true, 2, "shard-1") + for _, b := range testData.jobBlocks { + require.NoError(t, job.AppendMeta(b.meta)) + } + + userBucket := &bucket.ClientMock{} + for _, b := range testData.jobBlocks { + userBucket.MockAttributes(path.Join(b.meta.ULID.String(), block.MetaFilename), b.attrs, b.attrsErr) + } + + elapsed, meta, err := jobWaitPeriodElapsed(context.Background(), job, testData.waitPeriod, userBucket) + if testData.expectedErr != "" { + require.Error(t, err) + assert.ErrorContains(t, err, testData.expectedErr) + assert.False(t, elapsed) + assert.Equal(t, testData.expectedMeta, meta) + } else { + require.NoError(t, err) + assert.Equal(t, testData.expectedElapsed, elapsed) + assert.Equal(t, testData.expectedMeta, meta) + } + }) + } +} diff --git a/pkg/compactor/label_remover_filter.go b/pkg/compactor/label_remover_filter.go new file mode 100644 index 0000000000..6525453e26 --- /dev/null +++ b/pkg/compactor/label_remover_filter.go @@ -0,0 +1,34 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/grafana/mimir/blob/main/pkg/compactor/label_remover_filter.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Cortex Authors. + +package compactor + +import ( + "context" + + "github.com/oklog/ulid" + + "github.com/grafana/mimir/pkg/storage/tsdb/block" +) + +type LabelRemoverFilter struct { + labels []string +} + +// NewLabelRemoverFilter creates a LabelRemoverFilter. +func NewLabelRemoverFilter(labels []string) *LabelRemoverFilter { + return &LabelRemoverFilter{labels: labels} +} + +// Filter modifies external labels of existing blocks, removing given labels from the metadata of blocks that have it. +func (f *LabelRemoverFilter) Filter(_ context.Context, metas map[ulid.ULID]*block.Meta, _ block.GaugeVec) error { + for _, meta := range metas { + for _, l := range f.labels { + delete(meta.Thanos.Labels, l) + } + } + + return nil +} diff --git a/pkg/compactor/label_remover_filter_test.go b/pkg/compactor/label_remover_filter_test.go new file mode 100644 index 0000000000..0866f69e99 --- /dev/null +++ b/pkg/compactor/label_remover_filter_test.go @@ -0,0 +1,77 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/grafana/mimir/blob/main/pkg/compactor/label_remover_filter_test.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Cortex Authors. + +package compactor + +import ( + "context" + "testing" + + "github.com/oklog/ulid" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + + mimir_tsdb "github.com/grafana/mimir/pkg/storage/tsdb" + "github.com/grafana/mimir/pkg/storage/tsdb/block" +) + +func TestLabelRemoverFilter(t *testing.T) { + block1 := ulid.MustNew(1, nil) + block2 := ulid.MustNew(2, nil) + block3 := ulid.MustNew(3, nil) + + tests := map[string]struct { + labels []string + input map[ulid.ULID]map[string]string + expected map[ulid.ULID]map[string]string + }{ + "should remove configured labels": { + labels: []string{mimir_tsdb.DeprecatedIngesterIDExternalLabel}, + input: map[ulid.ULID]map[string]string{ + block1: {mimir_tsdb.DeprecatedIngesterIDExternalLabel: "ingester-0", mimir_tsdb.DeprecatedTenantIDExternalLabel: "user-1"}, + block2: {mimir_tsdb.DeprecatedIngesterIDExternalLabel: "ingester-0", mimir_tsdb.DeprecatedTenantIDExternalLabel: "user-1"}, + block3: {mimir_tsdb.DeprecatedIngesterIDExternalLabel: "ingester-0", mimir_tsdb.DeprecatedTenantIDExternalLabel: "user-1"}, + }, + expected: map[ulid.ULID]map[string]string{ + block1: {mimir_tsdb.DeprecatedTenantIDExternalLabel: "user-1"}, + block2: {mimir_tsdb.DeprecatedTenantIDExternalLabel: "user-1"}, + block3: {mimir_tsdb.DeprecatedTenantIDExternalLabel: "user-1"}, + }, + }, + + "should remove configured labels 2": { + labels: []string{mimir_tsdb.DeprecatedIngesterIDExternalLabel, mimir_tsdb.DeprecatedTenantIDExternalLabel}, + input: map[ulid.ULID]map[string]string{ + block1: {mimir_tsdb.DeprecatedIngesterIDExternalLabel: "ingester-0", mimir_tsdb.DeprecatedTenantIDExternalLabel: "user-1"}, + block2: {mimir_tsdb.DeprecatedIngesterIDExternalLabel: "ingester-0", mimir_tsdb.DeprecatedTenantIDExternalLabel: "user-1"}, + block3: {mimir_tsdb.DeprecatedIngesterIDExternalLabel: "ingester-0", mimir_tsdb.DeprecatedTenantIDExternalLabel: "user-1"}, + }, + expected: map[ulid.ULID]map[string]string{ + block1: {}, + block2: {}, + block3: {}, + }, + }, + } + + for testName, testData := range tests { + t.Run(testName, func(t *testing.T) { + metas := map[ulid.ULID]*block.Meta{} + for id, lbls := range testData.input { + metas[id] = &block.Meta{Thanos: block.ThanosMeta{Labels: lbls}} + } + + f := NewLabelRemoverFilter(testData.labels) + err := f.Filter(context.Background(), metas, nil) + require.NoError(t, err) + assert.Len(t, metas, len(testData.expected)) + + for expectedID, expectedLbls := range testData.expected { + assert.NotNil(t, metas[expectedID]) + assert.Equal(t, expectedLbls, metas[expectedID].Thanos.Labels) + } + }) + } +} diff --git a/pkg/compactor/shard_aware_deduplicate_filter.go b/pkg/compactor/shard_aware_deduplicate_filter.go new file mode 100644 index 0000000000..906ab183cf --- /dev/null +++ b/pkg/compactor/shard_aware_deduplicate_filter.go @@ -0,0 +1,293 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/grafana/mimir/blob/main/pkg/compactor/shard_aware_deduplicate_filter.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Cortex Authors. + +package compactor + +import ( + "context" + "sort" + + "github.com/oklog/ulid" + + "github.com/grafana/mimir/pkg/storage/sharding" + "github.com/grafana/mimir/pkg/storage/tsdb" + "github.com/grafana/mimir/pkg/storage/tsdb/block" +) + +const duplicateMeta = "duplicate" + +// ShardAwareDeduplicateFilter is a MetaFetcher filter that filters out older blocks that have exactly the same data. +// Not go-routine safe. +type ShardAwareDeduplicateFilter struct { + // List of duplicate IDs after last Filter call. + duplicateIDs []ulid.ULID +} + +// NewShardAwareDeduplicateFilter creates ShardAwareDeduplicateFilter. +func NewShardAwareDeduplicateFilter() *ShardAwareDeduplicateFilter { + return &ShardAwareDeduplicateFilter{} +} + +// Filter filters out from metas, the initial map of blocks, all the blocks that are contained in other, compacted, blocks. +// The removed blocks are source blocks of the blocks that remain in metas after the filtering is executed. +func (f *ShardAwareDeduplicateFilter) Filter(ctx context.Context, metas map[ulid.ULID]*block.Meta, synced block.GaugeVec) error { + f.duplicateIDs = f.duplicateIDs[:0] + + metasByResolution := make(map[int64][]*block.Meta) + for _, meta := range metas { + res := meta.Thanos.Downsample.Resolution + metasByResolution[res] = append(metasByResolution[res], meta) + } + + for res := range metasByResolution { + duplicateULIDs, err := f.findDuplicates(ctx, metasByResolution[res]) + if err != nil { + return err + } + + for id := range duplicateULIDs { + if metas[id] != nil { + f.duplicateIDs = append(f.duplicateIDs, id) + } + synced.WithLabelValues(duplicateMeta).Inc() + delete(metas, id) + } + } + + return nil +} + +// findDuplicates finds all the blocks from the input slice of blocks that are fully included in other blocks within the +// same slice. The found blocks are returned as a map which keys are blocks' ULIDs. +// +// For example consider the following blocks ("four base blocks merged and split into 2 separate shards, plus another level" test): +// +// ULID(1): {sources: []ulid.ULID{ULID(1)}}, +// ULID(2): {sources: []ulid.ULID{ULID(2)}}, +// ULID(3): {sources: []ulid.ULID{ULID(3)}}, +// ULID(4): {sources: []ulid.ULID{ULID(4)}}, +// +// ULID(5): {sources: []ulid.ULID{ULID(1), ULID(2)}, shardID: "1_of_2"}, +// ULID(6): {sources: []ulid.ULID{ULID(1), ULID(2)}, shardID: "2_of_2"}, +// +// ULID(7): {sources: []ulid.ULID{ULID(3), ULID(4)}, shardID: "1_of_2"}, +// ULID(8): {sources: []ulid.ULID{ULID(3), ULID(4)}, shardID: "2_of_2"}, +// +// ULID(9): {sources: []ulid.ULID{ULID(1), ULID(2), ULID(3), ULID(4)}, shardID: "1_of_2"}, +// ULID(10): {sources: []ulid.ULID{ULID(1), ULID(2), ULID(3), ULID(4)}, shardID: "2_of_2"}, +// +// Resulting tree will look like this: +// +// Root +// `--- ULID(1) +// | `--- ULID(5) +// | | `--- ULID(9) +// | | `--- ULID(10) +// | `--- ULID(6) +// | `--- ULID(9) +// | `--- ULID(10) +// `--- ULID(2) +// | `--- ULID(5) +// | | `--- ULID(9) +// | | `--- ULID(10) +// | `--- ULID(6) +// | `--- ULID(9) +// | `--- ULID(10) +// `--- ULID(3) +// | `--- ULID(7) +// | | `--- ULID(9) +// | | `--- ULID(10) +// | `--- ULID(8) +// | `--- ULID(9) +// | `--- ULID(10) +// `--- ULID(4) +// `--- ULID(7) +// | `--- ULID(9) +// | `--- ULID(10) +// `--- ULID(8) +// `--- ULID(9) +// `--- ULID(10) +// +// There is a lot of repetition in this tree, but individual block nodes are shared (it would be difficult to draw that though). +// So for example there is only one ULID(9) node, referenced from nodes 5, 6, 7, 8 (each of them also exists only once). See +// blockWithSuccessors structure -- it uses maps to pointers to handle all this cross-referencing correctly. +func (f *ShardAwareDeduplicateFilter) findDuplicates(ctx context.Context, input []*block.Meta) (map[ulid.ULID]struct{}, error) { + // We create a tree of blocks with successors (blockWithSuccessors) by + // 1) sorting the input blocks by number of sources, and + // 2) iterating through each input block, and adding it to the correct place in the tree of blocks with successors. + + // Sort blocks with fewer sources first. + sort.Slice(input, func(i, j int) bool { + ilen := len(input[i].Compaction.Sources) + jlen := len(input[j].Compaction.Sources) + + if ilen == jlen { + return input[i].ULID.Compare(input[j].ULID) < 0 + } + + return ilen < jlen + }) + + root := newBlockWithSuccessors(nil) + for _, meta := range input { + if err := ctx.Err(); err != nil { + return nil, err + } + + root.addSuccessorIfPossible(newBlockWithSuccessors(meta)) + } + + duplicateULIDs := make(map[ulid.ULID]struct{}) + root.getDuplicateBlocks(duplicateULIDs) + return duplicateULIDs, nil +} + +// DuplicateIDs returns slice of block ids that are filtered out by ShardAwareDeduplicateFilter. +func (f *ShardAwareDeduplicateFilter) DuplicateIDs() []ulid.ULID { + return f.duplicateIDs +} + +// blockWithSuccessors describes block (Meta) with other blocks, that contain the same sources as +// this block. We call such blocks "successors" here. For example, if there are blocks +// +// - A with sources 1 +// +// - B with sources 1, 2, 3 +// +// - C with sources 4, 5 +// +// - D with sources 1, 2, 3, 4, 5 +// +// Then B is a successor of A (A sources are subset of B sources, but not vice versa), and D is a successor of A, B and C. +type blockWithSuccessors struct { + meta *block.Meta // If meta is nil, then this is root node of the tree. + shardID string // Shard ID label value extracted from meta. If not empty, all successors must have the same shardID. + sources map[ulid.ULID]struct{} // Sources extracted from meta for easier comparison. + + successors map[ulid.ULID]*blockWithSuccessors +} + +func newBlockWithSuccessors(m *block.Meta) *blockWithSuccessors { + b := &blockWithSuccessors{meta: m} + if m != nil { + b.shardID = m.Thanos.Labels[tsdb.CompactorShardIDExternalLabel] + b.sources = make(map[ulid.ULID]struct{}, len(m.Compaction.Sources)) + for _, bid := range m.Compaction.Sources { + b.sources[bid] = struct{}{} + } + } + return b +} + +// isIncludedIn returns true, if *this* block is included in other block. +// If this block already has shard ID, then supplied metadata must use the same shard ID, +// in order to be considered "included" in other block. +func (b *blockWithSuccessors) isIncludedIn(other *blockWithSuccessors) bool { + if b.meta == nil { + return true + } + + if b.shardID != "" && b.shardID != other.shardID { + return false + } + + // All sources of this block must be in other block. + for bid := range b.sources { + if _, ok := other.sources[bid]; !ok { + return false + } + } + return true +} + +// addSuccessorIfPossible adds the given block as a direct or indirect successor of this block. +// The successor is added in the correct place in the tree of successors of this block. +// Returns true, if other block was added as successor (somewhere in the tree), false otherwise. +func (b *blockWithSuccessors) addSuccessorIfPossible(other *blockWithSuccessors) bool { + if b == other || !b.isIncludedIn(other) { + return false + } + + if _, ok := b.successors[other.meta.ULID]; ok { + return true + } + + // recursively add the other block as a successor of *all* direct or indirect successors of this block, if possible + added := false + for _, s := range b.successors { + if s.addSuccessorIfPossible(other) { + added = true + } + } + + // if the other block has not been added as a successor of any direct or indirect successor of this block, + // it must be added as a direct successor of this block + if !added { + if b.successors == nil { + b.successors = map[ulid.ULID]*blockWithSuccessors{} + } + b.successors[other.meta.ULID] = other + } + return true +} + +// isFullyIncludedInSuccessors returns true if this block is *fully* included in its own successor blocks. +// This is true under the following conditions: +// +// - if this block has a non-empty shardID, and it has *any* successors, then it is fully included in its successors. +// +// - if this block doesn't have a shardID, and it has a successor without a shardID, then it is fully included in that successor. +// +// - if this block doesn't have shardID, but *all* its successors do, and together they cover all shards, then it is fully included in its successors. +func (b *blockWithSuccessors) isFullyIncludedInSuccessors() bool { + if len(b.successors) == 0 { + return false + } + + // If there are any successors with same shard ID (all successors must have same shard ID), + // then this block must be included in them, since we don't do splitting into more shards at later levels anymore. + if b.shardID != "" { + // Double check that our invariant holds. + for _, s := range b.successors { + if s.shardID != b.shardID { + panic("successor has different shardID!") + } + } + return true + } + + shardCount := uint64(0) + shards := map[uint64]bool{} + + for _, s := range b.successors { + if s.shardID == "" { + return true + } + + index, count, err := sharding.ParseShardIDLabelValue(s.shardID) + // If we fail to parse shardID, we better not consider this block fully included in successors. + if err != nil { + return false + } + + if shardCount == 0 { + shardCount = count + } + shards[index] = true + } + + // If this condition is true, and all above checks passed, then this block is fully included in successors. + return uint64(len(shards)) == shardCount +} + +func (b *blockWithSuccessors) getDuplicateBlocks(output map[ulid.ULID]struct{}) { + if b.meta != nil && b.isFullyIncludedInSuccessors() { + output[b.meta.ULID] = struct{}{} + } + + for _, s := range b.successors { + s.getDuplicateBlocks(output) + } +} diff --git a/pkg/compactor/shard_aware_deduplicate_filter_test.go b/pkg/compactor/shard_aware_deduplicate_filter_test.go new file mode 100644 index 0000000000..ffb19ffaf1 --- /dev/null +++ b/pkg/compactor/shard_aware_deduplicate_filter_test.go @@ -0,0 +1,460 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/grafana/mimir/blob/main/pkg/compactor/shard_aware_deduplicate_filter_test.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Cortex Authors. + +package compactor + +import ( + "context" + "fmt" + "testing" + + "github.com/oklog/ulid" + "github.com/prometheus/client_golang/prometheus" + promtest "github.com/prometheus/client_golang/prometheus/testutil" + "github.com/prometheus/prometheus/tsdb" + "github.com/stretchr/testify/require" + + mimir_tsdb "github.com/grafana/mimir/pkg/storage/tsdb" + "github.com/grafana/mimir/pkg/storage/tsdb/block" + "github.com/grafana/mimir/pkg/util/extprom" +) + +func ULID(i int) ulid.ULID { return ulid.MustNew(uint64(i), nil) } + +type sourcesAndResolution struct { + sources []ulid.ULID + resolution int64 + shardID string +} + +func TestShardAwareDeduplicateFilter_Filter(t *testing.T) { + testcases := map[string]struct { + input map[ulid.ULID]sourcesAndResolution + expected []ulid.ULID // blocks in the output after duplicates are removed. + }{ + "3 non compacted blocks in bucket": { + input: map[ulid.ULID]sourcesAndResolution{ + ULID(1): {sources: []ulid.ULID{ULID(1)}}, + ULID(2): {sources: []ulid.ULID{ULID(2)}}, + ULID(3): {sources: []ulid.ULID{ULID(3)}}, + }, + expected: []ulid.ULID{ + ULID(1), + ULID(2), + ULID(3), + }, + }, + "compacted block without sources in bucket": { + input: map[ulid.ULID]sourcesAndResolution{ + ULID(6): {sources: []ulid.ULID{ULID(6)}}, + ULID(4): {sources: []ulid.ULID{ULID(1), ULID(3), ULID(2)}}, + ULID(5): {sources: []ulid.ULID{ULID(5)}}, + }, + expected: []ulid.ULID{ + ULID(4), + ULID(5), + ULID(6), + }, + }, + "two compacted blocks with same sources": { + input: map[ulid.ULID]sourcesAndResolution{ + ULID(3): {sources: []ulid.ULID{ULID(1), ULID(2)}}, + ULID(4): {sources: []ulid.ULID{ULID(1), ULID(2)}}, + ULID(5): {sources: []ulid.ULID{ULID(5)}}, + ULID(6): {sources: []ulid.ULID{ULID(6)}}, + }, + expected: []ulid.ULID{ + // ULID(4) is added after ULID(3), so ULID(4) becomes a "successor" of ULID(3), + // which makes ULID(3) to be considered a duplicate. + ULID(4), + ULID(5), + ULID(6), + }, + }, + "two compacted blocks with overlapping sources": { + input: map[ulid.ULID]sourcesAndResolution{ + ULID(4): {sources: []ulid.ULID{ULID(1), ULID(2)}}, + ULID(6): {sources: []ulid.ULID{ULID(6)}}, + ULID(5): {sources: []ulid.ULID{ULID(1), ULID(3), ULID(2)}}, + }, + expected: []ulid.ULID{ + ULID(5), + ULID(6), + }, + }, + "4 non compacted blocks and compacted block of level 2 in bucket": { + input: map[ulid.ULID]sourcesAndResolution{ + ULID(6): {sources: []ulid.ULID{ULID(6)}}, + ULID(1): {sources: []ulid.ULID{ULID(1)}}, + ULID(2): {sources: []ulid.ULID{ULID(2)}}, + ULID(3): {sources: []ulid.ULID{ULID(3)}}, + ULID(4): {sources: []ulid.ULID{ULID(2), ULID(1), ULID(3)}}, + }, + expected: []ulid.ULID{ + ULID(4), + ULID(6), + }, + }, + "3 compacted blocks of level 2 and one compacted block of level 3 in bucket": { + input: map[ulid.ULID]sourcesAndResolution{ + ULID(10): {sources: []ulid.ULID{ULID(1), ULID(2), ULID(3)}}, + ULID(11): {sources: []ulid.ULID{ULID(6), ULID(4), ULID(5)}}, + ULID(14): {sources: []ulid.ULID{ULID(14)}}, + ULID(1): {sources: []ulid.ULID{ULID(1)}}, + ULID(13): {sources: []ulid.ULID{ULID(1), ULID(6), ULID(2), ULID(3), ULID(5), ULID(7), ULID(4), ULID(8), ULID(9)}}, + ULID(12): {sources: []ulid.ULID{ULID(7), ULID(9), ULID(8)}}, + }, + expected: []ulid.ULID{ + ULID(14), + ULID(13), + }, + }, + "compacted blocks with overlapping sources": { + input: map[ulid.ULID]sourcesAndResolution{ + ULID(8): {sources: []ulid.ULID{ULID(1), ULID(3), ULID(2), ULID(4)}}, + ULID(1): {sources: []ulid.ULID{ULID(1)}}, + ULID(5): {sources: []ulid.ULID{ULID(1), ULID(2)}}, + ULID(6): {sources: []ulid.ULID{ULID(1), ULID(3), ULID(2), ULID(4)}}, + ULID(7): {sources: []ulid.ULID{ULID(3), ULID(1), ULID(2)}}, + }, + expected: []ulid.ULID{ + ULID(8), + }, + }, + "compacted blocks of level 3 with overlapping sources of equal length": { + input: map[ulid.ULID]sourcesAndResolution{ + ULID(10): {sources: []ulid.ULID{ULID(1), ULID(2), ULID(6), ULID(7)}}, + ULID(1): {sources: []ulid.ULID{ULID(1)}}, + ULID(11): {sources: []ulid.ULID{ULID(6), ULID(8), ULID(1), ULID(2)}}, + }, + expected: []ulid.ULID{ + ULID(10), + ULID(11), + }, + }, + "compacted blocks of level 3 with overlapping sources of different length": { + input: map[ulid.ULID]sourcesAndResolution{ + ULID(10): {sources: []ulid.ULID{ULID(6), ULID(7), ULID(1), ULID(2)}}, + ULID(1): {sources: []ulid.ULID{ULID(1)}}, + ULID(5): {sources: []ulid.ULID{ULID(1), ULID(2)}}, + ULID(11): {sources: []ulid.ULID{ULID(2), ULID(3), ULID(1)}}, + }, + expected: []ulid.ULID{ + ULID(10), + ULID(11), + }, + }, + "blocks with same sources and different resolutions": { + input: map[ulid.ULID]sourcesAndResolution{ + ULID(1): {sources: []ulid.ULID{ULID(1)}, resolution: 0}, + ULID(2): {sources: []ulid.ULID{ULID(1)}, resolution: 1000}, + ULID(3): {sources: []ulid.ULID{ULID(1)}, resolution: 10000}, + }, + expected: []ulid.ULID{ + ULID(1), + ULID(2), + ULID(3), + }, + }, + "compacted blocks with overlapping sources and different resolutions": { + input: map[ulid.ULID]sourcesAndResolution{ + ULID(1): {sources: []ulid.ULID{ULID(1)}, resolution: 0}, + ULID(6): {sources: []ulid.ULID{ULID(6)}, resolution: 10000}, + ULID(4): {sources: []ulid.ULID{ULID(1), ULID(3), ULID(2)}, resolution: 0}, + ULID(5): {sources: []ulid.ULID{ULID(2), ULID(3), ULID(1)}, resolution: 1000}, + }, + expected: []ulid.ULID{ + ULID(4), + ULID(5), + ULID(6), + }, + }, + "compacted blocks of level 3 with overlapping sources of different length and different resolutions": { + input: map[ulid.ULID]sourcesAndResolution{ + ULID(10): {sources: []ulid.ULID{ULID(7), ULID(5), ULID(1), ULID(2)}, resolution: 0}, + ULID(12): {sources: []ulid.ULID{ULID(6), ULID(7), ULID(1)}, resolution: 10000}, + ULID(1): {sources: []ulid.ULID{ULID(1)}, resolution: 0}, + ULID(13): {sources: []ulid.ULID{ULID(1)}, resolution: 10000}, + ULID(5): {sources: []ulid.ULID{ULID(1), ULID(2)}, resolution: 0}, + ULID(11): {sources: []ulid.ULID{ULID(2), ULID(3), ULID(1)}, resolution: 0}, + }, + expected: []ulid.ULID{ + ULID(10), + ULID(11), + ULID(12), + }, + }, + + // Blocks with ShardID + "two blocks merged and split, with single shard": { + input: map[ulid.ULID]sourcesAndResolution{ + ULID(1): {sources: []ulid.ULID{ULID(1)}}, + ULID(2): {sources: []ulid.ULID{ULID(2)}}, + ULID(3): {sources: []ulid.ULID{ULID(1), ULID(2)}, shardID: "1_of_1"}, + }, + expected: []ulid.ULID{ + ULID(3), + }, + }, + + "block with invalid shardID cannot 'include' its source blocks": { + input: map[ulid.ULID]sourcesAndResolution{ + ULID(1): {sources: []ulid.ULID{ULID(1)}}, + ULID(2): {sources: []ulid.ULID{ULID(2)}}, + ULID(3): {sources: []ulid.ULID{ULID(1), ULID(2)}, shardID: "invalid"}, + ULID(4): {sources: []ulid.ULID{ULID(1), ULID(2)}, shardID: "0_of_5"}, + ULID(5): {sources: []ulid.ULID{ULID(1), ULID(2)}, shardID: "3_of_2"}, + }, + // No blocks are removed as duplicates. + expected: []ulid.ULID{ + ULID(1), + ULID(2), + ULID(3), + ULID(4), + ULID(5), + }, + }, + + "when invalid shard IDs present, no deduplication happens for source blocks": { + input: map[ulid.ULID]sourcesAndResolution{ + ULID(1): {sources: []ulid.ULID{ULID(1)}}, + ULID(2): {sources: []ulid.ULID{ULID(2)}}, + // invalid + ULID(3): {sources: []ulid.ULID{ULID(1), ULID(2)}, shardID: "invalid"}, + ULID(4): {sources: []ulid.ULID{ULID(1), ULID(2)}, shardID: "0_of_5"}, + ULID(5): {sources: []ulid.ULID{ULID(1), ULID(2)}, shardID: "3_of_2"}, + // good shards + ULID(6): {sources: []ulid.ULID{ULID(1), ULID(2)}, shardID: "1_of_2"}, + ULID(7): {sources: []ulid.ULID{ULID(1), ULID(2)}, shardID: "2_of_2"}, + }, + // Presence of invalid shards means that even valid shards are not + expected: []ulid.ULID{ + ULID(1), + ULID(2), + ULID(3), + ULID(4), + ULID(5), + ULID(6), + ULID(7), + }, + }, + + "two blocks merged and split, with two shards": { + input: map[ulid.ULID]sourcesAndResolution{ + ULID(1): {sources: []ulid.ULID{ULID(1)}}, + ULID(2): {sources: []ulid.ULID{ULID(2)}}, + ULID(3): {sources: []ulid.ULID{ULID(1), ULID(2)}, shardID: "1_of_2"}, + ULID(4): {sources: []ulid.ULID{ULID(1), ULID(2)}, shardID: "2_of_2"}, + }, + expected: []ulid.ULID{ + ULID(3), + ULID(4), + }, + }, + + "two blocks merged and split into two, one shard missing": { + input: map[ulid.ULID]sourcesAndResolution{ + ULID(1): {sources: []ulid.ULID{ULID(1)}}, + ULID(2): {sources: []ulid.ULID{ULID(2)}}, + ULID(3): {sources: []ulid.ULID{ULID(1), ULID(2)}, shardID: "1_of_2"}, + }, + expected: []ulid.ULID{ + ULID(1), + ULID(2), + ULID(3), + }, + }, + "four base blocks merged and split into 2 separate shards": { + input: map[ulid.ULID]sourcesAndResolution{ + ULID(1): {sources: []ulid.ULID{ULID(1)}}, + ULID(2): {sources: []ulid.ULID{ULID(2)}}, + ULID(3): {sources: []ulid.ULID{ULID(3)}}, + ULID(4): {sources: []ulid.ULID{ULID(4)}}, + + // shards of 1+2 + ULID(5): {sources: []ulid.ULID{ULID(1), ULID(2)}, shardID: "1_of_2"}, + ULID(6): {sources: []ulid.ULID{ULID(1), ULID(2)}, shardID: "2_of_2"}, + + // shards of 3+4 + ULID(7): {sources: []ulid.ULID{ULID(3), ULID(4)}, shardID: "1_of_2"}, + ULID(8): {sources: []ulid.ULID{ULID(3), ULID(4)}, shardID: "2_of_2"}, + }, + expected: []ulid.ULID{ + ULID(5), + ULID(6), + ULID(7), + ULID(8), + }, + }, + + "four base blocks merged and split into 2 separate shards, plus another level": { + input: map[ulid.ULID]sourcesAndResolution{ + ULID(1): {sources: []ulid.ULID{ULID(1)}}, + ULID(2): {sources: []ulid.ULID{ULID(2)}}, + ULID(3): {sources: []ulid.ULID{ULID(3)}}, + ULID(4): {sources: []ulid.ULID{ULID(4)}}, + + // shards of 1+2 + ULID(5): {sources: []ulid.ULID{ULID(1), ULID(2)}, shardID: "1_of_2"}, + ULID(6): {sources: []ulid.ULID{ULID(1), ULID(2)}, shardID: "2_of_2"}, + + // shards of 3+4 + ULID(7): {sources: []ulid.ULID{ULID(3), ULID(4)}, shardID: "1_of_2"}, + ULID(8): {sources: []ulid.ULID{ULID(3), ULID(4)}, shardID: "2_of_2"}, + + // Two shards of 1+2+3+4 blocks. These "win". + ULID(9): {sources: []ulid.ULID{ULID(1), ULID(2), ULID(3), ULID(4)}, shardID: "1_of_2"}, + ULID(10): {sources: []ulid.ULID{ULID(1), ULID(2), ULID(3), ULID(4)}, shardID: "2_of_2"}, + }, + expected: []ulid.ULID{ + ULID(9), + ULID(10), + }, + }, + + "four base blocks merged and split into 2 separate shards, plus another level, with various resolutions": { + input: map[ulid.ULID]sourcesAndResolution{ + ULID(1): {sources: []ulid.ULID{ULID(1)}}, + ULID(2): {sources: []ulid.ULID{ULID(2)}}, + ULID(3): {sources: []ulid.ULID{ULID(3)}, resolution: 100}, + ULID(4): {sources: []ulid.ULID{ULID(4)}, resolution: 100}, + + // shards of 1+2 + ULID(5): {sources: []ulid.ULID{ULID(1), ULID(2)}, shardID: "1_of_2"}, + ULID(6): {sources: []ulid.ULID{ULID(1), ULID(2)}, shardID: "2_of_2"}, + + // shards of 3+4 + ULID(7): {sources: []ulid.ULID{ULID(3), ULID(4)}, shardID: "1_of_2", resolution: 100}, + ULID(8): {sources: []ulid.ULID{ULID(3), ULID(4)}, shardID: "2_of_2", resolution: 100}, + }, + expected: []ulid.ULID{ + ULID(5), + ULID(6), + ULID(7), + ULID(8), + }, + }, + } + + for name, tcase := range testcases { + t.Run(name, func(t *testing.T) { + f := NewShardAwareDeduplicateFilter() + m := newTestFetcherMetrics() + + metas := make(map[ulid.ULID]*block.Meta, len(tcase.input)) + + inputLen := len(tcase.input) + for id, metaInfo := range tcase.input { + metas[id] = &block.Meta{ + BlockMeta: tsdb.BlockMeta{ + ULID: id, + Compaction: tsdb.BlockMetaCompaction{ + Sources: metaInfo.sources, + }, + }, + Thanos: block.ThanosMeta{ + Downsample: block.ThanosDownsample{ + Resolution: metaInfo.resolution, + }, + Labels: map[string]string{ + mimir_tsdb.CompactorShardIDExternalLabel: metaInfo.shardID, + }, + }, + } + } + + expected := make(map[ulid.ULID]*block.Meta, len(tcase.expected)) + for _, id := range tcase.expected { + m := metas[id] + require.NotNil(t, m) + expected[id] = m + } + + require.NoError(t, f.Filter(context.Background(), metas, m.Synced)) + require.Equal(t, expected, metas) + require.Equal(t, float64(inputLen-len(tcase.expected)), promtest.ToFloat64(m.Synced.WithLabelValues(duplicateMeta))) + + for _, id := range f.duplicateIDs { + require.NotNil(t, tcase.input[id]) + require.Nil(t, metas[id]) + } + }) + } +} + +func newTestFetcherMetrics() *block.FetcherMetrics { + return &block.FetcherMetrics{ + Synced: extprom.NewTxGaugeVec(nil, prometheus.GaugeOpts{}, []string{"state"}), + } +} + +func BenchmarkDeduplicateFilter_Filter(b *testing.B) { + var ( + reg prometheus.Registerer + count uint64 + ) + + dedupFilter := NewShardAwareDeduplicateFilter() + synced := extprom.NewTxGaugeVec(reg, prometheus.GaugeOpts{}, []string{"state"}) + + for blocksNum := 10; blocksNum <= 10000; blocksNum *= 10 { + var cases []map[ulid.ULID]*block.Meta + // blocksNum number of blocks with all of them unique ULID and unique 100 sources. + cases = append(cases, make(map[ulid.ULID]*block.Meta, blocksNum)) + for i := 0; i < blocksNum; i++ { + + id := ulid.MustNew(count, nil) + count++ + + cases[0][id] = &block.Meta{ + BlockMeta: tsdb.BlockMeta{ + ULID: id, + }, + } + + for j := 0; j < 100; j++ { + cases[0][id].Compaction.Sources = append(cases[0][id].Compaction.Sources, ulid.MustNew(count, nil)) + count++ + } + } + + // Case for running 3x resolution as they can be run concurrently. + // blocksNum number of blocks. all of them with unique ULID and unique 100 cases. + cases = append(cases, make(map[ulid.ULID]*block.Meta, 3*blocksNum)) + + for i := 0; i < blocksNum; i++ { + for _, res := range []int64{0, 5 * 60 * 1000, 60 * 60 * 1000} { + + id := ulid.MustNew(count, nil) + count++ + cases[1][id] = &block.Meta{ + BlockMeta: tsdb.BlockMeta{ + ULID: id, + }, + Thanos: block.ThanosMeta{ + Downsample: block.ThanosDownsample{Resolution: res}, + }, + } + for j := 0; j < 100; j++ { + cases[1][id].Compaction.Sources = append(cases[1][id].Compaction.Sources, ulid.MustNew(count, nil)) + count++ + } + + } + } + + b.Run(fmt.Sprintf("Block-%d", blocksNum), func(b *testing.B) { + for _, tcase := range cases { + b.ResetTimer() + b.Run("", func(b *testing.B) { + for n := 0; n <= b.N; n++ { + _ = dedupFilter.Filter(context.Background(), tcase, synced) + require.Equal(b, 0, len(dedupFilter.DuplicateIDs())) + } + }) + } + }) + } +} diff --git a/pkg/compactor/split_merge_compactor.go b/pkg/compactor/split_merge_compactor.go new file mode 100644 index 0000000000..3e0f23872b --- /dev/null +++ b/pkg/compactor/split_merge_compactor.go @@ -0,0 +1,46 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/grafana/mimir/blob/main/pkg/compactor/split_merge_compactor.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Cortex Authors. +package compactor + +import ( + "context" + + "github.com/go-kit/log" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/prometheus/tsdb" +) + +func splitAndMergeGrouperFactory(_ context.Context, cfg Config, cfgProvider ConfigProvider, userID string, logger log.Logger, _ prometheus.Registerer) Grouper { + return NewSplitAndMergeGrouper( + userID, + cfg.BlockRanges.ToMilliseconds(), + uint32(cfgProvider.CompactorSplitAndMergeShards(userID)), + uint32(cfgProvider.CompactorSplitGroups(userID)), + logger) +} + +func splitAndMergeCompactorFactory(ctx context.Context, cfg Config, logger log.Logger, reg prometheus.Registerer) (Compactor, Planner, error) { + // We don't need to customise the TSDB compactor so we're just using the Prometheus one. + compactor, err := tsdb.NewLeveledCompactor(ctx, reg, logger, cfg.BlockRanges.ToMilliseconds(), nil, nil, true) + if err != nil { + return nil, nil, err + } + + opts := tsdb.DefaultLeveledCompactorConcurrencyOptions() + opts.MaxOpeningBlocks = cfg.MaxOpeningBlocksConcurrency + opts.MaxClosingBlocks = cfg.MaxClosingBlocksConcurrency + opts.SymbolsFlushersCount = cfg.SymbolsFlushersConcurrency + + compactor.SetConcurrencyOptions(opts) + + planner := NewSplitAndMergePlanner(cfg.BlockRanges.ToMilliseconds()) + return compactor, planner, nil +} + +// configureSplitAndMergeCompactor updates the provided configuration injecting the split-and-merge compactor. +func configureSplitAndMergeCompactor(cfg *Config) { + cfg.BlocksGrouperFactory = splitAndMergeGrouperFactory + cfg.BlocksCompactorFactory = splitAndMergeCompactorFactory +} diff --git a/pkg/compactor/split_merge_compactor_test.go b/pkg/compactor/split_merge_compactor_test.go new file mode 100644 index 0000000000..42a8f1c6ec --- /dev/null +++ b/pkg/compactor/split_merge_compactor_test.go @@ -0,0 +1,877 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/grafana/mimir/blob/main/pkg/compactor/split_merge_compactor_test.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Cortex Authors. + +package compactor + +import ( + "context" + "os" + "path/filepath" + "strconv" + "strings" + "testing" + "time" + + "github.com/go-kit/log" + "github.com/grafana/dskit/flagext" + "github.com/grafana/dskit/services" + "github.com/grafana/dskit/test" + "github.com/oklog/ulid" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/testutil" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/tsdb" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/thanos-io/objstore" + + "github.com/grafana/mimir/pkg/storage/bucket" + "github.com/grafana/mimir/pkg/storage/sharding" + mimir_tsdb "github.com/grafana/mimir/pkg/storage/tsdb" + "github.com/grafana/mimir/pkg/storage/tsdb/block" + util_test "github.com/grafana/mimir/pkg/util/test" +) + +func TestMultitenantCompactor_ShouldSupportSplitAndMergeCompactor(t *testing.T) { + const ( + userID = "user-1" + numSeries = 100 + blockRange = 2 * time.Hour + ) + + var ( + blockRangeMillis = blockRange.Milliseconds() + compactionRanges = mimir_tsdb.DurationList{blockRange, 2 * blockRange, 4 * blockRange} + ) + + externalLabels := func(shardID string) map[string]string { + labels := map[string]string{} + + if shardID != "" { + labels[mimir_tsdb.CompactorShardIDExternalLabel] = shardID + } + return labels + } + + externalLabelsWithTenantID := func(shardID string) map[string]string { + labels := externalLabels(shardID) + labels[mimir_tsdb.DeprecatedTenantIDExternalLabel] = userID + return labels + } + + tests := map[string]struct { + numShards int + setup func(t *testing.T, bkt objstore.Bucket) []block.Meta + }{ + "overlapping blocks matching the 1st compaction range should be merged and split": { + numShards: 2, + setup: func(t *testing.T, bkt objstore.Bucket) []block.Meta { + block1 := createTSDBBlock(t, bkt, userID, blockRangeMillis, 2*blockRangeMillis, numSeries, externalLabels("")) + block2 := createTSDBBlock(t, bkt, userID, blockRangeMillis, 2*blockRangeMillis, numSeries, externalLabels("")) + + return []block.Meta{ + { + BlockMeta: tsdb.BlockMeta{ + MinTime: 1 * blockRangeMillis, + MaxTime: 2 * blockRangeMillis, + Compaction: tsdb.BlockMetaCompaction{ + Sources: []ulid.ULID{block1, block2}, + }, + }, + Thanos: block.ThanosMeta{ + Labels: map[string]string{ + mimir_tsdb.CompactorShardIDExternalLabel: "1_of_2", + }, + }, + }, { + BlockMeta: tsdb.BlockMeta{ + MinTime: 1 * blockRangeMillis, + MaxTime: 2 * blockRangeMillis, + Compaction: tsdb.BlockMetaCompaction{ + Sources: []ulid.ULID{block1, block2}, + }, + }, + Thanos: block.ThanosMeta{ + Labels: map[string]string{ + mimir_tsdb.CompactorShardIDExternalLabel: "2_of_2", + }, + }, + }, + } + }, + }, + "overlapping blocks matching the 1st compaction range with mixed tenant ID labels should be merged and split": { + numShards: 2, + setup: func(t *testing.T, bkt objstore.Bucket) []block.Meta { + block1 := createTSDBBlock(t, bkt, userID, blockRangeMillis, 2*blockRangeMillis, numSeries, externalLabels("")) // Doesn't have __org_id__ label + block2 := createTSDBBlock(t, bkt, userID, blockRangeMillis, 2*blockRangeMillis, numSeries, externalLabelsWithTenantID("")) // Has __org_id__ label + + return []block.Meta{ + { + BlockMeta: tsdb.BlockMeta{ + MinTime: 1 * blockRangeMillis, + MaxTime: 2 * blockRangeMillis, + Compaction: tsdb.BlockMetaCompaction{ + Sources: []ulid.ULID{block1, block2}, + }, + }, + Thanos: block.ThanosMeta{ + Labels: map[string]string{ + mimir_tsdb.CompactorShardIDExternalLabel: "1_of_2", + }, + }, + }, { + BlockMeta: tsdb.BlockMeta{ + MinTime: 1 * blockRangeMillis, + MaxTime: 2 * blockRangeMillis, + Compaction: tsdb.BlockMetaCompaction{ + Sources: []ulid.ULID{block1, block2}, + }, + }, + Thanos: block.ThanosMeta{ + Labels: map[string]string{ + mimir_tsdb.CompactorShardIDExternalLabel: "2_of_2", + }, + }, + }, + } + }, + }, + "overlapping blocks matching the beginning of the 1st compaction range should be merged and split": { + numShards: 2, + setup: func(t *testing.T, bkt objstore.Bucket) []block.Meta { + block1 := createTSDBBlock(t, bkt, userID, 0, (5 * time.Minute).Milliseconds(), numSeries, externalLabels("")) + block2 := createTSDBBlock(t, bkt, userID, time.Minute.Milliseconds(), (7 * time.Minute).Milliseconds(), numSeries, externalLabels("")) + + // Add another block as "most recent one" otherwise the previous blocks are not compacted + // because the most recent blocks must cover the full range to be compacted. + block3 := createTSDBBlock(t, bkt, userID, blockRangeMillis, blockRangeMillis+time.Minute.Milliseconds(), numSeries, externalLabels("")) + + return []block.Meta{ + { + BlockMeta: tsdb.BlockMeta{ + MinTime: 0, + MaxTime: (7 * time.Minute).Milliseconds(), + Compaction: tsdb.BlockMetaCompaction{ + Sources: []ulid.ULID{block1, block2}, + }, + }, + Thanos: block.ThanosMeta{ + Labels: map[string]string{ + mimir_tsdb.CompactorShardIDExternalLabel: "1_of_2", + }, + }, + }, { + BlockMeta: tsdb.BlockMeta{ + MinTime: 0, + MaxTime: (7 * time.Minute).Milliseconds(), + Compaction: tsdb.BlockMetaCompaction{ + Sources: []ulid.ULID{block1, block2}, + }, + }, + Thanos: block.ThanosMeta{ + Labels: map[string]string{ + mimir_tsdb.CompactorShardIDExternalLabel: "2_of_2", + }, + }, + }, { + // Not compacted. + BlockMeta: tsdb.BlockMeta{ + MinTime: blockRangeMillis, + MaxTime: blockRangeMillis + time.Minute.Milliseconds(), + Compaction: tsdb.BlockMetaCompaction{ + Sources: []ulid.ULID{block3}, + }, + }, + Thanos: block.ThanosMeta{ + Labels: map[string]string{}, + }, + }, + } + }, + }, + "non-overlapping blocks matching the beginning of the 1st compaction range (without gaps) should be merged and split": { + numShards: 2, + setup: func(t *testing.T, bkt objstore.Bucket) []block.Meta { + block1 := createTSDBBlock(t, bkt, userID, 0, (5 * time.Minute).Milliseconds(), numSeries, externalLabels("")) + block2 := createTSDBBlock(t, bkt, userID, (5 * time.Minute).Milliseconds(), (10 * time.Minute).Milliseconds(), numSeries, externalLabels("")) + + // Add another block as "most recent one" otherwise the previous blocks are not compacted + // because the most recent blocks must cover the full range to be compacted. + block3 := createTSDBBlock(t, bkt, userID, blockRangeMillis, blockRangeMillis+time.Minute.Milliseconds(), numSeries, externalLabels("")) + + return []block.Meta{ + { + BlockMeta: tsdb.BlockMeta{ + MinTime: 0, + MaxTime: (10 * time.Minute).Milliseconds(), + Compaction: tsdb.BlockMetaCompaction{ + Sources: []ulid.ULID{block1, block2}, + }, + }, + Thanos: block.ThanosMeta{ + Labels: map[string]string{ + mimir_tsdb.CompactorShardIDExternalLabel: "1_of_2", + }, + }, + }, { + BlockMeta: tsdb.BlockMeta{ + MinTime: 0, + MaxTime: (10 * time.Minute).Milliseconds(), + Compaction: tsdb.BlockMetaCompaction{ + Sources: []ulid.ULID{block1, block2}, + }, + }, + Thanos: block.ThanosMeta{ + Labels: map[string]string{ + mimir_tsdb.CompactorShardIDExternalLabel: "2_of_2", + }, + }, + }, { + // Not compacted. + BlockMeta: tsdb.BlockMeta{ + MinTime: blockRangeMillis, + MaxTime: blockRangeMillis + time.Minute.Milliseconds(), + Compaction: tsdb.BlockMetaCompaction{ + Sources: []ulid.ULID{block3}, + }, + }, + Thanos: block.ThanosMeta{ + Labels: map[string]string{}, + }, + }, + } + }, + }, + "non-overlapping blocks matching the beginning of the 1st compaction range (with gaps) should be merged and split": { + numShards: 2, + setup: func(t *testing.T, bkt objstore.Bucket) []block.Meta { + block1 := createTSDBBlock(t, bkt, userID, 0, (5 * time.Minute).Milliseconds(), numSeries, externalLabels("")) + block2 := createTSDBBlock(t, bkt, userID, (7 * time.Minute).Milliseconds(), (10 * time.Minute).Milliseconds(), numSeries, externalLabels("")) + + // Add another block as "most recent one" otherwise the previous blocks are not compacted + // because the most recent blocks must cover the full range to be compacted. + block3 := createTSDBBlock(t, bkt, userID, blockRangeMillis, blockRangeMillis+time.Minute.Milliseconds(), numSeries, externalLabels("")) + + return []block.Meta{ + { + BlockMeta: tsdb.BlockMeta{ + MinTime: 0, + MaxTime: (10 * time.Minute).Milliseconds(), + Compaction: tsdb.BlockMetaCompaction{ + Sources: []ulid.ULID{block1, block2}, + }, + }, + Thanos: block.ThanosMeta{ + Labels: map[string]string{ + mimir_tsdb.CompactorShardIDExternalLabel: "1_of_2", + }, + }, + }, { + BlockMeta: tsdb.BlockMeta{ + MinTime: 0, + MaxTime: (10 * time.Minute).Milliseconds(), + Compaction: tsdb.BlockMetaCompaction{ + Sources: []ulid.ULID{block1, block2}, + }, + }, + Thanos: block.ThanosMeta{ + Labels: map[string]string{ + mimir_tsdb.CompactorShardIDExternalLabel: "2_of_2", + }, + }, + }, { + // Not compacted. + BlockMeta: tsdb.BlockMeta{ + MinTime: blockRangeMillis, + MaxTime: blockRangeMillis + time.Minute.Milliseconds(), + Compaction: tsdb.BlockMetaCompaction{ + Sources: []ulid.ULID{block3}, + }, + }, + Thanos: block.ThanosMeta{ + Labels: map[string]string{}, + }, + }, + } + }, + }, + "smaller compaction ranges should take precedence over larger ones, and then re-iterate in subsequent compactions of increasing ranges": { + numShards: 2, + setup: func(t *testing.T, bkt objstore.Bucket) []block.Meta { + // Two split blocks in the 1st compaction range. + block1a := createTSDBBlock(t, bkt, userID, 1, blockRangeMillis, numSeries, externalLabels("1_of_2")) + block1b := createTSDBBlock(t, bkt, userID, 1, blockRangeMillis, numSeries, externalLabels("2_of_2")) + + // Two non-split overlapping blocks in the 1st compaction range. + block2 := createTSDBBlock(t, bkt, userID, blockRangeMillis, 2*blockRangeMillis, numSeries, externalLabels("")) + block3 := createTSDBBlock(t, bkt, userID, blockRangeMillis, 2*blockRangeMillis, numSeries, externalLabels("")) + + // Two split adjacent blocks in the 2nd compaction range. + block4a := createTSDBBlock(t, bkt, userID, 2*blockRangeMillis, 3*blockRangeMillis, numSeries, externalLabels("1_of_2")) + block4b := createTSDBBlock(t, bkt, userID, 2*blockRangeMillis, 3*blockRangeMillis, numSeries, externalLabels("2_of_2")) + block5a := createTSDBBlock(t, bkt, userID, 3*blockRangeMillis, 4*blockRangeMillis, numSeries, externalLabels("1_of_2")) + block5b := createTSDBBlock(t, bkt, userID, 3*blockRangeMillis, 4*blockRangeMillis, numSeries, externalLabels("2_of_2")) + + // Two non-adjacent non-split blocks in the 1st compaction range. + block6 := createTSDBBlock(t, bkt, userID, 4*blockRangeMillis, 5*blockRangeMillis, numSeries, externalLabels("")) + block7 := createTSDBBlock(t, bkt, userID, 7*blockRangeMillis, 8*blockRangeMillis, numSeries, externalLabels("")) + + return []block.Meta{ + // The two overlapping blocks (block2, block3) have been merged and split in the 1st range, + // and then compacted with block1 in 2nd range. Finally, they've been compacted with + // block4 and block5 in the 3rd range compaction (total levels: 4). + { + BlockMeta: tsdb.BlockMeta{ + MinTime: 1, + MaxTime: 4 * blockRangeMillis, + Compaction: tsdb.BlockMetaCompaction{ + Sources: []ulid.ULID{block1a, block2, block3, block4a, block5a}, + }, + }, + Thanos: block.ThanosMeta{ + Labels: map[string]string{ + mimir_tsdb.CompactorShardIDExternalLabel: "1_of_2", + }, + }, + }, + { + BlockMeta: tsdb.BlockMeta{ + MinTime: 1, + MaxTime: 4 * blockRangeMillis, + Compaction: tsdb.BlockMetaCompaction{ + Sources: []ulid.ULID{block1b, block2, block3, block4b, block5b}, + }, + }, + Thanos: block.ThanosMeta{ + Labels: map[string]string{ + mimir_tsdb.CompactorShardIDExternalLabel: "2_of_2", + }, + }, + }, + // The two non-adjacent blocks block6 and block7 are split individually first and then merged + // together in the 3rd range. + { + BlockMeta: tsdb.BlockMeta{ + MinTime: 4 * blockRangeMillis, + MaxTime: 8 * blockRangeMillis, + Compaction: tsdb.BlockMetaCompaction{ + Sources: []ulid.ULID{block6, block7}, + }, + }, + Thanos: block.ThanosMeta{ + Labels: map[string]string{ + mimir_tsdb.CompactorShardIDExternalLabel: "1_of_2", + }, + }, + }, + { + BlockMeta: tsdb.BlockMeta{ + MinTime: 4 * blockRangeMillis, + MaxTime: 8 * blockRangeMillis, + Compaction: tsdb.BlockMetaCompaction{ + Sources: []ulid.ULID{block6, block7}, + }, + }, + Thanos: block.ThanosMeta{ + Labels: map[string]string{ + mimir_tsdb.CompactorShardIDExternalLabel: "2_of_2", + }, + }, + }, + } + }, + }, + "overlapping and non-overlapping blocks within the same range should be split and compacted together": { + numShards: 2, + setup: func(t *testing.T, bkt objstore.Bucket) []block.Meta { + // Overlapping. + block1 := createTSDBBlock(t, bkt, userID, 0, (5 * time.Minute).Milliseconds(), numSeries, externalLabels("")) + block2 := createTSDBBlock(t, bkt, userID, time.Minute.Milliseconds(), (7 * time.Minute).Milliseconds(), numSeries, externalLabels("")) + + // Not overlapping. + block3 := createTSDBBlock(t, bkt, userID, time.Hour.Milliseconds(), (2 * time.Hour).Milliseconds(), numSeries, externalLabels("")) + + return []block.Meta{ + { + BlockMeta: tsdb.BlockMeta{ + MinTime: 0, + MaxTime: (2 * time.Hour).Milliseconds(), + Compaction: tsdb.BlockMetaCompaction{ + Sources: []ulid.ULID{block1, block2, block3}, + }, + }, + Thanos: block.ThanosMeta{ + Labels: map[string]string{ + mimir_tsdb.CompactorShardIDExternalLabel: "1_of_2", + }, + }, + }, { + BlockMeta: tsdb.BlockMeta{ + MinTime: 0, + MaxTime: (2 * time.Hour).Milliseconds(), + Compaction: tsdb.BlockMetaCompaction{ + Sources: []ulid.ULID{block1, block2, block3}, + }, + }, + Thanos: block.ThanosMeta{ + Labels: map[string]string{ + mimir_tsdb.CompactorShardIDExternalLabel: "2_of_2", + }, + }, + }, + } + }, + }, + "overlapping and non-overlapping blocks within the same range and mixed tenant ID label should be split and compacted together": { + numShards: 2, + setup: func(t *testing.T, bkt objstore.Bucket) []block.Meta { + // Overlapping. + block1 := createTSDBBlock(t, bkt, userID, 0, (5 * time.Minute).Milliseconds(), numSeries, externalLabels("")) // Without __org_id__ label + block2 := createTSDBBlock(t, bkt, userID, time.Minute.Milliseconds(), (7 * time.Minute).Milliseconds(), numSeries, externalLabelsWithTenantID("")) // With __org_id__ label + + // Not overlapping. + block3 := createTSDBBlock(t, bkt, userID, time.Hour.Milliseconds(), (2 * time.Hour).Milliseconds(), numSeries, externalLabelsWithTenantID("")) // With __org_id__ label + + return []block.Meta{ + { + BlockMeta: tsdb.BlockMeta{ + MinTime: 0, + MaxTime: (2 * time.Hour).Milliseconds(), + Compaction: tsdb.BlockMetaCompaction{ + Sources: []ulid.ULID{block1, block2, block3}, + }, + }, + Thanos: block.ThanosMeta{ + Labels: map[string]string{ + mimir_tsdb.CompactorShardIDExternalLabel: "1_of_2", + }, + }, + }, { + BlockMeta: tsdb.BlockMeta{ + MinTime: 0, + MaxTime: (2 * time.Hour).Milliseconds(), + Compaction: tsdb.BlockMetaCompaction{ + Sources: []ulid.ULID{block1, block2, block3}, + }, + }, + Thanos: block.ThanosMeta{ + Labels: map[string]string{ + mimir_tsdb.CompactorShardIDExternalLabel: "2_of_2", + }, + }, + }, + } + }, + }, + "should correctly handle empty blocks generated in the splitting stage": { + numShards: 2, + setup: func(t *testing.T, bkt objstore.Bucket) []block.Meta { + // Generate a block with only 1 series. This block will be split into 1 split block only, + // because the source block only has 1 series. + block1 := createTSDBBlock(t, bkt, userID, blockRangeMillis, 2*blockRangeMillis, 1, externalLabels("")) + + return []block.Meta{ + { + BlockMeta: tsdb.BlockMeta{ + MinTime: (2 * blockRangeMillis) - 1, // Because there's only 1 sample with timestamp=maxT-1 + MaxTime: 2 * blockRangeMillis, + Compaction: tsdb.BlockMetaCompaction{ + Sources: []ulid.ULID{block1}, + }, + }, + Thanos: block.ThanosMeta{ + Labels: map[string]string{ + mimir_tsdb.CompactorShardIDExternalLabel: "1_of_2", + }, + }, + }, + } + }, + }, + "splitting should be disabled if configured shards = 0": { + numShards: 0, + setup: func(t *testing.T, bkt objstore.Bucket) []block.Meta { + block1 := createTSDBBlock(t, bkt, userID, 0, (5 * time.Minute).Milliseconds(), numSeries, externalLabels("")) + block2 := createTSDBBlock(t, bkt, userID, (5 * time.Minute).Milliseconds(), (10 * time.Minute).Milliseconds(), numSeries, externalLabels("")) + + // Add another block as "most recent one" otherwise the previous blocks are not compacted + // because the most recent blocks must cover the full range to be compacted. + block3 := createTSDBBlock(t, bkt, userID, blockRangeMillis, blockRangeMillis+time.Minute.Milliseconds(), numSeries, externalLabels("")) + + return []block.Meta{ + // Compacted but not split. + { + BlockMeta: tsdb.BlockMeta{ + MinTime: 0, + MaxTime: (10 * time.Minute).Milliseconds(), + Compaction: tsdb.BlockMetaCompaction{ + Sources: []ulid.ULID{block1, block2}, + }, + }, + Thanos: block.ThanosMeta{ + Labels: map[string]string{}, + }, + }, { + // Not compacted. + BlockMeta: tsdb.BlockMeta{ + MinTime: blockRangeMillis, + MaxTime: blockRangeMillis + time.Minute.Milliseconds(), + Compaction: tsdb.BlockMetaCompaction{ + Sources: []ulid.ULID{block3}, + }, + }, + Thanos: block.ThanosMeta{ + Labels: map[string]string{}, + }, + }, + } + }, + }, + "splitting should be disabled but already split blocks should be merged correctly (respecting the shard) if configured shards = 0": { + numShards: 0, + setup: func(t *testing.T, bkt objstore.Bucket) []block.Meta { + // Two split blocks in the 1st compaction range. + block1a := createTSDBBlock(t, bkt, userID, 1, blockRangeMillis, numSeries, externalLabels("1_of_2")) + block1b := createTSDBBlock(t, bkt, userID, 1, blockRangeMillis, numSeries, externalLabels("2_of_2")) + + // Two non-split overlapping blocks in the 1st compaction range. + block2 := createTSDBBlock(t, bkt, userID, blockRangeMillis, 2*blockRangeMillis, numSeries, externalLabels("")) + block3 := createTSDBBlock(t, bkt, userID, blockRangeMillis, 2*blockRangeMillis, numSeries, externalLabels("")) + + // Two split adjacent blocks in the 2nd compaction range. + block4a := createTSDBBlock(t, bkt, userID, 2*blockRangeMillis, 3*blockRangeMillis, numSeries, externalLabels("1_of_2")) + block4b := createTSDBBlock(t, bkt, userID, 2*blockRangeMillis, 3*blockRangeMillis, numSeries, externalLabels("2_of_2")) + block5a := createTSDBBlock(t, bkt, userID, 3*blockRangeMillis, 4*blockRangeMillis, numSeries, externalLabels("1_of_2")) + block5b := createTSDBBlock(t, bkt, userID, 3*blockRangeMillis, 4*blockRangeMillis, numSeries, externalLabels("2_of_2")) + + // Two non-adjacent non-split blocks in the 1st compaction range. + block6 := createTSDBBlock(t, bkt, userID, 4*blockRangeMillis, 5*blockRangeMillis, numSeries, externalLabels("")) + block7 := createTSDBBlock(t, bkt, userID, 7*blockRangeMillis, 8*blockRangeMillis, numSeries, externalLabels("")) + + return []block.Meta{ + // Block1 have been compacted with block4 and block5 in the 3rd range compaction. + { + BlockMeta: tsdb.BlockMeta{ + MinTime: 1, + MaxTime: 4 * blockRangeMillis, + Compaction: tsdb.BlockMetaCompaction{ + Sources: []ulid.ULID{block1a, block4a, block5a}, + }, + }, + Thanos: block.ThanosMeta{ + Labels: map[string]string{ + mimir_tsdb.CompactorShardIDExternalLabel: "1_of_2", + }, + }, + }, + { + BlockMeta: tsdb.BlockMeta{ + MinTime: 1, + MaxTime: 4 * blockRangeMillis, + Compaction: tsdb.BlockMetaCompaction{ + Sources: []ulid.ULID{block1b, block4b, block5b}, + }, + }, + Thanos: block.ThanosMeta{ + Labels: map[string]string{ + mimir_tsdb.CompactorShardIDExternalLabel: "2_of_2", + }, + }, + }, + // The two overlapping blocks (block2, block3) have been merged in the 1st range. + { + BlockMeta: tsdb.BlockMeta{ + MinTime: blockRangeMillis, + MaxTime: 2 * blockRangeMillis, + Compaction: tsdb.BlockMetaCompaction{ + Sources: []ulid.ULID{block2, block3}, + }, + }, + Thanos: block.ThanosMeta{ + Labels: map[string]string{}, + }, + }, + // The two non-adjacent blocks block6 and block7 are merged together in the 3rd range. + { + BlockMeta: tsdb.BlockMeta{ + MinTime: 4 * blockRangeMillis, + MaxTime: 8 * blockRangeMillis, + Compaction: tsdb.BlockMetaCompaction{ + Sources: []ulid.ULID{block6, block7}, + }, + }, + Thanos: block.ThanosMeta{ + Labels: map[string]string{}, + }, + }, + } + }, + }, + "compaction on blocks containing native histograms": { + numShards: 2, + setup: func(t *testing.T, bkt objstore.Bucket) []block.Meta { + minT := blockRangeMillis + maxT := 2 * blockRangeMillis + + seriesID := 0 + + appendHistograms := func(db *tsdb.DB) { + db.EnableNativeHistograms() + + appendHistogram := func(seriesID int, ts int64) { + lbls := labels.FromStrings("series_id", strconv.Itoa(seriesID)) + + app := db.Appender(context.Background()) + _, err := app.AppendHistogram(0, lbls, ts, util_test.GenerateTestHistogram(seriesID), nil) + require.NoError(t, err) + + err = app.Commit() + require.NoError(t, err) + } + + for ts := minT; ts < maxT; ts += (maxT - minT) / int64(numSeries-1) { + appendHistogram(seriesID, ts) + seriesID++ + } + + appendHistogram(seriesID, maxT-1) + } + + block1 := createCustomTSDBBlock(t, bkt, userID, externalLabels(""), appendHistograms) + block2 := createCustomTSDBBlock(t, bkt, userID, externalLabels(""), appendHistograms) + + return []block.Meta{ + { + BlockMeta: tsdb.BlockMeta{ + MinTime: 1 * blockRangeMillis, + MaxTime: 2 * blockRangeMillis, + Compaction: tsdb.BlockMetaCompaction{ + Sources: []ulid.ULID{block1, block2}, + }, + }, + Thanos: block.ThanosMeta{ + Labels: map[string]string{ + mimir_tsdb.CompactorShardIDExternalLabel: "1_of_2", + }, + }, + }, + { + BlockMeta: tsdb.BlockMeta{ + MinTime: 1 * blockRangeMillis, + MaxTime: 2 * blockRangeMillis, + Compaction: tsdb.BlockMetaCompaction{ + Sources: []ulid.ULID{block1, block2}, + }, + }, + Thanos: block.ThanosMeta{ + Labels: map[string]string{ + mimir_tsdb.CompactorShardIDExternalLabel: "2_of_2", + }, + }, + }, + } + }, + }, + } + + for testName, testData := range tests { + t.Run(testName, func(t *testing.T) { + workDir := t.TempDir() + storageDir := t.TempDir() + fetcherDir := t.TempDir() + + storageCfg := mimir_tsdb.BlocksStorageConfig{} + flagext.DefaultValues(&storageCfg) + storageCfg.Bucket.Backend = bucket.Filesystem + storageCfg.Bucket.Filesystem.Directory = storageDir + + compactorCfg := prepareConfig(t) + compactorCfg.DataDir = workDir + compactorCfg.BlockRanges = compactionRanges + + cfgProvider := newMockConfigProvider() + cfgProvider.splitAndMergeShards[userID] = testData.numShards + + logger := log.NewLogfmtLogger(os.Stdout) + reg := prometheus.NewPedanticRegistry() + ctx := context.Background() + + // Create TSDB blocks in the storage and get the expected blocks. + bucketClient, err := bucket.NewClient(ctx, storageCfg.Bucket, "test", logger, nil) + require.NoError(t, err) + expected := testData.setup(t, bucketClient) + + c, err := NewMultitenantCompactor(compactorCfg, storageCfg, cfgProvider, logger, reg) + require.NoError(t, err) + require.NoError(t, services.StartAndAwaitRunning(context.Background(), c)) + t.Cleanup(func() { + require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) + }) + + // Wait until the first compaction run completed. + test.Poll(t, 15*time.Second, nil, func() interface{} { + return testutil.GatherAndCompare(reg, strings.NewReader(` + # HELP cortex_compactor_runs_completed_total Total number of compaction runs successfully completed. + # TYPE cortex_compactor_runs_completed_total counter + cortex_compactor_runs_completed_total 1 + `), "cortex_compactor_runs_completed_total") + }) + + // List back any (non deleted) block from the storage. + userBucket := bucket.NewUserBucketClient(userID, bucketClient, nil) + fetcher, err := block.NewMetaFetcher(logger, + 1, + userBucket, + fetcherDir, + reg, + nil, + ) + require.NoError(t, err) + metas, partials, err := fetcher.FetchWithoutMarkedForDeletion(ctx) + require.NoError(t, err) + require.Empty(t, partials) + + // Sort blocks by MinTime and labels so that we get a stable comparison. + actual := sortMetasByMinTime(convertMetasMapToSlice(metas)) + + // Compare actual blocks with the expected ones. + require.Len(t, actual, len(expected)) + for i, e := range expected { + assert.Equal(t, e.MinTime, actual[i].MinTime) + assert.Equal(t, e.MaxTime, actual[i].MaxTime) + assert.Equal(t, e.Compaction.Sources, actual[i].Compaction.Sources) + assert.Equal(t, e.Thanos.Labels, actual[i].Thanos.Labels) + } + }) + } +} + +func TestMultitenantCompactor_ShouldGuaranteeSeriesShardingConsistencyOverTheTime(t *testing.T) { + const ( + userID = "user-1" + numSeries = 100 + blockRange = 2 * time.Hour + numShards = 2 + ) + + var ( + blockRangeMillis = blockRange.Milliseconds() + compactionRanges = mimir_tsdb.DurationList{blockRange} + + // You should NEVER CHANGE the expected series here, otherwise it means you're introducing + // a backward incompatible change. + expectedSeriesIDByShard = map[string][]int{ + "1_of_2": {0, 1, 3, 4, 5, 6, 7, 11, 12, 15, 16, 17, 18, 19, 20, 21, 24, 25, 27, 31, 36, 37, 38, 40, 42, 45, 47, 50, 51, 52, 53, 54, 55, 57, 59, 60, 61, 63, 68, 70, 71, 72, 74, 77, 79, 80, 81, 82, 83, 84, 85, 86, 88, 89, 90, 91, 92, 94, 98, 100}, + "2_of_2": {2, 8, 9, 10, 13, 14, 22, 23, 26, 28, 29, 30, 32, 33, 34, 35, 39, 41, 43, 44, 46, 48, 49, 56, 58, 62, 64, 65, 66, 67, 69, 73, 75, 76, 78, 87, 93, 95, 96, 97, 99}, + } + ) + + workDir := t.TempDir() + storageDir := t.TempDir() + fetcherDir := t.TempDir() + + storageCfg := mimir_tsdb.BlocksStorageConfig{} + flagext.DefaultValues(&storageCfg) + storageCfg.Bucket.Backend = bucket.Filesystem + storageCfg.Bucket.Filesystem.Directory = storageDir + + compactorCfg := prepareConfig(t) + compactorCfg.DataDir = workDir + compactorCfg.BlockRanges = compactionRanges + + cfgProvider := newMockConfigProvider() + cfgProvider.splitAndMergeShards[userID] = numShards + + logger := log.NewLogfmtLogger(os.Stdout) + reg := prometheus.NewPedanticRegistry() + ctx := context.Background() + + bucketClient, err := bucket.NewClient(ctx, storageCfg.Bucket, "test", logger, nil) + require.NoError(t, err) + + // Create a TSDB block in the storage. + blockID := createTSDBBlock(t, bucketClient, userID, blockRangeMillis, 2*blockRangeMillis, numSeries, nil) + + c, err := NewMultitenantCompactor(compactorCfg, storageCfg, cfgProvider, logger, reg) + require.NoError(t, err) + require.NoError(t, services.StartAndAwaitRunning(context.Background(), c)) + t.Cleanup(func() { + require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) + }) + + // Wait until the first compaction run completed. + test.Poll(t, 15*time.Second, nil, func() interface{} { + return testutil.GatherAndCompare(reg, strings.NewReader(` + # HELP cortex_compactor_runs_completed_total Total number of compaction runs successfully completed. + # TYPE cortex_compactor_runs_completed_total counter + cortex_compactor_runs_completed_total 1 + `), "cortex_compactor_runs_completed_total") + }) + + // List back any (non deleted) block from the storage. + userBucket := bucket.NewUserBucketClient(userID, bucketClient, nil) + fetcher, err := block.NewMetaFetcher(logger, + 1, + userBucket, + fetcherDir, + reg, + nil, + ) + require.NoError(t, err) + metas, partials, err := fetcher.FetchWithoutMarkedForDeletion(ctx) + require.NoError(t, err) + require.Empty(t, partials) + + // Sort blocks by MinTime and labels so that we get a stable comparison. + actualMetas := sortMetasByMinTime(convertMetasMapToSlice(metas)) + + // Ensure the input block has been split. + require.Len(t, actualMetas, numShards) + for idx, actualMeta := range actualMetas { + assert.Equal(t, blockRangeMillis, actualMeta.MinTime) + assert.Equal(t, 2*blockRangeMillis, actualMeta.MaxTime) + assert.Equal(t, []ulid.ULID{blockID}, actualMeta.Compaction.Sources) + assert.Equal(t, sharding.FormatShardIDLabelValue(uint64(idx), numShards), actualMeta.Thanos.Labels[mimir_tsdb.CompactorShardIDExternalLabel]) + } + + // Ensure each split block contains the right series, based on a series labels + // hashing function which doesn't change over time. + for _, actualMeta := range actualMetas { + expectedSeriesIDs := expectedSeriesIDByShard[actualMeta.Thanos.Labels[mimir_tsdb.CompactorShardIDExternalLabel]] + + b, err := tsdb.OpenBlock(logger, filepath.Join(storageDir, userID, actualMeta.ULID.String()), nil) + require.NoError(t, err) + + indexReader, err := b.Index() + require.NoError(t, err) + + // Find all series in the block. + postings, err := indexReader.PostingsForMatchers(false, labels.MustNewMatcher(labels.MatchRegexp, "series_id", ".+")) + require.NoError(t, err) + + builder := labels.NewScratchBuilder(1) + for postings.Next() { + // Symbolize the series labels. + require.NoError(t, indexReader.Series(postings.At(), &builder, nil)) + + // Ensure the series below to the right shard. + seriesLabels := builder.Labels() + seriesID, err := strconv.Atoi(seriesLabels.Get("series_id")) + require.NoError(t, err) + assert.Contains(t, expectedSeriesIDs, seriesID, "series:", seriesLabels.String()) + } + + require.NoError(t, postings.Err()) + } +} + +func convertMetasMapToSlice(metas map[ulid.ULID]*block.Meta) []*block.Meta { + var out []*block.Meta + for _, m := range metas { + out = append(out, m) + } + return out +} diff --git a/pkg/compactor/split_merge_grouper.go b/pkg/compactor/split_merge_grouper.go new file mode 100644 index 0000000000..89c616df12 --- /dev/null +++ b/pkg/compactor/split_merge_grouper.go @@ -0,0 +1,380 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/grafana/mimir/blob/main/pkg/compactor/split_merge_grouper.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Cortex Authors. + +package compactor + +import ( + "fmt" + "math" + "sort" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/oklog/ulid" + "github.com/pkg/errors" + "github.com/prometheus/prometheus/model/labels" + + "github.com/grafana/mimir/pkg/storage/sharding" + mimir_tsdb "github.com/grafana/mimir/pkg/storage/tsdb" + "github.com/grafana/mimir/pkg/storage/tsdb/block" +) + +type SplitAndMergeGrouper struct { + userID string + ranges []int64 + logger log.Logger + + // Number of shards to split source blocks into. + shardCount uint32 + + // Number of groups that blocks used for splitting are grouped into. + splitGroupsCount uint32 +} + +// NewSplitAndMergeGrouper makes a new SplitAndMergeGrouper. The provided ranges must be sorted. +// If shardCount is 0, the splitting stage is disabled. +func NewSplitAndMergeGrouper( + userID string, + ranges []int64, + shardCount uint32, + splitGroupsCount uint32, + logger log.Logger, +) *SplitAndMergeGrouper { + return &SplitAndMergeGrouper{ + userID: userID, + ranges: ranges, + shardCount: shardCount, + splitGroupsCount: splitGroupsCount, + logger: logger, + } +} + +func (g *SplitAndMergeGrouper) Groups(blocks map[ulid.ULID]*block.Meta) (res []*Job, err error) { + flatBlocks := make([]*block.Meta, 0, len(blocks)) + for _, b := range blocks { + flatBlocks = append(flatBlocks, b) + } + + for _, job := range planCompaction(g.userID, flatBlocks, g.ranges, g.shardCount, g.splitGroupsCount) { + // Sanity check: if splitting is disabled, we don't expect any job for the split stage. + if g.shardCount <= 0 && job.stage == stageSplit { + return nil, errors.Errorf("unexpected split stage job because splitting is disabled: %s", job.String()) + } + + // The group key is used by the compactor as a unique identifier of the compaction job. + // Its content is not important for the compactor, but uniqueness must be guaranteed. + groupKey := fmt.Sprintf("%s-%s-%s-%d-%d", + defaultGroupKeyWithoutShardID(job.blocks[0].Thanos), + job.stage, + job.shardID, + job.rangeStart, + job.rangeEnd) + + // All the blocks within the same group have the same downsample + // resolution and external labels. + resolution := job.blocks[0].Thanos.Downsample.Resolution + externalLabels := labels.FromMap(job.blocks[0].Thanos.Labels) + + compactionJob := NewJob( + g.userID, + groupKey, + externalLabels, + resolution, + job.stage == stageSplit, + g.shardCount, + job.shardingKey(), + ) + + for _, m := range job.blocks { + if err := compactionJob.AppendMeta(m); err != nil { + return nil, errors.Wrap(err, "add block to compaction group") + } + } + + res = append(res, compactionJob) + level.Debug(g.logger).Log("msg", "grouper found a compactable blocks group", "groupKey", groupKey, "job", job.String()) + } + + return res, nil +} + +// planCompaction analyzes the input blocks and returns a list of compaction jobs that can be +// run concurrently. Each returned job may belong either to this compactor instance or another one +// in the cluster, so the caller should check if they belong to their instance before running them. +func planCompaction(userID string, blocks []*block.Meta, ranges []int64, shardCount, splitGroups uint32) (jobs []*job) { + if len(blocks) == 0 || len(ranges) == 0 { + return nil + } + + // First of all we have to group blocks using the default grouping, but not + // considering the shard ID in the external labels (because will be checked later). + mainGroups := map[string][]*block.Meta{} + for _, b := range blocks { + key := defaultGroupKeyWithoutShardID(b.Thanos) + mainGroups[key] = append(mainGroups[key], b) + } + + for _, mainBlocks := range mainGroups { + // Sort blocks by min time. + sortMetasByMinTime(mainBlocks) + + for _, tr := range ranges { + nextJob: + for _, job := range planCompactionByRange(userID, mainBlocks, tr, tr == ranges[0], shardCount, splitGroups) { + // We can plan a job only if it doesn't conflict with other jobs already planned. + // Since we run the planning for each compaction range in increasing order, we guarantee + // that a job for the current time range is planned only if there's no other job for the + // same shard ID and an overlapping smaller time range. + for _, j := range jobs { + if job.conflicts(j) { + continue nextJob + } + } + + jobs = append(jobs, job) + } + } + } + + // Ensure we don't compact the most recent blocks prematurely when another one of + // the same size still fits in the range. To do it, we consider a job valid only + // if its range is before the most recent block or if it fully covers the range. + highestMaxTime := getMaxTime(blocks) + + for idx := 0; idx < len(jobs); { + job := jobs[idx] + + // If the job covers a range before the most recent block, it's fine. + if job.rangeEnd <= highestMaxTime { + idx++ + continue + } + + // If the job covers the full range, it's fine. + if job.maxTime()-job.minTime() == job.rangeLength() { + idx++ + continue + } + + // We have found a job which would compact recent blocks prematurely, + // so we need to filter it out. + jobs = append(jobs[:idx], jobs[idx+1:]...) + } + + // Jobs will be sorted later using configured job sorting algorithm. + // Here we sort them by sharding key, to keep the output stable for testing. + sort.SliceStable(jobs, func(i, j int) bool { + if iKey, jKey := jobs[i].shardingKey(), jobs[j].shardingKey(); iKey != jKey { + return iKey < jKey + } + + // The sharding key could be equal but external labels can still be different. + return defaultGroupKeyWithoutShardID(jobs[i].blocks[0].Thanos) < defaultGroupKeyWithoutShardID(jobs[j].blocks[0].Thanos) + }) + + return jobs +} + +// planCompactionByRange analyze the input blocks and returns a list of compaction jobs to +// compact blocks for the given compaction time range. Input blocks MUST be sorted by MinTime. +func planCompactionByRange(userID string, blocks []*block.Meta, tr int64, isSmallestRange bool, shardCount, splitGroups uint32) (jobs []*job) { + groups := groupBlocksByRange(blocks, tr) + + for _, group := range groups { + // If this is the smallest time range and there's any non-split block, + // then we should plan a job to split blocks. + if shardCount > 0 && isSmallestRange { + if splitJobs := planSplitting(userID, group, splitGroups); len(splitJobs) > 0 { + jobs = append(jobs, splitJobs...) + continue + } + } + + // If we reach this point, all blocks for this time range have already been split + // (or we're not processing the smallest time range, or splitting is disabled). + // Then, we can check if there's any group of blocks to be merged together for each shard. + for shardID, shardBlocks := range groupBlocksByShardID(group.blocks) { + // No merging to do if there are less than 2 blocks. + if len(shardBlocks) < 2 { + continue + } + + jobs = append(jobs, &job{ + userID: userID, + stage: stageMerge, + shardID: shardID, + blocksGroup: blocksGroup{ + rangeStart: group.rangeStart, + rangeEnd: group.rangeEnd, + blocks: shardBlocks, + }, + }) + } + } + + return jobs +} + +// planSplitting returns a job to split the blocks in the input group or nil if there's nothing to do because +// all blocks in the group have already been split. +func planSplitting(userID string, group blocksGroup, splitGroups uint32) []*job { + blocks := group.getNonShardedBlocks() + if len(blocks) == 0 { + return nil + } + + jobs := map[uint32]*job{} + + if splitGroups == 0 { + splitGroups = 1 + } + + // The number of source blocks could be very large so, to have a better horizontal scaling, we should group + // the source blocks into N groups (where N = number of shards) and create a job for each group of blocks to + // merge and split. + for _, block := range blocks { + splitGroup := mimir_tsdb.HashBlockID(block.ULID) % splitGroups + + if jobs[splitGroup] == nil { + jobs[splitGroup] = &job{ + userID: userID, + stage: stageSplit, + shardID: sharding.FormatShardIDLabelValue(uint64(splitGroup), uint64(splitGroups)), + blocksGroup: blocksGroup{ + rangeStart: group.rangeStart, + rangeEnd: group.rangeEnd, + }, + } + } + + jobs[splitGroup].blocks = append(jobs[splitGroup].blocks, block) + } + + // Convert the output. + out := make([]*job, 0, len(jobs)) + for _, job := range jobs { + out = append(out, job) + } + + return out +} + +// groupBlocksByShardID groups the blocks by shard ID (read from the block external labels). +// If a block doesn't have any shard ID in the external labels, it will be grouped with the +// shard ID set to an empty string. +func groupBlocksByShardID(blocks []*block.Meta) map[string][]*block.Meta { + groups := map[string][]*block.Meta{} + + for _, block := range blocks { + // If the label doesn't exist, we'll group together such blocks using an + // empty string as shard ID. + shardID := block.Thanos.Labels[mimir_tsdb.CompactorShardIDExternalLabel] + groups[shardID] = append(groups[shardID], block) + } + + return groups +} + +// groupBlocksByRange groups the blocks by the time range. The range sequence starts at 0. +// Input blocks MUST be sorted by MinTime. +// +// For example, if we have blocks [0-10, 10-20, 50-60, 90-100] and the split range tr is 30 +// it returns [0-10, 10-20], [50-60], [90-100]. +func groupBlocksByRange(blocks []*block.Meta, tr int64) []blocksGroup { + var ret []blocksGroup + + for i := 0; i < len(blocks); { + var ( + group blocksGroup + m = blocks[i] + ) + + group.rangeStart = getRangeStart(m, tr) + group.rangeEnd = group.rangeStart + tr + + // Skip blocks that don't fall into the range. This can happen via mis-alignment or + // by being the multiple of the intended range. + if m.MaxTime > group.rangeEnd { + i++ + continue + } + + // Add all blocks to the current group that are within [t0, t0+tr]. + for ; i < len(blocks); i++ { + // If the block does not start within this group, then we should break the iteration + // and move it to the next group. + if blocks[i].MinTime >= group.rangeEnd { + break + } + + // If the block doesn't fall into this group, but it started within this group then it + // means it spans across multiple ranges and we should skip it. + if blocks[i].MaxTime > group.rangeEnd { + continue + } + + group.blocks = append(group.blocks, blocks[i]) + } + + if len(group.blocks) > 0 { + ret = append(ret, group) + } + } + + return ret +} + +func getRangeStart(m *block.Meta, tr int64) int64 { + // Compute start of aligned time range of size tr closest to the current block's start. + // This code has been copied from TSDB. + if m.MinTime >= 0 { + return tr * (m.MinTime / tr) + } + return tr * ((m.MinTime - tr + 1) / tr) +} + +func sortMetasByMinTime(metas []*block.Meta) []*block.Meta { + sort.Slice(metas, func(i, j int) bool { + if metas[i].BlockMeta.MinTime != metas[j].BlockMeta.MinTime { + return metas[i].BlockMeta.MinTime < metas[j].BlockMeta.MinTime + } + + // Compare labels in case of same MinTime to get stable results. + return labels.Compare(labels.FromMap(metas[i].Thanos.Labels), labels.FromMap(metas[j].Thanos.Labels)) < 0 + }) + + return metas +} + +// getMaxTime returns the highest max time across all input blocks. +func getMaxTime(blocks []*block.Meta) int64 { + maxTime := int64(math.MinInt64) + + for _, block := range blocks { + if block.MaxTime > maxTime { + maxTime = block.MaxTime + } + } + + return maxTime +} + +// defaultGroupKeyWithoutShardID returns the default group key excluding ShardIDLabelName +// when computing it. +func defaultGroupKeyWithoutShardID(meta block.ThanosMeta) string { + return defaultGroupKey(meta.Downsample.Resolution, labelsWithoutShard(meta.Labels)) +} + +// Return labels built from base, but without any label with name equal to mimir_tsdb.CompactorShardIDExternalLabel. +func labelsWithoutShard(base map[string]string) labels.Labels { + b := labels.NewScratchBuilder(len(base)) + for k, v := range base { + if k != mimir_tsdb.CompactorShardIDExternalLabel { + b.Add(k, v) + } + } + b.Sort() + return b.Labels() +} diff --git a/pkg/compactor/split_merge_grouper_test.go b/pkg/compactor/split_merge_grouper_test.go new file mode 100644 index 0000000000..b1e25a5d1a --- /dev/null +++ b/pkg/compactor/split_merge_grouper_test.go @@ -0,0 +1,794 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/grafana/mimir/blob/main/pkg/compactor/split_merge_grouper_test.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Cortex Authors. + +package compactor + +import ( + "testing" + + "github.com/oklog/ulid" + "github.com/prometheus/prometheus/tsdb" + "github.com/stretchr/testify/assert" + + mimir_tsdb "github.com/grafana/mimir/pkg/storage/tsdb" + "github.com/grafana/mimir/pkg/storage/tsdb/block" +) + +func TestPlanCompaction(t *testing.T) { + const userID = "user-1" + + block1 := ulid.MustNew(1, nil) // Hash: 283204220 + block2 := ulid.MustNew(2, nil) // Hash: 444110359 + block3 := ulid.MustNew(3, nil) // Hash: 3253786510 + block4 := ulid.MustNew(4, nil) // Hash: 122298081 + block5 := ulid.MustNew(5, nil) // Hash: 2931974232 + block6 := ulid.MustNew(6, nil) // Hash: 3092880371 + block7 := ulid.MustNew(7, nil) // Hash: 1607589226 + block8 := ulid.MustNew(8, nil) // Hash: 2771068093 + block9 := ulid.MustNew(9, nil) // Hash: 1285776948 + block10 := ulid.MustNew(10, nil) // Hash: 1446683087 + + tests := map[string]struct { + ranges []int64 + shardCount uint32 + splitGroups uint32 + blocks []*block.Meta + expected []*job + }{ + "no input blocks": { + ranges: []int64{20}, + blocks: nil, + expected: nil, + }, + "should split a single block if == smallest compaction range": { + ranges: []int64{20, 40}, + shardCount: 1, + blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 0, MaxTime: 20}}, + }, + expected: []*job{ + {userID: userID, stage: stageSplit, shardID: "1_of_1", blocksGroup: blocksGroup{ + rangeStart: 0, + rangeEnd: 20, + blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 0, MaxTime: 20}}, + }, + }}, + }, + }, + "should split a single block if < smallest compaction range": { + ranges: []int64{20, 40}, + shardCount: 1, + blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 10, MaxTime: 20}}, + }, + expected: []*job{ + {userID: userID, stage: stageSplit, shardID: "1_of_1", blocksGroup: blocksGroup{ + rangeStart: 0, + rangeEnd: 20, + blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 10, MaxTime: 20}}, + }, + }}, + }, + }, + "should NOT split a single block if == smallest compaction range but configured shards = 0": { + ranges: []int64{20, 40}, + shardCount: 0, + blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 0, MaxTime: 20}}, + }, + expected: []*job{}, + }, + "should merge and split multiple 1st level blocks within the same time range": { + ranges: []int64{10, 20}, + shardCount: 1, + blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 10, MaxTime: 20}}, + {BlockMeta: tsdb.BlockMeta{ULID: block2, MinTime: 10, MaxTime: 20}}, + }, + expected: []*job{ + {userID: userID, stage: stageSplit, shardID: "1_of_1", blocksGroup: blocksGroup{ + rangeStart: 10, + rangeEnd: 20, + blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 10, MaxTime: 20}}, + {BlockMeta: tsdb.BlockMeta{ULID: block2, MinTime: 10, MaxTime: 20}}, + }, + }}, + }, + }, + "should merge and split multiple 1st level blocks in different time ranges": { + ranges: []int64{10, 20}, + shardCount: 1, + blocks: []*block.Meta{ + // 1st level range [0, 10] + {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 0, MaxTime: 10}}, + {BlockMeta: tsdb.BlockMeta{ULID: block2, MinTime: 0, MaxTime: 10}}, + // 1st level range [10, 20] + {BlockMeta: tsdb.BlockMeta{ULID: block3, MinTime: 10, MaxTime: 20}}, + {BlockMeta: tsdb.BlockMeta{ULID: block4, MinTime: 10, MaxTime: 20}}, + }, + expected: []*job{ + {userID: userID, stage: stageSplit, shardID: "1_of_1", blocksGroup: blocksGroup{ + rangeStart: 0, + rangeEnd: 10, + blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 0, MaxTime: 10}}, + {BlockMeta: tsdb.BlockMeta{ULID: block2, MinTime: 0, MaxTime: 10}}, + }, + }}, + {userID: userID, stage: stageSplit, shardID: "1_of_1", blocksGroup: blocksGroup{ + rangeStart: 10, + rangeEnd: 20, + blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block3, MinTime: 10, MaxTime: 20}}, + {BlockMeta: tsdb.BlockMeta{ULID: block4, MinTime: 10, MaxTime: 20}}, + }, + }}, + }, + }, + "should merge and split multiple 1st level blocks in different time ranges, single split group": { + ranges: []int64{10, 20}, + shardCount: 2, + splitGroups: 1, + blocks: []*block.Meta{ + // 1st level range [0, 10] + {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 0, MaxTime: 10}}, + {BlockMeta: tsdb.BlockMeta{ULID: block2, MinTime: 0, MaxTime: 10}}, + // 1st level range [10, 20] + {BlockMeta: tsdb.BlockMeta{ULID: block3, MinTime: 10, MaxTime: 20}}, + {BlockMeta: tsdb.BlockMeta{ULID: block4, MinTime: 10, MaxTime: 20}}, + }, + expected: []*job{ + {userID: userID, stage: stageSplit, shardID: "1_of_1", blocksGroup: blocksGroup{ + rangeStart: 0, + rangeEnd: 10, + blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 0, MaxTime: 10}}, + {BlockMeta: tsdb.BlockMeta{ULID: block2, MinTime: 0, MaxTime: 10}}, + }, + }}, + {userID: userID, stage: stageSplit, shardID: "1_of_1", blocksGroup: blocksGroup{ + rangeStart: 10, + rangeEnd: 20, + blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block3, MinTime: 10, MaxTime: 20}}, + {BlockMeta: tsdb.BlockMeta{ULID: block4, MinTime: 10, MaxTime: 20}}, + }, + }}, + }, + }, + "should merge and split multiple 1st level blocks in different time ranges, two split groups": { + ranges: []int64{10, 20}, + shardCount: 2, + splitGroups: 2, + blocks: []*block.Meta{ + // 1st level range [0, 10] + {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 0, MaxTime: 10}}, + {BlockMeta: tsdb.BlockMeta{ULID: block2, MinTime: 0, MaxTime: 10}}, + // 1st level range [10, 20] + {BlockMeta: tsdb.BlockMeta{ULID: block3, MinTime: 10, MaxTime: 20}}, + {BlockMeta: tsdb.BlockMeta{ULID: block4, MinTime: 10, MaxTime: 20}}, + }, + expected: []*job{ + {userID: userID, stage: stageSplit, shardID: "1_of_2", blocksGroup: blocksGroup{ + rangeStart: 0, + rangeEnd: 10, + blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 0, MaxTime: 10}}, + }, + }}, + {userID: userID, stage: stageSplit, shardID: "2_of_2", blocksGroup: blocksGroup{ + rangeStart: 0, + rangeEnd: 10, + blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block2, MinTime: 0, MaxTime: 10}}, + }, + }}, + {userID: userID, stage: stageSplit, shardID: "1_of_2", blocksGroup: blocksGroup{ + rangeStart: 10, + rangeEnd: 20, + blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block3, MinTime: 10, MaxTime: 20}}, + }, + }}, + {userID: userID, stage: stageSplit, shardID: "2_of_2", blocksGroup: blocksGroup{ + rangeStart: 10, + rangeEnd: 20, + blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block4, MinTime: 10, MaxTime: 20}}, + }, + }}, + }, + }, + "should merge but NOT split multiple 1st level blocks in different time ranges if configured shards = 0": { + ranges: []int64{10, 20}, + shardCount: 0, + blocks: []*block.Meta{ + // 1st level range [0, 10] + {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 0, MaxTime: 10}}, + {BlockMeta: tsdb.BlockMeta{ULID: block2, MinTime: 0, MaxTime: 10}}, + // 1st level range [10, 20] + {BlockMeta: tsdb.BlockMeta{ULID: block3, MinTime: 10, MaxTime: 20}}, + {BlockMeta: tsdb.BlockMeta{ULID: block4, MinTime: 10, MaxTime: 20}}, + }, + expected: []*job{ + {userID: userID, stage: stageMerge, blocksGroup: blocksGroup{ + rangeStart: 0, + rangeEnd: 10, + blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 0, MaxTime: 10}}, + {BlockMeta: tsdb.BlockMeta{ULID: block2, MinTime: 0, MaxTime: 10}}, + }, + }}, + {userID: userID, stage: stageMerge, blocksGroup: blocksGroup{ + rangeStart: 10, + rangeEnd: 20, + blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block3, MinTime: 10, MaxTime: 20}}, + {BlockMeta: tsdb.BlockMeta{ULID: block4, MinTime: 10, MaxTime: 20}}, + }, + }}, + }, + }, + "should merge split blocks that can be compacted on the 2nd range only": { + ranges: []int64{10, 20}, + shardCount: 2, + blocks: []*block.Meta{ + // 2nd level range [0, 20] + {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 0, MaxTime: 10}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_2"}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block2, MinTime: 10, MaxTime: 20}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_2"}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block3, MinTime: 0, MaxTime: 10}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "2_of_2"}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block4, MinTime: 10, MaxTime: 20}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "2_of_2"}}}, + // 2nd level range [20, 40] + {BlockMeta: tsdb.BlockMeta{ULID: block5, MinTime: 20, MaxTime: 30}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_2"}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block6, MinTime: 30, MaxTime: 40}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_2"}}}, + }, + expected: []*job{ + {userID: userID, stage: stageMerge, shardID: "1_of_2", blocksGroup: blocksGroup{ + rangeStart: 0, + rangeEnd: 20, + blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 0, MaxTime: 10}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_2"}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block2, MinTime: 10, MaxTime: 20}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_2"}}}, + }, + }}, + {userID: userID, stage: stageMerge, shardID: "2_of_2", blocksGroup: blocksGroup{ + rangeStart: 0, + rangeEnd: 20, + blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block3, MinTime: 0, MaxTime: 10}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "2_of_2"}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block4, MinTime: 10, MaxTime: 20}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "2_of_2"}}}, + }, + }}, + {userID: userID, stage: stageMerge, shardID: "1_of_2", blocksGroup: blocksGroup{ + rangeStart: 20, + rangeEnd: 40, + blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block5, MinTime: 20, MaxTime: 30}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_2"}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block6, MinTime: 30, MaxTime: 40}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_2"}}}, + }, + }}, + }, + }, + "should not split non-split blocks if they're > smallest compaction range (do not split historical blocks after enabling splitting)": { + ranges: []int64{10, 20}, + shardCount: 2, + blocks: []*block.Meta{ + // 2nd level range [0, 20] + {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 0, MaxTime: 10}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_2"}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block2, MinTime: 10, MaxTime: 20}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_2"}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block3, MinTime: 0, MaxTime: 10}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "2_of_2"}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block4, MinTime: 10, MaxTime: 20}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "2_of_2"}}}, + // 2nd level range [20, 40] + {BlockMeta: tsdb.BlockMeta{ULID: block5, MinTime: 20, MaxTime: 40}}, + {BlockMeta: tsdb.BlockMeta{ULID: block6, MinTime: 20, MaxTime: 40}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_2"}}}, + }, + expected: []*job{ + {userID: userID, stage: stageMerge, shardID: "1_of_2", blocksGroup: blocksGroup{ + rangeStart: 0, + rangeEnd: 20, + blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 0, MaxTime: 10}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_2"}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block2, MinTime: 10, MaxTime: 20}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_2"}}}, + }, + }}, + {userID: userID, stage: stageMerge, shardID: "2_of_2", blocksGroup: blocksGroup{ + rangeStart: 0, + rangeEnd: 20, + blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block3, MinTime: 0, MaxTime: 10}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "2_of_2"}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block4, MinTime: 10, MaxTime: 20}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "2_of_2"}}}, + }, + }}, + }, + }, + "input blocks can be compacted on a mix of 1st and 2nd ranges, guaranteeing no overlaps and giving preference to smaller ranges": { + ranges: []int64{10, 20}, + shardCount: 1, + blocks: []*block.Meta{ + // To be split on 1st level range [0, 10] + {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 0, MaxTime: 10}}, + {BlockMeta: tsdb.BlockMeta{ULID: block2, MinTime: 7, MaxTime: 10}}, + // Not compacted because on 2nd level because the range [0, 20] + // has other 1st level range groups to be split first + {BlockMeta: tsdb.BlockMeta{ULID: block10, MinTime: 0, MaxTime: 10}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block3, MinTime: 10, MaxTime: 20}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, + // To be compacted on 2nd level range [20, 40] + {BlockMeta: tsdb.BlockMeta{ULID: block4, MinTime: 20, MaxTime: 30}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block5, MinTime: 30, MaxTime: 40}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, + // Already compacted on 2nd level range [40, 60] + {BlockMeta: tsdb.BlockMeta{ULID: block6, MinTime: 40, MaxTime: 60}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, + // Not compacted on 2nd level because the range [60, 80] + // has other 1st level range groups to be compacted first + {BlockMeta: tsdb.BlockMeta{ULID: block7, MinTime: 60, MaxTime: 70}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, + // To be compacted on 1st level range [70, 80] + {BlockMeta: tsdb.BlockMeta{ULID: block8, MinTime: 70, MaxTime: 80}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block9, MinTime: 75, MaxTime: 80}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, + }, + expected: []*job{ + {userID: userID, stage: stageSplit, shardID: "1_of_1", blocksGroup: blocksGroup{ + rangeStart: 0, + rangeEnd: 10, + blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 0, MaxTime: 10}}, + {BlockMeta: tsdb.BlockMeta{ULID: block2, MinTime: 7, MaxTime: 10}}, + }, + }}, + {userID: userID, stage: stageMerge, shardID: "1_of_1", blocksGroup: blocksGroup{ + rangeStart: 70, + rangeEnd: 80, + blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block8, MinTime: 70, MaxTime: 80}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block9, MinTime: 75, MaxTime: 80}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, + }, + }}, + {userID: userID, stage: stageMerge, shardID: "1_of_1", blocksGroup: blocksGroup{ + rangeStart: 20, + rangeEnd: 40, + blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block4, MinTime: 20, MaxTime: 30}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block5, MinTime: 30, MaxTime: 40}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, + }, + }}, + }, + }, + "input blocks have already been compacted with the largest range": { + ranges: []int64{10, 20, 40}, + shardCount: 1, + blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 0, MaxTime: 40}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block2, MinTime: 40, MaxTime: 70}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block3, MinTime: 80, MaxTime: 120}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, + }, + expected: nil, + }, + "input blocks match the largest range but can be compacted because overlapping": { + ranges: []int64{10, 20, 40}, + shardCount: 1, + blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 0, MaxTime: 40}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block2, MinTime: 40, MaxTime: 70}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block3, MinTime: 80, MaxTime: 120}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block4, MinTime: 80, MaxTime: 120}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, + }, + expected: []*job{ + {userID: userID, stage: stageMerge, shardID: "1_of_1", blocksGroup: blocksGroup{ + rangeStart: 80, + rangeEnd: 120, + blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block3, MinTime: 80, MaxTime: 120}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block4, MinTime: 80, MaxTime: 120}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, + }, + }}, + }, + }, + "a block with time range crossing two 1st level ranges should be NOT considered for 1st level splitting": { + ranges: []int64{20, 40}, + shardCount: 1, + blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 10, MaxTime: 20}}, + {BlockMeta: tsdb.BlockMeta{ULID: block2, MinTime: 10, MaxTime: 30}}, // This block spans across two 1st level ranges. + {BlockMeta: tsdb.BlockMeta{ULID: block3, MinTime: 20, MaxTime: 30}}, + {BlockMeta: tsdb.BlockMeta{ULID: block4, MinTime: 30, MaxTime: 40}}, + }, + expected: []*job{ + {userID: userID, stage: stageSplit, shardID: "1_of_1", blocksGroup: blocksGroup{ + rangeStart: 0, + rangeEnd: 20, + blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 10, MaxTime: 20}}, + }, + }}, + {userID: userID, stage: stageSplit, shardID: "1_of_1", blocksGroup: blocksGroup{ + rangeStart: 20, + rangeEnd: 40, + blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block3, MinTime: 20, MaxTime: 30}}, + {BlockMeta: tsdb.BlockMeta{ULID: block4, MinTime: 30, MaxTime: 40}}, + }, + }}, + }, + }, + "a block with time range crossing two 1st level ranges should BE considered for 2nd level compaction": { + ranges: []int64{20, 40}, + shardCount: 1, + blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 0, MaxTime: 20}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block2, MinTime: 10, MaxTime: 30}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, // This block spans across two 1st level ranges. + {BlockMeta: tsdb.BlockMeta{ULID: block3, MinTime: 20, MaxTime: 40}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, + }, + expected: []*job{ + {userID: userID, stage: stageMerge, shardID: "1_of_1", blocksGroup: blocksGroup{ + rangeStart: 0, + rangeEnd: 40, + blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 0, MaxTime: 20}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block2, MinTime: 10, MaxTime: 30}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block3, MinTime: 20, MaxTime: 40}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, + }, + }}, + }, + }, + "a block with time range larger then the largest compaction range should NOT be considered for compaction": { + ranges: []int64{10, 20, 40}, + shardCount: 1, + blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 0, MaxTime: 40}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block2, MinTime: 30, MaxTime: 150}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, // This block is larger then the largest compaction range. + {BlockMeta: tsdb.BlockMeta{ULID: block3, MinTime: 40, MaxTime: 70}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block4, MinTime: 80, MaxTime: 120}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block5, MinTime: 80, MaxTime: 120}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, + }, + expected: []*job{ + {userID: userID, stage: stageMerge, shardID: "1_of_1", blocksGroup: blocksGroup{ + rangeStart: 80, + rangeEnd: 120, + blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block4, MinTime: 80, MaxTime: 120}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block5, MinTime: 80, MaxTime: 120}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, + }, + }}, + }, + }, + "a range containing the most recent block shouldn't be prematurely compacted if doesn't cover the full range": { + ranges: []int64{10, 20, 40}, + shardCount: 1, + blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{MinTime: 5, MaxTime: 8}}, + {BlockMeta: tsdb.BlockMeta{MinTime: 7, MaxTime: 9}}, + {BlockMeta: tsdb.BlockMeta{MinTime: 10, MaxTime: 12}}, + {BlockMeta: tsdb.BlockMeta{MinTime: 13, MaxTime: 15}}, + }, + expected: []*job{ + {userID: userID, stage: stageSplit, shardID: "1_of_1", blocksGroup: blocksGroup{ + rangeStart: 0, + rangeEnd: 10, + blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{MinTime: 5, MaxTime: 8}}, + {BlockMeta: tsdb.BlockMeta{MinTime: 7, MaxTime: 9}}, + }, + }}, + }, + }, + "should not merge blocks within the same time range but with different external labels": { + ranges: []int64{10, 20}, + shardCount: 1, + blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 10, MaxTime: 20}}, + {BlockMeta: tsdb.BlockMeta{ULID: block2, MinTime: 10, MaxTime: 20}, Thanos: block.ThanosMeta{Labels: map[string]string{"another_group": "a"}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block3, MinTime: 10, MaxTime: 20}, Thanos: block.ThanosMeta{Labels: map[string]string{"another_group": "a"}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block4, MinTime: 10, MaxTime: 20}, Thanos: block.ThanosMeta{Labels: map[string]string{"another_group": "b"}}}, + }, + expected: []*job{ + {userID: userID, stage: stageSplit, shardID: "1_of_1", blocksGroup: blocksGroup{ + rangeStart: 10, + rangeEnd: 20, + blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 10, MaxTime: 20}}, + }, + }}, + {userID: userID, stage: stageSplit, shardID: "1_of_1", blocksGroup: blocksGroup{ + rangeStart: 10, + rangeEnd: 20, + blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block2, MinTime: 10, MaxTime: 20}, Thanos: block.ThanosMeta{Labels: map[string]string{"another_group": "a"}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block3, MinTime: 10, MaxTime: 20}, Thanos: block.ThanosMeta{Labels: map[string]string{"another_group": "a"}}}, + }, + }}, + {userID: userID, stage: stageSplit, shardID: "1_of_1", blocksGroup: blocksGroup{ + rangeStart: 10, + rangeEnd: 20, + blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block4, MinTime: 10, MaxTime: 20}, Thanos: block.ThanosMeta{Labels: map[string]string{"another_group": "b"}}}, + }, + }}, + }, + }, + } + + for testName, testData := range tests { + t.Run(testName, func(t *testing.T) { + actual := planCompaction(userID, testData.blocks, testData.ranges, testData.shardCount, testData.splitGroups) + + // Print the actual jobs (useful for debugging if tests fail). + t.Logf("got %d jobs:", len(actual)) + for _, job := range actual { + t.Logf("- %s", job.String()) + } + + assert.ElementsMatch(t, testData.expected, actual) + }) + } +} + +func TestPlanSplitting(t *testing.T) { + const userID = "user-1" + + block1 := ulid.MustNew(1, nil) // Hash: 283204220 + block2 := ulid.MustNew(2, nil) // Hash: 444110359 + block3 := ulid.MustNew(3, nil) // Hash: 3253786510 + block4 := ulid.MustNew(4, nil) // Hash: 122298081 + block5 := ulid.MustNew(5, nil) // Hash: 2931974232 + + tests := map[string]struct { + blocks blocksGroup + splitGroups uint32 + expected []*job + }{ + "should return nil if the input group is empty": { + blocks: blocksGroup{}, + splitGroups: 2, + expected: nil, + }, + "should return nil if the input group contains no non-sharded blocks": { + blocks: blocksGroup{ + rangeStart: 10, + rangeEnd: 20, + blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block1}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_2"}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block2}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "2_of_2"}}}, + }, + }, + splitGroups: 2, + expected: nil, + }, + "should return a split job if the input group contains 1 non-sharded block": { + blocks: blocksGroup{ + rangeStart: 10, + rangeEnd: 20, + blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block1}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_2"}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block2}}, + }, + }, + splitGroups: 2, + expected: []*job{ + { + blocksGroup: blocksGroup{ + rangeStart: 10, + rangeEnd: 20, + blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block2}}, + }, + }, + userID: userID, + stage: stageSplit, + shardID: "2_of_2", + }, + }, + }, + "should splitGroups split jobs if the input group contains multiple non-sharded blocks": { + blocks: blocksGroup{ + rangeStart: 10, + rangeEnd: 20, + blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block1}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_2"}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block2}}, + {BlockMeta: tsdb.BlockMeta{ULID: block3}}, + {BlockMeta: tsdb.BlockMeta{ULID: block4}}, + {BlockMeta: tsdb.BlockMeta{ULID: block5}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_2"}}}, + }, + }, + splitGroups: 2, + expected: []*job{ + { + blocksGroup: blocksGroup{ + rangeStart: 10, + rangeEnd: 20, + blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block3}}, + }, + }, + userID: userID, + stage: stageSplit, + shardID: "1_of_2", + }, { + blocksGroup: blocksGroup{ + rangeStart: 10, + rangeEnd: 20, + blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block2}}, + {BlockMeta: tsdb.BlockMeta{ULID: block4}}, + }, + }, + userID: userID, + stage: stageSplit, + shardID: "2_of_2", + }, + }, + }, + } + + for testName, testData := range tests { + t.Run(testName, func(t *testing.T) { + assert.ElementsMatch(t, testData.expected, planSplitting(userID, testData.blocks, testData.splitGroups)) + }) + } +} + +func TestGroupBlocksByShardID(t *testing.T) { + block1 := ulid.MustNew(1, nil) + block2 := ulid.MustNew(2, nil) + block3 := ulid.MustNew(3, nil) + block4 := ulid.MustNew(4, nil) + + tests := map[string]struct { + blocks []*block.Meta + expected map[string][]*block.Meta + }{ + "no input blocks": { + blocks: nil, + expected: map[string][]*block.Meta{}, + }, + "only 1 block in input with shard ID": { + blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block1}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1"}}}, + }, + expected: map[string][]*block.Meta{ + "1": { + {BlockMeta: tsdb.BlockMeta{ULID: block1}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1"}}}, + }, + }, + }, + "only 1 block in input without shard ID": { + blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block1}}, + }, + expected: map[string][]*block.Meta{ + "": { + {BlockMeta: tsdb.BlockMeta{ULID: block1}}, + }, + }, + }, + "multiple blocks per shard ID": { + blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block1}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1"}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block2}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "2"}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block3}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1"}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block4}}, + }, + expected: map[string][]*block.Meta{ + "": { + {BlockMeta: tsdb.BlockMeta{ULID: block4}}, + }, + "1": { + {BlockMeta: tsdb.BlockMeta{ULID: block1}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1"}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block3}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1"}}}, + }, + "2": { + {BlockMeta: tsdb.BlockMeta{ULID: block2}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "2"}}}, + }, + }, + }, + } + + for testName, testData := range tests { + t.Run(testName, func(t *testing.T) { + assert.Equal(t, testData.expected, groupBlocksByShardID(testData.blocks)) + }) + } +} + +func TestGroupBlocksByRange(t *testing.T) { + tests := map[string]struct { + timeRange int64 + blocks []*block.Meta + expected []blocksGroup + }{ + "no input blocks": { + timeRange: 20, + blocks: nil, + expected: nil, + }, + "only 1 block in input": { + timeRange: 20, + blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{MinTime: 10, MaxTime: 20}}, + }, + expected: []blocksGroup{ + {rangeStart: 0, rangeEnd: 20, blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{MinTime: 10, MaxTime: 20}}, + }}, + }, + }, + "only 1 block per range": { + timeRange: 20, + blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{MinTime: 10, MaxTime: 15}}, + {BlockMeta: tsdb.BlockMeta{MinTime: 40, MaxTime: 60}}, + }, + expected: []blocksGroup{ + {rangeStart: 0, rangeEnd: 20, blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{MinTime: 10, MaxTime: 15}}, + }}, + {rangeStart: 40, rangeEnd: 60, blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{MinTime: 40, MaxTime: 60}}, + }}, + }, + }, + "multiple blocks per range": { + timeRange: 20, + blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{MinTime: 10, MaxTime: 15}}, + {BlockMeta: tsdb.BlockMeta{MinTime: 10, MaxTime: 20}}, + {BlockMeta: tsdb.BlockMeta{MinTime: 40, MaxTime: 60}}, + {BlockMeta: tsdb.BlockMeta{MinTime: 50, MaxTime: 55}}, + }, + expected: []blocksGroup{ + {rangeStart: 0, rangeEnd: 20, blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{MinTime: 10, MaxTime: 15}}, + {BlockMeta: tsdb.BlockMeta{MinTime: 10, MaxTime: 20}}, + }}, + {rangeStart: 40, rangeEnd: 60, blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{MinTime: 40, MaxTime: 60}}, + {BlockMeta: tsdb.BlockMeta{MinTime: 50, MaxTime: 55}}, + }}, + }, + }, + "a block with time range larger then the range should be excluded": { + timeRange: 20, + blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{MinTime: 0, MaxTime: 20}}, + {BlockMeta: tsdb.BlockMeta{MinTime: 0, MaxTime: 40}}, // This block is larger then the range. + {BlockMeta: tsdb.BlockMeta{MinTime: 10, MaxTime: 20}}, + {BlockMeta: tsdb.BlockMeta{MinTime: 20, MaxTime: 30}}, + }, + expected: []blocksGroup{ + {rangeStart: 0, rangeEnd: 20, blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{MinTime: 0, MaxTime: 20}}, + {BlockMeta: tsdb.BlockMeta{MinTime: 10, MaxTime: 20}}, + }}, + {rangeStart: 20, rangeEnd: 40, blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{MinTime: 20, MaxTime: 30}}, + }}, + }, + }, + "blocks with different time ranges but all fitting within the input range": { + timeRange: 40, + blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{MinTime: 0, MaxTime: 20}}, + {BlockMeta: tsdb.BlockMeta{MinTime: 0, MaxTime: 40}}, + {BlockMeta: tsdb.BlockMeta{MinTime: 10, MaxTime: 20}}, + {BlockMeta: tsdb.BlockMeta{MinTime: 20, MaxTime: 30}}, + }, + expected: []blocksGroup{ + {rangeStart: 0, rangeEnd: 40, blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{MinTime: 0, MaxTime: 20}}, + {BlockMeta: tsdb.BlockMeta{MinTime: 0, MaxTime: 40}}, + {BlockMeta: tsdb.BlockMeta{MinTime: 10, MaxTime: 20}}, + {BlockMeta: tsdb.BlockMeta{MinTime: 20, MaxTime: 30}}, + }}, + }, + }, + } + + for testName, testData := range tests { + t.Run(testName, func(t *testing.T) { + assert.Equal(t, testData.expected, groupBlocksByRange(testData.blocks, testData.timeRange)) + }) + } +} diff --git a/pkg/compactor/split_merge_job.go b/pkg/compactor/split_merge_job.go new file mode 100644 index 0000000000..21a7dcec22 --- /dev/null +++ b/pkg/compactor/split_merge_job.go @@ -0,0 +1,148 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/grafana/mimir/blob/main/pkg/compactor/split_merge_job.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Cortex Authors. + +package compactor + +import ( + "fmt" + "strings" + "time" + + "github.com/prometheus/prometheus/model/labels" + "golang.org/x/exp/slices" + + "github.com/grafana/mimir/pkg/storage/tsdb" + "github.com/grafana/mimir/pkg/storage/tsdb/block" +) + +type compactionStage string + +const ( + stageSplit compactionStage = "split" + stageMerge compactionStage = "merge" +) + +// job holds a compaction job planned by the split merge compactor. +type job struct { + userID string + + // Source blocks that should be compacted together when running this job. + blocksGroup + + // The compaction stage of this job. + stage compactionStage + + // The shard blocks in this job belong to. Its exact value depends on the stage: + // + // - split: identifier of the group of blocks that are going to be merged together + // when splitting their series into multiple output blocks. + // + // - merge: value of the ShardIDLabelName of all blocks in this job (all blocks in + // the job share the same label value). + shardID string +} + +func (j *job) shardingKey() string { + return fmt.Sprintf("%s-%s-%d-%d-%s", j.userID, j.stage, j.rangeStart, j.rangeEnd, j.shardID) +} + +// conflicts returns true if the two jobs cannot be planned at the same time. +func (j *job) conflicts(other *job) bool { + // Never conflict if related to different users or if time ranges don't overlap. + if j.userID != other.userID || !j.overlaps(other.blocksGroup) { + return false + } + + // Blocks with different downsample resolution or external labels (excluding the shard ID) + // are never merged together, so they can't conflict. Since all blocks within the same job are expected to have the same + // downsample resolution and external labels, we just check the 1st block of each job. + if len(j.blocks) > 0 && len(other.blocks) > 0 { + myLabels := labelsWithoutShard(j.blocksGroup.blocks[0].Thanos.Labels) + otherLabels := labelsWithoutShard(other.blocksGroup.blocks[0].Thanos.Labels) + if !labels.Equal(myLabels, otherLabels) { + return false + } + if j.blocksGroup.blocks[0].Thanos.Downsample != other.blocksGroup.blocks[0].Thanos.Downsample { + return false + } + } + + // We should merge after all splitting has been done, so two overlapping jobs + // for different stages shouldn't coexist. + if j.stage != other.stage { + return true + } + + // At this point we have two overlapping jobs for the same stage. They conflict if + // belonging to the same shard. + return j.shardID == other.shardID +} + +func (j *job) String() string { + blocks := make([]string, 0, len(j.blocks)) + for _, block := range j.blocks { + minT := time.Unix(0, block.MinTime*int64(time.Millisecond)).UTC() + maxT := time.Unix(0, block.MaxTime*int64(time.Millisecond)).UTC() + blocks = append(blocks, fmt.Sprintf("%s (min time: %s, max time: %s)", block.ULID.String(), minT.String(), maxT.String())) + } + + // Keep the output stable for tests. + slices.Sort(blocks) + + return fmt.Sprintf("stage: %s, range start: %d, range end: %d, shard: %s, blocks: %s", + j.stage, j.rangeStart, j.rangeEnd, j.shardID, strings.Join(blocks, ",")) +} + +// blocksGroup holds a group of blocks within the same time range. +type blocksGroup struct { + rangeStart int64 // Included. + rangeEnd int64 // Excluded. + blocks []*block.Meta // Sorted by MinTime. +} + +// overlaps returns whether the group range overlaps with the input group. +func (g blocksGroup) overlaps(other blocksGroup) bool { + if g.rangeStart >= other.rangeEnd || other.rangeStart >= g.rangeEnd { + return false + } + + return true +} + +func (g blocksGroup) rangeLength() int64 { + return g.rangeEnd - g.rangeStart +} + +// minTime returns the lowest MinTime across all blocks in the group. +func (g blocksGroup) minTime() int64 { + // Blocks are expected to be sorted by MinTime. + return g.blocks[0].MinTime +} + +// maxTime returns the highest MaxTime across all blocks in the group. +func (g blocksGroup) maxTime() int64 { + max := g.blocks[0].MaxTime + + for _, b := range g.blocks[1:] { + if b.MaxTime > max { + max = b.MaxTime + } + } + + return max +} + +// getNonShardedBlocks returns the list of non-sharded blocks. +func (g blocksGroup) getNonShardedBlocks() []*block.Meta { + var out []*block.Meta + + for _, b := range g.blocks { + if value, ok := b.Thanos.Labels[tsdb.CompactorShardIDExternalLabel]; !ok || value == "" { + out = append(out, b) + } + } + + return out +} diff --git a/pkg/compactor/split_merge_job_test.go b/pkg/compactor/split_merge_job_test.go new file mode 100644 index 0000000000..ac56919c02 --- /dev/null +++ b/pkg/compactor/split_merge_job_test.go @@ -0,0 +1,262 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/grafana/mimir/blob/main/pkg/compactor/split_merge_job_test.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Cortex Authors. + +package compactor + +import ( + "encoding/json" + "testing" + + "github.com/oklog/ulid" + "github.com/prometheus/prometheus/tsdb" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + + mimir_tsdb "github.com/grafana/mimir/pkg/storage/tsdb" + "github.com/grafana/mimir/pkg/storage/tsdb/block" +) + +func TestJob_conflicts(t *testing.T) { + block1 := &block.Meta{BlockMeta: tsdb.BlockMeta{ULID: ulid.MustNew(1, nil)}} + block2 := &block.Meta{BlockMeta: tsdb.BlockMeta{ULID: ulid.MustNew(2, nil)}} + block3 := &block.Meta{BlockMeta: tsdb.BlockMeta{ULID: ulid.MustNew(3, nil)}} + block4 := &block.Meta{BlockMeta: tsdb.BlockMeta{ULID: ulid.MustNew(4, nil)}} + + copyMeta := func(meta *block.Meta) *block.Meta { + encoded, err := json.Marshal(meta) + require.NoError(t, err) + + decoded := block.Meta{} + require.NoError(t, json.Unmarshal(encoded, &decoded)) + + return &decoded + } + + withShardIDLabel := func(meta *block.Meta, shardID string) *block.Meta { + meta = copyMeta(meta) + meta.Thanos.Labels = map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: shardID} + return meta + } + + tests := map[string]struct { + first *job + second *job + expected bool + }{ + "should conflict if jobs compact different blocks but with overlapping time ranges and same shard": { + first: &job{ + stage: stageMerge, + shardID: "1_of_2", + blocksGroup: blocksGroup{ + rangeStart: 10, + rangeEnd: 20, + blocks: []*block.Meta{withShardIDLabel(block1, "1_of_2"), withShardIDLabel(block2, "1_of_2")}, + }, + }, + second: &job{ + stage: stageMerge, + shardID: "1_of_2", + blocksGroup: blocksGroup{ + rangeStart: 15, + rangeEnd: 25, + blocks: []*block.Meta{withShardIDLabel(block3, "1_of_2"), withShardIDLabel(block4, "1_of_2")}, + }, + }, + expected: true, + }, + "should NOT conflict if jobs compact different blocks with non-overlapping time ranges and same shard": { + first: &job{ + stage: stageMerge, + shardID: "1_of_2", + blocksGroup: blocksGroup{ + rangeStart: 10, + rangeEnd: 20, + blocks: []*block.Meta{withShardIDLabel(block1, "1_of_2"), withShardIDLabel(block2, "1_of_2")}, + }, + }, + second: &job{ + stage: stageMerge, + shardID: "1_of_2", + blocksGroup: blocksGroup{ + rangeStart: 20, + rangeEnd: 30, + blocks: []*block.Meta{withShardIDLabel(block3, "1_of_2"), withShardIDLabel(block4, "1_of_2")}, + }, + }, + expected: false, + }, + "should NOT conflict if jobs compact same blocks with overlapping time ranges but different shard": { + first: &job{ + stage: stageMerge, + shardID: "1_of_2", + blocksGroup: blocksGroup{ + rangeStart: 10, + rangeEnd: 20, + blocks: []*block.Meta{withShardIDLabel(block1, "1_of_2"), withShardIDLabel(block2, "1_of_2")}, + }, + }, + second: &job{ + stage: stageMerge, + shardID: "2_of_2", + blocksGroup: blocksGroup{ + rangeStart: 10, + rangeEnd: 20, + blocks: []*block.Meta{withShardIDLabel(block1, "2_of_2"), withShardIDLabel(block2, "2_of_2")}, + }, + }, + expected: false, + }, + "should conflict if jobs compact same blocks with overlapping time ranges and different shard but at a different stage": { + first: &job{ + stage: stageSplit, + shardID: "1_of_2", + blocksGroup: blocksGroup{ + rangeStart: 10, + rangeEnd: 20, + blocks: []*block.Meta{withShardIDLabel(block1, "1_of_2"), withShardIDLabel(block2, "1_of_2")}, + }, + }, + second: &job{ + stage: stageMerge, + shardID: "2_of_2", + blocksGroup: blocksGroup{ + rangeStart: 10, + rangeEnd: 20, + blocks: []*block.Meta{withShardIDLabel(block1, "2_of_2"), withShardIDLabel(block2, "2_of_2")}, + }, + }, + expected: true, + }, + "should conflict between split and merge jobs with overlapping time ranges": { + first: &job{ + stage: stageSplit, + shardID: "", + blocksGroup: blocksGroup{ + rangeStart: 10, + rangeEnd: 20, + blocks: []*block.Meta{block1, block2}, + }, + }, + second: &job{ + stage: stageMerge, + shardID: "1_of_2", + blocksGroup: blocksGroup{ + rangeStart: 0, + rangeEnd: 40, + blocks: []*block.Meta{withShardIDLabel(block3, "1_of_2"), withShardIDLabel(block4, "1_of_2")}, + }, + }, + expected: true, + }, + "should NOT conflict between split and merge jobs with non-overlapping time ranges": { + first: &job{ + stage: stageSplit, + shardID: "", + blocksGroup: blocksGroup{ + rangeStart: 10, + rangeEnd: 20, + blocks: []*block.Meta{block1, block2}, + }, + }, + second: &job{ + stage: stageMerge, + shardID: "1_of_2", + blocksGroup: blocksGroup{ + rangeStart: 20, + rangeEnd: 40, + blocks: []*block.Meta{withShardIDLabel(block3, "1_of_2"), withShardIDLabel(block4, "1_of_2")}, + }, + }, + expected: false, + }, + } + + for testName, testCase := range tests { + t.Run(testName, func(t *testing.T) { + assert.Equal(t, testCase.expected, testCase.first.conflicts(testCase.second)) + assert.Equal(t, testCase.expected, testCase.second.conflicts(testCase.first)) + }) + } +} + +func TestBlocksGroup_overlaps(t *testing.T) { + tests := []struct { + first blocksGroup + second blocksGroup + expected bool + }{ + { + first: blocksGroup{rangeStart: 10, rangeEnd: 20}, + second: blocksGroup{rangeStart: 20, rangeEnd: 30}, + expected: false, + }, { + first: blocksGroup{rangeStart: 10, rangeEnd: 20}, + second: blocksGroup{rangeStart: 19, rangeEnd: 30}, + expected: true, + }, { + first: blocksGroup{rangeStart: 10, rangeEnd: 21}, + second: blocksGroup{rangeStart: 20, rangeEnd: 30}, + expected: true, + }, { + first: blocksGroup{rangeStart: 10, rangeEnd: 20}, + second: blocksGroup{rangeStart: 12, rangeEnd: 18}, + expected: true, + }, + } + + for _, tc := range tests { + assert.Equal(t, tc.expected, tc.first.overlaps(tc.second)) + assert.Equal(t, tc.expected, tc.second.overlaps(tc.first)) + } +} + +func TestBlocksGroup_getNonShardedBlocks(t *testing.T) { + block1 := ulid.MustNew(1, nil) + block2 := ulid.MustNew(2, nil) + block3 := ulid.MustNew(3, nil) + + tests := map[string]struct { + input blocksGroup + expected []*block.Meta + }{ + "should return nil if the group is empty": { + input: blocksGroup{}, + expected: nil, + }, + "should return nil if the group contains only sharded blocks": { + input: blocksGroup{blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block1}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1"}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block2}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1"}}}, + }}, + expected: nil, + }, + "should return the list of non-sharded blocks if exist in the group": { + input: blocksGroup{blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block1}}, + {BlockMeta: tsdb.BlockMeta{ULID: block2}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1"}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block3}, Thanos: block.ThanosMeta{Labels: map[string]string{"key": "value"}}}, + }}, + expected: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block1}}, + {BlockMeta: tsdb.BlockMeta{ULID: block3}, Thanos: block.ThanosMeta{Labels: map[string]string{"key": "value"}}}, + }, + }, + "should consider non-sharded a block with the shard ID label but empty value": { + input: blocksGroup{blocks: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block1}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: ""}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block2}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1"}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block3}, Thanos: block.ThanosMeta{Labels: map[string]string{"key": "value"}}}, + }}, + expected: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block1}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: ""}}}, + {BlockMeta: tsdb.BlockMeta{ULID: block3}, Thanos: block.ThanosMeta{Labels: map[string]string{"key": "value"}}}, + }, + }, + } + + for _, tc := range tests { + assert.Equal(t, tc.expected, tc.input.getNonShardedBlocks()) + } +} diff --git a/pkg/compactor/split_merge_planner.go b/pkg/compactor/split_merge_planner.go new file mode 100644 index 0000000000..eddc06ccc1 --- /dev/null +++ b/pkg/compactor/split_merge_planner.go @@ -0,0 +1,53 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/grafana/mimir/blob/main/pkg/compactor/split_merge_planner.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Cortex Authors. + +package compactor + +import ( + "context" + "fmt" + + "github.com/grafana/mimir/pkg/storage/tsdb/block" +) + +type SplitAndMergePlanner struct { + ranges []int64 +} + +func NewSplitAndMergePlanner(ranges []int64) *SplitAndMergePlanner { + return &SplitAndMergePlanner{ + ranges: ranges, + } +} + +// Plan implements compact.Planner. +func (c *SplitAndMergePlanner) Plan(_ context.Context, metasByMinTime []*block.Meta) ([]*block.Meta, error) { + // The split-and-merge grouper creates single groups of blocks that are expected to be + // compacted together, so there's no real planning to do here (reason why this function is + // just a pass-through). However, we want to run extra checks before proceeding. + if len(metasByMinTime) == 0 { + return metasByMinTime, nil + } + + // Ensure all blocks fits within the largest range. This is a double check + // to ensure there's no bug in the previous blocks grouping, given this Plan() + // is just a pass-through. + largestRange := c.ranges[len(c.ranges)-1] + rangeStart := getRangeStart(metasByMinTime[0], largestRange) + rangeEnd := rangeStart + largestRange + + for _, b := range metasByMinTime { + if b.MinTime < rangeStart || b.MaxTime > rangeEnd { + return nil, fmt.Errorf("block %s with time range %d:%d is outside the largest expected range %d:%d", + b.ULID.String(), + b.MinTime, + b.MaxTime, + rangeStart, + rangeEnd) + } + } + + return metasByMinTime, nil +} diff --git a/pkg/compactor/split_merge_planner_test.go b/pkg/compactor/split_merge_planner_test.go new file mode 100644 index 0000000000..bd577e8466 --- /dev/null +++ b/pkg/compactor/split_merge_planner_test.go @@ -0,0 +1,85 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/grafana/mimir/blob/main/pkg/compactor/split_merge_planner_test.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Cortex Authors. + +package compactor + +import ( + "context" + "fmt" + "testing" + + "github.com/oklog/ulid" + "github.com/prometheus/prometheus/tsdb" + "github.com/stretchr/testify/assert" + + "github.com/grafana/mimir/pkg/storage/tsdb/block" +) + +func TestSplitAndMergePlanner_Plan(t *testing.T) { + block1 := ulid.MustNew(1, nil) + block2 := ulid.MustNew(2, nil) + block3 := ulid.MustNew(3, nil) + + tests := map[string]struct { + ranges []int64 + blocksByMinTime []*block.Meta + expectedErr error + }{ + "no blocks": { + ranges: []int64{20, 40, 60}, + blocksByMinTime: []*block.Meta{}, + }, + "a source block is larger then the largest range": { + ranges: []int64{20, 40, 60}, + blocksByMinTime: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 0, MaxTime: 20, Version: block.TSDBVersion1}}, + {BlockMeta: tsdb.BlockMeta{ULID: block2, MinTime: 10, MaxTime: 80, Version: block.TSDBVersion1}}, + {BlockMeta: tsdb.BlockMeta{ULID: block3, MinTime: 12, MaxTime: 15, Version: block.TSDBVersion1}}, + }, + expectedErr: fmt.Errorf("block %s with time range 10:80 is outside the largest expected range 0:60", + block2.String()), + }, + "source blocks are smaller then the largest range but compacted block is larger": { + ranges: []int64{20, 40, 60}, + blocksByMinTime: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 10, MaxTime: 20, Version: block.TSDBVersion1}}, + {BlockMeta: tsdb.BlockMeta{ULID: block2, MinTime: 30, MaxTime: 40, Version: block.TSDBVersion1}}, + {BlockMeta: tsdb.BlockMeta{ULID: block3, MinTime: 50, MaxTime: 70, Version: block.TSDBVersion1}}, + }, + expectedErr: fmt.Errorf("block %s with time range 50:70 is outside the largest expected range 0:60", + block3.String()), + }, + "source blocks and compacted block are smaller then the largest range but misaligned": { + ranges: []int64{20, 40, 60}, + blocksByMinTime: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 50, MaxTime: 70, Version: block.TSDBVersion1}}, + {BlockMeta: tsdb.BlockMeta{ULID: block2, MinTime: 70, MaxTime: 80, Version: block.TSDBVersion1}}, + }, + expectedErr: fmt.Errorf("block %s with time range 50:70 is outside the largest expected range 0:60", + block1.String()), + }, + "blocks fit within the largest range": { + ranges: []int64{20, 40, 60}, + blocksByMinTime: []*block.Meta{ + {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 10, MaxTime: 20, Version: block.TSDBVersion1}}, + {BlockMeta: tsdb.BlockMeta{ULID: block2, MinTime: 20, MaxTime: 40, Version: block.TSDBVersion1}}, + {BlockMeta: tsdb.BlockMeta{ULID: block3, MinTime: 20, MaxTime: 60, Version: block.TSDBVersion1}}, + }, + }, + } + + for testName, testData := range tests { + t.Run(testName, func(t *testing.T) { + c := NewSplitAndMergePlanner(testData.ranges) + actual, err := c.Plan(context.Background(), testData.blocksByMinTime) + assert.Equal(t, testData.expectedErr, err) + + if testData.expectedErr == nil { + // Since the planner is a pass-through we do expect to get the same input blocks on success. + assert.Equal(t, testData.blocksByMinTime, actual) + } + }) + } +} diff --git a/pkg/compactor/status.gohtml b/pkg/compactor/status.gohtml new file mode 100644 index 0000000000..811e353f6d --- /dev/null +++ b/pkg/compactor/status.gohtml @@ -0,0 +1,11 @@ + + + + + Compactor Ring + + +

Compactor Ring

+

{{ .Message }}

+ + \ No newline at end of file diff --git a/pkg/compactor/syncer_metrics.go b/pkg/compactor/syncer_metrics.go new file mode 100644 index 0000000000..09e1593031 --- /dev/null +++ b/pkg/compactor/syncer_metrics.go @@ -0,0 +1,90 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/grafana/mimir/blob/main/pkg/compactor/syncer_metrics.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Cortex Authors. + +package compactor + +import ( + "github.com/go-kit/log/level" + dskit_metrics "github.com/grafana/dskit/metrics" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" + + util_log "github.com/grafana/mimir/pkg/util/log" +) + +// Copied from Thanos, pkg/compact/compact.go. +// Here we aggregate metrics from all finished syncers. +type aggregatedSyncerMetrics struct { + metaSync prometheus.Counter + metaSyncFailures prometheus.Counter + metaSyncDuration *dskit_metrics.HistogramDataCollector // was prometheus.Histogram before + garbageCollections prometheus.Counter + garbageCollectionFailures prometheus.Counter + garbageCollectionDuration *dskit_metrics.HistogramDataCollector // was prometheus.Histogram before +} + +// Copied (and modified with Mimir prefix) from Thanos, pkg/compact/compact.go +// We also ignore "group" label, since we only use a single group. +func newAggregatedSyncerMetrics(reg prometheus.Registerer) *aggregatedSyncerMetrics { + var m aggregatedSyncerMetrics + + m.metaSync = promauto.With(reg).NewCounter(prometheus.CounterOpts{ + Name: "cortex_compactor_meta_syncs_total", + Help: "Total blocks metadata synchronization attempts.", + }) + m.metaSyncFailures = promauto.With(reg).NewCounter(prometheus.CounterOpts{ + Name: "cortex_compactor_meta_sync_failures_total", + Help: "Total blocks metadata synchronization failures.", + }) + m.metaSyncDuration = dskit_metrics.NewHistogramDataCollector(prometheus.NewDesc( + "cortex_compactor_meta_sync_duration_seconds", + "Duration of the blocks metadata synchronization in seconds.", + nil, nil)) + + m.garbageCollections = promauto.With(reg).NewCounter(prometheus.CounterOpts{ + Name: "cortex_compactor_garbage_collection_total", + Help: "Total number of garbage collection operations.", + }) + m.garbageCollectionFailures = promauto.With(reg).NewCounter(prometheus.CounterOpts{ + Name: "cortex_compactor_garbage_collection_failures_total", + Help: "Total number of failed garbage collection operations.", + }) + m.garbageCollectionDuration = dskit_metrics.NewHistogramDataCollector(prometheus.NewDesc( + "cortex_compactor_garbage_collection_duration_seconds", + "Time it took to perform garbage collection iteration.", + nil, nil)) + + if reg != nil { + reg.MustRegister(m.metaSyncDuration, m.garbageCollectionDuration) + } + + return &m +} + +func (m *aggregatedSyncerMetrics) gatherThanosSyncerMetrics(reg *prometheus.Registry) { + if m == nil { + return + } + + mf, err := reg.Gather() + if err != nil { + level.Warn(util_log.Logger).Log("msg", "failed to gather metrics from syncer registry after compaction", "err", err) + return + } + + mfm, err := dskit_metrics.NewMetricFamilyMap(mf) + if err != nil { + level.Warn(util_log.Logger).Log("msg", "failed to gather metrics from syncer registry after compaction", "err", err) + return + } + + m.metaSync.Add(mfm.SumCounters("blocks_meta_syncs_total")) + m.metaSyncFailures.Add(mfm.SumCounters("blocks_meta_sync_failures_total")) + m.metaSyncDuration.Add(mfm.SumHistograms("blocks_meta_sync_duration_seconds")) + + m.garbageCollections.Add(mfm.SumCounters("thanos_compact_garbage_collection_total")) + m.garbageCollectionFailures.Add(mfm.SumCounters("thanos_compact_garbage_collection_failures_total")) + m.garbageCollectionDuration.Add(mfm.SumHistograms("thanos_compact_garbage_collection_duration_seconds")) +} diff --git a/pkg/compactor/syncer_metrics_test.go b/pkg/compactor/syncer_metrics_test.go new file mode 100644 index 0000000000..e6be00f9ef --- /dev/null +++ b/pkg/compactor/syncer_metrics_test.go @@ -0,0 +1,148 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/grafana/mimir/blob/main/pkg/compactor/syncer_metrics_test.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Cortex Authors. + +package compactor + +import ( + "bytes" + "testing" + + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" + "github.com/prometheus/client_golang/prometheus/testutil" + "github.com/stretchr/testify/require" +) + +func TestSyncerMetrics(t *testing.T) { + reg := prometheus.NewPedanticRegistry() + + sm := newAggregatedSyncerMetrics(reg) + sm.gatherThanosSyncerMetrics(generateTestData(12345)) + sm.gatherThanosSyncerMetrics(generateTestData(76543)) + sm.gatherThanosSyncerMetrics(generateTestData(22222)) + // total base = 111110 + + err := testutil.GatherAndCompare(reg, bytes.NewBufferString(` + # HELP cortex_compactor_meta_syncs_total Total blocks metadata synchronization attempts. + # TYPE cortex_compactor_meta_syncs_total counter + cortex_compactor_meta_syncs_total 111110 + + # HELP cortex_compactor_meta_sync_failures_total Total blocks metadata synchronization failures. + # TYPE cortex_compactor_meta_sync_failures_total counter + cortex_compactor_meta_sync_failures_total 222220 + + # HELP cortex_compactor_meta_sync_duration_seconds Duration of the blocks metadata synchronization in seconds. + # TYPE cortex_compactor_meta_sync_duration_seconds histogram + # Observed values: 3.7035, 22.9629, 6.6666 (seconds) + cortex_compactor_meta_sync_duration_seconds_bucket{le="0.01"} 0 + cortex_compactor_meta_sync_duration_seconds_bucket{le="0.1"} 0 + cortex_compactor_meta_sync_duration_seconds_bucket{le="0.3"} 0 + cortex_compactor_meta_sync_duration_seconds_bucket{le="0.6"} 0 + cortex_compactor_meta_sync_duration_seconds_bucket{le="1"} 0 + cortex_compactor_meta_sync_duration_seconds_bucket{le="3"} 0 + cortex_compactor_meta_sync_duration_seconds_bucket{le="6"} 1 + cortex_compactor_meta_sync_duration_seconds_bucket{le="9"} 2 + cortex_compactor_meta_sync_duration_seconds_bucket{le="20"} 2 + cortex_compactor_meta_sync_duration_seconds_bucket{le="30"} 3 + cortex_compactor_meta_sync_duration_seconds_bucket{le="60"} 3 + cortex_compactor_meta_sync_duration_seconds_bucket{le="90"} 3 + cortex_compactor_meta_sync_duration_seconds_bucket{le="120"} 3 + cortex_compactor_meta_sync_duration_seconds_bucket{le="240"} 3 + cortex_compactor_meta_sync_duration_seconds_bucket{le="360"} 3 + cortex_compactor_meta_sync_duration_seconds_bucket{le="720"} 3 + cortex_compactor_meta_sync_duration_seconds_bucket{le="+Inf"} 3 + # rounding error + cortex_compactor_meta_sync_duration_seconds_sum 33.333000000000006 + cortex_compactor_meta_sync_duration_seconds_count 3 + + # HELP cortex_compactor_garbage_collection_total Total number of garbage collection operations. + # TYPE cortex_compactor_garbage_collection_total counter + cortex_compactor_garbage_collection_total 555550 + + # HELP cortex_compactor_garbage_collection_failures_total Total number of failed garbage collection operations. + # TYPE cortex_compactor_garbage_collection_failures_total counter + cortex_compactor_garbage_collection_failures_total 666660 + + # HELP cortex_compactor_garbage_collection_duration_seconds Time it took to perform garbage collection iteration. + # TYPE cortex_compactor_garbage_collection_duration_seconds histogram + # Observed values: 8.6415, 53.5801, 15.5554 + cortex_compactor_garbage_collection_duration_seconds_bucket{le="0.01"} 0 + cortex_compactor_garbage_collection_duration_seconds_bucket{le="0.1"} 0 + cortex_compactor_garbage_collection_duration_seconds_bucket{le="0.3"} 0 + cortex_compactor_garbage_collection_duration_seconds_bucket{le="0.6"} 0 + cortex_compactor_garbage_collection_duration_seconds_bucket{le="1"} 0 + cortex_compactor_garbage_collection_duration_seconds_bucket{le="3"} 0 + cortex_compactor_garbage_collection_duration_seconds_bucket{le="6"} 0 + cortex_compactor_garbage_collection_duration_seconds_bucket{le="9"} 1 + cortex_compactor_garbage_collection_duration_seconds_bucket{le="20"} 2 + cortex_compactor_garbage_collection_duration_seconds_bucket{le="30"} 2 + cortex_compactor_garbage_collection_duration_seconds_bucket{le="60"} 3 + cortex_compactor_garbage_collection_duration_seconds_bucket{le="90"} 3 + cortex_compactor_garbage_collection_duration_seconds_bucket{le="120"} 3 + cortex_compactor_garbage_collection_duration_seconds_bucket{le="240"} 3 + cortex_compactor_garbage_collection_duration_seconds_bucket{le="360"} 3 + cortex_compactor_garbage_collection_duration_seconds_bucket{le="720"} 3 + cortex_compactor_garbage_collection_duration_seconds_bucket{le="+Inf"} 3 + cortex_compactor_garbage_collection_duration_seconds_sum 77.777 + cortex_compactor_garbage_collection_duration_seconds_count 3 + `)) + require.NoError(t, err) +} + +func generateTestData(base float64) *prometheus.Registry { + r := prometheus.NewRegistry() + m := newTestSyncerMetrics(r) + m.metaSync.Add(1 * base) + m.metaSyncFailures.Add(2 * base) + m.metaSyncDuration.Observe(3 * base / 10000) + m.garbageCollections.Add(5 * base) + m.garbageCollectionFailures.Add(6 * base) + m.garbageCollectionDuration.Observe(7 * base / 10000) + return r +} + +// directly copied from Thanos (and renamed syncerMetrics to testSyncerMetrics to avoid conflict) +type testSyncerMetrics struct { + metaSync prometheus.Counter + metaSyncFailures prometheus.Counter + metaSyncDuration prometheus.Histogram + garbageCollections prometheus.Counter + garbageCollectionFailures prometheus.Counter + garbageCollectionDuration prometheus.Histogram +} + +func newTestSyncerMetrics(reg prometheus.Registerer) *testSyncerMetrics { + var m testSyncerMetrics + + m.metaSync = promauto.With(reg).NewCounter(prometheus.CounterOpts{ + Name: "blocks_meta_syncs_total", + Help: "Total blocks metadata synchronization attempts.", + }) + m.metaSyncFailures = promauto.With(reg).NewCounter(prometheus.CounterOpts{ + Name: "blocks_meta_sync_failures_total", + Help: "Total blocks metadata synchronization failures.", + }) + m.metaSyncDuration = promauto.With(reg).NewHistogram(prometheus.HistogramOpts{ + Name: "blocks_meta_sync_duration_seconds", + Help: "Duration of the blocks metadata synchronization in seconds.", + Buckets: []float64{0.01, 0.1, 0.3, 0.6, 1, 3, 6, 9, 20, 30, 60, 90, 120, 240, 360, 720}, + }) + + m.garbageCollections = promauto.With(reg).NewCounter(prometheus.CounterOpts{ + Name: "thanos_compact_garbage_collection_total", + Help: "Total number of garbage collection operations.", + }) + m.garbageCollectionFailures = promauto.With(reg).NewCounter(prometheus.CounterOpts{ + Name: "thanos_compact_garbage_collection_failures_total", + Help: "Total number of failed garbage collection operations.", + }) + m.garbageCollectionDuration = promauto.With(reg).NewHistogram(prometheus.HistogramOpts{ + Name: "thanos_compact_garbage_collection_duration_seconds", + Help: "Time it took to perform garbage collection iteration.", + Buckets: []float64{0.01, 0.1, 0.3, 0.6, 1, 3, 6, 9, 20, 30, 60, 90, 120, 240, 360, 720}, + }) + + return &m +} diff --git a/pkg/compactor/tenant_deletion_api.go b/pkg/compactor/tenant_deletion_api.go new file mode 100644 index 0000000000..c79630a02f --- /dev/null +++ b/pkg/compactor/tenant_deletion_api.go @@ -0,0 +1,98 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/grafana/mimir/blob/main/pkg/compactor/tenant_deletion_api.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Cortex Authors. + +package compactor + +import ( + "context" + "net/http" + "strings" + "time" + + "github.com/go-kit/log/level" + "github.com/grafana/dskit/tenant" + "github.com/oklog/ulid" + "github.com/pkg/errors" + + "github.com/grafana/mimir/pkg/storage/bucket" + mimir_tsdb "github.com/grafana/mimir/pkg/storage/tsdb" + "github.com/grafana/mimir/pkg/util" +) + +func (c *MultitenantCompactor) DeleteTenant(w http.ResponseWriter, r *http.Request) { + ctx := r.Context() + userID, err := tenant.TenantID(ctx) + if err != nil { + // When Mimir is running, it uses Auth Middleware for checking X-Scope-OrgID and injecting tenant into context. + // Auth Middleware sends http.StatusUnauthorized if X-Scope-OrgID is missing, so we do too here, for consistency. + http.Error(w, err.Error(), http.StatusUnauthorized) + return + } + + err = mimir_tsdb.WriteTenantDeletionMark(r.Context(), c.bucketClient, userID, c.cfgProvider, mimir_tsdb.NewTenantDeletionMark(time.Now())) + if err != nil { + level.Error(c.logger).Log("msg", "failed to write tenant deletion mark", "user", userID, "err", err) + + http.Error(w, err.Error(), http.StatusInternalServerError) + return + } + + level.Info(c.logger).Log("msg", "tenant deletion mark in blocks storage created", "user", userID) + + w.WriteHeader(http.StatusOK) +} + +type DeleteTenantStatusResponse struct { + TenantID string `json:"tenant_id"` + BlocksDeleted bool `json:"blocks_deleted"` +} + +func (c *MultitenantCompactor) DeleteTenantStatus(w http.ResponseWriter, r *http.Request) { + ctx := r.Context() + userID, err := tenant.TenantID(ctx) + if err != nil { + http.Error(w, err.Error(), http.StatusBadRequest) + return + } + + result := DeleteTenantStatusResponse{} + result.TenantID = userID + result.BlocksDeleted, err = c.isBlocksForUserDeleted(ctx, userID) + if err != nil { + http.Error(w, err.Error(), http.StatusInternalServerError) + return + } + + util.WriteJSONResponse(w, result) +} + +func (c *MultitenantCompactor) isBlocksForUserDeleted(ctx context.Context, userID string) (bool, error) { + errBlockFound := errors.New("block found") + + userBucket := bucket.NewUserBucketClient(userID, c.bucketClient, c.cfgProvider) + err := userBucket.Iter(ctx, "", func(s string) error { + s = strings.TrimSuffix(s, "/") + + _, err := ulid.Parse(s) + if err != nil { + // not block, keep looking + return nil + } + + // Used as shortcut to stop iteration. + return errBlockFound + }) + + if errors.Is(err, errBlockFound) { + return false, nil + } + + if err != nil { + return false, err + } + + // No blocks found, all good. + return true, nil +} diff --git a/pkg/compactor/tenant_deletion_api_test.go b/pkg/compactor/tenant_deletion_api_test.go new file mode 100644 index 0000000000..09e146b825 --- /dev/null +++ b/pkg/compactor/tenant_deletion_api_test.go @@ -0,0 +1,100 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/grafana/mimir/blob/main/pkg/compactor/tenant_deletion_api_test.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Cortex Authors. +package compactor + +import ( + "bytes" + "context" + "net/http" + "net/http/httptest" + "path" + "testing" + + "github.com/grafana/dskit/services" + "github.com/grafana/dskit/user" + "github.com/stretchr/testify/require" + "github.com/thanos-io/objstore" + + "github.com/grafana/mimir/pkg/storage/tsdb" +) + +func TestDeleteTenant(t *testing.T) { + bkt := objstore.NewInMemBucket() + cfg := prepareConfig(t) + c, _, _, _, _ := prepare(t, cfg, bkt) + require.NoError(t, services.StartAndAwaitRunning(context.Background(), c)) + t.Cleanup(stopServiceFn(t, c)) + + { + resp := httptest.NewRecorder() + c.DeleteTenant(resp, &http.Request{}) + require.Equal(t, http.StatusUnauthorized, resp.Code) + } + + { + ctx := context.Background() + ctx = user.InjectOrgID(ctx, "fake") + + req := &http.Request{} + resp := httptest.NewRecorder() + c.DeleteTenant(resp, req.WithContext(ctx)) + + require.Equal(t, http.StatusOK, resp.Code) + objs := bkt.Objects() + require.NotNil(t, objs[path.Join("fake", tsdb.TenantDeletionMarkPath)]) + } +} + +func TestDeleteTenantStatus(t *testing.T) { + const username = "user" + + for name, tc := range map[string]struct { + objects map[string][]byte + expectedBlocksDeleted bool + }{ + "empty": { + objects: nil, + expectedBlocksDeleted: true, + }, + + "no user objects": { + objects: map[string][]byte{ + "different-user/01EQK4QKFHVSZYVJ908Y7HH9E0/meta.json": []byte("data"), + }, + expectedBlocksDeleted: true, + }, + + "non-block files": { + objects: map[string][]byte{ + "user/deletion-mark.json": []byte("data"), + }, + expectedBlocksDeleted: true, + }, + + "block files": { + objects: map[string][]byte{ + "user/01EQK4QKFHVSZYVJ908Y7HH9E0/meta.json": []byte("data"), + }, + expectedBlocksDeleted: false, + }, + } { + t.Run(name, func(t *testing.T) { + bkt := objstore.NewInMemBucket() + // "upload" objects + for objName, data := range tc.objects { + require.NoError(t, bkt.Upload(context.Background(), objName, bytes.NewReader(data))) + } + + cfg := prepareConfig(t) + c, _, _, _, _ := prepare(t, cfg, bkt) + require.NoError(t, services.StartAndAwaitRunning(context.Background(), c)) + t.Cleanup(stopServiceFn(t, c)) + + res, err := c.isBlocksForUserDeleted(context.Background(), username) + require.NoError(t, err) + require.Equal(t, tc.expectedBlocksDeleted, res) + }) + } +} From f297930a9eafedc76c238f740dcf737cf279eb9c Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Fri, 8 Sep 2023 09:49:36 +0200 Subject: [PATCH 11/74] Fixing more code --- pkg/compactor/block_upload.go | 1722 ++++----- pkg/compactor/block_upload_test.go | 4226 ++++++++++----------- pkg/compactor/job.go | 14 +- pkg/compactor/label_remover_filter.go | 4 +- pkg/compactor/tenant_deletion_api.go | 182 +- pkg/compactor/tenant_deletion_api_test.go | 160 +- 6 files changed, 3154 insertions(+), 3154 deletions(-) diff --git a/pkg/compactor/block_upload.go b/pkg/compactor/block_upload.go index 2cfdb1a7c6..d288a5ad4d 100644 --- a/pkg/compactor/block_upload.go +++ b/pkg/compactor/block_upload.go @@ -1,864 +1,864 @@ -// SPDX-License-Identifier: AGPL-3.0-only -// Provenance-includes-location: https://github.com/grafana/mimir/blob/main/pkg/compactor/block_upload.go -// Provenance-includes-license: Apache-2.0 -// Provenance-includes-copyright: The Cortex Authors. +// // SPDX-License-Identifier: AGPL-3.0-only +// // Provenance-includes-location: https://github.com/grafana/mimir/blob/main/pkg/compactor/block_upload.go +// // Provenance-includes-license: Apache-2.0 +// // Provenance-includes-copyright: The Cortex Authors. package compactor -import ( - "bytes" - "context" - "encoding/binary" - "encoding/hex" - "encoding/json" - "fmt" - "io" - "net/http" - "os" - "path" - "path/filepath" - "sync" - "time" - - "github.com/go-kit/log" - "github.com/go-kit/log/level" - "github.com/gorilla/mux" - "github.com/grafana/dskit/tenant" - "github.com/grafana/regexp" - "github.com/oklog/ulid" - "github.com/pkg/errors" - "github.com/thanos-io/objstore" - - "github.com/grafana/mimir/pkg/storage/bucket" - "github.com/grafana/mimir/pkg/storage/sharding" - mimir_tsdb "github.com/grafana/mimir/pkg/storage/tsdb" - "github.com/grafana/mimir/pkg/storage/tsdb/block" - "github.com/grafana/mimir/pkg/util" - util_log "github.com/grafana/mimir/pkg/util/log" -) - -const ( - uploadingMetaFilename = "uploading-meta.json" // Name of the file that stores a block's meta file while it's being uploaded - validationFilename = "validation.json" // Name of the file that stores a heartbeat time and possibly an error message - validationHeartbeatInterval = 1 * time.Minute // Duration of time between heartbeats of an in-progress block upload validation - validationHeartbeatTimeout = 5 * time.Minute // Maximum duration of time to wait until a validation is able to be restarted - maximumMetaSizeBytes = 1 * 1024 * 1024 // 1 MiB, maximum allowed size of an uploaded block's meta.json file -) - -var ( - maxBlockUploadSizeBytesFormat = "block exceeds the maximum block size limit of %d bytes" - rePath = regexp.MustCompile(`^(index|chunks/\d{6})$`) -) - -// StartBlockUpload handles request for starting block upload. -// -// Starting the uploading of a block means to upload a meta file and verify that the upload can -// go ahead. In practice this means to check that the (complete) block isn't already in block -// storage, and that the meta file is valid. -func (c *MultitenantCompactor) StartBlockUpload(w http.ResponseWriter, r *http.Request) { - blockID, tenantID, err := c.parseBlockUploadParameters(r) - if err != nil { - http.Error(w, err.Error(), http.StatusBadRequest) - return - } - - ctx := r.Context() - requestID := hexTimeNowNano() - logger := log.With( - util_log.WithContext(ctx, c.logger), - "feature", "block upload", - "block", blockID, - "operation", "start block upload", - "request_id", requestID, - ) - - userBkt := bucket.NewUserBucketClient(tenantID, c.bucketClient, c.cfgProvider) - if _, _, err := c.checkBlockState(ctx, userBkt, blockID, false); err != nil { - writeBlockUploadError(err, "can't check block state", logger, w, requestID) - return - } - - content, err := io.ReadAll(http.MaxBytesReader(w, r.Body, maximumMetaSizeBytes)) - if err != nil { - if errors.As(err, new(*http.MaxBytesError)) { - err = httpError{ - message: fmt.Sprintf("The block metadata was too large (maximum size allowed is %d bytes)", maximumMetaSizeBytes), - statusCode: http.StatusRequestEntityTooLarge, - } - } - writeBlockUploadError(err, "failed reading body", logger, w, requestID) - return - } - - var meta block.Meta - if err := json.Unmarshal(content, &meta); err != nil { - err = httpError{ - message: "malformed request body", - statusCode: http.StatusBadRequest, - } - writeBlockUploadError(err, "failed unmarshaling block meta json", logger, w, requestID) - return - } - - if err := c.createBlockUpload(ctx, &meta, logger, userBkt, tenantID, blockID); err != nil { - writeBlockUploadError(err, "failed creating block upload", logger, w, requestID) - return - } - - level.Info(logger).Log("msg", "started block upload") - - w.WriteHeader(http.StatusOK) -} - -// FinishBlockUpload handles request for finishing block upload. -// -// Finishing block upload performs block validation, and if all checks pass, marks block as finished -// by uploading meta.json file. -func (c *MultitenantCompactor) FinishBlockUpload(w http.ResponseWriter, r *http.Request) { - blockID, tenantID, err := c.parseBlockUploadParameters(r) - if err != nil { - http.Error(w, err.Error(), http.StatusBadRequest) - return - } - - ctx := r.Context() - requestID := hexTimeNowNano() - logger := log.With( - util_log.WithContext(ctx, c.logger), - "feature", "block upload", - "block", blockID, - "operation", "complete block upload", - "request_id", requestID, - ) - - userBkt := bucket.NewUserBucketClient(tenantID, c.bucketClient, c.cfgProvider) - m, _, err := c.checkBlockState(ctx, userBkt, blockID, true) - if err != nil { - writeBlockUploadError(err, "can't check block state", logger, w, requestID) - return - } - - // This should not happen, as checkBlockState with requireUploadInProgress=true returns nil error - // only if uploading-meta.json file exists. - if m == nil { - http.Error(w, "internal error", http.StatusInternalServerError) - return - } - - if c.cfgProvider.CompactorBlockUploadValidationEnabled(tenantID) { - maxConcurrency := int64(c.compactorCfg.MaxBlockUploadValidationConcurrency) - currentValidations := c.blockUploadValidations.Inc() - decreaseActiveValidationsInDefer := true - defer func() { - if decreaseActiveValidationsInDefer { - c.blockUploadValidations.Dec() - } - }() - if maxConcurrency > 0 && currentValidations > maxConcurrency { - err := httpError{ - message: fmt.Sprintf("too many block upload validations in progress, limit is %d", maxConcurrency), - statusCode: http.StatusTooManyRequests, - } - writeBlockUploadError(err, "max concurrency was hit", logger, w, requestID) - return - } - // create validation file to signal that block validation has started - if err := c.uploadValidation(ctx, blockID, userBkt); err != nil { - writeBlockUploadError(err, "can't upload validation file", logger, w, requestID) - return - } - decreaseActiveValidationsInDefer = false - go c.validateAndCompleteBlockUpload(logger, tenantID, userBkt, blockID, m, func(ctx context.Context) error { - defer c.blockUploadValidations.Dec() - return c.validateBlock(ctx, logger, blockID, m, userBkt, tenantID) - }) - level.Info(logger).Log("msg", "validation process started") - } else { - if err := c.markBlockComplete(ctx, logger, tenantID, userBkt, blockID, m); err != nil { - writeBlockUploadError(err, "can't mark block as complete", logger, w, requestID) - return - } - level.Info(logger).Log("msg", "successfully finished block upload") - } - - w.WriteHeader(http.StatusOK) -} - -// parseBlockUploadParameters parses common parameters from the request: block ID, tenant and checks if tenant has uploads enabled. -func (c *MultitenantCompactor) parseBlockUploadParameters(r *http.Request) (ulid.ULID, string, error) { - blockID, err := ulid.Parse(mux.Vars(r)["block"]) - if err != nil { - return ulid.ULID{}, "", errors.New("invalid block ID") - } - - ctx := r.Context() - tenantID, err := tenant.TenantID(ctx) - if err != nil { - return ulid.ULID{}, "", errors.New("invalid tenant ID") - } - - if !c.cfgProvider.CompactorBlockUploadEnabled(tenantID) { - return ulid.ULID{}, "", errors.New("block upload is disabled") - } - - return blockID, tenantID, nil -} - -func writeBlockUploadError(err error, msg string, logger log.Logger, w http.ResponseWriter, requestID string) { - var httpErr httpError - if errors.As(err, &httpErr) { - level.Warn(logger).Log("msg", msg, "response", httpErr.message, "status", httpErr.statusCode) - http.Error(w, httpErr.message, httpErr.statusCode) - return - } - - level.Error(logger).Log("msg", msg, "err", err) - http.Error(w, fmt.Sprintf("internal server error (id %s)", requestID), http.StatusInternalServerError) -} - -// hexTimeNano returns a hex-encoded big-endian representation of the current time in nanoseconds, previously converted to uint64 and encoded as big-endian. -func hexTimeNowNano() string { - var buf [8]byte - binary.BigEndian.PutUint64(buf[:], uint64(time.Now().UTC().UnixNano())) - return hex.EncodeToString(buf[:]) -} - -func (c *MultitenantCompactor) createBlockUpload(ctx context.Context, meta *block.Meta, - logger log.Logger, userBkt objstore.Bucket, tenantID string, blockID ulid.ULID, -) error { - level.Debug(logger).Log("msg", "starting block upload") - - if msg := c.sanitizeMeta(logger, tenantID, blockID, meta); msg != "" { - return httpError{ - message: msg, - statusCode: http.StatusBadRequest, - } - } - - // validate data is within the retention period - retention := c.cfgProvider.CompactorBlocksRetentionPeriod(tenantID) - if retention > 0 { - threshold := time.Now().Add(-retention) - if time.UnixMilli(meta.MaxTime).Before(threshold) { - maxTimeStr := util.FormatTimeMillis(meta.MaxTime) - return httpError{ - message: fmt.Sprintf("block max time (%s) older than retention period", maxTimeStr), - statusCode: http.StatusUnprocessableEntity, - } - } - } - - return c.uploadMeta(ctx, logger, meta, blockID, uploadingMetaFilename, userBkt) -} - -// UploadBlockFile handles requests for uploading block files. -// It takes the mandatory query parameter "path", specifying the file's destination path. -func (c *MultitenantCompactor) UploadBlockFile(w http.ResponseWriter, r *http.Request) { - blockID, tenantID, err := c.parseBlockUploadParameters(r) - if err != nil { - http.Error(w, err.Error(), http.StatusBadRequest) - return - } - - ctx := r.Context() - requestID := hexTimeNowNano() - logger := log.With( - util_log.WithContext(ctx, c.logger), - "feature", "block upload", - "block", blockID, - "operation", "block file upload", - "request", requestID, - ) - - pth := r.URL.Query().Get("path") - if pth == "" { - err := httpError{statusCode: http.StatusBadRequest, message: "missing or invalid file path"} - writeBlockUploadError(err, "failed because file path is empty", logger, w, requestID) - return - } - - if path.Base(pth) == block.MetaFilename { - err := httpError{statusCode: http.StatusBadRequest, message: fmt.Sprintf("%s is not allowed", block.MetaFilename)} - writeBlockUploadError(err, "failed because block meta is not allowed", logger, w, requestID) - return - } - - if !rePath.MatchString(pth) { - err := httpError{statusCode: http.StatusBadRequest, message: fmt.Sprintf("invalid path: %q", pth)} - writeBlockUploadError(err, "failed because path is invalid", logger, w, requestID) - return - } - - if r.ContentLength == 0 { - err := httpError{statusCode: http.StatusBadRequest, message: "file cannot be empty"} - writeBlockUploadError(err, "failed because file is empty", logger, w, requestID) - return - } - - userBkt := bucket.NewUserBucketClient(tenantID, c.bucketClient, c.cfgProvider) - - m, _, err := c.checkBlockState(ctx, userBkt, blockID, true) - if err != nil { - writeBlockUploadError(err, "can't check block state", logger, w, requestID) - return - } - - // This should not happen. - if m == nil { - err := httpError{statusCode: http.StatusInternalServerError, message: "internal error"} - writeBlockUploadError(err, "block meta is nil but err is also nil", logger, w, requestID) - return - } - - // Check if file was specified in meta.json, and if it has expected size. - found := false - for _, f := range m.Thanos.Files { - if pth == f.RelPath { - found = true - - if r.ContentLength >= 0 && r.ContentLength != f.SizeBytes { - err := httpError{statusCode: http.StatusBadRequest, message: fmt.Sprintf("file size doesn't match %s", block.MetaFilename)} - writeBlockUploadError(err, "failed because file size didn't match", logger, w, requestID) - return - } - } - } - if !found { - err := httpError{statusCode: http.StatusBadRequest, message: "unexpected file"} - writeBlockUploadError(err, "failed because file was not found", logger, w, requestID) - return - } - - dst := path.Join(blockID.String(), pth) - - level.Debug(logger).Log("msg", "uploading block file to bucket", "destination", dst, "size", r.ContentLength) - reader := bodyReader{r: r} - if err := userBkt.Upload(ctx, dst, reader); err != nil { - // We don't know what caused the error; it could be the client's fault (e.g. killed - // connection), but internal server error is the safe choice here. - level.Error(logger).Log("msg", "failed uploading block file to bucket", "destination", dst, "err", err) - http.Error(w, fmt.Sprintf("internal server error (id %s)", requestID), http.StatusInternalServerError) - return - } - - level.Debug(logger).Log("msg", "finished uploading block file to bucket", "path", pth) - - w.WriteHeader(http.StatusOK) -} - -func (c *MultitenantCompactor) validateAndCompleteBlockUpload(logger log.Logger, tenantID string, userBkt objstore.Bucket, blockID ulid.ULID, meta *block.Meta, validation func(context.Context) error) { - level.Debug(logger).Log("msg", "completing block upload", "files", len(meta.Thanos.Files)) - - { - var wg sync.WaitGroup - ctx, cancel := context.WithCancel(context.Background()) - - // start a go routine that updates the validation file's timestamp every heartbeat interval - wg.Add(1) - go func() { - defer wg.Done() - c.periodicValidationUpdater(ctx, logger, blockID, userBkt, cancel, validationHeartbeatInterval) - }() - - if err := validation(ctx); err != nil { - level.Error(logger).Log("msg", "error while validating block", "err", err) - cancel() - wg.Wait() - err := c.uploadValidationWithError(context.Background(), blockID, userBkt, err.Error()) - if err != nil { - level.Error(logger).Log("msg", "error updating validation file after failed block validation", "err", err) - } - return - } - - cancel() - wg.Wait() // use waitgroup to ensure validation ts update is complete - } - - ctx := context.Background() - - if err := c.markBlockComplete(ctx, logger, tenantID, userBkt, blockID, meta); err != nil { - if err := c.uploadValidationWithError(ctx, blockID, userBkt, err.Error()); err != nil { - level.Error(logger).Log("msg", "error updating validation file after upload of metadata file failed", "err", err) - } - return - } - - if err := userBkt.Delete(ctx, path.Join(blockID.String(), validationFilename)); err != nil { - level.Warn(logger).Log("msg", fmt.Sprintf( - "failed to delete %s from block in object storage", validationFilename), "err", err) - return - } - - level.Info(logger).Log("msg", "successfully completed block upload") -} - -func (c *MultitenantCompactor) markBlockComplete(ctx context.Context, logger log.Logger, tenantID string, userBkt objstore.Bucket, blockID ulid.ULID, meta *block.Meta) error { - if err := c.uploadMeta(ctx, logger, meta, blockID, block.MetaFilename, userBkt); err != nil { - level.Error(logger).Log("msg", "error uploading block metadata file", "err", err) - return err - } - - if err := userBkt.Delete(ctx, path.Join(blockID.String(), uploadingMetaFilename)); err != nil { - // Not returning an error since the temporary meta file persisting is a harmless side effect - level.Warn(logger).Log("msg", fmt.Sprintf("failed to delete %s from block in object storage", uploadingMetaFilename), "err", err) - } - - // Increment metrics on successful block upload - c.blockUploadBlocks.WithLabelValues(tenantID).Inc() - c.blockUploadBytes.WithLabelValues(tenantID).Add(float64(meta.BlockBytes())) - c.blockUploadFiles.WithLabelValues(tenantID).Add(float64(len(meta.Thanos.Files))) - - return nil -} - -// sanitizeMeta sanitizes and validates a metadata.Meta object. If a validation error occurs, an error -// message gets returned, otherwise an empty string. -func (c *MultitenantCompactor) sanitizeMeta(logger log.Logger, userID string, blockID ulid.ULID, meta *block.Meta) string { - if meta == nil { - return "missing block metadata" - } - - // check that the blocks doesn't contain down-sampled data - if meta.Thanos.Downsample.Resolution > 0 { - return "block contains downsampled data" - } - - meta.ULID = blockID - for l, v := range meta.Thanos.Labels { - switch l { - // Preserve this label - case mimir_tsdb.CompactorShardIDExternalLabel: - if v == "" { - level.Debug(logger).Log("msg", "removing empty external label", - "label", l) - delete(meta.Thanos.Labels, l) - continue - } - - if _, _, err := sharding.ParseShardIDLabelValue(v); err != nil { - return fmt.Sprintf("invalid %s external label: %q", - mimir_tsdb.CompactorShardIDExternalLabel, v) - } - // Remove unused labels - case mimir_tsdb.DeprecatedTenantIDExternalLabel, mimir_tsdb.DeprecatedIngesterIDExternalLabel, mimir_tsdb.DeprecatedShardIDExternalLabel: - level.Debug(logger).Log("msg", "removing unused external label", - "label", l, "value", v) - delete(meta.Thanos.Labels, l) - default: - return fmt.Sprintf("unsupported external label: %s", l) - } - } - - meta.Compaction.Parents = nil - meta.Compaction.Sources = []ulid.ULID{blockID} - - for _, f := range meta.Thanos.Files { - if f.RelPath == block.MetaFilename { - continue - } - - if !rePath.MatchString(f.RelPath) { - return fmt.Sprintf("file with invalid path: %s", f.RelPath) - } - - if f.SizeBytes <= 0 { - return fmt.Sprintf("file with invalid size: %s", f.RelPath) - } - } - - if err := c.validateMaximumBlockSize(logger, meta.Thanos.Files, userID); err != nil { - return err.Error() - } - - if meta.Version != block.TSDBVersion1 { - return fmt.Sprintf("version must be %d", block.TSDBVersion1) - } - - // validate minTime/maxTime - // basic sanity check - if meta.MinTime < 0 || meta.MaxTime < 0 || meta.MaxTime < meta.MinTime { - return fmt.Sprintf("invalid minTime/maxTime: minTime=%d, maxTime=%d", - meta.MinTime, meta.MaxTime) - } - // validate that times are in the past - now := time.Now() - if meta.MinTime > now.UnixMilli() || meta.MaxTime > now.UnixMilli() { - return fmt.Sprintf("block time(s) greater than the present: minTime=%d, maxTime=%d", - meta.MinTime, meta.MaxTime) - } - - // Mark block source - meta.Thanos.Source = "upload" - - return "" -} - -func (c *MultitenantCompactor) uploadMeta(ctx context.Context, logger log.Logger, meta *block.Meta, blockID ulid.ULID, name string, userBkt objstore.Bucket) error { - if meta == nil { - return errors.New("missing block metadata") - } - dst := path.Join(blockID.String(), name) - level.Debug(logger).Log("msg", fmt.Sprintf("uploading %s to bucket", name), "dst", dst) - buf := bytes.NewBuffer(nil) - if err := json.NewEncoder(buf).Encode(meta); err != nil { - return errors.Wrap(err, "failed to encode block metadata") - } - if err := userBkt.Upload(ctx, dst, buf); err != nil { - return errors.Wrapf(err, "failed uploading %s to bucket", name) - } - - return nil -} - -func (c *MultitenantCompactor) createTemporaryBlockDirectory() (dir string, err error) { - blockDir, err := os.MkdirTemp(c.compactorCfg.DataDir, "upload") - if err != nil { - level.Error(c.logger).Log("msg", "failed to create temporary block directory", "err", err) - return "", errors.New("failed to create temporary block directory") - } - - level.Debug(c.logger).Log("msg", "created temporary block directory", "dir", blockDir) - return blockDir, nil -} - -func (c *MultitenantCompactor) removeTemporaryBlockDirectory(blockDir string) { - level.Debug(c.logger).Log("msg", "removing temporary block directory", "dir", blockDir) - if err := os.RemoveAll(blockDir); err != nil { - level.Warn(c.logger).Log("msg", "failed to remove temporary block directory", "path", blockDir, "err", err) - } -} - -func (c *MultitenantCompactor) prepareBlockForValidation(ctx context.Context, userBkt objstore.Bucket, blockID ulid.ULID) (string, error) { - blockDir, err := c.createTemporaryBlockDirectory() - if err != nil { - return "", err - } - - // download the block to local storage - level.Debug(c.logger).Log("msg", "downloading block from bucket", "block", blockID.String()) - err = objstore.DownloadDir(ctx, c.logger, userBkt, blockID.String(), blockID.String(), blockDir) - if err != nil { - c.removeTemporaryBlockDirectory(blockDir) - return "", errors.Wrap(err, "failed to download block") - } - - // rename the temporary meta file name to the expected one locally so that the block can be inspected - err = os.Rename(filepath.Join(blockDir, uploadingMetaFilename), filepath.Join(blockDir, block.MetaFilename)) - if err != nil { - level.Warn(c.logger).Log("msg", "could not rename temporary metadata file", "block", blockID.String(), "err", err) - c.removeTemporaryBlockDirectory(blockDir) - return "", errors.New("failed renaming while preparing block for validation") - } - - return blockDir, nil -} - -func (c *MultitenantCompactor) validateBlock(ctx context.Context, logger log.Logger, blockID ulid.ULID, blockMetadata *block.Meta, userBkt objstore.Bucket, userID string) error { - if err := c.validateMaximumBlockSize(logger, blockMetadata.Thanos.Files, userID); err != nil { - return err - } - - blockDir, err := c.prepareBlockForValidation(ctx, userBkt, blockID) - if err != nil { - return err - } - defer c.removeTemporaryBlockDirectory(blockDir) - - // check that all files listed in the metadata are present and the correct size - for _, f := range blockMetadata.Thanos.Files { - fi, err := os.Stat(filepath.Join(blockDir, filepath.FromSlash(f.RelPath))) - if err != nil { - return errors.Wrapf(err, "failed to stat %s", f.RelPath) - } - - if !fi.Mode().IsRegular() { - return errors.Errorf("not a file: %s", f.RelPath) - } - - if f.RelPath != block.MetaFilename && fi.Size() != f.SizeBytes { - return errors.Errorf("file size mismatch for %s", f.RelPath) - } - } - - // validate block - checkChunks := c.cfgProvider.CompactorBlockUploadVerifyChunks(userID) - err = block.VerifyBlock(c.logger, blockDir, blockMetadata.MinTime, blockMetadata.MaxTime, checkChunks) - if err != nil { - return errors.Wrap(err, "error validating block") - } - - return nil -} - -func (c *MultitenantCompactor) validateMaximumBlockSize(logger log.Logger, files []block.File, userID string) error { - maxBlockSizeBytes := c.cfgProvider.CompactorBlockUploadMaxBlockSizeBytes(userID) - if maxBlockSizeBytes <= 0 { - return nil - } - - blockSizeBytes := int64(0) - for _, f := range files { - if f.SizeBytes < 0 { - return errors.New("invalid negative file size in block metadata") - } - blockSizeBytes += f.SizeBytes - if blockSizeBytes < 0 { - // overflow - break - } - } - - if blockSizeBytes > maxBlockSizeBytes || blockSizeBytes < 0 { - level.Error(logger).Log("msg", "rejecting block upload for exceeding maximum size", "limit", maxBlockSizeBytes, "size", blockSizeBytes) - return fmt.Errorf(maxBlockUploadSizeBytesFormat, maxBlockSizeBytes) - } - return nil -} - -type httpError struct { - message string - statusCode int -} - -func (e httpError) Error() string { - return e.message -} - -type bodyReader struct { - r *http.Request -} - -// ObjectSize implements thanos.ObjectSizer. -func (r bodyReader) ObjectSize() (int64, error) { - if r.r.ContentLength < 0 { - return 0, fmt.Errorf("unknown size") - } - - return r.r.ContentLength, nil -} - -// Read implements io.Reader. -func (r bodyReader) Read(b []byte) (int, error) { - return r.r.Body.Read(b) -} - -type validationFile struct { - LastUpdate int64 // UnixMillis of last update time. - Error string // Error message if validation failed. -} - -type blockUploadStateResult struct { - State string `json:"result"` - Error string `json:"error,omitempty"` -} - -type blockUploadState int - -const ( - blockStateUnknown blockUploadState = iota // unknown, default value - blockIsComplete // meta.json file exists - blockUploadNotStarted // meta.json doesn't exist, uploading-meta.json doesn't exist - blockUploadInProgress // meta.json doesn't exist, but uploading-meta.json does - blockValidationInProgress // meta.json doesn't exist, uploading-meta.json exists, validation.json exists and is recent - blockValidationFailed - blockValidationStale -) - -func (c *MultitenantCompactor) GetBlockUploadStateHandler(w http.ResponseWriter, r *http.Request) { - blockID, tenantID, err := c.parseBlockUploadParameters(r) - if err != nil { - http.Error(w, err.Error(), http.StatusBadRequest) - return - } - requestID := hexTimeNowNano() - logger := log.With( - util_log.WithContext(r.Context(), c.logger), - "feature", "block upload", - "block", blockID, - "operation", "get block state", - "request_id", requestID, - ) - - userBkt := bucket.NewUserBucketClient(tenantID, c.bucketClient, c.cfgProvider) - if err != nil { - http.Error(w, err.Error(), http.StatusBadRequest) - return - } - - s, _, v, err := c.getBlockUploadState(r.Context(), userBkt, blockID) - if err != nil { - writeBlockUploadError(err, "can't get upload state", logger, w, requestID) - return - } - - res := blockUploadStateResult{} - - switch s { - case blockIsComplete: - res.State = "complete" - case blockUploadNotStarted: - http.Error(w, "block doesn't exist", http.StatusNotFound) - return - case blockValidationStale: - fallthrough - case blockUploadInProgress: - res.State = "uploading" - case blockValidationInProgress: - res.State = "validating" - case blockValidationFailed: - res.State = "failed" - res.Error = v.Error - } - - util.WriteJSONResponse(w, res) -} - -// checkBlockState checks blocks state and returns various HTTP status codes for individual states if block -// upload cannot start, finish or file cannot be uploaded to the block. -func (c *MultitenantCompactor) checkBlockState(ctx context.Context, userBkt objstore.Bucket, blockID ulid.ULID, requireUploadInProgress bool) (*block.Meta, *validationFile, error) { - s, m, v, err := c.getBlockUploadState(ctx, userBkt, blockID) - if err != nil { - return m, v, err - } - - switch s { - case blockIsComplete: - return m, v, httpError{message: "block already exists", statusCode: http.StatusConflict} - case blockValidationInProgress: - return m, v, httpError{message: "block validation in progress", statusCode: http.StatusBadRequest} - case blockUploadNotStarted: - if requireUploadInProgress { - return m, v, httpError{message: "block upload not started", statusCode: http.StatusNotFound} - } - return m, v, nil - case blockValidationStale: - // if validation is stale, we treat block as being in "upload in progress" state, and validation can start again. - fallthrough - case blockUploadInProgress: - return m, v, nil - case blockValidationFailed: - return m, v, httpError{message: "block validation failed", statusCode: http.StatusBadRequest} - } - - return m, v, httpError{message: "unknown block upload state", statusCode: http.StatusInternalServerError} -} - -// getBlockUploadState returns state of the block upload, and meta and validation objects, if they exist. -func (c *MultitenantCompactor) getBlockUploadState(ctx context.Context, userBkt objstore.Bucket, blockID ulid.ULID) (blockUploadState, *block.Meta, *validationFile, error) { - exists, err := userBkt.Exists(ctx, path.Join(blockID.String(), block.MetaFilename)) - if err != nil { - return blockStateUnknown, nil, nil, err - } - if exists { - return blockIsComplete, nil, nil, nil - } - - meta, err := c.loadUploadingMeta(ctx, userBkt, blockID) - if err != nil { - return blockStateUnknown, nil, nil, err - } - // If neither meta.json nor uploading-meta.json file exist, we say that the block doesn't exist. - if meta == nil { - return blockUploadNotStarted, nil, nil, err - } - - v, err := c.loadValidation(ctx, userBkt, blockID) - if err != nil { - return blockStateUnknown, meta, nil, err - } - if v == nil { - return blockUploadInProgress, meta, nil, err - } - if v.Error != "" { - return blockValidationFailed, meta, v, err - } - if time.Since(time.UnixMilli(v.LastUpdate)) < validationHeartbeatTimeout { - return blockValidationInProgress, meta, v, nil - } - return blockValidationStale, meta, v, nil -} - -func (c *MultitenantCompactor) loadUploadingMeta(ctx context.Context, userBkt objstore.Bucket, blockID ulid.ULID) (*block.Meta, error) { - r, err := userBkt.Get(ctx, path.Join(blockID.String(), uploadingMetaFilename)) - if err != nil { - if userBkt.IsObjNotFoundErr(err) { - return nil, nil - } - return nil, err - } - defer func() { _ = r.Close() }() - - v := &block.Meta{} - err = json.NewDecoder(r).Decode(v) - if err != nil { - return nil, err - } - - return v, nil -} - -func (c *MultitenantCompactor) loadValidation(ctx context.Context, userBkt objstore.Bucket, blockID ulid.ULID) (*validationFile, error) { - r, err := userBkt.Get(ctx, path.Join(blockID.String(), validationFilename)) - if err != nil { - if userBkt.IsObjNotFoundErr(err) { - return nil, nil - } - return nil, err - } - defer func() { _ = r.Close() }() - - v := &validationFile{} - err = json.NewDecoder(r).Decode(v) - if err != nil { - return nil, err - } - - return v, nil -} - -func (c *MultitenantCompactor) uploadValidationWithError(ctx context.Context, blockID ulid.ULID, - userBkt objstore.Bucket, errorStr string, -) error { - val := validationFile{ - LastUpdate: time.Now().UnixMilli(), - Error: errorStr, - } - dst := path.Join(blockID.String(), validationFilename) - if err := marshalAndUploadToBucket(ctx, userBkt, dst, val); err != nil { - return errors.Wrapf(err, "failed uploading %s to bucket", validationFilename) - } - return nil -} - -func (c *MultitenantCompactor) uploadValidation(ctx context.Context, blockID ulid.ULID, userBkt objstore.Bucket) error { - return c.uploadValidationWithError(ctx, blockID, userBkt, "") -} - -func (c *MultitenantCompactor) periodicValidationUpdater(ctx context.Context, logger log.Logger, blockID ulid.ULID, userBkt objstore.Bucket, cancelFn func(), interval time.Duration) { - ticker := time.NewTicker(interval) - defer ticker.Stop() - for { - select { - case <-ctx.Done(): - return - case <-ticker.C: - if err := c.uploadValidation(ctx, blockID, userBkt); err != nil { - level.Warn(logger).Log("msg", "error during periodic update of validation file", "err", err) - cancelFn() - return - } - } - } -} - -func marshalAndUploadToBucket(ctx context.Context, bkt objstore.Bucket, pth string, val interface{}) error { - buf, err := json.Marshal(val) - if err != nil { - return err - } - if err := bkt.Upload(ctx, pth, bytes.NewReader(buf)); err != nil { - return err - } - return nil -} +// import ( +// "bytes" +// "context" +// "encoding/binary" +// "encoding/hex" +// "encoding/json" +// "fmt" +// "io" +// "net/http" +// "os" +// "path" +// "path/filepath" +// "sync" +// "time" + +// "github.com/go-kit/log" +// "github.com/go-kit/log/level" +// "github.com/gorilla/mux" +// "github.com/grafana/dskit/tenant" +// "github.com/grafana/regexp" +// "github.com/oklog/ulid" +// "github.com/pkg/errors" +// "github.com/thanos-io/objstore" + +// "github.com/grafana/mimir/pkg/storage/bucket" +// "github.com/grafana/mimir/pkg/storage/sharding" +// mimir_tsdb "github.com/grafana/mimir/pkg/storage/tsdb" +// "github.com/grafana/mimir/pkg/storage/tsdb/block" +// "github.com/grafana/mimir/pkg/util" +// util_log "github.com/grafana/mimir/pkg/util/log" +// ) + +// const ( +// uploadingMetaFilename = "uploading-meta.json" // Name of the file that stores a block's meta file while it's being uploaded +// validationFilename = "validation.json" // Name of the file that stores a heartbeat time and possibly an error message +// validationHeartbeatInterval = 1 * time.Minute // Duration of time between heartbeats of an in-progress block upload validation +// validationHeartbeatTimeout = 5 * time.Minute // Maximum duration of time to wait until a validation is able to be restarted +// maximumMetaSizeBytes = 1 * 1024 * 1024 // 1 MiB, maximum allowed size of an uploaded block's meta.json file +// ) + +// var ( +// maxBlockUploadSizeBytesFormat = "block exceeds the maximum block size limit of %d bytes" +// rePath = regexp.MustCompile(`^(index|chunks/\d{6})$`) +// ) + +// // StartBlockUpload handles request for starting block upload. +// // +// // Starting the uploading of a block means to upload a meta file and verify that the upload can +// // go ahead. In practice this means to check that the (complete) block isn't already in block +// // storage, and that the meta file is valid. +// func (c *MultitenantCompactor) StartBlockUpload(w http.ResponseWriter, r *http.Request) { +// blockID, tenantID, err := c.parseBlockUploadParameters(r) +// if err != nil { +// http.Error(w, err.Error(), http.StatusBadRequest) +// return +// } + +// ctx := r.Context() +// requestID := hexTimeNowNano() +// logger := log.With( +// util_log.WithContext(ctx, c.logger), +// "feature", "block upload", +// "block", blockID, +// "operation", "start block upload", +// "request_id", requestID, +// ) + +// userBkt := bucket.NewUserBucketClient(tenantID, c.bucketClient, c.cfgProvider) +// if _, _, err := c.checkBlockState(ctx, userBkt, blockID, false); err != nil { +// writeBlockUploadError(err, "can't check block state", logger, w, requestID) +// return +// } + +// content, err := io.ReadAll(http.MaxBytesReader(w, r.Body, maximumMetaSizeBytes)) +// if err != nil { +// if errors.As(err, new(*http.MaxBytesError)) { +// err = httpError{ +// message: fmt.Sprintf("The block metadata was too large (maximum size allowed is %d bytes)", maximumMetaSizeBytes), +// statusCode: http.StatusRequestEntityTooLarge, +// } +// } +// writeBlockUploadError(err, "failed reading body", logger, w, requestID) +// return +// } + +// var meta block.Meta +// if err := json.Unmarshal(content, &meta); err != nil { +// err = httpError{ +// message: "malformed request body", +// statusCode: http.StatusBadRequest, +// } +// writeBlockUploadError(err, "failed unmarshaling block meta json", logger, w, requestID) +// return +// } + +// if err := c.createBlockUpload(ctx, &meta, logger, userBkt, tenantID, blockID); err != nil { +// writeBlockUploadError(err, "failed creating block upload", logger, w, requestID) +// return +// } + +// level.Info(logger).Log("msg", "started block upload") + +// w.WriteHeader(http.StatusOK) +// } + +// // FinishBlockUpload handles request for finishing block upload. +// // +// // Finishing block upload performs block validation, and if all checks pass, marks block as finished +// // by uploading meta.json file. +// func (c *MultitenantCompactor) FinishBlockUpload(w http.ResponseWriter, r *http.Request) { +// blockID, tenantID, err := c.parseBlockUploadParameters(r) +// if err != nil { +// http.Error(w, err.Error(), http.StatusBadRequest) +// return +// } + +// ctx := r.Context() +// requestID := hexTimeNowNano() +// logger := log.With( +// util_log.WithContext(ctx, c.logger), +// "feature", "block upload", +// "block", blockID, +// "operation", "complete block upload", +// "request_id", requestID, +// ) + +// userBkt := bucket.NewUserBucketClient(tenantID, c.bucketClient, c.cfgProvider) +// m, _, err := c.checkBlockState(ctx, userBkt, blockID, true) +// if err != nil { +// writeBlockUploadError(err, "can't check block state", logger, w, requestID) +// return +// } + +// // This should not happen, as checkBlockState with requireUploadInProgress=true returns nil error +// // only if uploading-meta.json file exists. +// if m == nil { +// http.Error(w, "internal error", http.StatusInternalServerError) +// return +// } + +// if c.cfgProvider.CompactorBlockUploadValidationEnabled(tenantID) { +// maxConcurrency := int64(c.compactorCfg.MaxBlockUploadValidationConcurrency) +// currentValidations := c.blockUploadValidations.Inc() +// decreaseActiveValidationsInDefer := true +// defer func() { +// if decreaseActiveValidationsInDefer { +// c.blockUploadValidations.Dec() +// } +// }() +// if maxConcurrency > 0 && currentValidations > maxConcurrency { +// err := httpError{ +// message: fmt.Sprintf("too many block upload validations in progress, limit is %d", maxConcurrency), +// statusCode: http.StatusTooManyRequests, +// } +// writeBlockUploadError(err, "max concurrency was hit", logger, w, requestID) +// return +// } +// // create validation file to signal that block validation has started +// if err := c.uploadValidation(ctx, blockID, userBkt); err != nil { +// writeBlockUploadError(err, "can't upload validation file", logger, w, requestID) +// return +// } +// decreaseActiveValidationsInDefer = false +// go c.validateAndCompleteBlockUpload(logger, tenantID, userBkt, blockID, m, func(ctx context.Context) error { +// defer c.blockUploadValidations.Dec() +// return c.validateBlock(ctx, logger, blockID, m, userBkt, tenantID) +// }) +// level.Info(logger).Log("msg", "validation process started") +// } else { +// if err := c.markBlockComplete(ctx, logger, tenantID, userBkt, blockID, m); err != nil { +// writeBlockUploadError(err, "can't mark block as complete", logger, w, requestID) +// return +// } +// level.Info(logger).Log("msg", "successfully finished block upload") +// } + +// w.WriteHeader(http.StatusOK) +// } + +// // parseBlockUploadParameters parses common parameters from the request: block ID, tenant and checks if tenant has uploads enabled. +// func (c *MultitenantCompactor) parseBlockUploadParameters(r *http.Request) (ulid.ULID, string, error) { +// blockID, err := ulid.Parse(mux.Vars(r)["block"]) +// if err != nil { +// return ulid.ULID{}, "", errors.New("invalid block ID") +// } + +// ctx := r.Context() +// tenantID, err := tenant.TenantID(ctx) +// if err != nil { +// return ulid.ULID{}, "", errors.New("invalid tenant ID") +// } + +// if !c.cfgProvider.CompactorBlockUploadEnabled(tenantID) { +// return ulid.ULID{}, "", errors.New("block upload is disabled") +// } + +// return blockID, tenantID, nil +// } + +// func writeBlockUploadError(err error, msg string, logger log.Logger, w http.ResponseWriter, requestID string) { +// var httpErr httpError +// if errors.As(err, &httpErr) { +// level.Warn(logger).Log("msg", msg, "response", httpErr.message, "status", httpErr.statusCode) +// http.Error(w, httpErr.message, httpErr.statusCode) +// return +// } + +// level.Error(logger).Log("msg", msg, "err", err) +// http.Error(w, fmt.Sprintf("internal server error (id %s)", requestID), http.StatusInternalServerError) +// } + +// // hexTimeNano returns a hex-encoded big-endian representation of the current time in nanoseconds, previously converted to uint64 and encoded as big-endian. +// func hexTimeNowNano() string { +// var buf [8]byte +// binary.BigEndian.PutUint64(buf[:], uint64(time.Now().UTC().UnixNano())) +// return hex.EncodeToString(buf[:]) +// } + +// func (c *MultitenantCompactor) createBlockUpload(ctx context.Context, meta *block.Meta, +// logger log.Logger, userBkt objstore.Bucket, tenantID string, blockID ulid.ULID, +// ) error { +// level.Debug(logger).Log("msg", "starting block upload") + +// if msg := c.sanitizeMeta(logger, tenantID, blockID, meta); msg != "" { +// return httpError{ +// message: msg, +// statusCode: http.StatusBadRequest, +// } +// } + +// // validate data is within the retention period +// retention := c.cfgProvider.CompactorBlocksRetentionPeriod(tenantID) +// if retention > 0 { +// threshold := time.Now().Add(-retention) +// if time.UnixMilli(meta.MaxTime).Before(threshold) { +// maxTimeStr := util.FormatTimeMillis(meta.MaxTime) +// return httpError{ +// message: fmt.Sprintf("block max time (%s) older than retention period", maxTimeStr), +// statusCode: http.StatusUnprocessableEntity, +// } +// } +// } + +// return c.uploadMeta(ctx, logger, meta, blockID, uploadingMetaFilename, userBkt) +// } + +// // UploadBlockFile handles requests for uploading block files. +// // It takes the mandatory query parameter "path", specifying the file's destination path. +// func (c *MultitenantCompactor) UploadBlockFile(w http.ResponseWriter, r *http.Request) { +// blockID, tenantID, err := c.parseBlockUploadParameters(r) +// if err != nil { +// http.Error(w, err.Error(), http.StatusBadRequest) +// return +// } + +// ctx := r.Context() +// requestID := hexTimeNowNano() +// logger := log.With( +// util_log.WithContext(ctx, c.logger), +// "feature", "block upload", +// "block", blockID, +// "operation", "block file upload", +// "request", requestID, +// ) + +// pth := r.URL.Query().Get("path") +// if pth == "" { +// err := httpError{statusCode: http.StatusBadRequest, message: "missing or invalid file path"} +// writeBlockUploadError(err, "failed because file path is empty", logger, w, requestID) +// return +// } + +// if path.Base(pth) == block.MetaFilename { +// err := httpError{statusCode: http.StatusBadRequest, message: fmt.Sprintf("%s is not allowed", block.MetaFilename)} +// writeBlockUploadError(err, "failed because block meta is not allowed", logger, w, requestID) +// return +// } + +// if !rePath.MatchString(pth) { +// err := httpError{statusCode: http.StatusBadRequest, message: fmt.Sprintf("invalid path: %q", pth)} +// writeBlockUploadError(err, "failed because path is invalid", logger, w, requestID) +// return +// } + +// if r.ContentLength == 0 { +// err := httpError{statusCode: http.StatusBadRequest, message: "file cannot be empty"} +// writeBlockUploadError(err, "failed because file is empty", logger, w, requestID) +// return +// } + +// userBkt := bucket.NewUserBucketClient(tenantID, c.bucketClient, c.cfgProvider) + +// m, _, err := c.checkBlockState(ctx, userBkt, blockID, true) +// if err != nil { +// writeBlockUploadError(err, "can't check block state", logger, w, requestID) +// return +// } + +// // This should not happen. +// if m == nil { +// err := httpError{statusCode: http.StatusInternalServerError, message: "internal error"} +// writeBlockUploadError(err, "block meta is nil but err is also nil", logger, w, requestID) +// return +// } + +// // Check if file was specified in meta.json, and if it has expected size. +// found := false +// for _, f := range m.Thanos.Files { +// if pth == f.RelPath { +// found = true + +// if r.ContentLength >= 0 && r.ContentLength != f.SizeBytes { +// err := httpError{statusCode: http.StatusBadRequest, message: fmt.Sprintf("file size doesn't match %s", block.MetaFilename)} +// writeBlockUploadError(err, "failed because file size didn't match", logger, w, requestID) +// return +// } +// } +// } +// if !found { +// err := httpError{statusCode: http.StatusBadRequest, message: "unexpected file"} +// writeBlockUploadError(err, "failed because file was not found", logger, w, requestID) +// return +// } + +// dst := path.Join(blockID.String(), pth) + +// level.Debug(logger).Log("msg", "uploading block file to bucket", "destination", dst, "size", r.ContentLength) +// reader := bodyReader{r: r} +// if err := userBkt.Upload(ctx, dst, reader); err != nil { +// // We don't know what caused the error; it could be the client's fault (e.g. killed +// // connection), but internal server error is the safe choice here. +// level.Error(logger).Log("msg", "failed uploading block file to bucket", "destination", dst, "err", err) +// http.Error(w, fmt.Sprintf("internal server error (id %s)", requestID), http.StatusInternalServerError) +// return +// } + +// level.Debug(logger).Log("msg", "finished uploading block file to bucket", "path", pth) + +// w.WriteHeader(http.StatusOK) +// } + +// func (c *MultitenantCompactor) validateAndCompleteBlockUpload(logger log.Logger, tenantID string, userBkt objstore.Bucket, blockID ulid.ULID, meta *block.Meta, validation func(context.Context) error) { +// level.Debug(logger).Log("msg", "completing block upload", "files", len(meta.Thanos.Files)) + +// { +// var wg sync.WaitGroup +// ctx, cancel := context.WithCancel(context.Background()) + +// // start a go routine that updates the validation file's timestamp every heartbeat interval +// wg.Add(1) +// go func() { +// defer wg.Done() +// c.periodicValidationUpdater(ctx, logger, blockID, userBkt, cancel, validationHeartbeatInterval) +// }() + +// if err := validation(ctx); err != nil { +// level.Error(logger).Log("msg", "error while validating block", "err", err) +// cancel() +// wg.Wait() +// err := c.uploadValidationWithError(context.Background(), blockID, userBkt, err.Error()) +// if err != nil { +// level.Error(logger).Log("msg", "error updating validation file after failed block validation", "err", err) +// } +// return +// } + +// cancel() +// wg.Wait() // use waitgroup to ensure validation ts update is complete +// } + +// ctx := context.Background() + +// if err := c.markBlockComplete(ctx, logger, tenantID, userBkt, blockID, meta); err != nil { +// if err := c.uploadValidationWithError(ctx, blockID, userBkt, err.Error()); err != nil { +// level.Error(logger).Log("msg", "error updating validation file after upload of metadata file failed", "err", err) +// } +// return +// } + +// if err := userBkt.Delete(ctx, path.Join(blockID.String(), validationFilename)); err != nil { +// level.Warn(logger).Log("msg", fmt.Sprintf( +// "failed to delete %s from block in object storage", validationFilename), "err", err) +// return +// } + +// level.Info(logger).Log("msg", "successfully completed block upload") +// } + +// func (c *MultitenantCompactor) markBlockComplete(ctx context.Context, logger log.Logger, tenantID string, userBkt objstore.Bucket, blockID ulid.ULID, meta *block.Meta) error { +// if err := c.uploadMeta(ctx, logger, meta, blockID, block.MetaFilename, userBkt); err != nil { +// level.Error(logger).Log("msg", "error uploading block metadata file", "err", err) +// return err +// } + +// if err := userBkt.Delete(ctx, path.Join(blockID.String(), uploadingMetaFilename)); err != nil { +// // Not returning an error since the temporary meta file persisting is a harmless side effect +// level.Warn(logger).Log("msg", fmt.Sprintf("failed to delete %s from block in object storage", uploadingMetaFilename), "err", err) +// } + +// // Increment metrics on successful block upload +// c.blockUploadBlocks.WithLabelValues(tenantID).Inc() +// c.blockUploadBytes.WithLabelValues(tenantID).Add(float64(meta.BlockBytes())) +// c.blockUploadFiles.WithLabelValues(tenantID).Add(float64(len(meta.Thanos.Files))) + +// return nil +// } + +// // sanitizeMeta sanitizes and validates a metadata.Meta object. If a validation error occurs, an error +// // message gets returned, otherwise an empty string. +// func (c *MultitenantCompactor) sanitizeMeta(logger log.Logger, userID string, blockID ulid.ULID, meta *block.Meta) string { +// if meta == nil { +// return "missing block metadata" +// } + +// // check that the blocks doesn't contain down-sampled data +// if meta.Thanos.Downsample.Resolution > 0 { +// return "block contains downsampled data" +// } + +// meta.ULID = blockID +// for l, v := range meta.Thanos.Labels { +// switch l { +// // Preserve this label +// case mimir_tsdb.CompactorShardIDExternalLabel: +// if v == "" { +// level.Debug(logger).Log("msg", "removing empty external label", +// "label", l) +// delete(meta.Thanos.Labels, l) +// continue +// } + +// if _, _, err := sharding.ParseShardIDLabelValue(v); err != nil { +// return fmt.Sprintf("invalid %s external label: %q", +// mimir_tsdb.CompactorShardIDExternalLabel, v) +// } +// // Remove unused labels +// case mimir_tsdb.DeprecatedTenantIDExternalLabel, mimir_tsdb.DeprecatedIngesterIDExternalLabel, mimir_tsdb.DeprecatedShardIDExternalLabel: +// level.Debug(logger).Log("msg", "removing unused external label", +// "label", l, "value", v) +// delete(meta.Thanos.Labels, l) +// default: +// return fmt.Sprintf("unsupported external label: %s", l) +// } +// } + +// meta.Compaction.Parents = nil +// meta.Compaction.Sources = []ulid.ULID{blockID} + +// for _, f := range meta.Thanos.Files { +// if f.RelPath == block.MetaFilename { +// continue +// } + +// if !rePath.MatchString(f.RelPath) { +// return fmt.Sprintf("file with invalid path: %s", f.RelPath) +// } + +// if f.SizeBytes <= 0 { +// return fmt.Sprintf("file with invalid size: %s", f.RelPath) +// } +// } + +// if err := c.validateMaximumBlockSize(logger, meta.Thanos.Files, userID); err != nil { +// return err.Error() +// } + +// if meta.Version != block.TSDBVersion1 { +// return fmt.Sprintf("version must be %d", block.TSDBVersion1) +// } + +// // validate minTime/maxTime +// // basic sanity check +// if meta.MinTime < 0 || meta.MaxTime < 0 || meta.MaxTime < meta.MinTime { +// return fmt.Sprintf("invalid minTime/maxTime: minTime=%d, maxTime=%d", +// meta.MinTime, meta.MaxTime) +// } +// // validate that times are in the past +// now := time.Now() +// if meta.MinTime > now.UnixMilli() || meta.MaxTime > now.UnixMilli() { +// return fmt.Sprintf("block time(s) greater than the present: minTime=%d, maxTime=%d", +// meta.MinTime, meta.MaxTime) +// } + +// // Mark block source +// meta.Thanos.Source = "upload" + +// return "" +// } + +// func (c *MultitenantCompactor) uploadMeta(ctx context.Context, logger log.Logger, meta *block.Meta, blockID ulid.ULID, name string, userBkt objstore.Bucket) error { +// if meta == nil { +// return errors.New("missing block metadata") +// } +// dst := path.Join(blockID.String(), name) +// level.Debug(logger).Log("msg", fmt.Sprintf("uploading %s to bucket", name), "dst", dst) +// buf := bytes.NewBuffer(nil) +// if err := json.NewEncoder(buf).Encode(meta); err != nil { +// return errors.Wrap(err, "failed to encode block metadata") +// } +// if err := userBkt.Upload(ctx, dst, buf); err != nil { +// return errors.Wrapf(err, "failed uploading %s to bucket", name) +// } + +// return nil +// } + +// func (c *MultitenantCompactor) createTemporaryBlockDirectory() (dir string, err error) { +// blockDir, err := os.MkdirTemp(c.compactorCfg.DataDir, "upload") +// if err != nil { +// level.Error(c.logger).Log("msg", "failed to create temporary block directory", "err", err) +// return "", errors.New("failed to create temporary block directory") +// } + +// level.Debug(c.logger).Log("msg", "created temporary block directory", "dir", blockDir) +// return blockDir, nil +// } + +// func (c *MultitenantCompactor) removeTemporaryBlockDirectory(blockDir string) { +// level.Debug(c.logger).Log("msg", "removing temporary block directory", "dir", blockDir) +// if err := os.RemoveAll(blockDir); err != nil { +// level.Warn(c.logger).Log("msg", "failed to remove temporary block directory", "path", blockDir, "err", err) +// } +// } + +// func (c *MultitenantCompactor) prepareBlockForValidation(ctx context.Context, userBkt objstore.Bucket, blockID ulid.ULID) (string, error) { +// blockDir, err := c.createTemporaryBlockDirectory() +// if err != nil { +// return "", err +// } + +// // download the block to local storage +// level.Debug(c.logger).Log("msg", "downloading block from bucket", "block", blockID.String()) +// err = objstore.DownloadDir(ctx, c.logger, userBkt, blockID.String(), blockID.String(), blockDir) +// if err != nil { +// c.removeTemporaryBlockDirectory(blockDir) +// return "", errors.Wrap(err, "failed to download block") +// } + +// // rename the temporary meta file name to the expected one locally so that the block can be inspected +// err = os.Rename(filepath.Join(blockDir, uploadingMetaFilename), filepath.Join(blockDir, block.MetaFilename)) +// if err != nil { +// level.Warn(c.logger).Log("msg", "could not rename temporary metadata file", "block", blockID.String(), "err", err) +// c.removeTemporaryBlockDirectory(blockDir) +// return "", errors.New("failed renaming while preparing block for validation") +// } + +// return blockDir, nil +// } + +// func (c *MultitenantCompactor) validateBlock(ctx context.Context, logger log.Logger, blockID ulid.ULID, blockMetadata *block.Meta, userBkt objstore.Bucket, userID string) error { +// if err := c.validateMaximumBlockSize(logger, blockMetadata.Thanos.Files, userID); err != nil { +// return err +// } + +// blockDir, err := c.prepareBlockForValidation(ctx, userBkt, blockID) +// if err != nil { +// return err +// } +// defer c.removeTemporaryBlockDirectory(blockDir) + +// // check that all files listed in the metadata are present and the correct size +// for _, f := range blockMetadata.Thanos.Files { +// fi, err := os.Stat(filepath.Join(blockDir, filepath.FromSlash(f.RelPath))) +// if err != nil { +// return errors.Wrapf(err, "failed to stat %s", f.RelPath) +// } + +// if !fi.Mode().IsRegular() { +// return errors.Errorf("not a file: %s", f.RelPath) +// } + +// if f.RelPath != block.MetaFilename && fi.Size() != f.SizeBytes { +// return errors.Errorf("file size mismatch for %s", f.RelPath) +// } +// } + +// // validate block +// checkChunks := c.cfgProvider.CompactorBlockUploadVerifyChunks(userID) +// err = block.VerifyBlock(c.logger, blockDir, blockMetadata.MinTime, blockMetadata.MaxTime, checkChunks) +// if err != nil { +// return errors.Wrap(err, "error validating block") +// } + +// return nil +// } + +// func (c *MultitenantCompactor) validateMaximumBlockSize(logger log.Logger, files []block.File, userID string) error { +// maxBlockSizeBytes := c.cfgProvider.CompactorBlockUploadMaxBlockSizeBytes(userID) +// if maxBlockSizeBytes <= 0 { +// return nil +// } + +// blockSizeBytes := int64(0) +// for _, f := range files { +// if f.SizeBytes < 0 { +// return errors.New("invalid negative file size in block metadata") +// } +// blockSizeBytes += f.SizeBytes +// if blockSizeBytes < 0 { +// // overflow +// break +// } +// } + +// if blockSizeBytes > maxBlockSizeBytes || blockSizeBytes < 0 { +// level.Error(logger).Log("msg", "rejecting block upload for exceeding maximum size", "limit", maxBlockSizeBytes, "size", blockSizeBytes) +// return fmt.Errorf(maxBlockUploadSizeBytesFormat, maxBlockSizeBytes) +// } +// return nil +// } + +// type httpError struct { +// message string +// statusCode int +// } + +// func (e httpError) Error() string { +// return e.message +// } + +// type bodyReader struct { +// r *http.Request +// } + +// // ObjectSize implements thanos.ObjectSizer. +// func (r bodyReader) ObjectSize() (int64, error) { +// if r.r.ContentLength < 0 { +// return 0, fmt.Errorf("unknown size") +// } + +// return r.r.ContentLength, nil +// } + +// // Read implements io.Reader. +// func (r bodyReader) Read(b []byte) (int, error) { +// return r.r.Body.Read(b) +// } + +// type validationFile struct { +// LastUpdate int64 // UnixMillis of last update time. +// Error string // Error message if validation failed. +// } + +// type blockUploadStateResult struct { +// State string `json:"result"` +// Error string `json:"error,omitempty"` +// } + +// type blockUploadState int + +// const ( +// blockStateUnknown blockUploadState = iota // unknown, default value +// blockIsComplete // meta.json file exists +// blockUploadNotStarted // meta.json doesn't exist, uploading-meta.json doesn't exist +// blockUploadInProgress // meta.json doesn't exist, but uploading-meta.json does +// blockValidationInProgress // meta.json doesn't exist, uploading-meta.json exists, validation.json exists and is recent +// blockValidationFailed +// blockValidationStale +// ) + +// func (c *MultitenantCompactor) GetBlockUploadStateHandler(w http.ResponseWriter, r *http.Request) { +// blockID, tenantID, err := c.parseBlockUploadParameters(r) +// if err != nil { +// http.Error(w, err.Error(), http.StatusBadRequest) +// return +// } +// requestID := hexTimeNowNano() +// logger := log.With( +// util_log.WithContext(r.Context(), c.logger), +// "feature", "block upload", +// "block", blockID, +// "operation", "get block state", +// "request_id", requestID, +// ) + +// userBkt := bucket.NewUserBucketClient(tenantID, c.bucketClient, c.cfgProvider) +// if err != nil { +// http.Error(w, err.Error(), http.StatusBadRequest) +// return +// } + +// s, _, v, err := c.getBlockUploadState(r.Context(), userBkt, blockID) +// if err != nil { +// writeBlockUploadError(err, "can't get upload state", logger, w, requestID) +// return +// } + +// res := blockUploadStateResult{} + +// switch s { +// case blockIsComplete: +// res.State = "complete" +// case blockUploadNotStarted: +// http.Error(w, "block doesn't exist", http.StatusNotFound) +// return +// case blockValidationStale: +// fallthrough +// case blockUploadInProgress: +// res.State = "uploading" +// case blockValidationInProgress: +// res.State = "validating" +// case blockValidationFailed: +// res.State = "failed" +// res.Error = v.Error +// } + +// util.WriteJSONResponse(w, res) +// } + +// // checkBlockState checks blocks state and returns various HTTP status codes for individual states if block +// // upload cannot start, finish or file cannot be uploaded to the block. +// func (c *MultitenantCompactor) checkBlockState(ctx context.Context, userBkt objstore.Bucket, blockID ulid.ULID, requireUploadInProgress bool) (*block.Meta, *validationFile, error) { +// s, m, v, err := c.getBlockUploadState(ctx, userBkt, blockID) +// if err != nil { +// return m, v, err +// } + +// switch s { +// case blockIsComplete: +// return m, v, httpError{message: "block already exists", statusCode: http.StatusConflict} +// case blockValidationInProgress: +// return m, v, httpError{message: "block validation in progress", statusCode: http.StatusBadRequest} +// case blockUploadNotStarted: +// if requireUploadInProgress { +// return m, v, httpError{message: "block upload not started", statusCode: http.StatusNotFound} +// } +// return m, v, nil +// case blockValidationStale: +// // if validation is stale, we treat block as being in "upload in progress" state, and validation can start again. +// fallthrough +// case blockUploadInProgress: +// return m, v, nil +// case blockValidationFailed: +// return m, v, httpError{message: "block validation failed", statusCode: http.StatusBadRequest} +// } + +// return m, v, httpError{message: "unknown block upload state", statusCode: http.StatusInternalServerError} +// } + +// // getBlockUploadState returns state of the block upload, and meta and validation objects, if they exist. +// func (c *MultitenantCompactor) getBlockUploadState(ctx context.Context, userBkt objstore.Bucket, blockID ulid.ULID) (blockUploadState, *block.Meta, *validationFile, error) { +// exists, err := userBkt.Exists(ctx, path.Join(blockID.String(), block.MetaFilename)) +// if err != nil { +// return blockStateUnknown, nil, nil, err +// } +// if exists { +// return blockIsComplete, nil, nil, nil +// } + +// meta, err := c.loadUploadingMeta(ctx, userBkt, blockID) +// if err != nil { +// return blockStateUnknown, nil, nil, err +// } +// // If neither meta.json nor uploading-meta.json file exist, we say that the block doesn't exist. +// if meta == nil { +// return blockUploadNotStarted, nil, nil, err +// } + +// v, err := c.loadValidation(ctx, userBkt, blockID) +// if err != nil { +// return blockStateUnknown, meta, nil, err +// } +// if v == nil { +// return blockUploadInProgress, meta, nil, err +// } +// if v.Error != "" { +// return blockValidationFailed, meta, v, err +// } +// if time.Since(time.UnixMilli(v.LastUpdate)) < validationHeartbeatTimeout { +// return blockValidationInProgress, meta, v, nil +// } +// return blockValidationStale, meta, v, nil +// } + +// func (c *MultitenantCompactor) loadUploadingMeta(ctx context.Context, userBkt objstore.Bucket, blockID ulid.ULID) (*block.Meta, error) { +// r, err := userBkt.Get(ctx, path.Join(blockID.String(), uploadingMetaFilename)) +// if err != nil { +// if userBkt.IsObjNotFoundErr(err) { +// return nil, nil +// } +// return nil, err +// } +// defer func() { _ = r.Close() }() + +// v := &block.Meta{} +// err = json.NewDecoder(r).Decode(v) +// if err != nil { +// return nil, err +// } + +// return v, nil +// } + +// func (c *MultitenantCompactor) loadValidation(ctx context.Context, userBkt objstore.Bucket, blockID ulid.ULID) (*validationFile, error) { +// r, err := userBkt.Get(ctx, path.Join(blockID.String(), validationFilename)) +// if err != nil { +// if userBkt.IsObjNotFoundErr(err) { +// return nil, nil +// } +// return nil, err +// } +// defer func() { _ = r.Close() }() + +// v := &validationFile{} +// err = json.NewDecoder(r).Decode(v) +// if err != nil { +// return nil, err +// } + +// return v, nil +// } + +// func (c *MultitenantCompactor) uploadValidationWithError(ctx context.Context, blockID ulid.ULID, +// userBkt objstore.Bucket, errorStr string, +// ) error { +// val := validationFile{ +// LastUpdate: time.Now().UnixMilli(), +// Error: errorStr, +// } +// dst := path.Join(blockID.String(), validationFilename) +// if err := marshalAndUploadToBucket(ctx, userBkt, dst, val); err != nil { +// return errors.Wrapf(err, "failed uploading %s to bucket", validationFilename) +// } +// return nil +// } + +// func (c *MultitenantCompactor) uploadValidation(ctx context.Context, blockID ulid.ULID, userBkt objstore.Bucket) error { +// return c.uploadValidationWithError(ctx, blockID, userBkt, "") +// } + +// func (c *MultitenantCompactor) periodicValidationUpdater(ctx context.Context, logger log.Logger, blockID ulid.ULID, userBkt objstore.Bucket, cancelFn func(), interval time.Duration) { +// ticker := time.NewTicker(interval) +// defer ticker.Stop() +// for { +// select { +// case <-ctx.Done(): +// return +// case <-ticker.C: +// if err := c.uploadValidation(ctx, blockID, userBkt); err != nil { +// level.Warn(logger).Log("msg", "error during periodic update of validation file", "err", err) +// cancelFn() +// return +// } +// } +// } +// } + +// func marshalAndUploadToBucket(ctx context.Context, bkt objstore.Bucket, pth string, val interface{}) error { +// buf, err := json.Marshal(val) +// if err != nil { +// return err +// } +// if err := bkt.Upload(ctx, pth, bytes.NewReader(buf)); err != nil { +// return err +// } +// return nil +// } diff --git a/pkg/compactor/block_upload_test.go b/pkg/compactor/block_upload_test.go index b6395e4e06..87a93d4926 100644 --- a/pkg/compactor/block_upload_test.go +++ b/pkg/compactor/block_upload_test.go @@ -1,2116 +1,2116 @@ -// SPDX-License-Identifier: AGPL-3.0-only -// Provenance-includes-location: https://github.com/grafana/mimir/blob/main/pkg/compactor/block_upload_test.go -// Provenance-includes-license: Apache-2.0 -// Provenance-includes-copyright: The Cortex Authors. +// // SPDX-License-Identifier: AGPL-3.0-only +// // Provenance-includes-location: https://github.com/grafana/mimir/blob/main/pkg/compactor/block_upload_test.go +// // Provenance-includes-license: Apache-2.0 +// // Provenance-includes-copyright: The Cortex Authors. package compactor -import ( - "bytes" - "context" - "encoding/json" - "fmt" - "io" - "math" - "net/http" - "net/http/httptest" - "net/url" - "os" - "path" - "path/filepath" - "strings" - "sync" - "testing" - "time" - - "github.com/go-kit/log" - "github.com/gorilla/mux" - "github.com/grafana/dskit/test" - "github.com/grafana/dskit/user" - "github.com/oklog/ulid" - "github.com/pkg/errors" - "github.com/prometheus/client_golang/prometheus" - "github.com/prometheus/client_golang/prometheus/promauto" - promtest "github.com/prometheus/client_golang/prometheus/testutil" - "github.com/prometheus/prometheus/model/labels" - "github.com/prometheus/prometheus/tsdb" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/mock" - "github.com/stretchr/testify/require" - "github.com/thanos-io/objstore" - - "github.com/grafana/mimir/pkg/storage/bucket" - mimir_tsdb "github.com/grafana/mimir/pkg/storage/tsdb" - "github.com/grafana/mimir/pkg/storage/tsdb/block" -) - -func verifyUploadedMeta(t *testing.T, bkt *bucket.ClientMock, expMeta block.Meta) { - var call mock.Call - for _, c := range bkt.Calls { - if c.Method == "Upload" { - call = c - break - } - } - - rdr := call.Arguments[2].(io.Reader) - var gotMeta block.Meta - require.NoError(t, json.NewDecoder(rdr).Decode(&gotMeta)) - assert.Equal(t, expMeta, gotMeta) -} - -// Test MultitenantCompactor.StartBlockUpload -func TestMultitenantCompactor_StartBlockUpload(t *testing.T) { - const tenantID = "test" - const blockID = "01G3FZ0JWJYJC0ZM6Y9778P6KD" - bULID := ulid.MustParse(blockID) - now := time.Now().UnixMilli() - validMeta := block.Meta{ - BlockMeta: tsdb.BlockMeta{ - ULID: bULID, - Version: block.TSDBVersion1, - MinTime: now - 1000, - MaxTime: now, - }, - Thanos: block.ThanosMeta{ - Labels: map[string]string{ - mimir_tsdb.CompactorShardIDExternalLabel: "1_of_3", - }, - Files: []block.File{ - { - RelPath: block.MetaFilename, - }, - { - RelPath: "index", - SizeBytes: 1, - }, - { - RelPath: "chunks/000001", - SizeBytes: 1024, - }, - }, - }, - } - - metaPath := path.Join(tenantID, blockID, block.MetaFilename) - uploadingMetaPath := path.Join(tenantID, blockID, fmt.Sprintf("uploading-%s", block.MetaFilename)) - - setUpPartialBlock := func(bkt *bucket.ClientMock) { - bkt.MockExists(path.Join(tenantID, blockID, block.MetaFilename), false, nil) - setUpGet(bkt, path.Join(tenantID, blockID, uploadingMetaFilename), nil, bucket.ErrObjectDoesNotExist) - } - setUpUpload := func(bkt *bucket.ClientMock) { - setUpPartialBlock(bkt) - bkt.MockUpload(uploadingMetaPath, nil) - } - - verifyUpload := func(t *testing.T, bkt *bucket.ClientMock, labels map[string]string) { - t.Helper() - - expMeta := validMeta - expMeta.Compaction.Parents = nil - expMeta.Compaction.Sources = []ulid.ULID{expMeta.ULID} - expMeta.Thanos.Source = "upload" - expMeta.Thanos.Labels = labels - verifyUploadedMeta(t, bkt, expMeta) - } - - testCases := []struct { - name string - tenantID string - blockID string - body string - meta *block.Meta - retention time.Duration - disableBlockUpload bool - expBadRequest string - expConflict string - expUnprocessableEntity string - expEntityTooLarge string - expInternalServerError bool - setUpBucketMock func(bkt *bucket.ClientMock) - verifyUpload func(*testing.T, *bucket.ClientMock) - maxBlockUploadSizeBytes int64 - }{ - { - name: "missing tenant ID", - tenantID: "", - blockID: blockID, - expBadRequest: "invalid tenant ID", - }, - { - name: "missing block ID", - tenantID: tenantID, - blockID: "", - expBadRequest: "invalid block ID", - }, - { - name: "invalid block ID", - tenantID: tenantID, - blockID: "1234", - expBadRequest: "invalid block ID", - }, - { - name: "missing body", - tenantID: tenantID, - blockID: blockID, - expBadRequest: "malformed request body", - setUpBucketMock: setUpPartialBlock, - }, - { - name: "malformed body", - tenantID: tenantID, - blockID: blockID, - setUpBucketMock: setUpPartialBlock, - body: "{", - expBadRequest: "malformed request body", - }, - { - name: "invalid file path", - tenantID: tenantID, - blockID: blockID, - setUpBucketMock: setUpPartialBlock, - meta: &block.Meta{ - Thanos: block.ThanosMeta{ - Files: []block.File{ - { - RelPath: "chunks/invalid-file", - SizeBytes: 1024, - }, - }, - }, - }, - expBadRequest: "file with invalid path: chunks/invalid-file", - }, - { - name: "contains downsampled data", - tenantID: tenantID, - blockID: blockID, - setUpBucketMock: setUpPartialBlock, - meta: &block.Meta{ - Thanos: block.ThanosMeta{ - Downsample: block.ThanosDownsample{ - Resolution: 1000, - }, - Files: []block.File{ - { - RelPath: block.MetaFilename, - }, - { - RelPath: "index", - SizeBytes: 1, - }, - { - RelPath: "chunks/000001", - }, - }, - }, - }, - expBadRequest: "block contains downsampled data", - }, - { - name: "missing file size", - tenantID: tenantID, - blockID: blockID, - setUpBucketMock: setUpPartialBlock, - meta: &block.Meta{ - Thanos: block.ThanosMeta{ - Files: []block.File{ - { - RelPath: block.MetaFilename, - }, - { - RelPath: "index", - SizeBytes: 1, - }, - { - RelPath: "chunks/000001", - }, - }, - }, - }, - expBadRequest: "file with invalid size: chunks/000001", - }, - { - name: "invalid minTime", - tenantID: tenantID, - blockID: blockID, - setUpBucketMock: setUpPartialBlock, - meta: &block.Meta{ - BlockMeta: tsdb.BlockMeta{ - ULID: bULID, - Version: block.TSDBVersion1, - MinTime: -1, - MaxTime: 0, - }, - }, - expBadRequest: "invalid minTime/maxTime: minTime=-1, maxTime=0", - }, - { - name: "invalid maxTime", - tenantID: tenantID, - blockID: blockID, - setUpBucketMock: setUpPartialBlock, - meta: &block.Meta{ - BlockMeta: tsdb.BlockMeta{ - ULID: bULID, - Version: block.TSDBVersion1, - MinTime: 0, - MaxTime: -1, - }, - }, - expBadRequest: "invalid minTime/maxTime: minTime=0, maxTime=-1", - }, - { - name: "maxTime before minTime", - tenantID: tenantID, - blockID: blockID, - setUpBucketMock: setUpPartialBlock, - meta: &block.Meta{ - BlockMeta: tsdb.BlockMeta{ - ULID: bULID, - Version: block.TSDBVersion1, - MinTime: 1, - MaxTime: 0, - }, - }, - expBadRequest: "invalid minTime/maxTime: minTime=1, maxTime=0", - }, - { - name: "block before retention period", - tenantID: tenantID, - blockID: blockID, - retention: 10 * time.Second, - setUpBucketMock: setUpPartialBlock, - meta: &block.Meta{ - BlockMeta: tsdb.BlockMeta{ - ULID: bULID, - Version: block.TSDBVersion1, - MinTime: 0, - MaxTime: 1000, - }, - }, - expUnprocessableEntity: "block max time (1970-01-01 00:00:01 +0000 UTC) older than retention period", - }, - { - name: "invalid version", - tenantID: tenantID, - blockID: blockID, - setUpBucketMock: setUpPartialBlock, - meta: &block.Meta{ - BlockMeta: tsdb.BlockMeta{ - ULID: bULID, - Version: 0, - }, - }, - expBadRequest: fmt.Sprintf("version must be %d", block.TSDBVersion1), - }, - { - name: "ignore retention period if == 0", - tenantID: tenantID, - blockID: blockID, - retention: 0, - setUpBucketMock: setUpUpload, - meta: &block.Meta{ - BlockMeta: tsdb.BlockMeta{ - ULID: bULID, - Version: block.TSDBVersion1, - MinTime: 0, - MaxTime: 1000, - }, - Thanos: block.ThanosMeta{ - Labels: map[string]string{ - mimir_tsdb.CompactorShardIDExternalLabel: "1_of_3", - }, - Files: []block.File{ - { - RelPath: block.MetaFilename, - }, - { - RelPath: "index", - SizeBytes: 1, - }, - { - RelPath: "chunks/000001", - SizeBytes: 1024, - }, - }, - }, - }, - }, - { - name: "ignore retention period if < 0", - tenantID: tenantID, - blockID: blockID, - retention: -1, - setUpBucketMock: setUpUpload, - meta: &block.Meta{ - BlockMeta: tsdb.BlockMeta{ - ULID: bULID, - Version: block.TSDBVersion1, - MinTime: 0, - MaxTime: 1000, - }, - Thanos: block.ThanosMeta{ - Labels: map[string]string{ - mimir_tsdb.CompactorShardIDExternalLabel: "1_of_3", - }, - Files: []block.File{ - { - RelPath: block.MetaFilename, - }, - { - RelPath: "index", - SizeBytes: 1, - }, - { - RelPath: "chunks/000001", - SizeBytes: 1024, - }, - }, - }, - }, - }, - { - name: "invalid compactor shard ID label", - tenantID: tenantID, - blockID: blockID, - setUpBucketMock: setUpPartialBlock, - meta: &block.Meta{ - BlockMeta: tsdb.BlockMeta{ - ULID: bULID, - Version: block.TSDBVersion1, - }, - Thanos: block.ThanosMeta{ - Labels: map[string]string{ - mimir_tsdb.CompactorShardIDExternalLabel: "test", - }, - }, - }, - expBadRequest: fmt.Sprintf(`invalid %s external label: "test"`, mimir_tsdb.CompactorShardIDExternalLabel), - }, - { - name: "failure checking for complete block", - tenantID: tenantID, - blockID: blockID, - setUpBucketMock: func(bkt *bucket.ClientMock) { - bkt.MockExists(path.Join(tenantID, blockID, block.MetaFilename), false, fmt.Errorf("test")) - }, - expInternalServerError: true, - }, - { - name: "complete block already exists", - tenantID: tenantID, - blockID: blockID, - setUpBucketMock: func(bkt *bucket.ClientMock) { - bkt.MockExists(path.Join(tenantID, blockID, block.MetaFilename), true, nil) - }, - expConflict: "block already exists", - }, - { - name: "failure uploading meta file", - tenantID: tenantID, - blockID: blockID, - setUpBucketMock: func(bkt *bucket.ClientMock) { - setUpPartialBlock(bkt) - bkt.MockUpload(uploadingMetaPath, fmt.Errorf("test")) - }, - meta: &validMeta, - expInternalServerError: true, - }, - { - name: "too large of a request body", - tenantID: tenantID, - blockID: blockID, - setUpBucketMock: setUpPartialBlock, - body: strings.Repeat("A", maximumMetaSizeBytes+1), - expEntityTooLarge: fmt.Sprintf("The block metadata was too large (maximum size allowed is %d bytes)", maximumMetaSizeBytes), - }, - { - name: "block upload disabled", - tenantID: tenantID, - blockID: blockID, - disableBlockUpload: true, - expBadRequest: "block upload is disabled", - }, - { - name: "max block size exceeded", - tenantID: tenantID, - blockID: blockID, - setUpBucketMock: setUpPartialBlock, - meta: &validMeta, - maxBlockUploadSizeBytes: 1, - expBadRequest: fmt.Sprintf(maxBlockUploadSizeBytesFormat, 1), - }, - { - name: "valid request", - tenantID: tenantID, - blockID: blockID, - setUpBucketMock: setUpUpload, - meta: &validMeta, - verifyUpload: func(t *testing.T, bkt *bucket.ClientMock) { - verifyUpload(t, bkt, map[string]string{ - mimir_tsdb.CompactorShardIDExternalLabel: "1_of_3", - }) - }, - }, - { - name: "valid request with empty compactor shard ID label", - tenantID: tenantID, - blockID: blockID, - setUpBucketMock: setUpUpload, - meta: &block.Meta{ - BlockMeta: tsdb.BlockMeta{ - ULID: bULID, - Version: block.TSDBVersion1, - MinTime: now - 1000, - MaxTime: now, - }, - Thanos: block.ThanosMeta{ - Labels: map[string]string{ - mimir_tsdb.CompactorShardIDExternalLabel: "", - }, - Files: []block.File{ - { - RelPath: block.MetaFilename, - }, - { - RelPath: "index", - SizeBytes: 1, - }, - { - RelPath: "chunks/000001", - SizeBytes: 1024, - }, - }, - }, - }, - verifyUpload: func(t *testing.T, bkt *bucket.ClientMock) { - verifyUpload(t, bkt, map[string]string{}) - }, - }, - { - name: "valid request without compactor shard ID label", - tenantID: tenantID, - blockID: blockID, - setUpBucketMock: setUpUpload, - meta: &block.Meta{ - BlockMeta: tsdb.BlockMeta{ - ULID: bULID, - Version: block.TSDBVersion1, - MinTime: now - 1000, - MaxTime: now, - }, - Thanos: block.ThanosMeta{ - Files: []block.File{ - { - RelPath: block.MetaFilename, - }, - { - RelPath: "index", - SizeBytes: 1, - }, - { - RelPath: "chunks/000001", - SizeBytes: 1024, - }, - }, - }, - }, - verifyUpload: func(t *testing.T, bkt *bucket.ClientMock) { - verifyUpload(t, bkt, nil) - }, - }, - { - name: "valid request with different block ID in meta file", - tenantID: tenantID, - blockID: blockID, - setUpBucketMock: setUpUpload, - meta: &block.Meta{ - BlockMeta: tsdb.BlockMeta{ - ULID: ulid.MustParse("11A2FZ0JWJYJC0ZM6Y9778P6KD"), - Version: block.TSDBVersion1, - MinTime: now - 1000, - MaxTime: now, - }, - Thanos: block.ThanosMeta{ - Files: []block.File{ - { - RelPath: block.MetaFilename, - }, - { - RelPath: "index", - SizeBytes: 1, - }, - { - RelPath: "chunks/000001", - SizeBytes: 1024, - }, - }, - }, - }, - verifyUpload: func(t *testing.T, bkt *bucket.ClientMock) { - verifyUpload(t, bkt, nil) - }, - }, - } - for _, tc := range testCases { - t.Run(tc.name, func(t *testing.T) { - var bkt bucket.ClientMock - if tc.setUpBucketMock != nil { - tc.setUpBucketMock(&bkt) - } - - cfgProvider := newMockConfigProvider() - cfgProvider.userRetentionPeriods[tenantID] = tc.retention - cfgProvider.blockUploadEnabled[tenantID] = !tc.disableBlockUpload - cfgProvider.blockUploadMaxBlockSizeBytes[tenantID] = tc.maxBlockUploadSizeBytes - c := &MultitenantCompactor{ - logger: log.NewNopLogger(), - bucketClient: &bkt, - cfgProvider: cfgProvider, - } - var rdr io.Reader - if tc.body != "" { - rdr = strings.NewReader(tc.body) - } else if tc.meta != nil { - buf := bytes.NewBuffer(nil) - require.NoError(t, json.NewEncoder(buf).Encode(tc.meta)) - rdr = buf - } - r := httptest.NewRequest(http.MethodPost, fmt.Sprintf("/api/v1/upload/block/%s/start", tc.blockID), rdr) - if tc.tenantID != "" { - r = r.WithContext(user.InjectOrgID(r.Context(), tc.tenantID)) - } - if tc.blockID != "" { - r = mux.SetURLVars(r, map[string]string{"block": tc.blockID}) - } - w := httptest.NewRecorder() - c.StartBlockUpload(w, r) - - resp := w.Result() - body, err := io.ReadAll(resp.Body) - require.NoError(t, err) - - switch { - case tc.expInternalServerError: - assert.Equal(t, http.StatusInternalServerError, resp.StatusCode) - assert.Regexp(t, "internal server error \\(id [0-9a-f]{16}\\)\n", string(body)) - case tc.expBadRequest != "": - assert.Equal(t, http.StatusBadRequest, resp.StatusCode) - assert.Equal(t, fmt.Sprintf("%s\n", tc.expBadRequest), string(body)) - case tc.expConflict != "": - assert.Equal(t, http.StatusConflict, resp.StatusCode) - assert.Equal(t, fmt.Sprintf("%s\n", tc.expConflict), string(body)) - case tc.expUnprocessableEntity != "": - assert.Equal(t, http.StatusUnprocessableEntity, resp.StatusCode) - assert.Equal(t, fmt.Sprintf("%s\n", tc.expUnprocessableEntity), string(body)) - case tc.expEntityTooLarge != "": - assert.Equal(t, http.StatusRequestEntityTooLarge, resp.StatusCode) - assert.Equal(t, fmt.Sprintf("%s\n", tc.expEntityTooLarge), string(body)) - default: - assert.Equal(t, http.StatusOK, resp.StatusCode) - assert.Empty(t, string(body)) - } - - bkt.AssertExpectations(t) - - if tc.verifyUpload != nil { - tc.verifyUpload(t, &bkt) - } - }) - } - - downloadMeta := func(t *testing.T, bkt *objstore.InMemBucket, pth string) block.Meta { - t.Helper() - - ctx := context.Background() - rdr, err := bkt.Get(ctx, pth) - require.NoError(t, err) - t.Cleanup(func() { - _ = rdr.Close() - }) - var gotMeta block.Meta - require.NoError(t, json.NewDecoder(rdr).Decode(&gotMeta)) - return gotMeta - } - - // Additional test cases using an in-memory bucket for state testing - extraCases := []struct { - name string - setUp func(*testing.T, *objstore.InMemBucket) block.Meta - verifyBucket func(*testing.T, *objstore.InMemBucket) - expBadRequest string - expConflict string - }{ - { - name: "valid request when both in-flight meta file and complete meta file exist in object storage", - setUp: func(t *testing.T, bkt *objstore.InMemBucket) block.Meta { - marshalAndUploadJSON(t, bkt, uploadingMetaPath, validMeta) - marshalAndUploadJSON(t, bkt, metaPath, validMeta) - return validMeta - }, - verifyBucket: func(t *testing.T, bkt *objstore.InMemBucket) { - assert.Equal(t, validMeta, downloadMeta(t, bkt, uploadingMetaPath)) - assert.Equal(t, validMeta, downloadMeta(t, bkt, metaPath)) - }, - expConflict: "block already exists", - }, - { - name: "invalid request when in-flight meta file exists in object storage", - setUp: func(t *testing.T, bkt *objstore.InMemBucket) block.Meta { - marshalAndUploadJSON(t, bkt, uploadingMetaPath, validMeta) - - meta := validMeta - // Invalid version - meta.Version = 0 - return meta - }, - verifyBucket: func(t *testing.T, bkt *objstore.InMemBucket) { - assert.Equal(t, validMeta, downloadMeta(t, bkt, uploadingMetaPath)) - }, - expBadRequest: fmt.Sprintf("version must be %d", block.TSDBVersion1), - }, - { - name: "valid request when same in-flight meta file exists in object storage", - setUp: func(t *testing.T, bkt *objstore.InMemBucket) block.Meta { - marshalAndUploadJSON(t, bkt, uploadingMetaPath, validMeta) - return validMeta - }, - verifyBucket: func(t *testing.T, bkt *objstore.InMemBucket) { - expMeta := validMeta - expMeta.Compaction.Sources = []ulid.ULID{expMeta.ULID} - expMeta.Thanos.Source = "upload" - assert.Equal(t, expMeta, downloadMeta(t, bkt, uploadingMetaPath)) - }, - }, - { - name: "valid request when different in-flight meta file exists in object storage", - setUp: func(t *testing.T, bkt *objstore.InMemBucket) block.Meta { - meta := validMeta - meta.MinTime -= 1000 - meta.MaxTime -= 1000 - marshalAndUploadJSON(t, bkt, uploadingMetaPath, meta) - - // Return meta file that differs from the one in bucket - return validMeta - }, - verifyBucket: func(t *testing.T, bkt *objstore.InMemBucket) { - expMeta := validMeta - expMeta.Compaction.Sources = []ulid.ULID{expMeta.ULID} - expMeta.Thanos.Source = "upload" - assert.Equal(t, expMeta, downloadMeta(t, bkt, uploadingMetaPath)) - }, - }, - } - for _, tc := range extraCases { - t.Run(tc.name, func(t *testing.T) { - bkt := objstore.NewInMemBucket() - meta := tc.setUp(t, bkt) - metaJSON, err := json.Marshal(meta) - require.NoError(t, err) - - cfgProvider := newMockConfigProvider() - cfgProvider.blockUploadEnabled[tenantID] = true - c := &MultitenantCompactor{ - logger: log.NewNopLogger(), - bucketClient: bkt, - cfgProvider: cfgProvider, - } - r := httptest.NewRequest(http.MethodPost, fmt.Sprintf("/api/v1/upload/block/%s/start", blockID), bytes.NewReader(metaJSON)) - r = r.WithContext(user.InjectOrgID(r.Context(), tenantID)) - r = mux.SetURLVars(r, map[string]string{"block": blockID}) - w := httptest.NewRecorder() - c.StartBlockUpload(w, r) - - resp := w.Result() - body, err := io.ReadAll(resp.Body) - require.NoError(t, err) - switch { - case tc.expBadRequest != "": - assert.Equal(t, http.StatusBadRequest, resp.StatusCode) - assert.Equal(t, fmt.Sprintf("%s\n", tc.expBadRequest), string(body)) - case tc.expConflict != "": - assert.Equal(t, http.StatusConflict, resp.StatusCode) - assert.Equal(t, fmt.Sprintf("%s\n", tc.expConflict), string(body)) - default: - assert.Equal(t, http.StatusOK, resp.StatusCode) - assert.Empty(t, string(body)) - } - }) - } -} - -// Test MultitenantCompactor.UploadBlockFile -func TestMultitenantCompactor_UploadBlockFile(t *testing.T) { - const tenantID = "test" - const blockID = "01G3FZ0JWJYJC0ZM6Y9778P6KD" - uploadingMetaFilename := fmt.Sprintf("uploading-%s", block.MetaFilename) - uploadingMetaPath := path.Join(tenantID, blockID, uploadingMetaFilename) - metaPath := path.Join(tenantID, blockID, block.MetaFilename) - - chunkBodyContent := "content" - validMeta := block.Meta{ - BlockMeta: tsdb.BlockMeta{ - ULID: ulid.MustParse(blockID), - }, - Thanos: block.ThanosMeta{ - Labels: map[string]string{ - mimir_tsdb.CompactorShardIDExternalLabel: "1_of_3", - }, - Files: []block.File{ - { - RelPath: "index", - SizeBytes: 1, - }, - { - RelPath: "chunks/000001", - SizeBytes: int64(len(chunkBodyContent)), - }, - }, - }, - } - - setupFnForValidRequest := func(bkt *bucket.ClientMock) { - bkt.MockExists(metaPath, false, nil) - - b, err := json.Marshal(validMeta) - setUpGet(bkt, path.Join(tenantID, blockID, uploadingMetaFilename), b, err) - setUpGet(bkt, path.Join(tenantID, blockID, validationFilename), nil, bucket.ErrObjectDoesNotExist) - - bkt.MockUpload(path.Join(tenantID, blockID, "chunks/000001"), nil) - } - - verifyFuncForValidRequest := func(t *testing.T, bkt *bucket.ClientMock, expContent string) { - var call mock.Call - for _, c := range bkt.Calls { - if c.Method == "Upload" { - call = c - break - } - } - - rdr := call.Arguments[2].(io.Reader) - got, err := io.ReadAll(rdr) - require.NoError(t, err) - assert.Equal(t, []byte(expContent), got) - } - - testCases := []struct { - name string - tenantID string - blockID string - path string - body string - unknownContentLength bool - disableBlockUpload bool - expBadRequest string - expConflict string - expNotFound string - expInternalServerError bool - setUpBucketMock func(bkt *bucket.ClientMock) - verifyUpload func(*testing.T, *bucket.ClientMock, string) - }{ - { - name: "without tenant ID", - blockID: blockID, - path: "chunks/000001", - expBadRequest: "invalid tenant ID", - }, - { - name: "without block ID", - tenantID: tenantID, - path: "chunks/000001", - expBadRequest: "invalid block ID", - }, - { - name: "invalid block ID", - tenantID: tenantID, - blockID: "1234", - path: "chunks/000001", - expBadRequest: "invalid block ID", - }, - { - name: "without path", - tenantID: tenantID, - blockID: blockID, - expBadRequest: "missing or invalid file path", - }, - { - name: "invalid path", - tenantID: tenantID, - blockID: blockID, - path: "../chunks/000001", - expBadRequest: `invalid path: "../chunks/000001"`, - }, - { - name: "empty file", - tenantID: tenantID, - blockID: blockID, - path: "chunks/000001", - expBadRequest: "file cannot be empty", - }, - { - name: "attempt block metadata file", - tenantID: tenantID, - blockID: blockID, - path: block.MetaFilename, - body: "content", - expBadRequest: fmt.Sprintf("%s is not allowed", block.MetaFilename), - }, - { - name: "attempt in-flight block metadata file", - tenantID: tenantID, - blockID: blockID, - path: uploadingMetaFilename, - body: "content", - expBadRequest: fmt.Sprintf("invalid path: %q", uploadingMetaFilename), - }, - { - name: "block upload disabled", - tenantID: tenantID, - blockID: blockID, - disableBlockUpload: true, - path: "chunks/000001", - expBadRequest: "block upload is disabled", - }, - { - name: "complete block already exists", - tenantID: tenantID, - blockID: blockID, - path: "chunks/000001", - body: "content", - setUpBucketMock: func(bkt *bucket.ClientMock) { - bkt.MockExists(metaPath, true, nil) - }, - expConflict: "block already exists", - }, - { - name: "failure checking for complete block", - tenantID: tenantID, - blockID: blockID, - path: "chunks/000001", - body: chunkBodyContent, - setUpBucketMock: func(bkt *bucket.ClientMock) { - bkt.MockExists(metaPath, false, fmt.Errorf("test")) - }, - expInternalServerError: true, - }, - { - name: "failure checking for in-flight meta file", - tenantID: tenantID, - blockID: blockID, - path: "chunks/000001", - body: chunkBodyContent, - setUpBucketMock: func(bkt *bucket.ClientMock) { - bkt.MockExists(metaPath, false, nil) - setUpGet(bkt, path.Join(tenantID, blockID, uploadingMetaFilename), nil, fmt.Errorf("test")) - }, - expInternalServerError: true, - }, - { - name: "missing in-flight meta file", - tenantID: tenantID, - blockID: blockID, - path: "chunks/000001", - body: chunkBodyContent, - setUpBucketMock: func(bkt *bucket.ClientMock) { - bkt.MockExists(metaPath, false, nil) - setUpGet(bkt, path.Join(tenantID, blockID, uploadingMetaFilename), nil, bucket.ErrObjectDoesNotExist) - }, - expNotFound: "block upload not started", - }, - { - name: "file upload fails", - tenantID: tenantID, - blockID: blockID, - path: "chunks/000001", - body: chunkBodyContent, - setUpBucketMock: func(bkt *bucket.ClientMock) { - bkt.MockExists(metaPath, false, nil) - - b, err := json.Marshal(validMeta) - setUpGet(bkt, path.Join(tenantID, blockID, uploadingMetaFilename), b, err) - setUpGet(bkt, path.Join(tenantID, blockID, validationFilename), nil, bucket.ErrObjectDoesNotExist) - - bkt.MockUpload(path.Join(tenantID, blockID, "chunks/000001"), fmt.Errorf("test")) - }, - expInternalServerError: true, - }, - { - name: "invalid file size", - tenantID: tenantID, - blockID: blockID, - path: "chunks/000001", - body: chunkBodyContent + chunkBodyContent, - setUpBucketMock: func(bkt *bucket.ClientMock) { - bkt.MockExists(metaPath, false, nil) - - b, err := json.Marshal(validMeta) - setUpGet(bkt, path.Join(tenantID, blockID, uploadingMetaFilename), b, err) - setUpGet(bkt, path.Join(tenantID, blockID, validationFilename), nil, bucket.ErrObjectDoesNotExist) - }, - expBadRequest: "file size doesn't match meta.json", - }, - { - name: "unexpected file", - tenantID: tenantID, - blockID: blockID, - path: "chunks/111111", - body: chunkBodyContent, - setUpBucketMock: func(bkt *bucket.ClientMock) { - bkt.MockExists(metaPath, false, nil) - - b, err := json.Marshal(validMeta) - setUpGet(bkt, path.Join(tenantID, blockID, uploadingMetaFilename), b, err) - setUpGet(bkt, path.Join(tenantID, blockID, validationFilename), nil, bucket.ErrObjectDoesNotExist) - }, - expBadRequest: "unexpected file", - }, - { - name: "valid request", - tenantID: tenantID, - blockID: blockID, - path: "chunks/000001", - body: chunkBodyContent, - setUpBucketMock: setupFnForValidRequest, - verifyUpload: verifyFuncForValidRequest, - }, - { - name: "valid request, with unknown content-length", - tenantID: tenantID, - blockID: blockID, - path: "chunks/000001", - body: chunkBodyContent, - unknownContentLength: true, - setUpBucketMock: setupFnForValidRequest, - verifyUpload: verifyFuncForValidRequest, - }, - } - for _, tc := range testCases { - t.Run(tc.name, func(t *testing.T) { - var bkt bucket.ClientMock - if tc.setUpBucketMock != nil { - tc.setUpBucketMock(&bkt) - } - - cfgProvider := newMockConfigProvider() - cfgProvider.blockUploadEnabled[tc.tenantID] = !tc.disableBlockUpload - c := &MultitenantCompactor{ - logger: log.NewNopLogger(), - bucketClient: &bkt, - cfgProvider: cfgProvider, - } - var rdr io.Reader - if tc.body != "" { - rdr = strings.NewReader(tc.body) - } - r := httptest.NewRequest(http.MethodPost, fmt.Sprintf("/api/v1/upload/block/%s/files?path=%s", blockID, url.QueryEscape(tc.path)), rdr) - if tc.tenantID != "" { - r = r.WithContext(user.InjectOrgID(r.Context(), tenantID)) - } - if tc.blockID != "" { - r = mux.SetURLVars(r, map[string]string{"block": tc.blockID}) - } - if tc.body != "" { - r.ContentLength = int64(len(tc.body)) - if tc.unknownContentLength { - r.ContentLength = -1 - } - } - w := httptest.NewRecorder() - c.UploadBlockFile(w, r) - - resp := w.Result() - body, err := io.ReadAll(resp.Body) - require.NoError(t, err) - - switch { - case tc.expBadRequest != "": - assert.Equal(t, http.StatusBadRequest, resp.StatusCode) - assert.Equal(t, fmt.Sprintf("%s\n", tc.expBadRequest), string(body)) - case tc.expConflict != "": - assert.Equal(t, http.StatusConflict, resp.StatusCode) - assert.Equal(t, fmt.Sprintf("%s\n", tc.expConflict), string(body)) - case tc.expNotFound != "": - assert.Equal(t, http.StatusNotFound, resp.StatusCode) - assert.Equal(t, fmt.Sprintf("%s\n", tc.expNotFound), string(body)) - case tc.expInternalServerError: - assert.Equal(t, http.StatusInternalServerError, resp.StatusCode) - assert.Regexp(t, "internal server error \\(id [0-9a-f]{16}\\)\n", string(body)) - default: - assert.Equal(t, http.StatusOK, resp.StatusCode) - assert.Empty(t, string(body)) - } - - bkt.AssertExpectations(t) - - if tc.verifyUpload != nil { - tc.verifyUpload(t, &bkt, tc.body) - } - }) - } - - type file struct { - path string - content string - } - - // Additional test cases using an in-memory bucket for state testing - extraCases := []struct { - name string - files []file - setUpBucket func(*testing.T, *objstore.InMemBucket) - verifyBucket func(*testing.T, *objstore.InMemBucket, []file) - }{ - { - name: "multiple sequential uploads of same file", - files: []file{ - { - path: "chunks/000001", - content: strings.Repeat("a", len(chunkBodyContent)), - }, - { - path: "chunks/000001", - content: strings.Repeat("b", len(chunkBodyContent)), - }, - }, - setUpBucket: func(t *testing.T, bkt *objstore.InMemBucket) { - marshalAndUploadJSON(t, bkt, uploadingMetaPath, validMeta) - }, - verifyBucket: func(t *testing.T, bkt *objstore.InMemBucket, files []file) { - t.Helper() - - ctx := context.Background() - rdr, err := bkt.Get(ctx, path.Join(tenantID, blockID, files[1].path)) - require.NoError(t, err) - t.Cleanup(func() { - _ = rdr.Close() - }) - - content, err := io.ReadAll(rdr) - require.NoError(t, err) - assert.Equal(t, files[1].content, string(content)) - }, - }, - } - for _, tc := range extraCases { - t.Run(tc.name, func(t *testing.T) { - bkt := objstore.NewInMemBucket() - tc.setUpBucket(t, bkt) - cfgProvider := newMockConfigProvider() - cfgProvider.blockUploadEnabled[tenantID] = true - c := &MultitenantCompactor{ - logger: log.NewNopLogger(), - bucketClient: bkt, - cfgProvider: cfgProvider, - } - - for _, f := range tc.files { - rdr := strings.NewReader(f.content) - r := httptest.NewRequest(http.MethodPost, fmt.Sprintf("/api/v1/upload/block/%s/files?path=%s", blockID, url.QueryEscape(f.path)), rdr) - urlVars := map[string]string{ - "block": blockID, - } - r = mux.SetURLVars(r, urlVars) - r = r.WithContext(user.InjectOrgID(r.Context(), tenantID)) - w := httptest.NewRecorder() - c.UploadBlockFile(w, r) - - resp := w.Result() - body, err := io.ReadAll(resp.Body) - require.NoError(t, err) - require.Equal(t, http.StatusOK, resp.StatusCode) - require.Empty(t, body) - } - - tc.verifyBucket(t, bkt, tc.files) - }) - } -} - -func setUpGet(bkt *bucket.ClientMock, pth string, content []byte, err error) { - bkt.On("Get", mock.Anything, pth).Return(func(_ context.Context, _ string) (io.ReadCloser, error) { - return io.NopCloser(bytes.NewReader(content)), err - }) -} - -// Test MultitenantCompactor.FinishBlockUpload -func TestMultitenantCompactor_FinishBlockUpload(t *testing.T) { - const tenantID = "test" - const blockID = "01G3FZ0JWJYJC0ZM6Y9778P6KD" - uploadingMetaPath := path.Join(tenantID, blockID, uploadingMetaFilename) - metaPath := path.Join(tenantID, blockID, block.MetaFilename) - injectedError := fmt.Errorf("injected error") - validMeta := block.Meta{ - BlockMeta: tsdb.BlockMeta{ - Version: block.TSDBVersion1, - ULID: ulid.MustParse(blockID), - }, - Thanos: block.ThanosMeta{ - Labels: map[string]string{ - mimir_tsdb.CompactorShardIDExternalLabel: "1_of_3", - }, - Files: []block.File{ - { - RelPath: "index", - SizeBytes: 1, - }, - { - RelPath: "chunks/000001", - SizeBytes: 2, - }, - }, - }, - } - - validSetup := func(t *testing.T, bkt objstore.Bucket) { - err := marshalAndUploadToBucket(context.Background(), bkt, uploadingMetaPath, validMeta) - require.NoError(t, err) - for _, file := range validMeta.Thanos.Files { - content := bytes.NewReader(make([]byte, file.SizeBytes)) - err = bkt.Upload(context.Background(), path.Join(tenantID, blockID, file.RelPath), content) - require.NoError(t, err) - } - } - - testCases := []struct { - name string - tenantID string - blockID string - setUpBucket func(*testing.T, objstore.Bucket) - errorInjector func(op bucket.Operation, name string) error - disableBlockUpload bool - enableValidation bool // should only be set to true for tests that fail before validation is started - maxConcurrency int - setConcurrency int64 - expBadRequest string - expConflict string - expNotFound string - expTooManyRequests bool - expInternalServerError bool - }{ - { - name: "without tenant ID", - blockID: blockID, - expBadRequest: "invalid tenant ID", - }, - { - name: "without block ID", - tenantID: tenantID, - expBadRequest: "invalid block ID", - }, - { - name: "invalid block ID", - tenantID: tenantID, - blockID: "1234", - expBadRequest: "invalid block ID", - }, - { - name: "block upload disabled", - tenantID: tenantID, - blockID: blockID, - disableBlockUpload: true, - expBadRequest: "block upload is disabled", - }, - { - name: "complete block already exists", - tenantID: tenantID, - blockID: blockID, - setUpBucket: func(t *testing.T, bkt objstore.Bucket) { - err := marshalAndUploadToBucket(context.Background(), bkt, metaPath, validMeta) - require.NoError(t, err) - }, - expConflict: "block already exists", - }, - { - name: "checking for complete block fails", - tenantID: tenantID, - blockID: blockID, - errorInjector: bucket.InjectErrorOn(bucket.OpExists, metaPath, injectedError), - expInternalServerError: true, - }, - { - name: "missing in-flight meta file", - tenantID: tenantID, - blockID: blockID, - expNotFound: "block upload not started", - }, - { - name: "downloading in-flight meta file fails", - tenantID: tenantID, - blockID: blockID, - setUpBucket: func(t *testing.T, bkt objstore.Bucket) { - err := marshalAndUploadToBucket(context.Background(), bkt, uploadingMetaPath, validMeta) - require.NoError(t, err) - }, - errorInjector: bucket.InjectErrorOn(bucket.OpGet, uploadingMetaPath, injectedError), - expInternalServerError: true, - }, - { - name: "corrupt in-flight meta file", - tenantID: tenantID, - blockID: blockID, - setUpBucket: func(t *testing.T, bkt objstore.Bucket) { - err := bkt.Upload(context.Background(), uploadingMetaPath, bytes.NewReader([]byte("{"))) - require.NoError(t, err) - }, - expInternalServerError: true, - }, - { - name: "uploading meta file fails", - tenantID: tenantID, - blockID: blockID, - setUpBucket: validSetup, - errorInjector: bucket.InjectErrorOn(bucket.OpUpload, metaPath, injectedError), - expInternalServerError: true, - }, - { - name: "too many concurrent validations", - tenantID: tenantID, - blockID: blockID, - setUpBucket: validSetup, - enableValidation: true, - maxConcurrency: 2, - setConcurrency: 2, - expTooManyRequests: true, - }, - } - for _, tc := range testCases { - t.Run(tc.name, func(t *testing.T) { - bkt := objstore.NewInMemBucket() - injectedBkt := bucket.ErrorInjectedBucketClient{ - Bucket: bkt, - Injector: tc.errorInjector, - } - if tc.setUpBucket != nil { - tc.setUpBucket(t, bkt) - } - - cfgProvider := newMockConfigProvider() - cfgProvider.blockUploadEnabled[tc.tenantID] = !tc.disableBlockUpload - cfgProvider.blockUploadValidationEnabled[tc.tenantID] = tc.enableValidation - c := &MultitenantCompactor{ - logger: log.NewNopLogger(), - bucketClient: &injectedBkt, - cfgProvider: cfgProvider, - } - c.compactorCfg.MaxBlockUploadValidationConcurrency = tc.maxConcurrency - if tc.setConcurrency > 0 { - c.blockUploadValidations.Add(tc.setConcurrency) - } - - c.compactorCfg.DataDir = t.TempDir() - - r := httptest.NewRequest(http.MethodPost, fmt.Sprintf("/api/v1/upload/block/%s/finish", tc.blockID), nil) - if tc.tenantID != "" { - r = r.WithContext(user.InjectOrgID(r.Context(), tenantID)) - } - if tc.blockID != "" { - r = mux.SetURLVars(r, map[string]string{"block": tc.blockID}) - } - w := httptest.NewRecorder() - c.FinishBlockUpload(w, r) - - resp := w.Result() - body, err := io.ReadAll(resp.Body) - require.NoError(t, err) - - switch { - case tc.expBadRequest != "": - assert.Equal(t, http.StatusBadRequest, resp.StatusCode) - assert.Equal(t, fmt.Sprintf("%s\n", tc.expBadRequest), string(body)) - case tc.expConflict != "": - assert.Equal(t, http.StatusConflict, resp.StatusCode) - assert.Equal(t, fmt.Sprintf("%s\n", tc.expConflict), string(body)) - case tc.expNotFound != "": - assert.Equal(t, http.StatusNotFound, resp.StatusCode) - assert.Equal(t, fmt.Sprintf("%s\n", tc.expNotFound), string(body)) - case tc.expInternalServerError: - assert.Equal(t, http.StatusInternalServerError, resp.StatusCode) - assert.Regexp(t, "internal server error \\(id [0-9a-f]{16}\\)\n", string(body)) - case tc.expTooManyRequests: - assert.Equal(t, http.StatusTooManyRequests, resp.StatusCode) - assert.Equal(t, "too many block upload validations in progress, limit is 2\n", string(body)) - default: - assert.Equal(t, http.StatusOK, resp.StatusCode) - assert.Empty(t, string(body)) - exists, err := bkt.Exists(context.Background(), path.Join(tc.blockID, block.MetaFilename)) - require.NoError(t, err) - require.True(t, exists) - } - }) - } -} - -func TestMultitenantCompactor_ValidateAndComplete(t *testing.T) { - const tenantID = "test" - const blockID = "01G3FZ0JWJYJC0ZM6Y9778P6KD" - injectedError := fmt.Errorf("injected error") - - uploadingMetaPath := path.Join(tenantID, blockID, uploadingMetaFilename) - validationPath := path.Join(tenantID, blockID, validationFilename) - metaPath := path.Join(tenantID, blockID, block.MetaFilename) - - validationSucceeds := func(_ context.Context) error { return nil } - - testCases := []struct { - name string - errorInjector func(op bucket.Operation, name string) error - validation func(context.Context) error - expectValidationFile bool - expectErrorInValidationFile bool - expectTempUploadingMeta bool - expectMeta bool - }{ - { - name: "validation fails", - validation: func(_ context.Context) error { return injectedError }, - expectValidationFile: true, - expectErrorInValidationFile: true, - expectTempUploadingMeta: true, - expectMeta: false, - }, - { - name: "validation fails, uploading error fails", - errorInjector: bucket.InjectErrorOn(bucket.OpUpload, validationPath, injectedError), - validation: func(_ context.Context) error { return injectedError }, - expectValidationFile: true, - expectErrorInValidationFile: false, - expectTempUploadingMeta: true, - expectMeta: false, - }, - { - name: "uploading meta file fails", - errorInjector: bucket.InjectErrorOn(bucket.OpUpload, metaPath, injectedError), - validation: validationSucceeds, - expectValidationFile: true, - expectErrorInValidationFile: true, - expectTempUploadingMeta: true, - expectMeta: false, - }, - { - name: "uploading meta file fails, uploading error fails", - errorInjector: func(op bucket.Operation, target string) error { - if op == bucket.OpUpload && (target == metaPath || target == validationPath) { - return injectedError - } - return nil - }, - validation: validationSucceeds, - expectValidationFile: true, - expectErrorInValidationFile: false, - expectTempUploadingMeta: true, - expectMeta: false, - }, - { - name: "removing in-flight meta file fails", - errorInjector: bucket.InjectErrorOn(bucket.OpDelete, uploadingMetaPath, injectedError), - validation: validationSucceeds, - expectValidationFile: false, - expectTempUploadingMeta: true, - expectMeta: true, - }, - { - name: "removing validation file fails", - errorInjector: bucket.InjectErrorOn(bucket.OpDelete, validationPath, injectedError), - validation: validationSucceeds, - expectValidationFile: true, - expectErrorInValidationFile: false, - expectTempUploadingMeta: false, - expectMeta: true, - }, - { - name: "valid request", - validation: validationSucceeds, - expectValidationFile: false, - expectTempUploadingMeta: false, - expectMeta: true, - }, - } - - for _, tc := range testCases { - t.Run(tc.name, func(t *testing.T) { - bkt := objstore.NewInMemBucket() - var injectedBkt objstore.Bucket = bkt - if tc.errorInjector != nil { - injectedBkt = &bucket.ErrorInjectedBucketClient{ - Bucket: bkt, - Injector: tc.errorInjector, - } - } - cfgProvider := newMockConfigProvider() - c := &MultitenantCompactor{ - logger: log.NewNopLogger(), - bucketClient: injectedBkt, - cfgProvider: cfgProvider, - blockUploadBlocks: promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{tenantID}), - blockUploadBytes: promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{tenantID}), - blockUploadFiles: promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{tenantID}), - } - userBkt := bucket.NewUserBucketClient(tenantID, injectedBkt, cfgProvider) - - meta := block.Meta{} - marshalAndUploadJSON(t, bkt, uploadingMetaPath, meta) - v := validationFile{} - marshalAndUploadJSON(t, bkt, validationPath, v) - - c.validateAndCompleteBlockUpload(log.NewNopLogger(), tenantID, userBkt, ulid.MustParse(blockID), &meta, tc.validation) - - tempUploadingMetaExists, err := bkt.Exists(context.Background(), uploadingMetaPath) - require.NoError(t, err) - require.Equal(t, tempUploadingMetaExists, tc.expectTempUploadingMeta) - - metaExists, err := bkt.Exists(context.Background(), metaPath) - require.NoError(t, err) - require.Equal(t, metaExists, tc.expectMeta) - - if !tc.expectValidationFile { - exists, err := bkt.Exists(context.Background(), validationPath) - require.NoError(t, err) - require.False(t, exists) - return - } - - r, err := bkt.Get(context.Background(), validationPath) - require.NoError(t, err) - decoder := json.NewDecoder(r) - err = decoder.Decode(&v) - require.NoError(t, err) - - if tc.expectErrorInValidationFile { - require.NotEmpty(t, v.Error) - } else { - require.Empty(t, v.Error) - } - }) - } -} - -func TestMultitenantCompactor_ValidateBlock(t *testing.T) { - const tenantID = "test" - ctx := context.Background() - tmpDir := t.TempDir() - bkt := objstore.NewInMemBucket() - - type Missing uint8 - const ( - MissingMeta Missing = 1 << iota - MissingIndex - MissingChunks - ) - - validLabels := func() []labels.Labels { - return []labels.Labels{ - labels.FromStrings("a", "1"), - labels.FromStrings("b", "2"), - labels.FromStrings("c", "3"), - } - } - - testCases := []struct { - name string - lbls func() []labels.Labels - metaInject func(meta *block.Meta) - indexInject func(fname string) - chunkInject func(fname string) - populateFileList bool - maximumBlockSize int64 - verifyChunks bool - missing Missing - expectError bool - expectedMsg string - }{ - { - name: "valid block", - lbls: validLabels, - verifyChunks: true, - expectError: false, - populateFileList: true, - }, - { - name: "maximum block size exceeded", - lbls: validLabels, - populateFileList: true, - maximumBlockSize: 1, - expectError: true, - expectedMsg: fmt.Sprintf(maxBlockUploadSizeBytesFormat, 1), - }, - { - name: "missing meta file", - lbls: validLabels, - missing: MissingMeta, - expectError: true, - expectedMsg: "failed renaming while preparing block for validation", - }, - { - name: "missing index file", - lbls: validLabels, - missing: MissingIndex, - expectError: true, - expectedMsg: "error validating block: open index file:", - }, - { - name: "missing chunks file", - lbls: validLabels, - populateFileList: true, - missing: MissingChunks, - expectError: true, - expectedMsg: "failed to stat chunks/", - }, - { - name: "file size mismatch", - lbls: validLabels, - metaInject: func(meta *block.Meta) { - require.Greater(t, len(meta.Thanos.Files), 0) - meta.Thanos.Files[0].SizeBytes += 10 - }, - populateFileList: true, - expectError: true, - expectedMsg: "file size mismatch", - }, - { - name: "empty index file", - lbls: validLabels, - indexInject: func(fname string) { - require.NoError(t, os.Truncate(fname, 0)) - }, - expectError: true, - expectedMsg: "error validating block: open index file: mmap, size 0: invalid argument", - }, - { - name: "index file invalid magic number", - lbls: validLabels, - indexInject: func(fname string) { - flipByteAt(t, fname, 0) // guaranteed to be a magic number byte - }, - expectError: true, - expectedMsg: "error validating block: open index file: invalid magic number", - }, - { - name: "out of order labels", - lbls: func() []labels.Labels { - b := labels.NewScratchBuilder(2) - b.Add("d", "4") - b.Add("a", "1") - oooLabels := []labels.Labels{ - b.Labels(), // Haven't called Sort(), so they will be out of order. - labels.FromStrings("b", "2"), - labels.FromStrings("c", "3"), - } - return oooLabels - }, - expectError: true, - expectedMsg: "error validating block: index contains 1 postings with out of order labels", - }, - { - name: "segment file invalid magic number", - lbls: validLabels, - chunkInject: func(fname string) { - flipByteAt(t, fname, 0) // guaranteed to be a magic number byte - }, - verifyChunks: true, - expectError: true, - expectedMsg: "invalid magic number", - }, - { - name: "segment file invalid checksum", - lbls: validLabels, - chunkInject: func(fname string) { - flipByteAt(t, fname, 12) // guaranteed to be a data byte - }, - populateFileList: true, - verifyChunks: true, - expectError: true, - expectedMsg: "checksum mismatch", - }, - { - name: "empty segment file", - lbls: validLabels, - chunkInject: func(fname string) { - require.NoError(t, os.Truncate(fname, 0)) - }, - verifyChunks: true, - expectError: true, - expectedMsg: "size 0: invalid argument", - }, - } - - for _, tc := range testCases { - t.Run(tc.name, func(t *testing.T) { - // create a test block - now := time.Now() - blockID, err := block.CreateBlock(ctx, tmpDir, tc.lbls(), 300, now.Add(-2*time.Hour).UnixMilli(), now.UnixMilli(), labels.EmptyLabels()) - require.NoError(t, err) - testDir := filepath.Join(tmpDir, blockID.String()) - meta, err := block.ReadMetaFromDir(testDir) - require.NoError(t, err) - if tc.populateFileList { - stats, err := block.GatherFileStats(testDir) - require.NoError(t, err) - meta.Thanos.Files = stats - } - - // create a compactor - cfgProvider := newMockConfigProvider() - cfgProvider.blockUploadValidationEnabled[tenantID] = true - cfgProvider.verifyChunks[tenantID] = tc.verifyChunks - cfgProvider.blockUploadMaxBlockSizeBytes[tenantID] = tc.maximumBlockSize - c := &MultitenantCompactor{ - logger: log.NewNopLogger(), - bucketClient: bkt, - cfgProvider: cfgProvider, - } - - // upload the block - require.NoError(t, block.Upload(ctx, log.NewNopLogger(), bkt, testDir, nil)) - // remove meta.json as we will be uploading a new one with the uploading meta name - require.NoError(t, bkt.Delete(ctx, path.Join(blockID.String(), block.MetaFilename))) - - // handle meta file - if tc.metaInject != nil { - tc.metaInject(meta) - } - var metaBody bytes.Buffer - require.NoError(t, meta.Write(&metaBody)) - - // replace index file - if tc.indexInject != nil { - indexFile := filepath.Join(testDir, block.IndexFilename) - indexObject := path.Join(blockID.String(), block.IndexFilename) - require.NoError(t, bkt.Delete(ctx, indexObject)) - tc.indexInject(indexFile) - uploadLocalFileToBucket(ctx, t, bkt, indexFile, indexObject) - } - - // replace segment file - if tc.chunkInject != nil { - segmentFile := filepath.Join(testDir, block.ChunksDirname, "000001") - segmentObject := path.Join(blockID.String(), block.ChunksDirname, "000001") - require.NoError(t, bkt.Delete(ctx, segmentObject)) - tc.chunkInject(segmentFile) - uploadLocalFileToBucket(ctx, t, bkt, segmentFile, segmentObject) - } - - // delete any files that should be missing - if tc.missing&MissingIndex != 0 { - require.NoError(t, bkt.Delete(ctx, path.Join(blockID.String(), block.IndexFilename))) - } - - if tc.missing&MissingChunks != 0 { - chunkDir := path.Join(blockID.String(), block.ChunksDirname) - err := bkt.Iter(ctx, chunkDir, func(name string) error { - require.NoError(t, bkt.Delete(ctx, name)) - return nil - }) - require.NoError(t, err) - } - - // only upload renamed meta file if it is not meant to be missing - if tc.missing&MissingMeta == 0 { - // rename to uploading meta file as that is what validateBlock expects - require.NoError(t, bkt.Upload(ctx, path.Join(blockID.String(), uploadingMetaFilename), &metaBody)) - } - - // validate the block - err = c.validateBlock(ctx, c.logger, blockID, meta, bkt, tenantID) - if tc.expectError { - require.Error(t, err) - require.Contains(t, err.Error(), tc.expectedMsg) - } else { - require.NoError(t, err) - } - }) - } -} - -func TestMultitenantCompactor_PeriodicValidationUpdater(t *testing.T) { - const tenantID = "test" - const blockID = "01G3FZ0JWJYJC0ZM6Y9778P6KD" - injectedError := fmt.Errorf("injected error") - validationPath := path.Join(tenantID, blockID, validationFilename) - - heartbeatInterval := 50 * time.Millisecond - - validationExists := func(t *testing.T, bkt objstore.Bucket) bool { - exists, err := bkt.Exists(context.Background(), validationPath) - require.NoError(t, err) - return exists - } - - testCases := []struct { - name string - errorInjector func(op bucket.Operation, name string) error - cancelContext bool - assertions func(t *testing.T, ctx context.Context, bkt objstore.Bucket) - }{ - { - name: "updating validation file fails", - errorInjector: bucket.InjectErrorOn(bucket.OpUpload, validationPath, injectedError), - assertions: func(t *testing.T, ctx context.Context, bkt objstore.Bucket) { - <-ctx.Done() - require.True(t, errors.Is(context.Canceled, ctx.Err())) - require.False(t, validationExists(t, bkt)) - }, - }, - { - name: "updating validation file succeeds", - assertions: func(t *testing.T, ctx context.Context, bkt objstore.Bucket) { - test.Poll(t, heartbeatInterval*2, true, func() interface{} { - return validationExists(t, bkt) - }) - - v := validationFile{} - r, err := bkt.Get(context.Background(), validationPath) - require.NoError(t, err) - decoder := json.NewDecoder(r) - err = decoder.Decode(&v) - require.NoError(t, err) - require.NotEqual(t, 0, v.LastUpdate) - require.Empty(t, v.Error) - }, - }, - { - name: "context cancelled before update", - cancelContext: true, - assertions: func(t *testing.T, ctx context.Context, bkt objstore.Bucket) { - require.False(t, validationExists(t, bkt)) - }, - }, - } - - for _, tc := range testCases { - t.Run(tc.name, func(t *testing.T) { - bkt := objstore.NewInMemBucket() - var injectedBkt objstore.Bucket = bkt - if tc.errorInjector != nil { - injectedBkt = &bucket.ErrorInjectedBucketClient{ - Bucket: bkt, - Injector: tc.errorInjector, - } - } - - cfgProvider := newMockConfigProvider() - c := &MultitenantCompactor{ - logger: log.NewNopLogger(), - bucketClient: injectedBkt, - cfgProvider: cfgProvider, - } - userBkt := bucket.NewUserBucketClient(tenantID, injectedBkt, cfgProvider) - ctx, cancel := context.WithCancel(context.Background()) - - heartbeatInterval := heartbeatInterval - if tc.cancelContext { - cancel() - heartbeatInterval = 1 * time.Hour // to avoid racing a heartbeat - } - - var wg sync.WaitGroup - wg.Add(1) - go func() { - defer wg.Done() - c.periodicValidationUpdater(ctx, log.NewNopLogger(), ulid.MustParse(blockID), userBkt, cancel, heartbeatInterval) - }() - - if !tc.cancelContext { - time.Sleep(heartbeatInterval) - } - - tc.assertions(t, ctx, bkt) - - cancel() - wg.Wait() - }) - } -} - -func TestMultitenantCompactor_GetBlockUploadStateHandler(t *testing.T) { - const ( - tenantID = "tenant" - blockID = "01G8X9GA8R6N8F75FW1J18G83N" - ) - - type testcase struct { - setupBucket func(t *testing.T, bkt objstore.Bucket) - disableBlockUpload bool - expectedStatusCode int - expectedBody string - } - - for name, tc := range map[string]testcase{ - "block doesn't exist": { - expectedStatusCode: http.StatusNotFound, - expectedBody: "block doesn't exist", - }, - - "complete block": { - setupBucket: func(t *testing.T, bkt objstore.Bucket) { - marshalAndUploadJSON(t, bkt, path.Join(tenantID, blockID, block.MetaFilename), block.Meta{}) - }, - expectedStatusCode: http.StatusOK, - expectedBody: `{"result":"complete"}`, - }, - - "upload in progress": { - setupBucket: func(t *testing.T, bkt objstore.Bucket) { - marshalAndUploadJSON(t, bkt, path.Join(tenantID, blockID, uploadingMetaFilename), block.Meta{}) - }, - expectedStatusCode: http.StatusOK, - expectedBody: `{"result":"uploading"}`, - }, - - "validating": { - setupBucket: func(t *testing.T, bkt objstore.Bucket) { - marshalAndUploadJSON(t, bkt, path.Join(tenantID, blockID, uploadingMetaFilename), block.Meta{}) - marshalAndUploadJSON(t, bkt, path.Join(tenantID, blockID, validationFilename), validationFile{LastUpdate: time.Now().UnixMilli()}) - }, - expectedStatusCode: http.StatusOK, - expectedBody: `{"result":"validating"}`, - }, - - "validation failed": { - setupBucket: func(t *testing.T, bkt objstore.Bucket) { - marshalAndUploadJSON(t, bkt, path.Join(tenantID, blockID, uploadingMetaFilename), block.Meta{}) - marshalAndUploadJSON(t, bkt, path.Join(tenantID, blockID, validationFilename), validationFile{LastUpdate: time.Now().UnixMilli(), Error: "error during validation"}) - }, - expectedStatusCode: http.StatusOK, - expectedBody: `{"result":"failed","error":"error during validation"}`, - }, - - "stale validation file": { - setupBucket: func(t *testing.T, bkt objstore.Bucket) { - marshalAndUploadJSON(t, bkt, path.Join(tenantID, blockID, uploadingMetaFilename), block.Meta{}) - marshalAndUploadJSON(t, bkt, path.Join(tenantID, blockID, validationFilename), validationFile{LastUpdate: time.Now().Add(-10 * time.Minute).UnixMilli()}) - }, - expectedStatusCode: http.StatusOK, - expectedBody: `{"result":"uploading"}`, - }, - } { - t.Run(name, func(t *testing.T) { - bkt := objstore.NewInMemBucket() - if tc.setupBucket != nil { - tc.setupBucket(t, bkt) - } - - cfgProvider := newMockConfigProvider() - cfgProvider.blockUploadEnabled[tenantID] = !tc.disableBlockUpload - - c := &MultitenantCompactor{ - logger: log.NewNopLogger(), - bucketClient: bkt, - cfgProvider: cfgProvider, - } - - r := httptest.NewRequest(http.MethodGet, fmt.Sprintf("/api/v1/upload/block/%s/check", blockID), nil) - urlVars := map[string]string{"block": blockID} - r = mux.SetURLVars(r, urlVars) - r = r.WithContext(user.InjectOrgID(r.Context(), tenantID)) - - w := httptest.NewRecorder() - c.GetBlockUploadStateHandler(w, r) - resp := w.Result() - - body, err := io.ReadAll(resp.Body) - - require.NoError(t, err) - require.Equal(t, tc.expectedStatusCode, resp.StatusCode) - require.Equal(t, tc.expectedBody, strings.TrimSpace(string(body))) - }) - } -} - -func TestMultitenantCompactor_ValidateMaximumBlockSize(t *testing.T) { - const userID = "user" - - type testCase struct { - maximumBlockSize int64 - fileSizes []int64 - expectErr bool - } - - for name, tc := range map[string]testCase{ - "no limit": { - maximumBlockSize: 0, - fileSizes: []int64{math.MaxInt64}, - expectErr: false, - }, - "under limit": { - maximumBlockSize: 4, - fileSizes: []int64{1, 2}, - expectErr: false, - }, - "under limit - zero size file included": { - maximumBlockSize: 2, - fileSizes: []int64{1, 0}, - expectErr: false, - }, - "under limit - negative size file included": { - maximumBlockSize: 2, - fileSizes: []int64{2, -1}, - expectErr: true, - }, - "exact limit": { - maximumBlockSize: 3, - fileSizes: []int64{1, 2}, - expectErr: false, - }, - "over limit": { - maximumBlockSize: 1, - fileSizes: []int64{1, 1}, - expectErr: true, - }, - "overflow": { - maximumBlockSize: math.MaxInt64, - fileSizes: []int64{math.MaxInt64, math.MaxInt64, math.MaxInt64}, - expectErr: true, - }, - } { - t.Run(name, func(t *testing.T) { - files := make([]block.File, len(tc.fileSizes)) - for i, size := range tc.fileSizes { - files[i] = block.File{SizeBytes: size} - } - - cfgProvider := newMockConfigProvider() - cfgProvider.blockUploadMaxBlockSizeBytes[userID] = tc.maximumBlockSize - c := &MultitenantCompactor{ - logger: log.NewNopLogger(), - cfgProvider: cfgProvider, - } - - err := c.validateMaximumBlockSize(c.logger, files, userID) - if tc.expectErr { - require.Error(t, err) - } else { - require.NoError(t, err) - } - }) - } -} - -func TestMultitenantCompactor_MarkBlockComplete(t *testing.T) { - const tenantID = "test" - const blockID = "01G3FZ0JWJYJC0ZM6Y9778P6KD" - injectedError := fmt.Errorf("injected error") - - uploadingMetaPath := path.Join(tenantID, blockID, uploadingMetaFilename) - metaPath := path.Join(tenantID, blockID, block.MetaFilename) - testCases := []struct { - name string - errorInjector func(op bucket.Operation, name string) error - expectSuccess bool - }{ - { - name: "marking block complete succeeds", - expectSuccess: true, - }, - { - name: "uploading meta file fails", - errorInjector: bucket.InjectErrorOn(bucket.OpUpload, metaPath, injectedError), - }, - { - name: "deleting uploading meta file fails", - errorInjector: bucket.InjectErrorOn(bucket.OpDelete, uploadingMetaPath, injectedError), - expectSuccess: true, - }, - } - for _, tc := range testCases { - t.Run(tc.name, func(t *testing.T) { - bkt := objstore.NewInMemBucket() - var injectedBkt objstore.Bucket = bkt - if tc.errorInjector != nil { - injectedBkt = &bucket.ErrorInjectedBucketClient{ - Bucket: bkt, - Injector: tc.errorInjector, - } - } - cfgProvider := newMockConfigProvider() - c := &MultitenantCompactor{ - logger: log.NewNopLogger(), - bucketClient: injectedBkt, - cfgProvider: cfgProvider, - blockUploadBlocks: promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{tenantID}), - blockUploadBytes: promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{tenantID}), - blockUploadFiles: promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{tenantID}), - } - userBkt := bucket.NewUserBucketClient(tenantID, injectedBkt, cfgProvider) - - meta := block.Meta{ - Thanos: block.ThanosMeta{ - Files: []block.File{ - { - RelPath: "chunks/000001", - SizeBytes: 42, - }, - { - RelPath: "index", - SizeBytes: 17, - }, - { - RelPath: "meta.json", - }, - }, - }, - } - marshalAndUploadJSON(t, bkt, uploadingMetaPath, meta) - - ctx := context.Background() - err := c.markBlockComplete(ctx, log.NewNopLogger(), tenantID, userBkt, ulid.MustParse(blockID), &meta) - if tc.expectSuccess { - require.NoError(t, err) - assert.Equal(t, 1.0, promtest.ToFloat64(c.blockUploadBlocks.WithLabelValues(tenantID))) - assert.Equal(t, 59.0, promtest.ToFloat64(c.blockUploadBytes.WithLabelValues(tenantID))) - assert.Equal(t, 3.0, promtest.ToFloat64(c.blockUploadFiles.WithLabelValues(tenantID))) - } else { - require.Error(t, err) - assert.Equal(t, 0.0, promtest.ToFloat64(c.blockUploadBlocks.WithLabelValues(tenantID))) - assert.Equal(t, 0.0, promtest.ToFloat64(c.blockUploadBytes.WithLabelValues(tenantID))) - assert.Equal(t, 0.0, promtest.ToFloat64(c.blockUploadFiles.WithLabelValues(tenantID))) - } - }) - } -} - -// marshalAndUploadJSON is a test helper for uploading a meta file to a certain path in a bucket. -func marshalAndUploadJSON(t *testing.T, bkt objstore.Bucket, pth string, val interface{}) { - t.Helper() - err := marshalAndUploadToBucket(context.Background(), bkt, pth, val) - require.NoError(t, err) -} - -func uploadLocalFileToBucket(ctx context.Context, t *testing.T, bkt objstore.Bucket, src, dst string) { - t.Helper() - fd, err := os.Open(src) - require.NoError(t, err) - defer func(fd *os.File) { - err := fd.Close() - require.NoError(t, err) - }(fd) - require.NoError(t, bkt.Upload(ctx, dst, fd)) -} - -// flipByteAt flips a byte at a given offset in a file. -func flipByteAt(t *testing.T, fname string, offset int64) { - fd, err := os.OpenFile(fname, os.O_RDWR, 0o644) - require.NoError(t, err) - defer func(fd *os.File) { - err := fd.Close() - require.NoError(t, err) - }(fd) - var b [1]byte - _, err = fd.ReadAt(b[:], offset) - require.NoError(t, err) - // alter the byte - b[0] = 0xff - b[0] - _, err = fd.WriteAt(b[:], offset) - require.NoError(t, err) -} - -func TestHexTimeNowNano(t *testing.T) { - v := hexTimeNowNano() - require.Len(t, v, 16, "Should have exactly 16 characters") - - require.NotEqual(t, strings.Repeat("0", 16), v, "Should not be all zeros") - time.Sleep(time.Nanosecond) - require.NotEqual(t, v, hexTimeNowNano(), "Should generate a different one.") -} +// import ( +// "bytes" +// "context" +// "encoding/json" +// "fmt" +// "io" +// "math" +// "net/http" +// "net/http/httptest" +// "net/url" +// "os" +// "path" +// "path/filepath" +// "strings" +// "sync" +// "testing" +// "time" + +// "github.com/go-kit/log" +// "github.com/gorilla/mux" +// "github.com/grafana/dskit/test" +// "github.com/grafana/dskit/user" +// "github.com/oklog/ulid" +// "github.com/pkg/errors" +// "github.com/prometheus/client_golang/prometheus" +// "github.com/prometheus/client_golang/prometheus/promauto" +// promtest "github.com/prometheus/client_golang/prometheus/testutil" +// "github.com/prometheus/prometheus/model/labels" +// "github.com/prometheus/prometheus/tsdb" +// "github.com/stretchr/testify/assert" +// "github.com/stretchr/testify/mock" +// "github.com/stretchr/testify/require" +// "github.com/thanos-io/objstore" + +// "github.com/grafana/mimir/pkg/storage/bucket" +// mimir_tsdb "github.com/grafana/mimir/pkg/storage/tsdb" +// "github.com/grafana/mimir/pkg/storage/tsdb/block" +// ) + +// func verifyUploadedMeta(t *testing.T, bkt *bucket.ClientMock, expMeta block.Meta) { +// var call mock.Call +// for _, c := range bkt.Calls { +// if c.Method == "Upload" { +// call = c +// break +// } +// } + +// rdr := call.Arguments[2].(io.Reader) +// var gotMeta block.Meta +// require.NoError(t, json.NewDecoder(rdr).Decode(&gotMeta)) +// assert.Equal(t, expMeta, gotMeta) +// } + +// // Test MultitenantCompactor.StartBlockUpload +// func TestMultitenantCompactor_StartBlockUpload(t *testing.T) { +// const tenantID = "test" +// const blockID = "01G3FZ0JWJYJC0ZM6Y9778P6KD" +// bULID := ulid.MustParse(blockID) +// now := time.Now().UnixMilli() +// validMeta := block.Meta{ +// BlockMeta: tsdb.BlockMeta{ +// ULID: bULID, +// Version: block.TSDBVersion1, +// MinTime: now - 1000, +// MaxTime: now, +// }, +// Thanos: block.ThanosMeta{ +// Labels: map[string]string{ +// mimir_tsdb.CompactorShardIDExternalLabel: "1_of_3", +// }, +// Files: []block.File{ +// { +// RelPath: block.MetaFilename, +// }, +// { +// RelPath: "index", +// SizeBytes: 1, +// }, +// { +// RelPath: "chunks/000001", +// SizeBytes: 1024, +// }, +// }, +// }, +// } + +// metaPath := path.Join(tenantID, blockID, block.MetaFilename) +// uploadingMetaPath := path.Join(tenantID, blockID, fmt.Sprintf("uploading-%s", block.MetaFilename)) + +// setUpPartialBlock := func(bkt *bucket.ClientMock) { +// bkt.MockExists(path.Join(tenantID, blockID, block.MetaFilename), false, nil) +// setUpGet(bkt, path.Join(tenantID, blockID, uploadingMetaFilename), nil, bucket.ErrObjectDoesNotExist) +// } +// setUpUpload := func(bkt *bucket.ClientMock) { +// setUpPartialBlock(bkt) +// bkt.MockUpload(uploadingMetaPath, nil) +// } + +// verifyUpload := func(t *testing.T, bkt *bucket.ClientMock, labels map[string]string) { +// t.Helper() + +// expMeta := validMeta +// expMeta.Compaction.Parents = nil +// expMeta.Compaction.Sources = []ulid.ULID{expMeta.ULID} +// expMeta.Thanos.Source = "upload" +// expMeta.Thanos.Labels = labels +// verifyUploadedMeta(t, bkt, expMeta) +// } + +// testCases := []struct { +// name string +// tenantID string +// blockID string +// body string +// meta *block.Meta +// retention time.Duration +// disableBlockUpload bool +// expBadRequest string +// expConflict string +// expUnprocessableEntity string +// expEntityTooLarge string +// expInternalServerError bool +// setUpBucketMock func(bkt *bucket.ClientMock) +// verifyUpload func(*testing.T, *bucket.ClientMock) +// maxBlockUploadSizeBytes int64 +// }{ +// { +// name: "missing tenant ID", +// tenantID: "", +// blockID: blockID, +// expBadRequest: "invalid tenant ID", +// }, +// { +// name: "missing block ID", +// tenantID: tenantID, +// blockID: "", +// expBadRequest: "invalid block ID", +// }, +// { +// name: "invalid block ID", +// tenantID: tenantID, +// blockID: "1234", +// expBadRequest: "invalid block ID", +// }, +// { +// name: "missing body", +// tenantID: tenantID, +// blockID: blockID, +// expBadRequest: "malformed request body", +// setUpBucketMock: setUpPartialBlock, +// }, +// { +// name: "malformed body", +// tenantID: tenantID, +// blockID: blockID, +// setUpBucketMock: setUpPartialBlock, +// body: "{", +// expBadRequest: "malformed request body", +// }, +// { +// name: "invalid file path", +// tenantID: tenantID, +// blockID: blockID, +// setUpBucketMock: setUpPartialBlock, +// meta: &block.Meta{ +// Thanos: block.ThanosMeta{ +// Files: []block.File{ +// { +// RelPath: "chunks/invalid-file", +// SizeBytes: 1024, +// }, +// }, +// }, +// }, +// expBadRequest: "file with invalid path: chunks/invalid-file", +// }, +// { +// name: "contains downsampled data", +// tenantID: tenantID, +// blockID: blockID, +// setUpBucketMock: setUpPartialBlock, +// meta: &block.Meta{ +// Thanos: block.ThanosMeta{ +// Downsample: block.ThanosDownsample{ +// Resolution: 1000, +// }, +// Files: []block.File{ +// { +// RelPath: block.MetaFilename, +// }, +// { +// RelPath: "index", +// SizeBytes: 1, +// }, +// { +// RelPath: "chunks/000001", +// }, +// }, +// }, +// }, +// expBadRequest: "block contains downsampled data", +// }, +// { +// name: "missing file size", +// tenantID: tenantID, +// blockID: blockID, +// setUpBucketMock: setUpPartialBlock, +// meta: &block.Meta{ +// Thanos: block.ThanosMeta{ +// Files: []block.File{ +// { +// RelPath: block.MetaFilename, +// }, +// { +// RelPath: "index", +// SizeBytes: 1, +// }, +// { +// RelPath: "chunks/000001", +// }, +// }, +// }, +// }, +// expBadRequest: "file with invalid size: chunks/000001", +// }, +// { +// name: "invalid minTime", +// tenantID: tenantID, +// blockID: blockID, +// setUpBucketMock: setUpPartialBlock, +// meta: &block.Meta{ +// BlockMeta: tsdb.BlockMeta{ +// ULID: bULID, +// Version: block.TSDBVersion1, +// MinTime: -1, +// MaxTime: 0, +// }, +// }, +// expBadRequest: "invalid minTime/maxTime: minTime=-1, maxTime=0", +// }, +// { +// name: "invalid maxTime", +// tenantID: tenantID, +// blockID: blockID, +// setUpBucketMock: setUpPartialBlock, +// meta: &block.Meta{ +// BlockMeta: tsdb.BlockMeta{ +// ULID: bULID, +// Version: block.TSDBVersion1, +// MinTime: 0, +// MaxTime: -1, +// }, +// }, +// expBadRequest: "invalid minTime/maxTime: minTime=0, maxTime=-1", +// }, +// { +// name: "maxTime before minTime", +// tenantID: tenantID, +// blockID: blockID, +// setUpBucketMock: setUpPartialBlock, +// meta: &block.Meta{ +// BlockMeta: tsdb.BlockMeta{ +// ULID: bULID, +// Version: block.TSDBVersion1, +// MinTime: 1, +// MaxTime: 0, +// }, +// }, +// expBadRequest: "invalid minTime/maxTime: minTime=1, maxTime=0", +// }, +// { +// name: "block before retention period", +// tenantID: tenantID, +// blockID: blockID, +// retention: 10 * time.Second, +// setUpBucketMock: setUpPartialBlock, +// meta: &block.Meta{ +// BlockMeta: tsdb.BlockMeta{ +// ULID: bULID, +// Version: block.TSDBVersion1, +// MinTime: 0, +// MaxTime: 1000, +// }, +// }, +// expUnprocessableEntity: "block max time (1970-01-01 00:00:01 +0000 UTC) older than retention period", +// }, +// { +// name: "invalid version", +// tenantID: tenantID, +// blockID: blockID, +// setUpBucketMock: setUpPartialBlock, +// meta: &block.Meta{ +// BlockMeta: tsdb.BlockMeta{ +// ULID: bULID, +// Version: 0, +// }, +// }, +// expBadRequest: fmt.Sprintf("version must be %d", block.TSDBVersion1), +// }, +// { +// name: "ignore retention period if == 0", +// tenantID: tenantID, +// blockID: blockID, +// retention: 0, +// setUpBucketMock: setUpUpload, +// meta: &block.Meta{ +// BlockMeta: tsdb.BlockMeta{ +// ULID: bULID, +// Version: block.TSDBVersion1, +// MinTime: 0, +// MaxTime: 1000, +// }, +// Thanos: block.ThanosMeta{ +// Labels: map[string]string{ +// mimir_tsdb.CompactorShardIDExternalLabel: "1_of_3", +// }, +// Files: []block.File{ +// { +// RelPath: block.MetaFilename, +// }, +// { +// RelPath: "index", +// SizeBytes: 1, +// }, +// { +// RelPath: "chunks/000001", +// SizeBytes: 1024, +// }, +// }, +// }, +// }, +// }, +// { +// name: "ignore retention period if < 0", +// tenantID: tenantID, +// blockID: blockID, +// retention: -1, +// setUpBucketMock: setUpUpload, +// meta: &block.Meta{ +// BlockMeta: tsdb.BlockMeta{ +// ULID: bULID, +// Version: block.TSDBVersion1, +// MinTime: 0, +// MaxTime: 1000, +// }, +// Thanos: block.ThanosMeta{ +// Labels: map[string]string{ +// mimir_tsdb.CompactorShardIDExternalLabel: "1_of_3", +// }, +// Files: []block.File{ +// { +// RelPath: block.MetaFilename, +// }, +// { +// RelPath: "index", +// SizeBytes: 1, +// }, +// { +// RelPath: "chunks/000001", +// SizeBytes: 1024, +// }, +// }, +// }, +// }, +// }, +// { +// name: "invalid compactor shard ID label", +// tenantID: tenantID, +// blockID: blockID, +// setUpBucketMock: setUpPartialBlock, +// meta: &block.Meta{ +// BlockMeta: tsdb.BlockMeta{ +// ULID: bULID, +// Version: block.TSDBVersion1, +// }, +// Thanos: block.ThanosMeta{ +// Labels: map[string]string{ +// mimir_tsdb.CompactorShardIDExternalLabel: "test", +// }, +// }, +// }, +// expBadRequest: fmt.Sprintf(`invalid %s external label: "test"`, mimir_tsdb.CompactorShardIDExternalLabel), +// }, +// { +// name: "failure checking for complete block", +// tenantID: tenantID, +// blockID: blockID, +// setUpBucketMock: func(bkt *bucket.ClientMock) { +// bkt.MockExists(path.Join(tenantID, blockID, block.MetaFilename), false, fmt.Errorf("test")) +// }, +// expInternalServerError: true, +// }, +// { +// name: "complete block already exists", +// tenantID: tenantID, +// blockID: blockID, +// setUpBucketMock: func(bkt *bucket.ClientMock) { +// bkt.MockExists(path.Join(tenantID, blockID, block.MetaFilename), true, nil) +// }, +// expConflict: "block already exists", +// }, +// { +// name: "failure uploading meta file", +// tenantID: tenantID, +// blockID: blockID, +// setUpBucketMock: func(bkt *bucket.ClientMock) { +// setUpPartialBlock(bkt) +// bkt.MockUpload(uploadingMetaPath, fmt.Errorf("test")) +// }, +// meta: &validMeta, +// expInternalServerError: true, +// }, +// { +// name: "too large of a request body", +// tenantID: tenantID, +// blockID: blockID, +// setUpBucketMock: setUpPartialBlock, +// body: strings.Repeat("A", maximumMetaSizeBytes+1), +// expEntityTooLarge: fmt.Sprintf("The block metadata was too large (maximum size allowed is %d bytes)", maximumMetaSizeBytes), +// }, +// { +// name: "block upload disabled", +// tenantID: tenantID, +// blockID: blockID, +// disableBlockUpload: true, +// expBadRequest: "block upload is disabled", +// }, +// { +// name: "max block size exceeded", +// tenantID: tenantID, +// blockID: blockID, +// setUpBucketMock: setUpPartialBlock, +// meta: &validMeta, +// maxBlockUploadSizeBytes: 1, +// expBadRequest: fmt.Sprintf(maxBlockUploadSizeBytesFormat, 1), +// }, +// { +// name: "valid request", +// tenantID: tenantID, +// blockID: blockID, +// setUpBucketMock: setUpUpload, +// meta: &validMeta, +// verifyUpload: func(t *testing.T, bkt *bucket.ClientMock) { +// verifyUpload(t, bkt, map[string]string{ +// mimir_tsdb.CompactorShardIDExternalLabel: "1_of_3", +// }) +// }, +// }, +// { +// name: "valid request with empty compactor shard ID label", +// tenantID: tenantID, +// blockID: blockID, +// setUpBucketMock: setUpUpload, +// meta: &block.Meta{ +// BlockMeta: tsdb.BlockMeta{ +// ULID: bULID, +// Version: block.TSDBVersion1, +// MinTime: now - 1000, +// MaxTime: now, +// }, +// Thanos: block.ThanosMeta{ +// Labels: map[string]string{ +// mimir_tsdb.CompactorShardIDExternalLabel: "", +// }, +// Files: []block.File{ +// { +// RelPath: block.MetaFilename, +// }, +// { +// RelPath: "index", +// SizeBytes: 1, +// }, +// { +// RelPath: "chunks/000001", +// SizeBytes: 1024, +// }, +// }, +// }, +// }, +// verifyUpload: func(t *testing.T, bkt *bucket.ClientMock) { +// verifyUpload(t, bkt, map[string]string{}) +// }, +// }, +// { +// name: "valid request without compactor shard ID label", +// tenantID: tenantID, +// blockID: blockID, +// setUpBucketMock: setUpUpload, +// meta: &block.Meta{ +// BlockMeta: tsdb.BlockMeta{ +// ULID: bULID, +// Version: block.TSDBVersion1, +// MinTime: now - 1000, +// MaxTime: now, +// }, +// Thanos: block.ThanosMeta{ +// Files: []block.File{ +// { +// RelPath: block.MetaFilename, +// }, +// { +// RelPath: "index", +// SizeBytes: 1, +// }, +// { +// RelPath: "chunks/000001", +// SizeBytes: 1024, +// }, +// }, +// }, +// }, +// verifyUpload: func(t *testing.T, bkt *bucket.ClientMock) { +// verifyUpload(t, bkt, nil) +// }, +// }, +// { +// name: "valid request with different block ID in meta file", +// tenantID: tenantID, +// blockID: blockID, +// setUpBucketMock: setUpUpload, +// meta: &block.Meta{ +// BlockMeta: tsdb.BlockMeta{ +// ULID: ulid.MustParse("11A2FZ0JWJYJC0ZM6Y9778P6KD"), +// Version: block.TSDBVersion1, +// MinTime: now - 1000, +// MaxTime: now, +// }, +// Thanos: block.ThanosMeta{ +// Files: []block.File{ +// { +// RelPath: block.MetaFilename, +// }, +// { +// RelPath: "index", +// SizeBytes: 1, +// }, +// { +// RelPath: "chunks/000001", +// SizeBytes: 1024, +// }, +// }, +// }, +// }, +// verifyUpload: func(t *testing.T, bkt *bucket.ClientMock) { +// verifyUpload(t, bkt, nil) +// }, +// }, +// } +// for _, tc := range testCases { +// t.Run(tc.name, func(t *testing.T) { +// var bkt bucket.ClientMock +// if tc.setUpBucketMock != nil { +// tc.setUpBucketMock(&bkt) +// } + +// cfgProvider := newMockConfigProvider() +// cfgProvider.userRetentionPeriods[tenantID] = tc.retention +// cfgProvider.blockUploadEnabled[tenantID] = !tc.disableBlockUpload +// cfgProvider.blockUploadMaxBlockSizeBytes[tenantID] = tc.maxBlockUploadSizeBytes +// c := &MultitenantCompactor{ +// logger: log.NewNopLogger(), +// bucketClient: &bkt, +// cfgProvider: cfgProvider, +// } +// var rdr io.Reader +// if tc.body != "" { +// rdr = strings.NewReader(tc.body) +// } else if tc.meta != nil { +// buf := bytes.NewBuffer(nil) +// require.NoError(t, json.NewEncoder(buf).Encode(tc.meta)) +// rdr = buf +// } +// r := httptest.NewRequest(http.MethodPost, fmt.Sprintf("/api/v1/upload/block/%s/start", tc.blockID), rdr) +// if tc.tenantID != "" { +// r = r.WithContext(user.InjectOrgID(r.Context(), tc.tenantID)) +// } +// if tc.blockID != "" { +// r = mux.SetURLVars(r, map[string]string{"block": tc.blockID}) +// } +// w := httptest.NewRecorder() +// c.StartBlockUpload(w, r) + +// resp := w.Result() +// body, err := io.ReadAll(resp.Body) +// require.NoError(t, err) + +// switch { +// case tc.expInternalServerError: +// assert.Equal(t, http.StatusInternalServerError, resp.StatusCode) +// assert.Regexp(t, "internal server error \\(id [0-9a-f]{16}\\)\n", string(body)) +// case tc.expBadRequest != "": +// assert.Equal(t, http.StatusBadRequest, resp.StatusCode) +// assert.Equal(t, fmt.Sprintf("%s\n", tc.expBadRequest), string(body)) +// case tc.expConflict != "": +// assert.Equal(t, http.StatusConflict, resp.StatusCode) +// assert.Equal(t, fmt.Sprintf("%s\n", tc.expConflict), string(body)) +// case tc.expUnprocessableEntity != "": +// assert.Equal(t, http.StatusUnprocessableEntity, resp.StatusCode) +// assert.Equal(t, fmt.Sprintf("%s\n", tc.expUnprocessableEntity), string(body)) +// case tc.expEntityTooLarge != "": +// assert.Equal(t, http.StatusRequestEntityTooLarge, resp.StatusCode) +// assert.Equal(t, fmt.Sprintf("%s\n", tc.expEntityTooLarge), string(body)) +// default: +// assert.Equal(t, http.StatusOK, resp.StatusCode) +// assert.Empty(t, string(body)) +// } + +// bkt.AssertExpectations(t) + +// if tc.verifyUpload != nil { +// tc.verifyUpload(t, &bkt) +// } +// }) +// } + +// downloadMeta := func(t *testing.T, bkt *objstore.InMemBucket, pth string) block.Meta { +// t.Helper() + +// ctx := context.Background() +// rdr, err := bkt.Get(ctx, pth) +// require.NoError(t, err) +// t.Cleanup(func() { +// _ = rdr.Close() +// }) +// var gotMeta block.Meta +// require.NoError(t, json.NewDecoder(rdr).Decode(&gotMeta)) +// return gotMeta +// } + +// // Additional test cases using an in-memory bucket for state testing +// extraCases := []struct { +// name string +// setUp func(*testing.T, *objstore.InMemBucket) block.Meta +// verifyBucket func(*testing.T, *objstore.InMemBucket) +// expBadRequest string +// expConflict string +// }{ +// { +// name: "valid request when both in-flight meta file and complete meta file exist in object storage", +// setUp: func(t *testing.T, bkt *objstore.InMemBucket) block.Meta { +// marshalAndUploadJSON(t, bkt, uploadingMetaPath, validMeta) +// marshalAndUploadJSON(t, bkt, metaPath, validMeta) +// return validMeta +// }, +// verifyBucket: func(t *testing.T, bkt *objstore.InMemBucket) { +// assert.Equal(t, validMeta, downloadMeta(t, bkt, uploadingMetaPath)) +// assert.Equal(t, validMeta, downloadMeta(t, bkt, metaPath)) +// }, +// expConflict: "block already exists", +// }, +// { +// name: "invalid request when in-flight meta file exists in object storage", +// setUp: func(t *testing.T, bkt *objstore.InMemBucket) block.Meta { +// marshalAndUploadJSON(t, bkt, uploadingMetaPath, validMeta) + +// meta := validMeta +// // Invalid version +// meta.Version = 0 +// return meta +// }, +// verifyBucket: func(t *testing.T, bkt *objstore.InMemBucket) { +// assert.Equal(t, validMeta, downloadMeta(t, bkt, uploadingMetaPath)) +// }, +// expBadRequest: fmt.Sprintf("version must be %d", block.TSDBVersion1), +// }, +// { +// name: "valid request when same in-flight meta file exists in object storage", +// setUp: func(t *testing.T, bkt *objstore.InMemBucket) block.Meta { +// marshalAndUploadJSON(t, bkt, uploadingMetaPath, validMeta) +// return validMeta +// }, +// verifyBucket: func(t *testing.T, bkt *objstore.InMemBucket) { +// expMeta := validMeta +// expMeta.Compaction.Sources = []ulid.ULID{expMeta.ULID} +// expMeta.Thanos.Source = "upload" +// assert.Equal(t, expMeta, downloadMeta(t, bkt, uploadingMetaPath)) +// }, +// }, +// { +// name: "valid request when different in-flight meta file exists in object storage", +// setUp: func(t *testing.T, bkt *objstore.InMemBucket) block.Meta { +// meta := validMeta +// meta.MinTime -= 1000 +// meta.MaxTime -= 1000 +// marshalAndUploadJSON(t, bkt, uploadingMetaPath, meta) + +// // Return meta file that differs from the one in bucket +// return validMeta +// }, +// verifyBucket: func(t *testing.T, bkt *objstore.InMemBucket) { +// expMeta := validMeta +// expMeta.Compaction.Sources = []ulid.ULID{expMeta.ULID} +// expMeta.Thanos.Source = "upload" +// assert.Equal(t, expMeta, downloadMeta(t, bkt, uploadingMetaPath)) +// }, +// }, +// } +// for _, tc := range extraCases { +// t.Run(tc.name, func(t *testing.T) { +// bkt := objstore.NewInMemBucket() +// meta := tc.setUp(t, bkt) +// metaJSON, err := json.Marshal(meta) +// require.NoError(t, err) + +// cfgProvider := newMockConfigProvider() +// cfgProvider.blockUploadEnabled[tenantID] = true +// c := &MultitenantCompactor{ +// logger: log.NewNopLogger(), +// bucketClient: bkt, +// cfgProvider: cfgProvider, +// } +// r := httptest.NewRequest(http.MethodPost, fmt.Sprintf("/api/v1/upload/block/%s/start", blockID), bytes.NewReader(metaJSON)) +// r = r.WithContext(user.InjectOrgID(r.Context(), tenantID)) +// r = mux.SetURLVars(r, map[string]string{"block": blockID}) +// w := httptest.NewRecorder() +// c.StartBlockUpload(w, r) + +// resp := w.Result() +// body, err := io.ReadAll(resp.Body) +// require.NoError(t, err) +// switch { +// case tc.expBadRequest != "": +// assert.Equal(t, http.StatusBadRequest, resp.StatusCode) +// assert.Equal(t, fmt.Sprintf("%s\n", tc.expBadRequest), string(body)) +// case tc.expConflict != "": +// assert.Equal(t, http.StatusConflict, resp.StatusCode) +// assert.Equal(t, fmt.Sprintf("%s\n", tc.expConflict), string(body)) +// default: +// assert.Equal(t, http.StatusOK, resp.StatusCode) +// assert.Empty(t, string(body)) +// } +// }) +// } +// } + +// // Test MultitenantCompactor.UploadBlockFile +// func TestMultitenantCompactor_UploadBlockFile(t *testing.T) { +// const tenantID = "test" +// const blockID = "01G3FZ0JWJYJC0ZM6Y9778P6KD" +// uploadingMetaFilename := fmt.Sprintf("uploading-%s", block.MetaFilename) +// uploadingMetaPath := path.Join(tenantID, blockID, uploadingMetaFilename) +// metaPath := path.Join(tenantID, blockID, block.MetaFilename) + +// chunkBodyContent := "content" +// validMeta := block.Meta{ +// BlockMeta: tsdb.BlockMeta{ +// ULID: ulid.MustParse(blockID), +// }, +// Thanos: block.ThanosMeta{ +// Labels: map[string]string{ +// mimir_tsdb.CompactorShardIDExternalLabel: "1_of_3", +// }, +// Files: []block.File{ +// { +// RelPath: "index", +// SizeBytes: 1, +// }, +// { +// RelPath: "chunks/000001", +// SizeBytes: int64(len(chunkBodyContent)), +// }, +// }, +// }, +// } + +// setupFnForValidRequest := func(bkt *bucket.ClientMock) { +// bkt.MockExists(metaPath, false, nil) + +// b, err := json.Marshal(validMeta) +// setUpGet(bkt, path.Join(tenantID, blockID, uploadingMetaFilename), b, err) +// setUpGet(bkt, path.Join(tenantID, blockID, validationFilename), nil, bucket.ErrObjectDoesNotExist) + +// bkt.MockUpload(path.Join(tenantID, blockID, "chunks/000001"), nil) +// } + +// verifyFuncForValidRequest := func(t *testing.T, bkt *bucket.ClientMock, expContent string) { +// var call mock.Call +// for _, c := range bkt.Calls { +// if c.Method == "Upload" { +// call = c +// break +// } +// } + +// rdr := call.Arguments[2].(io.Reader) +// got, err := io.ReadAll(rdr) +// require.NoError(t, err) +// assert.Equal(t, []byte(expContent), got) +// } + +// testCases := []struct { +// name string +// tenantID string +// blockID string +// path string +// body string +// unknownContentLength bool +// disableBlockUpload bool +// expBadRequest string +// expConflict string +// expNotFound string +// expInternalServerError bool +// setUpBucketMock func(bkt *bucket.ClientMock) +// verifyUpload func(*testing.T, *bucket.ClientMock, string) +// }{ +// { +// name: "without tenant ID", +// blockID: blockID, +// path: "chunks/000001", +// expBadRequest: "invalid tenant ID", +// }, +// { +// name: "without block ID", +// tenantID: tenantID, +// path: "chunks/000001", +// expBadRequest: "invalid block ID", +// }, +// { +// name: "invalid block ID", +// tenantID: tenantID, +// blockID: "1234", +// path: "chunks/000001", +// expBadRequest: "invalid block ID", +// }, +// { +// name: "without path", +// tenantID: tenantID, +// blockID: blockID, +// expBadRequest: "missing or invalid file path", +// }, +// { +// name: "invalid path", +// tenantID: tenantID, +// blockID: blockID, +// path: "../chunks/000001", +// expBadRequest: `invalid path: "../chunks/000001"`, +// }, +// { +// name: "empty file", +// tenantID: tenantID, +// blockID: blockID, +// path: "chunks/000001", +// expBadRequest: "file cannot be empty", +// }, +// { +// name: "attempt block metadata file", +// tenantID: tenantID, +// blockID: blockID, +// path: block.MetaFilename, +// body: "content", +// expBadRequest: fmt.Sprintf("%s is not allowed", block.MetaFilename), +// }, +// { +// name: "attempt in-flight block metadata file", +// tenantID: tenantID, +// blockID: blockID, +// path: uploadingMetaFilename, +// body: "content", +// expBadRequest: fmt.Sprintf("invalid path: %q", uploadingMetaFilename), +// }, +// { +// name: "block upload disabled", +// tenantID: tenantID, +// blockID: blockID, +// disableBlockUpload: true, +// path: "chunks/000001", +// expBadRequest: "block upload is disabled", +// }, +// { +// name: "complete block already exists", +// tenantID: tenantID, +// blockID: blockID, +// path: "chunks/000001", +// body: "content", +// setUpBucketMock: func(bkt *bucket.ClientMock) { +// bkt.MockExists(metaPath, true, nil) +// }, +// expConflict: "block already exists", +// }, +// { +// name: "failure checking for complete block", +// tenantID: tenantID, +// blockID: blockID, +// path: "chunks/000001", +// body: chunkBodyContent, +// setUpBucketMock: func(bkt *bucket.ClientMock) { +// bkt.MockExists(metaPath, false, fmt.Errorf("test")) +// }, +// expInternalServerError: true, +// }, +// { +// name: "failure checking for in-flight meta file", +// tenantID: tenantID, +// blockID: blockID, +// path: "chunks/000001", +// body: chunkBodyContent, +// setUpBucketMock: func(bkt *bucket.ClientMock) { +// bkt.MockExists(metaPath, false, nil) +// setUpGet(bkt, path.Join(tenantID, blockID, uploadingMetaFilename), nil, fmt.Errorf("test")) +// }, +// expInternalServerError: true, +// }, +// { +// name: "missing in-flight meta file", +// tenantID: tenantID, +// blockID: blockID, +// path: "chunks/000001", +// body: chunkBodyContent, +// setUpBucketMock: func(bkt *bucket.ClientMock) { +// bkt.MockExists(metaPath, false, nil) +// setUpGet(bkt, path.Join(tenantID, blockID, uploadingMetaFilename), nil, bucket.ErrObjectDoesNotExist) +// }, +// expNotFound: "block upload not started", +// }, +// { +// name: "file upload fails", +// tenantID: tenantID, +// blockID: blockID, +// path: "chunks/000001", +// body: chunkBodyContent, +// setUpBucketMock: func(bkt *bucket.ClientMock) { +// bkt.MockExists(metaPath, false, nil) + +// b, err := json.Marshal(validMeta) +// setUpGet(bkt, path.Join(tenantID, blockID, uploadingMetaFilename), b, err) +// setUpGet(bkt, path.Join(tenantID, blockID, validationFilename), nil, bucket.ErrObjectDoesNotExist) + +// bkt.MockUpload(path.Join(tenantID, blockID, "chunks/000001"), fmt.Errorf("test")) +// }, +// expInternalServerError: true, +// }, +// { +// name: "invalid file size", +// tenantID: tenantID, +// blockID: blockID, +// path: "chunks/000001", +// body: chunkBodyContent + chunkBodyContent, +// setUpBucketMock: func(bkt *bucket.ClientMock) { +// bkt.MockExists(metaPath, false, nil) + +// b, err := json.Marshal(validMeta) +// setUpGet(bkt, path.Join(tenantID, blockID, uploadingMetaFilename), b, err) +// setUpGet(bkt, path.Join(tenantID, blockID, validationFilename), nil, bucket.ErrObjectDoesNotExist) +// }, +// expBadRequest: "file size doesn't match meta.json", +// }, +// { +// name: "unexpected file", +// tenantID: tenantID, +// blockID: blockID, +// path: "chunks/111111", +// body: chunkBodyContent, +// setUpBucketMock: func(bkt *bucket.ClientMock) { +// bkt.MockExists(metaPath, false, nil) + +// b, err := json.Marshal(validMeta) +// setUpGet(bkt, path.Join(tenantID, blockID, uploadingMetaFilename), b, err) +// setUpGet(bkt, path.Join(tenantID, blockID, validationFilename), nil, bucket.ErrObjectDoesNotExist) +// }, +// expBadRequest: "unexpected file", +// }, +// { +// name: "valid request", +// tenantID: tenantID, +// blockID: blockID, +// path: "chunks/000001", +// body: chunkBodyContent, +// setUpBucketMock: setupFnForValidRequest, +// verifyUpload: verifyFuncForValidRequest, +// }, +// { +// name: "valid request, with unknown content-length", +// tenantID: tenantID, +// blockID: blockID, +// path: "chunks/000001", +// body: chunkBodyContent, +// unknownContentLength: true, +// setUpBucketMock: setupFnForValidRequest, +// verifyUpload: verifyFuncForValidRequest, +// }, +// } +// for _, tc := range testCases { +// t.Run(tc.name, func(t *testing.T) { +// var bkt bucket.ClientMock +// if tc.setUpBucketMock != nil { +// tc.setUpBucketMock(&bkt) +// } + +// cfgProvider := newMockConfigProvider() +// cfgProvider.blockUploadEnabled[tc.tenantID] = !tc.disableBlockUpload +// c := &MultitenantCompactor{ +// logger: log.NewNopLogger(), +// bucketClient: &bkt, +// cfgProvider: cfgProvider, +// } +// var rdr io.Reader +// if tc.body != "" { +// rdr = strings.NewReader(tc.body) +// } +// r := httptest.NewRequest(http.MethodPost, fmt.Sprintf("/api/v1/upload/block/%s/files?path=%s", blockID, url.QueryEscape(tc.path)), rdr) +// if tc.tenantID != "" { +// r = r.WithContext(user.InjectOrgID(r.Context(), tenantID)) +// } +// if tc.blockID != "" { +// r = mux.SetURLVars(r, map[string]string{"block": tc.blockID}) +// } +// if tc.body != "" { +// r.ContentLength = int64(len(tc.body)) +// if tc.unknownContentLength { +// r.ContentLength = -1 +// } +// } +// w := httptest.NewRecorder() +// c.UploadBlockFile(w, r) + +// resp := w.Result() +// body, err := io.ReadAll(resp.Body) +// require.NoError(t, err) + +// switch { +// case tc.expBadRequest != "": +// assert.Equal(t, http.StatusBadRequest, resp.StatusCode) +// assert.Equal(t, fmt.Sprintf("%s\n", tc.expBadRequest), string(body)) +// case tc.expConflict != "": +// assert.Equal(t, http.StatusConflict, resp.StatusCode) +// assert.Equal(t, fmt.Sprintf("%s\n", tc.expConflict), string(body)) +// case tc.expNotFound != "": +// assert.Equal(t, http.StatusNotFound, resp.StatusCode) +// assert.Equal(t, fmt.Sprintf("%s\n", tc.expNotFound), string(body)) +// case tc.expInternalServerError: +// assert.Equal(t, http.StatusInternalServerError, resp.StatusCode) +// assert.Regexp(t, "internal server error \\(id [0-9a-f]{16}\\)\n", string(body)) +// default: +// assert.Equal(t, http.StatusOK, resp.StatusCode) +// assert.Empty(t, string(body)) +// } + +// bkt.AssertExpectations(t) + +// if tc.verifyUpload != nil { +// tc.verifyUpload(t, &bkt, tc.body) +// } +// }) +// } + +// type file struct { +// path string +// content string +// } + +// // Additional test cases using an in-memory bucket for state testing +// extraCases := []struct { +// name string +// files []file +// setUpBucket func(*testing.T, *objstore.InMemBucket) +// verifyBucket func(*testing.T, *objstore.InMemBucket, []file) +// }{ +// { +// name: "multiple sequential uploads of same file", +// files: []file{ +// { +// path: "chunks/000001", +// content: strings.Repeat("a", len(chunkBodyContent)), +// }, +// { +// path: "chunks/000001", +// content: strings.Repeat("b", len(chunkBodyContent)), +// }, +// }, +// setUpBucket: func(t *testing.T, bkt *objstore.InMemBucket) { +// marshalAndUploadJSON(t, bkt, uploadingMetaPath, validMeta) +// }, +// verifyBucket: func(t *testing.T, bkt *objstore.InMemBucket, files []file) { +// t.Helper() + +// ctx := context.Background() +// rdr, err := bkt.Get(ctx, path.Join(tenantID, blockID, files[1].path)) +// require.NoError(t, err) +// t.Cleanup(func() { +// _ = rdr.Close() +// }) + +// content, err := io.ReadAll(rdr) +// require.NoError(t, err) +// assert.Equal(t, files[1].content, string(content)) +// }, +// }, +// } +// for _, tc := range extraCases { +// t.Run(tc.name, func(t *testing.T) { +// bkt := objstore.NewInMemBucket() +// tc.setUpBucket(t, bkt) +// cfgProvider := newMockConfigProvider() +// cfgProvider.blockUploadEnabled[tenantID] = true +// c := &MultitenantCompactor{ +// logger: log.NewNopLogger(), +// bucketClient: bkt, +// cfgProvider: cfgProvider, +// } + +// for _, f := range tc.files { +// rdr := strings.NewReader(f.content) +// r := httptest.NewRequest(http.MethodPost, fmt.Sprintf("/api/v1/upload/block/%s/files?path=%s", blockID, url.QueryEscape(f.path)), rdr) +// urlVars := map[string]string{ +// "block": blockID, +// } +// r = mux.SetURLVars(r, urlVars) +// r = r.WithContext(user.InjectOrgID(r.Context(), tenantID)) +// w := httptest.NewRecorder() +// c.UploadBlockFile(w, r) + +// resp := w.Result() +// body, err := io.ReadAll(resp.Body) +// require.NoError(t, err) +// require.Equal(t, http.StatusOK, resp.StatusCode) +// require.Empty(t, body) +// } + +// tc.verifyBucket(t, bkt, tc.files) +// }) +// } +// } + +// func setUpGet(bkt *bucket.ClientMock, pth string, content []byte, err error) { +// bkt.On("Get", mock.Anything, pth).Return(func(_ context.Context, _ string) (io.ReadCloser, error) { +// return io.NopCloser(bytes.NewReader(content)), err +// }) +// } + +// // Test MultitenantCompactor.FinishBlockUpload +// func TestMultitenantCompactor_FinishBlockUpload(t *testing.T) { +// const tenantID = "test" +// const blockID = "01G3FZ0JWJYJC0ZM6Y9778P6KD" +// uploadingMetaPath := path.Join(tenantID, blockID, uploadingMetaFilename) +// metaPath := path.Join(tenantID, blockID, block.MetaFilename) +// injectedError := fmt.Errorf("injected error") +// validMeta := block.Meta{ +// BlockMeta: tsdb.BlockMeta{ +// Version: block.TSDBVersion1, +// ULID: ulid.MustParse(blockID), +// }, +// Thanos: block.ThanosMeta{ +// Labels: map[string]string{ +// mimir_tsdb.CompactorShardIDExternalLabel: "1_of_3", +// }, +// Files: []block.File{ +// { +// RelPath: "index", +// SizeBytes: 1, +// }, +// { +// RelPath: "chunks/000001", +// SizeBytes: 2, +// }, +// }, +// }, +// } + +// validSetup := func(t *testing.T, bkt objstore.Bucket) { +// err := marshalAndUploadToBucket(context.Background(), bkt, uploadingMetaPath, validMeta) +// require.NoError(t, err) +// for _, file := range validMeta.Thanos.Files { +// content := bytes.NewReader(make([]byte, file.SizeBytes)) +// err = bkt.Upload(context.Background(), path.Join(tenantID, blockID, file.RelPath), content) +// require.NoError(t, err) +// } +// } + +// testCases := []struct { +// name string +// tenantID string +// blockID string +// setUpBucket func(*testing.T, objstore.Bucket) +// errorInjector func(op bucket.Operation, name string) error +// disableBlockUpload bool +// enableValidation bool // should only be set to true for tests that fail before validation is started +// maxConcurrency int +// setConcurrency int64 +// expBadRequest string +// expConflict string +// expNotFound string +// expTooManyRequests bool +// expInternalServerError bool +// }{ +// { +// name: "without tenant ID", +// blockID: blockID, +// expBadRequest: "invalid tenant ID", +// }, +// { +// name: "without block ID", +// tenantID: tenantID, +// expBadRequest: "invalid block ID", +// }, +// { +// name: "invalid block ID", +// tenantID: tenantID, +// blockID: "1234", +// expBadRequest: "invalid block ID", +// }, +// { +// name: "block upload disabled", +// tenantID: tenantID, +// blockID: blockID, +// disableBlockUpload: true, +// expBadRequest: "block upload is disabled", +// }, +// { +// name: "complete block already exists", +// tenantID: tenantID, +// blockID: blockID, +// setUpBucket: func(t *testing.T, bkt objstore.Bucket) { +// err := marshalAndUploadToBucket(context.Background(), bkt, metaPath, validMeta) +// require.NoError(t, err) +// }, +// expConflict: "block already exists", +// }, +// { +// name: "checking for complete block fails", +// tenantID: tenantID, +// blockID: blockID, +// errorInjector: bucket.InjectErrorOn(bucket.OpExists, metaPath, injectedError), +// expInternalServerError: true, +// }, +// { +// name: "missing in-flight meta file", +// tenantID: tenantID, +// blockID: blockID, +// expNotFound: "block upload not started", +// }, +// { +// name: "downloading in-flight meta file fails", +// tenantID: tenantID, +// blockID: blockID, +// setUpBucket: func(t *testing.T, bkt objstore.Bucket) { +// err := marshalAndUploadToBucket(context.Background(), bkt, uploadingMetaPath, validMeta) +// require.NoError(t, err) +// }, +// errorInjector: bucket.InjectErrorOn(bucket.OpGet, uploadingMetaPath, injectedError), +// expInternalServerError: true, +// }, +// { +// name: "corrupt in-flight meta file", +// tenantID: tenantID, +// blockID: blockID, +// setUpBucket: func(t *testing.T, bkt objstore.Bucket) { +// err := bkt.Upload(context.Background(), uploadingMetaPath, bytes.NewReader([]byte("{"))) +// require.NoError(t, err) +// }, +// expInternalServerError: true, +// }, +// { +// name: "uploading meta file fails", +// tenantID: tenantID, +// blockID: blockID, +// setUpBucket: validSetup, +// errorInjector: bucket.InjectErrorOn(bucket.OpUpload, metaPath, injectedError), +// expInternalServerError: true, +// }, +// { +// name: "too many concurrent validations", +// tenantID: tenantID, +// blockID: blockID, +// setUpBucket: validSetup, +// enableValidation: true, +// maxConcurrency: 2, +// setConcurrency: 2, +// expTooManyRequests: true, +// }, +// } +// for _, tc := range testCases { +// t.Run(tc.name, func(t *testing.T) { +// bkt := objstore.NewInMemBucket() +// injectedBkt := bucket.ErrorInjectedBucketClient{ +// Bucket: bkt, +// Injector: tc.errorInjector, +// } +// if tc.setUpBucket != nil { +// tc.setUpBucket(t, bkt) +// } + +// cfgProvider := newMockConfigProvider() +// cfgProvider.blockUploadEnabled[tc.tenantID] = !tc.disableBlockUpload +// cfgProvider.blockUploadValidationEnabled[tc.tenantID] = tc.enableValidation +// c := &MultitenantCompactor{ +// logger: log.NewNopLogger(), +// bucketClient: &injectedBkt, +// cfgProvider: cfgProvider, +// } +// c.compactorCfg.MaxBlockUploadValidationConcurrency = tc.maxConcurrency +// if tc.setConcurrency > 0 { +// c.blockUploadValidations.Add(tc.setConcurrency) +// } + +// c.compactorCfg.DataDir = t.TempDir() + +// r := httptest.NewRequest(http.MethodPost, fmt.Sprintf("/api/v1/upload/block/%s/finish", tc.blockID), nil) +// if tc.tenantID != "" { +// r = r.WithContext(user.InjectOrgID(r.Context(), tenantID)) +// } +// if tc.blockID != "" { +// r = mux.SetURLVars(r, map[string]string{"block": tc.blockID}) +// } +// w := httptest.NewRecorder() +// c.FinishBlockUpload(w, r) + +// resp := w.Result() +// body, err := io.ReadAll(resp.Body) +// require.NoError(t, err) + +// switch { +// case tc.expBadRequest != "": +// assert.Equal(t, http.StatusBadRequest, resp.StatusCode) +// assert.Equal(t, fmt.Sprintf("%s\n", tc.expBadRequest), string(body)) +// case tc.expConflict != "": +// assert.Equal(t, http.StatusConflict, resp.StatusCode) +// assert.Equal(t, fmt.Sprintf("%s\n", tc.expConflict), string(body)) +// case tc.expNotFound != "": +// assert.Equal(t, http.StatusNotFound, resp.StatusCode) +// assert.Equal(t, fmt.Sprintf("%s\n", tc.expNotFound), string(body)) +// case tc.expInternalServerError: +// assert.Equal(t, http.StatusInternalServerError, resp.StatusCode) +// assert.Regexp(t, "internal server error \\(id [0-9a-f]{16}\\)\n", string(body)) +// case tc.expTooManyRequests: +// assert.Equal(t, http.StatusTooManyRequests, resp.StatusCode) +// assert.Equal(t, "too many block upload validations in progress, limit is 2\n", string(body)) +// default: +// assert.Equal(t, http.StatusOK, resp.StatusCode) +// assert.Empty(t, string(body)) +// exists, err := bkt.Exists(context.Background(), path.Join(tc.blockID, block.MetaFilename)) +// require.NoError(t, err) +// require.True(t, exists) +// } +// }) +// } +// } + +// func TestMultitenantCompactor_ValidateAndComplete(t *testing.T) { +// const tenantID = "test" +// const blockID = "01G3FZ0JWJYJC0ZM6Y9778P6KD" +// injectedError := fmt.Errorf("injected error") + +// uploadingMetaPath := path.Join(tenantID, blockID, uploadingMetaFilename) +// validationPath := path.Join(tenantID, blockID, validationFilename) +// metaPath := path.Join(tenantID, blockID, block.MetaFilename) + +// validationSucceeds := func(_ context.Context) error { return nil } + +// testCases := []struct { +// name string +// errorInjector func(op bucket.Operation, name string) error +// validation func(context.Context) error +// expectValidationFile bool +// expectErrorInValidationFile bool +// expectTempUploadingMeta bool +// expectMeta bool +// }{ +// { +// name: "validation fails", +// validation: func(_ context.Context) error { return injectedError }, +// expectValidationFile: true, +// expectErrorInValidationFile: true, +// expectTempUploadingMeta: true, +// expectMeta: false, +// }, +// { +// name: "validation fails, uploading error fails", +// errorInjector: bucket.InjectErrorOn(bucket.OpUpload, validationPath, injectedError), +// validation: func(_ context.Context) error { return injectedError }, +// expectValidationFile: true, +// expectErrorInValidationFile: false, +// expectTempUploadingMeta: true, +// expectMeta: false, +// }, +// { +// name: "uploading meta file fails", +// errorInjector: bucket.InjectErrorOn(bucket.OpUpload, metaPath, injectedError), +// validation: validationSucceeds, +// expectValidationFile: true, +// expectErrorInValidationFile: true, +// expectTempUploadingMeta: true, +// expectMeta: false, +// }, +// { +// name: "uploading meta file fails, uploading error fails", +// errorInjector: func(op bucket.Operation, target string) error { +// if op == bucket.OpUpload && (target == metaPath || target == validationPath) { +// return injectedError +// } +// return nil +// }, +// validation: validationSucceeds, +// expectValidationFile: true, +// expectErrorInValidationFile: false, +// expectTempUploadingMeta: true, +// expectMeta: false, +// }, +// { +// name: "removing in-flight meta file fails", +// errorInjector: bucket.InjectErrorOn(bucket.OpDelete, uploadingMetaPath, injectedError), +// validation: validationSucceeds, +// expectValidationFile: false, +// expectTempUploadingMeta: true, +// expectMeta: true, +// }, +// { +// name: "removing validation file fails", +// errorInjector: bucket.InjectErrorOn(bucket.OpDelete, validationPath, injectedError), +// validation: validationSucceeds, +// expectValidationFile: true, +// expectErrorInValidationFile: false, +// expectTempUploadingMeta: false, +// expectMeta: true, +// }, +// { +// name: "valid request", +// validation: validationSucceeds, +// expectValidationFile: false, +// expectTempUploadingMeta: false, +// expectMeta: true, +// }, +// } + +// for _, tc := range testCases { +// t.Run(tc.name, func(t *testing.T) { +// bkt := objstore.NewInMemBucket() +// var injectedBkt objstore.Bucket = bkt +// if tc.errorInjector != nil { +// injectedBkt = &bucket.ErrorInjectedBucketClient{ +// Bucket: bkt, +// Injector: tc.errorInjector, +// } +// } +// cfgProvider := newMockConfigProvider() +// c := &MultitenantCompactor{ +// logger: log.NewNopLogger(), +// bucketClient: injectedBkt, +// cfgProvider: cfgProvider, +// blockUploadBlocks: promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{tenantID}), +// blockUploadBytes: promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{tenantID}), +// blockUploadFiles: promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{tenantID}), +// } +// userBkt := bucket.NewUserBucketClient(tenantID, injectedBkt, cfgProvider) + +// meta := block.Meta{} +// marshalAndUploadJSON(t, bkt, uploadingMetaPath, meta) +// v := validationFile{} +// marshalAndUploadJSON(t, bkt, validationPath, v) + +// c.validateAndCompleteBlockUpload(log.NewNopLogger(), tenantID, userBkt, ulid.MustParse(blockID), &meta, tc.validation) + +// tempUploadingMetaExists, err := bkt.Exists(context.Background(), uploadingMetaPath) +// require.NoError(t, err) +// require.Equal(t, tempUploadingMetaExists, tc.expectTempUploadingMeta) + +// metaExists, err := bkt.Exists(context.Background(), metaPath) +// require.NoError(t, err) +// require.Equal(t, metaExists, tc.expectMeta) + +// if !tc.expectValidationFile { +// exists, err := bkt.Exists(context.Background(), validationPath) +// require.NoError(t, err) +// require.False(t, exists) +// return +// } + +// r, err := bkt.Get(context.Background(), validationPath) +// require.NoError(t, err) +// decoder := json.NewDecoder(r) +// err = decoder.Decode(&v) +// require.NoError(t, err) + +// if tc.expectErrorInValidationFile { +// require.NotEmpty(t, v.Error) +// } else { +// require.Empty(t, v.Error) +// } +// }) +// } +// } + +// func TestMultitenantCompactor_ValidateBlock(t *testing.T) { +// const tenantID = "test" +// ctx := context.Background() +// tmpDir := t.TempDir() +// bkt := objstore.NewInMemBucket() + +// type Missing uint8 +// const ( +// MissingMeta Missing = 1 << iota +// MissingIndex +// MissingChunks +// ) + +// validLabels := func() []labels.Labels { +// return []labels.Labels{ +// labels.FromStrings("a", "1"), +// labels.FromStrings("b", "2"), +// labels.FromStrings("c", "3"), +// } +// } + +// testCases := []struct { +// name string +// lbls func() []labels.Labels +// metaInject func(meta *block.Meta) +// indexInject func(fname string) +// chunkInject func(fname string) +// populateFileList bool +// maximumBlockSize int64 +// verifyChunks bool +// missing Missing +// expectError bool +// expectedMsg string +// }{ +// { +// name: "valid block", +// lbls: validLabels, +// verifyChunks: true, +// expectError: false, +// populateFileList: true, +// }, +// { +// name: "maximum block size exceeded", +// lbls: validLabels, +// populateFileList: true, +// maximumBlockSize: 1, +// expectError: true, +// expectedMsg: fmt.Sprintf(maxBlockUploadSizeBytesFormat, 1), +// }, +// { +// name: "missing meta file", +// lbls: validLabels, +// missing: MissingMeta, +// expectError: true, +// expectedMsg: "failed renaming while preparing block for validation", +// }, +// { +// name: "missing index file", +// lbls: validLabels, +// missing: MissingIndex, +// expectError: true, +// expectedMsg: "error validating block: open index file:", +// }, +// { +// name: "missing chunks file", +// lbls: validLabels, +// populateFileList: true, +// missing: MissingChunks, +// expectError: true, +// expectedMsg: "failed to stat chunks/", +// }, +// { +// name: "file size mismatch", +// lbls: validLabels, +// metaInject: func(meta *block.Meta) { +// require.Greater(t, len(meta.Thanos.Files), 0) +// meta.Thanos.Files[0].SizeBytes += 10 +// }, +// populateFileList: true, +// expectError: true, +// expectedMsg: "file size mismatch", +// }, +// { +// name: "empty index file", +// lbls: validLabels, +// indexInject: func(fname string) { +// require.NoError(t, os.Truncate(fname, 0)) +// }, +// expectError: true, +// expectedMsg: "error validating block: open index file: mmap, size 0: invalid argument", +// }, +// { +// name: "index file invalid magic number", +// lbls: validLabels, +// indexInject: func(fname string) { +// flipByteAt(t, fname, 0) // guaranteed to be a magic number byte +// }, +// expectError: true, +// expectedMsg: "error validating block: open index file: invalid magic number", +// }, +// { +// name: "out of order labels", +// lbls: func() []labels.Labels { +// b := labels.NewScratchBuilder(2) +// b.Add("d", "4") +// b.Add("a", "1") +// oooLabels := []labels.Labels{ +// b.Labels(), // Haven't called Sort(), so they will be out of order. +// labels.FromStrings("b", "2"), +// labels.FromStrings("c", "3"), +// } +// return oooLabels +// }, +// expectError: true, +// expectedMsg: "error validating block: index contains 1 postings with out of order labels", +// }, +// { +// name: "segment file invalid magic number", +// lbls: validLabels, +// chunkInject: func(fname string) { +// flipByteAt(t, fname, 0) // guaranteed to be a magic number byte +// }, +// verifyChunks: true, +// expectError: true, +// expectedMsg: "invalid magic number", +// }, +// { +// name: "segment file invalid checksum", +// lbls: validLabels, +// chunkInject: func(fname string) { +// flipByteAt(t, fname, 12) // guaranteed to be a data byte +// }, +// populateFileList: true, +// verifyChunks: true, +// expectError: true, +// expectedMsg: "checksum mismatch", +// }, +// { +// name: "empty segment file", +// lbls: validLabels, +// chunkInject: func(fname string) { +// require.NoError(t, os.Truncate(fname, 0)) +// }, +// verifyChunks: true, +// expectError: true, +// expectedMsg: "size 0: invalid argument", +// }, +// } + +// for _, tc := range testCases { +// t.Run(tc.name, func(t *testing.T) { +// // create a test block +// now := time.Now() +// blockID, err := block.CreateBlock(ctx, tmpDir, tc.lbls(), 300, now.Add(-2*time.Hour).UnixMilli(), now.UnixMilli(), labels.EmptyLabels()) +// require.NoError(t, err) +// testDir := filepath.Join(tmpDir, blockID.String()) +// meta, err := block.ReadMetaFromDir(testDir) +// require.NoError(t, err) +// if tc.populateFileList { +// stats, err := block.GatherFileStats(testDir) +// require.NoError(t, err) +// meta.Thanos.Files = stats +// } + +// // create a compactor +// cfgProvider := newMockConfigProvider() +// cfgProvider.blockUploadValidationEnabled[tenantID] = true +// cfgProvider.verifyChunks[tenantID] = tc.verifyChunks +// cfgProvider.blockUploadMaxBlockSizeBytes[tenantID] = tc.maximumBlockSize +// c := &MultitenantCompactor{ +// logger: log.NewNopLogger(), +// bucketClient: bkt, +// cfgProvider: cfgProvider, +// } + +// // upload the block +// require.NoError(t, block.Upload(ctx, log.NewNopLogger(), bkt, testDir, nil)) +// // remove meta.json as we will be uploading a new one with the uploading meta name +// require.NoError(t, bkt.Delete(ctx, path.Join(blockID.String(), block.MetaFilename))) + +// // handle meta file +// if tc.metaInject != nil { +// tc.metaInject(meta) +// } +// var metaBody bytes.Buffer +// require.NoError(t, meta.Write(&metaBody)) + +// // replace index file +// if tc.indexInject != nil { +// indexFile := filepath.Join(testDir, block.IndexFilename) +// indexObject := path.Join(blockID.String(), block.IndexFilename) +// require.NoError(t, bkt.Delete(ctx, indexObject)) +// tc.indexInject(indexFile) +// uploadLocalFileToBucket(ctx, t, bkt, indexFile, indexObject) +// } + +// // replace segment file +// if tc.chunkInject != nil { +// segmentFile := filepath.Join(testDir, block.ChunksDirname, "000001") +// segmentObject := path.Join(blockID.String(), block.ChunksDirname, "000001") +// require.NoError(t, bkt.Delete(ctx, segmentObject)) +// tc.chunkInject(segmentFile) +// uploadLocalFileToBucket(ctx, t, bkt, segmentFile, segmentObject) +// } + +// // delete any files that should be missing +// if tc.missing&MissingIndex != 0 { +// require.NoError(t, bkt.Delete(ctx, path.Join(blockID.String(), block.IndexFilename))) +// } + +// if tc.missing&MissingChunks != 0 { +// chunkDir := path.Join(blockID.String(), block.ChunksDirname) +// err := bkt.Iter(ctx, chunkDir, func(name string) error { +// require.NoError(t, bkt.Delete(ctx, name)) +// return nil +// }) +// require.NoError(t, err) +// } + +// // only upload renamed meta file if it is not meant to be missing +// if tc.missing&MissingMeta == 0 { +// // rename to uploading meta file as that is what validateBlock expects +// require.NoError(t, bkt.Upload(ctx, path.Join(blockID.String(), uploadingMetaFilename), &metaBody)) +// } + +// // validate the block +// err = c.validateBlock(ctx, c.logger, blockID, meta, bkt, tenantID) +// if tc.expectError { +// require.Error(t, err) +// require.Contains(t, err.Error(), tc.expectedMsg) +// } else { +// require.NoError(t, err) +// } +// }) +// } +// } + +// func TestMultitenantCompactor_PeriodicValidationUpdater(t *testing.T) { +// const tenantID = "test" +// const blockID = "01G3FZ0JWJYJC0ZM6Y9778P6KD" +// injectedError := fmt.Errorf("injected error") +// validationPath := path.Join(tenantID, blockID, validationFilename) + +// heartbeatInterval := 50 * time.Millisecond + +// validationExists := func(t *testing.T, bkt objstore.Bucket) bool { +// exists, err := bkt.Exists(context.Background(), validationPath) +// require.NoError(t, err) +// return exists +// } + +// testCases := []struct { +// name string +// errorInjector func(op bucket.Operation, name string) error +// cancelContext bool +// assertions func(t *testing.T, ctx context.Context, bkt objstore.Bucket) +// }{ +// { +// name: "updating validation file fails", +// errorInjector: bucket.InjectErrorOn(bucket.OpUpload, validationPath, injectedError), +// assertions: func(t *testing.T, ctx context.Context, bkt objstore.Bucket) { +// <-ctx.Done() +// require.True(t, errors.Is(context.Canceled, ctx.Err())) +// require.False(t, validationExists(t, bkt)) +// }, +// }, +// { +// name: "updating validation file succeeds", +// assertions: func(t *testing.T, ctx context.Context, bkt objstore.Bucket) { +// test.Poll(t, heartbeatInterval*2, true, func() interface{} { +// return validationExists(t, bkt) +// }) + +// v := validationFile{} +// r, err := bkt.Get(context.Background(), validationPath) +// require.NoError(t, err) +// decoder := json.NewDecoder(r) +// err = decoder.Decode(&v) +// require.NoError(t, err) +// require.NotEqual(t, 0, v.LastUpdate) +// require.Empty(t, v.Error) +// }, +// }, +// { +// name: "context cancelled before update", +// cancelContext: true, +// assertions: func(t *testing.T, ctx context.Context, bkt objstore.Bucket) { +// require.False(t, validationExists(t, bkt)) +// }, +// }, +// } + +// for _, tc := range testCases { +// t.Run(tc.name, func(t *testing.T) { +// bkt := objstore.NewInMemBucket() +// var injectedBkt objstore.Bucket = bkt +// if tc.errorInjector != nil { +// injectedBkt = &bucket.ErrorInjectedBucketClient{ +// Bucket: bkt, +// Injector: tc.errorInjector, +// } +// } + +// cfgProvider := newMockConfigProvider() +// c := &MultitenantCompactor{ +// logger: log.NewNopLogger(), +// bucketClient: injectedBkt, +// cfgProvider: cfgProvider, +// } +// userBkt := bucket.NewUserBucketClient(tenantID, injectedBkt, cfgProvider) +// ctx, cancel := context.WithCancel(context.Background()) + +// heartbeatInterval := heartbeatInterval +// if tc.cancelContext { +// cancel() +// heartbeatInterval = 1 * time.Hour // to avoid racing a heartbeat +// } + +// var wg sync.WaitGroup +// wg.Add(1) +// go func() { +// defer wg.Done() +// c.periodicValidationUpdater(ctx, log.NewNopLogger(), ulid.MustParse(blockID), userBkt, cancel, heartbeatInterval) +// }() + +// if !tc.cancelContext { +// time.Sleep(heartbeatInterval) +// } + +// tc.assertions(t, ctx, bkt) + +// cancel() +// wg.Wait() +// }) +// } +// } + +// func TestMultitenantCompactor_GetBlockUploadStateHandler(t *testing.T) { +// const ( +// tenantID = "tenant" +// blockID = "01G8X9GA8R6N8F75FW1J18G83N" +// ) + +// type testcase struct { +// setupBucket func(t *testing.T, bkt objstore.Bucket) +// disableBlockUpload bool +// expectedStatusCode int +// expectedBody string +// } + +// for name, tc := range map[string]testcase{ +// "block doesn't exist": { +// expectedStatusCode: http.StatusNotFound, +// expectedBody: "block doesn't exist", +// }, + +// "complete block": { +// setupBucket: func(t *testing.T, bkt objstore.Bucket) { +// marshalAndUploadJSON(t, bkt, path.Join(tenantID, blockID, block.MetaFilename), block.Meta{}) +// }, +// expectedStatusCode: http.StatusOK, +// expectedBody: `{"result":"complete"}`, +// }, + +// "upload in progress": { +// setupBucket: func(t *testing.T, bkt objstore.Bucket) { +// marshalAndUploadJSON(t, bkt, path.Join(tenantID, blockID, uploadingMetaFilename), block.Meta{}) +// }, +// expectedStatusCode: http.StatusOK, +// expectedBody: `{"result":"uploading"}`, +// }, + +// "validating": { +// setupBucket: func(t *testing.T, bkt objstore.Bucket) { +// marshalAndUploadJSON(t, bkt, path.Join(tenantID, blockID, uploadingMetaFilename), block.Meta{}) +// marshalAndUploadJSON(t, bkt, path.Join(tenantID, blockID, validationFilename), validationFile{LastUpdate: time.Now().UnixMilli()}) +// }, +// expectedStatusCode: http.StatusOK, +// expectedBody: `{"result":"validating"}`, +// }, + +// "validation failed": { +// setupBucket: func(t *testing.T, bkt objstore.Bucket) { +// marshalAndUploadJSON(t, bkt, path.Join(tenantID, blockID, uploadingMetaFilename), block.Meta{}) +// marshalAndUploadJSON(t, bkt, path.Join(tenantID, blockID, validationFilename), validationFile{LastUpdate: time.Now().UnixMilli(), Error: "error during validation"}) +// }, +// expectedStatusCode: http.StatusOK, +// expectedBody: `{"result":"failed","error":"error during validation"}`, +// }, + +// "stale validation file": { +// setupBucket: func(t *testing.T, bkt objstore.Bucket) { +// marshalAndUploadJSON(t, bkt, path.Join(tenantID, blockID, uploadingMetaFilename), block.Meta{}) +// marshalAndUploadJSON(t, bkt, path.Join(tenantID, blockID, validationFilename), validationFile{LastUpdate: time.Now().Add(-10 * time.Minute).UnixMilli()}) +// }, +// expectedStatusCode: http.StatusOK, +// expectedBody: `{"result":"uploading"}`, +// }, +// } { +// t.Run(name, func(t *testing.T) { +// bkt := objstore.NewInMemBucket() +// if tc.setupBucket != nil { +// tc.setupBucket(t, bkt) +// } + +// cfgProvider := newMockConfigProvider() +// cfgProvider.blockUploadEnabled[tenantID] = !tc.disableBlockUpload + +// c := &MultitenantCompactor{ +// logger: log.NewNopLogger(), +// bucketClient: bkt, +// cfgProvider: cfgProvider, +// } + +// r := httptest.NewRequest(http.MethodGet, fmt.Sprintf("/api/v1/upload/block/%s/check", blockID), nil) +// urlVars := map[string]string{"block": blockID} +// r = mux.SetURLVars(r, urlVars) +// r = r.WithContext(user.InjectOrgID(r.Context(), tenantID)) + +// w := httptest.NewRecorder() +// c.GetBlockUploadStateHandler(w, r) +// resp := w.Result() + +// body, err := io.ReadAll(resp.Body) + +// require.NoError(t, err) +// require.Equal(t, tc.expectedStatusCode, resp.StatusCode) +// require.Equal(t, tc.expectedBody, strings.TrimSpace(string(body))) +// }) +// } +// } + +// func TestMultitenantCompactor_ValidateMaximumBlockSize(t *testing.T) { +// const userID = "user" + +// type testCase struct { +// maximumBlockSize int64 +// fileSizes []int64 +// expectErr bool +// } + +// for name, tc := range map[string]testCase{ +// "no limit": { +// maximumBlockSize: 0, +// fileSizes: []int64{math.MaxInt64}, +// expectErr: false, +// }, +// "under limit": { +// maximumBlockSize: 4, +// fileSizes: []int64{1, 2}, +// expectErr: false, +// }, +// "under limit - zero size file included": { +// maximumBlockSize: 2, +// fileSizes: []int64{1, 0}, +// expectErr: false, +// }, +// "under limit - negative size file included": { +// maximumBlockSize: 2, +// fileSizes: []int64{2, -1}, +// expectErr: true, +// }, +// "exact limit": { +// maximumBlockSize: 3, +// fileSizes: []int64{1, 2}, +// expectErr: false, +// }, +// "over limit": { +// maximumBlockSize: 1, +// fileSizes: []int64{1, 1}, +// expectErr: true, +// }, +// "overflow": { +// maximumBlockSize: math.MaxInt64, +// fileSizes: []int64{math.MaxInt64, math.MaxInt64, math.MaxInt64}, +// expectErr: true, +// }, +// } { +// t.Run(name, func(t *testing.T) { +// files := make([]block.File, len(tc.fileSizes)) +// for i, size := range tc.fileSizes { +// files[i] = block.File{SizeBytes: size} +// } + +// cfgProvider := newMockConfigProvider() +// cfgProvider.blockUploadMaxBlockSizeBytes[userID] = tc.maximumBlockSize +// c := &MultitenantCompactor{ +// logger: log.NewNopLogger(), +// cfgProvider: cfgProvider, +// } + +// err := c.validateMaximumBlockSize(c.logger, files, userID) +// if tc.expectErr { +// require.Error(t, err) +// } else { +// require.NoError(t, err) +// } +// }) +// } +// } + +// func TestMultitenantCompactor_MarkBlockComplete(t *testing.T) { +// const tenantID = "test" +// const blockID = "01G3FZ0JWJYJC0ZM6Y9778P6KD" +// injectedError := fmt.Errorf("injected error") + +// uploadingMetaPath := path.Join(tenantID, blockID, uploadingMetaFilename) +// metaPath := path.Join(tenantID, blockID, block.MetaFilename) +// testCases := []struct { +// name string +// errorInjector func(op bucket.Operation, name string) error +// expectSuccess bool +// }{ +// { +// name: "marking block complete succeeds", +// expectSuccess: true, +// }, +// { +// name: "uploading meta file fails", +// errorInjector: bucket.InjectErrorOn(bucket.OpUpload, metaPath, injectedError), +// }, +// { +// name: "deleting uploading meta file fails", +// errorInjector: bucket.InjectErrorOn(bucket.OpDelete, uploadingMetaPath, injectedError), +// expectSuccess: true, +// }, +// } +// for _, tc := range testCases { +// t.Run(tc.name, func(t *testing.T) { +// bkt := objstore.NewInMemBucket() +// var injectedBkt objstore.Bucket = bkt +// if tc.errorInjector != nil { +// injectedBkt = &bucket.ErrorInjectedBucketClient{ +// Bucket: bkt, +// Injector: tc.errorInjector, +// } +// } +// cfgProvider := newMockConfigProvider() +// c := &MultitenantCompactor{ +// logger: log.NewNopLogger(), +// bucketClient: injectedBkt, +// cfgProvider: cfgProvider, +// blockUploadBlocks: promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{tenantID}), +// blockUploadBytes: promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{tenantID}), +// blockUploadFiles: promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{tenantID}), +// } +// userBkt := bucket.NewUserBucketClient(tenantID, injectedBkt, cfgProvider) + +// meta := block.Meta{ +// Thanos: block.ThanosMeta{ +// Files: []block.File{ +// { +// RelPath: "chunks/000001", +// SizeBytes: 42, +// }, +// { +// RelPath: "index", +// SizeBytes: 17, +// }, +// { +// RelPath: "meta.json", +// }, +// }, +// }, +// } +// marshalAndUploadJSON(t, bkt, uploadingMetaPath, meta) + +// ctx := context.Background() +// err := c.markBlockComplete(ctx, log.NewNopLogger(), tenantID, userBkt, ulid.MustParse(blockID), &meta) +// if tc.expectSuccess { +// require.NoError(t, err) +// assert.Equal(t, 1.0, promtest.ToFloat64(c.blockUploadBlocks.WithLabelValues(tenantID))) +// assert.Equal(t, 59.0, promtest.ToFloat64(c.blockUploadBytes.WithLabelValues(tenantID))) +// assert.Equal(t, 3.0, promtest.ToFloat64(c.blockUploadFiles.WithLabelValues(tenantID))) +// } else { +// require.Error(t, err) +// assert.Equal(t, 0.0, promtest.ToFloat64(c.blockUploadBlocks.WithLabelValues(tenantID))) +// assert.Equal(t, 0.0, promtest.ToFloat64(c.blockUploadBytes.WithLabelValues(tenantID))) +// assert.Equal(t, 0.0, promtest.ToFloat64(c.blockUploadFiles.WithLabelValues(tenantID))) +// } +// }) +// } +// } + +// // marshalAndUploadJSON is a test helper for uploading a meta file to a certain path in a bucket. +// func marshalAndUploadJSON(t *testing.T, bkt objstore.Bucket, pth string, val interface{}) { +// t.Helper() +// err := marshalAndUploadToBucket(context.Background(), bkt, pth, val) +// require.NoError(t, err) +// } + +// func uploadLocalFileToBucket(ctx context.Context, t *testing.T, bkt objstore.Bucket, src, dst string) { +// t.Helper() +// fd, err := os.Open(src) +// require.NoError(t, err) +// defer func(fd *os.File) { +// err := fd.Close() +// require.NoError(t, err) +// }(fd) +// require.NoError(t, bkt.Upload(ctx, dst, fd)) +// } + +// // flipByteAt flips a byte at a given offset in a file. +// func flipByteAt(t *testing.T, fname string, offset int64) { +// fd, err := os.OpenFile(fname, os.O_RDWR, 0o644) +// require.NoError(t, err) +// defer func(fd *os.File) { +// err := fd.Close() +// require.NoError(t, err) +// }(fd) +// var b [1]byte +// _, err = fd.ReadAt(b[:], offset) +// require.NoError(t, err) +// // alter the byte +// b[0] = 0xff - b[0] +// _, err = fd.WriteAt(b[:], offset) +// require.NoError(t, err) +// } + +// func TestHexTimeNowNano(t *testing.T) { +// v := hexTimeNowNano() +// require.Len(t, v, 16, "Should have exactly 16 characters") + +// require.NotEqual(t, strings.Repeat("0", 16), v, "Should not be all zeros") +// time.Sleep(time.Nanosecond) +// require.NotEqual(t, v, hexTimeNowNano(), "Should generate a different one.") +// } diff --git a/pkg/compactor/job.go b/pkg/compactor/job.go index 53d224a892..2ac15690b9 100644 --- a/pkg/compactor/job.go +++ b/pkg/compactor/job.go @@ -15,9 +15,9 @@ import ( "github.com/oklog/ulid" "github.com/pkg/errors" "github.com/prometheus/prometheus/model/labels" - "github.com/thanos-io/objstore" - "github.com/grafana/mimir/pkg/storage/tsdb/block" + "github.com/grafana/pyroscope/pkg/objstore" + "github.com/grafana/pyroscope/pkg/phlaredb/block" ) // Job holds a compaction job, which consists of a group of blocks that should be compacted together. @@ -60,10 +60,10 @@ func (job *Job) Key() string { // AppendMeta the block with the given meta to the job. func (job *Job) AppendMeta(meta *block.Meta) error { - if !labels.Equal(job.labels, labels.FromMap(meta.Thanos.Labels)) { + if !labels.Equal(job.labels, labels.FromMap(meta.Labels)) { return errors.New("block and group labels do not match") } - if job.resolution != meta.Thanos.Downsample.Resolution { + if job.resolution != meta.Downsample.Resolution { return errors.New("block and group resolution do not match") } @@ -88,7 +88,7 @@ func (job *Job) IDs() (ids []ulid.ULID) { // MinTime returns the min time across all job's blocks. func (job *Job) MinTime() int64 { if len(job.metasByMinTime) > 0 { - return job.metasByMinTime[0].MinTime + return int64(job.metasByMinTime[0].MinTime) } return math.MaxInt64 } @@ -97,8 +97,8 @@ func (job *Job) MinTime() int64 { func (job *Job) MaxTime() int64 { max := int64(math.MinInt64) for _, m := range job.metasByMinTime { - if m.MaxTime > max { - max = m.MaxTime + if int64(m.MaxTime) > max { + max = int64(m.MaxTime) } } return max diff --git a/pkg/compactor/label_remover_filter.go b/pkg/compactor/label_remover_filter.go index 6525453e26..435678c207 100644 --- a/pkg/compactor/label_remover_filter.go +++ b/pkg/compactor/label_remover_filter.go @@ -10,7 +10,7 @@ import ( "github.com/oklog/ulid" - "github.com/grafana/mimir/pkg/storage/tsdb/block" + "github.com/grafana/pyroscope/pkg/phlaredb/block" ) type LabelRemoverFilter struct { @@ -26,7 +26,7 @@ func NewLabelRemoverFilter(labels []string) *LabelRemoverFilter { func (f *LabelRemoverFilter) Filter(_ context.Context, metas map[ulid.ULID]*block.Meta, _ block.GaugeVec) error { for _, meta := range metas { for _, l := range f.labels { - delete(meta.Thanos.Labels, l) + delete(meta.Labels, l) } } diff --git a/pkg/compactor/tenant_deletion_api.go b/pkg/compactor/tenant_deletion_api.go index c79630a02f..470e04fda7 100644 --- a/pkg/compactor/tenant_deletion_api.go +++ b/pkg/compactor/tenant_deletion_api.go @@ -5,94 +5,94 @@ package compactor -import ( - "context" - "net/http" - "strings" - "time" - - "github.com/go-kit/log/level" - "github.com/grafana/dskit/tenant" - "github.com/oklog/ulid" - "github.com/pkg/errors" - - "github.com/grafana/mimir/pkg/storage/bucket" - mimir_tsdb "github.com/grafana/mimir/pkg/storage/tsdb" - "github.com/grafana/mimir/pkg/util" -) - -func (c *MultitenantCompactor) DeleteTenant(w http.ResponseWriter, r *http.Request) { - ctx := r.Context() - userID, err := tenant.TenantID(ctx) - if err != nil { - // When Mimir is running, it uses Auth Middleware for checking X-Scope-OrgID and injecting tenant into context. - // Auth Middleware sends http.StatusUnauthorized if X-Scope-OrgID is missing, so we do too here, for consistency. - http.Error(w, err.Error(), http.StatusUnauthorized) - return - } - - err = mimir_tsdb.WriteTenantDeletionMark(r.Context(), c.bucketClient, userID, c.cfgProvider, mimir_tsdb.NewTenantDeletionMark(time.Now())) - if err != nil { - level.Error(c.logger).Log("msg", "failed to write tenant deletion mark", "user", userID, "err", err) - - http.Error(w, err.Error(), http.StatusInternalServerError) - return - } - - level.Info(c.logger).Log("msg", "tenant deletion mark in blocks storage created", "user", userID) - - w.WriteHeader(http.StatusOK) -} - -type DeleteTenantStatusResponse struct { - TenantID string `json:"tenant_id"` - BlocksDeleted bool `json:"blocks_deleted"` -} - -func (c *MultitenantCompactor) DeleteTenantStatus(w http.ResponseWriter, r *http.Request) { - ctx := r.Context() - userID, err := tenant.TenantID(ctx) - if err != nil { - http.Error(w, err.Error(), http.StatusBadRequest) - return - } - - result := DeleteTenantStatusResponse{} - result.TenantID = userID - result.BlocksDeleted, err = c.isBlocksForUserDeleted(ctx, userID) - if err != nil { - http.Error(w, err.Error(), http.StatusInternalServerError) - return - } - - util.WriteJSONResponse(w, result) -} - -func (c *MultitenantCompactor) isBlocksForUserDeleted(ctx context.Context, userID string) (bool, error) { - errBlockFound := errors.New("block found") - - userBucket := bucket.NewUserBucketClient(userID, c.bucketClient, c.cfgProvider) - err := userBucket.Iter(ctx, "", func(s string) error { - s = strings.TrimSuffix(s, "/") - - _, err := ulid.Parse(s) - if err != nil { - // not block, keep looking - return nil - } - - // Used as shortcut to stop iteration. - return errBlockFound - }) - - if errors.Is(err, errBlockFound) { - return false, nil - } - - if err != nil { - return false, err - } - - // No blocks found, all good. - return true, nil -} +// import ( +// "context" +// "net/http" +// "strings" +// "time" + +// "github.com/go-kit/log/level" +// "github.com/grafana/dskit/tenant" +// "github.com/oklog/ulid" +// "github.com/pkg/errors" + +// "github.com/grafana/mimir/pkg/storage/bucket" +// mimir_tsdb "github.com/grafana/mimir/pkg/storage/tsdb" +// "github.com/grafana/mimir/pkg/util" +// ) + +// func (c *MultitenantCompactor) DeleteTenant(w http.ResponseWriter, r *http.Request) { +// ctx := r.Context() +// userID, err := tenant.TenantID(ctx) +// if err != nil { +// // When Mimir is running, it uses Auth Middleware for checking X-Scope-OrgID and injecting tenant into context. +// // Auth Middleware sends http.StatusUnauthorized if X-Scope-OrgID is missing, so we do too here, for consistency. +// http.Error(w, err.Error(), http.StatusUnauthorized) +// return +// } + +// err = mimir_tsdb.WriteTenantDeletionMark(r.Context(), c.bucketClient, userID, c.cfgProvider, mimir_tsdb.NewTenantDeletionMark(time.Now())) +// if err != nil { +// level.Error(c.logger).Log("msg", "failed to write tenant deletion mark", "user", userID, "err", err) + +// http.Error(w, err.Error(), http.StatusInternalServerError) +// return +// } + +// level.Info(c.logger).Log("msg", "tenant deletion mark in blocks storage created", "user", userID) + +// w.WriteHeader(http.StatusOK) +// } + +// type DeleteTenantStatusResponse struct { +// TenantID string `json:"tenant_id"` +// BlocksDeleted bool `json:"blocks_deleted"` +// } + +// func (c *MultitenantCompactor) DeleteTenantStatus(w http.ResponseWriter, r *http.Request) { +// ctx := r.Context() +// userID, err := tenant.TenantID(ctx) +// if err != nil { +// http.Error(w, err.Error(), http.StatusBadRequest) +// return +// } + +// result := DeleteTenantStatusResponse{} +// result.TenantID = userID +// result.BlocksDeleted, err = c.isBlocksForUserDeleted(ctx, userID) +// if err != nil { +// http.Error(w, err.Error(), http.StatusInternalServerError) +// return +// } + +// util.WriteJSONResponse(w, result) +// } + +// func (c *MultitenantCompactor) isBlocksForUserDeleted(ctx context.Context, userID string) (bool, error) { +// errBlockFound := errors.New("block found") + +// userBucket := bucket.NewUserBucketClient(userID, c.bucketClient, c.cfgProvider) +// err := userBucket.Iter(ctx, "", func(s string) error { +// s = strings.TrimSuffix(s, "/") + +// _, err := ulid.Parse(s) +// if err != nil { +// // not block, keep looking +// return nil +// } + +// // Used as shortcut to stop iteration. +// return errBlockFound +// }) + +// if errors.Is(err, errBlockFound) { +// return false, nil +// } + +// if err != nil { +// return false, err +// } + +// // No blocks found, all good. +// return true, nil +// } diff --git a/pkg/compactor/tenant_deletion_api_test.go b/pkg/compactor/tenant_deletion_api_test.go index 09e146b825..cf961e0c47 100644 --- a/pkg/compactor/tenant_deletion_api_test.go +++ b/pkg/compactor/tenant_deletion_api_test.go @@ -4,97 +4,97 @@ // Provenance-includes-copyright: The Cortex Authors. package compactor -import ( - "bytes" - "context" - "net/http" - "net/http/httptest" - "path" - "testing" +// import ( +// "bytes" +// "context" +// "net/http" +// "net/http/httptest" +// "path" +// "testing" - "github.com/grafana/dskit/services" - "github.com/grafana/dskit/user" - "github.com/stretchr/testify/require" - "github.com/thanos-io/objstore" +// "github.com/grafana/dskit/services" +// "github.com/grafana/dskit/user" +// "github.com/stretchr/testify/require" +// "github.com/thanos-io/objstore" - "github.com/grafana/mimir/pkg/storage/tsdb" -) +// "github.com/grafana/mimir/pkg/storage/tsdb" +// ) -func TestDeleteTenant(t *testing.T) { - bkt := objstore.NewInMemBucket() - cfg := prepareConfig(t) - c, _, _, _, _ := prepare(t, cfg, bkt) - require.NoError(t, services.StartAndAwaitRunning(context.Background(), c)) - t.Cleanup(stopServiceFn(t, c)) +// func TestDeleteTenant(t *testing.T) { +// bkt := objstore.NewInMemBucket() +// cfg := prepareConfig(t) +// c, _, _, _, _ := prepare(t, cfg, bkt) +// require.NoError(t, services.StartAndAwaitRunning(context.Background(), c)) +// t.Cleanup(stopServiceFn(t, c)) - { - resp := httptest.NewRecorder() - c.DeleteTenant(resp, &http.Request{}) - require.Equal(t, http.StatusUnauthorized, resp.Code) - } +// { +// resp := httptest.NewRecorder() +// c.DeleteTenant(resp, &http.Request{}) +// require.Equal(t, http.StatusUnauthorized, resp.Code) +// } - { - ctx := context.Background() - ctx = user.InjectOrgID(ctx, "fake") +// { +// ctx := context.Background() +// ctx = user.InjectOrgID(ctx, "fake") - req := &http.Request{} - resp := httptest.NewRecorder() - c.DeleteTenant(resp, req.WithContext(ctx)) +// req := &http.Request{} +// resp := httptest.NewRecorder() +// c.DeleteTenant(resp, req.WithContext(ctx)) - require.Equal(t, http.StatusOK, resp.Code) - objs := bkt.Objects() - require.NotNil(t, objs[path.Join("fake", tsdb.TenantDeletionMarkPath)]) - } -} +// require.Equal(t, http.StatusOK, resp.Code) +// objs := bkt.Objects() +// require.NotNil(t, objs[path.Join("fake", tsdb.TenantDeletionMarkPath)]) +// } +// } -func TestDeleteTenantStatus(t *testing.T) { - const username = "user" +// func TestDeleteTenantStatus(t *testing.T) { +// const username = "user" - for name, tc := range map[string]struct { - objects map[string][]byte - expectedBlocksDeleted bool - }{ - "empty": { - objects: nil, - expectedBlocksDeleted: true, - }, +// for name, tc := range map[string]struct { +// objects map[string][]byte +// expectedBlocksDeleted bool +// }{ +// "empty": { +// objects: nil, +// expectedBlocksDeleted: true, +// }, - "no user objects": { - objects: map[string][]byte{ - "different-user/01EQK4QKFHVSZYVJ908Y7HH9E0/meta.json": []byte("data"), - }, - expectedBlocksDeleted: true, - }, +// "no user objects": { +// objects: map[string][]byte{ +// "different-user/01EQK4QKFHVSZYVJ908Y7HH9E0/meta.json": []byte("data"), +// }, +// expectedBlocksDeleted: true, +// }, - "non-block files": { - objects: map[string][]byte{ - "user/deletion-mark.json": []byte("data"), - }, - expectedBlocksDeleted: true, - }, +// "non-block files": { +// objects: map[string][]byte{ +// "user/deletion-mark.json": []byte("data"), +// }, +// expectedBlocksDeleted: true, +// }, - "block files": { - objects: map[string][]byte{ - "user/01EQK4QKFHVSZYVJ908Y7HH9E0/meta.json": []byte("data"), - }, - expectedBlocksDeleted: false, - }, - } { - t.Run(name, func(t *testing.T) { - bkt := objstore.NewInMemBucket() - // "upload" objects - for objName, data := range tc.objects { - require.NoError(t, bkt.Upload(context.Background(), objName, bytes.NewReader(data))) - } +// "block files": { +// objects: map[string][]byte{ +// "user/01EQK4QKFHVSZYVJ908Y7HH9E0/meta.json": []byte("data"), +// }, +// expectedBlocksDeleted: false, +// }, +// } { +// t.Run(name, func(t *testing.T) { +// bkt := objstore.NewInMemBucket() +// // "upload" objects +// for objName, data := range tc.objects { +// require.NoError(t, bkt.Upload(context.Background(), objName, bytes.NewReader(data))) +// } - cfg := prepareConfig(t) - c, _, _, _, _ := prepare(t, cfg, bkt) - require.NoError(t, services.StartAndAwaitRunning(context.Background(), c)) - t.Cleanup(stopServiceFn(t, c)) +// cfg := prepareConfig(t) +// c, _, _, _, _ := prepare(t, cfg, bkt) +// require.NoError(t, services.StartAndAwaitRunning(context.Background(), c)) +// t.Cleanup(stopServiceFn(t, c)) - res, err := c.isBlocksForUserDeleted(context.Background(), username) - require.NoError(t, err) - require.Equal(t, tc.expectedBlocksDeleted, res) - }) - } -} +// res, err := c.isBlocksForUserDeleted(context.Background(), username) +// require.NoError(t, err) +// require.Equal(t, tc.expectedBlocksDeleted, res) +// }) +// } +// } From a26efaf30b569f26429aa3569ffd2bd7d8bd2794 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Fri, 8 Sep 2023 09:51:44 +0200 Subject: [PATCH 12/74] Introduce a meta fetcher that synchronize filtered metadata of all block locally --- pkg/phlaredb/block/block.go | 7 +- pkg/phlaredb/block/fetcher.go | 606 ++++++++++++++++++++++ pkg/phlaredb/block/global_markers.go | 88 ++++ pkg/phlaredb/block/global_markers_test.go | 96 ++++ pkg/phlaredb/block/markers.go | 131 +++++ pkg/phlaredb/block/metadata.go | 22 +- pkg/util/extprom/tx_gauge.go | 102 ++++ 7 files changed, 1046 insertions(+), 6 deletions(-) create mode 100644 pkg/phlaredb/block/fetcher.go create mode 100644 pkg/phlaredb/block/global_markers.go create mode 100644 pkg/phlaredb/block/global_markers_test.go create mode 100644 pkg/phlaredb/block/markers.go create mode 100644 pkg/util/extprom/tx_gauge.go diff --git a/pkg/phlaredb/block/block.go b/pkg/phlaredb/block/block.go index a41fd2eb9a..40e9222604 100644 --- a/pkg/phlaredb/block/block.go +++ b/pkg/phlaredb/block/block.go @@ -20,9 +20,8 @@ import ( ) const ( - IndexFilename = "index.tsdb" - ParquetSuffix = ".parquet" - DeletionMarkFilename = "deletion-mark.json" + IndexFilename = "index.tsdb" + ParquetSuffix = ".parquet" HostnameLabel = "__hostname__" ) @@ -74,7 +73,7 @@ func upload(ctx context.Context, logger log.Logger, bkt objstore.Bucket, bdir st return errors.Wrap(err, "not a block dir") } - meta, err := ReadFromDir(bdir) + meta, err := ReadMetaFromDir(bdir) if err != nil { // No meta or broken meta file. return errors.Wrap(err, "read meta") diff --git a/pkg/phlaredb/block/fetcher.go b/pkg/phlaredb/block/fetcher.go new file mode 100644 index 0000000000..969c490a52 --- /dev/null +++ b/pkg/phlaredb/block/fetcher.go @@ -0,0 +1,606 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/grafana/mimir/blob/main/pkg/storage/tsdb/block/fetcher.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Thanos Authors. + +package block + +import ( + "context" + "encoding/json" + "io" + "os" + "path" + "path/filepath" + "sync" + "time" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/golang/groupcache/singleflight" + "github.com/grafana/dskit/multierror" + "github.com/grafana/dskit/runutil" + "github.com/oklog/ulid" + "github.com/pkg/errors" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" + "golang.org/x/sync/errgroup" + + "github.com/grafana/pyroscope/pkg/objstore" + "github.com/grafana/pyroscope/pkg/util/extprom" +) + +// FetcherMetrics holds metrics tracked by the metadata fetcher. This struct and its fields are exported +// to allow depending projects (eg. Cortex) to implement their own custom metadata fetcher while tracking +// compatible metrics. +type FetcherMetrics struct { + Syncs prometheus.Counter + SyncFailures prometheus.Counter + SyncDuration prometheus.Histogram + + Synced *extprom.TxGaugeVec +} + +// Submit applies new values for metrics tracked by transaction GaugeVec. +func (s *FetcherMetrics) Submit() { + s.Synced.Submit() +} + +// ResetTx starts new transaction for metrics tracked by transaction GaugeVec. +func (s *FetcherMetrics) ResetTx() { + s.Synced.ResetTx() +} + +const ( + fetcherSubSys = "blocks_meta" + + CorruptedMeta = "corrupted-meta-json" + NoMeta = "no-meta-json" + LoadedMeta = "loaded" + FailedMeta = "failed" + + // Synced label values. + labelExcludedMeta = "label-excluded" + timeExcludedMeta = "time-excluded" + duplicateMeta = "duplicate" + // Blocks that are marked for deletion can be loaded as well. This is done to make sure that we load blocks that are meant to be deleted, + // but don't have a replacement block yet. + MarkedForDeletionMeta = "marked-for-deletion" + + // MarkedForNoCompactionMeta is label for blocks which are loaded but also marked for no compaction. This label is also counted in `loaded` label metric. + MarkedForNoCompactionMeta = "marked-for-no-compact" +) + +func NewFetcherMetrics(reg prometheus.Registerer, syncedExtraLabels [][]string) *FetcherMetrics { + var m FetcherMetrics + + m.Syncs = promauto.With(reg).NewCounter(prometheus.CounterOpts{ + Subsystem: fetcherSubSys, + Name: "syncs_total", + Help: "Total blocks metadata synchronization attempts", + }) + m.SyncFailures = promauto.With(reg).NewCounter(prometheus.CounterOpts{ + Subsystem: fetcherSubSys, + Name: "sync_failures_total", + Help: "Total blocks metadata synchronization failures", + }) + m.SyncDuration = promauto.With(reg).NewHistogram(prometheus.HistogramOpts{ + Subsystem: fetcherSubSys, + Name: "sync_duration_seconds", + Help: "Duration of the blocks metadata synchronization in seconds", + Buckets: []float64{0.01, 1, 10, 100, 300, 600, 1000}, + }) + m.Synced = extprom.NewTxGaugeVec( + reg, + prometheus.GaugeOpts{ + Subsystem: fetcherSubSys, + Name: "synced", + Help: "Number of block metadata synced", + }, + []string{"state"}, + append([][]string{ + {CorruptedMeta}, + {NoMeta}, + {LoadedMeta}, + {FailedMeta}, + {labelExcludedMeta}, + {timeExcludedMeta}, + {duplicateMeta}, + {MarkedForDeletionMeta}, + {MarkedForNoCompactionMeta}, + }, syncedExtraLabels...)..., + ) + return &m +} + +type MetadataFetcher interface { + Fetch(ctx context.Context) (metas map[ulid.ULID]*Meta, partial map[ulid.ULID]error, err error) +} + +// GaugeVec hides something like a Prometheus GaugeVec or an extprom.TxGaugeVec. +type GaugeVec interface { + WithLabelValues(lvs ...string) prometheus.Gauge +} + +// MetadataFilter allows filtering or modifying metas from the provided map or returns error. +type MetadataFilter interface { + Filter(ctx context.Context, metas map[ulid.ULID]*Meta, synced GaugeVec) error +} + +// MetaFetcher is a struct that synchronizes filtered metadata of all block in the object storage with the local state. +// Go-routine safe. +type MetaFetcher struct { + logger log.Logger + concurrency int + bkt objstore.BucketReader + metrics *FetcherMetrics + filters []MetadataFilter + + // Optional local directory to cache meta.json files. + cacheDir string + g singleflight.Group + + mtx sync.Mutex + cached map[ulid.ULID]*Meta +} + +// NewMetaFetcher returns a MetaFetcher. +func NewMetaFetcher(logger log.Logger, concurrency int, bkt objstore.BucketReader, dir string, reg prometheus.Registerer, filters []MetadataFilter) (*MetaFetcher, error) { + if logger == nil { + logger = log.NewNopLogger() + } + + cacheDir := "" + if dir != "" { + cacheDir = filepath.Join(dir, "meta-syncer") + if err := os.MkdirAll(cacheDir, os.ModePerm); err != nil { + return nil, err + } + } + + return &MetaFetcher{ + logger: log.With(logger, "component", "block.MetaFetcher"), + concurrency: concurrency, + bkt: bkt, + cacheDir: cacheDir, + cached: map[ulid.ULID]*Meta{}, + metrics: NewFetcherMetrics(reg, nil), + filters: filters, + }, nil +} + +var ( + ErrorSyncMetaNotFound = errors.New("meta.json not found") + ErrorSyncMetaCorrupted = errors.New("meta.json corrupted") +) + +// loadMeta returns metadata from object storage or error. +// It returns ErrorSyncMetaNotFound and ErrorSyncMetaCorrupted sentinel errors in those cases. +func (f *MetaFetcher) loadMeta(ctx context.Context, id ulid.ULID) (*Meta, error) { + var ( + metaFile = path.Join(id.String(), MetaFilename) + cachedBlockDir = filepath.Join(f.cacheDir, id.String()) + ) + + // Block meta.json file is immutable, so we lookup the cache as first thing without issuing + // any API call to the object storage. This significantly reduce the pressure on the object + // storage. + // + // Details of all possible cases: + // + // - The block upload is in progress: the meta.json file is guaranteed to be uploaded at last. + // When we'll try to read it from object storage (later on), it will fail with ErrorSyncMetaNotFound + // which is correctly handled by the caller (partial block). + // + // - The block upload is completed: this is the normal case. meta.json file still exists in the + // object storage and it's expected to match the locally cached one (because it's immutable by design). + // - The block has been marked for deletion: the deletion hasn't started yet, so the full block (including + // the meta.json file) is still in the object storage. This case is not different than the previous one. + // + // - The block deletion is in progress: loadMeta() function may return the cached meta.json while it should + // return ErrorSyncMetaNotFound. This is a race condition that could happen even if we check the meta.json + // file in the storage, because the deletion could start right after we check it but before the MetaFetcher + // completes its sync. + // + // - The block has been deleted: the loadMeta() function will not be called at all, because the block + // was not discovered while iterating the bucket since all its files were already deleted. + if m, seen := f.cached[id]; seen { + return m, nil + } + + // Best effort load from local dir. + if f.cacheDir != "" { + m, err := ReadMetaFromDir(cachedBlockDir) + if err == nil { + return m, nil + } + + if !errors.Is(err, os.ErrNotExist) { + level.Warn(f.logger).Log("msg", "best effort read of the local meta.json failed; removing cached block dir", "dir", cachedBlockDir, "err", err) + if err := os.RemoveAll(cachedBlockDir); err != nil { + level.Warn(f.logger).Log("msg", "best effort remove of cached dir failed; ignoring", "dir", cachedBlockDir, "err", err) + } + } + } + + // todo(cyriltovena): we should use ReaderWithExpectedErrs(f.bkt.IsObjNotFoundErr) here, to avoid counting IsObjNotFoundErr as an error + // since this is expected + r, err := f.bkt.Get(ctx, metaFile) + if f.bkt.IsObjNotFoundErr(err) { + // Meta.json was deleted between bkt.Exists and here. + return nil, errors.Wrapf(ErrorSyncMetaNotFound, "%v", err) + } + if err != nil { + return nil, errors.Wrapf(err, "get meta file: %v", metaFile) + } + + defer runutil.CloseWithLogOnErr(f.logger, r, "close bkt meta get") + + metaContent, err := io.ReadAll(r) + if err != nil { + return nil, errors.Wrapf(err, "read meta file: %v", metaFile) + } + + m := &Meta{} + if err := json.Unmarshal(metaContent, m); err != nil { + return nil, errors.Wrapf(ErrorSyncMetaCorrupted, "meta.json %v unmarshal: %v", metaFile, err) + } + + if !m.Version.IsValid() { + return nil, errors.Errorf("unexpected meta file: %s version: %d", metaFile, m.Version) + } + + // Best effort cache in local dir. + if f.cacheDir != "" { + if err := os.MkdirAll(cachedBlockDir, os.ModePerm); err != nil { + level.Warn(f.logger).Log("msg", "best effort mkdir of the meta.json block dir failed; ignoring", "dir", cachedBlockDir, "err", err) + } + + if _, err := m.WriteToFile(f.logger, cachedBlockDir); err != nil { + level.Warn(f.logger).Log("msg", "best effort save of the meta.json to local dir failed; ignoring", "dir", cachedBlockDir, "err", err) + } + } + return m, nil +} + +type response struct { + metas map[ulid.ULID]*Meta + partial map[ulid.ULID]error + + // If metaErr > 0 it means incomplete view, so some metas, failed to be loaded. + metaErrs multierror.MultiError + + // Track the number of blocks not returned because of various reasons. + noMetasCount float64 + corruptedMetasCount float64 + markedForDeletionCount float64 +} + +func (f *MetaFetcher) fetchMetadata(ctx context.Context, excludeMarkedForDeletion bool) (interface{}, error) { + var ( + resp = response{ + metas: make(map[ulid.ULID]*Meta), + partial: make(map[ulid.ULID]error), + } + eg errgroup.Group + ch = make(chan ulid.ULID, f.concurrency) + mtx sync.Mutex + ) + level.Debug(f.logger).Log("msg", "fetching meta data", "concurrency", f.concurrency) + + // Get the list of blocks marked for deletion so that we'll exclude them (if required). + var markedForDeletion map[ulid.ULID]struct{} + if excludeMarkedForDeletion { + var err error + + markedForDeletion, err = ListBlockDeletionMarks(ctx, f.bkt) + if err != nil { + return nil, err + } + } + + // Run workers. + for i := 0; i < f.concurrency; i++ { + eg.Go(func() error { + for id := range ch { + meta, err := f.loadMeta(ctx, id) + if err == nil { + mtx.Lock() + resp.metas[id] = meta + mtx.Unlock() + continue + } + + if errors.Is(errors.Cause(err), ErrorSyncMetaNotFound) { + mtx.Lock() + resp.noMetasCount++ + mtx.Unlock() + } else if errors.Is(errors.Cause(err), ErrorSyncMetaCorrupted) { + mtx.Lock() + resp.corruptedMetasCount++ + mtx.Unlock() + } else { + mtx.Lock() + resp.metaErrs.Add(err) + mtx.Unlock() + continue + } + + mtx.Lock() + resp.partial[id] = err + mtx.Unlock() + } + return nil + }) + } + + // Workers scheduled, distribute blocks. + eg.Go(func() error { + defer close(ch) + return f.bkt.Iter(ctx, "", func(name string) error { + id, ok := IsBlockDir(name) + if !ok { + return nil + } + + // If requested, skip any block marked for deletion. + if _, marked := markedForDeletion[id]; excludeMarkedForDeletion && marked { + resp.markedForDeletionCount++ + return nil + } + + select { + case <-ctx.Done(): + return ctx.Err() + case ch <- id: + } + + return nil + }) + }) + + if err := eg.Wait(); err != nil { + return nil, errors.Wrap(err, "MetaFetcher: iter bucket") + } + + if len(resp.metaErrs) > 0 { + return resp, nil + } + + // Only for complete view of blocks update the cache. + cached := make(map[ulid.ULID]*Meta, len(resp.metas)) + for id, m := range resp.metas { + cached[id] = m + } + + f.mtx.Lock() + f.cached = cached + f.mtx.Unlock() + + // Best effort cleanup of disk-cached metas. + if f.cacheDir != "" { + fis, err := os.ReadDir(f.cacheDir) + names := make([]string, 0, len(fis)) + for _, fi := range fis { + names = append(names, fi.Name()) + } + if err != nil { + level.Warn(f.logger).Log("msg", "best effort remove of not needed cached dirs failed; ignoring", "err", err) + } else { + for _, n := range names { + id, ok := IsBlockDir(n) + if !ok { + continue + } + + if _, ok := resp.metas[id]; ok { + continue + } + + cachedBlockDir := filepath.Join(f.cacheDir, id.String()) + + // No such block loaded, remove the local dir. + if err := os.RemoveAll(cachedBlockDir); err != nil { + level.Warn(f.logger).Log("msg", "best effort remove of not needed cached dir failed; ignoring", "dir", cachedBlockDir, "err", err) + } + } + } + } + return resp, nil +} + +// Fetch returns all block metas as well as partial blocks (blocks without or with corrupted meta file) from the bucket. +// It's caller responsibility to not change the returned metadata files. Maps can be modified. +// +// Returned error indicates a failure in fetching metadata. Returned meta can be assumed as correct, with some blocks missing. +func (f *MetaFetcher) Fetch(ctx context.Context) (metas map[ulid.ULID]*Meta, partials map[ulid.ULID]error, err error) { + metas, partials, err = f.fetch(ctx, false) + return +} + +// FetchWithoutMarkedForDeletion returns all block metas as well as partial blocks (blocks without or with corrupted meta file) from the bucket. +// This function excludes all blocks for deletion (no deletion delay applied). +// It's caller responsibility to not change the returned metadata files. Maps can be modified. +// +// Returned error indicates a failure in fetching metadata. Returned meta can be assumed as correct, with some blocks missing. +func (f *MetaFetcher) FetchWithoutMarkedForDeletion(ctx context.Context) (metas map[ulid.ULID]*Meta, partials map[ulid.ULID]error, err error) { + metas, partials, err = f.fetch(ctx, true) + return +} + +func (f *MetaFetcher) fetch(ctx context.Context, excludeMarkedForDeletion bool) (_ map[ulid.ULID]*Meta, _ map[ulid.ULID]error, err error) { + start := time.Now() + defer func() { + f.metrics.SyncDuration.Observe(time.Since(start).Seconds()) + if err != nil { + f.metrics.SyncFailures.Inc() + } + }() + f.metrics.Syncs.Inc() + f.metrics.ResetTx() + + // Run this in thread safe run group. + v, err := f.g.Do("", func() (i interface{}, err error) { + // NOTE: First go routine context will go through. + return f.fetchMetadata(ctx, excludeMarkedForDeletion) + }) + if err != nil { + return nil, nil, err + } + resp := v.(response) + + // Copy as same response might be reused by different goroutines. + metas := make(map[ulid.ULID]*Meta, len(resp.metas)) + for id, m := range resp.metas { + metas[id] = m + } + + f.metrics.Synced.WithLabelValues(FailedMeta).Set(float64(len(resp.metaErrs))) + f.metrics.Synced.WithLabelValues(NoMeta).Set(resp.noMetasCount) + f.metrics.Synced.WithLabelValues(CorruptedMeta).Set(resp.corruptedMetasCount) + if excludeMarkedForDeletion { + f.metrics.Synced.WithLabelValues(MarkedForDeletionMeta).Set(resp.markedForDeletionCount) + } + + for _, filter := range f.filters { + // NOTE: filter can update synced metric accordingly to the reason of the exclude. + if err := filter.Filter(ctx, metas, f.metrics.Synced); err != nil { + return nil, nil, errors.Wrap(err, "filter metas") + } + } + + f.metrics.Synced.WithLabelValues(LoadedMeta).Set(float64(len(metas))) + f.metrics.Submit() + + if len(resp.metaErrs) > 0 { + return metas, resp.partial, errors.Wrap(resp.metaErrs.Err(), "incomplete view") + } + + level.Info(f.logger).Log("msg", "successfully synchronized block metadata", "duration", time.Since(start).String(), "duration_ms", time.Since(start).Milliseconds(), "cached", f.countCached(), "returned", len(metas), "partial", len(resp.partial)) + return metas, resp.partial, nil +} + +func (f *MetaFetcher) countCached() int { + f.mtx.Lock() + defer f.mtx.Unlock() + + return len(f.cached) +} + +// BlockIDLabel is a special label that will have an ULID of the meta.json being referenced to. +const BlockIDLabel = "__block_id" + +// IgnoreDeletionMarkFilter is a filter that filters out the blocks that are marked for deletion after a given delay. +// The delay duration is to make sure that the replacement block can be fetched before we filter out the old block. +// Delay is not considered when computing DeletionMarkBlocks map. +// Not go-routine safe. +type IgnoreDeletionMarkFilter struct { + logger log.Logger + delay time.Duration + concurrency int + bkt objstore.BucketReader + + mtx sync.Mutex + deletionMarkMap map[ulid.ULID]*DeletionMark +} + +// NewIgnoreDeletionMarkFilter creates IgnoreDeletionMarkFilter. +func NewIgnoreDeletionMarkFilter(logger log.Logger, bkt objstore.BucketReader, delay time.Duration, concurrency int) *IgnoreDeletionMarkFilter { + return &IgnoreDeletionMarkFilter{ + logger: logger, + bkt: bkt, + delay: delay, + concurrency: concurrency, + } +} + +// DeletionMarkBlocks returns block ids that were marked for deletion. +func (f *IgnoreDeletionMarkFilter) DeletionMarkBlocks() map[ulid.ULID]*DeletionMark { + f.mtx.Lock() + defer f.mtx.Unlock() + + deletionMarkMap := make(map[ulid.ULID]*DeletionMark, len(f.deletionMarkMap)) + for id, meta := range f.deletionMarkMap { + deletionMarkMap[id] = meta + } + + return deletionMarkMap +} + +// Filter filters out blocks that are marked for deletion after a given delay. +// It also returns the blocks that can be deleted since they were uploaded delay duration before current time. +func (f *IgnoreDeletionMarkFilter) Filter(ctx context.Context, metas map[ulid.ULID]*Meta, synced GaugeVec) error { + deletionMarkMap := make(map[ulid.ULID]*DeletionMark) + + // Make a copy of block IDs to check, in order to avoid concurrency issues + // between the scheduler and workers. + blockIDs := make([]ulid.ULID, 0, len(metas)) + for id := range metas { + blockIDs = append(blockIDs, id) + } + + var ( + eg errgroup.Group + ch = make(chan ulid.ULID, f.concurrency) + mtx sync.Mutex + ) + + for i := 0; i < f.concurrency; i++ { + eg.Go(func() error { + var lastErr error + for id := range ch { + m := &DeletionMark{} + if err := ReadMarker(ctx, f.logger, f.bkt, id.String(), m); err != nil { + if errors.Is(errors.Cause(err), ErrorMarkerNotFound) { + continue + } + if errors.Is(errors.Cause(err), ErrorUnmarshalMarker) { + level.Warn(f.logger).Log("msg", "found partial deletion-mark.json; if we will see it happening often for the same block, consider manually deleting deletion-mark.json from the object storage", "block", id, "err", err) + continue + } + // Remember the last error and continue to drain the channel. + lastErr = err + continue + } + + // Keep track of the blocks marked for deletion and filter them out if their + // deletion time is greater than the configured delay. + mtx.Lock() + deletionMarkMap[id] = m + if time.Since(time.Unix(m.DeletionTime, 0)).Seconds() > f.delay.Seconds() { + synced.WithLabelValues(MarkedForDeletionMeta).Inc() + delete(metas, id) + } + mtx.Unlock() + } + + return lastErr + }) + } + + // Workers scheduled, distribute blocks. + eg.Go(func() error { + defer close(ch) + + for _, id := range blockIDs { + select { + case ch <- id: + // Nothing to do. + case <-ctx.Done(): + return ctx.Err() + } + } + + return nil + }) + + if err := eg.Wait(); err != nil { + return errors.Wrap(err, "filter blocks marked for deletion") + } + + f.mtx.Lock() + f.deletionMarkMap = deletionMarkMap + f.mtx.Unlock() + + return nil +} diff --git a/pkg/phlaredb/block/global_markers.go b/pkg/phlaredb/block/global_markers.go new file mode 100644 index 0000000000..3720734859 --- /dev/null +++ b/pkg/phlaredb/block/global_markers.go @@ -0,0 +1,88 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/cortexproject/cortex/blob/master/pkg/storage/tsdb/bucketindex/markers.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Cortex Authors. + +package block + +import ( + "context" + "fmt" + "path" + "path/filepath" + "strings" + + "github.com/oklog/ulid" + "github.com/pkg/errors" + "github.com/thanos-io/objstore" +) + +const ( + MarkersPathname = "markers" +) + +func markFilepath(blockID ulid.ULID, markFilename string) string { + return fmt.Sprintf("%s/%s-%s", MarkersPathname, blockID.String(), markFilename) +} + +func isMarkFilename(name string, markFilename string) (ulid.ULID, bool) { + parts := strings.SplitN(name, "-", 2) + if len(parts) != 2 { + return ulid.ULID{}, false + } + + // Ensure the 2nd part matches the mark filename. + if parts[1] != markFilename { + return ulid.ULID{}, false + } + + // Ensure the 1st part is a valid block ID. + id, err := ulid.Parse(filepath.Base(parts[0])) + return id, err == nil +} + +// DeletionMarkFilepath returns the path, relative to the tenant's bucket location, +// of a block deletion mark in the bucket markers location. +func DeletionMarkFilepath(blockID ulid.ULID) string { + return markFilepath(blockID, DeletionMarkFilename) +} + +// IsDeletionMarkFilename returns whether the input filename matches the expected pattern +// of block deletion markers stored in the markers location. +func IsDeletionMarkFilename(name string) (ulid.ULID, bool) { + return isMarkFilename(name, DeletionMarkFilename) +} + +// NoCompactMarkFilepath returns the path, relative to the tenant's bucket location, +// of a no-compact block mark in the bucket markers location. +func NoCompactMarkFilepath(blockID ulid.ULID) string { + return markFilepath(blockID, NoCompactMarkFilename) +} + +// IsNoCompactMarkFilename returns true if input filename matches the expected +// pattern of block marker stored in the markers location. +func IsNoCompactMarkFilename(name string) (ulid.ULID, bool) { + return isMarkFilename(name, NoCompactMarkFilename) +} + +// ListBlockDeletionMarks looks for block deletion marks in the global markers location +// and returns a map containing all blocks having a deletion mark and their location in the +// bucket. +func ListBlockDeletionMarks(ctx context.Context, bkt objstore.BucketReader) (map[ulid.ULID]struct{}, error) { + discovered := map[ulid.ULID]struct{}{} + + // Find all markers in the storage. + err := bkt.Iter(ctx, MarkersPathname+"/", func(name string) error { + if err := ctx.Err(); err != nil { + return err + } + + if blockID, ok := IsDeletionMarkFilename(path.Base(name)); ok { + discovered[blockID] = struct{}{} + } + + return nil + }) + + return discovered, errors.Wrap(err, "list block deletion marks") +} diff --git a/pkg/phlaredb/block/global_markers_test.go b/pkg/phlaredb/block/global_markers_test.go new file mode 100644 index 0000000000..f47129a9fe --- /dev/null +++ b/pkg/phlaredb/block/global_markers_test.go @@ -0,0 +1,96 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/cortexproject/cortex/blob/master/pkg/storage/tsdb/bucketindex/markers_test.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Cortex Authors. + +package block + +import ( + "context" + "strings" + "testing" + + "github.com/oklog/ulid" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + + mimir_testutil "github.com/grafana/mimir/pkg/storage/tsdb/testutil" +) + +func TestDeletionMarkFilepath(t *testing.T) { + id := ulid.MustNew(1, nil) + + assert.Equal(t, "markers/"+id.String()+"-deletion-mark.json", DeletionMarkFilepath(id)) +} + +func TestIsDeletionMarkFilename(t *testing.T) { + expected := ulid.MustNew(1, nil) + + _, ok := IsDeletionMarkFilename("xxx") + assert.False(t, ok) + + _, ok = IsDeletionMarkFilename("xxx-deletion-mark.json") + assert.False(t, ok) + + _, ok = IsDeletionMarkFilename("tenant-deletion-mark.json") + assert.False(t, ok) + + actual, ok := IsDeletionMarkFilename(expected.String() + "-deletion-mark.json") + assert.True(t, ok) + assert.Equal(t, expected, actual) +} + +func TestNoCompactMarkFilepath(t *testing.T) { + id := ulid.MustNew(1, nil) + + assert.Equal(t, "markers/"+id.String()+"-no-compact-mark.json", NoCompactMarkFilepath(id)) +} + +func TestIsNoCompactMarkFilename(t *testing.T) { + expected := ulid.MustNew(1, nil) + + _, ok := IsNoCompactMarkFilename("xxx") + assert.False(t, ok) + + _, ok = IsNoCompactMarkFilename("xxx-no-compact-mark.json") + assert.False(t, ok) + + _, ok = IsNoCompactMarkFilename("tenant-no-compact-mark.json") + assert.False(t, ok) + + actual, ok := IsNoCompactMarkFilename(expected.String() + "-no-compact-mark.json") + assert.True(t, ok) + assert.Equal(t, expected, actual) +} + +func TestListBlockDeletionMarks(t *testing.T) { + var ( + ctx = context.Background() + block1 = ulid.MustNew(1, nil) + block2 = ulid.MustNew(2, nil) + block3 = ulid.MustNew(3, nil) + ) + + t.Run("should return an empty map on empty bucket", func(t *testing.T) { + bkt, _ := mimir_testutil.PrepareFilesystemBucket(t) + + actualMarks, actualErr := ListBlockDeletionMarks(ctx, bkt) + require.NoError(t, actualErr) + assert.Empty(t, actualMarks) + }) + + t.Run("should return a map with the locations of the block deletion marks found", func(t *testing.T) { + bkt, _ := mimir_testutil.PrepareFilesystemBucket(t) + + require.NoError(t, bkt.Upload(ctx, DeletionMarkFilepath(block1), strings.NewReader("{}"))) + require.NoError(t, bkt.Upload(ctx, NoCompactMarkFilepath(block2), strings.NewReader("{}"))) + require.NoError(t, bkt.Upload(ctx, DeletionMarkFilepath(block3), strings.NewReader("{}"))) + + actualMarks, actualErr := ListBlockDeletionMarks(ctx, bkt) + require.NoError(t, actualErr) + assert.Equal(t, map[ulid.ULID]struct{}{ + block1: {}, + block3: {}, + }, actualMarks) + }) +} diff --git a/pkg/phlaredb/block/markers.go b/pkg/phlaredb/block/markers.go new file mode 100644 index 0000000000..6c5000ed1e --- /dev/null +++ b/pkg/phlaredb/block/markers.go @@ -0,0 +1,131 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/thanos-io/thanos/blob/master/pkg/block/metadata/markers.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Thanos Authors. + +package block + +import ( + "context" + "encoding/json" + "io" + "path" + "time" + + "github.com/go-kit/log" + "github.com/grafana/dskit/runutil" + "github.com/oklog/ulid" + "github.com/pkg/errors" + + "github.com/grafana/pyroscope/pkg/objstore" +) + +const ( + // DeletionMarkFilename is the known json filename for optional file storing details about when block is marked for deletion. + // If such file is present in block dir, it means the block is meant to be deleted after certain delay. + DeletionMarkFilename = "deletion-mark.json" + // NoCompactMarkFilename is the known json filename for optional file storing details about why block has to be excluded from compaction. + // If such file is present in block dir, it means the block has to excluded from compaction (both vertical and horizontal) or rewrite (e.g deletions). + NoCompactMarkFilename = "no-compact-mark.json" + + // DeletionMarkVersion1 is the version of deletion-mark file supported by Thanos. + DeletionMarkVersion1 = 1 + // NoCompactMarkVersion1 is the version of no-compact-mark file supported by Thanos. + NoCompactMarkVersion1 = 1 +) + +var ( + // ErrorMarkerNotFound is the error when marker file is not found. + ErrorMarkerNotFound = errors.New("marker not found") + // ErrorUnmarshalMarker is the error when unmarshalling marker JSON file. + // This error can occur because marker has been partially uploaded to block storage + // or the marker file is not a valid json file. + ErrorUnmarshalMarker = errors.New("unmarshal marker JSON") +) + +type Marker interface { + markerFilename() string +} + +// DeletionMark stores block id and when block was marked for deletion. +type DeletionMark struct { + // ID of the tsdb block. + ID ulid.ULID `json:"id"` + // Version of the file. + Version int `json:"version"` + // Details is a human readable string giving details of reason. + Details string `json:"details,omitempty"` + + // DeletionTime is a unix timestamp of when the block was marked to be deleted. + DeletionTime int64 `json:"deletion_time"` +} + +func (m *DeletionMark) markerFilename() string { return DeletionMarkFilename } + +// NoCompactReason is a reason for a block to be excluded from compaction. +type NoCompactReason string + +const ( + // ManualNoCompactReason is a custom reason of excluding from compaction that should be added when no-compact mark is added for unknown/user specified reason. + ManualNoCompactReason NoCompactReason = "manual" + // IndexSizeExceedingNoCompactReason is a reason of index being too big (for example exceeding 64GB limit: https://github.com/thanos-io/thanos/issues/1424) + // This reason can be ignored when vertical block sharding will be implemented. + IndexSizeExceedingNoCompactReason = "index-size-exceeding" + // OutOfOrderChunksNoCompactReason is a reason of to no compact block with index contains out of order chunk so that the compaction is not blocked. + OutOfOrderChunksNoCompactReason = "block-index-out-of-order-chunk" +) + +// NoCompactMark marker stores reason of block being excluded from compaction if needed. +type NoCompactMark struct { + // ID of the tsdb block. + ID ulid.ULID `json:"id"` + // Version of the file. + Version int `json:"version"` + // Details is a human readable string giving details of reason. + Details string `json:"details,omitempty"` + + // NoCompactTime is a unix timestamp of when the block was marked for no compact. + NoCompactTime int64 `json:"no_compact_time"` + Reason NoCompactReason `json:"reason"` +} + +func (n *NoCompactMark) markerFilename() string { return NoCompactMarkFilename } + +// ReadMarker reads the given mark file from /.json in bucket. +// ReadMarker has a one-minute timeout for completing the read against the bucket. +// This protects against operations that can take unbounded time. +func ReadMarker(ctx context.Context, logger log.Logger, bkt objstore.BucketReader, dir string, marker Marker) error { + ctx, cancel := context.WithTimeout(ctx, time.Minute) + defer cancel() + + markerFile := path.Join(dir, marker.markerFilename()) + // todo(cyriltovena): we should use ReaderWithExpectedErrs(bkt.IsObjNotFoundErr) since it's expected to not find the marker file. + r, err := bkt.Get(ctx, markerFile) + if err != nil { + if bkt.IsObjNotFoundErr(err) { + return ErrorMarkerNotFound + } + return errors.Wrapf(err, "get file: %s", markerFile) + } + defer runutil.CloseWithLogOnErr(logger, r, "close bkt marker reader") + + metaContent, err := io.ReadAll(r) + if err != nil { + return errors.Wrapf(err, "read file: %s", markerFile) + } + + if err := json.Unmarshal(metaContent, marker); err != nil { + return errors.Wrapf(ErrorUnmarshalMarker, "file: %s; err: %v", markerFile, err.Error()) + } + switch marker.markerFilename() { + case NoCompactMarkFilename: + if version := marker.(*NoCompactMark).Version; version != NoCompactMarkVersion1 { + return errors.Errorf("unexpected no-compact-mark file version %d, expected %d", version, NoCompactMarkVersion1) + } + case DeletionMarkFilename: + if version := marker.(*DeletionMark).Version; version != DeletionMarkVersion1 { + return errors.Errorf("unexpected deletion-mark file version %d, expected %d", version, DeletionMarkVersion1) + } + } + return nil +} diff --git a/pkg/phlaredb/block/metadata.go b/pkg/phlaredb/block/metadata.go index 3451a06d5e..f55f63c02c 100644 --- a/pkg/phlaredb/block/metadata.go +++ b/pkg/phlaredb/block/metadata.go @@ -62,6 +62,16 @@ const ( MetaVersion3 = MetaVersion(3) ) +// IsValid returns true if the version is valid. +func (v MetaVersion) IsValid() bool { + switch v { + case MetaVersion1, MetaVersion2, MetaVersion3: + return true + default: + return false + } +} + type BlockStats struct { NumSamples uint64 `json:"numSamples,omitempty"` NumSeries uint64 `json:"numSeries,omitempty"` @@ -115,6 +125,13 @@ type Meta struct { // Source is a real upload source of the block. Source SourceType `json:"source,omitempty"` + + // Downsample is a downsampling resolution of the block. 0 means no downsampling. + Downsample `json:"downsample"` +} + +type Downsample struct { + Resolution int64 `json:"resolution"` } func (m *Meta) FileByRelPath(name string) *File { @@ -212,6 +229,7 @@ func (meta *Meta) WriteTo(w io.Writer) (int64, error) { return int64(wrapped.n), enc.Encode(meta) } +// WriteToFile writes the encoded meta into /meta.json. func (meta *Meta) WriteToFile(logger log.Logger, dir string) (int64, error) { // Make any changes to the file appear atomic. path := filepath.Join(dir, MetaFilename) @@ -255,8 +273,8 @@ func (meta *Meta) TSDBBlockMeta() tsdb.BlockMeta { } } -// ReadFromDir reads the given meta from /meta.json. -func ReadFromDir(dir string) (*Meta, error) { +// ReadMetaFromDir reads the given meta from /meta.json. +func ReadMetaFromDir(dir string) (*Meta, error) { f, err := os.Open(filepath.Join(dir, filepath.Clean(MetaFilename))) if err != nil { return nil, err diff --git a/pkg/util/extprom/tx_gauge.go b/pkg/util/extprom/tx_gauge.go new file mode 100644 index 0000000000..cd50948fea --- /dev/null +++ b/pkg/util/extprom/tx_gauge.go @@ -0,0 +1,102 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/thanos-io/thanos/blob/main/pkg/extprom/tx_gauge.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Thanos Authors. + +package extprom + +import ( + "sync" + + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" +) + +type TxGaugeVec struct { + current *prometheus.GaugeVec + mtx sync.Mutex + newMetricVal func() *prometheus.GaugeVec + + tx *prometheus.GaugeVec +} + +// NewTxGaugeVec is a prometheus.GaugeVec that allows to start atomic metric value transaction. +// It might be useful if long process that wants to update a GaugeVec but wants to build/accumulate those metrics +// in a concurrent way without exposing partial state to Prometheus. +// Caller can also use this as normal GaugeVec. +// +// Additionally it allows to init LabelValues on each transaction. +// NOTE: This is quite naive implementation creating new prometheus.GaugeVec on each `ResetTx`, use wisely. +func NewTxGaugeVec(reg prometheus.Registerer, opts prometheus.GaugeOpts, labelNames []string, initLabelValues ...[]string) *TxGaugeVec { + // Nil as we will register it on our own later. + f := func() *prometheus.GaugeVec { + g := promauto.With(nil).NewGaugeVec(opts, labelNames) + for _, vals := range initLabelValues { + g.WithLabelValues(vals...) + } + return g + } + tx := &TxGaugeVec{ + current: f(), + newMetricVal: f, + } + if reg != nil { + reg.MustRegister(tx) + } + return tx +} + +// ResetTx starts new transaction. Not goroutine-safe. +func (tx *TxGaugeVec) ResetTx() { + tx.tx = tx.newMetricVal() +} + +// Submit atomically and fully applies new values from existing transaction GaugeVec. Not goroutine-safe. +func (tx *TxGaugeVec) Submit() { + if tx.tx == nil { + return + } + + tx.mtx.Lock() + tx.current = tx.tx + tx.mtx.Unlock() +} + +// Describe is used in Register. +func (tx *TxGaugeVec) Describe(ch chan<- *prometheus.Desc) { + tx.mtx.Lock() + defer tx.mtx.Unlock() + + tx.current.Describe(ch) +} + +// Collect is used by Registered. +func (tx *TxGaugeVec) Collect(ch chan<- prometheus.Metric) { + tx.mtx.Lock() + defer tx.mtx.Unlock() + + tx.current.Collect(ch) +} + +// With works as GetMetricWith, but panics where GetMetricWithLabels would have +// returned an error. Not returning an error allows shortcuts like +// +// myVec.With(prometheus.Labels{"code": "404", "method": "GET"}).Add(42) +func (tx *TxGaugeVec) With(labels prometheus.Labels) prometheus.Gauge { + if tx.tx == nil { + tx.ResetTx() + } + return tx.tx.With(labels) +} + +// WithLabelValues works as GetMetricWithLabelValues, but panics where +// GetMetricWithLabelValues would have returned an error. Not returning an +// error allows shortcuts like +// +// myVec.WithLabelValues("404", "GET").Add(42) +func (tx *TxGaugeVec) WithLabelValues(lvs ...string) prometheus.Gauge { + if tx.tx == nil { + tx.ResetTx() + } + return tx.tx.WithLabelValues(lvs...) +} From 14c47f8b6d98c8e797697a3536f32271bcf45240 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Fri, 8 Sep 2023 15:15:23 +0200 Subject: [PATCH 13/74] Fixes all tests --- pkg/objstore/objstore.go | 32 ++ .../providers/filesystem/bucket_client.go | 24 ++ pkg/objstore/reader.go | 22 ++ pkg/objstore/sse_bucket_client.go | 14 +- pkg/objstore/sse_bucket_client_test.go | 4 +- pkg/objstore/testutil/objstore.go | 31 ++ pkg/objstore/user_bucket_client.go | 16 + pkg/phlaredb/block/block.go | 33 ++ pkg/phlaredb/block/fetcher_test.go | 345 ++++++++++++++++++ pkg/phlaredb/block/global_markers.go | 3 +- .../block/global_markers_bucket_client.go | 199 ++++++++++ .../global_markers_bucket_client_test.go | 338 +++++++++++++++++ pkg/phlaredb/block/global_markers_test.go | 6 +- pkg/phlaredb/block/testutil/create_block.go | 58 +++ pkg/phlaredb/compact_test.go | 2 +- pkg/phlaredb/head.go | 4 +- pkg/phlaredb/phlaredb.go | 4 +- pkg/phlaredb/sample_merge_test.go | 6 +- 18 files changed, 1122 insertions(+), 19 deletions(-) create mode 100644 pkg/objstore/testutil/objstore.go create mode 100644 pkg/objstore/user_bucket_client.go create mode 100644 pkg/phlaredb/block/fetcher_test.go create mode 100644 pkg/phlaredb/block/global_markers_bucket_client.go create mode 100644 pkg/phlaredb/block/global_markers_bucket_client_test.go create mode 100644 pkg/phlaredb/block/testutil/create_block.go diff --git a/pkg/objstore/objstore.go b/pkg/objstore/objstore.go index ff2d1b8edd..ec22f935c8 100644 --- a/pkg/objstore/objstore.go +++ b/pkg/objstore/objstore.go @@ -21,6 +21,21 @@ type BucketReader interface { ReaderAt(ctx context.Context, filename string) (ReaderAtCloser, error) } +// IsOpFailureExpectedFunc allows to mark certain errors as expected, so they will not increment objstore_bucket_operation_failures_total metric. +type IsOpFailureExpectedFunc func(error) bool + +type InstrumentedBucket interface { + Bucket + // WithExpectedErrs allows to specify a filter that marks certain errors as expected, so it will not increment + // objstore_bucket_operation_failures_total metric. + WithExpectedErrs(IsOpFailureExpectedFunc) Bucket + + // ReaderWithExpectedErrs allows to specify a filter that marks certain errors as expected, so it will not increment + // objstore_bucket_operation_failures_total metric. + // TODO(bwplotka): Remove this when moved to Go 1.14 and replace with InstrumentedBucketReader. + ReaderWithExpectedErrs(IsOpFailureExpectedFunc) BucketReader +} + type PrefixedBucket struct { Bucket prefix string @@ -118,3 +133,20 @@ func (p *PrefixedBucket) Delete(ctx context.Context, name string) error { func (p *PrefixedBucket) Name() string { return p.Bucket.Name() } + +// ReaderWithExpectedErrs implements objstore.Bucket. +func (p *PrefixedBucket) ReaderWithExpectedErrs(fn IsOpFailureExpectedFunc) BucketReader { + return p.WithExpectedErrs(fn) +} + +// WithExpectedErrs implements objstore.Bucket. +func (p *PrefixedBucket) WithExpectedErrs(fn IsOpFailureExpectedFunc) Bucket { + if ib, ok := p.Bucket.(InstrumentedBucket); ok { + return &PrefixedBucket{ + Bucket: ib.WithExpectedErrs(fn), + prefix: p.prefix, + } + } + + return p +} diff --git a/pkg/objstore/providers/filesystem/bucket_client.go b/pkg/objstore/providers/filesystem/bucket_client.go index d1e544a5a3..4d83d0168d 100644 --- a/pkg/objstore/providers/filesystem/bucket_client.go +++ b/pkg/objstore/providers/filesystem/bucket_client.go @@ -8,6 +8,7 @@ import ( thanosobjstore "github.com/thanos-io/objstore" "github.com/thanos-io/objstore/providers/filesystem" + "github.com/grafana/pyroscope/pkg/objstore" phlareobjstore "github.com/grafana/pyroscope/pkg/objstore" ) @@ -44,6 +45,29 @@ func (b *Bucket) ReaderAt(ctx context.Context, filename string) (phlareobjstore. return &FileReaderAt{File: f}, nil } +// ReaderWithExpectedErrs implements objstore.Bucket. +func (b *Bucket) ReaderWithExpectedErrs(fn objstore.IsOpFailureExpectedFunc) objstore.BucketReader { + return b.WithExpectedErrs(fn) +} + +// WithExpectedErrs implements objstore.Bucket. +func (b *Bucket) WithExpectedErrs(fn objstore.IsOpFailureExpectedFunc) objstore.Bucket { + if ib, ok := b.Bucket.(objstore.InstrumentedBucket); ok { + return &Bucket{ + rootDir: b.rootDir, + Bucket: ib.WithExpectedErrs(fn), + } + } + if ib, ok := b.Bucket.(thanosobjstore.InstrumentedBucket); ok { + return &Bucket{ + rootDir: b.rootDir, + Bucket: ib.WithExpectedErrs(func(err error) bool { return fn(err) }), + } + } + + return b +} + type FileReaderAt struct { *os.File } diff --git a/pkg/objstore/reader.go b/pkg/objstore/reader.go index bb1f1f9681..30cb96a6e1 100644 --- a/pkg/objstore/reader.go +++ b/pkg/objstore/reader.go @@ -33,6 +33,28 @@ func (b *ReaderAtBucket) ReaderAt(ctx context.Context, name string) (ReaderAtClo }, nil } +// ReaderWithExpectedErrs implements objstore.Bucket. +func (b *ReaderAtBucket) ReaderWithExpectedErrs(fn IsOpFailureExpectedFunc) BucketReader { + return b.WithExpectedErrs(fn) +} + +// WithExpectedErrs implements objstore.Bucket. +func (b *ReaderAtBucket) WithExpectedErrs(fn IsOpFailureExpectedFunc) Bucket { + if ib, ok := b.Bucket.(InstrumentedBucket); ok { + return &ReaderAtBucket{ + Bucket: ib.WithExpectedErrs(fn), + } + } + + if ib, ok := b.Bucket.(objstore.InstrumentedBucket); ok { + return &ReaderAtBucket{ + Bucket: ib.WithExpectedErrs(func(err error) bool { return fn(err) }), + } + } + + return b +} + type GetRangeReader interface { GetRange(ctx context.Context, name string, off, length int64) (io.ReadCloser, error) } diff --git a/pkg/objstore/sse_bucket_client.go b/pkg/objstore/sse_bucket_client.go index 53e2b0b900..a89c21d863 100644 --- a/pkg/objstore/sse_bucket_client.go +++ b/pkg/objstore/sse_bucket_client.go @@ -33,12 +33,12 @@ type TenantConfigProvider interface { // storage server-side encryption (SSE) for a given user. type SSEBucketClient struct { userID string - bucket objstore.Bucket + bucket Bucket cfgProvider TenantConfigProvider } // NewSSEBucketClient makes a new SSEBucketClient. The cfgProvider can be nil. -func NewSSEBucketClient(userID string, bucket objstore.Bucket, cfgProvider TenantConfigProvider) *SSEBucketClient { +func NewSSEBucketClient(userID string, bucket Bucket, cfgProvider TenantConfigProvider) InstrumentedBucket { return &SSEBucketClient{ userID: userID, bucket: bucket, @@ -51,6 +51,10 @@ func (b *SSEBucketClient) Close() error { return b.bucket.Close() } +func (b *SSEBucketClient) ReaderAt(ctx context.Context, name string) (ReaderAtCloser, error) { + return b.bucket.ReaderAt(ctx, name) +} + // Upload the contents of the reader as an object into the bucket. func (b *SSEBucketClient) Upload(ctx context.Context, name string, r io.Reader) error { if sse, err := b.getCustomS3SSEConfig(); err != nil { @@ -134,13 +138,13 @@ func (b *SSEBucketClient) Attributes(ctx context.Context, name string) (objstore } // ReaderWithExpectedErrs implements objstore.Bucket. -func (b *SSEBucketClient) ReaderWithExpectedErrs(fn objstore.IsOpFailureExpectedFunc) objstore.BucketReader { +func (b *SSEBucketClient) ReaderWithExpectedErrs(fn IsOpFailureExpectedFunc) BucketReader { return b.WithExpectedErrs(fn) } // WithExpectedErrs implements objstore.Bucket. -func (b *SSEBucketClient) WithExpectedErrs(fn objstore.IsOpFailureExpectedFunc) objstore.Bucket { - if ib, ok := b.bucket.(objstore.InstrumentedBucket); ok { +func (b *SSEBucketClient) WithExpectedErrs(fn IsOpFailureExpectedFunc) Bucket { + if ib, ok := b.bucket.(InstrumentedBucket); ok { return &SSEBucketClient{ userID: b.userID, bucket: ib.WithExpectedErrs(fn), diff --git a/pkg/objstore/sse_bucket_client_test.go b/pkg/objstore/sse_bucket_client_test.go index 2db029ef79..ee2f721cc3 100644 --- a/pkg/objstore/sse_bucket_client_test.go +++ b/pkg/objstore/sse_bucket_client_test.go @@ -69,9 +69,9 @@ func TestSSEBucketClient_Upload_ShouldInjectCustomSSEConfig(t *testing.T) { var sseBkt objstore.Bucket if testData.withExpectedErrs { - sseBkt = NewSSEBucketClient("user-1", s3Client, cfgProvider).WithExpectedErrs(s3Client.IsObjNotFoundErr) + sseBkt = NewSSEBucketClient("user-1", NewBucket(s3Client), cfgProvider).WithExpectedErrs(s3Client.IsObjNotFoundErr) } else { - sseBkt = NewSSEBucketClient("user-1", s3Client, cfgProvider) + sseBkt = NewSSEBucketClient("user-1", NewBucket(s3Client), cfgProvider) } err = sseBkt.Upload(context.Background(), "test", strings.NewReader("test")) diff --git a/pkg/objstore/testutil/objstore.go b/pkg/objstore/testutil/objstore.go new file mode 100644 index 0000000000..03f284e52b --- /dev/null +++ b/pkg/objstore/testutil/objstore.go @@ -0,0 +1,31 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/cortexproject/cortex/blob/master/pkg/storage/tsdb/testutil/objstore.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Cortex Authors. + +package testutil + +import ( + "context" + "testing" + + "github.com/stretchr/testify/require" + + "github.com/grafana/pyroscope/pkg/objstore" + "github.com/grafana/pyroscope/pkg/objstore/client" + "github.com/grafana/pyroscope/pkg/objstore/providers/filesystem" +) + +func NewFilesystemBucket(t testing.TB, ctx context.Context, storageDir string) (objstore.Bucket, string) { + bkt, err := client.NewBucket(ctx, client.Config{ + StorageBackendConfig: client.StorageBackendConfig{ + Backend: client.Filesystem, + Filesystem: filesystem.Config{ + Directory: storageDir, + }, + }, + }, "test") + require.NoError(t, err) + + return bkt, storageDir +} diff --git a/pkg/objstore/user_bucket_client.go b/pkg/objstore/user_bucket_client.go new file mode 100644 index 0000000000..18605244d0 --- /dev/null +++ b/pkg/objstore/user_bucket_client.go @@ -0,0 +1,16 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/cortexproject/cortex/blob/master/pkg/storage/bucket/user_bucket_client.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Cortex Authors. + +package objstore + +// NewUserBucketClient returns a bucket client to use to access the storage on behalf of the provided user. +// The cfgProvider can be nil. +func NewUserBucketClient(userID string, bucket Bucket, cfgProvider TenantConfigProvider) InstrumentedBucket { + // Inject the user/tenant prefix. + bucket = NewPrefixedBucket(bucket, userID) + + // Inject the SSE config. + return NewSSEBucketClient(userID, bucket, cfgProvider) +} diff --git a/pkg/phlaredb/block/block.go b/pkg/phlaredb/block/block.go index 40e9222604..c3ccf94984 100644 --- a/pkg/phlaredb/block/block.go +++ b/pkg/phlaredb/block/block.go @@ -1,6 +1,7 @@ package block import ( + "bytes" "context" "encoding/json" "io" @@ -8,12 +9,14 @@ import ( "path" "path/filepath" "strings" + "time" "github.com/go-kit/log" "github.com/go-kit/log/level" "github.com/grafana/dskit/runutil" "github.com/oklog/ulid" "github.com/pkg/errors" + "github.com/prometheus/client_golang/prometheus" "github.com/thanos-io/objstore" "github.com/grafana/pyroscope/pkg/util/fnv32" @@ -132,6 +135,36 @@ func cleanUp(logger log.Logger, bkt objstore.Bucket, id ulid.ULID, err error) er return err } +// MarkForDeletion creates a file which stores information about when the block was marked for deletion. +func MarkForDeletion(ctx context.Context, logger log.Logger, bkt objstore.Bucket, id ulid.ULID, details string, markedForDeletion prometheus.Counter) error { + deletionMarkFile := path.Join(id.String(), DeletionMarkFilename) + deletionMarkExists, err := bkt.Exists(ctx, deletionMarkFile) + if err != nil { + return errors.Wrapf(err, "check exists %s in bucket", deletionMarkFile) + } + if deletionMarkExists { + level.Warn(logger).Log("msg", "requested to mark for deletion, but file already exists; this should not happen; investigate", "err", errors.Errorf("file %s already exists in bucket", deletionMarkFile)) + return nil + } + + deletionMark, err := json.Marshal(DeletionMark{ + ID: id, + DeletionTime: time.Now().Unix(), + Version: DeletionMarkVersion1, + Details: details, + }) + if err != nil { + return errors.Wrap(err, "json encode deletion mark") + } + + if err := bkt.Upload(ctx, deletionMarkFile, bytes.NewBuffer(deletionMark)); err != nil { + return errors.Wrapf(err, "upload file %s to bucket", deletionMarkFile) + } + markedForDeletion.Inc() + level.Info(logger).Log("msg", "block has been marked for deletion", "block", id) + return nil +} + // Delete removes directory that is meant to be block directory. // NOTE: Always prefer this method for deleting blocks. // - We have to delete block's files in the certain order (meta.json first and deletion-mark.json last) diff --git a/pkg/phlaredb/block/fetcher_test.go b/pkg/phlaredb/block/fetcher_test.go new file mode 100644 index 0000000000..c9794bdcc3 --- /dev/null +++ b/pkg/phlaredb/block/fetcher_test.go @@ -0,0 +1,345 @@ +// SPDX-License-Identifier: AGPL-3.0-only + +package block_test + +import ( + "context" + "path" + "path/filepath" + "strings" + "testing" + + "github.com/go-kit/log" + "github.com/oklog/ulid" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" + "github.com/prometheus/client_golang/prometheus/testutil" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + + objstore_testutil "github.com/grafana/pyroscope/pkg/objstore/testutil" + phlarecontext "github.com/grafana/pyroscope/pkg/phlare/context" + "github.com/grafana/pyroscope/pkg/phlaredb/block" + block_testutil "github.com/grafana/pyroscope/pkg/phlaredb/block/testutil" + "github.com/grafana/pyroscope/pkg/pprof/testhelper" +) + +func TestMetaFetcher_Fetch_ShouldReturnDiscoveredBlocksIncludingMarkedForDeletion(t *testing.T) { + var ( + ctx = context.Background() + reg = prometheus.NewPedanticRegistry() + logger = log.NewNopLogger() + ) + + // Create a bucket client with global markers. + bkt, _ := objstore_testutil.NewFilesystemBucket(t, ctx, t.TempDir()) + bkt = block.BucketWithGlobalMarkers(bkt) + + f, err := block.NewMetaFetcher(logger, 10, bkt, t.TempDir(), reg, nil) + require.NoError(t, err) + + t.Run("should return no metas and no partials on no block in the storage", func(t *testing.T) { + actualMetas, actualPartials, actualErr := f.Fetch(ctx) + require.NoError(t, actualErr) + require.Empty(t, actualMetas) + require.Empty(t, actualPartials) + + assert.NoError(t, testutil.GatherAndCompare(reg, strings.NewReader(` + # HELP blocks_meta_sync_failures_total Total blocks metadata synchronization failures + # TYPE blocks_meta_sync_failures_total counter + blocks_meta_sync_failures_total 0 + + # HELP blocks_meta_synced Number of block metadata synced + # TYPE blocks_meta_synced gauge + blocks_meta_synced{state="corrupted-meta-json"} 0 + blocks_meta_synced{state="duplicate"} 0 + blocks_meta_synced{state="failed"} 0 + blocks_meta_synced{state="label-excluded"} 0 + blocks_meta_synced{state="loaded"} 0 + blocks_meta_synced{state="marked-for-deletion"} 0 + blocks_meta_synced{state="marked-for-no-compact"} 0 + blocks_meta_synced{state="no-meta-json"} 0 + blocks_meta_synced{state="time-excluded"} 0 + + # HELP blocks_meta_syncs_total Total blocks metadata synchronization attempts + # TYPE blocks_meta_syncs_total counter + blocks_meta_syncs_total 1 + `), "blocks_meta_syncs_total", "blocks_meta_sync_failures_total", "blocks_meta_synced")) + }) + + // Upload a block. + block1ID, block1Dir := createTestBlock(t) + require.NoError(t, block.Upload(ctx, logger, bkt, block1Dir)) + + // Upload a partial block. + block2ID, block2Dir := createTestBlock(t) + require.NoError(t, block.Upload(ctx, logger, bkt, block2Dir)) + require.NoError(t, bkt.Delete(ctx, path.Join(block2ID.String(), block.MetaFilename))) + + t.Run("should return metas and partials on some blocks in the storage", func(t *testing.T) { + actualMetas, actualPartials, actualErr := f.Fetch(ctx) + require.NoError(t, actualErr) + require.Len(t, actualMetas, 1) + require.Contains(t, actualMetas, block1ID) + require.Len(t, actualPartials, 1) + require.Contains(t, actualPartials, block2ID) + + assert.NoError(t, testutil.GatherAndCompare(reg, strings.NewReader(` + # HELP blocks_meta_sync_failures_total Total blocks metadata synchronization failures + # TYPE blocks_meta_sync_failures_total counter + blocks_meta_sync_failures_total 0 + + # HELP blocks_meta_synced Number of block metadata synced + # TYPE blocks_meta_synced gauge + blocks_meta_synced{state="corrupted-meta-json"} 0 + blocks_meta_synced{state="duplicate"} 0 + blocks_meta_synced{state="failed"} 0 + blocks_meta_synced{state="label-excluded"} 0 + blocks_meta_synced{state="loaded"} 1 + blocks_meta_synced{state="marked-for-deletion"} 0 + blocks_meta_synced{state="marked-for-no-compact"} 0 + blocks_meta_synced{state="no-meta-json"} 1 + blocks_meta_synced{state="time-excluded"} 0 + + # HELP blocks_meta_syncs_total Total blocks metadata synchronization attempts + # TYPE blocks_meta_syncs_total counter + blocks_meta_syncs_total 2 + `), "blocks_meta_syncs_total", "blocks_meta_sync_failures_total", "blocks_meta_synced")) + }) + + // Upload a block and mark it for deletion. + block3ID, block3Dir := createTestBlock(t) + require.NoError(t, block.Upload(ctx, logger, bkt, block3Dir)) + require.NoError(t, block.MarkForDeletion(ctx, logger, bkt, block3ID, "", promauto.With(nil).NewCounter(prometheus.CounterOpts{}))) + + t.Run("should include blocks marked for deletion", func(t *testing.T) { + actualMetas, actualPartials, actualErr := f.Fetch(ctx) + require.NoError(t, actualErr) + require.Len(t, actualMetas, 2) + require.Contains(t, actualMetas, block1ID) + require.Contains(t, actualMetas, block3ID) + require.Len(t, actualPartials, 1) + require.Contains(t, actualPartials, block2ID) + + assert.NoError(t, testutil.GatherAndCompare(reg, strings.NewReader(` + # HELP blocks_meta_sync_failures_total Total blocks metadata synchronization failures + # TYPE blocks_meta_sync_failures_total counter + blocks_meta_sync_failures_total 0 + + # HELP blocks_meta_synced Number of block metadata synced + # TYPE blocks_meta_synced gauge + blocks_meta_synced{state="corrupted-meta-json"} 0 + blocks_meta_synced{state="duplicate"} 0 + blocks_meta_synced{state="failed"} 0 + blocks_meta_synced{state="label-excluded"} 0 + blocks_meta_synced{state="loaded"} 2 + blocks_meta_synced{state="marked-for-deletion"} 0 + blocks_meta_synced{state="marked-for-no-compact"} 0 + blocks_meta_synced{state="no-meta-json"} 1 + blocks_meta_synced{state="time-excluded"} 0 + + # HELP blocks_meta_syncs_total Total blocks metadata synchronization attempts + # TYPE blocks_meta_syncs_total counter + blocks_meta_syncs_total 3 + `), "blocks_meta_syncs_total", "blocks_meta_sync_failures_total", "blocks_meta_synced")) + }) +} + +func TestMetaFetcher_FetchWithoutMarkedForDeletion_ShouldReturnDiscoveredBlocksExcludingMarkedForDeletion(t *testing.T) { + var ( + ctx = context.Background() + reg = prometheus.NewPedanticRegistry() + logger = log.NewNopLogger() + ) + + // Create a bucket client with global markers. + bkt, _ := objstore_testutil.NewFilesystemBucket(t, ctx, t.TempDir()) + bkt = block.BucketWithGlobalMarkers(bkt) + + f, err := block.NewMetaFetcher(logger, 10, bkt, t.TempDir(), reg, nil) + require.NoError(t, err) + + t.Run("should return no metas and no partials on no block in the storage", func(t *testing.T) { + actualMetas, actualPartials, actualErr := f.FetchWithoutMarkedForDeletion(ctx) + require.NoError(t, actualErr) + require.Empty(t, actualMetas) + require.Empty(t, actualPartials) + + assert.NoError(t, testutil.GatherAndCompare(reg, strings.NewReader(` + # HELP blocks_meta_sync_failures_total Total blocks metadata synchronization failures + # TYPE blocks_meta_sync_failures_total counter + blocks_meta_sync_failures_total 0 + + # HELP blocks_meta_synced Number of block metadata synced + # TYPE blocks_meta_synced gauge + blocks_meta_synced{state="corrupted-meta-json"} 0 + blocks_meta_synced{state="duplicate"} 0 + blocks_meta_synced{state="failed"} 0 + blocks_meta_synced{state="label-excluded"} 0 + blocks_meta_synced{state="loaded"} 0 + blocks_meta_synced{state="marked-for-deletion"} 0 + blocks_meta_synced{state="marked-for-no-compact"} 0 + blocks_meta_synced{state="no-meta-json"} 0 + blocks_meta_synced{state="time-excluded"} 0 + + # HELP blocks_meta_syncs_total Total blocks metadata synchronization attempts + # TYPE blocks_meta_syncs_total counter + blocks_meta_syncs_total 1 + `), "blocks_meta_syncs_total", "blocks_meta_sync_failures_total", "blocks_meta_synced")) + }) + + // Upload a block. + block1ID, block1Dir := createTestBlock(t) + require.NoError(t, block.Upload(ctx, logger, bkt, block1Dir)) + + // Upload a partial block. + block2ID, block2Dir := createTestBlock(t) + require.NoError(t, block.Upload(ctx, logger, bkt, block2Dir)) + require.NoError(t, bkt.Delete(ctx, path.Join(block2ID.String(), block.MetaFilename))) + + t.Run("should return metas and partials on some blocks in the storage", func(t *testing.T) { + actualMetas, actualPartials, actualErr := f.FetchWithoutMarkedForDeletion(ctx) + require.NoError(t, actualErr) + require.Len(t, actualMetas, 1) + require.Contains(t, actualMetas, block1ID) + require.Len(t, actualPartials, 1) + require.Contains(t, actualPartials, block2ID) + + assert.NoError(t, testutil.GatherAndCompare(reg, strings.NewReader(` + # HELP blocks_meta_sync_failures_total Total blocks metadata synchronization failures + # TYPE blocks_meta_sync_failures_total counter + blocks_meta_sync_failures_total 0 + + # HELP blocks_meta_synced Number of block metadata synced + # TYPE blocks_meta_synced gauge + blocks_meta_synced{state="corrupted-meta-json"} 0 + blocks_meta_synced{state="duplicate"} 0 + blocks_meta_synced{state="failed"} 0 + blocks_meta_synced{state="label-excluded"} 0 + blocks_meta_synced{state="loaded"} 1 + blocks_meta_synced{state="marked-for-deletion"} 0 + blocks_meta_synced{state="marked-for-no-compact"} 0 + blocks_meta_synced{state="no-meta-json"} 1 + blocks_meta_synced{state="time-excluded"} 0 + + # HELP blocks_meta_syncs_total Total blocks metadata synchronization attempts + # TYPE blocks_meta_syncs_total counter + blocks_meta_syncs_total 2 + `), "blocks_meta_syncs_total", "blocks_meta_sync_failures_total", "blocks_meta_synced")) + }) + + // Upload a block and mark it for deletion. + block3ID, block3Dir := createTestBlock(t) + require.NoError(t, block.Upload(ctx, logger, bkt, block3Dir)) + require.NoError(t, block.MarkForDeletion(ctx, logger, bkt, block3ID, "", promauto.With(nil).NewCounter(prometheus.CounterOpts{}))) + + t.Run("should include blocks marked for deletion", func(t *testing.T) { + actualMetas, actualPartials, actualErr := f.FetchWithoutMarkedForDeletion(ctx) + require.NoError(t, actualErr) + require.Len(t, actualMetas, 1) + require.Contains(t, actualMetas, block1ID) + require.Len(t, actualPartials, 1) + require.Contains(t, actualPartials, block2ID) + + assert.NoError(t, testutil.GatherAndCompare(reg, strings.NewReader(` + # HELP blocks_meta_sync_failures_total Total blocks metadata synchronization failures + # TYPE blocks_meta_sync_failures_total counter + blocks_meta_sync_failures_total 0 + + # HELP blocks_meta_synced Number of block metadata synced + # TYPE blocks_meta_synced gauge + blocks_meta_synced{state="corrupted-meta-json"} 0 + blocks_meta_synced{state="duplicate"} 0 + blocks_meta_synced{state="failed"} 0 + blocks_meta_synced{state="label-excluded"} 0 + blocks_meta_synced{state="loaded"} 1 + blocks_meta_synced{state="marked-for-deletion"} 1 + blocks_meta_synced{state="marked-for-no-compact"} 0 + blocks_meta_synced{state="no-meta-json"} 1 + blocks_meta_synced{state="time-excluded"} 0 + + # HELP blocks_meta_syncs_total Total blocks metadata synchronization attempts + # TYPE blocks_meta_syncs_total counter + blocks_meta_syncs_total 3 + `), "blocks_meta_syncs_total", "blocks_meta_sync_failures_total", "blocks_meta_synced")) + }) +} + +func TestMetaFetcher_ShouldNotIssueAnyAPICallToObjectStorageIfAllBlockMetasAreCached(t *testing.T) { + var ( + ctx = context.Background() + logger = log.NewNopLogger() + fetcherDir = t.TempDir() + ) + bkt, _ := objstore_testutil.NewFilesystemBucket(t, ctx, fetcherDir) + + // Upload few blocks. + block1ID, block1Dir := createTestBlock(t) + require.NoError(t, block.Upload(ctx, logger, bkt, block1Dir)) + block2ID, block2Dir := createTestBlock(t) + require.NoError(t, block.Upload(ctx, logger, bkt, block2Dir)) + + // Create a fetcher and fetch block metas to populate the cache on disk. + reg1 := prometheus.NewPedanticRegistry() + ctx = phlarecontext.WithRegistry(ctx, reg1) + bkt, _ = objstore_testutil.NewFilesystemBucket(t, ctx, fetcherDir) + fetcher1, err := block.NewMetaFetcher(logger, 10, bkt, fetcherDir, nil, nil) + require.NoError(t, err) + actualMetas, _, actualErr := fetcher1.Fetch(ctx) + require.NoError(t, actualErr) + require.Len(t, actualMetas, 2) + require.Contains(t, actualMetas, block1ID) + require.Contains(t, actualMetas, block2ID) + + assert.NoError(t, testutil.GatherAndCompare(reg1, strings.NewReader(` + # HELP objstore_bucket_operations_total Total number of all attempted operations against a bucket. + # TYPE objstore_bucket_operations_total counter + objstore_bucket_operations_total{bucket="test",operation="attributes"} 0 + objstore_bucket_operations_total{bucket="test",operation="exists"} 0 + objstore_bucket_operations_total{bucket="test",operation="delete"} 0 + objstore_bucket_operations_total{bucket="test",operation="upload"} 0 + objstore_bucket_operations_total{bucket="test",operation="get"} 2 + objstore_bucket_operations_total{bucket="test",operation="get_range"} 0 + objstore_bucket_operations_total{bucket="test",operation="iter"} 1 + `), "objstore_bucket_operations_total")) + + // Create a new fetcher and fetch blocks again. This time we expect all meta.json to be loaded from cache. + reg2 := prometheus.NewPedanticRegistry() + ctx = phlarecontext.WithRegistry(ctx, reg2) + bkt, _ = objstore_testutil.NewFilesystemBucket(t, ctx, fetcherDir) + fetcher2, err := block.NewMetaFetcher(logger, 10, bkt, fetcherDir, nil, nil) + require.NoError(t, err) + actualMetas, _, actualErr = fetcher2.Fetch(ctx) + require.NoError(t, actualErr) + require.Len(t, actualMetas, 2) + require.Contains(t, actualMetas, block1ID) + require.Contains(t, actualMetas, block2ID) + + assert.NoError(t, testutil.GatherAndCompare(reg2, strings.NewReader(` + # HELP objstore_bucket_operations_total Total number of all attempted operations against a bucket. + # TYPE objstore_bucket_operations_total counter + objstore_bucket_operations_total{bucket="test",operation="attributes"} 0 + objstore_bucket_operations_total{bucket="test",operation="delete"} 0 + objstore_bucket_operations_total{bucket="test",operation="exists"} 0 + objstore_bucket_operations_total{bucket="test",operation="get"} 0 + objstore_bucket_operations_total{bucket="test",operation="get_range"} 0 + objstore_bucket_operations_total{bucket="test",operation="iter"} 1 + objstore_bucket_operations_total{bucket="test",operation="upload"} 0 + `), "objstore_bucket_operations_total")) +} + +func createTestBlock(t *testing.T) (blockID ulid.ULID, blockDir string) { + meta, dir, err := block_testutil.CreateBlock(t, func() []*testhelper.ProfileBuilder { + return []*testhelper.ProfileBuilder{ + testhelper.NewProfileBuilder(int64(1)). + CPUProfile(). + WithLabels( + "job", "a", + ).ForStacktraceString("foo", "bar", "baz").AddSamples(1), + } + }) + require.NoError(t, err) + blockID = meta.ULID + blockDir = filepath.Join(dir, blockID.String()) + return +} diff --git a/pkg/phlaredb/block/global_markers.go b/pkg/phlaredb/block/global_markers.go index 3720734859..03028e36ba 100644 --- a/pkg/phlaredb/block/global_markers.go +++ b/pkg/phlaredb/block/global_markers.go @@ -14,7 +14,8 @@ import ( "github.com/oklog/ulid" "github.com/pkg/errors" - "github.com/thanos-io/objstore" + + "github.com/grafana/pyroscope/pkg/objstore" ) const ( diff --git a/pkg/phlaredb/block/global_markers_bucket_client.go b/pkg/phlaredb/block/global_markers_bucket_client.go new file mode 100644 index 0000000000..4da30574e2 --- /dev/null +++ b/pkg/phlaredb/block/global_markers_bucket_client.go @@ -0,0 +1,199 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/cortexproject/cortex/blob/master/pkg/storage/tsdb/bucketindex/markers_bucket_client.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Cortex Authors. + +package block + +import ( + "bytes" + "context" + "io" + "path" + + "github.com/grafana/dskit/multierror" + "github.com/oklog/ulid" + thanosobjstore "github.com/thanos-io/objstore" + + "github.com/grafana/pyroscope/pkg/objstore" +) + +// globalMarkersBucket is a bucket client which stores markers (eg. block deletion marks) in a per-tenant +// global location too. +type globalMarkersBucket struct { + parent objstore.Bucket +} + +// BucketWithGlobalMarkers wraps the input bucket into a bucket which also keeps track of markers +// in the global markers location. +func BucketWithGlobalMarkers(b objstore.Bucket) objstore.Bucket { + return &globalMarkersBucket{ + parent: b, + } +} + +// Upload implements objstore.Bucket. +func (b *globalMarkersBucket) Upload(ctx context.Context, name string, r io.Reader) error { + globalMarkPath := getGlobalMarkPathFromBlockMark(name) + if globalMarkPath == "" { + return b.parent.Upload(ctx, name, r) + } + + // Read the marker. + body, err := io.ReadAll(r) + if err != nil { + return err + } + + // Upload it to the original location. + if err := b.parent.Upload(ctx, name, bytes.NewBuffer(body)); err != nil { + return err + } + + // Upload it to the global markers location too. + return b.parent.Upload(ctx, globalMarkPath, bytes.NewBuffer(body)) +} + +// Delete implements objstore.Bucket. +func (b *globalMarkersBucket) Delete(ctx context.Context, name string) error { + // Call the parent. Only return error here (without deleting global marker too) if error is different than "not found". + err1 := b.parent.Delete(ctx, name) + if err1 != nil && !b.parent.IsObjNotFoundErr(err1) { + return err1 + } + + // Delete the marker in the global markers location too. + globalMarkPath := getGlobalMarkPathFromBlockMark(name) + if globalMarkPath == "" { + return err1 + } + + var err2 error + if err := b.parent.Delete(ctx, globalMarkPath); err != nil { + if !b.parent.IsObjNotFoundErr(err) { + err2 = err + } + } + + if err1 != nil { + // In this case err1 is "ObjNotFound". If we tried to wrap it together with err2, we would need to + // handle this possibility in globalMarkersBucket.IsObjNotFoundErr(). Instead we just ignore err2, if any. + return err1 + } + + return err2 +} + +// Name implements objstore.Bucket. +func (b *globalMarkersBucket) Name() string { + return b.parent.Name() +} + +// Close implements objstore.Bucket. +func (b *globalMarkersBucket) Close() error { + return b.parent.Close() +} + +// Iter implements objstore.Bucket. +func (b *globalMarkersBucket) Iter(ctx context.Context, dir string, f func(string) error, options ...thanosobjstore.IterOption) error { + return b.parent.Iter(ctx, dir, f, options...) +} + +// Get implements objstore.Bucket. +func (b *globalMarkersBucket) Get(ctx context.Context, name string) (io.ReadCloser, error) { + return b.parent.Get(ctx, name) +} + +// GetRange implements objstore.Bucket. +func (b *globalMarkersBucket) GetRange(ctx context.Context, name string, off, length int64) (io.ReadCloser, error) { + return b.parent.GetRange(ctx, name, off, length) +} + +// Exists implements objstore.Bucket. +func (b *globalMarkersBucket) Exists(ctx context.Context, name string) (bool, error) { + globalMarkPath := getGlobalMarkPathFromBlockMark(name) + if globalMarkPath == "" { + return b.parent.Exists(ctx, name) + } + + // Report "exists" only if BOTH (block-local, and global) files exist, otherwise Thanos + // code will never try to upload the file again, if it finds that it exist. + ok1, err1 := b.parent.Exists(ctx, name) + ok2, err2 := b.parent.Exists(ctx, globalMarkPath) + + var me multierror.MultiError + me.Add(err1) + me.Add(err2) + + return ok1 && ok2, me.Err() +} + +// IsObjNotFoundErr implements objstore.Bucket. +func (b *globalMarkersBucket) IsObjNotFoundErr(err error) bool { + return b.parent.IsObjNotFoundErr(err) +} + +// IsCustomerManagedKeyError implements objstore.Bucket. +func (b *globalMarkersBucket) IsCustomerManagedKeyError(err error) bool { + return b.parent.IsCustomerManagedKeyError(err) +} + +// Attributes implements objstore.Bucket. +func (b *globalMarkersBucket) Attributes(ctx context.Context, name string) (thanosobjstore.ObjectAttributes, error) { + return b.parent.Attributes(ctx, name) +} + +// Attributes implements objstore.ReaderAt. +func (b *globalMarkersBucket) ReaderAt(ctx context.Context, filename string) (objstore.ReaderAtCloser, error) { + return b.parent.ReaderAt(ctx, filename) +} + +// ReaderWithExpectedErrs implements objstore.Bucket. +func (b *globalMarkersBucket) ReaderWithExpectedErrs(fn objstore.IsOpFailureExpectedFunc) objstore.BucketReader { + return b.WithExpectedErrs(fn) +} + +// WithExpectedErrs implements objstore.Bucket. +func (b *globalMarkersBucket) WithExpectedErrs(fn objstore.IsOpFailureExpectedFunc) objstore.Bucket { + if ib, ok := b.parent.(objstore.InstrumentedBucket); ok { + return &globalMarkersBucket{ + parent: ib.WithExpectedErrs(fn), + } + } + + return b +} + +// getGlobalMarkPathFromBlockMark returns path to global mark, if name points to a block-local mark file. If name +// doesn't point to a block-local mark file, returns empty string. +func getGlobalMarkPathFromBlockMark(name string) string { + if blockID, ok := isDeletionMark(name); ok { + return path.Clean(path.Join(path.Dir(name), "../", DeletionMarkFilepath(blockID))) + } + + if blockID, ok := isNoCompactMark(name); ok { + return path.Clean(path.Join(path.Dir(name), "../", NoCompactMarkFilepath(blockID))) + } + + return "" +} + +func isDeletionMark(name string) (ulid.ULID, bool) { + if path.Base(name) != DeletionMarkFilename { + return ulid.ULID{}, false + } + + // Parse the block ID in the path. If there's no block ID, then it's not the per-block + // deletion mark. + return IsBlockDir(path.Dir(name)) +} + +func isNoCompactMark(name string) (ulid.ULID, bool) { + if path.Base(name) != NoCompactMarkFilename { + return ulid.ULID{}, false + } + + // Parse the block ID in the path. If there's no block ID, then it's not the per-block + // no-compact mark. + return IsBlockDir(path.Dir(name)) +} diff --git a/pkg/phlaredb/block/global_markers_bucket_client_test.go b/pkg/phlaredb/block/global_markers_bucket_client_test.go new file mode 100644 index 0000000000..763da95efa --- /dev/null +++ b/pkg/phlaredb/block/global_markers_bucket_client_test.go @@ -0,0 +1,338 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/cortexproject/cortex/blob/master/pkg/storage/tsdb/bucketindex/markers_bucket_client_test.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Cortex Authors. + +package block + +import ( + "bytes" + "context" + "path" + "strings" + "testing" + + "github.com/oklog/ulid" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/testutil" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + + "github.com/grafana/pyroscope/pkg/objstore" + objstore_testutil "github.com/grafana/pyroscope/pkg/objstore/testutil" + phlarecontext "github.com/grafana/pyroscope/pkg/phlare/context" +) + +func TestGlobalMarkersBucket_Delete_ShouldSucceedIfDeletionMarkDoesNotExistInTheBlockButExistInTheGlobalLocation(t *testing.T) { + ctx := context.Background() + + // Create a mocked block deletion mark in the global location. + blockID := ulid.MustNew(1, nil) + for _, globalPath := range []string{DeletionMarkFilepath(blockID), NoCompactMarkFilepath(blockID)} { + bkt, _ := objstore_testutil.NewFilesystemBucket(t, ctx, t.TempDir()) + bkt = BucketWithGlobalMarkers(bkt) + + require.NoError(t, bkt.Upload(ctx, globalPath, strings.NewReader("{}"))) + + // Ensure it exists before deleting it. + ok, err := bkt.Exists(ctx, globalPath) + require.NoError(t, err) + require.True(t, ok) + + require.NoError(t, bkt.Delete(ctx, globalPath)) + + // Ensure has been actually deleted. + ok, err = bkt.Exists(ctx, globalPath) + require.NoError(t, err) + require.False(t, ok) + } +} + +func TestGlobalMarkersBucket_DeleteShouldDeleteGlobalMarkIfBlockMarkerDoesntExist(t *testing.T) { + ctx := context.Background() + + blockID := ulid.MustNew(1, nil) + + for name, tc := range map[string]struct { + blockMarker string + globalMarker string + }{ + "deletion mark": { + blockMarker: path.Join(blockID.String(), DeletionMarkFilename), + globalMarker: DeletionMarkFilepath(blockID), + }, + "no compact": { + blockMarker: path.Join(blockID.String(), NoCompactMarkFilename), + globalMarker: NoCompactMarkFilepath(blockID), + }, + } { + t.Run(name, func(t *testing.T) { + // Create a mocked block deletion mark in the global location. + bkt, _ := objstore_testutil.NewFilesystemBucket(t, ctx, t.TempDir()) + bkt = BucketWithGlobalMarkers(bkt) + + // Upload global only + require.NoError(t, bkt.Upload(ctx, tc.globalMarker, strings.NewReader("{}"))) + + // Verify global exists. + verifyPathExists(t, bkt, tc.globalMarker, true) + + // Delete block marker. + err := bkt.Delete(ctx, tc.blockMarker) + require.NoError(t, err) + + // Ensure global one been actually deleted. + verifyPathExists(t, bkt, tc.globalMarker, false) + }) + } +} + +func TestUploadToGlobalMarkerPath(t *testing.T) { + blockID := ulid.MustNew(1, nil) + for name, tc := range map[string]struct { + blockMarker string + globalMarker string + }{ + "deletion mark": { + blockMarker: path.Join(blockID.String(), DeletionMarkFilename), + globalMarker: DeletionMarkFilepath(blockID), + }, + "no compact": { + blockMarker: path.Join(blockID.String(), NoCompactMarkFilename), + globalMarker: NoCompactMarkFilepath(blockID), + }, + } { + t.Run(name, func(t *testing.T) { + bkt, _ := objstore_testutil.NewFilesystemBucket(t, context.Background(), t.TempDir()) + bkt = BucketWithGlobalMarkers(bkt) + + // Verify that uploading block mark file uploads it to the global markers location too. + require.NoError(t, bkt.Upload(context.Background(), tc.blockMarker, strings.NewReader("mark file"))) + + verifyPathExists(t, bkt, tc.globalMarker, true) + }) + } +} + +func TestGlobalMarkersBucket_ExistShouldReportTrueOnlyIfBothExist(t *testing.T) { + blockID := ulid.MustNew(1, nil) + + for name, tc := range map[string]struct { + blockMarker string + globalMarker string + }{ + "deletion mark": { + blockMarker: path.Join(blockID.String(), DeletionMarkFilename), + globalMarker: DeletionMarkFilepath(blockID), + }, + "no compact": { + blockMarker: path.Join(blockID.String(), NoCompactMarkFilename), + globalMarker: NoCompactMarkFilepath(blockID), + }, + } { + t.Run(name, func(t *testing.T) { + bkt, _ := objstore_testutil.NewFilesystemBucket(t, context.Background(), t.TempDir()) + bkt = BucketWithGlobalMarkers(bkt) + + // Upload to global marker only + require.NoError(t, bkt.Upload(context.Background(), tc.globalMarker, strings.NewReader("mark file"))) + + // Verify global exists, but block marker doesn't. + verifyPathExists(t, bkt, tc.globalMarker, true) + verifyPathExists(t, bkt, tc.blockMarker, false) + + // Now upload to block marker (also overwrites global) + require.NoError(t, bkt.Upload(context.Background(), tc.blockMarker, strings.NewReader("mark file"))) + + // Verify global exists and block marker does too. + verifyPathExists(t, bkt, tc.globalMarker, true) + verifyPathExists(t, bkt, tc.blockMarker, true) + + // Now delete global file, and only keep block. + require.NoError(t, bkt.Delete(context.Background(), tc.globalMarker)) + + // Verify global doesn't exist anymore. Block marker also returns false, even though it *does* exist. + verifyPathExists(t, bkt, tc.globalMarker, false) + verifyPathExists(t, bkt, tc.blockMarker, false) + }) + } +} + +func verifyPathExists(t *testing.T, bkt objstore.Bucket, name string, expected bool) { + t.Helper() + + ok, err := bkt.Exists(context.Background(), name) + require.NoError(t, err) + require.Equal(t, expected, ok) +} + +func TestGlobalMarkersBucket_getGlobalMarkPathFromBlockMark(t *testing.T) { + type testCase struct { + name string + expected string + } + + tests := []testCase{ + {name: "", expected: ""}, + {name: "01FV060K6XXCS8BCD2CH6C3GBR/index", expected: ""}, + } + + for _, marker := range []string{DeletionMarkFilename, NoCompactMarkFilename} { + tests = append(tests, testCase{name: marker, expected: ""}) + tests = append(tests, testCase{name: "01FV060K6XXCS8BCD2CH6C3GBR/" + marker, expected: "markers/01FV060K6XXCS8BCD2CH6C3GBR-" + marker}) + tests = append(tests, testCase{name: "/path/to/01FV060K6XXCS8BCD2CH6C3GBR/" + marker, expected: "/path/to/markers/01FV060K6XXCS8BCD2CH6C3GBR-" + marker}) + tests = append(tests, testCase{name: "invalid-block-id/" + marker, expected: ""}) + } + + for _, tc := range tests { + t.Run(tc.name, func(t *testing.T) { + result := getGlobalMarkPathFromBlockMark(tc.name) + assert.Equal(t, tc.expected, result) + }) + } +} + +func TestGlobalMarkersBucket_isDeletionMark(t *testing.T) { + block1 := ulid.MustNew(1, nil) + + tests := []struct { + name string + expectedOk bool + expectedID ulid.ULID + }{ + { + name: "", + expectedOk: false, + }, { + name: "deletion-mark.json", + expectedOk: false, + }, { + name: block1.String() + "/index", + expectedOk: false, + }, { + name: block1.String() + "/deletion-mark.json", + expectedOk: true, + expectedID: block1, + }, { + name: "/path/to/" + block1.String() + "/deletion-mark.json", + expectedOk: true, + expectedID: block1, + }, + } + + for _, tc := range tests { + t.Run(tc.name, func(t *testing.T) { + actualID, actualOk := isDeletionMark(tc.name) + assert.Equal(t, tc.expectedOk, actualOk) + assert.Equal(t, tc.expectedID, actualID) + }) + } +} + +func TestGlobalMarkersBucket_isNoCompactMark(t *testing.T) { + block1 := ulid.MustNew(1, nil) + + tests := []struct { + name string + expectedOk bool + expectedID ulid.ULID + }{ + { + name: "", + expectedOk: false, + }, { + name: "no-compact-mark.json", + expectedOk: false, + }, { + name: block1.String() + "/index", + expectedOk: false, + }, { + name: block1.String() + "/no-compact-mark.json", + expectedOk: true, + expectedID: block1, + }, { + name: "/path/to/" + block1.String() + "/no-compact-mark.json", + expectedOk: true, + expectedID: block1, + }, + } + + for _, tc := range tests { + t.Run(tc.name, func(t *testing.T) { + actualID, actualOk := isNoCompactMark(tc.name) + assert.Equal(t, tc.expectedOk, actualOk) + assert.Equal(t, tc.expectedID, actualID) + }) + } +} + +func TestBucketWithGlobalMarkers_ShouldWorkCorrectlyWithBucketMetrics(t *testing.T) { + reg := prometheus.NewPedanticRegistry() + ctx := phlarecontext.WithRegistry(context.Background(), reg) + // We wrap the underlying filesystem bucket client with metrics, + // global markers (intentionally in the middle of the chain) and + // user prefix. + bkt, _ := objstore_testutil.NewFilesystemBucket(t, ctx, t.TempDir()) + bkt = BucketWithGlobalMarkers(bkt) + userBkt := objstore.NewUserBucketClient("user-1", bkt, nil) + + reader, err := userBkt.Get(ctx, "does-not-exist") + require.Error(t, err) + require.Nil(t, reader) + assert.True(t, bkt.IsObjNotFoundErr(err)) + + // Should track the failure. + assert.NoError(t, testutil.GatherAndCompare(reg, bytes.NewBufferString(` + # HELP objstore_bucket_operation_failures_total Total number of operations against a bucket that failed, but were not expected to fail in certain way from caller perspective. Those errors have to be investigated. + # TYPE objstore_bucket_operation_failures_total counter + objstore_bucket_operation_failures_total{bucket="test",operation="attributes"} 0 + objstore_bucket_operation_failures_total{bucket="test",operation="delete"} 0 + objstore_bucket_operation_failures_total{bucket="test",operation="exists"} 0 + objstore_bucket_operation_failures_total{bucket="test",operation="get"} 1 + objstore_bucket_operation_failures_total{bucket="test",operation="get_range"} 0 + objstore_bucket_operation_failures_total{bucket="test",operation="iter"} 0 + objstore_bucket_operation_failures_total{bucket="test",operation="upload"} 0 + # HELP objstore_bucket_operations_total Total number of all attempted operations against a bucket. + # TYPE objstore_bucket_operations_total counter + objstore_bucket_operations_total{bucket="test",operation="attributes"} 0 + objstore_bucket_operations_total{bucket="test",operation="delete"} 0 + objstore_bucket_operations_total{bucket="test",operation="exists"} 0 + objstore_bucket_operations_total{bucket="test",operation="get"} 1 + objstore_bucket_operations_total{bucket="test",operation="get_range"} 0 + objstore_bucket_operations_total{bucket="test",operation="iter"} 0 + objstore_bucket_operations_total{bucket="test",operation="upload"} 0 + `), + "objstore_bucket_operations_total", + "objstore_bucket_operation_failures_total", + )) + + reader, err = userBkt.ReaderWithExpectedErrs(userBkt.IsObjNotFoundErr).Get(ctx, "does-not-exist") + require.Error(t, err) + require.Nil(t, reader) + assert.True(t, bkt.IsObjNotFoundErr(err)) + + // Should not track the failure. + assert.NoError(t, testutil.GatherAndCompare(reg, bytes.NewBufferString(` + # HELP objstore_bucket_operation_failures_total Total number of operations against a bucket that failed, but were not expected to fail in certain way from caller perspective. Those errors have to be investigated. + # TYPE objstore_bucket_operation_failures_total counter + objstore_bucket_operation_failures_total{bucket="test",operation="attributes"} 0 + objstore_bucket_operation_failures_total{bucket="test",operation="delete"} 0 + objstore_bucket_operation_failures_total{bucket="test",operation="exists"} 0 + objstore_bucket_operation_failures_total{bucket="test",operation="get"} 1 + objstore_bucket_operation_failures_total{bucket="test",operation="get_range"} 0 + objstore_bucket_operation_failures_total{bucket="test",operation="iter"} 0 + objstore_bucket_operation_failures_total{bucket="test",operation="upload"} 0 + # HELP objstore_bucket_operations_total Total number of all attempted operations against a bucket. + # TYPE objstore_bucket_operations_total counter + objstore_bucket_operations_total{bucket="test",operation="attributes"} 0 + objstore_bucket_operations_total{bucket="test",operation="delete"} 0 + objstore_bucket_operations_total{bucket="test",operation="exists"} 0 + objstore_bucket_operations_total{bucket="test",operation="get"} 2 + objstore_bucket_operations_total{bucket="test",operation="get_range"} 0 + objstore_bucket_operations_total{bucket="test",operation="iter"} 0 + objstore_bucket_operations_total{bucket="test",operation="upload"} 0 + `), + "objstore_bucket_operations_total", + "objstore_bucket_operation_failures_total", + )) +} diff --git a/pkg/phlaredb/block/global_markers_test.go b/pkg/phlaredb/block/global_markers_test.go index f47129a9fe..c7a7d636e4 100644 --- a/pkg/phlaredb/block/global_markers_test.go +++ b/pkg/phlaredb/block/global_markers_test.go @@ -14,7 +14,7 @@ import ( "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" - mimir_testutil "github.com/grafana/mimir/pkg/storage/tsdb/testutil" + "github.com/grafana/pyroscope/pkg/objstore/testutil" ) func TestDeletionMarkFilepath(t *testing.T) { @@ -72,7 +72,7 @@ func TestListBlockDeletionMarks(t *testing.T) { ) t.Run("should return an empty map on empty bucket", func(t *testing.T) { - bkt, _ := mimir_testutil.PrepareFilesystemBucket(t) + bkt, _ := testutil.NewFilesystemBucket(t, ctx, t.TempDir()) actualMarks, actualErr := ListBlockDeletionMarks(ctx, bkt) require.NoError(t, actualErr) @@ -80,7 +80,7 @@ func TestListBlockDeletionMarks(t *testing.T) { }) t.Run("should return a map with the locations of the block deletion marks found", func(t *testing.T) { - bkt, _ := mimir_testutil.PrepareFilesystemBucket(t) + bkt, _ := testutil.NewFilesystemBucket(t, ctx, t.TempDir()) require.NoError(t, bkt.Upload(ctx, DeletionMarkFilepath(block1), strings.NewReader("{}"))) require.NoError(t, bkt.Upload(ctx, NoCompactMarkFilepath(block2), strings.NewReader("{}"))) diff --git a/pkg/phlaredb/block/testutil/create_block.go b/pkg/phlaredb/block/testutil/create_block.go new file mode 100644 index 0000000000..920ce92036 --- /dev/null +++ b/pkg/phlaredb/block/testutil/create_block.go @@ -0,0 +1,58 @@ +package testutil + +import ( + "context" + "path/filepath" + "testing" + "time" + + "github.com/prometheus/common/model" + "github.com/stretchr/testify/require" + + phlaremodel "github.com/grafana/pyroscope/pkg/model" + "github.com/grafana/pyroscope/pkg/phlaredb" + "github.com/grafana/pyroscope/pkg/phlaredb/block" + "github.com/grafana/pyroscope/pkg/pprof/testhelper" +) + +type noLimit struct{} + +func (n noLimit) AllowProfile(fp model.Fingerprint, lbs phlaremodel.Labels, tsNano int64) error { + return nil +} + +func (n noLimit) Stop() {} + +// CreateBlock creates a block with the given profiles. +// Returns the block metadata, the directory where the block is stored, and an error if any. +func CreateBlock(t testing.TB, generator func() []*testhelper.ProfileBuilder) (block.Meta, string, error) { + t.Helper() + dir := t.TempDir() + ctx := context.Background() + h, err := phlaredb.NewHead(ctx, phlaredb.Config{ + DataPath: dir, + MaxBlockDuration: 24 * time.Hour, + Parquet: &phlaredb.ParquetConfig{ + MaxBufferRowCount: 10, + }, + }, noLimit{}) + require.NoError(t, err) + + // ingest. + for _, p := range generator() { + require.NoError(t, h.Ingest(ctx, p.Profile, p.UUID, p.Labels...)) + } + + require.NoError(t, h.Flush(ctx)) + require.NoError(t, h.Move()) + localDir := filepath.Join(dir, phlaredb.PathLocal) + metaMap, err := block.ListBlocks(localDir, time.Time{}) + require.NoError(t, err) + require.Len(t, metaMap, 1) + var meta *block.Meta + for _, m := range metaMap { + meta = m + } + require.NotNil(t, meta) + return *meta, localDir, nil +} diff --git a/pkg/phlaredb/compact_test.go b/pkg/phlaredb/compact_test.go index 1966f49a3c..040e98d829 100644 --- a/pkg/phlaredb/compact_test.go +++ b/pkg/phlaredb/compact_test.go @@ -555,7 +555,7 @@ func newBlock(t *testing.T, generator func() []*testhelper.ProfileBuilder) Block StoragePrefix: "local", }, "test") require.NoError(t, err) - metaMap, err := block.ListBlocks(filepath.Join(dir, pathLocal), time.Time{}) + metaMap, err := block.ListBlocks(filepath.Join(dir, PathLocal), time.Time{}) require.NoError(t, err) require.Len(t, metaMap, 1) var meta *block.Meta diff --git a/pkg/phlaredb/head.go b/pkg/phlaredb/head.go index dd8f1fb9de..712d864220 100644 --- a/pkg/phlaredb/head.go +++ b/pkg/phlaredb/head.go @@ -75,7 +75,7 @@ type Head struct { const ( pathHead = "head" - pathLocal = "local" + PathLocal = "local" defaultFolderMode = 0o755 ) @@ -95,7 +95,7 @@ func NewHead(phlarectx context.Context, cfg Config, limiter TenantLimiter) (*Hea limiter: limiter, } h.headPath = filepath.Join(cfg.DataPath, pathHead, h.meta.ULID.String()) - h.localPath = filepath.Join(cfg.DataPath, pathLocal, h.meta.ULID.String()) + h.localPath = filepath.Join(cfg.DataPath, PathLocal, h.meta.ULID.String()) if cfg.Parquet != nil { h.parquetConfig = cfg.Parquet diff --git a/pkg/phlaredb/phlaredb.go b/pkg/phlaredb/phlaredb.go index f61f4fb9fe..cfe052756e 100644 --- a/pkg/phlaredb/phlaredb.go +++ b/pkg/phlaredb/phlaredb.go @@ -116,7 +116,7 @@ func New(phlarectx context.Context, cfg Config, limiter TenantLimiter, fs phlare f.wg.Add(1) go f.loop() - f.blockQuerier = NewBlockQuerier(phlarectx, phlareobj.NewPrefixedBucket(fs, pathLocal)) + f.blockQuerier = NewBlockQuerier(phlarectx, phlareobj.NewPrefixedBucket(fs, PathLocal)) // do an initial querier sync ctx := context.Background() @@ -127,7 +127,7 @@ func New(phlarectx context.Context, cfg Config, limiter TenantLimiter, fs phlare } func (f *PhlareDB) LocalDataPath() string { - return filepath.Join(f.cfg.DataPath, pathLocal) + return filepath.Join(f.cfg.DataPath, PathLocal) } func (f *PhlareDB) BlockMetas(ctx context.Context) ([]*block.Meta, error) { diff --git a/pkg/phlaredb/sample_merge_test.go b/pkg/phlaredb/sample_merge_test.go index 14d3a73a5d..14b67ca4a6 100644 --- a/pkg/phlaredb/sample_merge_test.go +++ b/pkg/phlaredb/sample_merge_test.go @@ -104,7 +104,7 @@ func TestMergeSampleByStacktraces(t *testing.T) { require.NoError(t, db.Flush(context.Background())) - b, err := filesystem.NewBucket(filepath.Join(contextDataDir(ctx), pathLocal)) + b, err := filesystem.NewBucket(filepath.Join(contextDataDir(ctx), PathLocal)) require.NoError(t, err) // open resulting block @@ -319,7 +319,7 @@ func TestMergeSampleByLabels(t *testing.T) { require.NoError(t, db.Flush(context.Background())) - b, err := filesystem.NewBucket(filepath.Join(contextDataDir(ctx), pathLocal)) + b, err := filesystem.NewBucket(filepath.Join(contextDataDir(ctx), PathLocal)) require.NoError(t, err) // open resulting block @@ -483,7 +483,7 @@ func TestMergePprof(t *testing.T) { require.NoError(t, db.Flush(context.Background())) - b, err := filesystem.NewBucket(filepath.Join(contextDataDir(ctx), pathLocal)) + b, err := filesystem.NewBucket(filepath.Join(contextDataDir(ctx), PathLocal)) require.NoError(t, err) // open resulting block From 9de29ce6b35d9948d0baf18f80ea49c1dc987bb4 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Fri, 8 Sep 2023 15:25:52 +0200 Subject: [PATCH 14/74] make go/mod --- ebpf/go.mod | 2 +- ebpf/go.sum | 4 ++-- go.mod | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/ebpf/go.mod b/ebpf/go.mod index 974590cb3a..ed8b8adc7e 100644 --- a/ebpf/go.mod +++ b/ebpf/go.mod @@ -12,7 +12,7 @@ require ( github.com/pkg/errors v0.9.1 github.com/prometheus/client_golang v1.16.0 github.com/prometheus/common v0.44.0 - github.com/prometheus/prometheus v0.46.0 + github.com/prometheus/prometheus v0.47.0 github.com/samber/lo v1.38.1 github.com/stretchr/testify v1.8.4 golang.org/x/exp v0.0.0-20230713183714-613f0c0eb8a1 diff --git a/ebpf/go.sum b/ebpf/go.sum index 05ddfca2c3..c4e358fdc7 100644 --- a/ebpf/go.sum +++ b/ebpf/go.sum @@ -43,8 +43,8 @@ github.com/prometheus/common v0.44.0 h1:+5BrQJwiBB9xsMygAB3TNvpQKOwlkc25LbISbrdO github.com/prometheus/common v0.44.0/go.mod h1:ofAIvZbQ1e/nugmZGz4/qCb9Ap1VoSTIO7x0VV9VvuY= github.com/prometheus/procfs v0.11.0 h1:5EAgkfkMl659uZPbe9AS2N68a7Cc1TJbPEuGzFuRbyk= github.com/prometheus/procfs v0.11.0/go.mod h1:nwNm2aOCAYw8uTR/9bWRREkZFxAUcWzPHWJq+XBB/FM= -github.com/prometheus/prometheus v0.46.0 h1:9JSdXnsuT6YsbODEhSQMwxNkGwPExfmzqG73vCMk/Kw= -github.com/prometheus/prometheus v0.46.0/go.mod h1:10L5IJE5CEsjee1FnOcVswYXlPIscDWWt3IJ2UDYrz4= +github.com/prometheus/prometheus v0.47.0 h1:tIJJKZGlmrMVsvIt6rMfB8he7CRHEc8ZxS5ubcZtbkM= +github.com/prometheus/prometheus v0.47.0/go.mod h1:J/bmOSjgH7lFxz2gZhrWEZs2i64vMS+HIuZfmYNhJ/M= github.com/rogpeppe/go-internal v1.11.0 h1:cWPaGQEPrBb5/AsnsZesgZZ9yb1OQ+GOISoDNXVBh4M= github.com/rogpeppe/go-internal v1.11.0/go.mod h1:ddIwULY96R17DhadqLgMfk9H9tvdUzkipdSkR5nkCZA= github.com/samber/lo v1.38.1 h1:j2XEAqXKb09Am4ebOg31SpvzUTTs6EN3VfgeLUhPdXM= diff --git a/go.mod b/go.mod index 60c634cc63..03b07054fe 100644 --- a/go.mod +++ b/go.mod @@ -18,6 +18,7 @@ require ( github.com/go-kit/log v0.2.1 github.com/gogo/protobuf v1.3.2 github.com/gogo/status v1.1.1 + github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da github.com/golang/protobuf v1.5.3 github.com/google/go-cmp v0.5.9 github.com/google/pprof v0.0.0-20230705174524-200ffdc848b8 @@ -130,7 +131,6 @@ require ( github.com/go-task/slim-sprig v0.0.0-20230315185526-52ccab3ef572 // indirect github.com/gogo/googleapis v1.4.1 // indirect github.com/golang-jwt/jwt/v4 v4.5.0 // indirect - github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da // indirect github.com/golang/snappy v0.0.4 // indirect github.com/google/btree v1.1.2 // indirect github.com/google/go-querystring v1.1.0 // indirect From e3930f0bf849bc0d261574c9103386502c73aa46 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Fri, 8 Sep 2023 16:44:40 +0200 Subject: [PATCH 15/74] Adds a tenant scanner --- pkg/objstore/client_mock.go | 176 ++++++++++++++++++ pkg/phlaredb/bucket/tenant_deletion_mark.go | 82 ++++++++ .../bucket/tenant_deletion_mark_test.go | 58 ++++++ pkg/phlaredb/bucket/tenant_scanner.go | 74 ++++++++ pkg/phlaredb/bucket/tenant_scanner_test.go | 67 +++++++ 5 files changed, 457 insertions(+) create mode 100644 pkg/objstore/client_mock.go create mode 100644 pkg/phlaredb/bucket/tenant_deletion_mark.go create mode 100644 pkg/phlaredb/bucket/tenant_deletion_mark_test.go create mode 100644 pkg/phlaredb/bucket/tenant_scanner.go create mode 100644 pkg/phlaredb/bucket/tenant_scanner_test.go diff --git a/pkg/objstore/client_mock.go b/pkg/objstore/client_mock.go new file mode 100644 index 0000000000..9b285938bf --- /dev/null +++ b/pkg/objstore/client_mock.go @@ -0,0 +1,176 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/cortexproject/cortex/blob/master/pkg/storage/bucket/client_mock.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Cortex Authors. + +package objstore + +import ( + "bytes" + "context" + "errors" + "io" + "time" + + "github.com/stretchr/testify/mock" + "github.com/thanos-io/objstore" +) + +// ErrObjectDoesNotExist is used in tests to simulate objstore.Bucket.IsObjNotFoundErr(). +var ErrObjectDoesNotExist = errors.New("object does not exist") + +// ClientMock mocks objstore.Bucket +type ClientMock struct { + mock.Mock +} + +// Upload mocks objstore.Bucket.Upload() +func (m *ClientMock) Upload(ctx context.Context, name string, r io.Reader) error { + args := m.Called(ctx, name, r) + return args.Error(0) +} + +func (m *ClientMock) MockUpload(name string, err error) { + m.On("Upload", mock.Anything, name, mock.Anything).Return(err) +} + +// Delete mocks objstore.Bucket.Delete() +func (m *ClientMock) Delete(ctx context.Context, name string) error { + args := m.Called(ctx, name) + return args.Error(0) +} + +// Name mocks objstore.Bucket.Name() +func (m *ClientMock) Name() string { + return "mock" +} + +// Iter mocks objstore.Bucket.Iter() +func (m *ClientMock) Iter(ctx context.Context, dir string, f func(string) error, options ...objstore.IterOption) error { + args := m.Called(ctx, dir, f, options) + return args.Error(0) +} + +// MockIter is a convenient method to mock Iter() +func (m *ClientMock) MockIter(prefix string, objects []string, err error) { + m.MockIterWithCallback(prefix, objects, err, nil) +} + +// MockIterWithCallback is a convenient method to mock Iter() and get a callback called when the Iter +// API is called. +func (m *ClientMock) MockIterWithCallback(prefix string, objects []string, err error, cb func()) { + m.On("Iter", mock.Anything, prefix, mock.Anything, mock.Anything).Return(err).Run(func(args mock.Arguments) { + if cb != nil { + cb() + } + + f := args.Get(2).(func(string) error) + + for _, o := range objects { + if f(o) != nil { + break + } + } + }) +} + +// Get mocks objstore.Bucket.Get() +func (m *ClientMock) Get(ctx context.Context, name string) (io.ReadCloser, error) { + args := m.Called(ctx, name) + + // Allow to mock the Get() with a function which is called each time. + if fn, ok := args.Get(0).(func(ctx context.Context, name string) (io.ReadCloser, error)); ok { + return fn(ctx, name) + } + + val, err := args.Get(0), args.Error(1) + if val == nil { + return nil, err + } + return val.(io.ReadCloser), err +} + +func (m *ClientMock) ReaderAt(ctx context.Context, name string) (ReaderAtCloser, error) { + args := m.Called(ctx, name) + + // Allow to mock the ReaderAt() with a function which is called each time. + if fn, ok := args.Get(0).(func(ctx context.Context, name string) (ReaderAtCloser, error)); ok { + return fn(ctx, name) + } + + val, err := args.Get(0), args.Error(1) + if val == nil { + return nil, err + } + return val.(ReaderAtCloser), err +} + +// MockGet is a convenient method to mock Get() and Exists() +func (m *ClientMock) MockGet(name, content string, err error) { + m.MockGetAndLastModified(name, content, time.Now(), err) +} + +func (m *ClientMock) MockGetAndLastModified(name, content string, lastModified time.Time, err error) { + if content != "" { + m.On("Exists", mock.Anything, name).Return(true, err) + m.On("Attributes", mock.Anything, name).Return(objstore.ObjectAttributes{ + Size: int64(len(content)), + LastModified: lastModified, + }, nil) + + // Since we return an ReadCloser and it can be consumed only once, + // each time the mocked Get() is called we do create a new one, so + // that getting the same mocked object twice works as expected. + m.On("Get", mock.Anything, name).Return(func(_ context.Context, _ string) (io.ReadCloser, error) { + return io.NopCloser(bytes.NewReader([]byte(content))), err + }) + } else { + m.On("Exists", mock.Anything, name).Return(false, err) + m.On("Get", mock.Anything, name).Return(nil, ErrObjectDoesNotExist) + m.On("Attributes", mock.Anything, name).Return(nil, ErrObjectDoesNotExist) + } +} + +func (m *ClientMock) MockAttributes(name string, attrs objstore.ObjectAttributes, err error) { + m.On("Attributes", mock.Anything, name).Return(attrs, err) +} + +func (m *ClientMock) MockDelete(name string, err error) { + m.On("Delete", mock.Anything, name).Return(err) +} + +func (m *ClientMock) MockExists(name string, exists bool, err error) { + m.On("Exists", mock.Anything, name).Return(exists, err) +} + +// GetRange mocks objstore.Bucket.GetRange() +func (m *ClientMock) GetRange(ctx context.Context, name string, off, length int64) (io.ReadCloser, error) { + args := m.Called(ctx, name, off, length) + return args.Get(0).(io.ReadCloser), args.Error(1) +} + +// Exists mocks objstore.Bucket.Exists() +func (m *ClientMock) Exists(ctx context.Context, name string) (bool, error) { + args := m.Called(ctx, name) + return args.Bool(0), args.Error(1) +} + +// IsObjNotFoundErr mocks objstore.Bucket.IsObjNotFoundErr() +func (m *ClientMock) IsObjNotFoundErr(err error) bool { + return errors.Is(err, ErrObjectDoesNotExist) +} + +func (m *ClientMock) IsCustomerManagedKeyError(_ error) bool { + return false +} + +// ObjectSize mocks objstore.Bucket.Attributes() +func (m *ClientMock) Attributes(ctx context.Context, name string) (objstore.ObjectAttributes, error) { + args := m.Called(ctx, name) + return args.Get(0).(objstore.ObjectAttributes), args.Error(1) +} + +// Close mocks objstore.Bucket.Close() +func (m *ClientMock) Close() error { + return nil +} diff --git a/pkg/phlaredb/bucket/tenant_deletion_mark.go b/pkg/phlaredb/bucket/tenant_deletion_mark.go new file mode 100644 index 0000000000..635b74cf15 --- /dev/null +++ b/pkg/phlaredb/bucket/tenant_deletion_mark.go @@ -0,0 +1,82 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/cortexproject/cortex/blob/master/pkg/storage/tsdb/tenant_deletion_mark.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Cortex Authors. + +package bucket + +import ( + "bytes" + "context" + "encoding/json" + "path" + "time" + + "github.com/go-kit/log/level" + "github.com/pkg/errors" + + "github.com/grafana/pyroscope/pkg/objstore" + util_log "github.com/grafana/pyroscope/pkg/util" +) + +// Relative to user-specific prefix. +const TenantDeletionMarkPath = "markers/tenant-deletion-mark.json" + +type TenantDeletionMark struct { + // Unix timestamp when deletion marker was created. + DeletionTime int64 `json:"deletion_time"` + + // Unix timestamp when cleanup was finished. + FinishedTime int64 `json:"finished_time,omitempty"` +} + +func NewTenantDeletionMark(deletionTime time.Time) *TenantDeletionMark { + return &TenantDeletionMark{DeletionTime: deletionTime.Unix()} +} + +// Checks for deletion mark for tenant. Errors other than "object not found" are returned. +func TenantDeletionMarkExists(ctx context.Context, bkt objstore.BucketReader, userID string) (bool, error) { + markerFile := path.Join(userID, TenantDeletionMarkPath) + + return bkt.Exists(ctx, markerFile) +} + +// Uploads deletion mark to the tenant location in the bucket. +func WriteTenantDeletionMark(ctx context.Context, bkt objstore.Bucket, userID string, cfgProvider objstore.TenantConfigProvider, mark *TenantDeletionMark) error { + bkt = objstore.NewUserBucketClient(userID, bkt, cfgProvider) + + data, err := json.Marshal(mark) + if err != nil { + return errors.Wrap(err, "serialize tenant deletion mark") + } + + return errors.Wrap(bkt.Upload(ctx, TenantDeletionMarkPath, bytes.NewReader(data)), "upload tenant deletion mark") +} + +// Returns tenant deletion mark for given user, if it exists. If it doesn't exist, returns nil mark, and no error. +func ReadTenantDeletionMark(ctx context.Context, bkt objstore.BucketReader, userID string) (*TenantDeletionMark, error) { + markerFile := path.Join(userID, TenantDeletionMarkPath) + + r, err := bkt.Get(ctx, markerFile) + if err != nil { + if bkt.IsObjNotFoundErr(err) { + return nil, nil + } + + return nil, errors.Wrapf(err, "failed to read deletion mark object: %s", markerFile) + } + + mark := &TenantDeletionMark{} + err = json.NewDecoder(r).Decode(mark) + + // Close reader before dealing with decode error. + if closeErr := r.Close(); closeErr != nil { + level.Warn(util_log.Logger).Log("msg", "failed to close bucket reader", "err", closeErr) + } + + if err != nil { + return nil, errors.Wrapf(err, "failed to decode deletion mark object: %s", markerFile) + } + + return mark, nil +} diff --git a/pkg/phlaredb/bucket/tenant_deletion_mark_test.go b/pkg/phlaredb/bucket/tenant_deletion_mark_test.go new file mode 100644 index 0000000000..5ee5d35574 --- /dev/null +++ b/pkg/phlaredb/bucket/tenant_deletion_mark_test.go @@ -0,0 +1,58 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/cortexproject/cortex/blob/master/pkg/storage/tsdb/tenant_deletion_mark_test.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Cortex Authors. + +package bucket + +import ( + "bytes" + "context" + "testing" + + "github.com/stretchr/testify/require" + "github.com/thanos-io/objstore" + + phlareobj "github.com/grafana/pyroscope/pkg/objstore" +) + +func TestTenantDeletionMarkExists(t *testing.T) { + const username = "user" + + for name, tc := range map[string]struct { + objects map[string][]byte + exists bool + }{ + "empty": { + objects: nil, + exists: false, + }, + + "mark doesn't exist": { + objects: map[string][]byte{ + "user/01EQK4QKFHVSZYVJ908Y7HH9E0/meta.json": []byte("data"), + }, + exists: false, + }, + + "mark exists": { + objects: map[string][]byte{ + "user/01EQK4QKFHVSZYVJ908Y7HH9E0/meta.json": []byte("data"), + "user/" + TenantDeletionMarkPath: []byte("data"), + }, + exists: true, + }, + } { + t.Run(name, func(t *testing.T) { + bkt := objstore.NewInMemBucket() + // "upload" objects + for objName, data := range tc.objects { + require.NoError(t, bkt.Upload(context.Background(), objName, bytes.NewReader(data))) + } + + res, err := TenantDeletionMarkExists(context.Background(), phlareobj.NewBucket(bkt), username) + require.NoError(t, err) + require.Equal(t, tc.exists, res) + }) + } +} diff --git a/pkg/phlaredb/bucket/tenant_scanner.go b/pkg/phlaredb/bucket/tenant_scanner.go new file mode 100644 index 0000000000..65e17927f2 --- /dev/null +++ b/pkg/phlaredb/bucket/tenant_scanner.go @@ -0,0 +1,74 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/cortexproject/cortex/blob/master/pkg/storage/tsdb/users_scanner.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Cortex Authors. + +package bucket + +import ( + "context" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + + "github.com/grafana/pyroscope/pkg/objstore" +) + +// AllTenants returns true to each call and should be used whenever the UsersScanner should not filter out +// any user due to sharding. +func AllTenants(_ string) (bool, error) { + return true, nil +} + +type TenantsScanner struct { + bucketClient objstore.Bucket + logger log.Logger + isOwned func(userID string) (bool, error) +} + +func NewTenantsScanner(bucketClient objstore.Bucket, isOwned func(userID string) (bool, error), logger log.Logger) *TenantsScanner { + return &TenantsScanner{ + bucketClient: bucketClient, + logger: logger, + isOwned: isOwned, + } +} + +// ScanTenants returns a fresh list of users found in the storage, that are not marked for deletion, +// and list of users marked for deletion. +// +// If sharding is enabled, returned lists contains only the users owned by this instance. +func (s *TenantsScanner) ScanTenants(ctx context.Context) (users, markedForDeletion []string, err error) { + users, err = ListUsers(ctx, s.bucketClient) + if err != nil { + return nil, nil, err + } + + // Check users for being owned by instance, and split users into non-deleted and deleted. + // We do these checks after listing all users, to improve cacheability of Iter (result is only cached at the end of Iter call). + for ix := 0; ix < len(users); { + userID := users[ix] + + // Check if it's owned by this instance. + owned, err := s.isOwned(userID) + if err != nil { + level.Warn(s.logger).Log("msg", "unable to check if user is owned by this shard", "user", userID, "err", err) + } else if !owned { + users = append(users[:ix], users[ix+1:]...) + continue + } + + deletionMarkExists, err := TenantDeletionMarkExists(ctx, s.bucketClient, userID) + if err != nil { + level.Warn(s.logger).Log("msg", "unable to check if user is marked for deletion", "user", userID, "err", err) + } else if deletionMarkExists { + users = append(users[:ix], users[ix+1:]...) + markedForDeletion = append(markedForDeletion, userID) + continue + } + + ix++ + } + + return users, markedForDeletion, nil +} diff --git a/pkg/phlaredb/bucket/tenant_scanner_test.go b/pkg/phlaredb/bucket/tenant_scanner_test.go new file mode 100644 index 0000000000..5c5e61e1c4 --- /dev/null +++ b/pkg/phlaredb/bucket/tenant_scanner_test.go @@ -0,0 +1,67 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/cortexproject/cortex/blob/master/pkg/storage/tsdb/users_scanner_test.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Cortex Authors. + +package bucket + +import ( + "context" + "errors" + "path" + "testing" + + "github.com/go-kit/log" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + + "github.com/grafana/pyroscope/pkg/objstore" +) + +func TestUsersScanner_ScanUsers_ShouldReturnedOwnedUsersOnly(t *testing.T) { + bucketClient := &objstore.ClientMock{} + bucketClient.MockIter("", []string{"user-1", "user-2", "user-3", "user-4"}, nil) + bucketClient.MockExists(path.Join("user-1", TenantDeletionMarkPath), false, nil) + bucketClient.MockExists(path.Join("user-3", TenantDeletionMarkPath), true, nil) + + isOwned := func(userID string) (bool, error) { + return userID == "user-1" || userID == "user-3", nil + } + + s := NewTenantsScanner(bucketClient, isOwned, log.NewNopLogger()) + actual, deleted, err := s.ScanTenants(context.Background()) + require.NoError(t, err) + assert.Equal(t, []string{"user-1"}, actual) + assert.Equal(t, []string{"user-3"}, deleted) +} + +func TestUsersScanner_ScanUsers_ShouldReturnUsersForWhichOwnerCheckOrTenantDeletionCheckFailed(t *testing.T) { + expected := []string{"user-1", "user-2"} + + bucketClient := &objstore.ClientMock{} + bucketClient.MockIter("", expected, nil) + bucketClient.MockExists(path.Join("user-1", TenantDeletionMarkPath), false, nil) + bucketClient.MockExists(path.Join("user-2", TenantDeletionMarkPath), false, errors.New("fail")) + + isOwned := func(userID string) (bool, error) { + return false, errors.New("failed to check if user is owned") + } + + s := NewTenantsScanner(bucketClient, isOwned, log.NewNopLogger()) + actual, deleted, err := s.ScanTenants(context.Background()) + require.NoError(t, err) + assert.Equal(t, expected, actual) + assert.Empty(t, deleted) +} + +func TestUsersScanner_ScanUsers_ShouldNotReturnPrefixedUsedByMimirInternals(t *testing.T) { + bucketClient := &objstore.ClientMock{} + bucketClient.MockIter("", []string{"user-1", "user-2", PyroscopeInternalsPrefix}, nil) + bucketClient.MockExists(path.Join("user-1", TenantDeletionMarkPath), false, nil) + bucketClient.MockExists(path.Join("user-2", TenantDeletionMarkPath), false, nil) + + s := NewTenantsScanner(bucketClient, AllTenants, log.NewNopLogger()) + actual, _, err := s.ScanTenants(context.Background()) + require.NoError(t, err) + assert.Equal(t, []string{"user-1", "user-2"}, actual) +} From 26979c4df718e1c534d8b6802aa8ae6d93821499 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Mon, 11 Sep 2023 08:36:05 +0200 Subject: [PATCH 16/74] Working on the blocks cleaner --- pkg/compactor/blocks_cleaner.go | 64 ++++++++++++++-------------- pkg/compactor/blocks_cleaner_test.go | 2 +- pkg/compactor/compactor.go | 5 ++- 3 files changed, 36 insertions(+), 35 deletions(-) diff --git a/pkg/compactor/blocks_cleaner.go b/pkg/compactor/blocks_cleaner.go index 806a5e4170..8556cecafc 100644 --- a/pkg/compactor/blocks_cleaner.go +++ b/pkg/compactor/blocks_cleaner.go @@ -21,15 +21,15 @@ import ( "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" - "github.com/thanos-io/objstore" + thanos_objstore "github.com/thanos-io/objstore" - "github.com/grafana/mimir/pkg/storage/bucket" - mimir_tsdb "github.com/grafana/mimir/pkg/storage/tsdb" - "github.com/grafana/mimir/pkg/storage/tsdb/block" "github.com/grafana/mimir/pkg/storage/tsdb/bucketindex" - "github.com/grafana/mimir/pkg/util" - util_log "github.com/grafana/mimir/pkg/util/log" - "github.com/grafana/mimir/pkg/util/validation" + "github.com/grafana/pyroscope/pkg/objstore" + "github.com/grafana/pyroscope/pkg/phlaredb/block" + "github.com/grafana/pyroscope/pkg/phlaredb/bucket" + "github.com/grafana/pyroscope/pkg/util" + util_log "github.com/grafana/pyroscope/pkg/util" + "github.com/grafana/pyroscope/pkg/util/validation" ) const ( @@ -48,13 +48,13 @@ type BlocksCleanerConfig struct { type BlocksCleaner struct { services.Service - cfg BlocksCleanerConfig - cfgProvider ConfigProvider - logger log.Logger - bucketClient objstore.Bucket - usersScanner *mimir_tsdb.UsersScanner - ownUser func(userID string) (bool, error) - singleFlight *concurrency.LimitedConcurrencySingleFlight + cfg BlocksCleanerConfig + cfgProvider ConfigProvider + logger log.Logger + bucketClient objstore.Bucket + tenantsScanner *bucket.TenantsScanner + ownUser func(userID string) (bool, error) + singleFlight *concurrency.LimitedConcurrencySingleFlight // Keep track of the last owned users. lastOwnedUsers []string @@ -76,13 +76,13 @@ type BlocksCleaner struct { func NewBlocksCleaner(cfg BlocksCleanerConfig, bucketClient objstore.Bucket, ownUser func(userID string) (bool, error), cfgProvider ConfigProvider, logger log.Logger, reg prometheus.Registerer) *BlocksCleaner { c := &BlocksCleaner{ - cfg: cfg, - bucketClient: bucketClient, - usersScanner: mimir_tsdb.NewUsersScanner(bucketClient, ownUser, logger), - ownUser: ownUser, - cfgProvider: cfgProvider, - singleFlight: concurrency.NewLimitedConcurrencySingleFlight(cfg.CleanupConcurrency), - logger: log.With(logger, "component", "cleaner"), + cfg: cfg, + bucketClient: bucketClient, + tenantsScanner: bucket.NewTenantsScanner(bucketClient, ownUser, logger), + ownUser: ownUser, + cfgProvider: cfgProvider, + singleFlight: concurrency.NewLimitedConcurrencySingleFlight(cfg.CleanupConcurrency), + logger: log.With(logger, "component", "cleaner"), runsStarted: promauto.With(reg).NewCounter(prometheus.CounterOpts{ Name: "cortex_compactor_block_cleanup_started_total", Help: "Total number of blocks cleanup runs started.", @@ -213,7 +213,7 @@ func (c *BlocksCleaner) instrumentFinishedCleanupRun(err error, logger log.Logge // refreshOwnedUsers is not required to be concurrency safe, but a single instance of this function // could run concurrently with the cleanup job for any tenant. func (c *BlocksCleaner) refreshOwnedUsers(ctx context.Context) ([]string, map[string]bool, error) { - users, deleted, err := c.usersScanner.ScanUsers(ctx) + users, deleted, err := c.tenantsScanner.ScanTenants(ctx) if err != nil { return nil, nil, errors.Wrap(err, "failed to discover users from bucket") } @@ -264,14 +264,14 @@ func (c *BlocksCleaner) deleteRemainingData(ctx context.Context, userBucket objs level.Info(userLogger).Log("msg", "deleted bucket index for tenant with no blocks remaining") // Delete markers folder - if deleted, err := bucket.DeletePrefix(ctx, userBucket, block.MarkersPathname, userLogger); err != nil { + if deleted, err := objstore.DeletePrefix(ctx, userBucket, block.MarkersPathname, userLogger); err != nil { return errors.Wrap(err, "failed to delete marker files") } else if deleted > 0 { level.Info(userLogger).Log("msg", "deleted marker files for tenant with no blocks remaining", "count", deleted) } // Delete debug folder - if deleted, err := bucket.DeletePrefix(ctx, userBucket, block.DebugMetas, userLogger); err != nil { + if deleted, err := objstore.DeletePrefix(ctx, userBucket, block.DebugMetas, userLogger); err != nil { return errors.Wrap(err, "failed to delete "+block.DebugMetas) } else if deleted > 0 { level.Info(userLogger).Log("msg", "deleted files under "+block.DebugMetas+" for tenant with no blocks remaining", "count", deleted) @@ -282,7 +282,7 @@ func (c *BlocksCleaner) deleteRemainingData(ctx context.Context, userBucket objs // deleteUserMarkedForDeletion removes blocks and remaining data for tenant marked for deletion. func (c *BlocksCleaner) deleteUserMarkedForDeletion(ctx context.Context, userID string, userLogger log.Logger) error { - userBucket := bucket.NewUserBucketClient(userID, c.bucketClient, c.cfgProvider) + userBucket := objstore.NewUserBucketClient(userID, c.bucketClient, c.cfgProvider) level.Info(userLogger).Log("msg", "deleting blocks for tenant marked for deletion") @@ -341,7 +341,7 @@ func (c *BlocksCleaner) deleteUserMarkedForDeletion(ctx context.Context, userID level.Info(userLogger).Log("msg", "deleted blocks for tenant marked for deletion", "deletedBlocks", deletedBlocks) } - mark, err := mimir_tsdb.ReadTenantDeletionMark(ctx, c.bucketClient, userID) + mark, err := bucket.ReadTenantDeletionMark(ctx, c.bucketClient, userID) if err != nil { return errors.Wrap(err, "failed to read tenant deletion mark") } @@ -355,7 +355,7 @@ func (c *BlocksCleaner) deleteUserMarkedForDeletion(ctx context.Context, userID if deletedBlocks > 0 || mark.FinishedTime == 0 { level.Info(userLogger).Log("msg", "updating finished time in tenant deletion mark") mark.FinishedTime = time.Now().Unix() - return errors.Wrap(mimir_tsdb.WriteTenantDeletionMark(ctx, c.bucketClient, userID, c.cfgProvider, mark), "failed to update tenant deletion mark") + return errors.Wrap(bucket.WriteTenantDeletionMark(ctx, c.bucketClient, userID, c.cfgProvider, mark), "failed to update tenant deletion mark") } if time.Since(time.Unix(mark.FinishedTime, 0)) < c.cfg.TenantCleanupDelay { @@ -365,14 +365,14 @@ func (c *BlocksCleaner) deleteUserMarkedForDeletion(ctx context.Context, userID level.Info(userLogger).Log("msg", "cleaning up remaining blocks data for tenant marked for deletion") // Let's do final cleanup of tenant. - if deleted, err := bucket.DeletePrefix(ctx, userBucket, block.DebugMetas, userLogger); err != nil { + if deleted, err := objstore.DeletePrefix(ctx, userBucket, block.DebugMetas, userLogger); err != nil { return errors.Wrap(err, "failed to delete "+block.DebugMetas) } else if deleted > 0 { level.Info(userLogger).Log("msg", "deleted files under "+block.DebugMetas+" for tenant marked for deletion", "count", deleted) } // Tenant deletion mark file is inside Markers as well. - if deleted, err := bucket.DeletePrefix(ctx, userBucket, block.MarkersPathname, userLogger); err != nil { + if deleted, err := objstore.DeletePrefix(ctx, userBucket, block.MarkersPathname, userLogger); err != nil { return errors.Wrap(err, "failed to delete marker files") } else if deleted > 0 { level.Info(userLogger).Log("msg", "deleted marker files for tenant marked for deletion", "count", deleted) @@ -382,7 +382,7 @@ func (c *BlocksCleaner) deleteUserMarkedForDeletion(ctx context.Context, userID } func (c *BlocksCleaner) cleanUser(ctx context.Context, userID string, userLogger log.Logger) (returnErr error) { - userBucket := bucket.NewUserBucketClient(userID, c.bucketClient, c.cfgProvider) + userBucket := objstore.NewUserBucketClient(userID, c.bucketClient, c.cfgProvider) startTime := time.Now() level.Info(userLogger).Log("msg", "started blocks cleanup and maintenance") @@ -616,7 +616,7 @@ func stalePartialBlockLastModifiedTime(ctx context.Context, blockID ulid.ULID, u err := userBucket.WithExpectedErrs(func(err error) bool { return errors.Is(err, errStopIter) // sentinel error }).Iter(ctx, blockID.String(), func(name string) error { - if strings.HasSuffix(name, objstore.DirDelim) { + if strings.HasSuffix(name, thanos_objstore.DirDelim) { return nil } attrib, err := userBucket.Attributes(ctx, name) @@ -630,7 +630,7 @@ func stalePartialBlockLastModifiedTime(ctx context.Context, blockID ulid.ULID, u lastModified = attrib.LastModified } return nil - }, objstore.WithRecursiveIter) + }, thanos_objstore.WithRecursiveIter) if errors.Is(err, errStopIter) { return time.Time{}, nil diff --git a/pkg/compactor/blocks_cleaner_test.go b/pkg/compactor/blocks_cleaner_test.go index 728edd5558..193e28e101 100644 --- a/pkg/compactor/blocks_cleaner_test.go +++ b/pkg/compactor/blocks_cleaner_test.go @@ -378,7 +378,7 @@ func TestBlocksCleaner_ShouldRemoveMetricsForTenantsNotBelongingAnymoreToTheShar )) // Override the users scanner to reconfigure it to only return a subset of users. - cleaner.usersScanner = tsdb.NewUsersScanner(bucketClient, func(userID string) (bool, error) { return userID == "user-1", nil }, logger) + cleaner.tenantsScanner = tsdb.NewUsersScanner(bucketClient, func(userID string) (bool, error) { return userID == "user-1", nil }, logger) // Create new blocks, to double check expected metrics have changed. createTSDBBlock(t, bucketClient, "user-1", 40, 50, 2, nil) diff --git a/pkg/compactor/compactor.go b/pkg/compactor/compactor.go index 339c4e6316..b7bf6a123f 100644 --- a/pkg/compactor/compactor.go +++ b/pkg/compactor/compactor.go @@ -26,7 +26,6 @@ import ( "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" - "github.com/thanos-io/objstore" "go.uber.org/atomic" "github.com/grafana/mimir/pkg/storage/bucket" @@ -34,6 +33,8 @@ import ( "github.com/grafana/mimir/pkg/storage/tsdb/block" "github.com/grafana/mimir/pkg/util" util_log "github.com/grafana/mimir/pkg/util/log" + + "github.com/grafana/pyroscope/pkg/objstore" ) const ( @@ -184,7 +185,7 @@ func (cfg *Config) Validate() error { // ConfigProvider defines the per-tenant config provider for the MultitenantCompactor. type ConfigProvider interface { - bucket.TenantConfigProvider + objstore.TenantConfigProvider // CompactorBlocksRetentionPeriod returns the retention period for a given user. CompactorBlocksRetentionPeriod(user string) time.Duration From 56187ba6571fe1620e5f470f70797a85d9a3b18c Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Mon, 11 Sep 2023 11:15:40 +0200 Subject: [PATCH 17/74] Adds bucketindex package --- pkg/phlaredb/block/testutil/mock_block.go | 95 ++++ pkg/phlaredb/bucketindex/index.go | 189 +++++++ pkg/phlaredb/bucketindex/index_test.go | 133 +++++ pkg/phlaredb/bucketindex/loader.go | 273 ++++++++++ pkg/phlaredb/bucketindex/loader_test.go | 587 ++++++++++++++++++++++ pkg/phlaredb/bucketindex/storage.go | 103 ++++ pkg/phlaredb/bucketindex/storage_test.go | 122 +++++ pkg/phlaredb/bucketindex/updater.go | 233 +++++++++ pkg/phlaredb/bucketindex/updater_test.go | 267 ++++++++++ 9 files changed, 2002 insertions(+) create mode 100644 pkg/phlaredb/block/testutil/mock_block.go create mode 100644 pkg/phlaredb/bucketindex/index.go create mode 100644 pkg/phlaredb/bucketindex/index_test.go create mode 100644 pkg/phlaredb/bucketindex/loader.go create mode 100644 pkg/phlaredb/bucketindex/loader_test.go create mode 100644 pkg/phlaredb/bucketindex/storage.go create mode 100644 pkg/phlaredb/bucketindex/storage_test.go create mode 100644 pkg/phlaredb/bucketindex/updater.go create mode 100644 pkg/phlaredb/bucketindex/updater_test.go diff --git a/pkg/phlaredb/block/testutil/mock_block.go b/pkg/phlaredb/block/testutil/mock_block.go new file mode 100644 index 0000000000..3919ad5c6e --- /dev/null +++ b/pkg/phlaredb/block/testutil/mock_block.go @@ -0,0 +1,95 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/cortexproject/cortex/blob/master/pkg/storage/tsdb/testutil/block_mock.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Cortex Authors. + +package testutil + +import ( + "context" + "crypto/rand" + "encoding/json" + "fmt" + "strings" + "testing" + "time" + + "github.com/oklog/ulid" + "github.com/prometheus/common/model" + "github.com/prometheus/prometheus/tsdb" + "github.com/stretchr/testify/require" + + "github.com/grafana/pyroscope/pkg/objstore" + "github.com/grafana/pyroscope/pkg/phlaredb/block" +) + +func MockStorageBlock(t testing.TB, bucket objstore.Bucket, userID string, minT, maxT model.Time) block.Meta { + return MockStorageBlockWithExtLabels(t, bucket, userID, minT, maxT, nil) +} + +func MockStorageBlockWithExtLabels(t testing.TB, bucket objstore.Bucket, userID string, minT, maxT model.Time, externalLabels map[string]string) block.Meta { + // Generate a block ID whose timestamp matches the maxT (for simplicity we assume it + // has been compacted and shipped in zero time, even if not realistic). + id := ulid.MustNew(uint64(maxT), rand.Reader) + + meta := block.Meta{ + Version: 1, + ULID: id, + MinTime: minT, + MaxTime: maxT, + Compaction: tsdb.BlockMetaCompaction{ + Level: 1, + Sources: []ulid.ULID{id}, + }, + Labels: externalLabels, + } + + metaContent, err := json.Marshal(meta) + require.NoError(t, err, "failed to marshal mocked block meta") + + metaContentReader := strings.NewReader(string(metaContent)) + metaPath := fmt.Sprintf("%s/%s/meta.json", userID, id.String()) + require.NoError(t, bucket.Upload(context.Background(), metaPath, metaContentReader)) + + // Upload an empty index, just to make sure the meta.json is not the only object in the block location. + indexPath := fmt.Sprintf("%s/%s/index", userID, id.String()) + require.NoError(t, bucket.Upload(context.Background(), indexPath, strings.NewReader(""))) + + return meta +} + +func MockStorageDeletionMark(t testing.TB, bucket objstore.Bucket, userID string, meta block.Meta) *block.DeletionMark { + mark := block.DeletionMark{ + ID: meta.ULID, + DeletionTime: time.Now().Add(-time.Minute).Unix(), + Version: block.DeletionMarkVersion1, + } + + markContent, err := json.Marshal(mark) + require.NoError(t, err, "failed to marshal mocked deletion mark") + + markContentReader := strings.NewReader(string(markContent)) + markPath := fmt.Sprintf("%s/%s/%s", userID, meta.ULID.String(), block.DeletionMarkFilename) + require.NoError(t, bucket.Upload(context.Background(), markPath, markContentReader)) + + return &mark +} + +func MockNoCompactMark(t testing.TB, bucket objstore.Bucket, userID string, meta block.Meta) *block.NoCompactMark { + mark := block.NoCompactMark{ + ID: meta.ULID, + NoCompactTime: time.Now().Unix(), + Version: block.DeletionMarkVersion1, + Details: "details", + Reason: block.ManualNoCompactReason, + } + + markContent, err := json.Marshal(mark) + require.NoError(t, err, "failed to marshal mocked no-compact mark") + + markContentReader := strings.NewReader(string(markContent)) + markPath := fmt.Sprintf("%s/%s/%s", userID, meta.ULID.String(), block.NoCompactMarkFilename) + require.NoError(t, bucket.Upload(context.Background(), markPath, markContentReader)) + + return &mark +} diff --git a/pkg/phlaredb/bucketindex/index.go b/pkg/phlaredb/bucketindex/index.go new file mode 100644 index 0000000000..57a40f3dd8 --- /dev/null +++ b/pkg/phlaredb/bucketindex/index.go @@ -0,0 +1,189 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/cortexproject/cortex/blob/master/pkg/storage/tsdb/bucketindex/index.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Cortex Authors. + +package bucketindex + +import ( + "fmt" + "strings" + "time" + + "github.com/oklog/ulid" + "github.com/prometheus/common/model" + + "github.com/grafana/pyroscope/pkg/phlaredb/block" + "github.com/grafana/pyroscope/pkg/phlaredb/sharding" + "github.com/grafana/pyroscope/pkg/util" +) + +const ( + IndexFilename = "bucket-index.json" + IndexCompressedFilename = IndexFilename + ".gz" + IndexVersion1 = 1 + IndexVersion2 = 2 // Added CompactorShardID field. + SegmentsFormatUnknown = "" + + // SegmentsFormat1Based6Digits defined segments numbered with 6 digits numbers in a sequence starting from number 1 + // eg. (000001, 000002, 000003). + SegmentsFormat1Based6Digits = "1b6d" +) + +// Index contains all known blocks and markers of a tenant. +type Index struct { + // Version of the index format. + Version int `json:"version"` + + // List of complete blocks (partial blocks are excluded from the index). + Blocks Blocks `json:"blocks"` + + // List of block deletion marks. + BlockDeletionMarks BlockDeletionMarks `json:"block_deletion_marks"` + + // UpdatedAt is a unix timestamp (seconds precision) of when the index has been updated + // (written in the storage) the last time. + UpdatedAt int64 `json:"updated_at"` +} + +func (idx *Index) GetUpdatedAt() time.Time { + return time.Unix(idx.UpdatedAt, 0) +} + +// RemoveBlock removes block and its deletion mark (if any) from index. +func (idx *Index) RemoveBlock(id ulid.ULID) { + for i := 0; i < len(idx.Blocks); i++ { + if idx.Blocks[i].ID == id { + idx.Blocks = append(idx.Blocks[:i], idx.Blocks[i+1:]...) + break + } + } + + for i := 0; i < len(idx.BlockDeletionMarks); i++ { + if idx.BlockDeletionMarks[i].ID == id { + idx.BlockDeletionMarks = append(idx.BlockDeletionMarks[:i], idx.BlockDeletionMarks[i+1:]...) + break + } + } +} + +// Block holds the information about a block in the index. +type Block struct { + // Block ID. + ID ulid.ULID `json:"block_id"` + + // MinTime and MaxTime specify the time range all samples in the block are in (millis precision). + MinTime model.Time `json:"min_time"` + MaxTime model.Time `json:"max_time"` + + // UploadedAt is a unix timestamp (seconds precision) of when the block has been completed to be uploaded + // to the storage. + UploadedAt int64 `json:"uploaded_at"` + + // Block's compactor shard ID, copied from tsdb.CompactorShardIDExternalLabel label. + CompactorShardID string `json:"compactor_shard_id,omitempty"` +} + +// Within returns whether the block contains samples within the provided range. +// Input minT and maxT are both inclusive. +func (m *Block) Within(minT, maxT model.Time) bool { + return block.InRange(m.MinTime, m.MaxTime, minT, maxT) +} + +func (m *Block) GetUploadedAt() time.Time { + return time.Unix(m.UploadedAt, 0) +} + +func (m *Block) String() string { + minT := util.TimeFromMillis(int64(m.MinTime)).UTC() + maxT := util.TimeFromMillis(int64(m.MaxTime)).UTC() + + shard := m.CompactorShardID + if shard == "" { + shard = "none" + } + + return fmt.Sprintf("%s (min time: %s max time: %s, compactor shard: %s)", m.ID, minT.String(), maxT.String(), shard) +} + +func BlockFromMeta(meta block.Meta) *Block { + return &Block{ + ID: meta.ULID, + MinTime: meta.MinTime, + MaxTime: meta.MaxTime, + CompactorShardID: meta.Labels[sharding.CompactorShardIDLabel], + } +} + +// BlockDeletionMark holds the information about a block's deletion mark in the index. +type BlockDeletionMark struct { + // Block ID. + ID ulid.ULID `json:"block_id"` + + // DeletionTime is a unix timestamp (seconds precision) of when the block was marked to be deleted. + DeletionTime int64 `json:"deletion_time"` +} + +func (m *BlockDeletionMark) GetDeletionTime() time.Time { + return time.Unix(m.DeletionTime, 0) +} + +// BlockDeletionMark returns the block deletion mark. +func (m *BlockDeletionMark) BlockDeletionMark() *block.DeletionMark { + return &block.DeletionMark{ + ID: m.ID, + Version: block.DeletionMarkVersion1, + DeletionTime: m.DeletionTime, + } +} + +func DeletionMarkFromBlockMarker(mark *block.DeletionMark) *BlockDeletionMark { + return &BlockDeletionMark{ + ID: mark.ID, + DeletionTime: mark.DeletionTime, + } +} + +// BlockDeletionMarks holds a set of block deletion marks in the index. No ordering guaranteed. +type BlockDeletionMarks []*BlockDeletionMark + +func (s BlockDeletionMarks) GetULIDs() []ulid.ULID { + ids := make([]ulid.ULID, len(s)) + for i, m := range s { + ids[i] = m.ID + } + return ids +} + +func (s BlockDeletionMarks) Clone() BlockDeletionMarks { + clone := make(BlockDeletionMarks, len(s)) + for i, m := range s { + v := *m + clone[i] = &v + } + return clone +} + +// Blocks holds a set of blocks in the index. No ordering guaranteed. +type Blocks []*Block + +func (s Blocks) GetULIDs() []ulid.ULID { + ids := make([]ulid.ULID, len(s)) + for i, m := range s { + ids[i] = m.ID + } + return ids +} + +func (s Blocks) String() string { + b := strings.Builder{} + + for idx, m := range s { + if idx > 0 { + b.WriteString(", ") + } + b.WriteString(m.String()) + } + + return b.String() +} diff --git a/pkg/phlaredb/bucketindex/index_test.go b/pkg/phlaredb/bucketindex/index_test.go new file mode 100644 index 0000000000..1884ca7527 --- /dev/null +++ b/pkg/phlaredb/bucketindex/index_test.go @@ -0,0 +1,133 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/cortexproject/cortex/blob/master/pkg/storage/tsdb/bucketindex/index_test.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Cortex Authors. + +package bucketindex + +import ( + "testing" + + "github.com/oklog/ulid" + "github.com/prometheus/common/model" + "github.com/stretchr/testify/assert" + + "github.com/grafana/pyroscope/pkg/phlaredb/block" + "github.com/grafana/pyroscope/pkg/phlaredb/sharding" +) + +func TestIndex_RemoveBlock(t *testing.T) { + block1 := ulid.MustNew(1, nil) + block2 := ulid.MustNew(2, nil) + block3 := ulid.MustNew(3, nil) + idx := &Index{ + Blocks: Blocks{{ID: block1}, {ID: block2}, {ID: block3}}, + BlockDeletionMarks: BlockDeletionMarks{{ID: block2}, {ID: block3}}, + } + + idx.RemoveBlock(block2) + assert.ElementsMatch(t, []ulid.ULID{block1, block3}, idx.Blocks.GetULIDs()) + assert.ElementsMatch(t, []ulid.ULID{block3}, idx.BlockDeletionMarks.GetULIDs()) +} + +func TestBlockFromMeta(t *testing.T) { + blockID := ulid.MustNew(1, nil) + + tests := map[string]struct { + meta block.Meta + expected Block + }{ + "meta.json": { + meta: block.Meta{ + ULID: blockID, + MinTime: model.Time(10), + MaxTime: model.Time(20), + Labels: map[string]string{ + sharding.CompactorShardIDLabel: "1_of_8", + }, + }, + expected: Block{ + ID: blockID, + MinTime: model.Time(10), + MaxTime: model.Time(20), + CompactorShardID: "1_of_8", + }, + }, + } + + for testName, testData := range tests { + t.Run(testName, func(t *testing.T) { + assert.Equal(t, testData.expected, *BlockFromMeta(testData.meta)) + }) + } +} + +func TestBlock_Within(t *testing.T) { + tests := []struct { + block *Block + minT int64 + maxT int64 + expected bool + }{ + { + block: &Block{MinTime: 10, MaxTime: 20}, + minT: 5, + maxT: 9, + expected: false, + }, { + block: &Block{MinTime: 10, MaxTime: 20}, + minT: 5, + maxT: 10, + expected: true, + }, { + block: &Block{MinTime: 10, MaxTime: 20}, + minT: 5, + maxT: 10, + expected: true, + }, { + block: &Block{MinTime: 10, MaxTime: 20}, + minT: 11, + maxT: 13, + expected: true, + }, { + block: &Block{MinTime: 10, MaxTime: 20}, + minT: 19, + maxT: 21, + expected: true, + }, { + block: &Block{MinTime: 10, MaxTime: 20}, + minT: 20, + maxT: 21, + expected: true, + }, + } + + for _, tc := range tests { + assert.Equal(t, tc.expected, tc.block.Within(model.Time(tc.minT), model.Time(tc.maxT))) + } +} + +func TestBlockDeletionMark_DeletionMark(t *testing.T) { + block1 := ulid.MustNew(1, nil) + mark := &BlockDeletionMark{ID: block1, DeletionTime: 1} + + assert.Equal(t, &block.DeletionMark{ + ID: block1, + Version: block.DeletionMarkVersion1, + DeletionTime: 1, + }, mark.BlockDeletionMark()) +} + +func TestBlockDeletionMarks_Clone(t *testing.T) { + block1 := ulid.MustNew(1, nil) + block2 := ulid.MustNew(2, nil) + orig := BlockDeletionMarks{{ID: block1, DeletionTime: 1}, {ID: block2, DeletionTime: 2}} + + // The clone must be identical. + clone := orig.Clone() + assert.Equal(t, orig, clone) + + // Changes to the original shouldn't be reflected to the clone. + orig[0].DeletionTime = -1 + assert.Equal(t, int64(1), clone[0].DeletionTime) +} diff --git a/pkg/phlaredb/bucketindex/loader.go b/pkg/phlaredb/bucketindex/loader.go new file mode 100644 index 0000000000..ed68f2065c --- /dev/null +++ b/pkg/phlaredb/bucketindex/loader.go @@ -0,0 +1,273 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/cortexproject/cortex/blob/master/pkg/storage/tsdb/bucketindex/loader.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Cortex Authors. + +package bucketindex + +import ( + "context" + "sync" + "time" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/grafana/dskit/services" + "github.com/pkg/errors" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" + "go.uber.org/atomic" + + "github.com/grafana/pyroscope/pkg/objstore" + "github.com/grafana/pyroscope/pkg/util" +) + +type LoaderConfig struct { + CheckInterval time.Duration + UpdateOnStaleInterval time.Duration + UpdateOnErrorInterval time.Duration + IdleTimeout time.Duration +} + +// Loader is responsible to lazy load bucket indexes and, once loaded for the first time, +// keep them updated in background. Loaded indexes are automatically offloaded once the +// idle timeout expires. +type Loader struct { + services.Service + + bkt objstore.Bucket + logger log.Logger + cfg LoaderConfig + cfgProvider objstore.TenantConfigProvider + + indexesMx sync.RWMutex + indexes map[string]*cachedIndex + + // Metrics. + loadAttempts prometheus.Counter + loadFailures prometheus.Counter + loadDuration prometheus.Histogram + loaded prometheus.GaugeFunc +} + +// NewLoader makes a new Loader. +func NewLoader(cfg LoaderConfig, bucketClient objstore.Bucket, cfgProvider objstore.TenantConfigProvider, logger log.Logger, reg prometheus.Registerer) *Loader { + l := &Loader{ + bkt: bucketClient, + logger: logger, + cfg: cfg, + cfgProvider: cfgProvider, + indexes: map[string]*cachedIndex{}, + + loadAttempts: promauto.With(reg).NewCounter(prometheus.CounterOpts{ + Name: "pyroscope_bucket_index_loads_total", + Help: "Total number of bucket index loading attempts.", + }), + loadFailures: promauto.With(reg).NewCounter(prometheus.CounterOpts{ + Name: "pyroscope_bucket_index_load_failures_total", + Help: "Total number of bucket index loading failures.", + }), + loadDuration: promauto.With(reg).NewHistogram(prometheus.HistogramOpts{ + Name: "pyroscope_bucket_index_load_duration_seconds", + Help: "Duration of the a single bucket index loading operation in seconds.", + Buckets: []float64{0.01, 0.02, 0.05, 0.1, 0.2, 0.3, 1, 10}, + }), + } + + l.loaded = promauto.With(reg).NewGaugeFunc(prometheus.GaugeOpts{ + Name: "pyroscope_bucket_index_loaded", + Help: "Number of bucket indexes currently loaded in-memory.", + }, l.countLoadedIndexesMetric) + + // Apply a jitter to the sync frequency in order to increase the probability + // of hitting the shared cache (if any). + checkInterval := util.DurationWithJitter(cfg.CheckInterval, 0.2) + l.Service = services.NewTimerService(checkInterval, nil, l.checkCachedIndexes, nil) + + return l +} + +// GetIndex returns the bucket index for the given user. It returns the in-memory cached +// index if available, or load it from the bucket otherwise. +func (l *Loader) GetIndex(ctx context.Context, userID string) (*Index, error) { + l.indexesMx.RLock() + if entry := l.indexes[userID]; entry != nil { + idx := entry.index + err := entry.err + l.indexesMx.RUnlock() + + // We don't check if the index is stale because it's the responsibility + // of the background job to keep it updated. + entry.requestedAt.Store(time.Now().Unix()) + return idx, err + } + l.indexesMx.RUnlock() + + startTime := time.Now() + l.loadAttempts.Inc() + idx, err := ReadIndex(ctx, l.bkt, userID, l.cfgProvider, l.logger) + if err != nil { + // Cache the error, to avoid hammering the object store in case of persistent issues + // (eg. corrupted bucket index or not existing). + l.cacheIndex(userID, nil, err) + + if errors.Is(err, ErrIndexNotFound) { + level.Warn(l.logger).Log("msg", "bucket index not found", "user", userID) + } else { + // We don't track ErrIndexNotFound as failure because it's a legit case (eg. a tenant just + // started to remote write and its blocks haven't uploaded to storage yet). + l.loadFailures.Inc() + level.Error(l.logger).Log("msg", "unable to load bucket index", "user", userID, "err", err) + } + + return nil, err + } + + // Cache the index. + l.cacheIndex(userID, idx, nil) + + elapsedTime := time.Since(startTime) + l.loadDuration.Observe(elapsedTime.Seconds()) + level.Info(l.logger).Log("msg", "loaded bucket index", "user", userID, "duration", elapsedTime) + return idx, nil +} + +func (l *Loader) cacheIndex(userID string, idx *Index, err error) { + l.indexesMx.Lock() + defer l.indexesMx.Unlock() + + // Not an issue if, due to concurrency, another index was already cached + // and we overwrite it: last will win. + l.indexes[userID] = newCachedIndex(idx, err) +} + +// checkCachedIndexes checks all cached indexes and, for each of them, does two things: +// 1. Offload indexes not requested since >= idle timeout +// 2. Update indexes which have been updated last time since >= update timeout +func (l *Loader) checkCachedIndexes(ctx context.Context) error { + // Build a list of users for which we should update or delete the index. + toUpdate, toDelete := l.checkCachedIndexesToUpdateAndDelete() + + // Delete unused indexes. + for _, userID := range toDelete { + l.deleteCachedIndex(userID) + } + + // Update actively used indexes. + for _, userID := range toUpdate { + l.updateCachedIndex(ctx, userID) + } + + // Never return error, otherwise the service terminates. + return nil +} + +func (l *Loader) checkCachedIndexesToUpdateAndDelete() (toUpdate, toDelete []string) { + now := time.Now() + + l.indexesMx.RLock() + defer l.indexesMx.RUnlock() + + for userID, entry := range l.indexes { + // Given ErrIndexNotFound is a legit case and assuming UpdateOnErrorInterval is lower than + // UpdateOnStaleInterval, we don't consider ErrIndexNotFound as an error with regards to the + // refresh interval and so it will updated once stale. + isError := entry.err != nil && !errors.Is(entry.err, ErrIndexNotFound) + + switch { + case now.Sub(entry.getRequestedAt()) >= l.cfg.IdleTimeout: + toDelete = append(toDelete, userID) + case isError && now.Sub(entry.getUpdatedAt()) >= l.cfg.UpdateOnErrorInterval: + toUpdate = append(toUpdate, userID) + case !isError && now.Sub(entry.getUpdatedAt()) >= l.cfg.UpdateOnStaleInterval: + toUpdate = append(toUpdate, userID) + } + } + + return +} + +func (l *Loader) updateCachedIndex(ctx context.Context, userID string) { + l.loadAttempts.Inc() + startTime := time.Now() + idx, err := ReadIndex(ctx, l.bkt, userID, l.cfgProvider, l.logger) + if err != nil && !errors.Is(err, ErrIndexNotFound) { + l.loadFailures.Inc() + level.Warn(l.logger).Log("msg", "unable to update bucket index", "user", userID, "err", err) + return + } + + l.loadDuration.Observe(time.Since(startTime).Seconds()) + + // We cache it either it was successfully refreshed or wasn't found. An use case for caching the ErrIndexNotFound + // is when a tenant has rules configured but hasn't started remote writing yet. Rules will be evaluated and + // bucket index loaded by the ruler. + l.indexesMx.Lock() + l.indexes[userID].index = idx + l.indexes[userID].err = err + l.indexes[userID].setUpdatedAt(startTime) + l.indexesMx.Unlock() +} + +func (l *Loader) deleteCachedIndex(userID string) { + l.indexesMx.Lock() + delete(l.indexes, userID) + l.indexesMx.Unlock() + + level.Info(l.logger).Log("msg", "unloaded bucket index", "user", userID, "reason", "idle") +} + +func (l *Loader) countLoadedIndexesMetric() float64 { + l.indexesMx.RLock() + defer l.indexesMx.RUnlock() + + count := 0 + for _, idx := range l.indexes { + if idx.index != nil { + count++ + } + } + return float64(count) +} + +type cachedIndex struct { + // We cache either the index or the error occurred while fetching it. They're + // mutually exclusive. + index *Index + err error + + // Unix timestamp (seconds) of when the index has been updated from the storage the last time. + updatedAt atomic.Int64 + + // Unix timestamp (seconds) of when the index has been requested the last time. + requestedAt atomic.Int64 +} + +func newCachedIndex(idx *Index, err error) *cachedIndex { + entry := &cachedIndex{ + index: idx, + err: err, + } + + now := time.Now() + entry.setUpdatedAt(now) + entry.setRequestedAt(now) + + return entry +} + +func (i *cachedIndex) setUpdatedAt(ts time.Time) { + i.updatedAt.Store(ts.Unix()) +} + +func (i *cachedIndex) getUpdatedAt() time.Time { + return time.Unix(i.updatedAt.Load(), 0) +} + +func (i *cachedIndex) setRequestedAt(ts time.Time) { + i.requestedAt.Store(ts.Unix()) +} + +func (i *cachedIndex) getRequestedAt() time.Time { + return time.Unix(i.requestedAt.Load(), 0) +} diff --git a/pkg/phlaredb/bucketindex/loader_test.go b/pkg/phlaredb/bucketindex/loader_test.go new file mode 100644 index 0000000000..65eb8cc773 --- /dev/null +++ b/pkg/phlaredb/bucketindex/loader_test.go @@ -0,0 +1,587 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/cortexproject/cortex/blob/master/pkg/storage/tsdb/bucketindex/loader_test.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Cortex Authors. + +package bucketindex + +import ( + "bytes" + "context" + "path" + "strings" + "testing" + "time" + + "github.com/go-kit/log" + "github.com/grafana/dskit/services" + "github.com/grafana/dskit/test" + "github.com/oklog/ulid" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/testutil" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + + objstore_testutil "github.com/grafana/pyroscope/pkg/objstore/testutil" +) + +func TestLoader_GetIndex_ShouldLazyLoadBucketIndex(t *testing.T) { + ctx := context.Background() + reg := prometheus.NewPedanticRegistry() + bkt, _ := objstore_testutil.NewFilesystemBucket(t, ctx, t.TempDir()) + + // Create a bucket index. + idx := &Index{ + Version: IndexVersion1, + Blocks: Blocks{ + {ID: ulid.MustNew(1, nil), MinTime: 10, MaxTime: 20}, + }, + BlockDeletionMarks: nil, + UpdatedAt: time.Now().Unix(), + } + require.NoError(t, WriteIndex(ctx, bkt, "user-1", nil, idx)) + + // Create the loader. + loader := NewLoader(prepareLoaderConfig(), bkt, nil, log.NewNopLogger(), reg) + require.NoError(t, services.StartAndAwaitRunning(ctx, loader)) + t.Cleanup(func() { + require.NoError(t, services.StopAndAwaitTerminated(ctx, loader)) + }) + + // Ensure no index has been loaded yet. + assert.NoError(t, testutil.GatherAndCompare(reg, bytes.NewBufferString(` + # HELP pyroscope_bucket_index_load_failures_total Total number of bucket index loading failures. + # TYPE pyroscope_bucket_index_load_failures_total counter + pyroscope_bucket_index_load_failures_total 0 + # HELP pyroscope_bucket_index_loaded Number of bucket indexes currently loaded in-memory. + # TYPE pyroscope_bucket_index_loaded gauge + pyroscope_bucket_index_loaded 0 + # HELP pyroscope_bucket_index_loads_total Total number of bucket index loading attempts. + # TYPE pyroscope_bucket_index_loads_total counter + pyroscope_bucket_index_loads_total 0 + `), + "pyroscope_bucket_index_loads_total", + "pyroscope_bucket_index_load_failures_total", + "pyroscope_bucket_index_loaded", + )) + + // Request the index multiple times. + for i := 0; i < 10; i++ { + actualIdx, err := loader.GetIndex(ctx, "user-1") + require.NoError(t, err) + assert.Equal(t, idx, actualIdx) + } + + // Ensure metrics have been updated accordingly. + assert.NoError(t, testutil.GatherAndCompare(reg, bytes.NewBufferString(` + # HELP pyroscope_bucket_index_load_failures_total Total number of bucket index loading failures. + # TYPE pyroscope_bucket_index_load_failures_total counter + pyroscope_bucket_index_load_failures_total 0 + # HELP pyroscope_bucket_index_loaded Number of bucket indexes currently loaded in-memory. + # TYPE pyroscope_bucket_index_loaded gauge + pyroscope_bucket_index_loaded 1 + # HELP pyroscope_bucket_index_loads_total Total number of bucket index loading attempts. + # TYPE pyroscope_bucket_index_loads_total counter + pyroscope_bucket_index_loads_total 1 + `), + "pyroscope_bucket_index_loads_total", + "pyroscope_bucket_index_load_failures_total", + "pyroscope_bucket_index_loaded", + )) +} + +func TestLoader_GetIndex_ShouldCacheError(t *testing.T) { + ctx := context.Background() + reg := prometheus.NewPedanticRegistry() + bkt, _ := objstore_testutil.NewFilesystemBucket(t, ctx, t.TempDir()) + + // Create the loader. + loader := NewLoader(prepareLoaderConfig(), bkt, nil, log.NewNopLogger(), reg) + require.NoError(t, services.StartAndAwaitRunning(ctx, loader)) + t.Cleanup(func() { + require.NoError(t, services.StopAndAwaitTerminated(ctx, loader)) + }) + + // Write a corrupted index. + require.NoError(t, bkt.Upload(ctx, path.Join("user-1", IndexCompressedFilename), strings.NewReader("invalid!}"))) + + // Request the index multiple times. + for i := 0; i < 10; i++ { + _, err := loader.GetIndex(ctx, "user-1") + require.Equal(t, ErrIndexCorrupted, err) + } + + // Ensure metrics have been updated accordingly. + assert.NoError(t, testutil.GatherAndCompare(reg, bytes.NewBufferString(` + # HELP pyroscope_bucket_index_load_failures_total Total number of bucket index loading failures. + # TYPE pyroscope_bucket_index_load_failures_total counter + pyroscope_bucket_index_load_failures_total 1 + # HELP pyroscope_bucket_index_loaded Number of bucket indexes currently loaded in-memory. + # TYPE pyroscope_bucket_index_loaded gauge + pyroscope_bucket_index_loaded 0 + # HELP pyroscope_bucket_index_loads_total Total number of bucket index loading attempts. + # TYPE pyroscope_bucket_index_loads_total counter + pyroscope_bucket_index_loads_total 1 + `), + "pyroscope_bucket_index_loads_total", + "pyroscope_bucket_index_load_failures_total", + "pyroscope_bucket_index_loaded", + )) +} + +func TestLoader_GetIndex_ShouldCacheIndexNotFoundError(t *testing.T) { + ctx := context.Background() + reg := prometheus.NewPedanticRegistry() + bkt, _ := objstore_testutil.NewFilesystemBucket(t, ctx, t.TempDir()) + + // Create the loader. + loader := NewLoader(prepareLoaderConfig(), bkt, nil, log.NewNopLogger(), reg) + require.NoError(t, services.StartAndAwaitRunning(ctx, loader)) + t.Cleanup(func() { + require.NoError(t, services.StopAndAwaitTerminated(ctx, loader)) + }) + + // Request the index multiple times. + for i := 0; i < 10; i++ { + _, err := loader.GetIndex(ctx, "user-1") + require.Equal(t, ErrIndexNotFound, err) + } + + // Ensure metrics have been updated accordingly. + assert.NoError(t, testutil.GatherAndCompare(reg, bytes.NewBufferString(` + # HELP pyroscope_bucket_index_load_failures_total Total number of bucket index loading failures. + # TYPE pyroscope_bucket_index_load_failures_total counter + pyroscope_bucket_index_load_failures_total 0 + # HELP pyroscope_bucket_index_loaded Number of bucket indexes currently loaded in-memory. + # TYPE pyroscope_bucket_index_loaded gauge + pyroscope_bucket_index_loaded 0 + # HELP pyroscope_bucket_index_loads_total Total number of bucket index loading attempts. + # TYPE pyroscope_bucket_index_loads_total counter + pyroscope_bucket_index_loads_total 1 + `), + "pyroscope_bucket_index_loads_total", + "pyroscope_bucket_index_load_failures_total", + "pyroscope_bucket_index_loaded", + )) +} + +func TestLoader_ShouldUpdateIndexInBackgroundOnPreviousLoadSuccess(t *testing.T) { + ctx := context.Background() + reg := prometheus.NewPedanticRegistry() + bkt, _ := objstore_testutil.NewFilesystemBucket(t, ctx, t.TempDir()) + + // Create a bucket index. + idx := &Index{ + Version: IndexVersion1, + Blocks: Blocks{ + {ID: ulid.MustNew(1, nil), MinTime: 10, MaxTime: 20}, + }, + BlockDeletionMarks: nil, + UpdatedAt: time.Now().Unix(), + } + require.NoError(t, WriteIndex(ctx, bkt, "user-1", nil, idx)) + + // Create the loader. + cfg := LoaderConfig{ + CheckInterval: time.Second, + UpdateOnStaleInterval: time.Second, + UpdateOnErrorInterval: time.Hour, // Intentionally high to not hit it. + IdleTimeout: time.Hour, // Intentionally high to not hit it. + } + + loader := NewLoader(cfg, bkt, nil, log.NewNopLogger(), reg) + require.NoError(t, services.StartAndAwaitRunning(ctx, loader)) + t.Cleanup(func() { + require.NoError(t, services.StopAndAwaitTerminated(ctx, loader)) + }) + + actualIdx, err := loader.GetIndex(ctx, "user-1") + require.NoError(t, err) + assert.Equal(t, idx, actualIdx) + + // Update the bucket index. + idx.Blocks = append(idx.Blocks, &Block{ID: ulid.MustNew(2, nil), MinTime: 20, MaxTime: 30}) + require.NoError(t, WriteIndex(ctx, bkt, "user-1", nil, idx)) + + // Wait until the index has been updated in background. + test.Poll(t, 3*time.Second, 2, func() interface{} { + actualIdx, err := loader.GetIndex(ctx, "user-1") + if err != nil { + return 0 + } + return len(actualIdx.Blocks) + }) + + actualIdx, err = loader.GetIndex(ctx, "user-1") + require.NoError(t, err) + assert.Equal(t, idx, actualIdx) + + // Ensure metrics have been updated accordingly. + assert.NoError(t, testutil.GatherAndCompare(reg, bytes.NewBufferString(` + # HELP pyroscope_bucket_index_load_failures_total Total number of bucket index loading failures. + # TYPE pyroscope_bucket_index_load_failures_total counter + pyroscope_bucket_index_load_failures_total 0 + # HELP pyroscope_bucket_index_loaded Number of bucket indexes currently loaded in-memory. + # TYPE pyroscope_bucket_index_loaded gauge + pyroscope_bucket_index_loaded 1 + `), + "pyroscope_bucket_index_load_failures_total", + "pyroscope_bucket_index_loaded", + )) +} + +func TestLoader_ShouldUpdateIndexInBackgroundOnPreviousLoadFailure(t *testing.T) { + ctx := context.Background() + reg := prometheus.NewPedanticRegistry() + bkt, _ := objstore_testutil.NewFilesystemBucket(t, ctx, t.TempDir()) + + // Write a corrupted index. + require.NoError(t, bkt.Upload(ctx, path.Join("user-1", IndexCompressedFilename), strings.NewReader("invalid!}"))) + + // Create the loader. + cfg := LoaderConfig{ + CheckInterval: time.Second, + UpdateOnStaleInterval: time.Hour, // Intentionally high to not hit it. + UpdateOnErrorInterval: time.Second, + IdleTimeout: time.Hour, // Intentionally high to not hit it. + } + + loader := NewLoader(cfg, bkt, nil, log.NewNopLogger(), reg) + require.NoError(t, services.StartAndAwaitRunning(ctx, loader)) + t.Cleanup(func() { + require.NoError(t, services.StopAndAwaitTerminated(ctx, loader)) + }) + + _, err := loader.GetIndex(ctx, "user-1") + assert.Equal(t, ErrIndexCorrupted, err) + + // Upload the bucket index. + idx := &Index{ + Version: IndexVersion1, + Blocks: Blocks{ + {ID: ulid.MustNew(1, nil), MinTime: 10, MaxTime: 20}, + }, + BlockDeletionMarks: nil, + UpdatedAt: time.Now().Unix(), + } + require.NoError(t, WriteIndex(ctx, bkt, "user-1", nil, idx)) + + // Wait until the index has been updated in background. + test.Poll(t, 3*time.Second, nil, func() interface{} { + _, err := loader.GetIndex(ctx, "user-1") + return err + }) + + actualIdx, err := loader.GetIndex(ctx, "user-1") + require.NoError(t, err) + assert.Equal(t, idx, actualIdx) + + // Ensure metrics have been updated accordingly. + assert.NoError(t, testutil.GatherAndCompare(reg, bytes.NewBufferString(` + # HELP pyroscope_bucket_index_loaded Number of bucket indexes currently loaded in-memory. + # TYPE pyroscope_bucket_index_loaded gauge + pyroscope_bucket_index_loaded 1 + `), + "pyroscope_bucket_index_loaded", + )) +} + +func TestLoader_ShouldUpdateIndexInBackgroundOnPreviousIndexNotFound(t *testing.T) { + ctx := context.Background() + reg := prometheus.NewPedanticRegistry() + bkt, _ := objstore_testutil.NewFilesystemBucket(t, ctx, t.TempDir()) + + // Create the loader. + cfg := LoaderConfig{ + CheckInterval: time.Second, + UpdateOnStaleInterval: time.Second, + UpdateOnErrorInterval: time.Hour, // Intentionally high to not hit it. + IdleTimeout: time.Hour, // Intentionally high to not hit it. + } + + loader := NewLoader(cfg, bkt, nil, log.NewNopLogger(), reg) + require.NoError(t, services.StartAndAwaitRunning(ctx, loader)) + t.Cleanup(func() { + require.NoError(t, services.StopAndAwaitTerminated(ctx, loader)) + }) + + _, err := loader.GetIndex(ctx, "user-1") + assert.Equal(t, ErrIndexNotFound, err) + + // Upload the bucket index. + idx := &Index{ + Version: IndexVersion1, + Blocks: Blocks{ + {ID: ulid.MustNew(1, nil), MinTime: 10, MaxTime: 20}, + }, + BlockDeletionMarks: nil, + UpdatedAt: time.Now().Unix(), + } + require.NoError(t, WriteIndex(ctx, bkt, "user-1", nil, idx)) + + // Wait until the index has been updated in background. + test.Poll(t, 3*time.Second, nil, func() interface{} { + _, err := loader.GetIndex(ctx, "user-1") + return err + }) + + actualIdx, err := loader.GetIndex(ctx, "user-1") + require.NoError(t, err) + assert.Equal(t, idx, actualIdx) + + // Ensure metrics have been updated accordingly. + assert.NoError(t, testutil.GatherAndCompare(reg, bytes.NewBufferString(` + # HELP pyroscope_bucket_index_loaded Number of bucket indexes currently loaded in-memory. + # TYPE pyroscope_bucket_index_loaded gauge + pyroscope_bucket_index_loaded 1 + `), + "pyroscope_bucket_index_loaded", + )) +} + +func TestLoader_ShouldNotCacheCriticalErrorOnBackgroundUpdates(t *testing.T) { + ctx := context.Background() + reg := prometheus.NewPedanticRegistry() + bkt, _ := objstore_testutil.NewFilesystemBucket(t, ctx, t.TempDir()) + + // Create a bucket index. + idx := &Index{ + Version: IndexVersion1, + Blocks: Blocks{ + {ID: ulid.MustNew(1, nil), MinTime: 10, MaxTime: 20}, + }, + BlockDeletionMarks: nil, + UpdatedAt: time.Now().Unix(), + } + require.NoError(t, WriteIndex(ctx, bkt, "user-1", nil, idx)) + + // Create the loader. + cfg := LoaderConfig{ + CheckInterval: time.Second, + UpdateOnStaleInterval: time.Second, + UpdateOnErrorInterval: time.Second, + IdleTimeout: time.Hour, // Intentionally high to not hit it. + } + + loader := NewLoader(cfg, bkt, nil, log.NewNopLogger(), reg) + require.NoError(t, services.StartAndAwaitRunning(ctx, loader)) + t.Cleanup(func() { + require.NoError(t, services.StopAndAwaitTerminated(ctx, loader)) + }) + + actualIdx, err := loader.GetIndex(ctx, "user-1") + require.NoError(t, err) + assert.Equal(t, idx, actualIdx) + + // Write a corrupted index. + require.NoError(t, bkt.Upload(ctx, path.Join("user-1", IndexCompressedFilename), strings.NewReader("invalid!}"))) + + // Wait until the first failure has been tracked. + test.Poll(t, 3*time.Second, true, func() interface{} { + return testutil.ToFloat64(loader.loadFailures) > 0 + }) + + actualIdx, err = loader.GetIndex(ctx, "user-1") + require.NoError(t, err) + assert.Equal(t, idx, actualIdx) + + // Ensure metrics have been updated accordingly. + assert.NoError(t, testutil.GatherAndCompare(reg, bytes.NewBufferString(` + # HELP pyroscope_bucket_index_loaded Number of bucket indexes currently loaded in-memory. + # TYPE pyroscope_bucket_index_loaded gauge + pyroscope_bucket_index_loaded 1 + `), + "pyroscope_bucket_index_loaded", + )) +} + +func TestLoader_ShouldCacheIndexNotFoundOnBackgroundUpdates(t *testing.T) { + ctx := context.Background() + reg := prometheus.NewPedanticRegistry() + bkt, _ := objstore_testutil.NewFilesystemBucket(t, ctx, t.TempDir()) + + // Create a bucket index. + idx := &Index{ + Version: IndexVersion1, + Blocks: Blocks{ + {ID: ulid.MustNew(1, nil), MinTime: 10, MaxTime: 20}, + }, + BlockDeletionMarks: nil, + UpdatedAt: time.Now().Unix(), + } + require.NoError(t, WriteIndex(ctx, bkt, "user-1", nil, idx)) + + // Create the loader. + cfg := LoaderConfig{ + CheckInterval: time.Second, + UpdateOnStaleInterval: time.Second, + UpdateOnErrorInterval: time.Second, + IdleTimeout: time.Hour, // Intentionally high to not hit it. + } + + loader := NewLoader(cfg, bkt, nil, log.NewNopLogger(), reg) + require.NoError(t, services.StartAndAwaitRunning(ctx, loader)) + t.Cleanup(func() { + require.NoError(t, services.StopAndAwaitTerminated(ctx, loader)) + }) + + actualIdx, err := loader.GetIndex(ctx, "user-1") + require.NoError(t, err) + assert.Equal(t, idx, actualIdx) + + // Delete the bucket index. + require.NoError(t, DeleteIndex(ctx, bkt, "user-1", nil)) + + // Wait until the next index load attempt occurs. + prevLoads := testutil.ToFloat64(loader.loadAttempts) + test.Poll(t, 3*time.Second, true, func() interface{} { + return testutil.ToFloat64(loader.loadAttempts) > prevLoads + }) + + // We expect the bucket index is not considered loaded because of the error. + assert.NoError(t, testutil.GatherAndCompare(reg, bytes.NewBufferString(` + # HELP pyroscope_bucket_index_loaded Number of bucket indexes currently loaded in-memory. + # TYPE pyroscope_bucket_index_loaded gauge + pyroscope_bucket_index_loaded 0 + `), + "pyroscope_bucket_index_loaded", + )) + + // Try to get the index again. We expect no load attempt because the error has been cached. + prevLoads = testutil.ToFloat64(loader.loadAttempts) + actualIdx, err = loader.GetIndex(ctx, "user-1") + assert.Equal(t, ErrIndexNotFound, err) + assert.Nil(t, actualIdx) + assert.Equal(t, prevLoads, testutil.ToFloat64(loader.loadAttempts)) +} + +func TestLoader_ShouldOffloadIndexIfNotFoundDuringBackgroundUpdates(t *testing.T) { + ctx := context.Background() + reg := prometheus.NewPedanticRegistry() + bkt, _ := objstore_testutil.NewFilesystemBucket(t, ctx, t.TempDir()) + + // Create a bucket index. + idx := &Index{ + Version: IndexVersion1, + Blocks: Blocks{ + {ID: ulid.MustNew(1, nil), MinTime: 10, MaxTime: 20}, + }, + BlockDeletionMarks: nil, + UpdatedAt: time.Now().Unix(), + } + require.NoError(t, WriteIndex(ctx, bkt, "user-1", nil, idx)) + + // Create the loader. + cfg := LoaderConfig{ + CheckInterval: time.Second, + UpdateOnStaleInterval: time.Second, + UpdateOnErrorInterval: time.Second, + IdleTimeout: time.Hour, // Intentionally high to not hit it. + } + + loader := NewLoader(cfg, bkt, nil, log.NewNopLogger(), reg) + require.NoError(t, services.StartAndAwaitRunning(ctx, loader)) + t.Cleanup(func() { + require.NoError(t, services.StopAndAwaitTerminated(ctx, loader)) + }) + + actualIdx, err := loader.GetIndex(ctx, "user-1") + require.NoError(t, err) + assert.Equal(t, idx, actualIdx) + + // Delete the index + require.NoError(t, DeleteIndex(ctx, bkt, "user-1", nil)) + + // Wait until the index is offloaded. + test.Poll(t, 3*time.Second, float64(0), func() interface{} { + return testutil.ToFloat64(loader.loaded) + }) + + _, err = loader.GetIndex(ctx, "user-1") + require.Equal(t, ErrIndexNotFound, err) + + // Ensure metrics have been updated accordingly. + assert.NoError(t, testutil.GatherAndCompare(reg, bytes.NewBufferString(` + # HELP pyroscope_bucket_index_loaded Number of bucket indexes currently loaded in-memory. + # TYPE pyroscope_bucket_index_loaded gauge + pyroscope_bucket_index_loaded 0 + `), + "pyroscope_bucket_index_loaded", + )) +} + +func TestLoader_ShouldOffloadIndexIfIdleTimeoutIsReachedDuringBackgroundUpdates(t *testing.T) { + ctx := context.Background() + reg := prometheus.NewPedanticRegistry() + bkt, _ := objstore_testutil.NewFilesystemBucket(t, ctx, t.TempDir()) + + // Create a bucket index. + idx := &Index{ + Version: IndexVersion1, + Blocks: Blocks{ + {ID: ulid.MustNew(1, nil), MinTime: 10, MaxTime: 20}, + }, + BlockDeletionMarks: nil, + UpdatedAt: time.Now().Unix(), + } + require.NoError(t, WriteIndex(ctx, bkt, "user-1", nil, idx)) + + // Create the loader. + cfg := LoaderConfig{ + CheckInterval: time.Second, + UpdateOnStaleInterval: time.Second, + UpdateOnErrorInterval: time.Second, + IdleTimeout: 0, // Offload at first check. + } + + loader := NewLoader(cfg, bkt, nil, log.NewNopLogger(), reg) + require.NoError(t, services.StartAndAwaitRunning(ctx, loader)) + t.Cleanup(func() { + require.NoError(t, services.StopAndAwaitTerminated(ctx, loader)) + }) + + actualIdx, err := loader.GetIndex(ctx, "user-1") + require.NoError(t, err) + assert.Equal(t, idx, actualIdx) + + // Wait until the index is offloaded. + test.Poll(t, 3*time.Second, float64(0), func() interface{} { + return testutil.ToFloat64(loader.loaded) + }) + + // Ensure metrics have been updated accordingly. + assert.NoError(t, testutil.GatherAndCompare(reg, bytes.NewBufferString(` + # HELP pyroscope_bucket_index_loaded Number of bucket indexes currently loaded in-memory. + # TYPE pyroscope_bucket_index_loaded gauge + pyroscope_bucket_index_loaded 0 + # HELP pyroscope_bucket_index_loads_total Total number of bucket index loading attempts. + # TYPE pyroscope_bucket_index_loads_total counter + pyroscope_bucket_index_loads_total 1 + `), + "pyroscope_bucket_index_loaded", + "pyroscope_bucket_index_loads_total", + )) + + // Load it again. + actualIdx, err = loader.GetIndex(ctx, "user-1") + require.NoError(t, err) + assert.Equal(t, idx, actualIdx) + + // Ensure metrics have been updated accordingly. + assert.NoError(t, testutil.GatherAndCompare(reg, bytes.NewBufferString(` + # HELP pyroscope_bucket_index_loads_total Total number of bucket index loading attempts. + # TYPE pyroscope_bucket_index_loads_total counter + pyroscope_bucket_index_loads_total 2 + `), + "pyroscope_bucket_index_loads_total", + )) +} + +func prepareLoaderConfig() LoaderConfig { + return LoaderConfig{ + CheckInterval: time.Minute, + UpdateOnStaleInterval: 15 * time.Minute, + UpdateOnErrorInterval: time.Minute, + IdleTimeout: time.Hour, + } +} diff --git a/pkg/phlaredb/bucketindex/storage.go b/pkg/phlaredb/bucketindex/storage.go new file mode 100644 index 0000000000..397f1b5770 --- /dev/null +++ b/pkg/phlaredb/bucketindex/storage.go @@ -0,0 +1,103 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/cortexproject/cortex/blob/master/pkg/storage/tsdb/bucketindex/storage.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Cortex Authors. + +package bucketindex + +import ( + "bytes" + "compress/gzip" + "context" + "encoding/json" + "time" + + "github.com/go-kit/log" + "github.com/grafana/dskit/runutil" + "github.com/pkg/errors" + + "github.com/grafana/pyroscope/pkg/objstore" +) + +var ( + ErrIndexNotFound = errors.New("bucket index not found") + ErrIndexCorrupted = errors.New("bucket index corrupted") +) + +// ReadIndex reads, parses and returns a bucket index from the bucket. +// ReadIndex has a one-minute timeout for completing the read against the bucket. +// One minute is hard-coded to a reasonably high value to protect against operations that can take unbounded time. +func ReadIndex(ctx context.Context, bkt objstore.Bucket, userID string, cfgProvider objstore.TenantConfigProvider, logger log.Logger) (*Index, error) { + ctx, cancel := context.WithTimeout(ctx, time.Minute) + defer cancel() + + userBkt := objstore.NewUserBucketClient(userID, bkt, cfgProvider) + + // Get the bucket index. + reader, err := userBkt.WithExpectedErrs(userBkt.IsObjNotFoundErr).Get(ctx, IndexCompressedFilename) + if err != nil { + if userBkt.IsObjNotFoundErr(err) { + return nil, ErrIndexNotFound + } + return nil, errors.Wrap(err, "read bucket index") + } + defer runutil.CloseWithLogOnErr(logger, reader, "close bucket index reader") + + // Read all the content. + gzipReader, err := gzip.NewReader(reader) + if err != nil { + return nil, ErrIndexCorrupted + } + defer runutil.CloseWithLogOnErr(logger, gzipReader, "close bucket index gzip reader") + + // Deserialize it. + index := &Index{} + d := json.NewDecoder(gzipReader) + if err := d.Decode(index); err != nil { + return nil, ErrIndexCorrupted + } + + return index, nil +} + +// WriteIndex uploads the provided index to the storage. +func WriteIndex(ctx context.Context, bkt objstore.Bucket, userID string, cfgProvider objstore.TenantConfigProvider, idx *Index) error { + bkt = objstore.NewUserBucketClient(userID, bkt, cfgProvider) + + // Marshal the index. + content, err := json.Marshal(idx) + if err != nil { + return errors.Wrap(err, "marshal bucket index") + } + + // Compress it. + var gzipContent bytes.Buffer + gzip := gzip.NewWriter(&gzipContent) + gzip.Name = IndexFilename + + if _, err := gzip.Write(content); err != nil { + return errors.Wrap(err, "gzip bucket index") + } + if err := gzip.Close(); err != nil { + return errors.Wrap(err, "close gzip bucket index") + } + + // Upload the index to the storage. + if err := bkt.Upload(ctx, IndexCompressedFilename, &gzipContent); err != nil { + return errors.Wrap(err, "upload bucket index") + } + + return nil +} + +// DeleteIndex deletes the bucket index from the storage. No error is returned if the index +// does not exist. +func DeleteIndex(ctx context.Context, bkt objstore.Bucket, userID string, cfgProvider objstore.TenantConfigProvider) error { + bkt = objstore.NewUserBucketClient(userID, bkt, cfgProvider) + + err := bkt.Delete(ctx, IndexCompressedFilename) + if err != nil && !bkt.IsObjNotFoundErr(err) { + return errors.Wrap(err, "delete bucket index") + } + return nil +} diff --git a/pkg/phlaredb/bucketindex/storage_test.go b/pkg/phlaredb/bucketindex/storage_test.go new file mode 100644 index 0000000000..a04a618af1 --- /dev/null +++ b/pkg/phlaredb/bucketindex/storage_test.go @@ -0,0 +1,122 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/cortexproject/cortex/blob/master/pkg/storage/tsdb/bucketindex/storage_test.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Cortex Authors. + +package bucketindex + +import ( + "context" + "path" + "strings" + "testing" + + "github.com/go-kit/log" + "github.com/prometheus/common/model" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + + objstore_testutil "github.com/grafana/pyroscope/pkg/objstore/testutil" + "github.com/grafana/pyroscope/pkg/phlaredb/block" + block_testutil "github.com/grafana/pyroscope/pkg/phlaredb/block/testutil" +) + +func TestReadIndex_ShouldReturnErrorIfIndexDoesNotExist(t *testing.T) { + bkt, _ := objstore_testutil.NewFilesystemBucket(t, context.Background(), t.TempDir()) + + idx, err := ReadIndex(context.Background(), bkt, "user-1", nil, log.NewNopLogger()) + require.Equal(t, ErrIndexNotFound, err) + require.Nil(t, idx) +} + +func TestReadIndex_ShouldReturnErrorIfIndexIsCorrupted(t *testing.T) { + const userID = "user-1" + + ctx := context.Background() + bkt, _ := objstore_testutil.NewFilesystemBucket(t, ctx, t.TempDir()) + + // Write a corrupted index. + require.NoError(t, bkt.Upload(ctx, path.Join(userID, IndexCompressedFilename), strings.NewReader("invalid!}"))) + + idx, err := ReadIndex(ctx, bkt, userID, nil, log.NewNopLogger()) + require.Equal(t, ErrIndexCorrupted, err) + require.Nil(t, idx) +} + +func TestReadIndex_ShouldReturnTheParsedIndexOnSuccess(t *testing.T) { + const userID = "user-1" + + ctx := context.Background() + logger := log.NewNopLogger() + + bkt, _ := objstore_testutil.NewFilesystemBucket(t, ctx, t.TempDir()) + + // Mock some blocks in the storage. + bkt = block.BucketWithGlobalMarkers(bkt) + block_testutil.MockStorageBlock(t, bkt, userID, 10, 20) + block_testutil.MockStorageBlock(t, bkt, userID, 20, 30) + block_testutil.MockStorageDeletionMark(t, bkt, userID, block_testutil.MockStorageBlock(t, bkt, userID, 30, 40)) + + // Write the index. + u := NewUpdater(bkt, userID, nil, logger) + expectedIdx, _, err := u.UpdateIndex(ctx, nil) + require.NoError(t, err) + require.NoError(t, WriteIndex(ctx, bkt, userID, nil, expectedIdx)) + + // Read it back and compare. + actualIdx, err := ReadIndex(ctx, bkt, userID, nil, logger) + require.NoError(t, err) + assert.Equal(t, expectedIdx, actualIdx) +} + +func BenchmarkReadIndex(b *testing.B) { + const ( + numBlocks = 1000 + numBlockDeletionMarks = 100 + userID = "user-1" + ) + + ctx := context.Background() + logger := log.NewNopLogger() + + bkt, _ := objstore_testutil.NewFilesystemBucket(b, ctx, b.TempDir()) + + // Mock some blocks and deletion marks in the storage. + bkt = block.BucketWithGlobalMarkers(bkt) + for i := 0; i < numBlocks; i++ { + minT := int64(i * 10) + maxT := int64((i + 1) * 10) + + meta := block_testutil.MockStorageBlock(b, bkt, userID, model.Time(minT), model.Time(maxT)) + + if i < numBlockDeletionMarks { + block_testutil.MockStorageDeletionMark(b, bkt, userID, meta) + } + } + + // Write the index. + u := NewUpdater(bkt, userID, nil, logger) + idx, _, err := u.UpdateIndex(ctx, nil) + require.NoError(b, err) + require.NoError(b, WriteIndex(ctx, bkt, userID, nil, idx)) + + // Read it back once just to make sure the index contains the expected data. + idx, err = ReadIndex(ctx, bkt, userID, nil, logger) + require.NoError(b, err) + require.Len(b, idx.Blocks, numBlocks) + require.Len(b, idx.BlockDeletionMarks, numBlockDeletionMarks) + + b.ResetTimer() + + for n := 0; n < b.N; n++ { + _, err := ReadIndex(ctx, bkt, userID, nil, logger) + require.NoError(b, err) + } +} + +func TestDeleteIndex_ShouldNotReturnErrorIfIndexDoesNotExist(t *testing.T) { + ctx := context.Background() + bkt, _ := objstore_testutil.NewFilesystemBucket(t, ctx, t.TempDir()) + + assert.NoError(t, DeleteIndex(ctx, bkt, "user-1", nil)) +} diff --git a/pkg/phlaredb/bucketindex/updater.go b/pkg/phlaredb/bucketindex/updater.go new file mode 100644 index 0000000000..36ef8586be --- /dev/null +++ b/pkg/phlaredb/bucketindex/updater.go @@ -0,0 +1,233 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/cortexproject/cortex/blob/master/pkg/storage/tsdb/bucketindex/updater.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Cortex Authors. + +package bucketindex + +import ( + "context" + "encoding/json" + "io" + "path" + "time" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/grafana/dskit/runutil" + "github.com/oklog/ulid" + "github.com/pkg/errors" + + "github.com/grafana/pyroscope/pkg/objstore" + "github.com/grafana/pyroscope/pkg/phlaredb/block" +) + +var ( + ErrBlockMetaNotFound = block.ErrorSyncMetaNotFound + ErrBlockMetaCorrupted = block.ErrorSyncMetaCorrupted + ErrBlockDeletionMarkNotFound = errors.New("block deletion mark not found") + ErrBlockDeletionMarkCorrupted = errors.New("block deletion mark corrupted") +) + +// Updater is responsible to generate an update in-memory bucket index. +type Updater struct { + bkt objstore.InstrumentedBucket + logger log.Logger +} + +func NewUpdater(bkt objstore.Bucket, userID string, cfgProvider objstore.TenantConfigProvider, logger log.Logger) *Updater { + return &Updater{ + bkt: objstore.NewUserBucketClient(userID, bkt, cfgProvider), + logger: logger, + } +} + +// UpdateIndex generates the bucket index and returns it, without storing it to the storage. +// If the old index is not passed in input, then the bucket index will be generated from scratch. +func (w *Updater) UpdateIndex(ctx context.Context, old *Index) (*Index, map[ulid.ULID]error, error) { + var oldBlocks []*Block + var oldBlockDeletionMarks []*BlockDeletionMark + + // Use the old index if provided, and it is using the latest version format. + if old != nil && old.Version == IndexVersion2 { + oldBlocks = old.Blocks + oldBlockDeletionMarks = old.BlockDeletionMarks + } + + blocks, partials, err := w.updateBlocks(ctx, oldBlocks) + if err != nil { + return nil, nil, err + } + + blockDeletionMarks, err := w.updateBlockDeletionMarks(ctx, oldBlockDeletionMarks) + if err != nil { + return nil, nil, err + } + + return &Index{ + Version: IndexVersion2, + Blocks: blocks, + BlockDeletionMarks: blockDeletionMarks, + UpdatedAt: time.Now().Unix(), + }, partials, nil +} + +func (w *Updater) updateBlocks(ctx context.Context, old []*Block) (blocks []*Block, partials map[ulid.ULID]error, _ error) { + discovered := map[ulid.ULID]struct{}{} + partials = map[ulid.ULID]error{} + + // Find all blocks in the storage. + err := w.bkt.Iter(ctx, "", func(name string) error { + if id, ok := block.IsBlockDir(name); ok { + discovered[id] = struct{}{} + } + return nil + }) + if err != nil { + return nil, nil, errors.Wrap(err, "list blocks") + } + + // Since blocks are immutable, all blocks already existing in the index can just be copied. + for _, b := range old { + if _, ok := discovered[b.ID]; ok { + blocks = append(blocks, b) + delete(discovered, b.ID) + } + } + + level.Info(w.logger).Log("msg", "listed all blocks in storage", "newly_discovered", len(discovered), "existing", len(old)) + + // Remaining blocks are new ones and we have to fetch the meta.json for each of them, in order + // to find out if their upload has been completed (meta.json is uploaded last) and get the block + // information to store in the bucket index. + for id := range discovered { + b, err := w.updateBlockIndexEntry(ctx, id) + if err == nil { + blocks = append(blocks, b) + continue + } + + if errors.Is(err, ErrBlockMetaNotFound) { + partials[id] = err + level.Warn(w.logger).Log("msg", "skipped partial block when updating bucket index", "block", id.String()) + continue + } + if errors.Is(err, ErrBlockMetaCorrupted) { + partials[id] = err + level.Error(w.logger).Log("msg", "skipped block with corrupted meta.json when updating bucket index", "block", id.String(), "err", err) + continue + } + return nil, nil, err + } + level.Info(w.logger).Log("msg", "fetched blocks metas for newly discovered blocks", "total_blocks", len(blocks), "partial_errors", len(partials)) + + return blocks, partials, nil +} + +func (w *Updater) updateBlockIndexEntry(ctx context.Context, id ulid.ULID) (*Block, error) { + // Set a generous timeout for fetching the meta.json and getting the attributes of the same file. + // This protects against operations that can take unbounded time. + ctx, cancel := context.WithTimeout(ctx, time.Minute) + defer cancel() + + metaFile := path.Join(id.String(), block.MetaFilename) + + // Get the block's meta.json file. + r, err := w.bkt.Get(ctx, metaFile) + if w.bkt.IsObjNotFoundErr(err) { + return nil, ErrBlockMetaNotFound + } + if err != nil { + return nil, errors.Wrapf(err, "get block meta file: %v", metaFile) + } + defer runutil.CloseWithLogOnErr(w.logger, r, "close get block meta file") + + metaContent, err := io.ReadAll(r) + if err != nil { + return nil, errors.Wrapf(err, "read block meta file: %v", metaFile) + } + + // Unmarshal it. + m := block.Meta{} + if err := json.Unmarshal(metaContent, &m); err != nil { + return nil, errors.Wrapf(ErrBlockMetaCorrupted, "unmarshal block meta file %s: %v", metaFile, err) + } + + if !m.Version.IsValid() { + return nil, errors.Errorf("unexpected block meta version: %s version: %d", metaFile, m.Version) + } + + block := BlockFromMeta(m) + + // Get the meta.json attributes. + attrs, err := w.bkt.Attributes(ctx, metaFile) + if err != nil { + return nil, errors.Wrapf(err, "read meta file attributes: %v", metaFile) + } + + // Since the meta.json file is the last file of a block being uploaded and it's immutable + // we can safely assume that the last modified timestamp of the meta.json is the time when + // the block has completed to be uploaded. + block.UploadedAt = attrs.LastModified.Unix() + + return block, nil +} + +func (w *Updater) updateBlockDeletionMarks(ctx context.Context, old []*BlockDeletionMark) ([]*BlockDeletionMark, error) { + out := make([]*BlockDeletionMark, 0, len(old)) + + // Find all markers in the storage. + discovered, err := block.ListBlockDeletionMarks(ctx, w.bkt) + if err != nil { + return nil, err + } + + level.Info(w.logger).Log("msg", "listed deletion markers", "count", len(discovered)) + + // Since deletion marks are immutable, all markers already existing in the index can just be copied. + for _, m := range old { + if _, ok := discovered[m.ID]; ok { + out = append(out, m) + delete(discovered, m.ID) + } + } + + // Remaining markers are new ones and we have to fetch them. + for id := range discovered { + m, err := w.updateBlockDeletionMarkIndexEntry(ctx, id) + if errors.Is(err, ErrBlockDeletionMarkNotFound) { + // This could happen if the block is permanently deleted between the "list objects" and now. + level.Warn(w.logger).Log("msg", "skipped missing block deletion mark when updating bucket index", "block", id.String()) + continue + } + if errors.Is(err, ErrBlockDeletionMarkCorrupted) { + level.Error(w.logger).Log("msg", "skipped corrupted block deletion mark when updating bucket index", "block", id.String(), "err", err) + continue + } + if err != nil { + return nil, err + } + + out = append(out, m) + } + + level.Info(w.logger).Log("msg", "updated deletion markers for recently marked blocks", "count", len(discovered), "total_deletion_markers", len(out)) + + return out, nil +} + +func (w *Updater) updateBlockDeletionMarkIndexEntry(ctx context.Context, id ulid.ULID) (*BlockDeletionMark, error) { + m := block.DeletionMark{} + + if err := block.ReadMarker(ctx, w.logger, w.bkt, id.String(), &m); err != nil { + if errors.Is(err, block.ErrorMarkerNotFound) { + return nil, errors.Wrap(ErrBlockDeletionMarkNotFound, err.Error()) + } + if errors.Is(err, block.ErrorUnmarshalMarker) { + return nil, errors.Wrap(ErrBlockDeletionMarkCorrupted, err.Error()) + } + return nil, err + } + + return DeletionMarkFromBlockMarker(&m), nil +} diff --git a/pkg/phlaredb/bucketindex/updater_test.go b/pkg/phlaredb/bucketindex/updater_test.go new file mode 100644 index 0000000000..a73481695e --- /dev/null +++ b/pkg/phlaredb/bucketindex/updater_test.go @@ -0,0 +1,267 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/cortexproject/cortex/blob/master/pkg/storage/tsdb/bucketindex/updater_test.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Cortex Authors. + +package bucketindex + +import ( + "bytes" + "context" + "path" + "testing" + "time" + + "github.com/go-kit/log" + "github.com/oklog/ulid" + "github.com/pkg/errors" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + + "github.com/grafana/pyroscope/pkg/objstore" + objstore_testutil "github.com/grafana/pyroscope/pkg/objstore/testutil" + "github.com/grafana/pyroscope/pkg/phlaredb/block" + block_testutil "github.com/grafana/pyroscope/pkg/phlaredb/block/testutil" + "github.com/grafana/pyroscope/pkg/phlaredb/sharding" +) + +func TestUpdater_UpdateIndex(t *testing.T) { + const userID = "user-1" + + ctx := context.Background() + logger := log.NewNopLogger() + bkt, _ := objstore_testutil.NewFilesystemBucket(t, ctx, t.TempDir()) + + // Generate the initial index. + bkt = block.BucketWithGlobalMarkers(bkt) + block1 := block_testutil.MockStorageBlockWithExtLabels(t, bkt, userID, 10, 20, nil) + block_testutil.MockNoCompactMark(t, bkt, userID, block1) // no-compact mark is ignored by bucket index updater. + block2 := block_testutil.MockStorageBlockWithExtLabels(t, bkt, userID, 20, 30, map[string]string{sharding.CompactorShardIDLabel: "1_of_5"}) + block2Mark := block_testutil.MockStorageDeletionMark(t, bkt, userID, block2) + + w := NewUpdater(bkt, userID, nil, logger) + returnedIdx, _, err := w.UpdateIndex(ctx, nil) + require.NoError(t, err) + assertBucketIndexEqual(t, returnedIdx, bkt, userID, + []block.Meta{block1, block2}, + []*block.DeletionMark{block2Mark}) + + // Create new blocks, and update the index. + block3 := block_testutil.MockStorageBlockWithExtLabels(t, bkt, userID, 30, 40, map[string]string{"aaa": "bbb"}) + block4 := block_testutil.MockStorageBlockWithExtLabels(t, bkt, userID, 40, 50, map[string]string{sharding.CompactorShardIDLabel: "2_of_5"}) + block4Mark := block_testutil.MockStorageDeletionMark(t, bkt, userID, block4) + + returnedIdx, _, err = w.UpdateIndex(ctx, returnedIdx) + require.NoError(t, err) + assertBucketIndexEqual(t, returnedIdx, bkt, userID, + []block.Meta{block1, block2, block3, block4}, + []*block.DeletionMark{block2Mark, block4Mark}) + + // Hard delete a block and update the index. + require.NoError(t, block.Delete(ctx, log.NewNopLogger(), objstore.NewUserBucketClient(userID, bkt, nil), block2.ULID)) + + returnedIdx, _, err = w.UpdateIndex(ctx, returnedIdx) + require.NoError(t, err) + assertBucketIndexEqual(t, returnedIdx, bkt, userID, + []block.Meta{block1, block3, block4}, + []*block.DeletionMark{block4Mark}) +} + +func TestUpdater_UpdateIndex_ShouldSkipPartialBlocks(t *testing.T) { + const userID = "user-1" + + ctx := context.Background() + logger := log.NewNopLogger() + + bkt, _ := objstore_testutil.NewFilesystemBucket(t, ctx, t.TempDir()) + + // Mock some blocks in the storage. + bkt = block.BucketWithGlobalMarkers(bkt) + block1 := block_testutil.MockStorageBlockWithExtLabels(t, bkt, userID, 10, 20, map[string]string{"hello": "world"}) + block2 := block_testutil.MockStorageBlockWithExtLabels(t, bkt, userID, 20, 30, map[string]string{sharding.CompactorShardIDLabel: "3_of_10"}) + block3 := block_testutil.MockStorageBlockWithExtLabels(t, bkt, userID, 30, 40, nil) + block2Mark := block_testutil.MockStorageDeletionMark(t, bkt, userID, block2) + + // No compact marks are ignored by bucket index. + block_testutil.MockNoCompactMark(t, bkt, userID, block3) + + // Delete a block's meta.json to simulate a partial block. + require.NoError(t, bkt.Delete(ctx, path.Join(userID, block3.ULID.String(), block.MetaFilename))) + + w := NewUpdater(bkt, userID, nil, logger) + idx, partials, err := w.UpdateIndex(ctx, nil) + require.NoError(t, err) + assertBucketIndexEqual(t, idx, bkt, userID, + []block.Meta{block1, block2}, + []*block.DeletionMark{block2Mark}) + + assert.Len(t, partials, 1) + assert.True(t, errors.Is(partials[block3.ULID], ErrBlockMetaNotFound)) +} + +func TestUpdater_UpdateIndex_ShouldSkipBlocksWithCorruptedMeta(t *testing.T) { + const userID = "user-1" + + ctx := context.Background() + logger := log.NewNopLogger() + + bkt, _ := objstore_testutil.NewFilesystemBucket(t, ctx, t.TempDir()) + + // Mock some blocks in the storage. + bkt = block.BucketWithGlobalMarkers(bkt) + block1 := block_testutil.MockStorageBlockWithExtLabels(t, bkt, userID, 10, 20, nil) + block2 := block_testutil.MockStorageBlockWithExtLabels(t, bkt, userID, 20, 30, map[string]string{sharding.CompactorShardIDLabel: "55_of_64"}) + block3 := block_testutil.MockStorageBlockWithExtLabels(t, bkt, userID, 30, 40, nil) + block2Mark := block_testutil.MockStorageDeletionMark(t, bkt, userID, block2) + + // Overwrite a block's meta.json with invalid data. + require.NoError(t, bkt.Upload(ctx, path.Join(userID, block3.ULID.String(), block.MetaFilename), bytes.NewReader([]byte("invalid!}")))) + + w := NewUpdater(bkt, userID, nil, logger) + idx, partials, err := w.UpdateIndex(ctx, nil) + require.NoError(t, err) + assertBucketIndexEqual(t, idx, bkt, userID, + []block.Meta{block1, block2}, + []*block.DeletionMark{block2Mark}) + + assert.Len(t, partials, 1) + assert.True(t, errors.Is(partials[block3.ULID], ErrBlockMetaCorrupted)) +} + +func TestUpdater_UpdateIndex_ShouldSkipCorruptedDeletionMarks(t *testing.T) { + const userID = "user-1" + + ctx := context.Background() + logger := log.NewNopLogger() + + bkt, _ := objstore_testutil.NewFilesystemBucket(t, ctx, t.TempDir()) + + // Mock some blocks in the storage. + bkt = block.BucketWithGlobalMarkers(bkt) + block1 := block_testutil.MockStorageBlockWithExtLabels(t, bkt, userID, 10, 20, nil) + block2 := block_testutil.MockStorageBlockWithExtLabels(t, bkt, userID, 20, 30, nil) + block3 := block_testutil.MockStorageBlockWithExtLabels(t, bkt, userID, 30, 40, map[string]string{sharding.CompactorShardIDLabel: "2_of_7"}) + block2Mark := block_testutil.MockStorageDeletionMark(t, bkt, userID, block2) + + // Overwrite a block's deletion-mark.json with invalid data. + require.NoError(t, bkt.Upload(ctx, path.Join(userID, block2Mark.ID.String(), block.DeletionMarkFilename), bytes.NewReader([]byte("invalid!}")))) + + w := NewUpdater(bkt, userID, nil, logger) + idx, partials, err := w.UpdateIndex(ctx, nil) + require.NoError(t, err) + assertBucketIndexEqual(t, idx, bkt, userID, + []block.Meta{block1, block2, block3}, + []*block.DeletionMark{}) + assert.Empty(t, partials) +} + +func TestUpdater_UpdateIndex_NoTenantInTheBucket(t *testing.T) { + const userID = "user-1" + + ctx := context.Background() + bkt, _ := objstore_testutil.NewFilesystemBucket(t, ctx, t.TempDir()) + + for _, oldIdx := range []*Index{nil, {}} { + w := NewUpdater(bkt, userID, nil, log.NewNopLogger()) + idx, partials, err := w.UpdateIndex(ctx, oldIdx) + + require.NoError(t, err) + assert.Equal(t, IndexVersion2, idx.Version) + assert.InDelta(t, time.Now().Unix(), idx.UpdatedAt, 2) + assert.Len(t, idx.Blocks, 0) + assert.Len(t, idx.BlockDeletionMarks, 0) + assert.Empty(t, partials) + } +} + +func TestUpdater_UpdateIndexFromVersion1ToVersion2(t *testing.T) { + const userID = "user-2" + + ctx := context.Background() + logger := log.NewNopLogger() + + bkt, _ := objstore_testutil.NewFilesystemBucket(t, ctx, t.TempDir()) + + // Generate blocks with compactor shard ID. + bkt = block.BucketWithGlobalMarkers(bkt) + block1 := block_testutil.MockStorageBlockWithExtLabels(t, bkt, userID, 10, 20, map[string]string{sharding.CompactorShardIDLabel: "1_of_4"}) + block2 := block_testutil.MockStorageBlockWithExtLabels(t, bkt, userID, 20, 30, map[string]string{sharding.CompactorShardIDLabel: "3_of_4"}) + + block1WithoutCompactorShardID := block1 + block1WithoutCompactorShardID.Labels = nil + + block2WithoutCompactorShardID := block2 + block2WithoutCompactorShardID.Labels = nil + + // Double check that original block1 and block2 still have compactor shards set. + require.Equal(t, "1_of_4", block1.Labels[sharding.CompactorShardIDLabel]) + require.Equal(t, "3_of_4", block2.Labels[sharding.CompactorShardIDLabel]) + + // Generate index (this produces V2 index, with compactor shard IDs). + w := NewUpdater(bkt, userID, nil, logger) + returnedIdx, _, err := w.UpdateIndex(ctx, nil) + require.NoError(t, err) + assertBucketIndexEqual(t, returnedIdx, bkt, userID, + []block.Meta{block1, block2}, + []*block.DeletionMark{}) + + // Now remove Compactor Shard ID from index. + for _, b := range returnedIdx.Blocks { + b.CompactorShardID = "" + } + + // Try to update existing index. Since we didn't change the version, updater will reuse the index, and not update CompactorShardID field. + returnedIdx, _, err = w.UpdateIndex(ctx, returnedIdx) + require.NoError(t, err) + assertBucketIndexEqual(t, returnedIdx, bkt, userID, + []block.Meta{block1WithoutCompactorShardID, block2WithoutCompactorShardID}, // No compactor shards in bucket index. + []*block.DeletionMark{}) + + // Now set index version to old version 1. Rerunning updater should rebuild index from scratch. + returnedIdx.Version = IndexVersion1 + + returnedIdx, _, err = w.UpdateIndex(ctx, returnedIdx) + require.NoError(t, err) + assertBucketIndexEqual(t, returnedIdx, bkt, userID, + []block.Meta{block1, block2}, // Compactor shards are back. + []*block.DeletionMark{}) +} + +func getBlockUploadedAt(t testing.TB, bkt objstore.Bucket, userID string, blockID ulid.ULID) int64 { + metaFile := path.Join(userID, blockID.String(), block.MetaFilename) + + attrs, err := bkt.Attributes(context.Background(), metaFile) + require.NoError(t, err) + + return attrs.LastModified.Unix() +} + +func assertBucketIndexEqual(t testing.TB, idx *Index, bkt objstore.Bucket, userID string, expectedBlocks []block.Meta, expectedDeletionMarks []*block.DeletionMark) { + assert.Equal(t, IndexVersion2, idx.Version) + assert.InDelta(t, time.Now().Unix(), idx.UpdatedAt, 2) + + // Build the list of expected block index entries. + var expectedBlockEntries []*Block + for _, b := range expectedBlocks { + expectedBlockEntries = append(expectedBlockEntries, &Block{ + ID: b.ULID, + MinTime: b.MinTime, + MaxTime: b.MaxTime, + UploadedAt: getBlockUploadedAt(t, bkt, userID, b.ULID), + CompactorShardID: b.Labels[sharding.CompactorShardIDLabel], + }) + } + + assert.ElementsMatch(t, expectedBlockEntries, idx.Blocks) + + // Build the list of expected block deletion mark index entries. + var expectedMarkEntries []*BlockDeletionMark + for _, m := range expectedDeletionMarks { + expectedMarkEntries = append(expectedMarkEntries, &BlockDeletionMark{ + ID: m.ID, + DeletionTime: m.DeletionTime, + }) + } + + assert.ElementsMatch(t, expectedMarkEntries, idx.BlockDeletionMarks) +} From d61cbd5d3c6b42dc50fd427bf480c2a5de489ec5 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Mon, 11 Sep 2023 15:07:05 +0200 Subject: [PATCH 18/74] Fixing and adapting more code --- pkg/compactor/blocks_cleaner.go | 47 +-- pkg/compactor/compactor_http.go | 2 +- pkg/compactor/compactor_ring.go | 2 +- pkg/compactor/job_sorting_test.go | 12 +- pkg/compactor/job_test.go | 20 +- pkg/compactor/label_remover_filter_test.go | 29 +- .../shard_aware_deduplicate_filter.go | 9 +- .../shard_aware_deduplicate_filter_test.go | 40 +- pkg/compactor/split_merge_grouper.go | 50 ++- pkg/compactor/split_merge_grouper_test.go | 357 +++++++++--------- pkg/compactor/split_merge_job.go | 21 +- pkg/compactor/split_merge_job_test.go | 39 +- pkg/compactor/split_merge_planner.go | 4 +- pkg/compactor/split_merge_planner_test.go | 25 +- pkg/compactor/syncer_metrics.go | 18 +- 15 files changed, 321 insertions(+), 354 deletions(-) diff --git a/pkg/compactor/blocks_cleaner.go b/pkg/compactor/blocks_cleaner.go index 8556cecafc..bd70c4e049 100644 --- a/pkg/compactor/blocks_cleaner.go +++ b/pkg/compactor/blocks_cleaner.go @@ -23,13 +23,12 @@ import ( "github.com/prometheus/client_golang/prometheus/promauto" thanos_objstore "github.com/thanos-io/objstore" - "github.com/grafana/mimir/pkg/storage/tsdb/bucketindex" "github.com/grafana/pyroscope/pkg/objstore" "github.com/grafana/pyroscope/pkg/phlaredb/block" "github.com/grafana/pyroscope/pkg/phlaredb/bucket" + "github.com/grafana/pyroscope/pkg/phlaredb/bucketindex" "github.com/grafana/pyroscope/pkg/util" - util_log "github.com/grafana/pyroscope/pkg/util" - "github.com/grafana/pyroscope/pkg/util/validation" + "github.com/grafana/pyroscope/pkg/validation" ) const ( @@ -84,27 +83,27 @@ func NewBlocksCleaner(cfg BlocksCleanerConfig, bucketClient objstore.Bucket, own singleFlight: concurrency.NewLimitedConcurrencySingleFlight(cfg.CleanupConcurrency), logger: log.With(logger, "component", "cleaner"), runsStarted: promauto.With(reg).NewCounter(prometheus.CounterOpts{ - Name: "cortex_compactor_block_cleanup_started_total", + Name: "pyroscope_compactor_block_cleanup_started_total", Help: "Total number of blocks cleanup runs started.", }), runsCompleted: promauto.With(reg).NewCounter(prometheus.CounterOpts{ - Name: "cortex_compactor_block_cleanup_completed_total", + Name: "pyroscope_compactor_block_cleanup_completed_total", Help: "Total number of blocks cleanup runs successfully completed.", }), runsFailed: promauto.With(reg).NewCounter(prometheus.CounterOpts{ - Name: "cortex_compactor_block_cleanup_failed_total", + Name: "pyroscope_compactor_block_cleanup_failed_total", Help: "Total number of blocks cleanup runs failed.", }), runsLastSuccess: promauto.With(reg).NewGauge(prometheus.GaugeOpts{ - Name: "cortex_compactor_block_cleanup_last_successful_run_timestamp_seconds", + Name: "pyroscope_compactor_block_cleanup_last_successful_run_timestamp_seconds", Help: "Unix timestamp of the last successful blocks cleanup run.", }), blocksCleanedTotal: promauto.With(reg).NewCounter(prometheus.CounterOpts{ - Name: "cortex_compactor_blocks_cleaned_total", + Name: "pyroscope_compactor_blocks_cleaned_total", Help: "Total number of blocks deleted.", }), blocksFailedTotal: promauto.With(reg).NewCounter(prometheus.CounterOpts{ - Name: "cortex_compactor_block_cleanup_failures_total", + Name: "pyroscope_compactor_block_cleanup_failures_total", Help: "Total number of blocks failed to be deleted.", }), blocksMarkedForDeletion: promauto.With(reg).NewCounter(prometheus.CounterOpts{ @@ -118,23 +117,23 @@ func NewBlocksCleaner(cfg BlocksCleanerConfig, bucketClient objstore.Bucket, own ConstLabels: prometheus.Labels{"reason": "partial"}, }), - // The following metrics don't have the "cortex_compactor" prefix because not strictly related to + // The following metrics don't have the "pyroscope_compactor" prefix because not strictly related to // the compactor. They're just tracked by the compactor because it's the most logical place where these // metrics can be tracked. tenantBlocks: promauto.With(reg).NewGaugeVec(prometheus.GaugeOpts{ - Name: "cortex_bucket_blocks_count", + Name: "pyroscope_bucket_blocks_count", Help: "Total number of blocks in the bucket. Includes blocks marked for deletion, but not partial blocks.", }, []string{"user"}), tenantMarkedBlocks: promauto.With(reg).NewGaugeVec(prometheus.GaugeOpts{ - Name: "cortex_bucket_blocks_marked_for_deletion_count", + Name: "pyroscope_bucket_blocks_marked_for_deletion_count", Help: "Total number of blocks marked for deletion in the bucket.", }, []string{"user"}), tenantPartialBlocks: promauto.With(reg).NewGaugeVec(prometheus.GaugeOpts{ - Name: "cortex_bucket_blocks_partials_count", + Name: "pyroscope_bucket_blocks_partials_count", Help: "Total number of partial blocks.", }, []string{"user"}), tenantBucketIndexLastUpdate: promauto.With(reg).NewGaugeVec(prometheus.GaugeOpts{ - Name: "cortex_bucket_index_last_successful_update_timestamp_seconds", + Name: "pyroscope_bucket_index_last_successful_update_timestamp_seconds", Help: "Timestamp of the last successful update of a tenant's bucket index.", }, []string{"user"}), } @@ -246,7 +245,7 @@ func (c *BlocksCleaner) cleanUsers(ctx context.Context, allUsers []string, isDel return errors.Wrap(err, "check own user") } - userLogger := util_log.WithUserID(userID, logger) + userLogger := util.LoggerWithUserID(userID, logger) if isDeleted[userID] { return errors.Wrapf(c.deleteUserMarkedForDeletion(ctx, userID, userLogger), "failed to delete user marked for deletion: %s", userID) } @@ -270,13 +269,6 @@ func (c *BlocksCleaner) deleteRemainingData(ctx context.Context, userBucket objs level.Info(userLogger).Log("msg", "deleted marker files for tenant with no blocks remaining", "count", deleted) } - // Delete debug folder - if deleted, err := objstore.DeletePrefix(ctx, userBucket, block.DebugMetas, userLogger); err != nil { - return errors.Wrap(err, "failed to delete "+block.DebugMetas) - } else if deleted > 0 { - level.Info(userLogger).Log("msg", "deleted files under "+block.DebugMetas+" for tenant with no blocks remaining", "count", deleted) - } - return nil } @@ -364,14 +356,7 @@ func (c *BlocksCleaner) deleteUserMarkedForDeletion(ctx context.Context, userID level.Info(userLogger).Log("msg", "cleaning up remaining blocks data for tenant marked for deletion") - // Let's do final cleanup of tenant. - if deleted, err := objstore.DeletePrefix(ctx, userBucket, block.DebugMetas, userLogger); err != nil { - return errors.Wrap(err, "failed to delete "+block.DebugMetas) - } else if deleted > 0 { - level.Info(userLogger).Log("msg", "deleted files under "+block.DebugMetas+" for tenant marked for deletion", "count", deleted) - } - - // Tenant deletion mark file is inside Markers as well. + // Let's do final cleanup of markers. if deleted, err := objstore.DeletePrefix(ctx, userBucket, block.MarkersPathname, userLogger); err != nil { return errors.Wrap(err, "failed to delete marker files") } else if deleted > 0 { @@ -597,7 +582,7 @@ func listBlocksOutsideRetentionPeriod(idx *bucketindex.Index, threshold time.Tim } for _, b := range idx.Blocks { - maxTime := time.Unix(b.MaxTime/1000, 0) + maxTime := time.Unix(int64(b.MaxTime)/1000, 0) if maxTime.Before(threshold) { if _, isMarked := marked[b.ID]; !isMarked { result = append(result, b) diff --git a/pkg/compactor/compactor_http.go b/pkg/compactor/compactor_http.go index 227d0894c3..3fd5190fdf 100644 --- a/pkg/compactor/compactor_http.go +++ b/pkg/compactor/compactor_http.go @@ -13,7 +13,7 @@ import ( "github.com/go-kit/log/level" "github.com/grafana/dskit/services" - util_log "github.com/grafana/mimir/pkg/util/log" + util_log "github.com/grafana/pyroscope/pkg/util" ) var ( diff --git a/pkg/compactor/compactor_ring.go b/pkg/compactor/compactor_ring.go index b2872d6409..02813bc86d 100644 --- a/pkg/compactor/compactor_ring.go +++ b/pkg/compactor/compactor_ring.go @@ -13,7 +13,7 @@ import ( "github.com/go-kit/log" "github.com/grafana/dskit/ring" - "github.com/grafana/mimir/pkg/util" + "github.com/grafana/pyroscope/pkg/util" ) const ( diff --git a/pkg/compactor/job_sorting_test.go b/pkg/compactor/job_sorting_test.go index 33c9ea2259..94e83a5475 100644 --- a/pkg/compactor/job_sorting_test.go +++ b/pkg/compactor/job_sorting_test.go @@ -9,10 +9,10 @@ import ( "testing" "github.com/oklog/ulid" - "github.com/prometheus/prometheus/tsdb" + "github.com/prometheus/common/model" "github.com/stretchr/testify/assert" - "github.com/grafana/mimir/pkg/storage/tsdb/block" + "github.com/grafana/pyroscope/pkg/phlaredb/block" ) func TestSortJobsBySmallestRangeOldestBlocksFirst(t *testing.T) { @@ -137,10 +137,8 @@ func TestSortJobsByNewestBlocksFirst(t *testing.T) { func mockMetaWithMinMax(id ulid.ULID, minTime, maxTime int64) *block.Meta { return &block.Meta{ - BlockMeta: tsdb.BlockMeta{ - ULID: id, - MinTime: minTime, - MaxTime: maxTime, - }, + ULID: id, + MinTime: model.Time(minTime), + MaxTime: model.Time(maxTime), } } diff --git a/pkg/compactor/job_test.go b/pkg/compactor/job_test.go index 35df7e65d0..517767ceca 100644 --- a/pkg/compactor/job_test.go +++ b/pkg/compactor/job_test.go @@ -19,19 +19,19 @@ import ( "github.com/stretchr/testify/require" "github.com/thanos-io/objstore" - "github.com/grafana/mimir/pkg/storage/bucket" - "github.com/grafana/mimir/pkg/storage/tsdb/block" + pyroscope_objstore "github.com/grafana/pyroscope/pkg/objstore" + "github.com/grafana/pyroscope/pkg/phlaredb/block" ) func TestJob_MinCompactionLevel(t *testing.T) { job := NewJob("user-1", "group-1", labels.EmptyLabels(), 0, true, 2, "shard-1") - require.NoError(t, job.AppendMeta(&block.Meta{BlockMeta: tsdb.BlockMeta{ULID: ulid.MustNew(1, nil), Compaction: tsdb.BlockMetaCompaction{Level: 2}}})) + require.NoError(t, job.AppendMeta(&block.Meta{ULID: ulid.MustNew(1, nil), Compaction: tsdb.BlockMetaCompaction{Level: 2}})) assert.Equal(t, 2, job.MinCompactionLevel()) - require.NoError(t, job.AppendMeta(&block.Meta{BlockMeta: tsdb.BlockMeta{ULID: ulid.MustNew(2, nil), Compaction: tsdb.BlockMetaCompaction{Level: 3}}})) + require.NoError(t, job.AppendMeta(&block.Meta{ULID: ulid.MustNew(2, nil), Compaction: tsdb.BlockMetaCompaction{Level: 3}})) assert.Equal(t, 2, job.MinCompactionLevel()) - require.NoError(t, job.AppendMeta(&block.Meta{BlockMeta: tsdb.BlockMeta{ULID: ulid.MustNew(3, nil), Compaction: tsdb.BlockMetaCompaction{Level: 1}}})) + require.NoError(t, job.AppendMeta(&block.Meta{ULID: ulid.MustNew(3, nil), Compaction: tsdb.BlockMetaCompaction{Level: 1}})) assert.Equal(t, 1, job.MinCompactionLevel()) } @@ -43,12 +43,12 @@ func TestJobWaitPeriodElapsed(t *testing.T) { } // Blocks with compaction level 1. - meta1 := &block.Meta{BlockMeta: tsdb.BlockMeta{ULID: ulid.MustNew(1, nil), Compaction: tsdb.BlockMetaCompaction{Level: 1}}} - meta2 := &block.Meta{BlockMeta: tsdb.BlockMeta{ULID: ulid.MustNew(2, nil), Compaction: tsdb.BlockMetaCompaction{Level: 1}}} + meta1 := &block.Meta{ULID: ulid.MustNew(1, nil), Compaction: tsdb.BlockMetaCompaction{Level: 1}} + meta2 := &block.Meta{ULID: ulid.MustNew(2, nil), Compaction: tsdb.BlockMetaCompaction{Level: 1}} // Blocks with compaction level 2. - meta3 := &block.Meta{BlockMeta: tsdb.BlockMeta{ULID: ulid.MustNew(3, nil), Compaction: tsdb.BlockMetaCompaction{Level: 2}}} - meta4 := &block.Meta{BlockMeta: tsdb.BlockMeta{ULID: ulid.MustNew(4, nil), Compaction: tsdb.BlockMetaCompaction{Level: 2}}} + meta3 := &block.Meta{ULID: ulid.MustNew(3, nil), Compaction: tsdb.BlockMetaCompaction{Level: 2}} + meta4 := &block.Meta{ULID: ulid.MustNew(4, nil), Compaction: tsdb.BlockMetaCompaction{Level: 2}} tests := map[string]struct { waitPeriod time.Duration @@ -114,7 +114,7 @@ func TestJobWaitPeriodElapsed(t *testing.T) { require.NoError(t, job.AppendMeta(b.meta)) } - userBucket := &bucket.ClientMock{} + userBucket := &pyroscope_objstore.ClientMock{} for _, b := range testData.jobBlocks { userBucket.MockAttributes(path.Join(b.meta.ULID.String(), block.MetaFilename), b.attrs, b.attrsErr) } diff --git a/pkg/compactor/label_remover_filter_test.go b/pkg/compactor/label_remover_filter_test.go index 0866f69e99..7142ac9f73 100644 --- a/pkg/compactor/label_remover_filter_test.go +++ b/pkg/compactor/label_remover_filter_test.go @@ -13,8 +13,7 @@ import ( "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" - mimir_tsdb "github.com/grafana/mimir/pkg/storage/tsdb" - "github.com/grafana/mimir/pkg/storage/tsdb/block" + "github.com/grafana/pyroscope/pkg/phlaredb/block" ) func TestLabelRemoverFilter(t *testing.T) { @@ -28,25 +27,25 @@ func TestLabelRemoverFilter(t *testing.T) { expected map[ulid.ULID]map[string]string }{ "should remove configured labels": { - labels: []string{mimir_tsdb.DeprecatedIngesterIDExternalLabel}, + labels: []string{block.HostnameLabel}, input: map[ulid.ULID]map[string]string{ - block1: {mimir_tsdb.DeprecatedIngesterIDExternalLabel: "ingester-0", mimir_tsdb.DeprecatedTenantIDExternalLabel: "user-1"}, - block2: {mimir_tsdb.DeprecatedIngesterIDExternalLabel: "ingester-0", mimir_tsdb.DeprecatedTenantIDExternalLabel: "user-1"}, - block3: {mimir_tsdb.DeprecatedIngesterIDExternalLabel: "ingester-0", mimir_tsdb.DeprecatedTenantIDExternalLabel: "user-1"}, + block1: {block.HostnameLabel: "ingester-0", "foo": "user-1"}, + block2: {block.HostnameLabel: "ingester-0", "foo": "user-1"}, + block3: {block.HostnameLabel: "ingester-0", "foo": "user-1"}, }, expected: map[ulid.ULID]map[string]string{ - block1: {mimir_tsdb.DeprecatedTenantIDExternalLabel: "user-1"}, - block2: {mimir_tsdb.DeprecatedTenantIDExternalLabel: "user-1"}, - block3: {mimir_tsdb.DeprecatedTenantIDExternalLabel: "user-1"}, + block1: {"foo": "user-1"}, + block2: {"foo": "user-1"}, + block3: {"foo": "user-1"}, }, }, "should remove configured labels 2": { - labels: []string{mimir_tsdb.DeprecatedIngesterIDExternalLabel, mimir_tsdb.DeprecatedTenantIDExternalLabel}, + labels: []string{block.HostnameLabel, "foo"}, input: map[ulid.ULID]map[string]string{ - block1: {mimir_tsdb.DeprecatedIngesterIDExternalLabel: "ingester-0", mimir_tsdb.DeprecatedTenantIDExternalLabel: "user-1"}, - block2: {mimir_tsdb.DeprecatedIngesterIDExternalLabel: "ingester-0", mimir_tsdb.DeprecatedTenantIDExternalLabel: "user-1"}, - block3: {mimir_tsdb.DeprecatedIngesterIDExternalLabel: "ingester-0", mimir_tsdb.DeprecatedTenantIDExternalLabel: "user-1"}, + block1: {block.HostnameLabel: "ingester-0", "foo": "user-1"}, + block2: {block.HostnameLabel: "ingester-0", "foo": "user-1"}, + block3: {block.HostnameLabel: "ingester-0", "foo": "user-1"}, }, expected: map[ulid.ULID]map[string]string{ block1: {}, @@ -60,7 +59,7 @@ func TestLabelRemoverFilter(t *testing.T) { t.Run(testName, func(t *testing.T) { metas := map[ulid.ULID]*block.Meta{} for id, lbls := range testData.input { - metas[id] = &block.Meta{Thanos: block.ThanosMeta{Labels: lbls}} + metas[id] = &block.Meta{Labels: lbls} } f := NewLabelRemoverFilter(testData.labels) @@ -70,7 +69,7 @@ func TestLabelRemoverFilter(t *testing.T) { for expectedID, expectedLbls := range testData.expected { assert.NotNil(t, metas[expectedID]) - assert.Equal(t, expectedLbls, metas[expectedID].Thanos.Labels) + assert.Equal(t, expectedLbls, metas[expectedID].Labels) } }) } diff --git a/pkg/compactor/shard_aware_deduplicate_filter.go b/pkg/compactor/shard_aware_deduplicate_filter.go index 906ab183cf..a42404bf5b 100644 --- a/pkg/compactor/shard_aware_deduplicate_filter.go +++ b/pkg/compactor/shard_aware_deduplicate_filter.go @@ -11,9 +11,8 @@ import ( "github.com/oklog/ulid" - "github.com/grafana/mimir/pkg/storage/sharding" - "github.com/grafana/mimir/pkg/storage/tsdb" - "github.com/grafana/mimir/pkg/storage/tsdb/block" + "github.com/grafana/pyroscope/pkg/phlaredb/block" + "github.com/grafana/pyroscope/pkg/phlaredb/sharding" ) const duplicateMeta = "duplicate" @@ -37,7 +36,7 @@ func (f *ShardAwareDeduplicateFilter) Filter(ctx context.Context, metas map[ulid metasByResolution := make(map[int64][]*block.Meta) for _, meta := range metas { - res := meta.Thanos.Downsample.Resolution + res := meta.Downsample.Resolution metasByResolution[res] = append(metasByResolution[res], meta) } @@ -172,7 +171,7 @@ type blockWithSuccessors struct { func newBlockWithSuccessors(m *block.Meta) *blockWithSuccessors { b := &blockWithSuccessors{meta: m} if m != nil { - b.shardID = m.Thanos.Labels[tsdb.CompactorShardIDExternalLabel] + b.shardID = m.Labels[sharding.CompactorShardIDLabel] b.sources = make(map[ulid.ULID]struct{}, len(m.Compaction.Sources)) for _, bid := range m.Compaction.Sources { b.sources[bid] = struct{}{} diff --git a/pkg/compactor/shard_aware_deduplicate_filter_test.go b/pkg/compactor/shard_aware_deduplicate_filter_test.go index ffb19ffaf1..330afa1fe3 100644 --- a/pkg/compactor/shard_aware_deduplicate_filter_test.go +++ b/pkg/compactor/shard_aware_deduplicate_filter_test.go @@ -16,9 +16,9 @@ import ( "github.com/prometheus/prometheus/tsdb" "github.com/stretchr/testify/require" - mimir_tsdb "github.com/grafana/mimir/pkg/storage/tsdb" - "github.com/grafana/mimir/pkg/storage/tsdb/block" - "github.com/grafana/mimir/pkg/util/extprom" + "github.com/grafana/pyroscope/pkg/phlaredb/block" + "github.com/grafana/pyroscope/pkg/phlaredb/sharding" + "github.com/grafana/pyroscope/pkg/util/extprom" ) func ULID(i int) ulid.ULID { return ulid.MustNew(uint64(i), nil) } @@ -348,19 +348,16 @@ func TestShardAwareDeduplicateFilter_Filter(t *testing.T) { inputLen := len(tcase.input) for id, metaInfo := range tcase.input { metas[id] = &block.Meta{ - BlockMeta: tsdb.BlockMeta{ - ULID: id, - Compaction: tsdb.BlockMetaCompaction{ - Sources: metaInfo.sources, - }, + ULID: id, + Compaction: tsdb.BlockMetaCompaction{ + Sources: metaInfo.sources, + }, + + Downsample: block.Downsample{ + Resolution: metaInfo.resolution, }, - Thanos: block.ThanosMeta{ - Downsample: block.ThanosDownsample{ - Resolution: metaInfo.resolution, - }, - Labels: map[string]string{ - mimir_tsdb.CompactorShardIDExternalLabel: metaInfo.shardID, - }, + Labels: map[string]string{ + sharding.CompactorShardIDLabel: metaInfo.shardID, }, } } @@ -409,9 +406,7 @@ func BenchmarkDeduplicateFilter_Filter(b *testing.B) { count++ cases[0][id] = &block.Meta{ - BlockMeta: tsdb.BlockMeta{ - ULID: id, - }, + ULID: id, } for j := 0; j < 100; j++ { @@ -430,12 +425,9 @@ func BenchmarkDeduplicateFilter_Filter(b *testing.B) { id := ulid.MustNew(count, nil) count++ cases[1][id] = &block.Meta{ - BlockMeta: tsdb.BlockMeta{ - ULID: id, - }, - Thanos: block.ThanosMeta{ - Downsample: block.ThanosDownsample{Resolution: res}, - }, + ULID: id, + + Downsample: block.Downsample{Resolution: res}, } for j := 0; j < 100; j++ { cases[1][id].Compaction.Sources = append(cases[1][id].Compaction.Sources, ulid.MustNew(count, nil)) diff --git a/pkg/compactor/split_merge_grouper.go b/pkg/compactor/split_merge_grouper.go index 89c616df12..d3df772a67 100644 --- a/pkg/compactor/split_merge_grouper.go +++ b/pkg/compactor/split_merge_grouper.go @@ -16,9 +16,8 @@ import ( "github.com/pkg/errors" "github.com/prometheus/prometheus/model/labels" - "github.com/grafana/mimir/pkg/storage/sharding" - mimir_tsdb "github.com/grafana/mimir/pkg/storage/tsdb" - "github.com/grafana/mimir/pkg/storage/tsdb/block" + "github.com/grafana/pyroscope/pkg/phlaredb/block" + "github.com/grafana/pyroscope/pkg/phlaredb/sharding" ) type SplitAndMergeGrouper struct { @@ -66,7 +65,7 @@ func (g *SplitAndMergeGrouper) Groups(blocks map[ulid.ULID]*block.Meta) (res []* // The group key is used by the compactor as a unique identifier of the compaction job. // Its content is not important for the compactor, but uniqueness must be guaranteed. groupKey := fmt.Sprintf("%s-%s-%s-%d-%d", - defaultGroupKeyWithoutShardID(job.blocks[0].Thanos), + defaultGroupKeyWithoutShardID(job.blocks[0]), job.stage, job.shardID, job.rangeStart, @@ -74,8 +73,8 @@ func (g *SplitAndMergeGrouper) Groups(blocks map[ulid.ULID]*block.Meta) (res []* // All the blocks within the same group have the same downsample // resolution and external labels. - resolution := job.blocks[0].Thanos.Downsample.Resolution - externalLabels := labels.FromMap(job.blocks[0].Thanos.Labels) + resolution := job.blocks[0].Downsample.Resolution + externalLabels := labels.FromMap(job.blocks[0].Labels) compactionJob := NewJob( g.userID, @@ -112,7 +111,7 @@ func planCompaction(userID string, blocks []*block.Meta, ranges []int64, shardCo // considering the shard ID in the external labels (because will be checked later). mainGroups := map[string][]*block.Meta{} for _, b := range blocks { - key := defaultGroupKeyWithoutShardID(b.Thanos) + key := defaultGroupKeyWithoutShardID(b) mainGroups[key] = append(mainGroups[key], b) } @@ -171,7 +170,7 @@ func planCompaction(userID string, blocks []*block.Meta, ranges []int64, shardCo } // The sharding key could be equal but external labels can still be different. - return defaultGroupKeyWithoutShardID(jobs[i].blocks[0].Thanos) < defaultGroupKeyWithoutShardID(jobs[j].blocks[0].Thanos) + return defaultGroupKeyWithoutShardID(jobs[i].blocks[0]) < defaultGroupKeyWithoutShardID(jobs[j].blocks[0]) }) return jobs @@ -234,8 +233,8 @@ func planSplitting(userID string, group blocksGroup, splitGroups uint32) []*job // The number of source blocks could be very large so, to have a better horizontal scaling, we should group // the source blocks into N groups (where N = number of shards) and create a job for each group of blocks to // merge and split. - for _, block := range blocks { - splitGroup := mimir_tsdb.HashBlockID(block.ULID) % splitGroups + for _, blk := range blocks { + splitGroup := block.HashBlockID(blk.ULID) % splitGroups if jobs[splitGroup] == nil { jobs[splitGroup] = &job{ @@ -249,7 +248,7 @@ func planSplitting(userID string, group blocksGroup, splitGroups uint32) []*job } } - jobs[splitGroup].blocks = append(jobs[splitGroup].blocks, block) + jobs[splitGroup].blocks = append(jobs[splitGroup].blocks, blk) } // Convert the output. @@ -270,7 +269,7 @@ func groupBlocksByShardID(blocks []*block.Meta) map[string][]*block.Meta { for _, block := range blocks { // If the label doesn't exist, we'll group together such blocks using an // empty string as shard ID. - shardID := block.Thanos.Labels[mimir_tsdb.CompactorShardIDExternalLabel] + shardID := block.Labels[sharding.CompactorShardIDLabel] groups[shardID] = append(groups[shardID], block) } @@ -296,7 +295,7 @@ func groupBlocksByRange(blocks []*block.Meta, tr int64) []blocksGroup { // Skip blocks that don't fall into the range. This can happen via mis-alignment or // by being the multiple of the intended range. - if m.MaxTime > group.rangeEnd { + if int64(m.MaxTime) > group.rangeEnd { i++ continue } @@ -305,13 +304,13 @@ func groupBlocksByRange(blocks []*block.Meta, tr int64) []blocksGroup { for ; i < len(blocks); i++ { // If the block does not start within this group, then we should break the iteration // and move it to the next group. - if blocks[i].MinTime >= group.rangeEnd { + if int64(blocks[i].MinTime) >= group.rangeEnd { break } // If the block doesn't fall into this group, but it started within this group then it // means it spans across multiple ranges and we should skip it. - if blocks[i].MaxTime > group.rangeEnd { + if int64(blocks[i].MaxTime) > group.rangeEnd { continue } @@ -328,21 +327,20 @@ func groupBlocksByRange(blocks []*block.Meta, tr int64) []blocksGroup { func getRangeStart(m *block.Meta, tr int64) int64 { // Compute start of aligned time range of size tr closest to the current block's start. - // This code has been copied from TSDB. if m.MinTime >= 0 { - return tr * (m.MinTime / tr) + return tr * (int64(m.MinTime) / tr) } - return tr * ((m.MinTime - tr + 1) / tr) + return tr * ((int64(m.MinTime) - tr + 1) / tr) } func sortMetasByMinTime(metas []*block.Meta) []*block.Meta { sort.Slice(metas, func(i, j int) bool { - if metas[i].BlockMeta.MinTime != metas[j].BlockMeta.MinTime { - return metas[i].BlockMeta.MinTime < metas[j].BlockMeta.MinTime + if metas[i].MinTime != metas[j].MinTime { + return metas[i].MinTime < metas[j].MinTime } // Compare labels in case of same MinTime to get stable results. - return labels.Compare(labels.FromMap(metas[i].Thanos.Labels), labels.FromMap(metas[j].Thanos.Labels)) < 0 + return labels.Compare(labels.FromMap(metas[i].Labels), labels.FromMap(metas[j].Labels)) < 0 }) return metas @@ -353,8 +351,8 @@ func getMaxTime(blocks []*block.Meta) int64 { maxTime := int64(math.MinInt64) for _, block := range blocks { - if block.MaxTime > maxTime { - maxTime = block.MaxTime + if int64(block.MaxTime) > maxTime { + maxTime = int64(block.MaxTime) } } @@ -363,15 +361,15 @@ func getMaxTime(blocks []*block.Meta) int64 { // defaultGroupKeyWithoutShardID returns the default group key excluding ShardIDLabelName // when computing it. -func defaultGroupKeyWithoutShardID(meta block.ThanosMeta) string { +func defaultGroupKeyWithoutShardID(meta *block.Meta) string { return defaultGroupKey(meta.Downsample.Resolution, labelsWithoutShard(meta.Labels)) } -// Return labels built from base, but without any label with name equal to mimir_tsdb.CompactorShardIDExternalLabel. +// Return labels built from base, but without any label with name equal to sharding.CompactorShardIDExternalLabel. func labelsWithoutShard(base map[string]string) labels.Labels { b := labels.NewScratchBuilder(len(base)) for k, v := range base { - if k != mimir_tsdb.CompactorShardIDExternalLabel { + if k != sharding.CompactorShardIDLabel { b.Add(k, v) } } diff --git a/pkg/compactor/split_merge_grouper_test.go b/pkg/compactor/split_merge_grouper_test.go index b1e25a5d1a..05f1e331f2 100644 --- a/pkg/compactor/split_merge_grouper_test.go +++ b/pkg/compactor/split_merge_grouper_test.go @@ -9,11 +9,10 @@ import ( "testing" "github.com/oklog/ulid" - "github.com/prometheus/prometheus/tsdb" "github.com/stretchr/testify/assert" - mimir_tsdb "github.com/grafana/mimir/pkg/storage/tsdb" - "github.com/grafana/mimir/pkg/storage/tsdb/block" + "github.com/grafana/pyroscope/pkg/phlaredb/block" + "github.com/grafana/pyroscope/pkg/phlaredb/sharding" ) func TestPlanCompaction(t *testing.T) { @@ -46,14 +45,14 @@ func TestPlanCompaction(t *testing.T) { ranges: []int64{20, 40}, shardCount: 1, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 0, MaxTime: 20}}, + {ULID: block1, MinTime: 0, MaxTime: 20}, }, expected: []*job{ {userID: userID, stage: stageSplit, shardID: "1_of_1", blocksGroup: blocksGroup{ rangeStart: 0, rangeEnd: 20, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 0, MaxTime: 20}}, + {ULID: block1, MinTime: 0, MaxTime: 20}, }, }}, }, @@ -62,14 +61,14 @@ func TestPlanCompaction(t *testing.T) { ranges: []int64{20, 40}, shardCount: 1, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 10, MaxTime: 20}}, + {ULID: block1, MinTime: 10, MaxTime: 20}, }, expected: []*job{ {userID: userID, stage: stageSplit, shardID: "1_of_1", blocksGroup: blocksGroup{ rangeStart: 0, rangeEnd: 20, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 10, MaxTime: 20}}, + {ULID: block1, MinTime: 10, MaxTime: 20}, }, }}, }, @@ -78,7 +77,7 @@ func TestPlanCompaction(t *testing.T) { ranges: []int64{20, 40}, shardCount: 0, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 0, MaxTime: 20}}, + {ULID: block1, MinTime: 0, MaxTime: 20}, }, expected: []*job{}, }, @@ -86,16 +85,16 @@ func TestPlanCompaction(t *testing.T) { ranges: []int64{10, 20}, shardCount: 1, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 10, MaxTime: 20}}, - {BlockMeta: tsdb.BlockMeta{ULID: block2, MinTime: 10, MaxTime: 20}}, + {ULID: block1, MinTime: 10, MaxTime: 20}, + {ULID: block2, MinTime: 10, MaxTime: 20}, }, expected: []*job{ {userID: userID, stage: stageSplit, shardID: "1_of_1", blocksGroup: blocksGroup{ rangeStart: 10, rangeEnd: 20, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 10, MaxTime: 20}}, - {BlockMeta: tsdb.BlockMeta{ULID: block2, MinTime: 10, MaxTime: 20}}, + {ULID: block1, MinTime: 10, MaxTime: 20}, + {ULID: block2, MinTime: 10, MaxTime: 20}, }, }}, }, @@ -105,27 +104,27 @@ func TestPlanCompaction(t *testing.T) { shardCount: 1, blocks: []*block.Meta{ // 1st level range [0, 10] - {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 0, MaxTime: 10}}, - {BlockMeta: tsdb.BlockMeta{ULID: block2, MinTime: 0, MaxTime: 10}}, + {ULID: block1, MinTime: 0, MaxTime: 10}, + {ULID: block2, MinTime: 0, MaxTime: 10}, // 1st level range [10, 20] - {BlockMeta: tsdb.BlockMeta{ULID: block3, MinTime: 10, MaxTime: 20}}, - {BlockMeta: tsdb.BlockMeta{ULID: block4, MinTime: 10, MaxTime: 20}}, + {ULID: block3, MinTime: 10, MaxTime: 20}, + {ULID: block4, MinTime: 10, MaxTime: 20}, }, expected: []*job{ {userID: userID, stage: stageSplit, shardID: "1_of_1", blocksGroup: blocksGroup{ rangeStart: 0, rangeEnd: 10, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 0, MaxTime: 10}}, - {BlockMeta: tsdb.BlockMeta{ULID: block2, MinTime: 0, MaxTime: 10}}, + {ULID: block1, MinTime: 0, MaxTime: 10}, + {ULID: block2, MinTime: 0, MaxTime: 10}, }, }}, {userID: userID, stage: stageSplit, shardID: "1_of_1", blocksGroup: blocksGroup{ rangeStart: 10, rangeEnd: 20, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{ULID: block3, MinTime: 10, MaxTime: 20}}, - {BlockMeta: tsdb.BlockMeta{ULID: block4, MinTime: 10, MaxTime: 20}}, + {ULID: block3, MinTime: 10, MaxTime: 20}, + {ULID: block4, MinTime: 10, MaxTime: 20}, }, }}, }, @@ -136,27 +135,27 @@ func TestPlanCompaction(t *testing.T) { splitGroups: 1, blocks: []*block.Meta{ // 1st level range [0, 10] - {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 0, MaxTime: 10}}, - {BlockMeta: tsdb.BlockMeta{ULID: block2, MinTime: 0, MaxTime: 10}}, + {ULID: block1, MinTime: 0, MaxTime: 10}, + {ULID: block2, MinTime: 0, MaxTime: 10}, // 1st level range [10, 20] - {BlockMeta: tsdb.BlockMeta{ULID: block3, MinTime: 10, MaxTime: 20}}, - {BlockMeta: tsdb.BlockMeta{ULID: block4, MinTime: 10, MaxTime: 20}}, + {ULID: block3, MinTime: 10, MaxTime: 20}, + {ULID: block4, MinTime: 10, MaxTime: 20}, }, expected: []*job{ {userID: userID, stage: stageSplit, shardID: "1_of_1", blocksGroup: blocksGroup{ rangeStart: 0, rangeEnd: 10, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 0, MaxTime: 10}}, - {BlockMeta: tsdb.BlockMeta{ULID: block2, MinTime: 0, MaxTime: 10}}, + {ULID: block1, MinTime: 0, MaxTime: 10}, + {ULID: block2, MinTime: 0, MaxTime: 10}, }, }}, {userID: userID, stage: stageSplit, shardID: "1_of_1", blocksGroup: blocksGroup{ rangeStart: 10, rangeEnd: 20, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{ULID: block3, MinTime: 10, MaxTime: 20}}, - {BlockMeta: tsdb.BlockMeta{ULID: block4, MinTime: 10, MaxTime: 20}}, + {ULID: block3, MinTime: 10, MaxTime: 20}, + {ULID: block4, MinTime: 10, MaxTime: 20}, }, }}, }, @@ -167,39 +166,39 @@ func TestPlanCompaction(t *testing.T) { splitGroups: 2, blocks: []*block.Meta{ // 1st level range [0, 10] - {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 0, MaxTime: 10}}, - {BlockMeta: tsdb.BlockMeta{ULID: block2, MinTime: 0, MaxTime: 10}}, + {ULID: block1, MinTime: 0, MaxTime: 10}, + {ULID: block2, MinTime: 0, MaxTime: 10}, // 1st level range [10, 20] - {BlockMeta: tsdb.BlockMeta{ULID: block3, MinTime: 10, MaxTime: 20}}, - {BlockMeta: tsdb.BlockMeta{ULID: block4, MinTime: 10, MaxTime: 20}}, + {ULID: block3, MinTime: 10, MaxTime: 20}, + {ULID: block4, MinTime: 10, MaxTime: 20}, }, expected: []*job{ {userID: userID, stage: stageSplit, shardID: "1_of_2", blocksGroup: blocksGroup{ rangeStart: 0, rangeEnd: 10, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 0, MaxTime: 10}}, + {ULID: block1, MinTime: 0, MaxTime: 10}, }, }}, {userID: userID, stage: stageSplit, shardID: "2_of_2", blocksGroup: blocksGroup{ rangeStart: 0, rangeEnd: 10, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{ULID: block2, MinTime: 0, MaxTime: 10}}, + {ULID: block2, MinTime: 0, MaxTime: 10}, }, }}, {userID: userID, stage: stageSplit, shardID: "1_of_2", blocksGroup: blocksGroup{ rangeStart: 10, rangeEnd: 20, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{ULID: block3, MinTime: 10, MaxTime: 20}}, + {ULID: block3, MinTime: 10, MaxTime: 20}, }, }}, {userID: userID, stage: stageSplit, shardID: "2_of_2", blocksGroup: blocksGroup{ rangeStart: 10, rangeEnd: 20, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{ULID: block4, MinTime: 10, MaxTime: 20}}, + {ULID: block4, MinTime: 10, MaxTime: 20}, }, }}, }, @@ -209,27 +208,27 @@ func TestPlanCompaction(t *testing.T) { shardCount: 0, blocks: []*block.Meta{ // 1st level range [0, 10] - {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 0, MaxTime: 10}}, - {BlockMeta: tsdb.BlockMeta{ULID: block2, MinTime: 0, MaxTime: 10}}, + {ULID: block1, MinTime: 0, MaxTime: 10}, + {ULID: block2, MinTime: 0, MaxTime: 10}, // 1st level range [10, 20] - {BlockMeta: tsdb.BlockMeta{ULID: block3, MinTime: 10, MaxTime: 20}}, - {BlockMeta: tsdb.BlockMeta{ULID: block4, MinTime: 10, MaxTime: 20}}, + {ULID: block3, MinTime: 10, MaxTime: 20}, + {ULID: block4, MinTime: 10, MaxTime: 20}, }, expected: []*job{ {userID: userID, stage: stageMerge, blocksGroup: blocksGroup{ rangeStart: 0, rangeEnd: 10, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 0, MaxTime: 10}}, - {BlockMeta: tsdb.BlockMeta{ULID: block2, MinTime: 0, MaxTime: 10}}, + {ULID: block1, MinTime: 0, MaxTime: 10}, + {ULID: block2, MinTime: 0, MaxTime: 10}, }, }}, {userID: userID, stage: stageMerge, blocksGroup: blocksGroup{ rangeStart: 10, rangeEnd: 20, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{ULID: block3, MinTime: 10, MaxTime: 20}}, - {BlockMeta: tsdb.BlockMeta{ULID: block4, MinTime: 10, MaxTime: 20}}, + {ULID: block3, MinTime: 10, MaxTime: 20}, + {ULID: block4, MinTime: 10, MaxTime: 20}, }, }}, }, @@ -239,37 +238,37 @@ func TestPlanCompaction(t *testing.T) { shardCount: 2, blocks: []*block.Meta{ // 2nd level range [0, 20] - {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 0, MaxTime: 10}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_2"}}}, - {BlockMeta: tsdb.BlockMeta{ULID: block2, MinTime: 10, MaxTime: 20}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_2"}}}, - {BlockMeta: tsdb.BlockMeta{ULID: block3, MinTime: 0, MaxTime: 10}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "2_of_2"}}}, - {BlockMeta: tsdb.BlockMeta{ULID: block4, MinTime: 10, MaxTime: 20}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "2_of_2"}}}, + {ULID: block1, MinTime: 0, MaxTime: 10, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_2"}}, + {ULID: block2, MinTime: 10, MaxTime: 20, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_2"}}, + {ULID: block3, MinTime: 0, MaxTime: 10, Labels: map[string]string{sharding.CompactorShardIDLabel: "2_of_2"}}, + {ULID: block4, MinTime: 10, MaxTime: 20, Labels: map[string]string{sharding.CompactorShardIDLabel: "2_of_2"}}, // 2nd level range [20, 40] - {BlockMeta: tsdb.BlockMeta{ULID: block5, MinTime: 20, MaxTime: 30}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_2"}}}, - {BlockMeta: tsdb.BlockMeta{ULID: block6, MinTime: 30, MaxTime: 40}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_2"}}}, + {ULID: block5, MinTime: 20, MaxTime: 30, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_2"}}, + {ULID: block6, MinTime: 30, MaxTime: 40, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_2"}}, }, expected: []*job{ {userID: userID, stage: stageMerge, shardID: "1_of_2", blocksGroup: blocksGroup{ rangeStart: 0, rangeEnd: 20, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 0, MaxTime: 10}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_2"}}}, - {BlockMeta: tsdb.BlockMeta{ULID: block2, MinTime: 10, MaxTime: 20}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_2"}}}, + {ULID: block1, MinTime: 0, MaxTime: 10, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_2"}}, + {ULID: block2, MinTime: 10, MaxTime: 20, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_2"}}, }, }}, {userID: userID, stage: stageMerge, shardID: "2_of_2", blocksGroup: blocksGroup{ rangeStart: 0, rangeEnd: 20, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{ULID: block3, MinTime: 0, MaxTime: 10}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "2_of_2"}}}, - {BlockMeta: tsdb.BlockMeta{ULID: block4, MinTime: 10, MaxTime: 20}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "2_of_2"}}}, + {ULID: block3, MinTime: 0, MaxTime: 10, Labels: map[string]string{sharding.CompactorShardIDLabel: "2_of_2"}}, + {ULID: block4, MinTime: 10, MaxTime: 20, Labels: map[string]string{sharding.CompactorShardIDLabel: "2_of_2"}}, }, }}, {userID: userID, stage: stageMerge, shardID: "1_of_2", blocksGroup: blocksGroup{ rangeStart: 20, rangeEnd: 40, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{ULID: block5, MinTime: 20, MaxTime: 30}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_2"}}}, - {BlockMeta: tsdb.BlockMeta{ULID: block6, MinTime: 30, MaxTime: 40}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_2"}}}, + {ULID: block5, MinTime: 20, MaxTime: 30, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_2"}}, + {ULID: block6, MinTime: 30, MaxTime: 40, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_2"}}, }, }}, }, @@ -279,29 +278,29 @@ func TestPlanCompaction(t *testing.T) { shardCount: 2, blocks: []*block.Meta{ // 2nd level range [0, 20] - {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 0, MaxTime: 10}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_2"}}}, - {BlockMeta: tsdb.BlockMeta{ULID: block2, MinTime: 10, MaxTime: 20}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_2"}}}, - {BlockMeta: tsdb.BlockMeta{ULID: block3, MinTime: 0, MaxTime: 10}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "2_of_2"}}}, - {BlockMeta: tsdb.BlockMeta{ULID: block4, MinTime: 10, MaxTime: 20}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "2_of_2"}}}, + {ULID: block1, MinTime: 0, MaxTime: 10, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_2"}}, + {ULID: block2, MinTime: 10, MaxTime: 20, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_2"}}, + {ULID: block3, MinTime: 0, MaxTime: 10, Labels: map[string]string{sharding.CompactorShardIDLabel: "2_of_2"}}, + {ULID: block4, MinTime: 10, MaxTime: 20, Labels: map[string]string{sharding.CompactorShardIDLabel: "2_of_2"}}, // 2nd level range [20, 40] - {BlockMeta: tsdb.BlockMeta{ULID: block5, MinTime: 20, MaxTime: 40}}, - {BlockMeta: tsdb.BlockMeta{ULID: block6, MinTime: 20, MaxTime: 40}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_2"}}}, + {ULID: block5, MinTime: 20, MaxTime: 40}, + {ULID: block6, MinTime: 20, MaxTime: 40, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_2"}}, }, expected: []*job{ {userID: userID, stage: stageMerge, shardID: "1_of_2", blocksGroup: blocksGroup{ rangeStart: 0, rangeEnd: 20, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 0, MaxTime: 10}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_2"}}}, - {BlockMeta: tsdb.BlockMeta{ULID: block2, MinTime: 10, MaxTime: 20}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_2"}}}, + {ULID: block1, MinTime: 0, MaxTime: 10, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_2"}}, + {ULID: block2, MinTime: 10, MaxTime: 20, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_2"}}, }, }}, {userID: userID, stage: stageMerge, shardID: "2_of_2", blocksGroup: blocksGroup{ rangeStart: 0, rangeEnd: 20, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{ULID: block3, MinTime: 0, MaxTime: 10}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "2_of_2"}}}, - {BlockMeta: tsdb.BlockMeta{ULID: block4, MinTime: 10, MaxTime: 20}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "2_of_2"}}}, + {ULID: block3, MinTime: 0, MaxTime: 10, Labels: map[string]string{sharding.CompactorShardIDLabel: "2_of_2"}}, + {ULID: block4, MinTime: 10, MaxTime: 20, Labels: map[string]string{sharding.CompactorShardIDLabel: "2_of_2"}}, }, }}, }, @@ -311,47 +310,47 @@ func TestPlanCompaction(t *testing.T) { shardCount: 1, blocks: []*block.Meta{ // To be split on 1st level range [0, 10] - {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 0, MaxTime: 10}}, - {BlockMeta: tsdb.BlockMeta{ULID: block2, MinTime: 7, MaxTime: 10}}, + {ULID: block1, MinTime: 0, MaxTime: 10}, + {ULID: block2, MinTime: 7, MaxTime: 10}, // Not compacted because on 2nd level because the range [0, 20] // has other 1st level range groups to be split first - {BlockMeta: tsdb.BlockMeta{ULID: block10, MinTime: 0, MaxTime: 10}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, - {BlockMeta: tsdb.BlockMeta{ULID: block3, MinTime: 10, MaxTime: 20}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, + {ULID: block10, MinTime: 0, MaxTime: 10, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, + {ULID: block3, MinTime: 10, MaxTime: 20, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, // To be compacted on 2nd level range [20, 40] - {BlockMeta: tsdb.BlockMeta{ULID: block4, MinTime: 20, MaxTime: 30}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, - {BlockMeta: tsdb.BlockMeta{ULID: block5, MinTime: 30, MaxTime: 40}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, + {ULID: block4, MinTime: 20, MaxTime: 30, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, + {ULID: block5, MinTime: 30, MaxTime: 40, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, // Already compacted on 2nd level range [40, 60] - {BlockMeta: tsdb.BlockMeta{ULID: block6, MinTime: 40, MaxTime: 60}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, + {ULID: block6, MinTime: 40, MaxTime: 60, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, // Not compacted on 2nd level because the range [60, 80] // has other 1st level range groups to be compacted first - {BlockMeta: tsdb.BlockMeta{ULID: block7, MinTime: 60, MaxTime: 70}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, + {ULID: block7, MinTime: 60, MaxTime: 70, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, // To be compacted on 1st level range [70, 80] - {BlockMeta: tsdb.BlockMeta{ULID: block8, MinTime: 70, MaxTime: 80}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, - {BlockMeta: tsdb.BlockMeta{ULID: block9, MinTime: 75, MaxTime: 80}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, + {ULID: block8, MinTime: 70, MaxTime: 80, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, + {ULID: block9, MinTime: 75, MaxTime: 80, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, }, expected: []*job{ {userID: userID, stage: stageSplit, shardID: "1_of_1", blocksGroup: blocksGroup{ rangeStart: 0, rangeEnd: 10, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 0, MaxTime: 10}}, - {BlockMeta: tsdb.BlockMeta{ULID: block2, MinTime: 7, MaxTime: 10}}, + {ULID: block1, MinTime: 0, MaxTime: 10}, + {ULID: block2, MinTime: 7, MaxTime: 10}, }, }}, {userID: userID, stage: stageMerge, shardID: "1_of_1", blocksGroup: blocksGroup{ rangeStart: 70, rangeEnd: 80, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{ULID: block8, MinTime: 70, MaxTime: 80}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, - {BlockMeta: tsdb.BlockMeta{ULID: block9, MinTime: 75, MaxTime: 80}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, + {ULID: block8, MinTime: 70, MaxTime: 80, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, + {ULID: block9, MinTime: 75, MaxTime: 80, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, }, }}, {userID: userID, stage: stageMerge, shardID: "1_of_1", blocksGroup: blocksGroup{ rangeStart: 20, rangeEnd: 40, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{ULID: block4, MinTime: 20, MaxTime: 30}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, - {BlockMeta: tsdb.BlockMeta{ULID: block5, MinTime: 30, MaxTime: 40}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, + {ULID: block4, MinTime: 20, MaxTime: 30, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, + {ULID: block5, MinTime: 30, MaxTime: 40, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, }, }}, }, @@ -360,9 +359,9 @@ func TestPlanCompaction(t *testing.T) { ranges: []int64{10, 20, 40}, shardCount: 1, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 0, MaxTime: 40}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, - {BlockMeta: tsdb.BlockMeta{ULID: block2, MinTime: 40, MaxTime: 70}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, - {BlockMeta: tsdb.BlockMeta{ULID: block3, MinTime: 80, MaxTime: 120}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, + {ULID: block1, MinTime: 0, MaxTime: 40, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, + {ULID: block2, MinTime: 40, MaxTime: 70, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, + {ULID: block3, MinTime: 80, MaxTime: 120, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, }, expected: nil, }, @@ -370,18 +369,18 @@ func TestPlanCompaction(t *testing.T) { ranges: []int64{10, 20, 40}, shardCount: 1, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 0, MaxTime: 40}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, - {BlockMeta: tsdb.BlockMeta{ULID: block2, MinTime: 40, MaxTime: 70}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, - {BlockMeta: tsdb.BlockMeta{ULID: block3, MinTime: 80, MaxTime: 120}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, - {BlockMeta: tsdb.BlockMeta{ULID: block4, MinTime: 80, MaxTime: 120}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, + {ULID: block1, MinTime: 0, MaxTime: 40, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, + {ULID: block2, MinTime: 40, MaxTime: 70, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, + {ULID: block3, MinTime: 80, MaxTime: 120, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, + {ULID: block4, MinTime: 80, MaxTime: 120, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, }, expected: []*job{ {userID: userID, stage: stageMerge, shardID: "1_of_1", blocksGroup: blocksGroup{ rangeStart: 80, rangeEnd: 120, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{ULID: block3, MinTime: 80, MaxTime: 120}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, - {BlockMeta: tsdb.BlockMeta{ULID: block4, MinTime: 80, MaxTime: 120}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, + {ULID: block3, MinTime: 80, MaxTime: 120, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, + {ULID: block4, MinTime: 80, MaxTime: 120, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, }, }}, }, @@ -390,25 +389,25 @@ func TestPlanCompaction(t *testing.T) { ranges: []int64{20, 40}, shardCount: 1, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 10, MaxTime: 20}}, - {BlockMeta: tsdb.BlockMeta{ULID: block2, MinTime: 10, MaxTime: 30}}, // This block spans across two 1st level ranges. - {BlockMeta: tsdb.BlockMeta{ULID: block3, MinTime: 20, MaxTime: 30}}, - {BlockMeta: tsdb.BlockMeta{ULID: block4, MinTime: 30, MaxTime: 40}}, + {ULID: block1, MinTime: 10, MaxTime: 20}, + {ULID: block2, MinTime: 10, MaxTime: 30}, // This block spans across two 1st level ranges. + {ULID: block3, MinTime: 20, MaxTime: 30}, + {ULID: block4, MinTime: 30, MaxTime: 40}, }, expected: []*job{ {userID: userID, stage: stageSplit, shardID: "1_of_1", blocksGroup: blocksGroup{ rangeStart: 0, rangeEnd: 20, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 10, MaxTime: 20}}, + {ULID: block1, MinTime: 10, MaxTime: 20}, }, }}, {userID: userID, stage: stageSplit, shardID: "1_of_1", blocksGroup: blocksGroup{ rangeStart: 20, rangeEnd: 40, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{ULID: block3, MinTime: 20, MaxTime: 30}}, - {BlockMeta: tsdb.BlockMeta{ULID: block4, MinTime: 30, MaxTime: 40}}, + {ULID: block3, MinTime: 20, MaxTime: 30}, + {ULID: block4, MinTime: 30, MaxTime: 40}, }, }}, }, @@ -417,18 +416,18 @@ func TestPlanCompaction(t *testing.T) { ranges: []int64{20, 40}, shardCount: 1, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 0, MaxTime: 20}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, - {BlockMeta: tsdb.BlockMeta{ULID: block2, MinTime: 10, MaxTime: 30}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, // This block spans across two 1st level ranges. - {BlockMeta: tsdb.BlockMeta{ULID: block3, MinTime: 20, MaxTime: 40}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, + {ULID: block1, MinTime: 0, MaxTime: 20, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, + {ULID: block2, MinTime: 10, MaxTime: 30, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, // This block spans across two 1st level ranges. + {ULID: block3, MinTime: 20, MaxTime: 40, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, }, expected: []*job{ {userID: userID, stage: stageMerge, shardID: "1_of_1", blocksGroup: blocksGroup{ rangeStart: 0, rangeEnd: 40, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 0, MaxTime: 20}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, - {BlockMeta: tsdb.BlockMeta{ULID: block2, MinTime: 10, MaxTime: 30}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, - {BlockMeta: tsdb.BlockMeta{ULID: block3, MinTime: 20, MaxTime: 40}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, + {ULID: block1, MinTime: 0, MaxTime: 20, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, + {ULID: block2, MinTime: 10, MaxTime: 30, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, + {ULID: block3, MinTime: 20, MaxTime: 40, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, }, }}, }, @@ -437,19 +436,19 @@ func TestPlanCompaction(t *testing.T) { ranges: []int64{10, 20, 40}, shardCount: 1, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 0, MaxTime: 40}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, - {BlockMeta: tsdb.BlockMeta{ULID: block2, MinTime: 30, MaxTime: 150}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, // This block is larger then the largest compaction range. - {BlockMeta: tsdb.BlockMeta{ULID: block3, MinTime: 40, MaxTime: 70}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, - {BlockMeta: tsdb.BlockMeta{ULID: block4, MinTime: 80, MaxTime: 120}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, - {BlockMeta: tsdb.BlockMeta{ULID: block5, MinTime: 80, MaxTime: 120}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, + {ULID: block1, MinTime: 0, MaxTime: 40, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, + {ULID: block2, MinTime: 30, MaxTime: 150, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, // This block is larger then the largest compaction range. + {ULID: block3, MinTime: 40, MaxTime: 70, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, + {ULID: block4, MinTime: 80, MaxTime: 120, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, + {ULID: block5, MinTime: 80, MaxTime: 120, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, }, expected: []*job{ {userID: userID, stage: stageMerge, shardID: "1_of_1", blocksGroup: blocksGroup{ rangeStart: 80, rangeEnd: 120, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{ULID: block4, MinTime: 80, MaxTime: 120}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, - {BlockMeta: tsdb.BlockMeta{ULID: block5, MinTime: 80, MaxTime: 120}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_1"}}}, + {ULID: block4, MinTime: 80, MaxTime: 120, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, + {ULID: block5, MinTime: 80, MaxTime: 120, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, }, }}, }, @@ -458,18 +457,18 @@ func TestPlanCompaction(t *testing.T) { ranges: []int64{10, 20, 40}, shardCount: 1, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{MinTime: 5, MaxTime: 8}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 7, MaxTime: 9}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 10, MaxTime: 12}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 13, MaxTime: 15}}, + {MinTime: 5, MaxTime: 8}, + {MinTime: 7, MaxTime: 9}, + {MinTime: 10, MaxTime: 12}, + {MinTime: 13, MaxTime: 15}, }, expected: []*job{ {userID: userID, stage: stageSplit, shardID: "1_of_1", blocksGroup: blocksGroup{ rangeStart: 0, rangeEnd: 10, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{MinTime: 5, MaxTime: 8}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 7, MaxTime: 9}}, + {MinTime: 5, MaxTime: 8}, + {MinTime: 7, MaxTime: 9}, }, }}, }, @@ -478,32 +477,32 @@ func TestPlanCompaction(t *testing.T) { ranges: []int64{10, 20}, shardCount: 1, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 10, MaxTime: 20}}, - {BlockMeta: tsdb.BlockMeta{ULID: block2, MinTime: 10, MaxTime: 20}, Thanos: block.ThanosMeta{Labels: map[string]string{"another_group": "a"}}}, - {BlockMeta: tsdb.BlockMeta{ULID: block3, MinTime: 10, MaxTime: 20}, Thanos: block.ThanosMeta{Labels: map[string]string{"another_group": "a"}}}, - {BlockMeta: tsdb.BlockMeta{ULID: block4, MinTime: 10, MaxTime: 20}, Thanos: block.ThanosMeta{Labels: map[string]string{"another_group": "b"}}}, + {ULID: block1, MinTime: 10, MaxTime: 20}, + {ULID: block2, MinTime: 10, MaxTime: 20, Labels: map[string]string{"another_group": "a"}}, + {ULID: block3, MinTime: 10, MaxTime: 20, Labels: map[string]string{"another_group": "a"}}, + {ULID: block4, MinTime: 10, MaxTime: 20, Labels: map[string]string{"another_group": "b"}}, }, expected: []*job{ {userID: userID, stage: stageSplit, shardID: "1_of_1", blocksGroup: blocksGroup{ rangeStart: 10, rangeEnd: 20, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 10, MaxTime: 20}}, + {ULID: block1, MinTime: 10, MaxTime: 20}, }, }}, {userID: userID, stage: stageSplit, shardID: "1_of_1", blocksGroup: blocksGroup{ rangeStart: 10, rangeEnd: 20, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{ULID: block2, MinTime: 10, MaxTime: 20}, Thanos: block.ThanosMeta{Labels: map[string]string{"another_group": "a"}}}, - {BlockMeta: tsdb.BlockMeta{ULID: block3, MinTime: 10, MaxTime: 20}, Thanos: block.ThanosMeta{Labels: map[string]string{"another_group": "a"}}}, + {ULID: block2, MinTime: 10, MaxTime: 20, Labels: map[string]string{"another_group": "a"}}, + {ULID: block3, MinTime: 10, MaxTime: 20, Labels: map[string]string{"another_group": "a"}}, }, }}, {userID: userID, stage: stageSplit, shardID: "1_of_1", blocksGroup: blocksGroup{ rangeStart: 10, rangeEnd: 20, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{ULID: block4, MinTime: 10, MaxTime: 20}, Thanos: block.ThanosMeta{Labels: map[string]string{"another_group": "b"}}}, + {ULID: block4, MinTime: 10, MaxTime: 20, Labels: map[string]string{"another_group": "b"}}, }, }}, }, @@ -549,8 +548,8 @@ func TestPlanSplitting(t *testing.T) { rangeStart: 10, rangeEnd: 20, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{ULID: block1}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_2"}}}, - {BlockMeta: tsdb.BlockMeta{ULID: block2}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "2_of_2"}}}, + {ULID: block1, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_2"}}, + {ULID: block2, Labels: map[string]string{sharding.CompactorShardIDLabel: "2_of_2"}}, }, }, splitGroups: 2, @@ -561,8 +560,8 @@ func TestPlanSplitting(t *testing.T) { rangeStart: 10, rangeEnd: 20, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{ULID: block1}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_2"}}}, - {BlockMeta: tsdb.BlockMeta{ULID: block2}}, + {ULID: block1, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_2"}}, + {ULID: block2}, }, }, splitGroups: 2, @@ -572,7 +571,7 @@ func TestPlanSplitting(t *testing.T) { rangeStart: 10, rangeEnd: 20, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{ULID: block2}}, + {ULID: block2}, }, }, userID: userID, @@ -586,11 +585,11 @@ func TestPlanSplitting(t *testing.T) { rangeStart: 10, rangeEnd: 20, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{ULID: block1}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_2"}}}, - {BlockMeta: tsdb.BlockMeta{ULID: block2}}, - {BlockMeta: tsdb.BlockMeta{ULID: block3}}, - {BlockMeta: tsdb.BlockMeta{ULID: block4}}, - {BlockMeta: tsdb.BlockMeta{ULID: block5}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1_of_2"}}}, + {ULID: block1, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_2"}}, + {ULID: block2}, + {ULID: block3}, + {ULID: block4}, + {ULID: block5, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_2"}}, }, }, splitGroups: 2, @@ -600,7 +599,7 @@ func TestPlanSplitting(t *testing.T) { rangeStart: 10, rangeEnd: 20, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{ULID: block3}}, + {ULID: block3}, }, }, userID: userID, @@ -611,8 +610,8 @@ func TestPlanSplitting(t *testing.T) { rangeStart: 10, rangeEnd: 20, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{ULID: block2}}, - {BlockMeta: tsdb.BlockMeta{ULID: block4}}, + {ULID: block2}, + {ULID: block4}, }, }, userID: userID, @@ -646,41 +645,41 @@ func TestGroupBlocksByShardID(t *testing.T) { }, "only 1 block in input with shard ID": { blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{ULID: block1}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1"}}}, + {ULID: block1, Labels: map[string]string{sharding.CompactorShardIDLabel: "1"}}, }, expected: map[string][]*block.Meta{ "1": { - {BlockMeta: tsdb.BlockMeta{ULID: block1}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1"}}}, + {ULID: block1, Labels: map[string]string{sharding.CompactorShardIDLabel: "1"}}, }, }, }, "only 1 block in input without shard ID": { blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{ULID: block1}}, + {ULID: block1}, }, expected: map[string][]*block.Meta{ "": { - {BlockMeta: tsdb.BlockMeta{ULID: block1}}, + {ULID: block1}, }, }, }, "multiple blocks per shard ID": { blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{ULID: block1}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1"}}}, - {BlockMeta: tsdb.BlockMeta{ULID: block2}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "2"}}}, - {BlockMeta: tsdb.BlockMeta{ULID: block3}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1"}}}, - {BlockMeta: tsdb.BlockMeta{ULID: block4}}, + {ULID: block1, Labels: map[string]string{sharding.CompactorShardIDLabel: "1"}}, + {ULID: block2, Labels: map[string]string{sharding.CompactorShardIDLabel: "2"}}, + {ULID: block3, Labels: map[string]string{sharding.CompactorShardIDLabel: "1"}}, + {ULID: block4}, }, expected: map[string][]*block.Meta{ "": { - {BlockMeta: tsdb.BlockMeta{ULID: block4}}, + {ULID: block4}, }, "1": { - {BlockMeta: tsdb.BlockMeta{ULID: block1}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1"}}}, - {BlockMeta: tsdb.BlockMeta{ULID: block3}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1"}}}, + {ULID: block1, Labels: map[string]string{sharding.CompactorShardIDLabel: "1"}}, + {ULID: block3, Labels: map[string]string{sharding.CompactorShardIDLabel: "1"}}, }, "2": { - {BlockMeta: tsdb.BlockMeta{ULID: block2}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "2"}}}, + {ULID: block2, Labels: map[string]string{sharding.CompactorShardIDLabel: "2"}}, }, }, }, @@ -707,80 +706,80 @@ func TestGroupBlocksByRange(t *testing.T) { "only 1 block in input": { timeRange: 20, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{MinTime: 10, MaxTime: 20}}, + {MinTime: 10, MaxTime: 20}, }, expected: []blocksGroup{ {rangeStart: 0, rangeEnd: 20, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{MinTime: 10, MaxTime: 20}}, + {MinTime: 10, MaxTime: 20}, }}, }, }, "only 1 block per range": { timeRange: 20, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{MinTime: 10, MaxTime: 15}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 40, MaxTime: 60}}, + {MinTime: 10, MaxTime: 15}, + {MinTime: 40, MaxTime: 60}, }, expected: []blocksGroup{ {rangeStart: 0, rangeEnd: 20, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{MinTime: 10, MaxTime: 15}}, + {MinTime: 10, MaxTime: 15}, }}, {rangeStart: 40, rangeEnd: 60, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{MinTime: 40, MaxTime: 60}}, + {MinTime: 40, MaxTime: 60}, }}, }, }, "multiple blocks per range": { timeRange: 20, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{MinTime: 10, MaxTime: 15}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 10, MaxTime: 20}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 40, MaxTime: 60}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 50, MaxTime: 55}}, + {MinTime: 10, MaxTime: 15}, + {MinTime: 10, MaxTime: 20}, + {MinTime: 40, MaxTime: 60}, + {MinTime: 50, MaxTime: 55}, }, expected: []blocksGroup{ {rangeStart: 0, rangeEnd: 20, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{MinTime: 10, MaxTime: 15}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 10, MaxTime: 20}}, + {MinTime: 10, MaxTime: 15}, + {MinTime: 10, MaxTime: 20}, }}, {rangeStart: 40, rangeEnd: 60, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{MinTime: 40, MaxTime: 60}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 50, MaxTime: 55}}, + {MinTime: 40, MaxTime: 60}, + {MinTime: 50, MaxTime: 55}, }}, }, }, "a block with time range larger then the range should be excluded": { timeRange: 20, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{MinTime: 0, MaxTime: 20}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 0, MaxTime: 40}}, // This block is larger then the range. - {BlockMeta: tsdb.BlockMeta{MinTime: 10, MaxTime: 20}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 20, MaxTime: 30}}, + {MinTime: 0, MaxTime: 20}, + {MinTime: 0, MaxTime: 40}, // This block is larger then the range. + {MinTime: 10, MaxTime: 20}, + {MinTime: 20, MaxTime: 30}, }, expected: []blocksGroup{ {rangeStart: 0, rangeEnd: 20, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{MinTime: 0, MaxTime: 20}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 10, MaxTime: 20}}, + {MinTime: 0, MaxTime: 20}, + {MinTime: 10, MaxTime: 20}, }}, {rangeStart: 20, rangeEnd: 40, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{MinTime: 20, MaxTime: 30}}, + {MinTime: 20, MaxTime: 30}, }}, }, }, "blocks with different time ranges but all fitting within the input range": { timeRange: 40, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{MinTime: 0, MaxTime: 20}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 0, MaxTime: 40}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 10, MaxTime: 20}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 20, MaxTime: 30}}, + {MinTime: 0, MaxTime: 20}, + {MinTime: 0, MaxTime: 40}, + {MinTime: 10, MaxTime: 20}, + {MinTime: 20, MaxTime: 30}, }, expected: []blocksGroup{ {rangeStart: 0, rangeEnd: 40, blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{MinTime: 0, MaxTime: 20}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 0, MaxTime: 40}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 10, MaxTime: 20}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 20, MaxTime: 30}}, + {MinTime: 0, MaxTime: 20}, + {MinTime: 0, MaxTime: 40}, + {MinTime: 10, MaxTime: 20}, + {MinTime: 20, MaxTime: 30}, }}, }, }, diff --git a/pkg/compactor/split_merge_job.go b/pkg/compactor/split_merge_job.go index 21a7dcec22..2033c4cf7a 100644 --- a/pkg/compactor/split_merge_job.go +++ b/pkg/compactor/split_merge_job.go @@ -8,13 +8,12 @@ package compactor import ( "fmt" "strings" - "time" "github.com/prometheus/prometheus/model/labels" "golang.org/x/exp/slices" - "github.com/grafana/mimir/pkg/storage/tsdb" - "github.com/grafana/mimir/pkg/storage/tsdb/block" + "github.com/grafana/pyroscope/pkg/phlaredb/block" + "github.com/grafana/pyroscope/pkg/phlaredb/sharding" ) type compactionStage string @@ -59,12 +58,12 @@ func (j *job) conflicts(other *job) bool { // are never merged together, so they can't conflict. Since all blocks within the same job are expected to have the same // downsample resolution and external labels, we just check the 1st block of each job. if len(j.blocks) > 0 && len(other.blocks) > 0 { - myLabels := labelsWithoutShard(j.blocksGroup.blocks[0].Thanos.Labels) - otherLabels := labelsWithoutShard(other.blocksGroup.blocks[0].Thanos.Labels) + myLabels := labelsWithoutShard(j.blocksGroup.blocks[0].Labels) + otherLabels := labelsWithoutShard(other.blocksGroup.blocks[0].Labels) if !labels.Equal(myLabels, otherLabels) { return false } - if j.blocksGroup.blocks[0].Thanos.Downsample != other.blocksGroup.blocks[0].Thanos.Downsample { + if j.blocksGroup.blocks[0].Downsample != other.blocksGroup.blocks[0].Downsample { return false } } @@ -83,8 +82,8 @@ func (j *job) conflicts(other *job) bool { func (j *job) String() string { blocks := make([]string, 0, len(j.blocks)) for _, block := range j.blocks { - minT := time.Unix(0, block.MinTime*int64(time.Millisecond)).UTC() - maxT := time.Unix(0, block.MaxTime*int64(time.Millisecond)).UTC() + minT := block.MinTime.Time().UTC() + maxT := block.MaxTime.Time().UTC() blocks = append(blocks, fmt.Sprintf("%s (min time: %s, max time: %s)", block.ULID.String(), minT.String(), maxT.String())) } @@ -118,7 +117,7 @@ func (g blocksGroup) rangeLength() int64 { // minTime returns the lowest MinTime across all blocks in the group. func (g blocksGroup) minTime() int64 { // Blocks are expected to be sorted by MinTime. - return g.blocks[0].MinTime + return int64(g.blocks[0].MinTime) } // maxTime returns the highest MaxTime across all blocks in the group. @@ -131,7 +130,7 @@ func (g blocksGroup) maxTime() int64 { } } - return max + return int64(max) } // getNonShardedBlocks returns the list of non-sharded blocks. @@ -139,7 +138,7 @@ func (g blocksGroup) getNonShardedBlocks() []*block.Meta { var out []*block.Meta for _, b := range g.blocks { - if value, ok := b.Thanos.Labels[tsdb.CompactorShardIDExternalLabel]; !ok || value == "" { + if value, ok := b.Labels[sharding.CompactorShardIDLabel]; !ok || value == "" { out = append(out, b) } } diff --git a/pkg/compactor/split_merge_job_test.go b/pkg/compactor/split_merge_job_test.go index ac56919c02..3b894d5723 100644 --- a/pkg/compactor/split_merge_job_test.go +++ b/pkg/compactor/split_merge_job_test.go @@ -10,19 +10,18 @@ import ( "testing" "github.com/oklog/ulid" - "github.com/prometheus/prometheus/tsdb" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" - mimir_tsdb "github.com/grafana/mimir/pkg/storage/tsdb" - "github.com/grafana/mimir/pkg/storage/tsdb/block" + "github.com/grafana/pyroscope/pkg/phlaredb/block" + sharding "github.com/grafana/pyroscope/pkg/phlaredb/sharding" ) func TestJob_conflicts(t *testing.T) { - block1 := &block.Meta{BlockMeta: tsdb.BlockMeta{ULID: ulid.MustNew(1, nil)}} - block2 := &block.Meta{BlockMeta: tsdb.BlockMeta{ULID: ulid.MustNew(2, nil)}} - block3 := &block.Meta{BlockMeta: tsdb.BlockMeta{ULID: ulid.MustNew(3, nil)}} - block4 := &block.Meta{BlockMeta: tsdb.BlockMeta{ULID: ulid.MustNew(4, nil)}} + block1 := &block.Meta{ULID: ulid.MustNew(1, nil)} + block2 := &block.Meta{ULID: ulid.MustNew(2, nil)} + block3 := &block.Meta{ULID: ulid.MustNew(3, nil)} + block4 := &block.Meta{ULID: ulid.MustNew(4, nil)} copyMeta := func(meta *block.Meta) *block.Meta { encoded, err := json.Marshal(meta) @@ -36,7 +35,7 @@ func TestJob_conflicts(t *testing.T) { withShardIDLabel := func(meta *block.Meta, shardID string) *block.Meta { meta = copyMeta(meta) - meta.Thanos.Labels = map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: shardID} + meta.Labels = map[string]string{sharding.CompactorShardIDLabel: shardID} return meta } @@ -227,31 +226,31 @@ func TestBlocksGroup_getNonShardedBlocks(t *testing.T) { }, "should return nil if the group contains only sharded blocks": { input: blocksGroup{blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{ULID: block1}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1"}}}, - {BlockMeta: tsdb.BlockMeta{ULID: block2}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1"}}}, + {ULID: block1, Labels: map[string]string{sharding.CompactorShardIDLabel: "1"}}, + {ULID: block2, Labels: map[string]string{sharding.CompactorShardIDLabel: "1"}}, }}, expected: nil, }, "should return the list of non-sharded blocks if exist in the group": { input: blocksGroup{blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{ULID: block1}}, - {BlockMeta: tsdb.BlockMeta{ULID: block2}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1"}}}, - {BlockMeta: tsdb.BlockMeta{ULID: block3}, Thanos: block.ThanosMeta{Labels: map[string]string{"key": "value"}}}, + {ULID: block1}, + {ULID: block2, Labels: map[string]string{sharding.CompactorShardIDLabel: "1"}}, + {ULID: block3, Labels: map[string]string{"key": "value"}}, }}, expected: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{ULID: block1}}, - {BlockMeta: tsdb.BlockMeta{ULID: block3}, Thanos: block.ThanosMeta{Labels: map[string]string{"key": "value"}}}, + {ULID: block1}, + {ULID: block3, Labels: map[string]string{"key": "value"}}, }, }, "should consider non-sharded a block with the shard ID label but empty value": { input: blocksGroup{blocks: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{ULID: block1}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: ""}}}, - {BlockMeta: tsdb.BlockMeta{ULID: block2}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: "1"}}}, - {BlockMeta: tsdb.BlockMeta{ULID: block3}, Thanos: block.ThanosMeta{Labels: map[string]string{"key": "value"}}}, + {ULID: block1, Labels: map[string]string{sharding.CompactorShardIDLabel: ""}}, + {ULID: block2, Labels: map[string]string{sharding.CompactorShardIDLabel: "1"}}, + {ULID: block3, Labels: map[string]string{"key": "value"}}, }}, expected: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{ULID: block1}, Thanos: block.ThanosMeta{Labels: map[string]string{mimir_tsdb.CompactorShardIDExternalLabel: ""}}}, - {BlockMeta: tsdb.BlockMeta{ULID: block3}, Thanos: block.ThanosMeta{Labels: map[string]string{"key": "value"}}}, + {ULID: block1, Labels: map[string]string{sharding.CompactorShardIDLabel: ""}}, + {ULID: block3, Labels: map[string]string{"key": "value"}}, }, }, } diff --git a/pkg/compactor/split_merge_planner.go b/pkg/compactor/split_merge_planner.go index eddc06ccc1..c486a4f2c9 100644 --- a/pkg/compactor/split_merge_planner.go +++ b/pkg/compactor/split_merge_planner.go @@ -9,7 +9,7 @@ import ( "context" "fmt" - "github.com/grafana/mimir/pkg/storage/tsdb/block" + "github.com/grafana/pyroscope/pkg/phlaredb/block" ) type SplitAndMergePlanner struct { @@ -39,7 +39,7 @@ func (c *SplitAndMergePlanner) Plan(_ context.Context, metasByMinTime []*block.M rangeEnd := rangeStart + largestRange for _, b := range metasByMinTime { - if b.MinTime < rangeStart || b.MaxTime > rangeEnd { + if int64(b.MinTime) < rangeStart || int64(b.MaxTime) > rangeEnd { return nil, fmt.Errorf("block %s with time range %d:%d is outside the largest expected range %d:%d", b.ULID.String(), b.MinTime, diff --git a/pkg/compactor/split_merge_planner_test.go b/pkg/compactor/split_merge_planner_test.go index bd577e8466..e52376fd43 100644 --- a/pkg/compactor/split_merge_planner_test.go +++ b/pkg/compactor/split_merge_planner_test.go @@ -11,10 +11,9 @@ import ( "testing" "github.com/oklog/ulid" - "github.com/prometheus/prometheus/tsdb" "github.com/stretchr/testify/assert" - "github.com/grafana/mimir/pkg/storage/tsdb/block" + "github.com/grafana/pyroscope/pkg/phlaredb/block" ) func TestSplitAndMergePlanner_Plan(t *testing.T) { @@ -34,9 +33,9 @@ func TestSplitAndMergePlanner_Plan(t *testing.T) { "a source block is larger then the largest range": { ranges: []int64{20, 40, 60}, blocksByMinTime: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 0, MaxTime: 20, Version: block.TSDBVersion1}}, - {BlockMeta: tsdb.BlockMeta{ULID: block2, MinTime: 10, MaxTime: 80, Version: block.TSDBVersion1}}, - {BlockMeta: tsdb.BlockMeta{ULID: block3, MinTime: 12, MaxTime: 15, Version: block.TSDBVersion1}}, + {ULID: block1, MinTime: 0, MaxTime: 20, Version: block.MetaVersion3}, + {ULID: block2, MinTime: 10, MaxTime: 80, Version: block.MetaVersion3}, + {ULID: block3, MinTime: 12, MaxTime: 15, Version: block.MetaVersion3}, }, expectedErr: fmt.Errorf("block %s with time range 10:80 is outside the largest expected range 0:60", block2.String()), @@ -44,9 +43,9 @@ func TestSplitAndMergePlanner_Plan(t *testing.T) { "source blocks are smaller then the largest range but compacted block is larger": { ranges: []int64{20, 40, 60}, blocksByMinTime: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 10, MaxTime: 20, Version: block.TSDBVersion1}}, - {BlockMeta: tsdb.BlockMeta{ULID: block2, MinTime: 30, MaxTime: 40, Version: block.TSDBVersion1}}, - {BlockMeta: tsdb.BlockMeta{ULID: block3, MinTime: 50, MaxTime: 70, Version: block.TSDBVersion1}}, + {ULID: block1, MinTime: 10, MaxTime: 20, Version: block.MetaVersion3}, + {ULID: block2, MinTime: 30, MaxTime: 40, Version: block.MetaVersion3}, + {ULID: block3, MinTime: 50, MaxTime: 70, Version: block.MetaVersion3}, }, expectedErr: fmt.Errorf("block %s with time range 50:70 is outside the largest expected range 0:60", block3.String()), @@ -54,8 +53,8 @@ func TestSplitAndMergePlanner_Plan(t *testing.T) { "source blocks and compacted block are smaller then the largest range but misaligned": { ranges: []int64{20, 40, 60}, blocksByMinTime: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 50, MaxTime: 70, Version: block.TSDBVersion1}}, - {BlockMeta: tsdb.BlockMeta{ULID: block2, MinTime: 70, MaxTime: 80, Version: block.TSDBVersion1}}, + {ULID: block1, MinTime: 50, MaxTime: 70, Version: block.MetaVersion3}, + {ULID: block2, MinTime: 70, MaxTime: 80, Version: block.MetaVersion3}, }, expectedErr: fmt.Errorf("block %s with time range 50:70 is outside the largest expected range 0:60", block1.String()), @@ -63,9 +62,9 @@ func TestSplitAndMergePlanner_Plan(t *testing.T) { "blocks fit within the largest range": { ranges: []int64{20, 40, 60}, blocksByMinTime: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{ULID: block1, MinTime: 10, MaxTime: 20, Version: block.TSDBVersion1}}, - {BlockMeta: tsdb.BlockMeta{ULID: block2, MinTime: 20, MaxTime: 40, Version: block.TSDBVersion1}}, - {BlockMeta: tsdb.BlockMeta{ULID: block3, MinTime: 20, MaxTime: 60, Version: block.TSDBVersion1}}, + {ULID: block1, MinTime: 10, MaxTime: 20, Version: block.MetaVersion3}, + {ULID: block2, MinTime: 20, MaxTime: 40, Version: block.MetaVersion3}, + {ULID: block3, MinTime: 20, MaxTime: 60, Version: block.MetaVersion3}, }, }, } diff --git a/pkg/compactor/syncer_metrics.go b/pkg/compactor/syncer_metrics.go index 09e1593031..9e860ae77e 100644 --- a/pkg/compactor/syncer_metrics.go +++ b/pkg/compactor/syncer_metrics.go @@ -11,10 +11,10 @@ import ( "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" - util_log "github.com/grafana/mimir/pkg/util/log" + util_log "github.com/grafana/pyroscope/pkg/util" ) -// Copied from Thanos, pkg/compact/compact.go. +// Copied from Mimir. // Here we aggregate metrics from all finished syncers. type aggregatedSyncerMetrics struct { metaSync prometheus.Counter @@ -25,34 +25,34 @@ type aggregatedSyncerMetrics struct { garbageCollectionDuration *dskit_metrics.HistogramDataCollector // was prometheus.Histogram before } -// Copied (and modified with Mimir prefix) from Thanos, pkg/compact/compact.go +// Copied (and modified with Pyroscope prefix) from Mimir. // We also ignore "group" label, since we only use a single group. func newAggregatedSyncerMetrics(reg prometheus.Registerer) *aggregatedSyncerMetrics { var m aggregatedSyncerMetrics m.metaSync = promauto.With(reg).NewCounter(prometheus.CounterOpts{ - Name: "cortex_compactor_meta_syncs_total", + Name: "pyroscope_compactor_meta_syncs_total", Help: "Total blocks metadata synchronization attempts.", }) m.metaSyncFailures = promauto.With(reg).NewCounter(prometheus.CounterOpts{ - Name: "cortex_compactor_meta_sync_failures_total", + Name: "pyroscope_compactor_meta_sync_failures_total", Help: "Total blocks metadata synchronization failures.", }) m.metaSyncDuration = dskit_metrics.NewHistogramDataCollector(prometheus.NewDesc( - "cortex_compactor_meta_sync_duration_seconds", + "pyroscope_compactor_meta_sync_duration_seconds", "Duration of the blocks metadata synchronization in seconds.", nil, nil)) m.garbageCollections = promauto.With(reg).NewCounter(prometheus.CounterOpts{ - Name: "cortex_compactor_garbage_collection_total", + Name: "pyroscope_compactor_garbage_collection_total", Help: "Total number of garbage collection operations.", }) m.garbageCollectionFailures = promauto.With(reg).NewCounter(prometheus.CounterOpts{ - Name: "cortex_compactor_garbage_collection_failures_total", + Name: "pyroscope_compactor_garbage_collection_failures_total", Help: "Total number of failed garbage collection operations.", }) m.garbageCollectionDuration = dskit_metrics.NewHistogramDataCollector(prometheus.NewDesc( - "cortex_compactor_garbage_collection_duration_seconds", + "pyroscope_compactor_garbage_collection_duration_seconds", "Time it took to perform garbage collection iteration.", nil, nil)) From 171c96023c61f1212ebe602032682e72b0271dfe Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Mon, 11 Sep 2023 16:01:26 +0200 Subject: [PATCH 19/74] Add missing minCompactor config --- pkg/validation/limits.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/pkg/validation/limits.go b/pkg/validation/limits.go index 73adbf51c3..8713af7ff4 100644 --- a/pkg/validation/limits.go +++ b/pkg/validation/limits.go @@ -12,6 +12,10 @@ import ( const ( bytesInMB = 1048576 + + // MinCompactorPartialBlockDeletionDelay is the minimum partial blocks deletion delay that can be configured in Mimir. + // Partial blocks are blocks that are not having meta file uploaded yet. + MinCompactorPartialBlockDeletionDelay = 4 * time.Hour ) // Limits describe all the limits for tenants; can be used to describe global default From 1650aba4e4517fed0547f9f78f2d25e791a0d02c Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Mon, 11 Sep 2023 17:52:43 +0200 Subject: [PATCH 20/74] Fixes as much as possible --- pkg/compactor/bucket_compactor.go | 264 +++++++----------------------- 1 file changed, 58 insertions(+), 206 deletions(-) diff --git a/pkg/compactor/bucket_compactor.go b/pkg/compactor/bucket_compactor.go index d4d28f8e5c..1a26dac3fb 100644 --- a/pkg/compactor/bucket_compactor.go +++ b/pkg/compactor/bucket_compactor.go @@ -25,12 +25,12 @@ import ( "github.com/prometheus/client_golang/prometheus/promauto" "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/tsdb" - "github.com/thanos-io/objstore" "go.uber.org/atomic" - "github.com/grafana/mimir/pkg/storage/sharding" - mimir_tsdb "github.com/grafana/mimir/pkg/storage/tsdb" - "github.com/grafana/mimir/pkg/storage/tsdb/block" + "github.com/grafana/pyroscope/pkg/objstore" + "github.com/grafana/pyroscope/pkg/phlaredb/block" + "github.com/grafana/pyroscope/pkg/phlaredb/sharding" + "github.com/grafana/pyroscope/pkg/util" ) type DeduplicateFilter interface { @@ -170,12 +170,12 @@ type Grouper interface { // DefaultGroupKey returns a unique identifier for the group the block belongs to, based on // the DefaultGrouper logic. It considers the downsampling resolution and the block's labels. -func DefaultGroupKey(meta block.ThanosMeta) string { +func DefaultGroupKey(meta block.Meta) string { return defaultGroupKey(meta.Downsample.Resolution, labels.FromMap(meta.Labels)) } func defaultGroupKey(res int64, lbls labels.Labels) string { - return fmt.Sprintf("%d@%v", res, labels.StableHash(lbls)) + return fmt.Sprintf("%d@%v", res, util.StableHash(lbls)) } func minTime(metas []*block.Meta) time.Time { @@ -190,7 +190,7 @@ func minTime(metas []*block.Meta) time.Time { } } - return time.Unix(0, minT*int64(time.Millisecond)).UTC() + return time.Unix(0, int64(minT)*int64(time.Millisecond)).UTC() } func maxTime(metas []*block.Meta) time.Time { @@ -205,7 +205,7 @@ func maxTime(metas []*block.Meta) time.Time { } } - return time.Unix(0, maxT*int64(time.Millisecond)).UTC() + return time.Unix(0, int64(maxT)*int64(time.Millisecond)).UTC() } // Planner returns blocks to compact. @@ -302,20 +302,9 @@ func (c *BucketCompactor) runCompactionJob(ctx context.Context, job *Job) (shoul } if err := stats.CriticalErr(); err != nil { - return errors.Wrapf(err, "block with not healthy index found %s; Compaction level %v; Labels: %v", bdir, meta.Compaction.Level, meta.Thanos.Labels) + return errors.Wrapf(err, "block with not healthy index found %s; Compaction level %v; Labels: %v", bdir, meta.Compaction.Level, meta.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(); err != nil { - return errors.Wrapf(err, "block id %s", meta.ULID) - } return nil }) if err != nil { @@ -376,20 +365,9 @@ func (c *BucketCompactor) runCompactionJob(ctx context.Context, job *Job) (shoul bdir := filepath.Join(subDir, blockToUpload.ulid.String()) - // When splitting is enabled, we need to inject the shard ID as external label. - newLabels := job.Labels().Map() - if job.UseSplitting() { - newLabels[mimir_tsdb.CompactorShardIDExternalLabel] = sharding.FormatShardIDLabelValue(uint64(blockToUpload.shardIndex), uint64(job.SplittingShards())) - } - - newMeta, err := block.InjectThanosMeta(jobLogger, bdir, block.ThanosMeta{ - Labels: newLabels, - Downsample: block.ThanosDownsample{Resolution: job.Resolution()}, - Source: block.CompactorSource, - SegmentFiles: block.GetSegmentFiles(bdir), - }, nil) + newMeta, err := block.ReadMetaFromDir(bdir) if err != nil { - return errors.Wrapf(err, "failed to finalize the block %s", bdir) + return errors.Wrapf(err, "failed to read meta the block dir %s", bdir) } if err = os.Remove(filepath.Join(bdir, "tombstones")); err != nil { @@ -397,12 +375,13 @@ func (c *BucketCompactor) runCompactionJob(ctx context.Context, job *Job) (shoul } // Ensure the compacted block is valid. + // todo validate blocks by opening. if err := block.VerifyBlock(jobLogger, bdir, newMeta.MinTime, newMeta.MaxTime, false); err != nil { return errors.Wrapf(err, "invalid result block %s", bdir) } begin := time.Now() - if err := block.Upload(ctx, jobLogger, c.bkt, bdir, nil); err != nil { + if err := block.Upload(ctx, jobLogger, c.bkt, bdir); err != nil { return errors.Wrapf(err, "upload of %s failed", blockToUpload.ulid) } @@ -448,19 +427,19 @@ func verifyCompactedBlocksTimeRanges(compIDs []ulid.ULID, sourceBlocksMinTime, s } // Ensure compacted block min/maxTime within source blocks min/maxTime - if meta.MinTime < sourceBlocksMinTime { + if int64(meta.MinTime) < sourceBlocksMinTime { return fmt.Errorf("invalid minTime for block %s, compacted block minTime %d is before source minTime %d", compID.String(), meta.MinTime, sourceBlocksMinTime) } - if meta.MaxTime > sourceBlocksMaxTime { + if int64(meta.MaxTime) > sourceBlocksMaxTime { return fmt.Errorf("invalid maxTime for block %s, compacted block maxTime %d is after source maxTime %d", compID.String(), meta.MaxTime, sourceBlocksMaxTime) } - if meta.MinTime == sourceBlocksMinTime { + if int64(meta.MinTime) == sourceBlocksMinTime { sourceBlocksMinTimeFound = true } - if meta.MaxTime == sourceBlocksMaxTime { + if int64(meta.MaxTime) == sourceBlocksMaxTime { sourceBlocksMaxTimeFound = true } } @@ -495,110 +474,6 @@ func convertCompactionResultToForEachJobs(compactedBlocks []ulid.ULID, splitJob return result } -type ulidWithShardIndex struct { - ulid ulid.ULID - shardIndex int -} - -// Issue347Error is a type wrapper for errors that should invoke repair process for broken block. -type Issue347Error struct { - err error - - id ulid.ULID -} - -func issue347Error(err error, brokenBlock ulid.ULID) Issue347Error { - return Issue347Error{err: err, id: brokenBlock} -} - -func (e Issue347Error) Error() string { - return e.err.Error() -} - -// IsIssue347Error returns true if the base error is a Issue347Error. -func IsIssue347Error(err error) bool { - _, ok := errors.Cause(err).(Issue347Error) - return ok -} - -// OutOfOrderChunkError is a type wrapper for OOO chunk error from validating block index. -type OutOfOrderChunksError struct { - err error - id ulid.ULID -} - -func (e OutOfOrderChunksError) Error() string { - return e.err.Error() -} - -func outOfOrderChunkError(err error, brokenBlock ulid.ULID) OutOfOrderChunksError { - return OutOfOrderChunksError{err: err, id: brokenBlock} -} - -// IsOutOfOrderChunk returns true if the base error is a OutOfOrderChunkError. -func IsOutOfOrderChunkError(err error) bool { - _, ok := errors.Cause(err).(OutOfOrderChunksError) - return ok -} - -// RepairIssue347 repairs the https://github.com/prometheus/tsdb/issues/347 issue when having issue347Error. -func RepairIssue347(ctx context.Context, logger log.Logger, bkt objstore.Bucket, blocksMarkedForDeletion prometheus.Counter, issue347Err error) error { - ie, ok := errors.Cause(issue347Err).(Issue347Error) - if !ok { - return errors.Errorf("Given error is not an issue347 error: %v", issue347Err) - } - - level.Info(logger).Log("msg", "Repairing block broken by https://github.com/prometheus/tsdb/issues/347", "id", ie.id, "err", issue347Err) - - tmpdir, err := os.MkdirTemp("", fmt.Sprintf("repair-issue-347-id-%s-", ie.id)) - if err != nil { - return err - } - - defer func() { - if err := os.RemoveAll(tmpdir); err != nil { - level.Warn(logger).Log("msg", "failed to remote tmpdir", "err", err, "tmpdir", tmpdir) - } - }() - - bdir := filepath.Join(tmpdir, ie.id.String()) - if err := block.Download(ctx, logger, bkt, ie.id, bdir); err != nil { - return errors.Wrapf(err, "download block %s", ie.id) - } - - meta, err := block.ReadMetaFromDir(bdir) - if err != nil { - return errors.Wrapf(err, "read meta from %s", bdir) - } - - resid, err := block.Repair(logger, tmpdir, ie.id, block.CompactorRepairSource, block.IgnoreIssue347OutsideChunk) - if err != nil { - return errors.Wrapf(err, "repair failed for block %s", ie.id) - } - - // Verify repaired id before uploading it. - if err := block.VerifyBlock(logger, filepath.Join(tmpdir, resid.String()), meta.MinTime, meta.MaxTime, false); err != nil { - return errors.Wrapf(err, "repaired block is invalid %s", resid) - } - - level.Info(logger).Log("msg", "uploading repaired block", "newID", resid) - if err = block.Upload(ctx, logger, bkt, filepath.Join(tmpdir, resid.String()), nil); err != nil { - return errors.Wrapf(err, "upload of %s failed", resid) - } - - level.Info(logger).Log("msg", "deleting broken block", "id", ie.id) - - // Spawn a new context so we always mark a block for deletion in full on shutdown. - delCtx, cancel := context.WithTimeout(context.Background(), 5*time.Minute) - defer cancel() - - // TODO(bplotka): Issue with this will introduce overlap that will halt compactor. Automate that (fix duplicate overlaps caused by this). - if err := block.MarkForDeletion(delCtx, logger, bkt, ie.id, "source of repaired block", blocksMarkedForDeletion); err != nil { - return errors.Wrapf(err, "marking old block %s for deletion has failed", ie.id) - } - return nil -} - func deleteBlock(bkt objstore.Bucket, id ulid.ULID, bdir string, logger log.Logger, blocksMarkedForDeletion prometheus.Counter) error { if err := os.RemoveAll(bdir); err != nil { return errors.Wrapf(err, "remove old block dir %s", id) @@ -614,6 +489,11 @@ func deleteBlock(bkt objstore.Bucket, id ulid.ULID, bdir string, logger log.Logg return nil } +type ulidWithShardIndex struct { + ulid ulid.ULID + shardIndex int +} + // BucketCompactorMetrics holds the metrics tracked by BucketCompactor. type BucketCompactorMetrics struct { groupCompactionRunsStarted prometheus.Counter @@ -630,33 +510,33 @@ type BucketCompactorMetrics struct { func NewBucketCompactorMetrics(blocksMarkedForDeletion prometheus.Counter, reg prometheus.Registerer) *BucketCompactorMetrics { return &BucketCompactorMetrics{ groupCompactionRunsStarted: promauto.With(reg).NewCounter(prometheus.CounterOpts{ - Name: "cortex_compactor_group_compaction_runs_started_total", + Name: "pyroscope_compactor_group_compaction_runs_started_total", Help: "Total number of group compaction attempts.", }), groupCompactionRunsCompleted: promauto.With(reg).NewCounter(prometheus.CounterOpts{ - Name: "cortex_compactor_group_compaction_runs_completed_total", + Name: "pyroscope_compactor_group_compaction_runs_completed_total", Help: "Total number of group completed compaction runs. This also includes compactor group runs that resulted with no compaction.", }), groupCompactionRunsFailed: promauto.With(reg).NewCounter(prometheus.CounterOpts{ - Name: "cortex_compactor_group_compactions_failures_total", + Name: "pyroscope_compactor_group_compactions_failures_total", Help: "Total number of failed group compactions.", }), groupCompactions: promauto.With(reg).NewCounter(prometheus.CounterOpts{ - Name: "cortex_compactor_group_compactions_total", + Name: "pyroscope_compactor_group_compactions_total", Help: "Total number of group compaction attempts that resulted in new block(s).", }), compactionBlocksVerificationFailed: promauto.With(reg).NewCounter(prometheus.CounterOpts{ - Name: "cortex_compactor_blocks_verification_failures_total", + Name: "pyroscope_compactor_blocks_verification_failures_total", Help: "Total number of failures when verifying min/max time ranges of compacted blocks.", }), blocksMarkedForDeletion: blocksMarkedForDeletion, blocksMarkedForNoCompact: promauto.With(reg).NewCounter(prometheus.CounterOpts{ - Name: "cortex_compactor_blocks_marked_for_no_compaction_total", + Name: "pyroscope_compactor_blocks_marked_for_no_compaction_total", Help: "Total number of blocks that were marked for no-compaction.", ConstLabels: prometheus.Labels{"reason": block.OutOfOrderChunksNoCompactReason}, }), blocksMaxTimeDelta: promauto.With(reg).NewHistogram(prometheus.HistogramOpts{ - Name: "cortex_compactor_block_max_time_delta_seconds", + Name: "pyroscope_compactor_block_max_time_delta_seconds", Help: "Difference between now and the max time of a block being compacted in seconds.", Buckets: prometheus.LinearBuckets(86400, 43200, 8), // 1 to 5 days, in 12 hour intervals }), @@ -672,20 +552,19 @@ var ownAllJobs = func(job *Job) (bool, error) { // BucketCompactor compacts blocks in a bucket. type BucketCompactor struct { - logger log.Logger - sy *Syncer - grouper Grouper - comp Compactor - planner Planner - compactDir string - bkt objstore.Bucket - concurrency int - skipBlocksWithOutOfOrderChunks bool - ownJob ownCompactionJobFunc - sortJobs JobsOrderFunc - waitPeriod time.Duration - blockSyncConcurrency int - metrics *BucketCompactorMetrics + logger log.Logger + sy *Syncer + grouper Grouper + comp Compactor + planner Planner + compactDir string + bkt objstore.Bucket + concurrency int + ownJob ownCompactionJobFunc + sortJobs JobsOrderFunc + waitPeriod time.Duration + blockSyncConcurrency int + metrics *BucketCompactorMetrics } // NewBucketCompactor creates a new bucket compactor. @@ -698,7 +577,6 @@ func NewBucketCompactor( compactDir string, bkt objstore.Bucket, concurrency int, - skipBlocksWithOutOfOrderChunks bool, ownJob ownCompactionJobFunc, sortJobs JobsOrderFunc, waitPeriod time.Duration, @@ -709,20 +587,19 @@ func NewBucketCompactor( return nil, errors.Errorf("invalid concurrency level (%d), concurrency level must be > 0", concurrency) } return &BucketCompactor{ - logger: logger, - sy: sy, - grouper: grouper, - planner: planner, - comp: comp, - compactDir: compactDir, - bkt: bkt, - concurrency: concurrency, - skipBlocksWithOutOfOrderChunks: skipBlocksWithOutOfOrderChunks, - ownJob: ownJob, - sortJobs: sortJobs, - waitPeriod: waitPeriod, - blockSyncConcurrency: blockSyncConcurrency, - metrics: metrics, + logger: logger, + sy: sy, + grouper: grouper, + planner: planner, + comp: comp, + compactDir: compactDir, + bkt: bkt, + concurrency: concurrency, + ownJob: ownJob, + sortJobs: sortJobs, + waitPeriod: waitPeriod, + blockSyncConcurrency: blockSyncConcurrency, + metrics: metrics, }, nil } @@ -796,31 +673,6 @@ func (c *BucketCompactor) Compact(ctx context.Context, maxCompactionTime time.Du // At this point the compaction has failed. c.metrics.groupCompactionRunsFailed.Inc() - if IsIssue347Error(err) { - if err := RepairIssue347(workCtx, c.logger, c.bkt, c.sy.metrics.blocksMarkedForDeletion, err); err == nil { - mtx.Lock() - finishedAllJobs = 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, - block.OutOfOrderChunksNoCompactReason, - "OutofOrderChunk: marking block with out-of-order series/chunks to as no compact to unblock compaction", c.metrics.blocksMarkedForNoCompact); err == nil { - mtx.Lock() - finishedAllJobs = false - mtx.Unlock() - continue - } - } errChan <- errors.Wrapf(err, "group %s", g.Key()) return } @@ -924,7 +776,7 @@ func (c *BucketCompactor) blockMaxTimeDeltas(now time.Time, jobs []*Job) []float for _, j := range jobs { for _, m := range j.Metas() { - out = append(out, now.Sub(time.UnixMilli(m.MaxTime)).Seconds()) + out = append(out, now.Sub(time.UnixMilli(int64(m.MaxTime))).Seconds()) } } @@ -969,13 +821,13 @@ var _ block.MetadataFilter = &NoCompactionMarkFilter{} // NoCompactionMarkFilter is a block.Fetcher filter that finds all blocks with no-compact marker files, and optionally // removes them from synced metas. type NoCompactionMarkFilter struct { - bkt objstore.InstrumentedBucketReader + bkt objstore.BucketReader noCompactMarkedMap map[ulid.ULID]struct{} removeNoCompactBlocks bool } // NewNoCompactionMarkFilter creates NoCompactionMarkFilter. -func NewNoCompactionMarkFilter(bkt objstore.InstrumentedBucketReader, removeNoCompactBlocks bool) *NoCompactionMarkFilter { +func NewNoCompactionMarkFilter(bkt objstore.BucketReader, removeNoCompactBlocks bool) *NoCompactionMarkFilter { return &NoCompactionMarkFilter{ bkt: bkt, removeNoCompactBlocks: removeNoCompactBlocks, From 54a37cf6f7ecd7543e12908e5d976f3a757ef845 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Tue, 12 Sep 2023 14:06:32 +0200 Subject: [PATCH 21/74] block: Adds Download and MarkForNoCompact --- pkg/phlaredb/block/block.go | 52 +++++ pkg/phlaredb/block/block_test.go | 362 +++++++++++++++++++++++++++++++ pkg/test/copy.go | 57 +++++ pkg/util/labels.go | 35 +++ 4 files changed, 506 insertions(+) create mode 100644 pkg/phlaredb/block/block_test.go create mode 100644 pkg/test/copy.go create mode 100644 pkg/util/labels.go diff --git a/pkg/phlaredb/block/block.go b/pkg/phlaredb/block/block.go index c3ccf94984..aeea22cbb6 100644 --- a/pkg/phlaredb/block/block.go +++ b/pkg/phlaredb/block/block.go @@ -52,6 +52,26 @@ func DownloadMeta(ctx context.Context, logger log.Logger, bkt objstore.Bucket, i return m, nil } +// Download downloads directory that is meant to be block directory. If any of the files +// have a hash calculated in the meta file and it matches with what is in the destination path then +// we do not download it. We always re-download the meta file. +func Download(ctx context.Context, logger log.Logger, bucket objstore.Bucket, id ulid.ULID, dst string, options ...objstore.DownloadOption) error { + if err := os.MkdirAll(dst, 0o750); err != nil { + return errors.Wrap(err, "create dir") + } + + if err := objstore.DownloadFile(ctx, logger, bucket, path.Join(id.String(), MetaFilename), filepath.Join(dst, MetaFilename)); err != nil { + return err + } + + ignoredPaths := []string{MetaFilename} + if err := objstore.DownloadDir(ctx, logger, bucket, id.String(), id.String(), dst, append(options, objstore.WithDownloadIgnoredPaths(ignoredPaths...))...); err != nil { + return err + } + + return nil +} + func IsBlockDir(path string) (id ulid.ULID, ok bool) { id, err := ulid.Parse(filepath.Base(path)) return id, err == nil @@ -233,6 +253,38 @@ func deleteDirRec(ctx context.Context, logger log.Logger, bkt objstore.Bucket, d }) } +// MarkForNoCompact creates a file which marks block to be not compacted. +func MarkForNoCompact(ctx context.Context, logger log.Logger, bkt objstore.Bucket, id ulid.ULID, reason NoCompactReason, details string, markedForNoCompact prometheus.Counter) error { + m := path.Join(id.String(), NoCompactMarkFilename) + noCompactMarkExists, err := bkt.Exists(ctx, m) + if err != nil { + return errors.Wrapf(err, "check exists %s in bucket", m) + } + if noCompactMarkExists { + level.Warn(logger).Log("msg", "requested to mark for no compaction, but file already exists; this should not happen; investigate", "err", errors.Errorf("file %s already exists in bucket", m)) + return nil + } + + noCompactMark, err := json.Marshal(NoCompactMark{ + ID: id, + Version: NoCompactMarkVersion1, + + NoCompactTime: time.Now().Unix(), + Reason: reason, + Details: details, + }) + if err != nil { + return errors.Wrap(err, "json encode no compact mark") + } + + if err := bkt.Upload(ctx, m, bytes.NewBuffer(noCompactMark)); err != nil { + return errors.Wrapf(err, "upload file %s to bucket", m) + } + markedForNoCompact.Inc() + level.Info(logger).Log("msg", "block has been marked for no compaction", "block", id) + return nil +} + // HashBlockID returns a 32-bit hash of the block ID useful for // ring-based sharding. func HashBlockID(id ulid.ULID) uint32 { diff --git a/pkg/phlaredb/block/block_test.go b/pkg/phlaredb/block/block_test.go new file mode 100644 index 0000000000..f5b17a67a9 --- /dev/null +++ b/pkg/phlaredb/block/block_test.go @@ -0,0 +1,362 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/thanos-io/thanos/blob/main/pkg/block/block_test.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Thanos Authors. + +package block_test + +import ( + "bytes" + "context" + "encoding/json" + "io" + "os" + "path" + "strings" + "testing" + "time" + + "github.com/go-kit/log" + "github.com/oklog/ulid" + "github.com/pkg/errors" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" + promtest "github.com/prometheus/client_golang/prometheus/testutil" + "github.com/stretchr/testify/require" + "github.com/thanos-io/objstore" + "go.uber.org/goleak" + + "github.com/grafana/pyroscope/pkg/phlaredb/block" + block_testutil "github.com/grafana/pyroscope/pkg/phlaredb/block/testutil" + "github.com/grafana/pyroscope/pkg/pprof/testhelper" + "github.com/grafana/pyroscope/pkg/test" +) + +func TestIsBlockDir(t *testing.T) { + for _, tc := range []struct { + input string + id ulid.ULID + bdir bool + }{ + { + input: "", + bdir: false, + }, + { + input: "something", + bdir: false, + }, + { + id: ulid.MustNew(1, nil), + input: ulid.MustNew(1, nil).String(), + bdir: true, + }, + { + id: ulid.MustNew(2, nil), + input: "/" + ulid.MustNew(2, nil).String(), + bdir: true, + }, + { + id: ulid.MustNew(3, nil), + input: "some/path/" + ulid.MustNew(3, nil).String(), + bdir: true, + }, + { + input: ulid.MustNew(4, nil).String() + "/something", + bdir: false, + }, + } { + t.Run(tc.input, func(t *testing.T) { + id, ok := block.IsBlockDir(tc.input) + require.Equal(t, tc.bdir, ok) + + if id.Compare(tc.id) != 0 { + t.Errorf("expected %s got %s", tc.id, id) + t.FailNow() + } + }) + } +} + +func TestDelete(t *testing.T) { + defer goleak.VerifyNone(t, goleak.IgnoreCurrent()) + ctx := context.Background() + + bkt := objstore.NewInMemBucket() + { + meta, dir, err := block_testutil.CreateBlock(t, func() []*testhelper.ProfileBuilder { + return []*testhelper.ProfileBuilder{ + testhelper.NewProfileBuilder(int64(1)). + CPUProfile(). + WithLabels( + "job", "a", + ).ForStacktraceString("foo", "bar", "baz").AddSamples(1), + } + }) + require.NoError(t, err) + require.NoError(t, block.Upload(ctx, log.NewNopLogger(), bkt, path.Join(dir, meta.ULID.String()))) + require.Equal(t, 9, len(bkt.Objects())) + + markedForDeletion := promauto.With(prometheus.NewRegistry()).NewCounter(prometheus.CounterOpts{Name: "test"}) + require.NoError(t, block.MarkForDeletion(ctx, log.NewNopLogger(), bkt, meta.ULID, "", markedForDeletion)) + + // Full delete. + require.NoError(t, block.Delete(ctx, log.NewNopLogger(), bkt, meta.ULID)) + require.Equal(t, 0, len(bkt.Objects())) + } + { + b2, tmpDir, err := block_testutil.CreateBlock(t, func() []*testhelper.ProfileBuilder { + return []*testhelper.ProfileBuilder{ + testhelper.NewProfileBuilder(int64(1)). + CPUProfile(). + WithLabels( + "job", "a", + ).ForStacktraceString("foo", "bar", "baz").AddSamples(1), + } + }) + require.NoError(t, err) + require.NoError(t, block.Upload(ctx, log.NewNopLogger(), bkt, path.Join(tmpDir, b2.ULID.String()))) + require.Equal(t, 9, len(bkt.Objects())) + + // Remove meta.json and check if delete can delete it. + require.NoError(t, bkt.Delete(ctx, path.Join(b2.ULID.String(), block.MetaFilename))) + require.NoError(t, block.Delete(ctx, log.NewNopLogger(), bkt, b2.ULID)) + require.Equal(t, 0, len(bkt.Objects())) + } +} + +func TestUpload(t *testing.T) { + ctx := context.Background() + + bkt := objstore.NewInMemBucket() + b1, tmpDir, err := block_testutil.CreateBlock(t, func() []*testhelper.ProfileBuilder { + return []*testhelper.ProfileBuilder{ + testhelper.NewProfileBuilder(int64(1)). + CPUProfile(). + WithLabels( + "a", "3", "b", "1", + ).ForStacktraceString("foo", "bar", "baz").AddSamples(1), + } + }) + + require.NoError(t, err) + require.NoError(t, os.MkdirAll(path.Join(tmpDir, "test", b1.ULID.String()), os.ModePerm)) + + t.Run("wrong dir", func(t *testing.T) { + // Wrong dir. + err := block.Upload(ctx, log.NewNopLogger(), bkt, path.Join(tmpDir, "not-existing")) + require.Error(t, err) + require.Contains(t, err.Error(), "/not-existing: no such file or directory") + }) + + t.Run("wrong existing dir (not a block)", func(t *testing.T) { + err := block.Upload(ctx, log.NewNopLogger(), bkt, path.Join(tmpDir, "test")) + require.EqualError(t, err, "not a block dir: ulid: bad data size when unmarshaling") + }) + + t.Run("empty block dir", func(t *testing.T) { + err := block.Upload(ctx, log.NewNopLogger(), bkt, path.Join(tmpDir, "test", b1.ULID.String())) + require.Error(t, err) + require.Contains(t, err.Error(), "/meta.json: no such file or directory") + }) + + t.Run("missing meta.json file", func(t *testing.T) { + test.Copy(t, path.Join(tmpDir, b1.ULID.String(), block.IndexFilename), path.Join(tmpDir, "test", b1.ULID.String(), block.IndexFilename)) + + // Missing meta.json file. + err := block.Upload(ctx, log.NewNopLogger(), bkt, path.Join(tmpDir, "test", b1.ULID.String())) + require.Error(t, err) + require.Contains(t, err.Error(), "/meta.json: no such file or directory") + }) + + test.Copy(t, path.Join(tmpDir, b1.ULID.String(), block.MetaFilename), path.Join(tmpDir, "test", b1.ULID.String(), block.MetaFilename)) + + t.Run("full block", func(t *testing.T) { + require.NoError(t, block.Upload(ctx, log.NewNopLogger(), bkt, path.Join(tmpDir, b1.ULID.String()))) + require.Equal(t, 9, len(bkt.Objects())) + objs := bkt.Objects() + require.Contains(t, objs, path.Join(b1.ULID.String(), block.MetaFilename)) + require.Contains(t, objs, path.Join(b1.ULID.String(), block.IndexFilename)) + require.Contains(t, objs, path.Join(b1.ULID.String(), "profiles.parquet")) + }) + + t.Run("upload is idempotent", func(t *testing.T) { + require.NoError(t, block.Upload(ctx, log.NewNopLogger(), bkt, path.Join(tmpDir, b1.ULID.String()))) + require.Equal(t, 9, len(bkt.Objects())) + objs := bkt.Objects() + require.Contains(t, objs, path.Join(b1.ULID.String(), block.MetaFilename)) + require.Contains(t, objs, path.Join(b1.ULID.String(), block.IndexFilename)) + require.Contains(t, objs, path.Join(b1.ULID.String(), "profiles.parquet")) + }) +} + +func TestMarkForDeletion(t *testing.T) { + defer goleak.VerifyNone(t, goleak.IgnoreCurrent()) + ctx := context.Background() + + for _, tcase := range []struct { + name string + preUpload func(t testing.TB, id ulid.ULID, bkt objstore.Bucket) + + blocksMarked int + }{ + { + name: "block marked for deletion", + preUpload: func(t testing.TB, id ulid.ULID, bkt objstore.Bucket) {}, + blocksMarked: 1, + }, + { + name: "block with deletion mark already, expected log and no metric increment", + preUpload: func(t testing.TB, id ulid.ULID, bkt objstore.Bucket) { + deletionMark, err := json.Marshal(block.DeletionMark{ + ID: id, + DeletionTime: time.Now().Unix(), + Version: block.DeletionMarkVersion1, + }) + require.NoError(t, err) + require.NoError(t, bkt.Upload(ctx, path.Join(id.String(), block.DeletionMarkFilename), bytes.NewReader(deletionMark))) + }, + blocksMarked: 0, + }, + } { + t.Run(tcase.name, func(t *testing.T) { + bkt := objstore.NewInMemBucket() + b1, tmpDir, err := block_testutil.CreateBlock(t, func() []*testhelper.ProfileBuilder { + return []*testhelper.ProfileBuilder{ + testhelper.NewProfileBuilder(int64(1)). + CPUProfile(). + WithLabels( + "a", "3", "b", "1", + ).ForStacktraceString("foo", "bar", "baz").AddSamples(1), + } + }) + id := b1.ULID + require.NoError(t, err) + + tcase.preUpload(t, id, bkt) + + require.NoError(t, block.Upload(ctx, log.NewNopLogger(), bkt, path.Join(tmpDir, id.String()))) + + c := promauto.With(nil).NewCounter(prometheus.CounterOpts{}) + err = block.MarkForDeletion(ctx, log.NewNopLogger(), bkt, id, "", c) + require.NoError(t, err) + require.Equal(t, float64(tcase.blocksMarked), promtest.ToFloat64(c)) + }) + } +} + +func TestMarkForNoCompact(t *testing.T) { + defer goleak.VerifyNone(t, goleak.IgnoreCurrent()) + ctx := context.Background() + + for _, tcase := range []struct { + name string + preUpload func(t testing.TB, id ulid.ULID, bkt objstore.Bucket) + + blocksMarked int + }{ + { + name: "block marked", + preUpload: func(t testing.TB, id ulid.ULID, bkt objstore.Bucket) {}, + blocksMarked: 1, + }, + { + name: "block with no-compact mark already, expected log and no metric increment", + preUpload: func(t testing.TB, id ulid.ULID, bkt objstore.Bucket) { + m, err := json.Marshal(block.NoCompactMark{ + ID: id, + NoCompactTime: time.Now().Unix(), + Version: block.NoCompactMarkVersion1, + }) + require.NoError(t, err) + require.NoError(t, bkt.Upload(ctx, path.Join(id.String(), block.NoCompactMarkFilename), bytes.NewReader(m))) + }, + blocksMarked: 0, + }, + } { + t.Run(tcase.name, func(t *testing.T) { + bkt := objstore.NewInMemBucket() + meta, tmpDir, err := block_testutil.CreateBlock(t, func() []*testhelper.ProfileBuilder { + return []*testhelper.ProfileBuilder{ + testhelper.NewProfileBuilder(int64(1)). + CPUProfile(). + WithLabels( + "a", "3", "b", "1", + ).ForStacktraceString("foo", "bar", "baz").AddSamples(1), + } + }) + id := meta.ULID + require.NoError(t, err) + + tcase.preUpload(t, id, bkt) + + require.NoError(t, block.Upload(ctx, log.NewNopLogger(), bkt, path.Join(tmpDir, id.String()))) + + c := promauto.With(nil).NewCounter(prometheus.CounterOpts{}) + err = block.MarkForNoCompact(ctx, log.NewNopLogger(), bkt, id, block.ManualNoCompactReason, "", c) + require.NoError(t, err) + require.Equal(t, float64(tcase.blocksMarked), promtest.ToFloat64(c)) + }) + } +} + +func TestUploadCleanup(t *testing.T) { + defer goleak.VerifyNone(t, goleak.IgnoreCurrent()) + + ctx := context.Background() + + bkt := objstore.NewInMemBucket() + meta, tmpDir, err := block_testutil.CreateBlock(t, func() []*testhelper.ProfileBuilder { + return []*testhelper.ProfileBuilder{ + testhelper.NewProfileBuilder(int64(1)). + CPUProfile(). + WithLabels( + "a", "3", "b", "1", + ).ForStacktraceString("foo", "bar", "baz").AddSamples(1), + } + }) + b1 := meta.ULID + require.NoError(t, err) + + { + errBkt := errBucket{Bucket: bkt, failSuffix: "/index.tsdb"} + + uploadErr := block.Upload(ctx, log.NewNopLogger(), errBkt, path.Join(tmpDir, b1.String())) + require.ErrorIs(t, uploadErr, errUploadFailed) + + // If upload of index fails, block is deleted. + require.Equal(t, 0, len(bkt.Objects())) + } + + { + errBkt := errBucket{Bucket: bkt, failSuffix: "/meta.json"} + + uploadErr := block.Upload(ctx, log.NewNopLogger(), errBkt, path.Join(tmpDir, b1.String())) + require.ErrorIs(t, uploadErr, errUploadFailed) + + // If upload of meta.json fails, nothing is cleaned up. + require.Equal(t, 9, len(bkt.Objects())) + require.Greater(t, len(bkt.Objects()[path.Join(b1.String(), block.IndexFilename)]), 0) + require.Greater(t, len(bkt.Objects()[path.Join(b1.String(), block.MetaFilename)]), 0) + } +} + +var errUploadFailed = errors.New("upload failed") + +type errBucket struct { + objstore.Bucket + + failSuffix string +} + +func (eb errBucket) Upload(ctx context.Context, name string, r io.Reader) error { + err := eb.Bucket.Upload(ctx, name, r) + if err != nil { + return err + } + + if strings.HasSuffix(name, eb.failSuffix) { + return errUploadFailed + } + return nil +} diff --git a/pkg/test/copy.go b/pkg/test/copy.go new file mode 100644 index 0000000000..ea5c7c05e5 --- /dev/null +++ b/pkg/test/copy.go @@ -0,0 +1,57 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/thanos-io/thanos/blob/main/pkg/testutil/e2eutil/copy.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Thanos Authors. + +package test + +import ( + "io" + "os" + "path/filepath" + "testing" + + "github.com/grafana/dskit/runutil" + "github.com/pkg/errors" + "github.com/stretchr/testify/require" +) + +func Copy(t testing.TB, src, dst string) { + require.NoError(t, copyRecursive(src, dst)) +} + +func copyRecursive(src, dst string) error { + return filepath.Walk(src, func(path string, info os.FileInfo, err error) error { + if err != nil { + return err + } + + relPath, err := filepath.Rel(src, path) + if err != nil { + return err + } + + if info.IsDir() { + return os.MkdirAll(filepath.Join(dst, relPath), os.ModePerm) + } + + if !info.Mode().IsRegular() { + return errors.Errorf("%s is not a regular file", path) + } + + source, err := os.Open(filepath.Clean(path)) + if err != nil { + return err + } + defer runutil.CloseWithErrCapture(&err, source, "close file") + + destination, err := os.Create(filepath.Join(dst, relPath)) + if err != nil { + return err + } + defer runutil.CloseWithErrCapture(&err, destination, "close file") + + _, err = io.Copy(destination, source) + return err + }) +} diff --git a/pkg/util/labels.go b/pkg/util/labels.go new file mode 100644 index 0000000000..82c0fa88ce --- /dev/null +++ b/pkg/util/labels.go @@ -0,0 +1,35 @@ +package util + +import ( + "github.com/cespare/xxhash/v2" + "github.com/prometheus/prometheus/model/labels" +) + +var seps = []byte{'\xff'} + +// StableHash is a labels hashing implementation which is guaranteed to not change over time. +// This function should be used whenever labels hashing backward compatibility must be guaranteed. +func StableHash(ls labels.Labels) uint64 { + // Use xxhash.Sum64(b) for fast path as it's faster. + b := make([]byte, 0, 1024) + for i, v := range ls { + if len(b)+len(v.Name)+len(v.Value)+2 >= cap(b) { + // If labels entry is 1KB+ do not allocate whole entry. + h := xxhash.New() + _, _ = h.Write(b) + for _, v := range ls[i:] { + _, _ = h.WriteString(v.Name) + _, _ = h.Write(seps) + _, _ = h.WriteString(v.Value) + _, _ = h.Write(seps) + } + return h.Sum64() + } + + b = append(b, v.Name...) + b = append(b, seps[0]) + b = append(b, v.Value...) + b = append(b, seps[0]) + } + return xxhash.Sum64(b) +} From 08abae6ac25939cdf2809e11de439cffb53f5090 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Tue, 12 Sep 2023 14:09:56 +0200 Subject: [PATCH 22/74] improve documentation --- pkg/phlaredb/block/block.go | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/pkg/phlaredb/block/block.go b/pkg/phlaredb/block/block.go index aeea22cbb6..240815119a 100644 --- a/pkg/phlaredb/block/block.go +++ b/pkg/phlaredb/block/block.go @@ -52,9 +52,7 @@ func DownloadMeta(ctx context.Context, logger log.Logger, bkt objstore.Bucket, i return m, nil } -// Download downloads directory that is meant to be block directory. If any of the files -// have a hash calculated in the meta file and it matches with what is in the destination path then -// we do not download it. We always re-download the meta file. +// Download downloads directory that is meant to be block directory. If a block file exists locally we won't download it. However we always re-download the meta file. func Download(ctx context.Context, logger log.Logger, bucket objstore.Bucket, id ulid.ULID, dst string, options ...objstore.DownloadOption) error { if err := os.MkdirAll(dst, 0o750); err != nil { return errors.Wrap(err, "create dir") From e99562cac6787fafe5725f8d64a731dbce084e1a Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Tue, 12 Sep 2023 16:38:01 +0200 Subject: [PATCH 23/74] wip --- pkg/compactor/bucket_compactor.go | 15 +++++---------- 1 file changed, 5 insertions(+), 10 deletions(-) diff --git a/pkg/compactor/bucket_compactor.go b/pkg/compactor/bucket_compactor.go index 1a26dac3fb..5a08c4ffee 100644 --- a/pkg/compactor/bucket_compactor.go +++ b/pkg/compactor/bucket_compactor.go @@ -28,6 +28,7 @@ import ( "go.uber.org/atomic" "github.com/grafana/pyroscope/pkg/objstore" + "github.com/grafana/pyroscope/pkg/phlaredb" "github.com/grafana/pyroscope/pkg/phlaredb/block" "github.com/grafana/pyroscope/pkg/phlaredb/sharding" "github.com/grafana/pyroscope/pkg/util" @@ -296,13 +297,8 @@ func (c *BucketCompactor) runCompactionJob(ctx context.Context, job *Job) (shoul } // Ensure all source blocks are valid. - stats, err := block.GatherBlockHealthStats(jobLogger, bdir, meta.MinTime, meta.MaxTime, false) - if err != nil { - return errors.Wrapf(err, "gather index issues for block %s", bdir) - } - - if err := stats.CriticalErr(); err != nil { - return errors.Wrapf(err, "block with not healthy index found %s; Compaction level %v; Labels: %v", bdir, meta.Compaction.Level, meta.Labels) + if err := phlaredb.ValidateLocalBlock(ctx, bdir); err != nil { + return errors.Wrapf(err, "invalid block %s", bdir) } return nil @@ -375,8 +371,7 @@ func (c *BucketCompactor) runCompactionJob(ctx context.Context, job *Job) (shoul } // Ensure the compacted block is valid. - // todo validate blocks by opening. - if err := block.VerifyBlock(jobLogger, bdir, newMeta.MinTime, newMeta.MaxTime, false); err != nil { + if err := phlaredb.ValidateLocalBlock(ctx, bdir); err != nil { return errors.Wrapf(err, "invalid result block %s", bdir) } @@ -386,7 +381,7 @@ func (c *BucketCompactor) runCompactionJob(ctx context.Context, job *Job) (shoul } elapsed := time.Since(begin) - level.Info(jobLogger).Log("msg", "uploaded block", "result_block", blockToUpload.ulid, "duration", elapsed, "duration_ms", elapsed.Milliseconds(), "external_labels", labels.FromMap(newLabels)) + level.Info(jobLogger).Log("msg", "uploaded block", "result_block", blockToUpload.ulid, "duration", elapsed, "duration_ms", elapsed.Milliseconds(), "labels", labels.FromMap(newMeta.Labels)) return nil }) if err != nil { From f88b292ea6093b54b97d6e2f60384171dfcbd22c Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Tue, 12 Sep 2023 16:39:58 +0200 Subject: [PATCH 24/74] Adds a way to validate local blocks --- pkg/phlaredb/validate.go | 35 ++++++++++++++++++++++++++++++++++ pkg/phlaredb/validate_test.go | 36 +++++++++++++++++++++++++++++++++++ 2 files changed, 71 insertions(+) create mode 100644 pkg/phlaredb/validate.go create mode 100644 pkg/phlaredb/validate_test.go diff --git a/pkg/phlaredb/validate.go b/pkg/phlaredb/validate.go new file mode 100644 index 0000000000..9c671dc0eb --- /dev/null +++ b/pkg/phlaredb/validate.go @@ -0,0 +1,35 @@ +package phlaredb + +import ( + "context" + "path" + + "github.com/grafana/dskit/runutil" + "github.com/grafana/pyroscope/pkg/objstore/client" + "github.com/grafana/pyroscope/pkg/objstore/providers/filesystem" + "github.com/grafana/pyroscope/pkg/phlaredb/block" + "github.com/grafana/pyroscope/pkg/util" +) + +// ValidateLocalBlock validates the block in the given directory is readable. +func ValidateLocalBlock(phlarectx context.Context, dir string) error { + meta, err := block.ReadMetaFromDir(dir) + if err != nil { + return err + } + + bkt, err := client.NewBucket(phlarectx, client.Config{ + StorageBackendConfig: client.StorageBackendConfig{ + Backend: client.Filesystem, + Filesystem: filesystem.Config{ + Directory: path.Dir(dir), + }, + }, + }, "validate") + if err != nil { + return err + } + q := NewSingleBlockQuerierFromMeta(phlarectx, bkt, meta) + defer runutil.CloseWithLogOnErr(util.Logger, q, "closing block querier") + return q.Open(phlarectx) +} diff --git a/pkg/phlaredb/validate_test.go b/pkg/phlaredb/validate_test.go new file mode 100644 index 0000000000..cb5db77bea --- /dev/null +++ b/pkg/phlaredb/validate_test.go @@ -0,0 +1,36 @@ +package phlaredb_test + +import ( + "context" + "os" + "path" + "testing" + + "github.com/grafana/pyroscope/pkg/phlaredb" + "github.com/grafana/pyroscope/pkg/phlaredb/block" + "github.com/grafana/pyroscope/pkg/phlaredb/block/testutil" + "github.com/grafana/pyroscope/pkg/pprof/testhelper" + "github.com/stretchr/testify/require" +) + +func Test_ValidateBlock(t *testing.T) { + meta, dir, err := testutil.CreateBlock(t, func() []*testhelper.ProfileBuilder { + return []*testhelper.ProfileBuilder{ + testhelper.NewProfileBuilder(int64(1)). + CPUProfile(). + WithLabels( + "job", "a", + ).ForStacktraceString("foo", "bar", "baz").AddSamples(1), + } + }) + require.NoError(t, err) + + err = phlaredb.ValidateLocalBlock(context.Background(), path.Join(dir, meta.ULID.String())) + require.NoError(t, err) + t.Run("should error when a file is missing", func(t *testing.T) { + os.Remove(path.Join(dir, meta.ULID.String(), block.IndexFilename)) + err = phlaredb.ValidateLocalBlock(context.Background(), path.Join(dir, meta.ULID.String())) + require.Error(t, err) + require.Contains(t, err.Error(), "no such file") + }) +} From 3324acc577993be79b5350636d246b2bb90ce377 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Tue, 12 Sep 2023 16:45:37 +0200 Subject: [PATCH 25/74] fixes wrong merge --- pkg/phlaredb/bucketindex/index.go | 3 --- 1 file changed, 3 deletions(-) diff --git a/pkg/phlaredb/bucketindex/index.go b/pkg/phlaredb/bucketindex/index.go index e8f5eab0b9..6ed88373a2 100644 --- a/pkg/phlaredb/bucketindex/index.go +++ b/pkg/phlaredb/bucketindex/index.go @@ -56,10 +56,7 @@ func (idx *Index) RemoveBlock(id ulid.ULID) { for i := 0; i < len(idx.BlockDeletionMarks); i++ { if idx.BlockDeletionMarks[i].ID == id { -<<<<<<< HEAD -======= idx.BlockDeletionMarks[i] = nil ->>>>>>> feat/validateblock2 idx.BlockDeletionMarks = append(idx.BlockDeletionMarks[:i], idx.BlockDeletionMarks[i+1:]...) break } From 07bc11857ae44a02b8789a038521b99fbe2caf7c Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Tue, 12 Sep 2023 23:22:01 +0200 Subject: [PATCH 26/74] more integration fixes --- pkg/compactor/compactor.go | 85 +++++++++++------------------- pkg/compactor/duration_list.go | 43 +++++++++++++++ pkg/tenant/allowed_tenants.go | 59 +++++++++++++++++++++ pkg/tenant/allowed_tenants_test.go | 52 ++++++++++++++++++ 4 files changed, 184 insertions(+), 55 deletions(-) create mode 100644 pkg/compactor/duration_list.go create mode 100644 pkg/tenant/allowed_tenants.go create mode 100644 pkg/tenant/allowed_tenants_test.go diff --git a/pkg/compactor/compactor.go b/pkg/compactor/compactor.go index b7bf6a123f..d0bc2e8be8 100644 --- a/pkg/compactor/compactor.go +++ b/pkg/compactor/compactor.go @@ -28,13 +28,11 @@ import ( "github.com/prometheus/client_golang/prometheus/promauto" "go.uber.org/atomic" - "github.com/grafana/mimir/pkg/storage/bucket" - mimir_tsdb "github.com/grafana/mimir/pkg/storage/tsdb" - "github.com/grafana/mimir/pkg/storage/tsdb/block" - "github.com/grafana/mimir/pkg/util" - util_log "github.com/grafana/mimir/pkg/util/log" - "github.com/grafana/pyroscope/pkg/objstore" + "github.com/grafana/pyroscope/pkg/phlaredb/block" + "github.com/grafana/pyroscope/pkg/phlaredb/bucket" + "github.com/grafana/pyroscope/pkg/tenant" + "github.com/grafana/pyroscope/pkg/util" ) const ( @@ -81,20 +79,20 @@ type BlocksCompactorFactory func( // Config holds the MultitenantCompactor config. type Config struct { - BlockRanges mimir_tsdb.DurationList `yaml:"block_ranges" category:"advanced"` - BlockSyncConcurrency int `yaml:"block_sync_concurrency" category:"advanced"` - MetaSyncConcurrency int `yaml:"meta_sync_concurrency" category:"advanced"` - DataDir string `yaml:"data_dir"` - CompactionInterval time.Duration `yaml:"compaction_interval" category:"advanced"` - CompactionRetries int `yaml:"compaction_retries" category:"advanced"` - CompactionConcurrency int `yaml:"compaction_concurrency" category:"advanced"` - CompactionWaitPeriod time.Duration `yaml:"first_level_compaction_wait_period"` - CleanupInterval time.Duration `yaml:"cleanup_interval" category:"advanced"` - CleanupConcurrency int `yaml:"cleanup_concurrency" category:"advanced"` - DeletionDelay time.Duration `yaml:"deletion_delay" category:"advanced"` - TenantCleanupDelay time.Duration `yaml:"tenant_cleanup_delay" category:"advanced"` - MaxCompactionTime time.Duration `yaml:"max_compaction_time" category:"advanced"` - NoBlocksFileCleanupEnabled bool `yaml:"no_blocks_file_cleanup_enabled" category:"experimental"` + BlockRanges DurationList `yaml:"block_ranges" category:"advanced"` + BlockSyncConcurrency int `yaml:"block_sync_concurrency" category:"advanced"` + MetaSyncConcurrency int `yaml:"meta_sync_concurrency" category:"advanced"` + DataDir string `yaml:"data_dir"` + CompactionInterval time.Duration `yaml:"compaction_interval" category:"advanced"` + CompactionRetries int `yaml:"compaction_retries" category:"advanced"` + CompactionConcurrency int `yaml:"compaction_concurrency" category:"advanced"` + CompactionWaitPeriod time.Duration `yaml:"first_level_compaction_wait_period"` + CleanupInterval time.Duration `yaml:"cleanup_interval" category:"advanced"` + CleanupConcurrency int `yaml:"cleanup_concurrency" category:"advanced"` + DeletionDelay time.Duration `yaml:"deletion_delay" category:"advanced"` + TenantCleanupDelay time.Duration `yaml:"tenant_cleanup_delay" category:"advanced"` + MaxCompactionTime time.Duration `yaml:"max_compaction_time" category:"advanced"` + NoBlocksFileCleanupEnabled bool `yaml:"no_blocks_file_cleanup_enabled" category:"experimental"` // Compactor concurrency options MaxOpeningBlocksConcurrency int `yaml:"max_opening_blocks_concurrency" category:"advanced"` // Number of goroutines opening blocks before compaction. @@ -125,7 +123,7 @@ type Config struct { func (cfg *Config) RegisterFlags(f *flag.FlagSet, logger log.Logger) { cfg.ShardingRing.RegisterFlags(f, logger) - cfg.BlockRanges = mimir_tsdb.DurationList{2 * time.Hour, 12 * time.Hour, 24 * time.Hour} + cfg.BlockRanges = DurationList{2 * time.Hour, 12 * time.Hour, 24 * time.Hour} cfg.retryMinBackoff = 10 * time.Second cfg.retryMaxBackoff = time.Minute @@ -223,7 +221,6 @@ type MultitenantCompactor struct { services.Service compactorCfg Config - storageCfg mimir_tsdb.BlocksStorageConfig cfgProvider ConfigProvider logger log.Logger parentLogger log.Logger @@ -231,7 +228,6 @@ type MultitenantCompactor struct { // Functions that creates bucket client, grouper, planner and compactor using the context. // Useful for injecting mock objects from tests. - bucketClientFactory func(ctx context.Context) (objstore.Bucket, error) blocksGrouperFactory BlocksGrouperFactory blocksCompactorFactory BlocksCompactorFactory @@ -281,11 +277,7 @@ type MultitenantCompactor struct { } // NewMultitenantCompactor makes a new MultitenantCompactor. -func NewMultitenantCompactor(compactorCfg Config, storageCfg mimir_tsdb.BlocksStorageConfig, cfgProvider ConfigProvider, logger log.Logger, registerer prometheus.Registerer) (*MultitenantCompactor, error) { - bucketClientFactory := func(ctx context.Context) (objstore.Bucket, error) { - return bucket.NewClient(ctx, storageCfg.Bucket, "compactor", logger, registerer) - } - +func NewMultitenantCompactor(compactorCfg Config, bucketClient objstore.Bucket, cfgProvider ConfigProvider, logger log.Logger, registerer prometheus.Registerer) (*MultitenantCompactor, error) { // Configure the compactor and grouper factories only if they weren't already set by a downstream project. if compactorCfg.BlocksGrouperFactory == nil || compactorCfg.BlocksCompactorFactory == nil { configureSplitAndMergeCompactor(&compactorCfg) @@ -294,7 +286,7 @@ func NewMultitenantCompactor(compactorCfg Config, storageCfg mimir_tsdb.BlocksSt blocksGrouperFactory := compactorCfg.BlocksGrouperFactory blocksCompactorFactory := compactorCfg.BlocksCompactorFactory - mimirCompactor, err := newMultitenantCompactor(compactorCfg, storageCfg, cfgProvider, logger, registerer, bucketClientFactory, blocksGrouperFactory, blocksCompactorFactory) + mimirCompactor, err := newMultitenantCompactor(compactorCfg, bucketClient, cfgProvider, logger, registerer, blocksGrouperFactory, blocksCompactorFactory) if err != nil { return nil, errors.Wrap(err, "failed to create blocks compactor") } @@ -304,23 +296,21 @@ func NewMultitenantCompactor(compactorCfg Config, storageCfg mimir_tsdb.BlocksSt func newMultitenantCompactor( compactorCfg Config, - storageCfg mimir_tsdb.BlocksStorageConfig, + bucketClient objstore.Bucket, cfgProvider ConfigProvider, logger log.Logger, registerer prometheus.Registerer, - bucketClientFactory func(ctx context.Context) (objstore.Bucket, error), blocksGrouperFactory BlocksGrouperFactory, blocksCompactorFactory BlocksCompactorFactory, ) (*MultitenantCompactor, error) { c := &MultitenantCompactor{ compactorCfg: compactorCfg, - storageCfg: storageCfg, cfgProvider: cfgProvider, parentLogger: logger, logger: log.With(logger, "component", "compactor"), registerer: registerer, syncerMetrics: newAggregatedSyncerMetrics(registerer), - bucketClientFactory: bucketClientFactory, + bucketClient: bucketClient, blocksGrouperFactory: blocksGrouperFactory, blocksCompactorFactory: blocksCompactorFactory, @@ -418,12 +408,6 @@ func newMultitenantCompactor( func (c *MultitenantCompactor) starting(ctx context.Context) error { var err error - // Create bucket client. - c.bucketClient, err = c.bucketClientFactory(ctx) - if err != nil { - return errors.Wrap(err, "failed to create bucket client") - } - // Create blocks compactor dependencies. c.blocksCompactor, c.blocksPlanner, err = c.blocksCompactorFactory(ctx, c.compactorCfg, c.logger, c.registerer) if err != nil { @@ -482,7 +466,7 @@ func (c *MultitenantCompactor) starting(ctx context.Context) error { } } - allowedTenants := util.NewAllowedTenants(c.compactorCfg.EnabledTenants, c.compactorCfg.DisabledTenants) + allowedTenants := tenant.NewAllowedTenants(c.compactorCfg.EnabledTenants, c.compactorCfg.DisabledTenants) c.shardingStrategy = newSplitAndMergeShardingStrategy(allowedTenants, c.ring, c.ringLifecycler, c.cfgProvider) // Create the blocks cleaner (service). @@ -628,7 +612,7 @@ func (c *MultitenantCompactor) compactUsers(ctx context.Context) { ownedUsers[userID] = struct{}{} - if markedForDeletion, err := mimir_tsdb.TenantDeletionMarkExists(ctx, c.bucketClient, userID); err != nil { + if markedForDeletion, err := bucket.TenantDeletionMarkExists(ctx, c.bucketClient, userID); err != nil { c.compactionRunSkippedTenants.Inc() level.Warn(c.logger).Log("msg", "unable to check if user is marked for deletion", "user", userID, "err", err) continue @@ -710,11 +694,11 @@ func (c *MultitenantCompactor) compactUserWithRetries(ctx context.Context, userI } func (c *MultitenantCompactor) compactUser(ctx context.Context, userID string) error { - userBucket := bucket.NewUserBucketClient(userID, c.bucketClient, c.cfgProvider) + userBucket := objstore.NewUserBucketClient(userID, c.bucketClient, c.cfgProvider) reg := prometheus.NewRegistry() defer c.syncerMetrics.gatherThanosSyncerMetrics(reg) - userLogger := util_log.WithUserID(userID, c.logger) + userLogger := util.LoggerWithUserID(userID, c.logger) // Filters out duplicate blocks that can be formed from two or more overlapping // blocks that fully submatches the source blocks of the older blocks. @@ -722,14 +706,6 @@ func (c *MultitenantCompactor) compactUser(ctx context.Context, userID string) e // List of filters to apply (order matters). fetcherFilters := []block.MetadataFilter{ - // Remove the ingester ID because we don't shard blocks anymore, while still - // honoring the shard ID if sharding was done in the past. - // Remove TenantID external label to make sure that we compact blocks with and without the label - // together. - NewLabelRemoverFilter([]string{ - mimir_tsdb.DeprecatedTenantIDExternalLabel, - mimir_tsdb.DeprecatedIngesterIDExternalLabel, - }), deduplicateBlocksFilter, // removes blocks that should not be compacted due to being marked so. NewNoCompactionMarkFilter(userBucket, true), @@ -768,7 +744,6 @@ func (c *MultitenantCompactor) compactUser(ctx context.Context, userID string) e path.Join(c.compactorCfg.DataDir, "compact"), userBucket, c.compactorCfg.CompactionConcurrency, - true, // Skip blocks with out of order chunks, and mark them for no-compaction. c.shardingStrategy.ownJob, c.jobsOrder, c.compactorCfg.CompactionWaitPeriod, @@ -810,7 +785,7 @@ func (c *MultitenantCompactor) discoverUsersWithRetries(ctx context.Context) ([] } func (c *MultitenantCompactor) discoverUsers(ctx context.Context) ([]string, error) { - return mimir_tsdb.ListUsers(ctx, c.bucketClient) + return bucket.ListUsers(ctx, c.bucketClient) } // shardingStrategy describes whether compactor "owns" given user or job. @@ -826,13 +801,13 @@ type shardingStrategy interface { // Each job is only owned and executed by single compactor. // Only one of compactors from user's shard will do cleanup. type splitAndMergeShardingStrategy struct { - allowedTenants *util.AllowedTenants + allowedTenants *tenant.AllowedTenants ring *ring.Ring ringLifecycler *ring.BasicLifecycler configProvider ConfigProvider } -func newSplitAndMergeShardingStrategy(allowedTenants *util.AllowedTenants, ring *ring.Ring, ringLifecycler *ring.BasicLifecycler, configProvider ConfigProvider) *splitAndMergeShardingStrategy { +func newSplitAndMergeShardingStrategy(allowedTenants *tenant.AllowedTenants, ring *ring.Ring, ringLifecycler *ring.BasicLifecycler, configProvider ConfigProvider) *splitAndMergeShardingStrategy { return &splitAndMergeShardingStrategy{ allowedTenants: allowedTenants, ring: ring, diff --git a/pkg/compactor/duration_list.go b/pkg/compactor/duration_list.go new file mode 100644 index 0000000000..1f3e5b15a3 --- /dev/null +++ b/pkg/compactor/duration_list.go @@ -0,0 +1,43 @@ +package compactor + +import ( + "strings" + "time" +) + +// DurationList is the block ranges for a tsdb +type DurationList []time.Duration + +// String implements the flag.Value interface +func (d *DurationList) String() string { + values := make([]string, 0, len(*d)) + for _, v := range *d { + values = append(values, v.String()) + } + + return strings.Join(values, ",") +} + +// Set implements the flag.Value interface +func (d *DurationList) Set(s string) error { + values := strings.Split(s, ",") + *d = make([]time.Duration, 0, len(values)) // flag.Parse may be called twice, so overwrite instead of append + for _, v := range values { + t, err := time.ParseDuration(v) + if err != nil { + return err + } + *d = append(*d, t) + } + return nil +} + +// ToMilliseconds returns the duration list in milliseconds +func (d *DurationList) ToMilliseconds() []int64 { + values := make([]int64, 0, len(*d)) + for _, t := range *d { + values = append(values, t.Milliseconds()) + } + + return values +} diff --git a/pkg/tenant/allowed_tenants.go b/pkg/tenant/allowed_tenants.go new file mode 100644 index 0000000000..9e83558d06 --- /dev/null +++ b/pkg/tenant/allowed_tenants.go @@ -0,0 +1,59 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/cortexproject/cortex/blob/master/pkg/util/allowed_tenants.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Cortex Authors. + +package tenant + +// AllowedTenants that can answer whether tenant is allowed or not based on configuration. +// Default value (nil) allows all tenants. +type AllowedTenants struct { + // If empty, all tenants are enabled. If not empty, only tenants in the map are enabled. + enabled map[string]struct{} + + // If empty, no tenants are disabled. If not empty, tenants in the map are disabled. + disabled map[string]struct{} +} + +// NewAllowedTenants builds new allowed tenants based on enabled and disabled tenants. +// If there are any enabled tenants, then only those tenants are allowed. +// If there are any disabled tenants, then tenant from that list, that would normally be allowed, is disabled instead. +func NewAllowedTenants(enabled []string, disabled []string) *AllowedTenants { + a := &AllowedTenants{} + + if len(enabled) > 0 { + a.enabled = make(map[string]struct{}, len(enabled)) + for _, u := range enabled { + a.enabled[u] = struct{}{} + } + } + + if len(disabled) > 0 { + a.disabled = make(map[string]struct{}, len(disabled)) + for _, u := range disabled { + a.disabled[u] = struct{}{} + } + } + + return a +} + +func (a *AllowedTenants) IsAllowed(tenantID string) bool { + if a == nil { + return true + } + + if len(a.enabled) > 0 { + if _, ok := a.enabled[tenantID]; !ok { + return false + } + } + + if len(a.disabled) > 0 { + if _, ok := a.disabled[tenantID]; ok { + return false + } + } + + return true +} diff --git a/pkg/tenant/allowed_tenants_test.go b/pkg/tenant/allowed_tenants_test.go new file mode 100644 index 0000000000..6647377dc7 --- /dev/null +++ b/pkg/tenant/allowed_tenants_test.go @@ -0,0 +1,52 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/cortexproject/cortex/blob/master/pkg/util/allowed_tenants_test.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Cortex Authors. + +package tenant + +import ( + "testing" + + "github.com/stretchr/testify/require" +) + +func TestAllowedTenants_NoConfig(t *testing.T) { + a := NewAllowedTenants(nil, nil) + require.True(t, a.IsAllowed("all")) + require.True(t, a.IsAllowed("tenants")) + require.True(t, a.IsAllowed("allowed")) +} + +func TestAllowedTenants_Enabled(t *testing.T) { + a := NewAllowedTenants([]string{"A", "B"}, nil) + require.True(t, a.IsAllowed("A")) + require.True(t, a.IsAllowed("B")) + require.False(t, a.IsAllowed("C")) + require.False(t, a.IsAllowed("D")) +} + +func TestAllowedTenants_Disabled(t *testing.T) { + a := NewAllowedTenants(nil, []string{"A", "B"}) + require.False(t, a.IsAllowed("A")) + require.False(t, a.IsAllowed("B")) + require.True(t, a.IsAllowed("C")) + require.True(t, a.IsAllowed("D")) +} + +func TestAllowedTenants_Combination(t *testing.T) { + a := NewAllowedTenants([]string{"A", "B"}, []string{"B", "C"}) + require.True(t, a.IsAllowed("A")) // enabled, and not disabled + require.False(t, a.IsAllowed("B")) // enabled, but also disabled + require.False(t, a.IsAllowed("C")) // disabled + require.False(t, a.IsAllowed("D")) // not enabled +} + +func TestAllowedTenants_Nil(t *testing.T) { + var a *Allowed + + // All tenants are allowed when using nil as allowed tenants. + require.True(t, a.IsAllowed("A")) + require.True(t, a.IsAllowed("B")) + require.True(t, a.IsAllowed("C")) +} From 8595c649f331cdba797b03cb8e6d9f4659a559b0 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Tue, 12 Sep 2023 23:24:25 +0200 Subject: [PATCH 27/74] fixes metrics namespace --- pkg/compactor/blocks_cleaner_test.go | 294 +++--- pkg/compactor/compactor.go | 32 +- pkg/compactor/compactor_test.go | 1008 +++++++++---------- pkg/compactor/split_merge_compactor_test.go | 16 +- pkg/compactor/syncer_metrics_test.go | 108 +- 5 files changed, 729 insertions(+), 729 deletions(-) diff --git a/pkg/compactor/blocks_cleaner_test.go b/pkg/compactor/blocks_cleaner_test.go index 193e28e101..fc8fb17ccf 100644 --- a/pkg/compactor/blocks_cleaner_test.go +++ b/pkg/compactor/blocks_cleaner_test.go @@ -191,22 +191,22 @@ func testBlocksCleanerWithOptions(t *testing.T, options testBlocksCleanerOptions } assert.NoError(t, testutil.GatherAndCompare(reg, strings.NewReader(` - # HELP cortex_bucket_blocks_count Total number of blocks in the bucket. Includes blocks marked for deletion, but not partial blocks. - # TYPE cortex_bucket_blocks_count gauge - cortex_bucket_blocks_count{user="user-1"} 2 - cortex_bucket_blocks_count{user="user-2"} 1 - # HELP cortex_bucket_blocks_marked_for_deletion_count Total number of blocks marked for deletion in the bucket. - # TYPE cortex_bucket_blocks_marked_for_deletion_count gauge - cortex_bucket_blocks_marked_for_deletion_count{user="user-1"} 1 - cortex_bucket_blocks_marked_for_deletion_count{user="user-2"} 0 - # HELP cortex_bucket_blocks_partials_count Total number of partial blocks. - # TYPE cortex_bucket_blocks_partials_count gauge - cortex_bucket_blocks_partials_count{user="user-1"} 2 - cortex_bucket_blocks_partials_count{user="user-2"} 0 + # HELP pyroscope_bucket_blocks_count Total number of blocks in the bucket. Includes blocks marked for deletion, but not partial blocks. + # TYPE pyroscope_bucket_blocks_count gauge + pyroscope_bucket_blocks_count{user="user-1"} 2 + pyroscope_bucket_blocks_count{user="user-2"} 1 + # HELP pyroscope_bucket_blocks_marked_for_deletion_count Total number of blocks marked for deletion in the bucket. + # TYPE pyroscope_bucket_blocks_marked_for_deletion_count gauge + pyroscope_bucket_blocks_marked_for_deletion_count{user="user-1"} 1 + pyroscope_bucket_blocks_marked_for_deletion_count{user="user-2"} 0 + # HELP pyroscope_bucket_blocks_partials_count Total number of partial blocks. + # TYPE pyroscope_bucket_blocks_partials_count gauge + pyroscope_bucket_blocks_partials_count{user="user-1"} 2 + pyroscope_bucket_blocks_partials_count{user="user-2"} 0 `), - "cortex_bucket_blocks_count", - "cortex_bucket_blocks_marked_for_deletion_count", - "cortex_bucket_blocks_partials_count", + "pyroscope_bucket_blocks_count", + "pyroscope_bucket_blocks_marked_for_deletion_count", + "pyroscope_bucket_blocks_partials_count", )) } @@ -359,22 +359,22 @@ func TestBlocksCleaner_ShouldRemoveMetricsForTenantsNotBelongingAnymoreToTheShar require.NoError(t, cleaner.runCleanupWithErr(ctx)) assert.NoError(t, testutil.GatherAndCompare(reg, strings.NewReader(` - # HELP cortex_bucket_blocks_count Total number of blocks in the bucket. Includes blocks marked for deletion, but not partial blocks. - # TYPE cortex_bucket_blocks_count gauge - cortex_bucket_blocks_count{user="user-1"} 2 - cortex_bucket_blocks_count{user="user-2"} 1 - # HELP cortex_bucket_blocks_marked_for_deletion_count Total number of blocks marked for deletion in the bucket. - # TYPE cortex_bucket_blocks_marked_for_deletion_count gauge - cortex_bucket_blocks_marked_for_deletion_count{user="user-1"} 0 - cortex_bucket_blocks_marked_for_deletion_count{user="user-2"} 0 - # HELP cortex_bucket_blocks_partials_count Total number of partial blocks. - # TYPE cortex_bucket_blocks_partials_count gauge - cortex_bucket_blocks_partials_count{user="user-1"} 0 - cortex_bucket_blocks_partials_count{user="user-2"} 0 + # HELP pyroscope_bucket_blocks_count Total number of blocks in the bucket. Includes blocks marked for deletion, but not partial blocks. + # TYPE pyroscope_bucket_blocks_count gauge + pyroscope_bucket_blocks_count{user="user-1"} 2 + pyroscope_bucket_blocks_count{user="user-2"} 1 + # HELP pyroscope_bucket_blocks_marked_for_deletion_count Total number of blocks marked for deletion in the bucket. + # TYPE pyroscope_bucket_blocks_marked_for_deletion_count gauge + pyroscope_bucket_blocks_marked_for_deletion_count{user="user-1"} 0 + pyroscope_bucket_blocks_marked_for_deletion_count{user="user-2"} 0 + # HELP pyroscope_bucket_blocks_partials_count Total number of partial blocks. + # TYPE pyroscope_bucket_blocks_partials_count gauge + pyroscope_bucket_blocks_partials_count{user="user-1"} 0 + pyroscope_bucket_blocks_partials_count{user="user-2"} 0 `), - "cortex_bucket_blocks_count", - "cortex_bucket_blocks_marked_for_deletion_count", - "cortex_bucket_blocks_partials_count", + "pyroscope_bucket_blocks_count", + "pyroscope_bucket_blocks_marked_for_deletion_count", + "pyroscope_bucket_blocks_partials_count", )) // Override the users scanner to reconfigure it to only return a subset of users. @@ -387,19 +387,19 @@ func TestBlocksCleaner_ShouldRemoveMetricsForTenantsNotBelongingAnymoreToTheShar require.NoError(t, cleaner.runCleanupWithErr(ctx)) assert.NoError(t, testutil.GatherAndCompare(reg, strings.NewReader(` - # HELP cortex_bucket_blocks_count Total number of blocks in the bucket. Includes blocks marked for deletion, but not partial blocks. - # TYPE cortex_bucket_blocks_count gauge - cortex_bucket_blocks_count{user="user-1"} 3 - # HELP cortex_bucket_blocks_marked_for_deletion_count Total number of blocks marked for deletion in the bucket. - # TYPE cortex_bucket_blocks_marked_for_deletion_count gauge - cortex_bucket_blocks_marked_for_deletion_count{user="user-1"} 0 - # HELP cortex_bucket_blocks_partials_count Total number of partial blocks. - # TYPE cortex_bucket_blocks_partials_count gauge - cortex_bucket_blocks_partials_count{user="user-1"} 0 + # HELP pyroscope_bucket_blocks_count Total number of blocks in the bucket. Includes blocks marked for deletion, but not partial blocks. + # TYPE pyroscope_bucket_blocks_count gauge + pyroscope_bucket_blocks_count{user="user-1"} 3 + # HELP pyroscope_bucket_blocks_marked_for_deletion_count Total number of blocks marked for deletion in the bucket. + # TYPE pyroscope_bucket_blocks_marked_for_deletion_count gauge + pyroscope_bucket_blocks_marked_for_deletion_count{user="user-1"} 0 + # HELP pyroscope_bucket_blocks_partials_count Total number of partial blocks. + # TYPE pyroscope_bucket_blocks_partials_count gauge + pyroscope_bucket_blocks_partials_count{user="user-1"} 0 `), - "cortex_bucket_blocks_count", - "cortex_bucket_blocks_marked_for_deletion_count", - "cortex_bucket_blocks_partials_count", + "pyroscope_bucket_blocks_count", + "pyroscope_bucket_blocks_marked_for_deletion_count", + "pyroscope_bucket_blocks_partials_count", )) } @@ -443,10 +443,10 @@ func TestBlocksCleaner_ShouldNotCleanupUserThatDoesntBelongToShardAnymore(t *tes // But there are no metrics for any user, because we did not in fact clean them. assert.NoError(t, testutil.GatherAndCompare(reg, strings.NewReader(` - # HELP cortex_bucket_blocks_count Total number of blocks in the bucket. Includes blocks marked for deletion, but not partial blocks. - # TYPE cortex_bucket_blocks_count gauge + # HELP pyroscope_bucket_blocks_count Total number of blocks in the bucket. Includes blocks marked for deletion, but not partial blocks. + # TYPE pyroscope_bucket_blocks_count gauge `), - "cortex_bucket_blocks_count", + "pyroscope_bucket_blocks_count", )) // Running cleanUsers again will see that users are no longer owned. @@ -557,22 +557,22 @@ func TestBlocksCleaner_ShouldRemoveBlocksOutsideRetentionPeriod(t *testing.T) { assertBlockExists("user-2", block4, true) assert.NoError(t, testutil.GatherAndCompare(reg, strings.NewReader(` - # HELP cortex_bucket_blocks_count Total number of blocks in the bucket. Includes blocks marked for deletion, but not partial blocks. - # TYPE cortex_bucket_blocks_count gauge - cortex_bucket_blocks_count{user="user-1"} 2 - cortex_bucket_blocks_count{user="user-2"} 2 - # HELP cortex_bucket_blocks_marked_for_deletion_count Total number of blocks marked for deletion in the bucket. - # TYPE cortex_bucket_blocks_marked_for_deletion_count gauge - cortex_bucket_blocks_marked_for_deletion_count{user="user-1"} 0 - cortex_bucket_blocks_marked_for_deletion_count{user="user-2"} 0 - # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. - # TYPE cortex_compactor_blocks_marked_for_deletion_total counter - cortex_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 - cortex_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 + # HELP pyroscope_bucket_blocks_count Total number of blocks in the bucket. Includes blocks marked for deletion, but not partial blocks. + # TYPE pyroscope_bucket_blocks_count gauge + pyroscope_bucket_blocks_count{user="user-1"} 2 + pyroscope_bucket_blocks_count{user="user-2"} 2 + # HELP pyroscope_bucket_blocks_marked_for_deletion_count Total number of blocks marked for deletion in the bucket. + # TYPE pyroscope_bucket_blocks_marked_for_deletion_count gauge + pyroscope_bucket_blocks_marked_for_deletion_count{user="user-1"} 0 + pyroscope_bucket_blocks_marked_for_deletion_count{user="user-2"} 0 + # HELP pyroscope_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. + # TYPE pyroscope_compactor_blocks_marked_for_deletion_total counter + pyroscope_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 + pyroscope_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 `), - "cortex_bucket_blocks_count", - "cortex_bucket_blocks_marked_for_deletion_count", - "cortex_compactor_blocks_marked_for_deletion_total", + "pyroscope_bucket_blocks_count", + "pyroscope_bucket_blocks_marked_for_deletion_count", + "pyroscope_compactor_blocks_marked_for_deletion_total", )) } @@ -599,22 +599,22 @@ func TestBlocksCleaner_ShouldRemoveBlocksOutsideRetentionPeriod(t *testing.T) { assertBlockExists("user-2", block4, true) assert.NoError(t, testutil.GatherAndCompare(reg, strings.NewReader(` - # HELP cortex_bucket_blocks_count Total number of blocks in the bucket. Includes blocks marked for deletion, but not partial blocks. - # TYPE cortex_bucket_blocks_count gauge - cortex_bucket_blocks_count{user="user-1"} 2 - cortex_bucket_blocks_count{user="user-2"} 2 - # HELP cortex_bucket_blocks_marked_for_deletion_count Total number of blocks marked for deletion in the bucket. - # TYPE cortex_bucket_blocks_marked_for_deletion_count gauge - cortex_bucket_blocks_marked_for_deletion_count{user="user-1"} 1 - cortex_bucket_blocks_marked_for_deletion_count{user="user-2"} 0 - # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. - # TYPE cortex_compactor_blocks_marked_for_deletion_total counter - cortex_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 - cortex_compactor_blocks_marked_for_deletion_total{reason="retention"} 1 + # HELP pyroscope_bucket_blocks_count Total number of blocks in the bucket. Includes blocks marked for deletion, but not partial blocks. + # TYPE pyroscope_bucket_blocks_count gauge + pyroscope_bucket_blocks_count{user="user-1"} 2 + pyroscope_bucket_blocks_count{user="user-2"} 2 + # HELP pyroscope_bucket_blocks_marked_for_deletion_count Total number of blocks marked for deletion in the bucket. + # TYPE pyroscope_bucket_blocks_marked_for_deletion_count gauge + pyroscope_bucket_blocks_marked_for_deletion_count{user="user-1"} 1 + pyroscope_bucket_blocks_marked_for_deletion_count{user="user-2"} 0 + # HELP pyroscope_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. + # TYPE pyroscope_compactor_blocks_marked_for_deletion_total counter + pyroscope_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 + pyroscope_compactor_blocks_marked_for_deletion_total{reason="retention"} 1 `), - "cortex_bucket_blocks_count", - "cortex_bucket_blocks_marked_for_deletion_count", - "cortex_compactor_blocks_marked_for_deletion_total", + "pyroscope_bucket_blocks_count", + "pyroscope_bucket_blocks_marked_for_deletion_count", + "pyroscope_compactor_blocks_marked_for_deletion_total", )) } @@ -638,22 +638,22 @@ func TestBlocksCleaner_ShouldRemoveBlocksOutsideRetentionPeriod(t *testing.T) { assertBlockExists("user-2", block4, true) assert.NoError(t, testutil.GatherAndCompare(reg, strings.NewReader(` - # HELP cortex_bucket_blocks_count Total number of blocks in the bucket. Includes blocks marked for deletion, but not partial blocks. - # TYPE cortex_bucket_blocks_count gauge - cortex_bucket_blocks_count{user="user-1"} 1 - cortex_bucket_blocks_count{user="user-2"} 2 - # HELP cortex_bucket_blocks_marked_for_deletion_count Total number of blocks marked for deletion in the bucket. - # TYPE cortex_bucket_blocks_marked_for_deletion_count gauge - cortex_bucket_blocks_marked_for_deletion_count{user="user-1"} 0 - cortex_bucket_blocks_marked_for_deletion_count{user="user-2"} 0 - # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. - # TYPE cortex_compactor_blocks_marked_for_deletion_total counter - cortex_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 - cortex_compactor_blocks_marked_for_deletion_total{reason="retention"} 1 + # HELP pyroscope_bucket_blocks_count Total number of blocks in the bucket. Includes blocks marked for deletion, but not partial blocks. + # TYPE pyroscope_bucket_blocks_count gauge + pyroscope_bucket_blocks_count{user="user-1"} 1 + pyroscope_bucket_blocks_count{user="user-2"} 2 + # HELP pyroscope_bucket_blocks_marked_for_deletion_count Total number of blocks marked for deletion in the bucket. + # TYPE pyroscope_bucket_blocks_marked_for_deletion_count gauge + pyroscope_bucket_blocks_marked_for_deletion_count{user="user-1"} 0 + pyroscope_bucket_blocks_marked_for_deletion_count{user="user-2"} 0 + # HELP pyroscope_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. + # TYPE pyroscope_compactor_blocks_marked_for_deletion_total counter + pyroscope_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 + pyroscope_compactor_blocks_marked_for_deletion_total{reason="retention"} 1 `), - "cortex_bucket_blocks_count", - "cortex_bucket_blocks_marked_for_deletion_count", - "cortex_compactor_blocks_marked_for_deletion_total", + "pyroscope_bucket_blocks_count", + "pyroscope_bucket_blocks_marked_for_deletion_count", + "pyroscope_compactor_blocks_marked_for_deletion_total", )) } @@ -668,22 +668,22 @@ func TestBlocksCleaner_ShouldRemoveBlocksOutsideRetentionPeriod(t *testing.T) { assertBlockExists("user-2", block4, false) assert.NoError(t, testutil.GatherAndCompare(reg, strings.NewReader(` - # HELP cortex_bucket_blocks_count Total number of blocks in the bucket. Includes blocks marked for deletion, but not partial blocks. - # TYPE cortex_bucket_blocks_count gauge - cortex_bucket_blocks_count{user="user-1"} 1 - cortex_bucket_blocks_count{user="user-2"} 0 - # HELP cortex_bucket_blocks_marked_for_deletion_count Total number of blocks marked for deletion in the bucket. - # TYPE cortex_bucket_blocks_marked_for_deletion_count gauge - cortex_bucket_blocks_marked_for_deletion_count{user="user-1"} 0 - cortex_bucket_blocks_marked_for_deletion_count{user="user-2"} 0 - # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. - # TYPE cortex_compactor_blocks_marked_for_deletion_total counter - cortex_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 - cortex_compactor_blocks_marked_for_deletion_total{reason="retention"} 3 + # HELP pyroscope_bucket_blocks_count Total number of blocks in the bucket. Includes blocks marked for deletion, but not partial blocks. + # TYPE pyroscope_bucket_blocks_count gauge + pyroscope_bucket_blocks_count{user="user-1"} 1 + pyroscope_bucket_blocks_count{user="user-2"} 0 + # HELP pyroscope_bucket_blocks_marked_for_deletion_count Total number of blocks marked for deletion in the bucket. + # TYPE pyroscope_bucket_blocks_marked_for_deletion_count gauge + pyroscope_bucket_blocks_marked_for_deletion_count{user="user-1"} 0 + pyroscope_bucket_blocks_marked_for_deletion_count{user="user-2"} 0 + # HELP pyroscope_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. + # TYPE pyroscope_compactor_blocks_marked_for_deletion_total counter + pyroscope_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 + pyroscope_compactor_blocks_marked_for_deletion_total{reason="retention"} 3 `), - "cortex_bucket_blocks_count", - "cortex_bucket_blocks_marked_for_deletion_count", - "cortex_compactor_blocks_marked_for_deletion_total", + "pyroscope_bucket_blocks_count", + "pyroscope_bucket_blocks_marked_for_deletion_count", + "pyroscope_compactor_blocks_marked_for_deletion_total", )) } } @@ -810,20 +810,20 @@ func TestBlocksCleaner_ShouldRemovePartialBlocksOutsideDelayPeriod(t *testing.T) checkBlock(t, "user-1", bucketClient, block2, true, false) require.NoError(t, testutil.GatherAndCompare(reg, strings.NewReader(` - # HELP cortex_bucket_blocks_count Total number of blocks in the bucket. Includes blocks marked for deletion, but not partial blocks. - # TYPE cortex_bucket_blocks_count gauge - cortex_bucket_blocks_count{user="user-1"} 1 - # HELP cortex_bucket_blocks_marked_for_deletion_count Total number of blocks marked for deletion in the bucket. - # TYPE cortex_bucket_blocks_marked_for_deletion_count gauge - cortex_bucket_blocks_marked_for_deletion_count{user="user-1"} 0 - # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. - # TYPE cortex_compactor_blocks_marked_for_deletion_total counter - cortex_compactor_blocks_marked_for_deletion_total{reason="partial"} 1 - cortex_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 + # HELP pyroscope_bucket_blocks_count Total number of blocks in the bucket. Includes blocks marked for deletion, but not partial blocks. + # TYPE pyroscope_bucket_blocks_count gauge + pyroscope_bucket_blocks_count{user="user-1"} 1 + # HELP pyroscope_bucket_blocks_marked_for_deletion_count Total number of blocks marked for deletion in the bucket. + # TYPE pyroscope_bucket_blocks_marked_for_deletion_count gauge + pyroscope_bucket_blocks_marked_for_deletion_count{user="user-1"} 0 + # HELP pyroscope_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. + # TYPE pyroscope_compactor_blocks_marked_for_deletion_total counter + pyroscope_compactor_blocks_marked_for_deletion_total{reason="partial"} 1 + pyroscope_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 `), - "cortex_bucket_blocks_count", - "cortex_bucket_blocks_marked_for_deletion_count", - "cortex_compactor_blocks_marked_for_deletion_total", + "pyroscope_bucket_blocks_count", + "pyroscope_bucket_blocks_marked_for_deletion_count", + "pyroscope_compactor_blocks_marked_for_deletion_total", )) } @@ -885,26 +885,26 @@ func TestBlocksCleaner_ShouldNotRemovePartialBlocksInsideDelayPeriod(t *testing. checkBlock(t, "user-1", bucketClient, block1, false, false) // No change for user-1 checkBlock(t, "user-2", bucketClient, block2, true, false) // Block with corrupted meta is NOT marked for deletion. - // The cortex_compactor_blocks_marked_for_deletion_total{reason="partial"} counter should be zero since for user-1 + // The pyroscope_compactor_blocks_marked_for_deletion_total{reason="partial"} counter should be zero since for user-1 // the time since modification is shorter than the delay, and for user-2, the metadata is corrupted but the file // is still present in the bucket so the block is not partial require.NoError(t, testutil.GatherAndCompare(reg, strings.NewReader(` - # HELP cortex_bucket_blocks_count Total number of blocks in the bucket. Includes blocks marked for deletion, but not partial blocks. - # TYPE cortex_bucket_blocks_count gauge - cortex_bucket_blocks_count{user="user-1"} 0 - cortex_bucket_blocks_count{user="user-2"} 0 - # HELP cortex_bucket_blocks_marked_for_deletion_count Total number of blocks marked for deletion in the bucket. - # TYPE cortex_bucket_blocks_marked_for_deletion_count gauge - cortex_bucket_blocks_marked_for_deletion_count{user="user-1"} 0 - cortex_bucket_blocks_marked_for_deletion_count{user="user-2"} 0 - # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. - # TYPE cortex_compactor_blocks_marked_for_deletion_total counter - cortex_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 - cortex_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 + # HELP pyroscope_bucket_blocks_count Total number of blocks in the bucket. Includes blocks marked for deletion, but not partial blocks. + # TYPE pyroscope_bucket_blocks_count gauge + pyroscope_bucket_blocks_count{user="user-1"} 0 + pyroscope_bucket_blocks_count{user="user-2"} 0 + # HELP pyroscope_bucket_blocks_marked_for_deletion_count Total number of blocks marked for deletion in the bucket. + # TYPE pyroscope_bucket_blocks_marked_for_deletion_count gauge + pyroscope_bucket_blocks_marked_for_deletion_count{user="user-1"} 0 + pyroscope_bucket_blocks_marked_for_deletion_count{user="user-2"} 0 + # HELP pyroscope_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. + # TYPE pyroscope_compactor_blocks_marked_for_deletion_total counter + pyroscope_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 + pyroscope_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 `), - "cortex_bucket_blocks_count", - "cortex_bucket_blocks_marked_for_deletion_count", - "cortex_compactor_blocks_marked_for_deletion_total", + "pyroscope_bucket_blocks_count", + "pyroscope_bucket_blocks_marked_for_deletion_count", + "pyroscope_compactor_blocks_marked_for_deletion_total", )) } @@ -950,20 +950,20 @@ func TestBlocksCleaner_ShouldNotRemovePartialBlocksIfConfiguredDelayIsInvalid(t assert.Contains(t, logs.String(), "partial blocks deletion has been disabled for tenant because the delay has been set lower than the minimum value allowed") require.NoError(t, testutil.GatherAndCompare(reg, strings.NewReader(` - # HELP cortex_bucket_blocks_count Total number of blocks in the bucket. Includes blocks marked for deletion, but not partial blocks. - # TYPE cortex_bucket_blocks_count gauge - cortex_bucket_blocks_count{user="user-1"} 0 - # HELP cortex_bucket_blocks_marked_for_deletion_count Total number of blocks marked for deletion in the bucket. - # TYPE cortex_bucket_blocks_marked_for_deletion_count gauge - cortex_bucket_blocks_marked_for_deletion_count{user="user-1"} 0 - # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. - # TYPE cortex_compactor_blocks_marked_for_deletion_total counter - cortex_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 - cortex_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 + # HELP pyroscope_bucket_blocks_count Total number of blocks in the bucket. Includes blocks marked for deletion, but not partial blocks. + # TYPE pyroscope_bucket_blocks_count gauge + pyroscope_bucket_blocks_count{user="user-1"} 0 + # HELP pyroscope_bucket_blocks_marked_for_deletion_count Total number of blocks marked for deletion in the bucket. + # TYPE pyroscope_bucket_blocks_marked_for_deletion_count gauge + pyroscope_bucket_blocks_marked_for_deletion_count{user="user-1"} 0 + # HELP pyroscope_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. + # TYPE pyroscope_compactor_blocks_marked_for_deletion_total counter + pyroscope_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 + pyroscope_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 `), - "cortex_bucket_blocks_count", - "cortex_bucket_blocks_marked_for_deletion_count", - "cortex_compactor_blocks_marked_for_deletion_total", + "pyroscope_bucket_blocks_count", + "pyroscope_bucket_blocks_marked_for_deletion_count", + "pyroscope_compactor_blocks_marked_for_deletion_total", )) } diff --git a/pkg/compactor/compactor.go b/pkg/compactor/compactor.go index d0bc2e8be8..b2c96a21d1 100644 --- a/pkg/compactor/compactor.go +++ b/pkg/compactor/compactor.go @@ -45,7 +45,7 @@ const ( ) const ( - blocksMarkedForDeletionName = "cortex_compactor_blocks_marked_for_deletion_total" + blocksMarkedForDeletionName = "gitcompactor_blocks_marked_for_deletion_total" blocksMarkedForDeletionHelp = "Total number of blocks marked for deletion in compactor." ) @@ -315,45 +315,45 @@ func newMultitenantCompactor( blocksCompactorFactory: blocksCompactorFactory, compactionRunsStarted: promauto.With(registerer).NewCounter(prometheus.CounterOpts{ - Name: "cortex_compactor_runs_started_total", + Name: "pyroscope_compactor_runs_started_total", Help: "Total number of compaction runs started.", }), compactionRunsCompleted: promauto.With(registerer).NewCounter(prometheus.CounterOpts{ - Name: "cortex_compactor_runs_completed_total", + Name: "pyroscope_compactor_runs_completed_total", Help: "Total number of compaction runs successfully completed.", }), compactionRunsErred: promauto.With(registerer).NewCounter(prometheus.CounterOpts{ - Name: "cortex_compactor_runs_failed_total", + Name: "pyroscope_compactor_runs_failed_total", Help: "Total number of compaction runs failed.", ConstLabels: map[string]string{"reason": "error"}, }), compactionRunsShutdown: promauto.With(registerer).NewCounter(prometheus.CounterOpts{ - Name: "cortex_compactor_runs_failed_total", + Name: "pyroscope_compactor_runs_failed_total", Help: "Total number of compaction runs failed.", ConstLabels: map[string]string{"reason": "shutdown"}, }), compactionRunsLastSuccess: promauto.With(registerer).NewGauge(prometheus.GaugeOpts{ - Name: "cortex_compactor_last_successful_run_timestamp_seconds", + Name: "pyroscope_compactor_last_successful_run_timestamp_seconds", Help: "Unix timestamp of the last successful compaction run.", }), compactionRunDiscoveredTenants: promauto.With(registerer).NewGauge(prometheus.GaugeOpts{ - Name: "cortex_compactor_tenants_discovered", + Name: "pyroscope_compactor_tenants_discovered", Help: "Number of tenants discovered during the current compaction run. Reset to 0 when compactor is idle.", }), compactionRunSkippedTenants: promauto.With(registerer).NewGauge(prometheus.GaugeOpts{ - Name: "cortex_compactor_tenants_skipped", + Name: "pyroscope_compactor_tenants_skipped", Help: "Number of tenants skipped during the current compaction run. Reset to 0 when compactor is idle.", }), compactionRunSucceededTenants: promauto.With(registerer).NewGauge(prometheus.GaugeOpts{ - Name: "cortex_compactor_tenants_processing_succeeded", + Name: "pyroscope_compactor_tenants_processing_succeeded", Help: "Number of tenants successfully processed during the current compaction run. Reset to 0 when compactor is idle.", }), compactionRunFailedTenants: promauto.With(registerer).NewGauge(prometheus.GaugeOpts{ - Name: "cortex_compactor_tenants_processing_failed", + Name: "pyroscope_compactor_tenants_processing_failed", Help: "Number of tenants failed processing during the current compaction run. Reset to 0 when compactor is idle.", }), compactionRunInterval: promauto.With(registerer).NewGauge(prometheus.GaugeOpts{ - Name: "cortex_compactor_compaction_interval_seconds", + Name: "pyroscope_compactor_compaction_interval_seconds", Help: "The configured interval on which compaction is run in seconds. Useful when compared to the last successful run metric to accurately detect multiple failed compaction runs.", }), blocksMarkedForDeletion: promauto.With(registerer).NewCounter(prometheus.CounterOpts{ @@ -362,21 +362,21 @@ func newMultitenantCompactor( ConstLabels: prometheus.Labels{"reason": "compaction"}, }), blockUploadBlocks: promauto.With(registerer).NewGaugeVec(prometheus.GaugeOpts{ - Name: "cortex_block_upload_api_blocks_total", + Name: "pyroscope_block_upload_api_blocks_total", Help: "Total number of blocks successfully uploaded and validated using the block upload API.", }, []string{"user"}), blockUploadBytes: promauto.With(registerer).NewGaugeVec(prometheus.GaugeOpts{ - Name: "cortex_block_upload_api_bytes_total", + Name: "pyroscope_block_upload_api_bytes_total", Help: "Total number of bytes from successfully uploaded and validated blocks using block upload API.", }, []string{"user"}), blockUploadFiles: promauto.With(registerer).NewGaugeVec(prometheus.GaugeOpts{ - Name: "cortex_block_upload_api_files_total", + Name: "pyroscope_block_upload_api_files_total", Help: "Total number of files from successfully uploaded and validated blocks using block upload API.", }, []string{"user"}), } promauto.With(registerer).NewGaugeFunc(prometheus.GaugeOpts{ - Name: "cortex_block_upload_validations_in_progress", + Name: "pyroscope_block_upload_validations_in_progress", Help: "Number of block upload validations currently running.", }, func() float64 { return float64(c.blockUploadValidations.Load()) @@ -489,7 +489,7 @@ func (c *MultitenantCompactor) starting(ctx context.Context) error { } func newRingAndLifecycler(cfg RingConfig, logger log.Logger, reg prometheus.Registerer) (*ring.Ring, *ring.BasicLifecycler, error) { - reg = prometheus.WrapRegistererWithPrefix("cortex_", reg) + reg = prometheus.WrapRegistererWithPrefix("pyroscope_", reg) kvStore, err := kv.NewClient(cfg.Common.KVStore, ring.GetCodec(), kv.RegistererWithKVName(reg, "compactor-lifecycler"), logger) if err != nil { return nil, nil, errors.Wrap(err, "failed to initialize compactors' KV store") diff --git a/pkg/compactor/compactor_test.go b/pkg/compactor/compactor_test.go index f755040c42..de94412ae2 100644 --- a/pkg/compactor/compactor_test.go +++ b/pkg/compactor/compactor_test.go @@ -175,108 +175,108 @@ func TestMultitenantCompactor_ShouldDoNothingOnNoUserBlocks(t *testing.T) { }, removeIgnoredLogs(strings.Split(strings.TrimSpace(logs.String()), "\n"))) assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` - # TYPE cortex_compactor_runs_started_total counter - # HELP cortex_compactor_runs_started_total Total number of compaction runs started. - cortex_compactor_runs_started_total 1 - - # TYPE cortex_compactor_runs_completed_total counter - # HELP cortex_compactor_runs_completed_total Total number of compaction runs successfully completed. - cortex_compactor_runs_completed_total 1 - - # TYPE cortex_compactor_runs_failed_total counter - # HELP cortex_compactor_runs_failed_total Total number of compaction runs failed. - cortex_compactor_runs_failed_total{reason="error"} 0 - cortex_compactor_runs_failed_total{reason="shutdown"} 0 - - # HELP cortex_compactor_garbage_collection_duration_seconds Time it took to perform garbage collection iteration. - # TYPE cortex_compactor_garbage_collection_duration_seconds histogram - cortex_compactor_garbage_collection_duration_seconds_bucket{le="+Inf"} 0 - cortex_compactor_garbage_collection_duration_seconds_sum 0 - cortex_compactor_garbage_collection_duration_seconds_count 0 - - # HELP cortex_compactor_garbage_collection_failures_total Total number of failed garbage collection operations. - # TYPE cortex_compactor_garbage_collection_failures_total counter - cortex_compactor_garbage_collection_failures_total 0 - - # HELP cortex_compactor_garbage_collection_total Total number of garbage collection operations. - # TYPE cortex_compactor_garbage_collection_total counter - cortex_compactor_garbage_collection_total 0 - - # HELP cortex_compactor_meta_sync_duration_seconds Duration of the blocks metadata synchronization in seconds. - # TYPE cortex_compactor_meta_sync_duration_seconds histogram - cortex_compactor_meta_sync_duration_seconds_bucket{le="+Inf"} 0 - cortex_compactor_meta_sync_duration_seconds_sum 0 - cortex_compactor_meta_sync_duration_seconds_count 0 - - # HELP cortex_compactor_meta_sync_failures_total Total blocks metadata synchronization failures. - # TYPE cortex_compactor_meta_sync_failures_total counter - cortex_compactor_meta_sync_failures_total 0 - - # HELP cortex_compactor_meta_syncs_total Total blocks metadata synchronization attempts. - # TYPE cortex_compactor_meta_syncs_total counter - cortex_compactor_meta_syncs_total 0 - - # HELP cortex_compactor_group_compaction_runs_completed_total Total number of group completed compaction runs. This also includes compactor group runs that resulted with no compaction. - # TYPE cortex_compactor_group_compaction_runs_completed_total counter - cortex_compactor_group_compaction_runs_completed_total 0 - - # HELP cortex_compactor_group_compaction_runs_started_total Total number of group compaction attempts. - # TYPE cortex_compactor_group_compaction_runs_started_total counter - cortex_compactor_group_compaction_runs_started_total 0 - - # HELP cortex_compactor_group_compactions_failures_total Total number of failed group compactions. - # TYPE cortex_compactor_group_compactions_failures_total counter - cortex_compactor_group_compactions_failures_total 0 - - # HELP cortex_compactor_group_compactions_total Total number of group compaction attempts that resulted in new block(s). - # TYPE cortex_compactor_group_compactions_total counter - cortex_compactor_group_compactions_total 0 - - # TYPE cortex_compactor_block_cleanup_failures_total counter - # HELP cortex_compactor_block_cleanup_failures_total Total number of blocks failed to be deleted. - cortex_compactor_block_cleanup_failures_total 0 - - # HELP cortex_compactor_blocks_cleaned_total Total number of blocks deleted. - # TYPE cortex_compactor_blocks_cleaned_total counter - cortex_compactor_blocks_cleaned_total 0 - - # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. - # TYPE cortex_compactor_blocks_marked_for_deletion_total counter - cortex_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 - cortex_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 - cortex_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 - - # TYPE cortex_compactor_block_cleanup_started_total counter - # HELP cortex_compactor_block_cleanup_started_total Total number of blocks cleanup runs started. - cortex_compactor_block_cleanup_started_total 1 - - # TYPE cortex_compactor_block_cleanup_completed_total counter - # HELP cortex_compactor_block_cleanup_completed_total Total number of blocks cleanup runs successfully completed. - cortex_compactor_block_cleanup_completed_total 1 - - # TYPE cortex_compactor_block_cleanup_failed_total counter - # HELP cortex_compactor_block_cleanup_failed_total Total number of blocks cleanup runs failed. - cortex_compactor_block_cleanup_failed_total 0 + # TYPE pyroscope_compactor_runs_started_total counter + # HELP pyroscope_compactor_runs_started_total Total number of compaction runs started. + pyroscope_compactor_runs_started_total 1 + + # TYPE pyroscope_compactor_runs_completed_total counter + # HELP pyroscope_compactor_runs_completed_total Total number of compaction runs successfully completed. + pyroscope_compactor_runs_completed_total 1 + + # TYPE pyroscope_compactor_runs_failed_total counter + # HELP pyroscope_compactor_runs_failed_total Total number of compaction runs failed. + pyroscope_compactor_runs_failed_total{reason="error"} 0 + pyroscope_compactor_runs_failed_total{reason="shutdown"} 0 + + # HELP pyroscope_compactor_garbage_collection_duration_seconds Time it took to perform garbage collection iteration. + # TYPE pyroscope_compactor_garbage_collection_duration_seconds histogram + pyroscope_compactor_garbage_collection_duration_seconds_bucket{le="+Inf"} 0 + pyroscope_compactor_garbage_collection_duration_seconds_sum 0 + pyroscope_compactor_garbage_collection_duration_seconds_count 0 + + # HELP pyroscope_compactor_garbage_collection_failures_total Total number of failed garbage collection operations. + # TYPE pyroscope_compactor_garbage_collection_failures_total counter + pyroscope_compactor_garbage_collection_failures_total 0 + + # HELP pyroscope_compactor_garbage_collection_total Total number of garbage collection operations. + # TYPE pyroscope_compactor_garbage_collection_total counter + pyroscope_compactor_garbage_collection_total 0 + + # HELP pyroscope_compactor_meta_sync_duration_seconds Duration of the blocks metadata synchronization in seconds. + # TYPE pyroscope_compactor_meta_sync_duration_seconds histogram + pyroscope_compactor_meta_sync_duration_seconds_bucket{le="+Inf"} 0 + pyroscope_compactor_meta_sync_duration_seconds_sum 0 + pyroscope_compactor_meta_sync_duration_seconds_count 0 + + # HELP pyroscope_compactor_meta_sync_failures_total Total blocks metadata synchronization failures. + # TYPE pyroscope_compactor_meta_sync_failures_total counter + pyroscope_compactor_meta_sync_failures_total 0 + + # HELP pyroscope_compactor_meta_syncs_total Total blocks metadata synchronization attempts. + # TYPE pyroscope_compactor_meta_syncs_total counter + pyroscope_compactor_meta_syncs_total 0 + + # HELP pyroscope_compactor_group_compaction_runs_completed_total Total number of group completed compaction runs. This also includes compactor group runs that resulted with no compaction. + # TYPE pyroscope_compactor_group_compaction_runs_completed_total counter + pyroscope_compactor_group_compaction_runs_completed_total 0 + + # HELP pyroscope_compactor_group_compaction_runs_started_total Total number of group compaction attempts. + # TYPE pyroscope_compactor_group_compaction_runs_started_total counter + pyroscope_compactor_group_compaction_runs_started_total 0 + + # HELP pyroscope_compactor_group_compactions_failures_total Total number of failed group compactions. + # TYPE pyroscope_compactor_group_compactions_failures_total counter + pyroscope_compactor_group_compactions_failures_total 0 + + # HELP pyroscope_compactor_group_compactions_total Total number of group compaction attempts that resulted in new block(s). + # TYPE pyroscope_compactor_group_compactions_total counter + pyroscope_compactor_group_compactions_total 0 + + # TYPE pyroscope_compactor_block_cleanup_failures_total counter + # HELP pyroscope_compactor_block_cleanup_failures_total Total number of blocks failed to be deleted. + pyroscope_compactor_block_cleanup_failures_total 0 + + # HELP pyroscope_compactor_blocks_cleaned_total Total number of blocks deleted. + # TYPE pyroscope_compactor_blocks_cleaned_total counter + pyroscope_compactor_blocks_cleaned_total 0 + + # HELP pyroscope_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. + # TYPE pyroscope_compactor_blocks_marked_for_deletion_total counter + pyroscope_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 + pyroscope_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 + pyroscope_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 + + # TYPE pyroscope_compactor_block_cleanup_started_total counter + # HELP pyroscope_compactor_block_cleanup_started_total Total number of blocks cleanup runs started. + pyroscope_compactor_block_cleanup_started_total 1 + + # TYPE pyroscope_compactor_block_cleanup_completed_total counter + # HELP pyroscope_compactor_block_cleanup_completed_total Total number of blocks cleanup runs successfully completed. + pyroscope_compactor_block_cleanup_completed_total 1 + + # TYPE pyroscope_compactor_block_cleanup_failed_total counter + # HELP pyroscope_compactor_block_cleanup_failed_total Total number of blocks cleanup runs failed. + pyroscope_compactor_block_cleanup_failed_total 0 `), - "cortex_compactor_runs_started_total", - "cortex_compactor_runs_completed_total", - "cortex_compactor_runs_failed_total", - "cortex_compactor_garbage_collection_duration_seconds", - "cortex_compactor_garbage_collection_failures_total", - "cortex_compactor_garbage_collection_total", - "cortex_compactor_meta_sync_duration_seconds", - "cortex_compactor_meta_sync_failures_total", - "cortex_compactor_meta_syncs_total", - "cortex_compactor_group_compaction_runs_completed_total", - "cortex_compactor_group_compaction_runs_started_total", - "cortex_compactor_group_compactions_failures_total", - "cortex_compactor_group_compactions_total", - "cortex_compactor_block_cleanup_failures_total", - "cortex_compactor_blocks_cleaned_total", - "cortex_compactor_blocks_marked_for_deletion_total", - "cortex_compactor_block_cleanup_started_total", - "cortex_compactor_block_cleanup_completed_total", - "cortex_compactor_block_cleanup_failed_total", + "pyroscope_compactor_runs_started_total", + "pyroscope_compactor_runs_completed_total", + "pyroscope_compactor_runs_failed_total", + "pyroscope_compactor_garbage_collection_duration_seconds", + "pyroscope_compactor_garbage_collection_failures_total", + "pyroscope_compactor_garbage_collection_total", + "pyroscope_compactor_meta_sync_duration_seconds", + "pyroscope_compactor_meta_sync_failures_total", + "pyroscope_compactor_meta_syncs_total", + "pyroscope_compactor_group_compaction_runs_completed_total", + "pyroscope_compactor_group_compaction_runs_started_total", + "pyroscope_compactor_group_compactions_failures_total", + "pyroscope_compactor_group_compactions_total", + "pyroscope_compactor_block_cleanup_failures_total", + "pyroscope_compactor_blocks_cleaned_total", + "pyroscope_compactor_blocks_marked_for_deletion_total", + "pyroscope_compactor_block_cleanup_started_total", + "pyroscope_compactor_block_cleanup_completed_total", + "pyroscope_compactor_block_cleanup_failed_total", )) } @@ -314,108 +314,108 @@ func TestMultitenantCompactor_ShouldRetryCompactionOnFailureWhileDiscoveringUser }, removeIgnoredLogs(strings.Split(strings.TrimSpace(logs.String()), "\n"))) assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` - # TYPE cortex_compactor_runs_started_total counter - # HELP cortex_compactor_runs_started_total Total number of compaction runs started. - cortex_compactor_runs_started_total 1 - - # TYPE cortex_compactor_runs_completed_total counter - # HELP cortex_compactor_runs_completed_total Total number of compaction runs successfully completed. - cortex_compactor_runs_completed_total 0 - - # TYPE cortex_compactor_runs_failed_total counter - # HELP cortex_compactor_runs_failed_total Total number of compaction runs failed. - cortex_compactor_runs_failed_total{reason="error"} 1 - cortex_compactor_runs_failed_total{reason="shutdown"} 0 - - # HELP cortex_compactor_garbage_collection_duration_seconds Time it took to perform garbage collection iteration. - # TYPE cortex_compactor_garbage_collection_duration_seconds histogram - cortex_compactor_garbage_collection_duration_seconds_bucket{le="+Inf"} 0 - cortex_compactor_garbage_collection_duration_seconds_sum 0 - cortex_compactor_garbage_collection_duration_seconds_count 0 - - # HELP cortex_compactor_garbage_collection_failures_total Total number of failed garbage collection operations. - # TYPE cortex_compactor_garbage_collection_failures_total counter - cortex_compactor_garbage_collection_failures_total 0 - - # HELP cortex_compactor_garbage_collection_total Total number of garbage collection operations. - # TYPE cortex_compactor_garbage_collection_total counter - cortex_compactor_garbage_collection_total 0 - - # HELP cortex_compactor_meta_sync_duration_seconds Duration of the blocks metadata synchronization in seconds. - # TYPE cortex_compactor_meta_sync_duration_seconds histogram - cortex_compactor_meta_sync_duration_seconds_bucket{le="+Inf"} 0 - cortex_compactor_meta_sync_duration_seconds_sum 0 - cortex_compactor_meta_sync_duration_seconds_count 0 - - # HELP cortex_compactor_meta_sync_failures_total Total blocks metadata synchronization failures. - # TYPE cortex_compactor_meta_sync_failures_total counter - cortex_compactor_meta_sync_failures_total 0 - - # HELP cortex_compactor_meta_syncs_total Total blocks metadata synchronization attempts. - # TYPE cortex_compactor_meta_syncs_total counter - cortex_compactor_meta_syncs_total 0 - - # HELP cortex_compactor_group_compaction_runs_completed_total Total number of group completed compaction runs. This also includes compactor group runs that resulted with no compaction. - # TYPE cortex_compactor_group_compaction_runs_completed_total counter - cortex_compactor_group_compaction_runs_completed_total 0 - - # HELP cortex_compactor_group_compaction_runs_started_total Total number of group compaction attempts. - # TYPE cortex_compactor_group_compaction_runs_started_total counter - cortex_compactor_group_compaction_runs_started_total 0 - - # HELP cortex_compactor_group_compactions_failures_total Total number of failed group compactions. - # TYPE cortex_compactor_group_compactions_failures_total counter - cortex_compactor_group_compactions_failures_total 0 - - # HELP cortex_compactor_group_compactions_total Total number of group compaction attempts that resulted in new block(s). - # TYPE cortex_compactor_group_compactions_total counter - cortex_compactor_group_compactions_total 0 - - # TYPE cortex_compactor_block_cleanup_failures_total counter - # HELP cortex_compactor_block_cleanup_failures_total Total number of blocks failed to be deleted. - cortex_compactor_block_cleanup_failures_total 0 - - # HELP cortex_compactor_blocks_cleaned_total Total number of blocks deleted. - # TYPE cortex_compactor_blocks_cleaned_total counter - cortex_compactor_blocks_cleaned_total 0 - - # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. - # TYPE cortex_compactor_blocks_marked_for_deletion_total counter - cortex_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 - cortex_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 - cortex_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 - - # TYPE cortex_compactor_block_cleanup_started_total counter - # HELP cortex_compactor_block_cleanup_started_total Total number of blocks cleanup runs started. - cortex_compactor_block_cleanup_started_total 1 - - # TYPE cortex_compactor_block_cleanup_completed_total counter - # HELP cortex_compactor_block_cleanup_completed_total Total number of blocks cleanup runs successfully completed. - cortex_compactor_block_cleanup_completed_total 0 - - # TYPE cortex_compactor_block_cleanup_failed_total counter - # HELP cortex_compactor_block_cleanup_failed_total Total number of blocks cleanup runs failed. - cortex_compactor_block_cleanup_failed_total 1 + # TYPE pyroscope_compactor_runs_started_total counter + # HELP pyroscope_compactor_runs_started_total Total number of compaction runs started. + pyroscope_compactor_runs_started_total 1 + + # TYPE pyroscope_compactor_runs_completed_total counter + # HELP pyroscope_compactor_runs_completed_total Total number of compaction runs successfully completed. + pyroscope_compactor_runs_completed_total 0 + + # TYPE pyroscope_compactor_runs_failed_total counter + # HELP pyroscope_compactor_runs_failed_total Total number of compaction runs failed. + pyroscope_compactor_runs_failed_total{reason="error"} 1 + pyroscope_compactor_runs_failed_total{reason="shutdown"} 0 + + # HELP pyroscope_compactor_garbage_collection_duration_seconds Time it took to perform garbage collection iteration. + # TYPE pyroscope_compactor_garbage_collection_duration_seconds histogram + pyroscope_compactor_garbage_collection_duration_seconds_bucket{le="+Inf"} 0 + pyroscope_compactor_garbage_collection_duration_seconds_sum 0 + pyroscope_compactor_garbage_collection_duration_seconds_count 0 + + # HELP pyroscope_compactor_garbage_collection_failures_total Total number of failed garbage collection operations. + # TYPE pyroscope_compactor_garbage_collection_failures_total counter + pyroscope_compactor_garbage_collection_failures_total 0 + + # HELP pyroscope_compactor_garbage_collection_total Total number of garbage collection operations. + # TYPE pyroscope_compactor_garbage_collection_total counter + pyroscope_compactor_garbage_collection_total 0 + + # HELP pyroscope_compactor_meta_sync_duration_seconds Duration of the blocks metadata synchronization in seconds. + # TYPE pyroscope_compactor_meta_sync_duration_seconds histogram + pyroscope_compactor_meta_sync_duration_seconds_bucket{le="+Inf"} 0 + pyroscope_compactor_meta_sync_duration_seconds_sum 0 + pyroscope_compactor_meta_sync_duration_seconds_count 0 + + # HELP pyroscope_compactor_meta_sync_failures_total Total blocks metadata synchronization failures. + # TYPE pyroscope_compactor_meta_sync_failures_total counter + pyroscope_compactor_meta_sync_failures_total 0 + + # HELP pyroscope_compactor_meta_syncs_total Total blocks metadata synchronization attempts. + # TYPE pyroscope_compactor_meta_syncs_total counter + pyroscope_compactor_meta_syncs_total 0 + + # HELP pyroscope_compactor_group_compaction_runs_completed_total Total number of group completed compaction runs. This also includes compactor group runs that resulted with no compaction. + # TYPE pyroscope_compactor_group_compaction_runs_completed_total counter + pyroscope_compactor_group_compaction_runs_completed_total 0 + + # HELP pyroscope_compactor_group_compaction_runs_started_total Total number of group compaction attempts. + # TYPE pyroscope_compactor_group_compaction_runs_started_total counter + pyroscope_compactor_group_compaction_runs_started_total 0 + + # HELP pyroscope_compactor_group_compactions_failures_total Total number of failed group compactions. + # TYPE pyroscope_compactor_group_compactions_failures_total counter + pyroscope_compactor_group_compactions_failures_total 0 + + # HELP pyroscope_compactor_group_compactions_total Total number of group compaction attempts that resulted in new block(s). + # TYPE pyroscope_compactor_group_compactions_total counter + pyroscope_compactor_group_compactions_total 0 + + # TYPE pyroscope_compactor_block_cleanup_failures_total counter + # HELP pyroscope_compactor_block_cleanup_failures_total Total number of blocks failed to be deleted. + pyroscope_compactor_block_cleanup_failures_total 0 + + # HELP pyroscope_compactor_blocks_cleaned_total Total number of blocks deleted. + # TYPE pyroscope_compactor_blocks_cleaned_total counter + pyroscope_compactor_blocks_cleaned_total 0 + + # HELP pyroscope_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. + # TYPE pyroscope_compactor_blocks_marked_for_deletion_total counter + pyroscope_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 + pyroscope_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 + pyroscope_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 + + # TYPE pyroscope_compactor_block_cleanup_started_total counter + # HELP pyroscope_compactor_block_cleanup_started_total Total number of blocks cleanup runs started. + pyroscope_compactor_block_cleanup_started_total 1 + + # TYPE pyroscope_compactor_block_cleanup_completed_total counter + # HELP pyroscope_compactor_block_cleanup_completed_total Total number of blocks cleanup runs successfully completed. + pyroscope_compactor_block_cleanup_completed_total 0 + + # TYPE pyroscope_compactor_block_cleanup_failed_total counter + # HELP pyroscope_compactor_block_cleanup_failed_total Total number of blocks cleanup runs failed. + pyroscope_compactor_block_cleanup_failed_total 1 `), - "cortex_compactor_runs_started_total", - "cortex_compactor_runs_completed_total", - "cortex_compactor_runs_failed_total", - "cortex_compactor_garbage_collection_duration_seconds", - "cortex_compactor_garbage_collection_failures_total", - "cortex_compactor_garbage_collection_total", - "cortex_compactor_meta_sync_duration_seconds", - "cortex_compactor_meta_sync_failures_total", - "cortex_compactor_meta_syncs_total", - "cortex_compactor_group_compaction_runs_completed_total", - "cortex_compactor_group_compaction_runs_started_total", - "cortex_compactor_group_compactions_failures_total", - "cortex_compactor_group_compactions_total", - "cortex_compactor_block_cleanup_failures_total", - "cortex_compactor_blocks_cleaned_total", - "cortex_compactor_blocks_marked_for_deletion_total", - "cortex_compactor_block_cleanup_started_total", - "cortex_compactor_block_cleanup_completed_total", - "cortex_compactor_block_cleanup_failed_total", + "pyroscope_compactor_runs_started_total", + "pyroscope_compactor_runs_completed_total", + "pyroscope_compactor_runs_failed_total", + "pyroscope_compactor_garbage_collection_duration_seconds", + "pyroscope_compactor_garbage_collection_failures_total", + "pyroscope_compactor_garbage_collection_total", + "pyroscope_compactor_meta_sync_duration_seconds", + "pyroscope_compactor_meta_sync_failures_total", + "pyroscope_compactor_meta_syncs_total", + "pyroscope_compactor_group_compaction_runs_completed_total", + "pyroscope_compactor_group_compaction_runs_started_total", + "pyroscope_compactor_group_compactions_failures_total", + "pyroscope_compactor_group_compactions_total", + "pyroscope_compactor_block_cleanup_failures_total", + "pyroscope_compactor_blocks_cleaned_total", + "pyroscope_compactor_blocks_marked_for_deletion_total", + "pyroscope_compactor_block_cleanup_started_total", + "pyroscope_compactor_block_cleanup_completed_total", + "pyroscope_compactor_block_cleanup_failed_total", )) } @@ -449,22 +449,22 @@ func TestMultitenantCompactor_ShouldIncrementCompactionErrorIfFailedToCompactASi require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` - # TYPE cortex_compactor_runs_started_total counter - # HELP cortex_compactor_runs_started_total Total number of compaction runs started. - cortex_compactor_runs_started_total 1 - - # TYPE cortex_compactor_runs_completed_total counter - # HELP cortex_compactor_runs_completed_total Total number of compaction runs successfully completed. - cortex_compactor_runs_completed_total 0 - - # TYPE cortex_compactor_runs_failed_total counter - # HELP cortex_compactor_runs_failed_total Total number of compaction runs failed. - cortex_compactor_runs_failed_total{reason="error"} 1 - cortex_compactor_runs_failed_total{reason="shutdown"} 0 + # TYPE pyroscope_compactor_runs_started_total counter + # HELP pyroscope_compactor_runs_started_total Total number of compaction runs started. + pyroscope_compactor_runs_started_total 1 + + # TYPE pyroscope_compactor_runs_completed_total counter + # HELP pyroscope_compactor_runs_completed_total Total number of compaction runs successfully completed. + pyroscope_compactor_runs_completed_total 0 + + # TYPE pyroscope_compactor_runs_failed_total counter + # HELP pyroscope_compactor_runs_failed_total Total number of compaction runs failed. + pyroscope_compactor_runs_failed_total{reason="error"} 1 + pyroscope_compactor_runs_failed_total{reason="shutdown"} 0 `), - "cortex_compactor_runs_started_total", - "cortex_compactor_runs_completed_total", - "cortex_compactor_runs_failed_total", + "pyroscope_compactor_runs_started_total", + "pyroscope_compactor_runs_completed_total", + "pyroscope_compactor_runs_failed_total", )) } @@ -502,22 +502,22 @@ func TestMultitenantCompactor_ShouldIncrementCompactionShutdownIfTheContextIsCan require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` - # TYPE cortex_compactor_runs_started_total counter - # HELP cortex_compactor_runs_started_total Total number of compaction runs started. - cortex_compactor_runs_started_total 1 - - # TYPE cortex_compactor_runs_completed_total counter - # HELP cortex_compactor_runs_completed_total Total number of compaction runs successfully completed. - cortex_compactor_runs_completed_total 0 - - # TYPE cortex_compactor_runs_failed_total counter - # HELP cortex_compactor_runs_failed_total Total number of compaction runs failed. - cortex_compactor_runs_failed_total{reason="error"} 0 - cortex_compactor_runs_failed_total{reason="shutdown"} 1 + # TYPE pyroscope_compactor_runs_started_total counter + # HELP pyroscope_compactor_runs_started_total Total number of compaction runs started. + pyroscope_compactor_runs_started_total 1 + + # TYPE pyroscope_compactor_runs_completed_total counter + # HELP pyroscope_compactor_runs_completed_total Total number of compaction runs successfully completed. + pyroscope_compactor_runs_completed_total 0 + + # TYPE pyroscope_compactor_runs_failed_total counter + # HELP pyroscope_compactor_runs_failed_total Total number of compaction runs failed. + pyroscope_compactor_runs_failed_total{reason="error"} 0 + pyroscope_compactor_runs_failed_total{reason="shutdown"} 1 `), - "cortex_compactor_runs_started_total", - "cortex_compactor_runs_completed_total", - "cortex_compactor_runs_failed_total", + "pyroscope_compactor_runs_started_total", + "pyroscope_compactor_runs_completed_total", + "pyroscope_compactor_runs_failed_total", )) } @@ -600,68 +600,68 @@ func TestMultitenantCompactor_ShouldIterateOverUsersAndRunCompaction(t *testing. // Instead of testing for shipper metrics, we only check our metrics here. // Real shipper metrics are too variable to embed into a test. testedMetrics := []string{ - "cortex_compactor_runs_started_total", "cortex_compactor_runs_completed_total", "cortex_compactor_runs_failed_total", - "cortex_compactor_blocks_cleaned_total", "cortex_compactor_block_cleanup_failures_total", "cortex_compactor_blocks_marked_for_deletion_total", - "cortex_compactor_block_cleanup_started_total", "cortex_compactor_block_cleanup_completed_total", "cortex_compactor_block_cleanup_failed_total", - "cortex_compactor_group_compaction_runs_completed_total", "cortex_compactor_group_compaction_runs_started_total", - "cortex_compactor_group_compactions_failures_total", "cortex_compactor_group_compactions_total", + "pyroscope_compactor_runs_started_total", "pyroscope_compactor_runs_completed_total", "pyroscope_compactor_runs_failed_total", + "pyroscope_compactor_blocks_cleaned_total", "pyroscope_compactor_block_cleanup_failures_total", "pyroscope_compactor_blocks_marked_for_deletion_total", + "pyroscope_compactor_block_cleanup_started_total", "pyroscope_compactor_block_cleanup_completed_total", "pyroscope_compactor_block_cleanup_failed_total", + "pyroscope_compactor_group_compaction_runs_completed_total", "pyroscope_compactor_group_compaction_runs_started_total", + "pyroscope_compactor_group_compactions_failures_total", "pyroscope_compactor_group_compactions_total", } assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` - # TYPE cortex_compactor_runs_started_total counter - # HELP cortex_compactor_runs_started_total Total number of compaction runs started. - cortex_compactor_runs_started_total 1 - - # TYPE cortex_compactor_runs_completed_total counter - # HELP cortex_compactor_runs_completed_total Total number of compaction runs successfully completed. - cortex_compactor_runs_completed_total 1 - - # TYPE cortex_compactor_runs_failed_total counter - # HELP cortex_compactor_runs_failed_total Total number of compaction runs failed. - cortex_compactor_runs_failed_total{reason="error"} 0 - cortex_compactor_runs_failed_total{reason="shutdown"} 0 - - # HELP cortex_compactor_group_compaction_runs_completed_total Total number of group completed compaction runs. This also includes compactor group runs that resulted with no compaction. - # TYPE cortex_compactor_group_compaction_runs_completed_total counter - cortex_compactor_group_compaction_runs_completed_total 2 - - # HELP cortex_compactor_group_compaction_runs_started_total Total number of group compaction attempts. - # TYPE cortex_compactor_group_compaction_runs_started_total counter - cortex_compactor_group_compaction_runs_started_total 2 - - # HELP cortex_compactor_group_compactions_failures_total Total number of failed group compactions. - # TYPE cortex_compactor_group_compactions_failures_total counter - cortex_compactor_group_compactions_failures_total 0 - - # HELP cortex_compactor_group_compactions_total Total number of group compaction attempts that resulted in new block(s). - # TYPE cortex_compactor_group_compactions_total counter - cortex_compactor_group_compactions_total 0 - - # TYPE cortex_compactor_block_cleanup_failures_total counter - # HELP cortex_compactor_block_cleanup_failures_total Total number of blocks failed to be deleted. - cortex_compactor_block_cleanup_failures_total 0 - - # HELP cortex_compactor_blocks_cleaned_total Total number of blocks deleted. - # TYPE cortex_compactor_blocks_cleaned_total counter - cortex_compactor_blocks_cleaned_total 0 - - # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. - # TYPE cortex_compactor_blocks_marked_for_deletion_total counter - cortex_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 - cortex_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 - cortex_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 - - # TYPE cortex_compactor_block_cleanup_started_total counter - # HELP cortex_compactor_block_cleanup_started_total Total number of blocks cleanup runs started. - cortex_compactor_block_cleanup_started_total 1 - - # TYPE cortex_compactor_block_cleanup_completed_total counter - # HELP cortex_compactor_block_cleanup_completed_total Total number of blocks cleanup runs successfully completed. - cortex_compactor_block_cleanup_completed_total 1 - - # TYPE cortex_compactor_block_cleanup_failed_total counter - # HELP cortex_compactor_block_cleanup_failed_total Total number of blocks cleanup runs failed. - cortex_compactor_block_cleanup_failed_total 0 + # TYPE pyroscope_compactor_runs_started_total counter + # HELP pyroscope_compactor_runs_started_total Total number of compaction runs started. + pyroscope_compactor_runs_started_total 1 + + # TYPE pyroscope_compactor_runs_completed_total counter + # HELP pyroscope_compactor_runs_completed_total Total number of compaction runs successfully completed. + pyroscope_compactor_runs_completed_total 1 + + # TYPE pyroscope_compactor_runs_failed_total counter + # HELP pyroscope_compactor_runs_failed_total Total number of compaction runs failed. + pyroscope_compactor_runs_failed_total{reason="error"} 0 + pyroscope_compactor_runs_failed_total{reason="shutdown"} 0 + + # HELP pyroscope_compactor_group_compaction_runs_completed_total Total number of group completed compaction runs. This also includes compactor group runs that resulted with no compaction. + # TYPE pyroscope_compactor_group_compaction_runs_completed_total counter + pyroscope_compactor_group_compaction_runs_completed_total 2 + + # HELP pyroscope_compactor_group_compaction_runs_started_total Total number of group compaction attempts. + # TYPE pyroscope_compactor_group_compaction_runs_started_total counter + pyroscope_compactor_group_compaction_runs_started_total 2 + + # HELP pyroscope_compactor_group_compactions_failures_total Total number of failed group compactions. + # TYPE pyroscope_compactor_group_compactions_failures_total counter + pyroscope_compactor_group_compactions_failures_total 0 + + # HELP pyroscope_compactor_group_compactions_total Total number of group compaction attempts that resulted in new block(s). + # TYPE pyroscope_compactor_group_compactions_total counter + pyroscope_compactor_group_compactions_total 0 + + # TYPE pyroscope_compactor_block_cleanup_failures_total counter + # HELP pyroscope_compactor_block_cleanup_failures_total Total number of blocks failed to be deleted. + pyroscope_compactor_block_cleanup_failures_total 0 + + # HELP pyroscope_compactor_blocks_cleaned_total Total number of blocks deleted. + # TYPE pyroscope_compactor_blocks_cleaned_total counter + pyroscope_compactor_blocks_cleaned_total 0 + + # HELP pyroscope_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. + # TYPE pyroscope_compactor_blocks_marked_for_deletion_total counter + pyroscope_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 + pyroscope_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 + pyroscope_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 + + # TYPE pyroscope_compactor_block_cleanup_started_total counter + # HELP pyroscope_compactor_block_cleanup_started_total Total number of blocks cleanup runs started. + pyroscope_compactor_block_cleanup_started_total 1 + + # TYPE pyroscope_compactor_block_cleanup_completed_total counter + # HELP pyroscope_compactor_block_cleanup_completed_total Total number of blocks cleanup runs successfully completed. + pyroscope_compactor_block_cleanup_completed_total 1 + + # TYPE pyroscope_compactor_block_cleanup_failed_total counter + # HELP pyroscope_compactor_block_cleanup_failed_total Total number of blocks cleanup runs failed. + pyroscope_compactor_block_cleanup_failed_total 0 `), testedMetrics...)) } @@ -805,49 +805,49 @@ func TestMultitenantCompactor_ShouldNotCompactBlocksMarkedForDeletion(t *testing // Instead of testing for shipper metrics, we only check our metrics here. // Real shipper metrics are too variable to embed into a test. testedMetrics := []string{ - "cortex_compactor_runs_started_total", "cortex_compactor_runs_completed_total", "cortex_compactor_runs_failed_total", - "cortex_compactor_blocks_cleaned_total", "cortex_compactor_block_cleanup_failures_total", "cortex_compactor_blocks_marked_for_deletion_total", - "cortex_compactor_block_cleanup_started_total", "cortex_compactor_block_cleanup_completed_total", "cortex_compactor_block_cleanup_failed_total", + "pyroscope_compactor_runs_started_total", "pyroscope_compactor_runs_completed_total", "pyroscope_compactor_runs_failed_total", + "pyroscope_compactor_blocks_cleaned_total", "pyroscope_compactor_block_cleanup_failures_total", "pyroscope_compactor_blocks_marked_for_deletion_total", + "pyroscope_compactor_block_cleanup_started_total", "pyroscope_compactor_block_cleanup_completed_total", "pyroscope_compactor_block_cleanup_failed_total", } assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` - # TYPE cortex_compactor_runs_started_total counter - # HELP cortex_compactor_runs_started_total Total number of compaction runs started. - cortex_compactor_runs_started_total 1 - - # TYPE cortex_compactor_runs_completed_total counter - # HELP cortex_compactor_runs_completed_total Total number of compaction runs successfully completed. - cortex_compactor_runs_completed_total 1 - - # TYPE cortex_compactor_runs_failed_total counter - # HELP cortex_compactor_runs_failed_total Total number of compaction runs failed. - cortex_compactor_runs_failed_total{reason="error"} 0 - cortex_compactor_runs_failed_total{reason="shutdown"} 0 - - # TYPE cortex_compactor_block_cleanup_failures_total counter - # HELP cortex_compactor_block_cleanup_failures_total Total number of blocks failed to be deleted. - cortex_compactor_block_cleanup_failures_total 0 - - # HELP cortex_compactor_blocks_cleaned_total Total number of blocks deleted. - # TYPE cortex_compactor_blocks_cleaned_total counter - cortex_compactor_blocks_cleaned_total 1 - - # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. - # TYPE cortex_compactor_blocks_marked_for_deletion_total counter - cortex_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 - cortex_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 - cortex_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 - - # TYPE cortex_compactor_block_cleanup_started_total counter - # HELP cortex_compactor_block_cleanup_started_total Total number of blocks cleanup runs started. - cortex_compactor_block_cleanup_started_total 1 - - # TYPE cortex_compactor_block_cleanup_completed_total counter - # HELP cortex_compactor_block_cleanup_completed_total Total number of blocks cleanup runs successfully completed. - cortex_compactor_block_cleanup_completed_total 1 - - # TYPE cortex_compactor_block_cleanup_failed_total counter - # HELP cortex_compactor_block_cleanup_failed_total Total number of blocks cleanup runs failed. - cortex_compactor_block_cleanup_failed_total 0 + # TYPE pyroscope_compactor_runs_started_total counter + # HELP pyroscope_compactor_runs_started_total Total number of compaction runs started. + pyroscope_compactor_runs_started_total 1 + + # TYPE pyroscope_compactor_runs_completed_total counter + # HELP pyroscope_compactor_runs_completed_total Total number of compaction runs successfully completed. + pyroscope_compactor_runs_completed_total 1 + + # TYPE pyroscope_compactor_runs_failed_total counter + # HELP pyroscope_compactor_runs_failed_total Total number of compaction runs failed. + pyroscope_compactor_runs_failed_total{reason="error"} 0 + pyroscope_compactor_runs_failed_total{reason="shutdown"} 0 + + # TYPE pyroscope_compactor_block_cleanup_failures_total counter + # HELP pyroscope_compactor_block_cleanup_failures_total Total number of blocks failed to be deleted. + pyroscope_compactor_block_cleanup_failures_total 0 + + # HELP pyroscope_compactor_blocks_cleaned_total Total number of blocks deleted. + # TYPE pyroscope_compactor_blocks_cleaned_total counter + pyroscope_compactor_blocks_cleaned_total 1 + + # HELP pyroscope_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. + # TYPE pyroscope_compactor_blocks_marked_for_deletion_total counter + pyroscope_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 + pyroscope_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 + pyroscope_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 + + # TYPE pyroscope_compactor_block_cleanup_started_total counter + # HELP pyroscope_compactor_block_cleanup_started_total Total number of blocks cleanup runs started. + pyroscope_compactor_block_cleanup_started_total 1 + + # TYPE pyroscope_compactor_block_cleanup_completed_total counter + # HELP pyroscope_compactor_block_cleanup_completed_total Total number of blocks cleanup runs successfully completed. + pyroscope_compactor_block_cleanup_completed_total 1 + + # TYPE pyroscope_compactor_block_cleanup_failed_total counter + # HELP pyroscope_compactor_block_cleanup_failed_total Total number of blocks cleanup runs failed. + pyroscope_compactor_block_cleanup_failed_total 0 `), testedMetrics...)) } @@ -962,50 +962,50 @@ func TestMultitenantCompactor_ShouldNotCompactBlocksForUsersMarkedForDeletion(t // Instead of testing for shipper metrics, we only check our metrics here. // Real shipper metrics are too variable to embed into a test. testedMetrics := []string{ - "cortex_compactor_runs_started_total", "cortex_compactor_runs_completed_total", "cortex_compactor_runs_failed_total", - "cortex_compactor_blocks_cleaned_total", "cortex_compactor_block_cleanup_failures_total", "cortex_compactor_blocks_marked_for_deletion_total", - "cortex_compactor_block_cleanup_started_total", "cortex_compactor_block_cleanup_completed_total", "cortex_compactor_block_cleanup_failed_total", - "cortex_bucket_blocks_count", "cortex_bucket_blocks_marked_for_deletion_count", "cortex_bucket_index_last_successful_update_timestamp_seconds", + "pyroscope_compactor_runs_started_total", "pyroscope_compactor_runs_completed_total", "pyroscope_compactor_runs_failed_total", + "pyroscope_compactor_blocks_cleaned_total", "pyroscope_compactor_block_cleanup_failures_total", "pyroscope_compactor_blocks_marked_for_deletion_total", + "pyroscope_compactor_block_cleanup_started_total", "pyroscope_compactor_block_cleanup_completed_total", "pyroscope_compactor_block_cleanup_failed_total", + "pyroscope_bucket_blocks_count", "pyroscope_bucket_blocks_marked_for_deletion_count", "pyroscope_bucket_index_last_successful_update_timestamp_seconds", } assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` - # TYPE cortex_compactor_runs_started_total counter - # HELP cortex_compactor_runs_started_total Total number of compaction runs started. - cortex_compactor_runs_started_total 1 - - # TYPE cortex_compactor_runs_completed_total counter - # HELP cortex_compactor_runs_completed_total Total number of compaction runs successfully completed. - cortex_compactor_runs_completed_total 1 - - # TYPE cortex_compactor_runs_failed_total counter - # HELP cortex_compactor_runs_failed_total Total number of compaction runs failed. - cortex_compactor_runs_failed_total{reason="error"} 0 - cortex_compactor_runs_failed_total{reason="shutdown"} 0 - - # TYPE cortex_compactor_block_cleanup_failures_total counter - # HELP cortex_compactor_block_cleanup_failures_total Total number of blocks failed to be deleted. - cortex_compactor_block_cleanup_failures_total 0 - - # HELP cortex_compactor_blocks_cleaned_total Total number of blocks deleted. - # TYPE cortex_compactor_blocks_cleaned_total counter - cortex_compactor_blocks_cleaned_total 1 - - # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. - # TYPE cortex_compactor_blocks_marked_for_deletion_total counter - cortex_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 - cortex_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 - cortex_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 - - # TYPE cortex_compactor_block_cleanup_started_total counter - # HELP cortex_compactor_block_cleanup_started_total Total number of blocks cleanup runs started. - cortex_compactor_block_cleanup_started_total 1 - - # TYPE cortex_compactor_block_cleanup_completed_total counter - # HELP cortex_compactor_block_cleanup_completed_total Total number of blocks cleanup runs successfully completed. - cortex_compactor_block_cleanup_completed_total 1 - - # TYPE cortex_compactor_block_cleanup_failed_total counter - # HELP cortex_compactor_block_cleanup_failed_total Total number of blocks cleanup runs failed. - cortex_compactor_block_cleanup_failed_total 0 + # TYPE pyroscope_compactor_runs_started_total counter + # HELP pyroscope_compactor_runs_started_total Total number of compaction runs started. + pyroscope_compactor_runs_started_total 1 + + # TYPE pyroscope_compactor_runs_completed_total counter + # HELP pyroscope_compactor_runs_completed_total Total number of compaction runs successfully completed. + pyroscope_compactor_runs_completed_total 1 + + # TYPE pyroscope_compactor_runs_failed_total counter + # HELP pyroscope_compactor_runs_failed_total Total number of compaction runs failed. + pyroscope_compactor_runs_failed_total{reason="error"} 0 + pyroscope_compactor_runs_failed_total{reason="shutdown"} 0 + + # TYPE pyroscope_compactor_block_cleanup_failures_total counter + # HELP pyroscope_compactor_block_cleanup_failures_total Total number of blocks failed to be deleted. + pyroscope_compactor_block_cleanup_failures_total 0 + + # HELP pyroscope_compactor_blocks_cleaned_total Total number of blocks deleted. + # TYPE pyroscope_compactor_blocks_cleaned_total counter + pyroscope_compactor_blocks_cleaned_total 1 + + # HELP pyroscope_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. + # TYPE pyroscope_compactor_blocks_marked_for_deletion_total counter + pyroscope_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 + pyroscope_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 + pyroscope_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 + + # TYPE pyroscope_compactor_block_cleanup_started_total counter + # HELP pyroscope_compactor_block_cleanup_started_total Total number of blocks cleanup runs started. + pyroscope_compactor_block_cleanup_started_total 1 + + # TYPE pyroscope_compactor_block_cleanup_completed_total counter + # HELP pyroscope_compactor_block_cleanup_completed_total Total number of blocks cleanup runs successfully completed. + pyroscope_compactor_block_cleanup_completed_total 1 + + # TYPE pyroscope_compactor_block_cleanup_failed_total counter + # HELP pyroscope_compactor_block_cleanup_failed_total Total number of blocks cleanup runs failed. + pyroscope_compactor_block_cleanup_failed_total 0 `), testedMetrics...)) } @@ -1092,49 +1092,49 @@ func TestMultitenantCompactor_ShouldCompactAllUsersOnShardingEnabledButOnlyOneIn }, removeIgnoredLogs(strings.Split(strings.TrimSpace(logs.String()), "\n"))) assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` - # TYPE cortex_compactor_runs_started_total counter - # HELP cortex_compactor_runs_started_total Total number of compaction runs started. - cortex_compactor_runs_started_total 1 - - # TYPE cortex_compactor_runs_completed_total counter - # HELP cortex_compactor_runs_completed_total Total number of compaction runs successfully completed. - cortex_compactor_runs_completed_total 1 - - # TYPE cortex_compactor_runs_failed_total counter - # HELP cortex_compactor_runs_failed_total Total number of compaction runs failed. - cortex_compactor_runs_failed_total{reason="error"} 0 - cortex_compactor_runs_failed_total{reason="shutdown"} 0 - - # HELP cortex_compactor_group_compaction_runs_completed_total Total number of group completed compaction runs. This also includes compactor group runs that resulted with no compaction. - # TYPE cortex_compactor_group_compaction_runs_completed_total counter - cortex_compactor_group_compaction_runs_completed_total 2 - - # HELP cortex_compactor_group_compaction_runs_started_total Total number of group compaction attempts. - # TYPE cortex_compactor_group_compaction_runs_started_total counter - cortex_compactor_group_compaction_runs_started_total 2 - - # HELP cortex_compactor_group_compactions_failures_total Total number of failed group compactions. - # TYPE cortex_compactor_group_compactions_failures_total counter - cortex_compactor_group_compactions_failures_total 0 - - # HELP cortex_compactor_group_compactions_total Total number of group compaction attempts that resulted in new block(s). - # TYPE cortex_compactor_group_compactions_total counter - cortex_compactor_group_compactions_total 0 - - # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. - # TYPE cortex_compactor_blocks_marked_for_deletion_total counter - cortex_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 - cortex_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 - cortex_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 + # TYPE pyroscope_compactor_runs_started_total counter + # HELP pyroscope_compactor_runs_started_total Total number of compaction runs started. + pyroscope_compactor_runs_started_total 1 + + # TYPE pyroscope_compactor_runs_completed_total counter + # HELP pyroscope_compactor_runs_completed_total Total number of compaction runs successfully completed. + pyroscope_compactor_runs_completed_total 1 + + # TYPE pyroscope_compactor_runs_failed_total counter + # HELP pyroscope_compactor_runs_failed_total Total number of compaction runs failed. + pyroscope_compactor_runs_failed_total{reason="error"} 0 + pyroscope_compactor_runs_failed_total{reason="shutdown"} 0 + + # HELP pyroscope_compactor_group_compaction_runs_completed_total Total number of group completed compaction runs. This also includes compactor group runs that resulted with no compaction. + # TYPE pyroscope_compactor_group_compaction_runs_completed_total counter + pyroscope_compactor_group_compaction_runs_completed_total 2 + + # HELP pyroscope_compactor_group_compaction_runs_started_total Total number of group compaction attempts. + # TYPE pyroscope_compactor_group_compaction_runs_started_total counter + pyroscope_compactor_group_compaction_runs_started_total 2 + + # HELP pyroscope_compactor_group_compactions_failures_total Total number of failed group compactions. + # TYPE pyroscope_compactor_group_compactions_failures_total counter + pyroscope_compactor_group_compactions_failures_total 0 + + # HELP pyroscope_compactor_group_compactions_total Total number of group compaction attempts that resulted in new block(s). + # TYPE pyroscope_compactor_group_compactions_total counter + pyroscope_compactor_group_compactions_total 0 + + # HELP pyroscope_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. + # TYPE pyroscope_compactor_blocks_marked_for_deletion_total counter + pyroscope_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 + pyroscope_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 + pyroscope_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 `), - "cortex_compactor_runs_started_total", - "cortex_compactor_runs_completed_total", - "cortex_compactor_runs_failed_total", - "cortex_compactor_group_compaction_runs_completed_total", - "cortex_compactor_group_compaction_runs_started_total", - "cortex_compactor_group_compactions_failures_total", - "cortex_compactor_group_compactions_total", - "cortex_compactor_blocks_marked_for_deletion_total", + "pyroscope_compactor_runs_started_total", + "pyroscope_compactor_runs_completed_total", + "pyroscope_compactor_runs_failed_total", + "pyroscope_compactor_group_compaction_runs_completed_total", + "pyroscope_compactor_group_compaction_runs_started_total", + "pyroscope_compactor_group_compactions_failures_total", + "pyroscope_compactor_group_compactions_total", + "pyroscope_compactor_blocks_marked_for_deletion_total", )) } @@ -1404,49 +1404,49 @@ func TestMultitenantCompactor_ShouldSkipCompactionForJobsNoMoreOwnedAfterPlannin }, removeIgnoredLogs(strings.Split(strings.TrimSpace(logs.String()), "\n"))) assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` - # TYPE cortex_compactor_runs_started_total counter - # HELP cortex_compactor_runs_started_total Total number of compaction runs started. - cortex_compactor_runs_started_total 1 - - # TYPE cortex_compactor_runs_completed_total counter - # HELP cortex_compactor_runs_completed_total Total number of compaction runs successfully completed. - cortex_compactor_runs_completed_total 1 - - # TYPE cortex_compactor_runs_failed_total counter - # HELP cortex_compactor_runs_failed_total Total number of compaction runs failed. - cortex_compactor_runs_failed_total{reason="error"} 0 - cortex_compactor_runs_failed_total{reason="shutdown"} 0 - - # HELP cortex_compactor_group_compaction_runs_completed_total Total number of group completed compaction runs. This also includes compactor group runs that resulted with no compaction. - # TYPE cortex_compactor_group_compaction_runs_completed_total counter - cortex_compactor_group_compaction_runs_completed_total 1 - - # HELP cortex_compactor_group_compaction_runs_started_total Total number of group compaction attempts. - # TYPE cortex_compactor_group_compaction_runs_started_total counter - cortex_compactor_group_compaction_runs_started_total 1 - - # HELP cortex_compactor_group_compactions_failures_total Total number of failed group compactions. - # TYPE cortex_compactor_group_compactions_failures_total counter - cortex_compactor_group_compactions_failures_total 0 - - # HELP cortex_compactor_group_compactions_total Total number of group compaction attempts that resulted in new block(s). - # TYPE cortex_compactor_group_compactions_total counter - cortex_compactor_group_compactions_total 0 - - # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. - # TYPE cortex_compactor_blocks_marked_for_deletion_total counter - cortex_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 - cortex_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 - cortex_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 + # TYPE pyroscope_compactor_runs_started_total counter + # HELP pyroscope_compactor_runs_started_total Total number of compaction runs started. + pyroscope_compactor_runs_started_total 1 + + # TYPE pyroscope_compactor_runs_completed_total counter + # HELP pyroscope_compactor_runs_completed_total Total number of compaction runs successfully completed. + pyroscope_compactor_runs_completed_total 1 + + # TYPE pyroscope_compactor_runs_failed_total counter + # HELP pyroscope_compactor_runs_failed_total Total number of compaction runs failed. + pyroscope_compactor_runs_failed_total{reason="error"} 0 + pyroscope_compactor_runs_failed_total{reason="shutdown"} 0 + + # HELP pyroscope_compactor_group_compaction_runs_completed_total Total number of group completed compaction runs. This also includes compactor group runs that resulted with no compaction. + # TYPE pyroscope_compactor_group_compaction_runs_completed_total counter + pyroscope_compactor_group_compaction_runs_completed_total 1 + + # HELP pyroscope_compactor_group_compaction_runs_started_total Total number of group compaction attempts. + # TYPE pyroscope_compactor_group_compaction_runs_started_total counter + pyroscope_compactor_group_compaction_runs_started_total 1 + + # HELP pyroscope_compactor_group_compactions_failures_total Total number of failed group compactions. + # TYPE pyroscope_compactor_group_compactions_failures_total counter + pyroscope_compactor_group_compactions_failures_total 0 + + # HELP pyroscope_compactor_group_compactions_total Total number of group compaction attempts that resulted in new block(s). + # TYPE pyroscope_compactor_group_compactions_total counter + pyroscope_compactor_group_compactions_total 0 + + # HELP pyroscope_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. + # TYPE pyroscope_compactor_blocks_marked_for_deletion_total counter + pyroscope_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 + pyroscope_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 + pyroscope_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 `), - "cortex_compactor_runs_started_total", - "cortex_compactor_runs_completed_total", - "cortex_compactor_runs_failed_total", - "cortex_compactor_group_compaction_runs_completed_total", - "cortex_compactor_group_compaction_runs_started_total", - "cortex_compactor_group_compactions_failures_total", - "cortex_compactor_group_compactions_total", - "cortex_compactor_blocks_marked_for_deletion_total", + "pyroscope_compactor_runs_started_total", + "pyroscope_compactor_runs_completed_total", + "pyroscope_compactor_runs_failed_total", + "pyroscope_compactor_group_compaction_runs_completed_total", + "pyroscope_compactor_group_compaction_runs_started_total", + "pyroscope_compactor_group_compactions_failures_total", + "pyroscope_compactor_group_compactions_total", + "pyroscope_compactor_blocks_marked_for_deletion_total", )) } @@ -1514,49 +1514,49 @@ func TestMultitenantCompactor_ShouldSkipCompactionForJobsWithFirstLevelCompactio fmt.Sprintf(`level=info component=compactor user=user-2 msg="skipping compaction job because blocks in this job were uploaded too recently (within wait period)" groupKey=0@17241709254077376921-merge--1574776800000-1574784000000 waitPeriodNotElapsedFor="%s (min time: 1574776800000, max time: 1574784000000)"`, user2Meta2.ULID.String())) assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` - # TYPE cortex_compactor_runs_started_total counter - # HELP cortex_compactor_runs_started_total Total number of compaction runs started. - cortex_compactor_runs_started_total 1 - - # TYPE cortex_compactor_runs_completed_total counter - # HELP cortex_compactor_runs_completed_total Total number of compaction runs successfully completed. - cortex_compactor_runs_completed_total 1 - - # TYPE cortex_compactor_runs_failed_total counter - # HELP cortex_compactor_runs_failed_total Total number of compaction runs failed. - cortex_compactor_runs_failed_total{reason="error"} 0 - cortex_compactor_runs_failed_total{reason="shutdown"} 0 - - # HELP cortex_compactor_group_compaction_runs_completed_total Total number of group completed compaction runs. This also includes compactor group runs that resulted with no compaction. - # TYPE cortex_compactor_group_compaction_runs_completed_total counter - cortex_compactor_group_compaction_runs_completed_total 1 - - # HELP cortex_compactor_group_compaction_runs_started_total Total number of group compaction attempts. - # TYPE cortex_compactor_group_compaction_runs_started_total counter - cortex_compactor_group_compaction_runs_started_total 1 - - # HELP cortex_compactor_group_compactions_failures_total Total number of failed group compactions. - # TYPE cortex_compactor_group_compactions_failures_total counter - cortex_compactor_group_compactions_failures_total 0 - - # HELP cortex_compactor_group_compactions_total Total number of group compaction attempts that resulted in new block(s). - # TYPE cortex_compactor_group_compactions_total counter - cortex_compactor_group_compactions_total 0 - - # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. - # TYPE cortex_compactor_blocks_marked_for_deletion_total counter - cortex_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 - cortex_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 - cortex_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 + # TYPE pyroscope_compactor_runs_started_total counter + # HELP pyroscope_compactor_runs_started_total Total number of compaction runs started. + pyroscope_compactor_runs_started_total 1 + + # TYPE pyroscope_compactor_runs_completed_total counter + # HELP pyroscope_compactor_runs_completed_total Total number of compaction runs successfully completed. + pyroscope_compactor_runs_completed_total 1 + + # TYPE pyroscope_compactor_runs_failed_total counter + # HELP pyroscope_compactor_runs_failed_total Total number of compaction runs failed. + pyroscope_compactor_runs_failed_total{reason="error"} 0 + pyroscope_compactor_runs_failed_total{reason="shutdown"} 0 + + # HELP pyroscope_compactor_group_compaction_runs_completed_total Total number of group completed compaction runs. This also includes compactor group runs that resulted with no compaction. + # TYPE pyroscope_compactor_group_compaction_runs_completed_total counter + pyroscope_compactor_group_compaction_runs_completed_total 1 + + # HELP pyroscope_compactor_group_compaction_runs_started_total Total number of group compaction attempts. + # TYPE pyroscope_compactor_group_compaction_runs_started_total counter + pyroscope_compactor_group_compaction_runs_started_total 1 + + # HELP pyroscope_compactor_group_compactions_failures_total Total number of failed group compactions. + # TYPE pyroscope_compactor_group_compactions_failures_total counter + pyroscope_compactor_group_compactions_failures_total 0 + + # HELP pyroscope_compactor_group_compactions_total Total number of group compaction attempts that resulted in new block(s). + # TYPE pyroscope_compactor_group_compactions_total counter + pyroscope_compactor_group_compactions_total 0 + + # HELP pyroscope_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. + # TYPE pyroscope_compactor_blocks_marked_for_deletion_total counter + pyroscope_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 + pyroscope_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 + pyroscope_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 `), - "cortex_compactor_runs_started_total", - "cortex_compactor_runs_completed_total", - "cortex_compactor_runs_failed_total", - "cortex_compactor_group_compaction_runs_completed_total", - "cortex_compactor_group_compaction_runs_started_total", - "cortex_compactor_group_compactions_failures_total", - "cortex_compactor_group_compactions_total", - "cortex_compactor_blocks_marked_for_deletion_total", + "pyroscope_compactor_runs_started_total", + "pyroscope_compactor_runs_completed_total", + "pyroscope_compactor_runs_failed_total", + "pyroscope_compactor_group_compaction_runs_completed_total", + "pyroscope_compactor_group_compaction_runs_started_total", + "pyroscope_compactor_group_compactions_failures_total", + "pyroscope_compactor_group_compactions_total", + "pyroscope_compactor_blocks_marked_for_deletion_total", )) } @@ -2242,11 +2242,11 @@ func TestMultitenantCompactor_OutOfOrderCompaction(t *testing.T) { require.Equal(t, block.NoCompactReason(block.OutOfOrderChunksNoCompactReason), m.Reason) assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` - # HELP cortex_compactor_blocks_marked_for_no_compaction_total Total number of blocks that were marked for no-compaction. - # TYPE cortex_compactor_blocks_marked_for_no_compaction_total counter - cortex_compactor_blocks_marked_for_no_compaction_total{reason="block-index-out-of-order-chunk"} 1 + # HELP pyroscope_compactor_blocks_marked_for_no_compaction_total Total number of blocks that were marked for no-compaction. + # TYPE pyroscope_compactor_blocks_marked_for_no_compaction_total counter + pyroscope_compactor_blocks_marked_for_no_compaction_total{reason="block-index-out-of-order-chunk"} 1 `), - "cortex_compactor_blocks_marked_for_no_compaction_total", + "pyroscope_compactor_blocks_marked_for_no_compaction_total", )) } diff --git a/pkg/compactor/split_merge_compactor_test.go b/pkg/compactor/split_merge_compactor_test.go index 42a8f1c6ec..caee820b71 100644 --- a/pkg/compactor/split_merge_compactor_test.go +++ b/pkg/compactor/split_merge_compactor_test.go @@ -714,10 +714,10 @@ func TestMultitenantCompactor_ShouldSupportSplitAndMergeCompactor(t *testing.T) // Wait until the first compaction run completed. test.Poll(t, 15*time.Second, nil, func() interface{} { return testutil.GatherAndCompare(reg, strings.NewReader(` - # HELP cortex_compactor_runs_completed_total Total number of compaction runs successfully completed. - # TYPE cortex_compactor_runs_completed_total counter - cortex_compactor_runs_completed_total 1 - `), "cortex_compactor_runs_completed_total") + # HELP pyroscope_compactor_runs_completed_total Total number of compaction runs successfully completed. + # TYPE pyroscope_compactor_runs_completed_total counter + pyroscope_compactor_runs_completed_total 1 + `), "pyroscope_compactor_runs_completed_total") }) // List back any (non deleted) block from the storage. @@ -805,10 +805,10 @@ func TestMultitenantCompactor_ShouldGuaranteeSeriesShardingConsistencyOverTheTim // Wait until the first compaction run completed. test.Poll(t, 15*time.Second, nil, func() interface{} { return testutil.GatherAndCompare(reg, strings.NewReader(` - # HELP cortex_compactor_runs_completed_total Total number of compaction runs successfully completed. - # TYPE cortex_compactor_runs_completed_total counter - cortex_compactor_runs_completed_total 1 - `), "cortex_compactor_runs_completed_total") + # HELP pyroscope_compactor_runs_completed_total Total number of compaction runs successfully completed. + # TYPE pyroscope_compactor_runs_completed_total counter + pyroscope_compactor_runs_completed_total 1 + `), "pyroscope_compactor_runs_completed_total") }) // List back any (non deleted) block from the storage. diff --git a/pkg/compactor/syncer_metrics_test.go b/pkg/compactor/syncer_metrics_test.go index e6be00f9ef..91da99db69 100644 --- a/pkg/compactor/syncer_metrics_test.go +++ b/pkg/compactor/syncer_metrics_test.go @@ -25,68 +25,68 @@ func TestSyncerMetrics(t *testing.T) { // total base = 111110 err := testutil.GatherAndCompare(reg, bytes.NewBufferString(` - # HELP cortex_compactor_meta_syncs_total Total blocks metadata synchronization attempts. - # TYPE cortex_compactor_meta_syncs_total counter - cortex_compactor_meta_syncs_total 111110 + # HELP pyroscope_compactor_meta_syncs_total Total blocks metadata synchronization attempts. + # TYPE pyroscope_compactor_meta_syncs_total counter + pyroscope_compactor_meta_syncs_total 111110 - # HELP cortex_compactor_meta_sync_failures_total Total blocks metadata synchronization failures. - # TYPE cortex_compactor_meta_sync_failures_total counter - cortex_compactor_meta_sync_failures_total 222220 + # HELP pyroscope_compactor_meta_sync_failures_total Total blocks metadata synchronization failures. + # TYPE pyroscope_compactor_meta_sync_failures_total counter + pyroscope_compactor_meta_sync_failures_total 222220 - # HELP cortex_compactor_meta_sync_duration_seconds Duration of the blocks metadata synchronization in seconds. - # TYPE cortex_compactor_meta_sync_duration_seconds histogram + # HELP pyroscope_compactor_meta_sync_duration_seconds Duration of the blocks metadata synchronization in seconds. + # TYPE pyroscope_compactor_meta_sync_duration_seconds histogram # Observed values: 3.7035, 22.9629, 6.6666 (seconds) - cortex_compactor_meta_sync_duration_seconds_bucket{le="0.01"} 0 - cortex_compactor_meta_sync_duration_seconds_bucket{le="0.1"} 0 - cortex_compactor_meta_sync_duration_seconds_bucket{le="0.3"} 0 - cortex_compactor_meta_sync_duration_seconds_bucket{le="0.6"} 0 - cortex_compactor_meta_sync_duration_seconds_bucket{le="1"} 0 - cortex_compactor_meta_sync_duration_seconds_bucket{le="3"} 0 - cortex_compactor_meta_sync_duration_seconds_bucket{le="6"} 1 - cortex_compactor_meta_sync_duration_seconds_bucket{le="9"} 2 - cortex_compactor_meta_sync_duration_seconds_bucket{le="20"} 2 - cortex_compactor_meta_sync_duration_seconds_bucket{le="30"} 3 - cortex_compactor_meta_sync_duration_seconds_bucket{le="60"} 3 - cortex_compactor_meta_sync_duration_seconds_bucket{le="90"} 3 - cortex_compactor_meta_sync_duration_seconds_bucket{le="120"} 3 - cortex_compactor_meta_sync_duration_seconds_bucket{le="240"} 3 - cortex_compactor_meta_sync_duration_seconds_bucket{le="360"} 3 - cortex_compactor_meta_sync_duration_seconds_bucket{le="720"} 3 - cortex_compactor_meta_sync_duration_seconds_bucket{le="+Inf"} 3 + pyroscope_compactor_meta_sync_duration_seconds_bucket{le="0.01"} 0 + pyroscope_compactor_meta_sync_duration_seconds_bucket{le="0.1"} 0 + pyroscope_compactor_meta_sync_duration_seconds_bucket{le="0.3"} 0 + pyroscope_compactor_meta_sync_duration_seconds_bucket{le="0.6"} 0 + pyroscope_compactor_meta_sync_duration_seconds_bucket{le="1"} 0 + pyroscope_compactor_meta_sync_duration_seconds_bucket{le="3"} 0 + pyroscope_compactor_meta_sync_duration_seconds_bucket{le="6"} 1 + pyroscope_compactor_meta_sync_duration_seconds_bucket{le="9"} 2 + pyroscope_compactor_meta_sync_duration_seconds_bucket{le="20"} 2 + pyroscope_compactor_meta_sync_duration_seconds_bucket{le="30"} 3 + pyroscope_compactor_meta_sync_duration_seconds_bucket{le="60"} 3 + pyroscope_compactor_meta_sync_duration_seconds_bucket{le="90"} 3 + pyroscope_compactor_meta_sync_duration_seconds_bucket{le="120"} 3 + pyroscope_compactor_meta_sync_duration_seconds_bucket{le="240"} 3 + pyroscope_compactor_meta_sync_duration_seconds_bucket{le="360"} 3 + pyroscope_compactor_meta_sync_duration_seconds_bucket{le="720"} 3 + pyroscope_compactor_meta_sync_duration_seconds_bucket{le="+Inf"} 3 # rounding error - cortex_compactor_meta_sync_duration_seconds_sum 33.333000000000006 - cortex_compactor_meta_sync_duration_seconds_count 3 + pyroscope_compactor_meta_sync_duration_seconds_sum 33.333000000000006 + pyroscope_compactor_meta_sync_duration_seconds_count 3 - # HELP cortex_compactor_garbage_collection_total Total number of garbage collection operations. - # TYPE cortex_compactor_garbage_collection_total counter - cortex_compactor_garbage_collection_total 555550 + # HELP pyroscope_compactor_garbage_collection_total Total number of garbage collection operations. + # TYPE pyroscope_compactor_garbage_collection_total counter + pyroscope_compactor_garbage_collection_total 555550 - # HELP cortex_compactor_garbage_collection_failures_total Total number of failed garbage collection operations. - # TYPE cortex_compactor_garbage_collection_failures_total counter - cortex_compactor_garbage_collection_failures_total 666660 + # HELP pyroscope_compactor_garbage_collection_failures_total Total number of failed garbage collection operations. + # TYPE pyroscope_compactor_garbage_collection_failures_total counter + pyroscope_compactor_garbage_collection_failures_total 666660 - # HELP cortex_compactor_garbage_collection_duration_seconds Time it took to perform garbage collection iteration. - # TYPE cortex_compactor_garbage_collection_duration_seconds histogram + # HELP pyroscope_compactor_garbage_collection_duration_seconds Time it took to perform garbage collection iteration. + # TYPE pyroscope_compactor_garbage_collection_duration_seconds histogram # Observed values: 8.6415, 53.5801, 15.5554 - cortex_compactor_garbage_collection_duration_seconds_bucket{le="0.01"} 0 - cortex_compactor_garbage_collection_duration_seconds_bucket{le="0.1"} 0 - cortex_compactor_garbage_collection_duration_seconds_bucket{le="0.3"} 0 - cortex_compactor_garbage_collection_duration_seconds_bucket{le="0.6"} 0 - cortex_compactor_garbage_collection_duration_seconds_bucket{le="1"} 0 - cortex_compactor_garbage_collection_duration_seconds_bucket{le="3"} 0 - cortex_compactor_garbage_collection_duration_seconds_bucket{le="6"} 0 - cortex_compactor_garbage_collection_duration_seconds_bucket{le="9"} 1 - cortex_compactor_garbage_collection_duration_seconds_bucket{le="20"} 2 - cortex_compactor_garbage_collection_duration_seconds_bucket{le="30"} 2 - cortex_compactor_garbage_collection_duration_seconds_bucket{le="60"} 3 - cortex_compactor_garbage_collection_duration_seconds_bucket{le="90"} 3 - cortex_compactor_garbage_collection_duration_seconds_bucket{le="120"} 3 - cortex_compactor_garbage_collection_duration_seconds_bucket{le="240"} 3 - cortex_compactor_garbage_collection_duration_seconds_bucket{le="360"} 3 - cortex_compactor_garbage_collection_duration_seconds_bucket{le="720"} 3 - cortex_compactor_garbage_collection_duration_seconds_bucket{le="+Inf"} 3 - cortex_compactor_garbage_collection_duration_seconds_sum 77.777 - cortex_compactor_garbage_collection_duration_seconds_count 3 + pyroscope_compactor_garbage_collection_duration_seconds_bucket{le="0.01"} 0 + pyroscope_compactor_garbage_collection_duration_seconds_bucket{le="0.1"} 0 + pyroscope_compactor_garbage_collection_duration_seconds_bucket{le="0.3"} 0 + pyroscope_compactor_garbage_collection_duration_seconds_bucket{le="0.6"} 0 + pyroscope_compactor_garbage_collection_duration_seconds_bucket{le="1"} 0 + pyroscope_compactor_garbage_collection_duration_seconds_bucket{le="3"} 0 + pyroscope_compactor_garbage_collection_duration_seconds_bucket{le="6"} 0 + pyroscope_compactor_garbage_collection_duration_seconds_bucket{le="9"} 1 + pyroscope_compactor_garbage_collection_duration_seconds_bucket{le="20"} 2 + pyroscope_compactor_garbage_collection_duration_seconds_bucket{le="30"} 2 + pyroscope_compactor_garbage_collection_duration_seconds_bucket{le="60"} 3 + pyroscope_compactor_garbage_collection_duration_seconds_bucket{le="90"} 3 + pyroscope_compactor_garbage_collection_duration_seconds_bucket{le="120"} 3 + pyroscope_compactor_garbage_collection_duration_seconds_bucket{le="240"} 3 + pyroscope_compactor_garbage_collection_duration_seconds_bucket{le="360"} 3 + pyroscope_compactor_garbage_collection_duration_seconds_bucket{le="720"} 3 + pyroscope_compactor_garbage_collection_duration_seconds_bucket{le="+Inf"} 3 + pyroscope_compactor_garbage_collection_duration_seconds_sum 77.777 + pyroscope_compactor_garbage_collection_duration_seconds_count 3 `)) require.NoError(t, err) } From 0fa0966cb1cedcda3ac8d1b4170ab6655bf71c5d Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Wed, 13 Sep 2023 00:44:22 +0200 Subject: [PATCH 28/74] more fixes --- pkg/compactor/bucket_compactor.go | 86 ++++++++++++++------------ pkg/compactor/compactor.go | 49 +++++---------- pkg/compactor/split_merge_compactor.go | 19 ------ pkg/phlaredb/compact.go | 5 +- pkg/tenant/allowed_tenants_test.go | 2 +- 5 files changed, 67 insertions(+), 94 deletions(-) diff --git a/pkg/compactor/bucket_compactor.go b/pkg/compactor/bucket_compactor.go index 5a08c4ffee..67f691aed2 100644 --- a/pkg/compactor/bucket_compactor.go +++ b/pkg/compactor/bucket_compactor.go @@ -24,10 +24,11 @@ import ( "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" "github.com/prometheus/prometheus/model/labels" - "github.com/prometheus/prometheus/tsdb" "go.uber.org/atomic" "github.com/grafana/pyroscope/pkg/objstore" + "github.com/grafana/pyroscope/pkg/objstore/client" + "github.com/grafana/pyroscope/pkg/objstore/providers/filesystem" "github.com/grafana/pyroscope/pkg/phlaredb" "github.com/grafana/pyroscope/pkg/phlaredb/block" "github.com/grafana/pyroscope/pkg/phlaredb/sharding" @@ -216,30 +217,6 @@ type Planner interface { Plan(ctx context.Context, metasByMinTime []*block.Meta) ([]*block.Meta, error) } -// Compactor provides compaction against an underlying storage of time series data. -// This is similar to tsdb.Compactor just without Plan method. -// TODO(bwplotka): Split the Planner from Compactor on upstream as well, so we can import it. -type Compactor interface { - // Write persists a Block into a directory. - // No Block is written when resulting Block has 0 samples, and returns empty ulid.ULID{}. - Write(dest string, b tsdb.BlockReader, mint, maxt int64, parent *tsdb.BlockMeta) (ulid.ULID, error) - - // Compact runs compaction against the provided directories. Must - // only be called concurrently with results of Plan(). - // Can optionally pass a list of already open blocks, - // to avoid having to reopen them. - // When resulting Block has 0 samples - // * No block is written. - // * The source dirs are marked Deletable. - // * Returns empty ulid.ULID{}. - Compact(dest string, dirs []string, open []*tsdb.Block) (ulid.ULID, error) - - // CompactWithSplitting merges and splits the source blocks into shardCount number of compacted blocks, - // and returns slice of block IDs. Position of returned block ID in the result slice corresponds to the shard index. - // If given compacted block has no series, corresponding block ID will be zero ULID value. - CompactWithSplitting(dest string, dirs []string, open []*tsdb.Block, shardCount uint64) (result []ulid.ULID, _ error) -} - // runCompactionJob plans and runs a single compaction against the provided job. The compacted result // is uploaded into the bucket the blocks were retrieved from. func (c *BucketCompactor) runCompactionJob(ctx context.Context, job *Job) (shouldRerun bool, compIDs []ulid.ULID, rerr error) { @@ -296,11 +273,6 @@ func (c *BucketCompactor) runCompactionJob(ctx context.Context, job *Job) (shoul return errors.Wrapf(err, "download block %s", meta.ULID) } - // Ensure all source blocks are valid. - if err := phlaredb.ValidateLocalBlock(ctx, bdir); err != nil { - return errors.Wrapf(err, "invalid block %s", bdir) - } - return nil }) if err != nil { @@ -312,21 +284,57 @@ func (c *BucketCompactor) runCompactionJob(ctx context.Context, job *Job) (shoul blocksToCompactDirs[ix] = filepath.Join(subDir, meta.ULID.String()) } + compactionBegin := time.Now() + + // todo: move this to a separate function. + localBucket, err := client.NewBucket(ctx, client.Config{ + StorageBackendConfig: client.StorageBackendConfig{ + Backend: client.Filesystem, + Filesystem: filesystem.Config{Directory: subDir}, + }, + }, "local-compactor") + if err != nil { + return false, nil, errors.Wrap(err, "create local bucket") + } + defer localBucket.Close() + + src := make([]phlaredb.BlockReader, len(toCompact)) + defer func() { + for _, b := range src { + if b != nil { + if err := b.Close(); err != nil { + level.Warn(jobLogger).Log("msg", "failed to close block", "err", err) + } + } + } + }() + err = concurrency.ForEachJob(ctx, len(src), c.blockOpenConcurrency, func(ctx context.Context, idx int) error { + meta := toCompact[idx] + b := phlaredb.NewSingleBlockQuerierFromMeta(ctx, localBucket, meta) + if err := b.Open(ctx); err != nil { + return errors.Wrapf(err, "open block %s", meta.ULID) + } + src[idx] = b + return nil + }) + if err != nil { + return false, nil, err + } elapsed := time.Since(downloadBegin) level.Info(jobLogger).Log("msg", "downloaded and verified blocks; compacting blocks", "blocks", len(blocksToCompactDirs), "plan", fmt.Sprintf("%v", blocksToCompactDirs), "duration", elapsed, "duration_ms", elapsed.Milliseconds()) - compactionBegin := time.Now() - + var out []block.Meta if job.UseSplitting() { - compIDs, err = c.comp.CompactWithSplitting(subDir, blocksToCompactDirs, nil, uint64(job.SplittingShards())) + out, err = phlaredb.CompactWithSplitting(ctx, src, uint64(job.SplittingShards()), subDir) } else { - var compID ulid.ULID - compID, err = c.comp.Compact(subDir, blocksToCompactDirs, nil) - compIDs = append(compIDs, compID) + out, err = phlaredb.CompactWithSplitting(ctx, src, 1, subDir) } if err != nil { return false, nil, errors.Wrapf(err, "compact blocks %v", blocksToCompactDirs) } + for _, o := range out { + compIDs = append(compIDs, o.ULID) + } if !hasNonZeroULIDs(compIDs) { // Prometheus compactor found that the compacted block would have no samples. @@ -550,7 +558,6 @@ type BucketCompactor struct { logger log.Logger sy *Syncer grouper Grouper - comp Compactor planner Planner compactDir string bkt objstore.Bucket @@ -559,6 +566,7 @@ type BucketCompactor struct { sortJobs JobsOrderFunc waitPeriod time.Duration blockSyncConcurrency int + blockOpenConcurrency int metrics *BucketCompactorMetrics } @@ -568,7 +576,6 @@ func NewBucketCompactor( sy *Syncer, grouper Grouper, planner Planner, - comp Compactor, compactDir string, bkt objstore.Bucket, concurrency int, @@ -576,6 +583,7 @@ func NewBucketCompactor( sortJobs JobsOrderFunc, waitPeriod time.Duration, blockSyncConcurrency int, + blockOpenConcurrency int, metrics *BucketCompactorMetrics, ) (*BucketCompactor, error) { if concurrency <= 0 { @@ -586,7 +594,6 @@ func NewBucketCompactor( sy: sy, grouper: grouper, planner: planner, - comp: comp, compactDir: compactDir, bkt: bkt, concurrency: concurrency, @@ -594,6 +601,7 @@ func NewBucketCompactor( sortJobs: sortJobs, waitPeriod: waitPeriod, blockSyncConcurrency: blockSyncConcurrency, + blockOpenConcurrency: blockOpenConcurrency, metrics: metrics, }, nil } diff --git a/pkg/compactor/compactor.go b/pkg/compactor/compactor.go index b2c96a21d1..8693e1a7ed 100644 --- a/pkg/compactor/compactor.go +++ b/pkg/compactor/compactor.go @@ -69,14 +69,6 @@ type BlocksGrouperFactory func( reg prometheus.Registerer, ) Grouper -// BlocksCompactorFactory builds and returns the compactor and planner to use to compact a tenant's blocks. -type BlocksCompactorFactory func( - ctx context.Context, - cfg Config, - logger log.Logger, - reg prometheus.Registerer, -) (Compactor, Planner, error) - // Config holds the MultitenantCompactor config. type Config struct { BlockRanges DurationList `yaml:"block_ranges" category:"advanced"` @@ -115,8 +107,7 @@ type Config struct { retryMaxBackoff time.Duration `yaml:"-"` // Allow downstream projects to customise the blocks compactor. - BlocksGrouperFactory BlocksGrouperFactory `yaml:"-"` - BlocksCompactorFactory BlocksCompactorFactory `yaml:"-"` + BlocksGrouperFactory BlocksGrouperFactory `yaml:"-"` } // RegisterFlags registers the MultitenantCompactor flags. @@ -228,15 +219,13 @@ type MultitenantCompactor struct { // Functions that creates bucket client, grouper, planner and compactor using the context. // Useful for injecting mock objects from tests. - blocksGrouperFactory BlocksGrouperFactory - blocksCompactorFactory BlocksCompactorFactory + blocksGrouperFactory BlocksGrouperFactory // Blocks cleaner is responsible to hard delete blocks marked for deletion. blocksCleaner *BlocksCleaner // Underlying compactor and planner used to compact TSDB blocks. - blocksCompactor Compactor - blocksPlanner Planner + blocksPlanner Planner // Client used to run operations on the bucket storing blocks. bucketClient objstore.Bucket @@ -279,14 +268,13 @@ type MultitenantCompactor struct { // NewMultitenantCompactor makes a new MultitenantCompactor. func NewMultitenantCompactor(compactorCfg Config, bucketClient objstore.Bucket, cfgProvider ConfigProvider, logger log.Logger, registerer prometheus.Registerer) (*MultitenantCompactor, error) { // Configure the compactor and grouper factories only if they weren't already set by a downstream project. - if compactorCfg.BlocksGrouperFactory == nil || compactorCfg.BlocksCompactorFactory == nil { + if compactorCfg.BlocksGrouperFactory == nil { configureSplitAndMergeCompactor(&compactorCfg) } blocksGrouperFactory := compactorCfg.BlocksGrouperFactory - blocksCompactorFactory := compactorCfg.BlocksCompactorFactory - mimirCompactor, err := newMultitenantCompactor(compactorCfg, bucketClient, cfgProvider, logger, registerer, blocksGrouperFactory, blocksCompactorFactory) + mimirCompactor, err := newMultitenantCompactor(compactorCfg, bucketClient, cfgProvider, logger, registerer, blocksGrouperFactory) if err != nil { return nil, errors.Wrap(err, "failed to create blocks compactor") } @@ -301,18 +289,17 @@ func newMultitenantCompactor( logger log.Logger, registerer prometheus.Registerer, blocksGrouperFactory BlocksGrouperFactory, - blocksCompactorFactory BlocksCompactorFactory, ) (*MultitenantCompactor, error) { c := &MultitenantCompactor{ - compactorCfg: compactorCfg, - cfgProvider: cfgProvider, - parentLogger: logger, - logger: log.With(logger, "component", "compactor"), - registerer: registerer, - syncerMetrics: newAggregatedSyncerMetrics(registerer), - bucketClient: bucketClient, - blocksGrouperFactory: blocksGrouperFactory, - blocksCompactorFactory: blocksCompactorFactory, + compactorCfg: compactorCfg, + cfgProvider: cfgProvider, + parentLogger: logger, + logger: log.With(logger, "component", "compactor"), + registerer: registerer, + syncerMetrics: newAggregatedSyncerMetrics(registerer), + bucketClient: bucketClient, + blocksGrouperFactory: blocksGrouperFactory, + blocksPlanner: NewSplitAndMergePlanner(compactorCfg.BlockRanges.ToMilliseconds()), compactionRunsStarted: promauto.With(registerer).NewCounter(prometheus.CounterOpts{ Name: "pyroscope_compactor_runs_started_total", @@ -408,12 +395,6 @@ func newMultitenantCompactor( func (c *MultitenantCompactor) starting(ctx context.Context) error { var err error - // Create blocks compactor dependencies. - c.blocksCompactor, c.blocksPlanner, err = c.blocksCompactorFactory(ctx, c.compactorCfg, c.logger, c.registerer) - if err != nil { - return errors.Wrap(err, "failed to initialize compactor dependencies") - } - // Wrap the bucket client to write block deletion marks in the global location too. c.bucketClient = block.BucketWithGlobalMarkers(c.bucketClient) @@ -740,7 +721,6 @@ func (c *MultitenantCompactor) compactUser(ctx context.Context, userID string) e syncer, c.blocksGrouperFactory(ctx, c.compactorCfg, c.cfgProvider, userID, userLogger, reg), c.blocksPlanner, - c.blocksCompactor, path.Join(c.compactorCfg.DataDir, "compact"), userBucket, c.compactorCfg.CompactionConcurrency, @@ -748,6 +728,7 @@ func (c *MultitenantCompactor) compactUser(ctx context.Context, userID string) e c.jobsOrder, c.compactorCfg.CompactionWaitPeriod, c.compactorCfg.BlockSyncConcurrency, + c.compactorCfg.MaxOpeningBlocksConcurrency, c.bucketCompactorMetrics, ) if err != nil { diff --git a/pkg/compactor/split_merge_compactor.go b/pkg/compactor/split_merge_compactor.go index 3e0f23872b..1995ca2e7a 100644 --- a/pkg/compactor/split_merge_compactor.go +++ b/pkg/compactor/split_merge_compactor.go @@ -9,7 +9,6 @@ import ( "github.com/go-kit/log" "github.com/prometheus/client_golang/prometheus" - "github.com/prometheus/prometheus/tsdb" ) func splitAndMergeGrouperFactory(_ context.Context, cfg Config, cfgProvider ConfigProvider, userID string, logger log.Logger, _ prometheus.Registerer) Grouper { @@ -21,26 +20,8 @@ func splitAndMergeGrouperFactory(_ context.Context, cfg Config, cfgProvider Conf logger) } -func splitAndMergeCompactorFactory(ctx context.Context, cfg Config, logger log.Logger, reg prometheus.Registerer) (Compactor, Planner, error) { - // We don't need to customise the TSDB compactor so we're just using the Prometheus one. - compactor, err := tsdb.NewLeveledCompactor(ctx, reg, logger, cfg.BlockRanges.ToMilliseconds(), nil, nil, true) - if err != nil { - return nil, nil, err - } - - opts := tsdb.DefaultLeveledCompactorConcurrencyOptions() - opts.MaxOpeningBlocks = cfg.MaxOpeningBlocksConcurrency - opts.MaxClosingBlocks = cfg.MaxClosingBlocksConcurrency - opts.SymbolsFlushersCount = cfg.SymbolsFlushersConcurrency - - compactor.SetConcurrencyOptions(opts) - - planner := NewSplitAndMergePlanner(cfg.BlockRanges.ToMilliseconds()) - return compactor, planner, nil -} // configureSplitAndMergeCompactor updates the provided configuration injecting the split-and-merge compactor. func configureSplitAndMergeCompactor(cfg *Config) { cfg.BlocksGrouperFactory = splitAndMergeGrouperFactory - cfg.BlocksCompactorFactory = splitAndMergeCompactorFactory } diff --git a/pkg/phlaredb/compact.go b/pkg/phlaredb/compact.go index 994c8fc831..aeadd09176 100644 --- a/pkg/phlaredb/compact.go +++ b/pkg/phlaredb/compact.go @@ -37,6 +37,7 @@ type BlockReader interface { Profiles() []parquet.RowGroup Index() IndexReader Symbols() symdb.SymbolsReader + Close() error } func Compact(ctx context.Context, src []BlockReader, dst string) (meta block.Meta, err error) { @@ -108,7 +109,9 @@ func CompactWithSplitting(ctx context.Context, src []BlockReader, shardsCount ui out := make([]block.Meta, 0, len(writers)) for shard, w := range writers { if w.meta.Stats.NumSamples > 0 { - w.meta.Labels[sharding.CompactorShardIDLabel] = sharding.FormatShardIDLabelValue(uint64(shard), shardsCount) + if shardsCount > 1 { + w.meta.Labels[sharding.CompactorShardIDLabel] = sharding.FormatShardIDLabelValue(uint64(shard), shardsCount) + } out = append(out, *w.meta) } } diff --git a/pkg/tenant/allowed_tenants_test.go b/pkg/tenant/allowed_tenants_test.go index 6647377dc7..23c113e9c5 100644 --- a/pkg/tenant/allowed_tenants_test.go +++ b/pkg/tenant/allowed_tenants_test.go @@ -43,7 +43,7 @@ func TestAllowedTenants_Combination(t *testing.T) { } func TestAllowedTenants_Nil(t *testing.T) { - var a *Allowed + var a *AllowedTenants // All tenants are allowed when using nil as allowed tenants. require.True(t, a.IsAllowed("A")) From 34ed7c6fc80b76ad903debdfaa66417b27a9c66c Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Wed, 13 Sep 2023 09:35:27 +0200 Subject: [PATCH 29/74] fixes block_cleaner_test --- pkg/compactor/blocks_cleaner_test.go | 91 ++++++++++++---------------- pkg/test/logger.go | 24 ++++++++ 2 files changed, 64 insertions(+), 51 deletions(-) create mode 100644 pkg/test/logger.go diff --git a/pkg/compactor/blocks_cleaner_test.go b/pkg/compactor/blocks_cleaner_test.go index fc8fb17ccf..ffba250c63 100644 --- a/pkg/compactor/blocks_cleaner_test.go +++ b/pkg/compactor/blocks_cleaner_test.go @@ -25,15 +25,14 @@ import ( "github.com/prometheus/client_golang/prometheus/testutil" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" - "github.com/thanos-io/objstore" - - "github.com/grafana/mimir/pkg/storage/bucket" - "github.com/grafana/mimir/pkg/storage/tsdb" - "github.com/grafana/mimir/pkg/storage/tsdb/block" - "github.com/grafana/mimir/pkg/storage/tsdb/bucketindex" - mimir_testutil "github.com/grafana/mimir/pkg/storage/tsdb/testutil" - "github.com/grafana/mimir/pkg/util" - "github.com/grafana/mimir/pkg/util/test" + + "github.com/grafana/pyroscope/pkg/objstore" + objstore_testutil "github.com/grafana/pyroscope/pkg/objstore/testutil" + "github.com/grafana/pyroscope/pkg/phlaredb/block" + "github.com/grafana/pyroscope/pkg/phlaredb/bucket" + "github.com/grafana/pyroscope/pkg/phlaredb/bucketindex" + "github.com/grafana/pyroscope/pkg/test" + "github.com/grafana/pyroscope/pkg/util" ) type testBlocksCleanerOptions struct { @@ -64,7 +63,7 @@ func TestBlocksCleaner(t *testing.T) { } func testBlocksCleanerWithOptions(t *testing.T, options testBlocksCleanerOptions) { - bucketClient, _ := mimir_testutil.PrepareFilesystemBucket(t) + bucketClient, _ := objstore_testutil.NewFilesystemBucket(t, context.Background(), t.TempDir()) bucketClient = block.BucketWithGlobalMarkers(bucketClient) // Create blocks. @@ -87,16 +86,14 @@ func testBlocksCleanerWithOptions(t *testing.T, options testBlocksCleanerOptions createDeletionMark(t, bucketClient, "user-2", block7, now.Add(-deletionDelay).Add(-time.Hour)) // Block reached the deletion threshold. // Blocks for user-3, marked for deletion. - require.NoError(t, tsdb.WriteTenantDeletionMark(context.Background(), bucketClient, "user-3", nil, tsdb.NewTenantDeletionMark(time.Now()))) + require.NoError(t, bucket.WriteTenantDeletionMark(context.Background(), bucketClient, "user-3", nil, bucket.NewTenantDeletionMark(time.Now()))) block9 := createTSDBBlock(t, bucketClient, "user-3", 10, 30, 2, nil) block10 := createTSDBBlock(t, bucketClient, "user-3", 30, 50, 2, nil) // User-4 with no more blocks, but couple of mark and debug files. Should be fully deleted. - user4Mark := tsdb.NewTenantDeletionMark(time.Now()) + user4Mark := bucket.NewTenantDeletionMark(time.Now()) user4Mark.FinishedTime = time.Now().Unix() - 60 // Set to check final user cleanup. - require.NoError(t, tsdb.WriteTenantDeletionMark(context.Background(), bucketClient, "user-4", nil, user4Mark)) - user4DebugMetaFile := path.Join("user-4", block.DebugMetas, "meta.json") - require.NoError(t, bucketClient.Upload(context.Background(), user4DebugMetaFile, strings.NewReader("some random content here"))) + require.NoError(t, bucket.WriteTenantDeletionMark(context.Background(), bucketClient, "user-4", nil, user4Mark)) cfg := BlocksCleanerConfig{ DeletionDelay: deletionDelay, @@ -110,7 +107,7 @@ func testBlocksCleanerWithOptions(t *testing.T, options testBlocksCleanerOptions logger := log.NewNopLogger() cfgProvider := newMockConfigProvider() - cleaner := NewBlocksCleaner(cfg, bucketClient, tsdb.AllUsers, cfgProvider, logger, reg) + cleaner := NewBlocksCleaner(cfg, bucketClient, bucket.AllTenants, cfgProvider, logger, reg) require.NoError(t, services.StartAndAwaitRunning(ctx, cleaner)) defer services.StopAndAwaitTerminated(ctx, cleaner) //nolint:errcheck @@ -141,10 +138,9 @@ func testBlocksCleanerWithOptions(t *testing.T, options testBlocksCleanerOptions {path: path.Join("user-3", block10.String(), block.MetaFilename), expectedExists: false}, {path: path.Join("user-3", block10.String(), "index"), expectedExists: false}, // Tenant deletion mark is not removed. - {path: path.Join("user-3", tsdb.TenantDeletionMarkPath), expectedExists: true}, + {path: path.Join("user-3", bucket.TenantDeletionMarkPath), expectedExists: true}, // User-4 is removed fully. - {path: path.Join("user-4", tsdb.TenantDeletionMarkPath), expectedExists: options.user4FilesExist}, - {path: path.Join("user-4", block.DebugMetas, "meta.json"), expectedExists: options.user4FilesExist}, + {path: path.Join("user-4", bucket.TenantDeletionMarkPath), expectedExists: options.user4FilesExist}, } { exists, err := bucketClient.Exists(ctx, tc.path) require.NoError(t, err) @@ -213,7 +209,7 @@ func testBlocksCleanerWithOptions(t *testing.T, options testBlocksCleanerOptions func TestBlocksCleaner_ShouldContinueOnBlockDeletionFailure(t *testing.T) { const userID = "user-1" - bucketClient, _ := mimir_testutil.PrepareFilesystemBucket(t) + bucketClient, _ := objstore_testutil.NewFilesystemBucket(t, context.Background(), t.TempDir()) bucketClient = block.BucketWithGlobalMarkers(bucketClient) // Create blocks. @@ -244,7 +240,7 @@ func TestBlocksCleaner_ShouldContinueOnBlockDeletionFailure(t *testing.T) { logger := log.NewNopLogger() cfgProvider := newMockConfigProvider() - cleaner := NewBlocksCleaner(cfg, bucketClient, tsdb.AllUsers, cfgProvider, logger, nil) + cleaner := NewBlocksCleaner(cfg, bucketClient, bucket.AllTenants, cfgProvider, logger, nil) require.NoError(t, services.StartAndAwaitRunning(ctx, cleaner)) defer services.StopAndAwaitTerminated(ctx, cleaner) //nolint:errcheck @@ -278,7 +274,7 @@ func TestBlocksCleaner_ShouldContinueOnBlockDeletionFailure(t *testing.T) { func TestBlocksCleaner_ShouldRebuildBucketIndexOnCorruptedOne(t *testing.T) { const userID = "user-1" - bucketClient, _ := mimir_testutil.PrepareFilesystemBucket(t) + bucketClient, _ := objstore_testutil.NewFilesystemBucket(t, context.Background(), t.TempDir()) bucketClient = block.BucketWithGlobalMarkers(bucketClient) // Create blocks. @@ -304,7 +300,7 @@ func TestBlocksCleaner_ShouldRebuildBucketIndexOnCorruptedOne(t *testing.T) { logger := log.NewNopLogger() cfgProvider := newMockConfigProvider() - cleaner := NewBlocksCleaner(cfg, bucketClient, tsdb.AllUsers, cfgProvider, logger, nil) + cleaner := NewBlocksCleaner(cfg, bucketClient, bucket.AllTenants, cfgProvider, logger, nil) require.NoError(t, services.StartAndAwaitRunning(ctx, cleaner)) defer services.StopAndAwaitTerminated(ctx, cleaner) //nolint:errcheck @@ -335,7 +331,7 @@ func TestBlocksCleaner_ShouldRebuildBucketIndexOnCorruptedOne(t *testing.T) { } func TestBlocksCleaner_ShouldRemoveMetricsForTenantsNotBelongingAnymoreToTheShard(t *testing.T) { - bucketClient, _ := mimir_testutil.PrepareFilesystemBucket(t) + bucketClient, _ := objstore_testutil.NewFilesystemBucket(t, context.Background(), t.TempDir()) bucketClient = block.BucketWithGlobalMarkers(bucketClient) // Create blocks. @@ -355,7 +351,7 @@ func TestBlocksCleaner_ShouldRemoveMetricsForTenantsNotBelongingAnymoreToTheShar reg := prometheus.NewPedanticRegistry() cfgProvider := newMockConfigProvider() - cleaner := NewBlocksCleaner(cfg, bucketClient, tsdb.AllUsers, cfgProvider, logger, reg) + cleaner := NewBlocksCleaner(cfg, bucketClient, bucket.AllTenants, cfgProvider, logger, reg) require.NoError(t, cleaner.runCleanupWithErr(ctx)) assert.NoError(t, testutil.GatherAndCompare(reg, strings.NewReader(` @@ -378,7 +374,7 @@ func TestBlocksCleaner_ShouldRemoveMetricsForTenantsNotBelongingAnymoreToTheShar )) // Override the users scanner to reconfigure it to only return a subset of users. - cleaner.tenantsScanner = tsdb.NewUsersScanner(bucketClient, func(userID string) (bool, error) { return userID == "user-1", nil }, logger) + cleaner.tenantsScanner = bucket.NewTenantsScanner(bucketClient, func(userID string) (bool, error) { return userID == "user-1", nil }, logger) // Create new blocks, to double check expected metrics have changed. createTSDBBlock(t, bucketClient, "user-1", 40, 50, 2, nil) @@ -404,7 +400,7 @@ func TestBlocksCleaner_ShouldRemoveMetricsForTenantsNotBelongingAnymoreToTheShar } func TestBlocksCleaner_ShouldNotCleanupUserThatDoesntBelongToShardAnymore(t *testing.T) { - bucketClient, _ := mimir_testutil.PrepareFilesystemBucket(t) + bucketClient, _ := objstore_testutil.NewFilesystemBucket(t, context.Background(), t.TempDir()) bucketClient = block.BucketWithGlobalMarkers(bucketClient) // Create blocks. @@ -455,7 +451,7 @@ func TestBlocksCleaner_ShouldNotCleanupUserThatDoesntBelongToShardAnymore(t *tes } func TestBlocksCleaner_ListBlocksOutsideRetentionPeriod(t *testing.T) { - bucketClient, _ := mimir_testutil.PrepareFilesystemBucket(t) + bucketClient, _ := objstore_testutil.NewFilesystemBucket(t, context.Background(), t.TempDir()) bucketClient = block.BucketWithGlobalMarkers(bucketClient) ctx := context.Background() logger := log.NewNopLogger() @@ -513,7 +509,7 @@ func TestBlocksCleaner_ListBlocksOutsideRetentionPeriod(t *testing.T) { } func TestBlocksCleaner_ShouldRemoveBlocksOutsideRetentionPeriod(t *testing.T) { - bucketClient, _ := mimir_testutil.PrepareFilesystemBucket(t) + bucketClient, _ := objstore_testutil.NewFilesystemBucket(t, context.Background(), t.TempDir()) bucketClient = block.BucketWithGlobalMarkers(bucketClient) ts := func(hours int) int64 { @@ -537,7 +533,7 @@ func TestBlocksCleaner_ShouldRemoveBlocksOutsideRetentionPeriod(t *testing.T) { reg := prometheus.NewPedanticRegistry() cfgProvider := newMockConfigProvider() - cleaner := NewBlocksCleaner(cfg, bucketClient, tsdb.AllUsers, cfgProvider, logger, reg) + cleaner := NewBlocksCleaner(cfg, bucketClient, bucket.AllTenants, cfgProvider, logger, reg) assertBlockExists := func(user string, blockID ulid.ULID, expectExists bool) { exists, err := bucketClient.Exists(ctx, path.Join(user, blockID.String(), block.MetaFilename)) @@ -699,7 +695,7 @@ func checkBlock(t *testing.T, user string, bucketClient objstore.Bucket, blockID } func TestBlocksCleaner_ShouldCleanUpFilesWhenNoMoreBlocksRemain(t *testing.T) { - bucketClient, _ := mimir_testutil.PrepareFilesystemBucket(t) + bucketClient, _ := objstore_testutil.NewFilesystemBucket(t, context.Background(), t.TempDir()) bucketClient = block.BucketWithGlobalMarkers(bucketClient) const userID = "user-1" @@ -725,10 +721,6 @@ func TestBlocksCleaner_ShouldCleanUpFilesWhenNoMoreBlocksRemain(t *testing.T) { require.NoError(t, err) assert.True(t, exists) - // Create a debug file that wouldn't otherwise be deleted by the cleaner - debugMetaFile := path.Join(userID, block.DebugMetas, "meta.json") - require.NoError(t, bucketClient.Upload(context.Background(), debugMetaFile, strings.NewReader("random content"))) - cfg := BlocksCleanerConfig{ DeletionDelay: deletionDelay, CleanupInterval: time.Minute, @@ -741,7 +733,7 @@ func TestBlocksCleaner_ShouldCleanUpFilesWhenNoMoreBlocksRemain(t *testing.T) { reg := prometheus.NewPedanticRegistry() cfgProvider := newMockConfigProvider() - cleaner := NewBlocksCleaner(cfg, bucketClient, tsdb.AllUsers, cfgProvider, logger, reg) + cleaner := NewBlocksCleaner(cfg, bucketClient, bucket.AllTenants, cfgProvider, logger, reg) require.NoError(t, cleaner.runCleanupWithErr(ctx)) // Check bucket index, markers and debug files have been deleted. @@ -749,16 +741,12 @@ func TestBlocksCleaner_ShouldCleanUpFilesWhenNoMoreBlocksRemain(t *testing.T) { require.NoError(t, err) assert.False(t, exists) - exists, err = bucketClient.Exists(ctx, debugMetaFile) - require.NoError(t, err) - assert.False(t, exists) - _, err = bucketindex.ReadIndex(ctx, bucketClient, userID, nil, logger) require.ErrorIs(t, err, bucketindex.ErrIndexNotFound) } func TestBlocksCleaner_ShouldRemovePartialBlocksOutsideDelayPeriod(t *testing.T) { - bucketClient, _ := mimir_testutil.PrepareFilesystemBucket(t) + bucketClient, _ := objstore_testutil.NewFilesystemBucket(t, context.Background(), t.TempDir()) bucketClient = block.BucketWithGlobalMarkers(bucketClient) ts := func(hours int) int64 { @@ -780,7 +768,7 @@ func TestBlocksCleaner_ShouldRemovePartialBlocksOutsideDelayPeriod(t *testing.T) reg := prometheus.NewPedanticRegistry() cfgProvider := newMockConfigProvider() - cleaner := NewBlocksCleaner(cfg, bucketClient, tsdb.AllUsers, cfgProvider, logger, reg) + cleaner := NewBlocksCleaner(cfg, bucketClient, bucket.AllTenants, cfgProvider, logger, reg) makeBlockPartial := func(user string, blockID ulid.ULID) { err := bucketClient.Delete(ctx, path.Join(user, blockID.String(), block.MetaFilename)) @@ -828,7 +816,7 @@ func TestBlocksCleaner_ShouldRemovePartialBlocksOutsideDelayPeriod(t *testing.T) } func TestBlocksCleaner_ShouldNotRemovePartialBlocksInsideDelayPeriod(t *testing.T) { - bucketClient, _ := mimir_testutil.PrepareFilesystemBucket(t) + bucketClient, _ := objstore_testutil.NewFilesystemBucket(t, context.Background(), t.TempDir()) bucketClient = block.BucketWithGlobalMarkers(bucketClient) ts := func(hours int) int64 { @@ -850,7 +838,7 @@ func TestBlocksCleaner_ShouldNotRemovePartialBlocksInsideDelayPeriod(t *testing. reg := prometheus.NewPedanticRegistry() cfgProvider := newMockConfigProvider() - cleaner := NewBlocksCleaner(cfg, bucketClient, tsdb.AllUsers, cfgProvider, logger, reg) + cleaner := NewBlocksCleaner(cfg, bucketClient, bucket.AllTenants, cfgProvider, logger, reg) makeBlockPartial := func(user string, blockID ulid.ULID) { err := bucketClient.Delete(ctx, path.Join(user, blockID.String(), block.MetaFilename)) @@ -914,7 +902,7 @@ func TestBlocksCleaner_ShouldNotRemovePartialBlocksIfConfiguredDelayIsInvalid(t logs := &concurrency.SyncBuffer{} logger := log.NewLogfmtLogger(logs) - bucketClient, _ := mimir_testutil.PrepareFilesystemBucket(t) + bucketClient, _ := objstore_testutil.NewFilesystemBucket(t, context.Background(), t.TempDir()) bucketClient = block.BucketWithGlobalMarkers(bucketClient) ts := func(hours int) int64 { @@ -942,7 +930,7 @@ func TestBlocksCleaner_ShouldNotRemovePartialBlocksIfConfiguredDelayIsInvalid(t checkBlock(t, "user-1", bucketClient, block1, false, false) // Run the cleanup. - cleaner := NewBlocksCleaner(cfg, bucketClient, tsdb.AllUsers, cfgProvider, logger, reg) + cleaner := NewBlocksCleaner(cfg, bucketClient, bucket.AllTenants, cfgProvider, logger, reg) require.NoError(t, cleaner.cleanUser(ctx, "user-1", logger)) // Ensure the block has NOT been marked for deletion. @@ -968,17 +956,18 @@ func TestBlocksCleaner_ShouldNotRemovePartialBlocksIfConfiguredDelayIsInvalid(t } func TestStalePartialBlockLastModifiedTime(t *testing.T) { - b, dir := mimir_testutil.PrepareFilesystemBucket(t) + dir := t.TempDir() + b, _ := objstore_testutil.NewFilesystemBucket(t, context.Background(), dir) - const tenant = "user" + const tenantId = "user" objectTime := time.Now().Add(-1 * time.Hour).Truncate(time.Second) // ignore milliseconds, as not all filesystems store them. - blockID := createTSDBBlock(t, b, tenant, objectTime.UnixMilli(), time.Now().UnixMilli(), 2, nil) + blockID := createTSDBBlock(t, b, tenantId, objectTime.UnixMilli(), time.Now().UnixMilli(), 2, nil) for _, f := range []string{"meta.json", "index", "chunks/000001", "tombstones"} { - require.NoError(t, os.Chtimes(filepath.Join(dir, tenant, blockID.String(), filepath.FromSlash(f)), objectTime, objectTime)) + require.NoError(t, os.Chtimes(filepath.Join(dir, tenantId, blockID.String(), filepath.FromSlash(f)), objectTime, objectTime)) } - userBucket := bucket.NewUserBucketClient(tenant, b, nil) + userBucket := objstore.NewUserBucketClient(tenantId, b, nil) emptyBlockID := ulid.ULID{} require.NotEqual(t, blockID, emptyBlockID) diff --git a/pkg/test/logger.go b/pkg/test/logger.go new file mode 100644 index 0000000000..8c0831e8f9 --- /dev/null +++ b/pkg/test/logger.go @@ -0,0 +1,24 @@ +// SPDX-License-Identifier: AGPL-3.0-only + +package test + +import ( + "testing" + + "github.com/go-kit/log" +) + +type testingLogger struct { + t testing.TB +} + +func NewTestingLogger(t testing.TB) log.Logger { + return &testingLogger{ + t: t, + } +} + +func (l *testingLogger) Log(keyvals ...interface{}) error { + l.t.Log(keyvals...) + return nil +} From 0647cd9eb7b5de5d34571ec21f8cb5ee42f10601 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Wed, 13 Sep 2023 09:37:26 +0200 Subject: [PATCH 30/74] remove upload block api --- pkg/compactor/block_upload.go | 864 ------------ pkg/compactor/block_upload_test.go | 2116 ---------------------------- 2 files changed, 2980 deletions(-) delete mode 100644 pkg/compactor/block_upload.go delete mode 100644 pkg/compactor/block_upload_test.go diff --git a/pkg/compactor/block_upload.go b/pkg/compactor/block_upload.go deleted file mode 100644 index d288a5ad4d..0000000000 --- a/pkg/compactor/block_upload.go +++ /dev/null @@ -1,864 +0,0 @@ -// // SPDX-License-Identifier: AGPL-3.0-only -// // Provenance-includes-location: https://github.com/grafana/mimir/blob/main/pkg/compactor/block_upload.go -// // Provenance-includes-license: Apache-2.0 -// // Provenance-includes-copyright: The Cortex Authors. - -package compactor - -// import ( -// "bytes" -// "context" -// "encoding/binary" -// "encoding/hex" -// "encoding/json" -// "fmt" -// "io" -// "net/http" -// "os" -// "path" -// "path/filepath" -// "sync" -// "time" - -// "github.com/go-kit/log" -// "github.com/go-kit/log/level" -// "github.com/gorilla/mux" -// "github.com/grafana/dskit/tenant" -// "github.com/grafana/regexp" -// "github.com/oklog/ulid" -// "github.com/pkg/errors" -// "github.com/thanos-io/objstore" - -// "github.com/grafana/mimir/pkg/storage/bucket" -// "github.com/grafana/mimir/pkg/storage/sharding" -// mimir_tsdb "github.com/grafana/mimir/pkg/storage/tsdb" -// "github.com/grafana/mimir/pkg/storage/tsdb/block" -// "github.com/grafana/mimir/pkg/util" -// util_log "github.com/grafana/mimir/pkg/util/log" -// ) - -// const ( -// uploadingMetaFilename = "uploading-meta.json" // Name of the file that stores a block's meta file while it's being uploaded -// validationFilename = "validation.json" // Name of the file that stores a heartbeat time and possibly an error message -// validationHeartbeatInterval = 1 * time.Minute // Duration of time between heartbeats of an in-progress block upload validation -// validationHeartbeatTimeout = 5 * time.Minute // Maximum duration of time to wait until a validation is able to be restarted -// maximumMetaSizeBytes = 1 * 1024 * 1024 // 1 MiB, maximum allowed size of an uploaded block's meta.json file -// ) - -// var ( -// maxBlockUploadSizeBytesFormat = "block exceeds the maximum block size limit of %d bytes" -// rePath = regexp.MustCompile(`^(index|chunks/\d{6})$`) -// ) - -// // StartBlockUpload handles request for starting block upload. -// // -// // Starting the uploading of a block means to upload a meta file and verify that the upload can -// // go ahead. In practice this means to check that the (complete) block isn't already in block -// // storage, and that the meta file is valid. -// func (c *MultitenantCompactor) StartBlockUpload(w http.ResponseWriter, r *http.Request) { -// blockID, tenantID, err := c.parseBlockUploadParameters(r) -// if err != nil { -// http.Error(w, err.Error(), http.StatusBadRequest) -// return -// } - -// ctx := r.Context() -// requestID := hexTimeNowNano() -// logger := log.With( -// util_log.WithContext(ctx, c.logger), -// "feature", "block upload", -// "block", blockID, -// "operation", "start block upload", -// "request_id", requestID, -// ) - -// userBkt := bucket.NewUserBucketClient(tenantID, c.bucketClient, c.cfgProvider) -// if _, _, err := c.checkBlockState(ctx, userBkt, blockID, false); err != nil { -// writeBlockUploadError(err, "can't check block state", logger, w, requestID) -// return -// } - -// content, err := io.ReadAll(http.MaxBytesReader(w, r.Body, maximumMetaSizeBytes)) -// if err != nil { -// if errors.As(err, new(*http.MaxBytesError)) { -// err = httpError{ -// message: fmt.Sprintf("The block metadata was too large (maximum size allowed is %d bytes)", maximumMetaSizeBytes), -// statusCode: http.StatusRequestEntityTooLarge, -// } -// } -// writeBlockUploadError(err, "failed reading body", logger, w, requestID) -// return -// } - -// var meta block.Meta -// if err := json.Unmarshal(content, &meta); err != nil { -// err = httpError{ -// message: "malformed request body", -// statusCode: http.StatusBadRequest, -// } -// writeBlockUploadError(err, "failed unmarshaling block meta json", logger, w, requestID) -// return -// } - -// if err := c.createBlockUpload(ctx, &meta, logger, userBkt, tenantID, blockID); err != nil { -// writeBlockUploadError(err, "failed creating block upload", logger, w, requestID) -// return -// } - -// level.Info(logger).Log("msg", "started block upload") - -// w.WriteHeader(http.StatusOK) -// } - -// // FinishBlockUpload handles request for finishing block upload. -// // -// // Finishing block upload performs block validation, and if all checks pass, marks block as finished -// // by uploading meta.json file. -// func (c *MultitenantCompactor) FinishBlockUpload(w http.ResponseWriter, r *http.Request) { -// blockID, tenantID, err := c.parseBlockUploadParameters(r) -// if err != nil { -// http.Error(w, err.Error(), http.StatusBadRequest) -// return -// } - -// ctx := r.Context() -// requestID := hexTimeNowNano() -// logger := log.With( -// util_log.WithContext(ctx, c.logger), -// "feature", "block upload", -// "block", blockID, -// "operation", "complete block upload", -// "request_id", requestID, -// ) - -// userBkt := bucket.NewUserBucketClient(tenantID, c.bucketClient, c.cfgProvider) -// m, _, err := c.checkBlockState(ctx, userBkt, blockID, true) -// if err != nil { -// writeBlockUploadError(err, "can't check block state", logger, w, requestID) -// return -// } - -// // This should not happen, as checkBlockState with requireUploadInProgress=true returns nil error -// // only if uploading-meta.json file exists. -// if m == nil { -// http.Error(w, "internal error", http.StatusInternalServerError) -// return -// } - -// if c.cfgProvider.CompactorBlockUploadValidationEnabled(tenantID) { -// maxConcurrency := int64(c.compactorCfg.MaxBlockUploadValidationConcurrency) -// currentValidations := c.blockUploadValidations.Inc() -// decreaseActiveValidationsInDefer := true -// defer func() { -// if decreaseActiveValidationsInDefer { -// c.blockUploadValidations.Dec() -// } -// }() -// if maxConcurrency > 0 && currentValidations > maxConcurrency { -// err := httpError{ -// message: fmt.Sprintf("too many block upload validations in progress, limit is %d", maxConcurrency), -// statusCode: http.StatusTooManyRequests, -// } -// writeBlockUploadError(err, "max concurrency was hit", logger, w, requestID) -// return -// } -// // create validation file to signal that block validation has started -// if err := c.uploadValidation(ctx, blockID, userBkt); err != nil { -// writeBlockUploadError(err, "can't upload validation file", logger, w, requestID) -// return -// } -// decreaseActiveValidationsInDefer = false -// go c.validateAndCompleteBlockUpload(logger, tenantID, userBkt, blockID, m, func(ctx context.Context) error { -// defer c.blockUploadValidations.Dec() -// return c.validateBlock(ctx, logger, blockID, m, userBkt, tenantID) -// }) -// level.Info(logger).Log("msg", "validation process started") -// } else { -// if err := c.markBlockComplete(ctx, logger, tenantID, userBkt, blockID, m); err != nil { -// writeBlockUploadError(err, "can't mark block as complete", logger, w, requestID) -// return -// } -// level.Info(logger).Log("msg", "successfully finished block upload") -// } - -// w.WriteHeader(http.StatusOK) -// } - -// // parseBlockUploadParameters parses common parameters from the request: block ID, tenant and checks if tenant has uploads enabled. -// func (c *MultitenantCompactor) parseBlockUploadParameters(r *http.Request) (ulid.ULID, string, error) { -// blockID, err := ulid.Parse(mux.Vars(r)["block"]) -// if err != nil { -// return ulid.ULID{}, "", errors.New("invalid block ID") -// } - -// ctx := r.Context() -// tenantID, err := tenant.TenantID(ctx) -// if err != nil { -// return ulid.ULID{}, "", errors.New("invalid tenant ID") -// } - -// if !c.cfgProvider.CompactorBlockUploadEnabled(tenantID) { -// return ulid.ULID{}, "", errors.New("block upload is disabled") -// } - -// return blockID, tenantID, nil -// } - -// func writeBlockUploadError(err error, msg string, logger log.Logger, w http.ResponseWriter, requestID string) { -// var httpErr httpError -// if errors.As(err, &httpErr) { -// level.Warn(logger).Log("msg", msg, "response", httpErr.message, "status", httpErr.statusCode) -// http.Error(w, httpErr.message, httpErr.statusCode) -// return -// } - -// level.Error(logger).Log("msg", msg, "err", err) -// http.Error(w, fmt.Sprintf("internal server error (id %s)", requestID), http.StatusInternalServerError) -// } - -// // hexTimeNano returns a hex-encoded big-endian representation of the current time in nanoseconds, previously converted to uint64 and encoded as big-endian. -// func hexTimeNowNano() string { -// var buf [8]byte -// binary.BigEndian.PutUint64(buf[:], uint64(time.Now().UTC().UnixNano())) -// return hex.EncodeToString(buf[:]) -// } - -// func (c *MultitenantCompactor) createBlockUpload(ctx context.Context, meta *block.Meta, -// logger log.Logger, userBkt objstore.Bucket, tenantID string, blockID ulid.ULID, -// ) error { -// level.Debug(logger).Log("msg", "starting block upload") - -// if msg := c.sanitizeMeta(logger, tenantID, blockID, meta); msg != "" { -// return httpError{ -// message: msg, -// statusCode: http.StatusBadRequest, -// } -// } - -// // validate data is within the retention period -// retention := c.cfgProvider.CompactorBlocksRetentionPeriod(tenantID) -// if retention > 0 { -// threshold := time.Now().Add(-retention) -// if time.UnixMilli(meta.MaxTime).Before(threshold) { -// maxTimeStr := util.FormatTimeMillis(meta.MaxTime) -// return httpError{ -// message: fmt.Sprintf("block max time (%s) older than retention period", maxTimeStr), -// statusCode: http.StatusUnprocessableEntity, -// } -// } -// } - -// return c.uploadMeta(ctx, logger, meta, blockID, uploadingMetaFilename, userBkt) -// } - -// // UploadBlockFile handles requests for uploading block files. -// // It takes the mandatory query parameter "path", specifying the file's destination path. -// func (c *MultitenantCompactor) UploadBlockFile(w http.ResponseWriter, r *http.Request) { -// blockID, tenantID, err := c.parseBlockUploadParameters(r) -// if err != nil { -// http.Error(w, err.Error(), http.StatusBadRequest) -// return -// } - -// ctx := r.Context() -// requestID := hexTimeNowNano() -// logger := log.With( -// util_log.WithContext(ctx, c.logger), -// "feature", "block upload", -// "block", blockID, -// "operation", "block file upload", -// "request", requestID, -// ) - -// pth := r.URL.Query().Get("path") -// if pth == "" { -// err := httpError{statusCode: http.StatusBadRequest, message: "missing or invalid file path"} -// writeBlockUploadError(err, "failed because file path is empty", logger, w, requestID) -// return -// } - -// if path.Base(pth) == block.MetaFilename { -// err := httpError{statusCode: http.StatusBadRequest, message: fmt.Sprintf("%s is not allowed", block.MetaFilename)} -// writeBlockUploadError(err, "failed because block meta is not allowed", logger, w, requestID) -// return -// } - -// if !rePath.MatchString(pth) { -// err := httpError{statusCode: http.StatusBadRequest, message: fmt.Sprintf("invalid path: %q", pth)} -// writeBlockUploadError(err, "failed because path is invalid", logger, w, requestID) -// return -// } - -// if r.ContentLength == 0 { -// err := httpError{statusCode: http.StatusBadRequest, message: "file cannot be empty"} -// writeBlockUploadError(err, "failed because file is empty", logger, w, requestID) -// return -// } - -// userBkt := bucket.NewUserBucketClient(tenantID, c.bucketClient, c.cfgProvider) - -// m, _, err := c.checkBlockState(ctx, userBkt, blockID, true) -// if err != nil { -// writeBlockUploadError(err, "can't check block state", logger, w, requestID) -// return -// } - -// // This should not happen. -// if m == nil { -// err := httpError{statusCode: http.StatusInternalServerError, message: "internal error"} -// writeBlockUploadError(err, "block meta is nil but err is also nil", logger, w, requestID) -// return -// } - -// // Check if file was specified in meta.json, and if it has expected size. -// found := false -// for _, f := range m.Thanos.Files { -// if pth == f.RelPath { -// found = true - -// if r.ContentLength >= 0 && r.ContentLength != f.SizeBytes { -// err := httpError{statusCode: http.StatusBadRequest, message: fmt.Sprintf("file size doesn't match %s", block.MetaFilename)} -// writeBlockUploadError(err, "failed because file size didn't match", logger, w, requestID) -// return -// } -// } -// } -// if !found { -// err := httpError{statusCode: http.StatusBadRequest, message: "unexpected file"} -// writeBlockUploadError(err, "failed because file was not found", logger, w, requestID) -// return -// } - -// dst := path.Join(blockID.String(), pth) - -// level.Debug(logger).Log("msg", "uploading block file to bucket", "destination", dst, "size", r.ContentLength) -// reader := bodyReader{r: r} -// if err := userBkt.Upload(ctx, dst, reader); err != nil { -// // We don't know what caused the error; it could be the client's fault (e.g. killed -// // connection), but internal server error is the safe choice here. -// level.Error(logger).Log("msg", "failed uploading block file to bucket", "destination", dst, "err", err) -// http.Error(w, fmt.Sprintf("internal server error (id %s)", requestID), http.StatusInternalServerError) -// return -// } - -// level.Debug(logger).Log("msg", "finished uploading block file to bucket", "path", pth) - -// w.WriteHeader(http.StatusOK) -// } - -// func (c *MultitenantCompactor) validateAndCompleteBlockUpload(logger log.Logger, tenantID string, userBkt objstore.Bucket, blockID ulid.ULID, meta *block.Meta, validation func(context.Context) error) { -// level.Debug(logger).Log("msg", "completing block upload", "files", len(meta.Thanos.Files)) - -// { -// var wg sync.WaitGroup -// ctx, cancel := context.WithCancel(context.Background()) - -// // start a go routine that updates the validation file's timestamp every heartbeat interval -// wg.Add(1) -// go func() { -// defer wg.Done() -// c.periodicValidationUpdater(ctx, logger, blockID, userBkt, cancel, validationHeartbeatInterval) -// }() - -// if err := validation(ctx); err != nil { -// level.Error(logger).Log("msg", "error while validating block", "err", err) -// cancel() -// wg.Wait() -// err := c.uploadValidationWithError(context.Background(), blockID, userBkt, err.Error()) -// if err != nil { -// level.Error(logger).Log("msg", "error updating validation file after failed block validation", "err", err) -// } -// return -// } - -// cancel() -// wg.Wait() // use waitgroup to ensure validation ts update is complete -// } - -// ctx := context.Background() - -// if err := c.markBlockComplete(ctx, logger, tenantID, userBkt, blockID, meta); err != nil { -// if err := c.uploadValidationWithError(ctx, blockID, userBkt, err.Error()); err != nil { -// level.Error(logger).Log("msg", "error updating validation file after upload of metadata file failed", "err", err) -// } -// return -// } - -// if err := userBkt.Delete(ctx, path.Join(blockID.String(), validationFilename)); err != nil { -// level.Warn(logger).Log("msg", fmt.Sprintf( -// "failed to delete %s from block in object storage", validationFilename), "err", err) -// return -// } - -// level.Info(logger).Log("msg", "successfully completed block upload") -// } - -// func (c *MultitenantCompactor) markBlockComplete(ctx context.Context, logger log.Logger, tenantID string, userBkt objstore.Bucket, blockID ulid.ULID, meta *block.Meta) error { -// if err := c.uploadMeta(ctx, logger, meta, blockID, block.MetaFilename, userBkt); err != nil { -// level.Error(logger).Log("msg", "error uploading block metadata file", "err", err) -// return err -// } - -// if err := userBkt.Delete(ctx, path.Join(blockID.String(), uploadingMetaFilename)); err != nil { -// // Not returning an error since the temporary meta file persisting is a harmless side effect -// level.Warn(logger).Log("msg", fmt.Sprintf("failed to delete %s from block in object storage", uploadingMetaFilename), "err", err) -// } - -// // Increment metrics on successful block upload -// c.blockUploadBlocks.WithLabelValues(tenantID).Inc() -// c.blockUploadBytes.WithLabelValues(tenantID).Add(float64(meta.BlockBytes())) -// c.blockUploadFiles.WithLabelValues(tenantID).Add(float64(len(meta.Thanos.Files))) - -// return nil -// } - -// // sanitizeMeta sanitizes and validates a metadata.Meta object. If a validation error occurs, an error -// // message gets returned, otherwise an empty string. -// func (c *MultitenantCompactor) sanitizeMeta(logger log.Logger, userID string, blockID ulid.ULID, meta *block.Meta) string { -// if meta == nil { -// return "missing block metadata" -// } - -// // check that the blocks doesn't contain down-sampled data -// if meta.Thanos.Downsample.Resolution > 0 { -// return "block contains downsampled data" -// } - -// meta.ULID = blockID -// for l, v := range meta.Thanos.Labels { -// switch l { -// // Preserve this label -// case mimir_tsdb.CompactorShardIDExternalLabel: -// if v == "" { -// level.Debug(logger).Log("msg", "removing empty external label", -// "label", l) -// delete(meta.Thanos.Labels, l) -// continue -// } - -// if _, _, err := sharding.ParseShardIDLabelValue(v); err != nil { -// return fmt.Sprintf("invalid %s external label: %q", -// mimir_tsdb.CompactorShardIDExternalLabel, v) -// } -// // Remove unused labels -// case mimir_tsdb.DeprecatedTenantIDExternalLabel, mimir_tsdb.DeprecatedIngesterIDExternalLabel, mimir_tsdb.DeprecatedShardIDExternalLabel: -// level.Debug(logger).Log("msg", "removing unused external label", -// "label", l, "value", v) -// delete(meta.Thanos.Labels, l) -// default: -// return fmt.Sprintf("unsupported external label: %s", l) -// } -// } - -// meta.Compaction.Parents = nil -// meta.Compaction.Sources = []ulid.ULID{blockID} - -// for _, f := range meta.Thanos.Files { -// if f.RelPath == block.MetaFilename { -// continue -// } - -// if !rePath.MatchString(f.RelPath) { -// return fmt.Sprintf("file with invalid path: %s", f.RelPath) -// } - -// if f.SizeBytes <= 0 { -// return fmt.Sprintf("file with invalid size: %s", f.RelPath) -// } -// } - -// if err := c.validateMaximumBlockSize(logger, meta.Thanos.Files, userID); err != nil { -// return err.Error() -// } - -// if meta.Version != block.TSDBVersion1 { -// return fmt.Sprintf("version must be %d", block.TSDBVersion1) -// } - -// // validate minTime/maxTime -// // basic sanity check -// if meta.MinTime < 0 || meta.MaxTime < 0 || meta.MaxTime < meta.MinTime { -// return fmt.Sprintf("invalid minTime/maxTime: minTime=%d, maxTime=%d", -// meta.MinTime, meta.MaxTime) -// } -// // validate that times are in the past -// now := time.Now() -// if meta.MinTime > now.UnixMilli() || meta.MaxTime > now.UnixMilli() { -// return fmt.Sprintf("block time(s) greater than the present: minTime=%d, maxTime=%d", -// meta.MinTime, meta.MaxTime) -// } - -// // Mark block source -// meta.Thanos.Source = "upload" - -// return "" -// } - -// func (c *MultitenantCompactor) uploadMeta(ctx context.Context, logger log.Logger, meta *block.Meta, blockID ulid.ULID, name string, userBkt objstore.Bucket) error { -// if meta == nil { -// return errors.New("missing block metadata") -// } -// dst := path.Join(blockID.String(), name) -// level.Debug(logger).Log("msg", fmt.Sprintf("uploading %s to bucket", name), "dst", dst) -// buf := bytes.NewBuffer(nil) -// if err := json.NewEncoder(buf).Encode(meta); err != nil { -// return errors.Wrap(err, "failed to encode block metadata") -// } -// if err := userBkt.Upload(ctx, dst, buf); err != nil { -// return errors.Wrapf(err, "failed uploading %s to bucket", name) -// } - -// return nil -// } - -// func (c *MultitenantCompactor) createTemporaryBlockDirectory() (dir string, err error) { -// blockDir, err := os.MkdirTemp(c.compactorCfg.DataDir, "upload") -// if err != nil { -// level.Error(c.logger).Log("msg", "failed to create temporary block directory", "err", err) -// return "", errors.New("failed to create temporary block directory") -// } - -// level.Debug(c.logger).Log("msg", "created temporary block directory", "dir", blockDir) -// return blockDir, nil -// } - -// func (c *MultitenantCompactor) removeTemporaryBlockDirectory(blockDir string) { -// level.Debug(c.logger).Log("msg", "removing temporary block directory", "dir", blockDir) -// if err := os.RemoveAll(blockDir); err != nil { -// level.Warn(c.logger).Log("msg", "failed to remove temporary block directory", "path", blockDir, "err", err) -// } -// } - -// func (c *MultitenantCompactor) prepareBlockForValidation(ctx context.Context, userBkt objstore.Bucket, blockID ulid.ULID) (string, error) { -// blockDir, err := c.createTemporaryBlockDirectory() -// if err != nil { -// return "", err -// } - -// // download the block to local storage -// level.Debug(c.logger).Log("msg", "downloading block from bucket", "block", blockID.String()) -// err = objstore.DownloadDir(ctx, c.logger, userBkt, blockID.String(), blockID.String(), blockDir) -// if err != nil { -// c.removeTemporaryBlockDirectory(blockDir) -// return "", errors.Wrap(err, "failed to download block") -// } - -// // rename the temporary meta file name to the expected one locally so that the block can be inspected -// err = os.Rename(filepath.Join(blockDir, uploadingMetaFilename), filepath.Join(blockDir, block.MetaFilename)) -// if err != nil { -// level.Warn(c.logger).Log("msg", "could not rename temporary metadata file", "block", blockID.String(), "err", err) -// c.removeTemporaryBlockDirectory(blockDir) -// return "", errors.New("failed renaming while preparing block for validation") -// } - -// return blockDir, nil -// } - -// func (c *MultitenantCompactor) validateBlock(ctx context.Context, logger log.Logger, blockID ulid.ULID, blockMetadata *block.Meta, userBkt objstore.Bucket, userID string) error { -// if err := c.validateMaximumBlockSize(logger, blockMetadata.Thanos.Files, userID); err != nil { -// return err -// } - -// blockDir, err := c.prepareBlockForValidation(ctx, userBkt, blockID) -// if err != nil { -// return err -// } -// defer c.removeTemporaryBlockDirectory(blockDir) - -// // check that all files listed in the metadata are present and the correct size -// for _, f := range blockMetadata.Thanos.Files { -// fi, err := os.Stat(filepath.Join(blockDir, filepath.FromSlash(f.RelPath))) -// if err != nil { -// return errors.Wrapf(err, "failed to stat %s", f.RelPath) -// } - -// if !fi.Mode().IsRegular() { -// return errors.Errorf("not a file: %s", f.RelPath) -// } - -// if f.RelPath != block.MetaFilename && fi.Size() != f.SizeBytes { -// return errors.Errorf("file size mismatch for %s", f.RelPath) -// } -// } - -// // validate block -// checkChunks := c.cfgProvider.CompactorBlockUploadVerifyChunks(userID) -// err = block.VerifyBlock(c.logger, blockDir, blockMetadata.MinTime, blockMetadata.MaxTime, checkChunks) -// if err != nil { -// return errors.Wrap(err, "error validating block") -// } - -// return nil -// } - -// func (c *MultitenantCompactor) validateMaximumBlockSize(logger log.Logger, files []block.File, userID string) error { -// maxBlockSizeBytes := c.cfgProvider.CompactorBlockUploadMaxBlockSizeBytes(userID) -// if maxBlockSizeBytes <= 0 { -// return nil -// } - -// blockSizeBytes := int64(0) -// for _, f := range files { -// if f.SizeBytes < 0 { -// return errors.New("invalid negative file size in block metadata") -// } -// blockSizeBytes += f.SizeBytes -// if blockSizeBytes < 0 { -// // overflow -// break -// } -// } - -// if blockSizeBytes > maxBlockSizeBytes || blockSizeBytes < 0 { -// level.Error(logger).Log("msg", "rejecting block upload for exceeding maximum size", "limit", maxBlockSizeBytes, "size", blockSizeBytes) -// return fmt.Errorf(maxBlockUploadSizeBytesFormat, maxBlockSizeBytes) -// } -// return nil -// } - -// type httpError struct { -// message string -// statusCode int -// } - -// func (e httpError) Error() string { -// return e.message -// } - -// type bodyReader struct { -// r *http.Request -// } - -// // ObjectSize implements thanos.ObjectSizer. -// func (r bodyReader) ObjectSize() (int64, error) { -// if r.r.ContentLength < 0 { -// return 0, fmt.Errorf("unknown size") -// } - -// return r.r.ContentLength, nil -// } - -// // Read implements io.Reader. -// func (r bodyReader) Read(b []byte) (int, error) { -// return r.r.Body.Read(b) -// } - -// type validationFile struct { -// LastUpdate int64 // UnixMillis of last update time. -// Error string // Error message if validation failed. -// } - -// type blockUploadStateResult struct { -// State string `json:"result"` -// Error string `json:"error,omitempty"` -// } - -// type blockUploadState int - -// const ( -// blockStateUnknown blockUploadState = iota // unknown, default value -// blockIsComplete // meta.json file exists -// blockUploadNotStarted // meta.json doesn't exist, uploading-meta.json doesn't exist -// blockUploadInProgress // meta.json doesn't exist, but uploading-meta.json does -// blockValidationInProgress // meta.json doesn't exist, uploading-meta.json exists, validation.json exists and is recent -// blockValidationFailed -// blockValidationStale -// ) - -// func (c *MultitenantCompactor) GetBlockUploadStateHandler(w http.ResponseWriter, r *http.Request) { -// blockID, tenantID, err := c.parseBlockUploadParameters(r) -// if err != nil { -// http.Error(w, err.Error(), http.StatusBadRequest) -// return -// } -// requestID := hexTimeNowNano() -// logger := log.With( -// util_log.WithContext(r.Context(), c.logger), -// "feature", "block upload", -// "block", blockID, -// "operation", "get block state", -// "request_id", requestID, -// ) - -// userBkt := bucket.NewUserBucketClient(tenantID, c.bucketClient, c.cfgProvider) -// if err != nil { -// http.Error(w, err.Error(), http.StatusBadRequest) -// return -// } - -// s, _, v, err := c.getBlockUploadState(r.Context(), userBkt, blockID) -// if err != nil { -// writeBlockUploadError(err, "can't get upload state", logger, w, requestID) -// return -// } - -// res := blockUploadStateResult{} - -// switch s { -// case blockIsComplete: -// res.State = "complete" -// case blockUploadNotStarted: -// http.Error(w, "block doesn't exist", http.StatusNotFound) -// return -// case blockValidationStale: -// fallthrough -// case blockUploadInProgress: -// res.State = "uploading" -// case blockValidationInProgress: -// res.State = "validating" -// case blockValidationFailed: -// res.State = "failed" -// res.Error = v.Error -// } - -// util.WriteJSONResponse(w, res) -// } - -// // checkBlockState checks blocks state and returns various HTTP status codes for individual states if block -// // upload cannot start, finish or file cannot be uploaded to the block. -// func (c *MultitenantCompactor) checkBlockState(ctx context.Context, userBkt objstore.Bucket, blockID ulid.ULID, requireUploadInProgress bool) (*block.Meta, *validationFile, error) { -// s, m, v, err := c.getBlockUploadState(ctx, userBkt, blockID) -// if err != nil { -// return m, v, err -// } - -// switch s { -// case blockIsComplete: -// return m, v, httpError{message: "block already exists", statusCode: http.StatusConflict} -// case blockValidationInProgress: -// return m, v, httpError{message: "block validation in progress", statusCode: http.StatusBadRequest} -// case blockUploadNotStarted: -// if requireUploadInProgress { -// return m, v, httpError{message: "block upload not started", statusCode: http.StatusNotFound} -// } -// return m, v, nil -// case blockValidationStale: -// // if validation is stale, we treat block as being in "upload in progress" state, and validation can start again. -// fallthrough -// case blockUploadInProgress: -// return m, v, nil -// case blockValidationFailed: -// return m, v, httpError{message: "block validation failed", statusCode: http.StatusBadRequest} -// } - -// return m, v, httpError{message: "unknown block upload state", statusCode: http.StatusInternalServerError} -// } - -// // getBlockUploadState returns state of the block upload, and meta and validation objects, if they exist. -// func (c *MultitenantCompactor) getBlockUploadState(ctx context.Context, userBkt objstore.Bucket, blockID ulid.ULID) (blockUploadState, *block.Meta, *validationFile, error) { -// exists, err := userBkt.Exists(ctx, path.Join(blockID.String(), block.MetaFilename)) -// if err != nil { -// return blockStateUnknown, nil, nil, err -// } -// if exists { -// return blockIsComplete, nil, nil, nil -// } - -// meta, err := c.loadUploadingMeta(ctx, userBkt, blockID) -// if err != nil { -// return blockStateUnknown, nil, nil, err -// } -// // If neither meta.json nor uploading-meta.json file exist, we say that the block doesn't exist. -// if meta == nil { -// return blockUploadNotStarted, nil, nil, err -// } - -// v, err := c.loadValidation(ctx, userBkt, blockID) -// if err != nil { -// return blockStateUnknown, meta, nil, err -// } -// if v == nil { -// return blockUploadInProgress, meta, nil, err -// } -// if v.Error != "" { -// return blockValidationFailed, meta, v, err -// } -// if time.Since(time.UnixMilli(v.LastUpdate)) < validationHeartbeatTimeout { -// return blockValidationInProgress, meta, v, nil -// } -// return blockValidationStale, meta, v, nil -// } - -// func (c *MultitenantCompactor) loadUploadingMeta(ctx context.Context, userBkt objstore.Bucket, blockID ulid.ULID) (*block.Meta, error) { -// r, err := userBkt.Get(ctx, path.Join(blockID.String(), uploadingMetaFilename)) -// if err != nil { -// if userBkt.IsObjNotFoundErr(err) { -// return nil, nil -// } -// return nil, err -// } -// defer func() { _ = r.Close() }() - -// v := &block.Meta{} -// err = json.NewDecoder(r).Decode(v) -// if err != nil { -// return nil, err -// } - -// return v, nil -// } - -// func (c *MultitenantCompactor) loadValidation(ctx context.Context, userBkt objstore.Bucket, blockID ulid.ULID) (*validationFile, error) { -// r, err := userBkt.Get(ctx, path.Join(blockID.String(), validationFilename)) -// if err != nil { -// if userBkt.IsObjNotFoundErr(err) { -// return nil, nil -// } -// return nil, err -// } -// defer func() { _ = r.Close() }() - -// v := &validationFile{} -// err = json.NewDecoder(r).Decode(v) -// if err != nil { -// return nil, err -// } - -// return v, nil -// } - -// func (c *MultitenantCompactor) uploadValidationWithError(ctx context.Context, blockID ulid.ULID, -// userBkt objstore.Bucket, errorStr string, -// ) error { -// val := validationFile{ -// LastUpdate: time.Now().UnixMilli(), -// Error: errorStr, -// } -// dst := path.Join(blockID.String(), validationFilename) -// if err := marshalAndUploadToBucket(ctx, userBkt, dst, val); err != nil { -// return errors.Wrapf(err, "failed uploading %s to bucket", validationFilename) -// } -// return nil -// } - -// func (c *MultitenantCompactor) uploadValidation(ctx context.Context, blockID ulid.ULID, userBkt objstore.Bucket) error { -// return c.uploadValidationWithError(ctx, blockID, userBkt, "") -// } - -// func (c *MultitenantCompactor) periodicValidationUpdater(ctx context.Context, logger log.Logger, blockID ulid.ULID, userBkt objstore.Bucket, cancelFn func(), interval time.Duration) { -// ticker := time.NewTicker(interval) -// defer ticker.Stop() -// for { -// select { -// case <-ctx.Done(): -// return -// case <-ticker.C: -// if err := c.uploadValidation(ctx, blockID, userBkt); err != nil { -// level.Warn(logger).Log("msg", "error during periodic update of validation file", "err", err) -// cancelFn() -// return -// } -// } -// } -// } - -// func marshalAndUploadToBucket(ctx context.Context, bkt objstore.Bucket, pth string, val interface{}) error { -// buf, err := json.Marshal(val) -// if err != nil { -// return err -// } -// if err := bkt.Upload(ctx, pth, bytes.NewReader(buf)); err != nil { -// return err -// } -// return nil -// } diff --git a/pkg/compactor/block_upload_test.go b/pkg/compactor/block_upload_test.go deleted file mode 100644 index 87a93d4926..0000000000 --- a/pkg/compactor/block_upload_test.go +++ /dev/null @@ -1,2116 +0,0 @@ -// // SPDX-License-Identifier: AGPL-3.0-only -// // Provenance-includes-location: https://github.com/grafana/mimir/blob/main/pkg/compactor/block_upload_test.go -// // Provenance-includes-license: Apache-2.0 -// // Provenance-includes-copyright: The Cortex Authors. - -package compactor - -// import ( -// "bytes" -// "context" -// "encoding/json" -// "fmt" -// "io" -// "math" -// "net/http" -// "net/http/httptest" -// "net/url" -// "os" -// "path" -// "path/filepath" -// "strings" -// "sync" -// "testing" -// "time" - -// "github.com/go-kit/log" -// "github.com/gorilla/mux" -// "github.com/grafana/dskit/test" -// "github.com/grafana/dskit/user" -// "github.com/oklog/ulid" -// "github.com/pkg/errors" -// "github.com/prometheus/client_golang/prometheus" -// "github.com/prometheus/client_golang/prometheus/promauto" -// promtest "github.com/prometheus/client_golang/prometheus/testutil" -// "github.com/prometheus/prometheus/model/labels" -// "github.com/prometheus/prometheus/tsdb" -// "github.com/stretchr/testify/assert" -// "github.com/stretchr/testify/mock" -// "github.com/stretchr/testify/require" -// "github.com/thanos-io/objstore" - -// "github.com/grafana/mimir/pkg/storage/bucket" -// mimir_tsdb "github.com/grafana/mimir/pkg/storage/tsdb" -// "github.com/grafana/mimir/pkg/storage/tsdb/block" -// ) - -// func verifyUploadedMeta(t *testing.T, bkt *bucket.ClientMock, expMeta block.Meta) { -// var call mock.Call -// for _, c := range bkt.Calls { -// if c.Method == "Upload" { -// call = c -// break -// } -// } - -// rdr := call.Arguments[2].(io.Reader) -// var gotMeta block.Meta -// require.NoError(t, json.NewDecoder(rdr).Decode(&gotMeta)) -// assert.Equal(t, expMeta, gotMeta) -// } - -// // Test MultitenantCompactor.StartBlockUpload -// func TestMultitenantCompactor_StartBlockUpload(t *testing.T) { -// const tenantID = "test" -// const blockID = "01G3FZ0JWJYJC0ZM6Y9778P6KD" -// bULID := ulid.MustParse(blockID) -// now := time.Now().UnixMilli() -// validMeta := block.Meta{ -// BlockMeta: tsdb.BlockMeta{ -// ULID: bULID, -// Version: block.TSDBVersion1, -// MinTime: now - 1000, -// MaxTime: now, -// }, -// Thanos: block.ThanosMeta{ -// Labels: map[string]string{ -// mimir_tsdb.CompactorShardIDExternalLabel: "1_of_3", -// }, -// Files: []block.File{ -// { -// RelPath: block.MetaFilename, -// }, -// { -// RelPath: "index", -// SizeBytes: 1, -// }, -// { -// RelPath: "chunks/000001", -// SizeBytes: 1024, -// }, -// }, -// }, -// } - -// metaPath := path.Join(tenantID, blockID, block.MetaFilename) -// uploadingMetaPath := path.Join(tenantID, blockID, fmt.Sprintf("uploading-%s", block.MetaFilename)) - -// setUpPartialBlock := func(bkt *bucket.ClientMock) { -// bkt.MockExists(path.Join(tenantID, blockID, block.MetaFilename), false, nil) -// setUpGet(bkt, path.Join(tenantID, blockID, uploadingMetaFilename), nil, bucket.ErrObjectDoesNotExist) -// } -// setUpUpload := func(bkt *bucket.ClientMock) { -// setUpPartialBlock(bkt) -// bkt.MockUpload(uploadingMetaPath, nil) -// } - -// verifyUpload := func(t *testing.T, bkt *bucket.ClientMock, labels map[string]string) { -// t.Helper() - -// expMeta := validMeta -// expMeta.Compaction.Parents = nil -// expMeta.Compaction.Sources = []ulid.ULID{expMeta.ULID} -// expMeta.Thanos.Source = "upload" -// expMeta.Thanos.Labels = labels -// verifyUploadedMeta(t, bkt, expMeta) -// } - -// testCases := []struct { -// name string -// tenantID string -// blockID string -// body string -// meta *block.Meta -// retention time.Duration -// disableBlockUpload bool -// expBadRequest string -// expConflict string -// expUnprocessableEntity string -// expEntityTooLarge string -// expInternalServerError bool -// setUpBucketMock func(bkt *bucket.ClientMock) -// verifyUpload func(*testing.T, *bucket.ClientMock) -// maxBlockUploadSizeBytes int64 -// }{ -// { -// name: "missing tenant ID", -// tenantID: "", -// blockID: blockID, -// expBadRequest: "invalid tenant ID", -// }, -// { -// name: "missing block ID", -// tenantID: tenantID, -// blockID: "", -// expBadRequest: "invalid block ID", -// }, -// { -// name: "invalid block ID", -// tenantID: tenantID, -// blockID: "1234", -// expBadRequest: "invalid block ID", -// }, -// { -// name: "missing body", -// tenantID: tenantID, -// blockID: blockID, -// expBadRequest: "malformed request body", -// setUpBucketMock: setUpPartialBlock, -// }, -// { -// name: "malformed body", -// tenantID: tenantID, -// blockID: blockID, -// setUpBucketMock: setUpPartialBlock, -// body: "{", -// expBadRequest: "malformed request body", -// }, -// { -// name: "invalid file path", -// tenantID: tenantID, -// blockID: blockID, -// setUpBucketMock: setUpPartialBlock, -// meta: &block.Meta{ -// Thanos: block.ThanosMeta{ -// Files: []block.File{ -// { -// RelPath: "chunks/invalid-file", -// SizeBytes: 1024, -// }, -// }, -// }, -// }, -// expBadRequest: "file with invalid path: chunks/invalid-file", -// }, -// { -// name: "contains downsampled data", -// tenantID: tenantID, -// blockID: blockID, -// setUpBucketMock: setUpPartialBlock, -// meta: &block.Meta{ -// Thanos: block.ThanosMeta{ -// Downsample: block.ThanosDownsample{ -// Resolution: 1000, -// }, -// Files: []block.File{ -// { -// RelPath: block.MetaFilename, -// }, -// { -// RelPath: "index", -// SizeBytes: 1, -// }, -// { -// RelPath: "chunks/000001", -// }, -// }, -// }, -// }, -// expBadRequest: "block contains downsampled data", -// }, -// { -// name: "missing file size", -// tenantID: tenantID, -// blockID: blockID, -// setUpBucketMock: setUpPartialBlock, -// meta: &block.Meta{ -// Thanos: block.ThanosMeta{ -// Files: []block.File{ -// { -// RelPath: block.MetaFilename, -// }, -// { -// RelPath: "index", -// SizeBytes: 1, -// }, -// { -// RelPath: "chunks/000001", -// }, -// }, -// }, -// }, -// expBadRequest: "file with invalid size: chunks/000001", -// }, -// { -// name: "invalid minTime", -// tenantID: tenantID, -// blockID: blockID, -// setUpBucketMock: setUpPartialBlock, -// meta: &block.Meta{ -// BlockMeta: tsdb.BlockMeta{ -// ULID: bULID, -// Version: block.TSDBVersion1, -// MinTime: -1, -// MaxTime: 0, -// }, -// }, -// expBadRequest: "invalid minTime/maxTime: minTime=-1, maxTime=0", -// }, -// { -// name: "invalid maxTime", -// tenantID: tenantID, -// blockID: blockID, -// setUpBucketMock: setUpPartialBlock, -// meta: &block.Meta{ -// BlockMeta: tsdb.BlockMeta{ -// ULID: bULID, -// Version: block.TSDBVersion1, -// MinTime: 0, -// MaxTime: -1, -// }, -// }, -// expBadRequest: "invalid minTime/maxTime: minTime=0, maxTime=-1", -// }, -// { -// name: "maxTime before minTime", -// tenantID: tenantID, -// blockID: blockID, -// setUpBucketMock: setUpPartialBlock, -// meta: &block.Meta{ -// BlockMeta: tsdb.BlockMeta{ -// ULID: bULID, -// Version: block.TSDBVersion1, -// MinTime: 1, -// MaxTime: 0, -// }, -// }, -// expBadRequest: "invalid minTime/maxTime: minTime=1, maxTime=0", -// }, -// { -// name: "block before retention period", -// tenantID: tenantID, -// blockID: blockID, -// retention: 10 * time.Second, -// setUpBucketMock: setUpPartialBlock, -// meta: &block.Meta{ -// BlockMeta: tsdb.BlockMeta{ -// ULID: bULID, -// Version: block.TSDBVersion1, -// MinTime: 0, -// MaxTime: 1000, -// }, -// }, -// expUnprocessableEntity: "block max time (1970-01-01 00:00:01 +0000 UTC) older than retention period", -// }, -// { -// name: "invalid version", -// tenantID: tenantID, -// blockID: blockID, -// setUpBucketMock: setUpPartialBlock, -// meta: &block.Meta{ -// BlockMeta: tsdb.BlockMeta{ -// ULID: bULID, -// Version: 0, -// }, -// }, -// expBadRequest: fmt.Sprintf("version must be %d", block.TSDBVersion1), -// }, -// { -// name: "ignore retention period if == 0", -// tenantID: tenantID, -// blockID: blockID, -// retention: 0, -// setUpBucketMock: setUpUpload, -// meta: &block.Meta{ -// BlockMeta: tsdb.BlockMeta{ -// ULID: bULID, -// Version: block.TSDBVersion1, -// MinTime: 0, -// MaxTime: 1000, -// }, -// Thanos: block.ThanosMeta{ -// Labels: map[string]string{ -// mimir_tsdb.CompactorShardIDExternalLabel: "1_of_3", -// }, -// Files: []block.File{ -// { -// RelPath: block.MetaFilename, -// }, -// { -// RelPath: "index", -// SizeBytes: 1, -// }, -// { -// RelPath: "chunks/000001", -// SizeBytes: 1024, -// }, -// }, -// }, -// }, -// }, -// { -// name: "ignore retention period if < 0", -// tenantID: tenantID, -// blockID: blockID, -// retention: -1, -// setUpBucketMock: setUpUpload, -// meta: &block.Meta{ -// BlockMeta: tsdb.BlockMeta{ -// ULID: bULID, -// Version: block.TSDBVersion1, -// MinTime: 0, -// MaxTime: 1000, -// }, -// Thanos: block.ThanosMeta{ -// Labels: map[string]string{ -// mimir_tsdb.CompactorShardIDExternalLabel: "1_of_3", -// }, -// Files: []block.File{ -// { -// RelPath: block.MetaFilename, -// }, -// { -// RelPath: "index", -// SizeBytes: 1, -// }, -// { -// RelPath: "chunks/000001", -// SizeBytes: 1024, -// }, -// }, -// }, -// }, -// }, -// { -// name: "invalid compactor shard ID label", -// tenantID: tenantID, -// blockID: blockID, -// setUpBucketMock: setUpPartialBlock, -// meta: &block.Meta{ -// BlockMeta: tsdb.BlockMeta{ -// ULID: bULID, -// Version: block.TSDBVersion1, -// }, -// Thanos: block.ThanosMeta{ -// Labels: map[string]string{ -// mimir_tsdb.CompactorShardIDExternalLabel: "test", -// }, -// }, -// }, -// expBadRequest: fmt.Sprintf(`invalid %s external label: "test"`, mimir_tsdb.CompactorShardIDExternalLabel), -// }, -// { -// name: "failure checking for complete block", -// tenantID: tenantID, -// blockID: blockID, -// setUpBucketMock: func(bkt *bucket.ClientMock) { -// bkt.MockExists(path.Join(tenantID, blockID, block.MetaFilename), false, fmt.Errorf("test")) -// }, -// expInternalServerError: true, -// }, -// { -// name: "complete block already exists", -// tenantID: tenantID, -// blockID: blockID, -// setUpBucketMock: func(bkt *bucket.ClientMock) { -// bkt.MockExists(path.Join(tenantID, blockID, block.MetaFilename), true, nil) -// }, -// expConflict: "block already exists", -// }, -// { -// name: "failure uploading meta file", -// tenantID: tenantID, -// blockID: blockID, -// setUpBucketMock: func(bkt *bucket.ClientMock) { -// setUpPartialBlock(bkt) -// bkt.MockUpload(uploadingMetaPath, fmt.Errorf("test")) -// }, -// meta: &validMeta, -// expInternalServerError: true, -// }, -// { -// name: "too large of a request body", -// tenantID: tenantID, -// blockID: blockID, -// setUpBucketMock: setUpPartialBlock, -// body: strings.Repeat("A", maximumMetaSizeBytes+1), -// expEntityTooLarge: fmt.Sprintf("The block metadata was too large (maximum size allowed is %d bytes)", maximumMetaSizeBytes), -// }, -// { -// name: "block upload disabled", -// tenantID: tenantID, -// blockID: blockID, -// disableBlockUpload: true, -// expBadRequest: "block upload is disabled", -// }, -// { -// name: "max block size exceeded", -// tenantID: tenantID, -// blockID: blockID, -// setUpBucketMock: setUpPartialBlock, -// meta: &validMeta, -// maxBlockUploadSizeBytes: 1, -// expBadRequest: fmt.Sprintf(maxBlockUploadSizeBytesFormat, 1), -// }, -// { -// name: "valid request", -// tenantID: tenantID, -// blockID: blockID, -// setUpBucketMock: setUpUpload, -// meta: &validMeta, -// verifyUpload: func(t *testing.T, bkt *bucket.ClientMock) { -// verifyUpload(t, bkt, map[string]string{ -// mimir_tsdb.CompactorShardIDExternalLabel: "1_of_3", -// }) -// }, -// }, -// { -// name: "valid request with empty compactor shard ID label", -// tenantID: tenantID, -// blockID: blockID, -// setUpBucketMock: setUpUpload, -// meta: &block.Meta{ -// BlockMeta: tsdb.BlockMeta{ -// ULID: bULID, -// Version: block.TSDBVersion1, -// MinTime: now - 1000, -// MaxTime: now, -// }, -// Thanos: block.ThanosMeta{ -// Labels: map[string]string{ -// mimir_tsdb.CompactorShardIDExternalLabel: "", -// }, -// Files: []block.File{ -// { -// RelPath: block.MetaFilename, -// }, -// { -// RelPath: "index", -// SizeBytes: 1, -// }, -// { -// RelPath: "chunks/000001", -// SizeBytes: 1024, -// }, -// }, -// }, -// }, -// verifyUpload: func(t *testing.T, bkt *bucket.ClientMock) { -// verifyUpload(t, bkt, map[string]string{}) -// }, -// }, -// { -// name: "valid request without compactor shard ID label", -// tenantID: tenantID, -// blockID: blockID, -// setUpBucketMock: setUpUpload, -// meta: &block.Meta{ -// BlockMeta: tsdb.BlockMeta{ -// ULID: bULID, -// Version: block.TSDBVersion1, -// MinTime: now - 1000, -// MaxTime: now, -// }, -// Thanos: block.ThanosMeta{ -// Files: []block.File{ -// { -// RelPath: block.MetaFilename, -// }, -// { -// RelPath: "index", -// SizeBytes: 1, -// }, -// { -// RelPath: "chunks/000001", -// SizeBytes: 1024, -// }, -// }, -// }, -// }, -// verifyUpload: func(t *testing.T, bkt *bucket.ClientMock) { -// verifyUpload(t, bkt, nil) -// }, -// }, -// { -// name: "valid request with different block ID in meta file", -// tenantID: tenantID, -// blockID: blockID, -// setUpBucketMock: setUpUpload, -// meta: &block.Meta{ -// BlockMeta: tsdb.BlockMeta{ -// ULID: ulid.MustParse("11A2FZ0JWJYJC0ZM6Y9778P6KD"), -// Version: block.TSDBVersion1, -// MinTime: now - 1000, -// MaxTime: now, -// }, -// Thanos: block.ThanosMeta{ -// Files: []block.File{ -// { -// RelPath: block.MetaFilename, -// }, -// { -// RelPath: "index", -// SizeBytes: 1, -// }, -// { -// RelPath: "chunks/000001", -// SizeBytes: 1024, -// }, -// }, -// }, -// }, -// verifyUpload: func(t *testing.T, bkt *bucket.ClientMock) { -// verifyUpload(t, bkt, nil) -// }, -// }, -// } -// for _, tc := range testCases { -// t.Run(tc.name, func(t *testing.T) { -// var bkt bucket.ClientMock -// if tc.setUpBucketMock != nil { -// tc.setUpBucketMock(&bkt) -// } - -// cfgProvider := newMockConfigProvider() -// cfgProvider.userRetentionPeriods[tenantID] = tc.retention -// cfgProvider.blockUploadEnabled[tenantID] = !tc.disableBlockUpload -// cfgProvider.blockUploadMaxBlockSizeBytes[tenantID] = tc.maxBlockUploadSizeBytes -// c := &MultitenantCompactor{ -// logger: log.NewNopLogger(), -// bucketClient: &bkt, -// cfgProvider: cfgProvider, -// } -// var rdr io.Reader -// if tc.body != "" { -// rdr = strings.NewReader(tc.body) -// } else if tc.meta != nil { -// buf := bytes.NewBuffer(nil) -// require.NoError(t, json.NewEncoder(buf).Encode(tc.meta)) -// rdr = buf -// } -// r := httptest.NewRequest(http.MethodPost, fmt.Sprintf("/api/v1/upload/block/%s/start", tc.blockID), rdr) -// if tc.tenantID != "" { -// r = r.WithContext(user.InjectOrgID(r.Context(), tc.tenantID)) -// } -// if tc.blockID != "" { -// r = mux.SetURLVars(r, map[string]string{"block": tc.blockID}) -// } -// w := httptest.NewRecorder() -// c.StartBlockUpload(w, r) - -// resp := w.Result() -// body, err := io.ReadAll(resp.Body) -// require.NoError(t, err) - -// switch { -// case tc.expInternalServerError: -// assert.Equal(t, http.StatusInternalServerError, resp.StatusCode) -// assert.Regexp(t, "internal server error \\(id [0-9a-f]{16}\\)\n", string(body)) -// case tc.expBadRequest != "": -// assert.Equal(t, http.StatusBadRequest, resp.StatusCode) -// assert.Equal(t, fmt.Sprintf("%s\n", tc.expBadRequest), string(body)) -// case tc.expConflict != "": -// assert.Equal(t, http.StatusConflict, resp.StatusCode) -// assert.Equal(t, fmt.Sprintf("%s\n", tc.expConflict), string(body)) -// case tc.expUnprocessableEntity != "": -// assert.Equal(t, http.StatusUnprocessableEntity, resp.StatusCode) -// assert.Equal(t, fmt.Sprintf("%s\n", tc.expUnprocessableEntity), string(body)) -// case tc.expEntityTooLarge != "": -// assert.Equal(t, http.StatusRequestEntityTooLarge, resp.StatusCode) -// assert.Equal(t, fmt.Sprintf("%s\n", tc.expEntityTooLarge), string(body)) -// default: -// assert.Equal(t, http.StatusOK, resp.StatusCode) -// assert.Empty(t, string(body)) -// } - -// bkt.AssertExpectations(t) - -// if tc.verifyUpload != nil { -// tc.verifyUpload(t, &bkt) -// } -// }) -// } - -// downloadMeta := func(t *testing.T, bkt *objstore.InMemBucket, pth string) block.Meta { -// t.Helper() - -// ctx := context.Background() -// rdr, err := bkt.Get(ctx, pth) -// require.NoError(t, err) -// t.Cleanup(func() { -// _ = rdr.Close() -// }) -// var gotMeta block.Meta -// require.NoError(t, json.NewDecoder(rdr).Decode(&gotMeta)) -// return gotMeta -// } - -// // Additional test cases using an in-memory bucket for state testing -// extraCases := []struct { -// name string -// setUp func(*testing.T, *objstore.InMemBucket) block.Meta -// verifyBucket func(*testing.T, *objstore.InMemBucket) -// expBadRequest string -// expConflict string -// }{ -// { -// name: "valid request when both in-flight meta file and complete meta file exist in object storage", -// setUp: func(t *testing.T, bkt *objstore.InMemBucket) block.Meta { -// marshalAndUploadJSON(t, bkt, uploadingMetaPath, validMeta) -// marshalAndUploadJSON(t, bkt, metaPath, validMeta) -// return validMeta -// }, -// verifyBucket: func(t *testing.T, bkt *objstore.InMemBucket) { -// assert.Equal(t, validMeta, downloadMeta(t, bkt, uploadingMetaPath)) -// assert.Equal(t, validMeta, downloadMeta(t, bkt, metaPath)) -// }, -// expConflict: "block already exists", -// }, -// { -// name: "invalid request when in-flight meta file exists in object storage", -// setUp: func(t *testing.T, bkt *objstore.InMemBucket) block.Meta { -// marshalAndUploadJSON(t, bkt, uploadingMetaPath, validMeta) - -// meta := validMeta -// // Invalid version -// meta.Version = 0 -// return meta -// }, -// verifyBucket: func(t *testing.T, bkt *objstore.InMemBucket) { -// assert.Equal(t, validMeta, downloadMeta(t, bkt, uploadingMetaPath)) -// }, -// expBadRequest: fmt.Sprintf("version must be %d", block.TSDBVersion1), -// }, -// { -// name: "valid request when same in-flight meta file exists in object storage", -// setUp: func(t *testing.T, bkt *objstore.InMemBucket) block.Meta { -// marshalAndUploadJSON(t, bkt, uploadingMetaPath, validMeta) -// return validMeta -// }, -// verifyBucket: func(t *testing.T, bkt *objstore.InMemBucket) { -// expMeta := validMeta -// expMeta.Compaction.Sources = []ulid.ULID{expMeta.ULID} -// expMeta.Thanos.Source = "upload" -// assert.Equal(t, expMeta, downloadMeta(t, bkt, uploadingMetaPath)) -// }, -// }, -// { -// name: "valid request when different in-flight meta file exists in object storage", -// setUp: func(t *testing.T, bkt *objstore.InMemBucket) block.Meta { -// meta := validMeta -// meta.MinTime -= 1000 -// meta.MaxTime -= 1000 -// marshalAndUploadJSON(t, bkt, uploadingMetaPath, meta) - -// // Return meta file that differs from the one in bucket -// return validMeta -// }, -// verifyBucket: func(t *testing.T, bkt *objstore.InMemBucket) { -// expMeta := validMeta -// expMeta.Compaction.Sources = []ulid.ULID{expMeta.ULID} -// expMeta.Thanos.Source = "upload" -// assert.Equal(t, expMeta, downloadMeta(t, bkt, uploadingMetaPath)) -// }, -// }, -// } -// for _, tc := range extraCases { -// t.Run(tc.name, func(t *testing.T) { -// bkt := objstore.NewInMemBucket() -// meta := tc.setUp(t, bkt) -// metaJSON, err := json.Marshal(meta) -// require.NoError(t, err) - -// cfgProvider := newMockConfigProvider() -// cfgProvider.blockUploadEnabled[tenantID] = true -// c := &MultitenantCompactor{ -// logger: log.NewNopLogger(), -// bucketClient: bkt, -// cfgProvider: cfgProvider, -// } -// r := httptest.NewRequest(http.MethodPost, fmt.Sprintf("/api/v1/upload/block/%s/start", blockID), bytes.NewReader(metaJSON)) -// r = r.WithContext(user.InjectOrgID(r.Context(), tenantID)) -// r = mux.SetURLVars(r, map[string]string{"block": blockID}) -// w := httptest.NewRecorder() -// c.StartBlockUpload(w, r) - -// resp := w.Result() -// body, err := io.ReadAll(resp.Body) -// require.NoError(t, err) -// switch { -// case tc.expBadRequest != "": -// assert.Equal(t, http.StatusBadRequest, resp.StatusCode) -// assert.Equal(t, fmt.Sprintf("%s\n", tc.expBadRequest), string(body)) -// case tc.expConflict != "": -// assert.Equal(t, http.StatusConflict, resp.StatusCode) -// assert.Equal(t, fmt.Sprintf("%s\n", tc.expConflict), string(body)) -// default: -// assert.Equal(t, http.StatusOK, resp.StatusCode) -// assert.Empty(t, string(body)) -// } -// }) -// } -// } - -// // Test MultitenantCompactor.UploadBlockFile -// func TestMultitenantCompactor_UploadBlockFile(t *testing.T) { -// const tenantID = "test" -// const blockID = "01G3FZ0JWJYJC0ZM6Y9778P6KD" -// uploadingMetaFilename := fmt.Sprintf("uploading-%s", block.MetaFilename) -// uploadingMetaPath := path.Join(tenantID, blockID, uploadingMetaFilename) -// metaPath := path.Join(tenantID, blockID, block.MetaFilename) - -// chunkBodyContent := "content" -// validMeta := block.Meta{ -// BlockMeta: tsdb.BlockMeta{ -// ULID: ulid.MustParse(blockID), -// }, -// Thanos: block.ThanosMeta{ -// Labels: map[string]string{ -// mimir_tsdb.CompactorShardIDExternalLabel: "1_of_3", -// }, -// Files: []block.File{ -// { -// RelPath: "index", -// SizeBytes: 1, -// }, -// { -// RelPath: "chunks/000001", -// SizeBytes: int64(len(chunkBodyContent)), -// }, -// }, -// }, -// } - -// setupFnForValidRequest := func(bkt *bucket.ClientMock) { -// bkt.MockExists(metaPath, false, nil) - -// b, err := json.Marshal(validMeta) -// setUpGet(bkt, path.Join(tenantID, blockID, uploadingMetaFilename), b, err) -// setUpGet(bkt, path.Join(tenantID, blockID, validationFilename), nil, bucket.ErrObjectDoesNotExist) - -// bkt.MockUpload(path.Join(tenantID, blockID, "chunks/000001"), nil) -// } - -// verifyFuncForValidRequest := func(t *testing.T, bkt *bucket.ClientMock, expContent string) { -// var call mock.Call -// for _, c := range bkt.Calls { -// if c.Method == "Upload" { -// call = c -// break -// } -// } - -// rdr := call.Arguments[2].(io.Reader) -// got, err := io.ReadAll(rdr) -// require.NoError(t, err) -// assert.Equal(t, []byte(expContent), got) -// } - -// testCases := []struct { -// name string -// tenantID string -// blockID string -// path string -// body string -// unknownContentLength bool -// disableBlockUpload bool -// expBadRequest string -// expConflict string -// expNotFound string -// expInternalServerError bool -// setUpBucketMock func(bkt *bucket.ClientMock) -// verifyUpload func(*testing.T, *bucket.ClientMock, string) -// }{ -// { -// name: "without tenant ID", -// blockID: blockID, -// path: "chunks/000001", -// expBadRequest: "invalid tenant ID", -// }, -// { -// name: "without block ID", -// tenantID: tenantID, -// path: "chunks/000001", -// expBadRequest: "invalid block ID", -// }, -// { -// name: "invalid block ID", -// tenantID: tenantID, -// blockID: "1234", -// path: "chunks/000001", -// expBadRequest: "invalid block ID", -// }, -// { -// name: "without path", -// tenantID: tenantID, -// blockID: blockID, -// expBadRequest: "missing or invalid file path", -// }, -// { -// name: "invalid path", -// tenantID: tenantID, -// blockID: blockID, -// path: "../chunks/000001", -// expBadRequest: `invalid path: "../chunks/000001"`, -// }, -// { -// name: "empty file", -// tenantID: tenantID, -// blockID: blockID, -// path: "chunks/000001", -// expBadRequest: "file cannot be empty", -// }, -// { -// name: "attempt block metadata file", -// tenantID: tenantID, -// blockID: blockID, -// path: block.MetaFilename, -// body: "content", -// expBadRequest: fmt.Sprintf("%s is not allowed", block.MetaFilename), -// }, -// { -// name: "attempt in-flight block metadata file", -// tenantID: tenantID, -// blockID: blockID, -// path: uploadingMetaFilename, -// body: "content", -// expBadRequest: fmt.Sprintf("invalid path: %q", uploadingMetaFilename), -// }, -// { -// name: "block upload disabled", -// tenantID: tenantID, -// blockID: blockID, -// disableBlockUpload: true, -// path: "chunks/000001", -// expBadRequest: "block upload is disabled", -// }, -// { -// name: "complete block already exists", -// tenantID: tenantID, -// blockID: blockID, -// path: "chunks/000001", -// body: "content", -// setUpBucketMock: func(bkt *bucket.ClientMock) { -// bkt.MockExists(metaPath, true, nil) -// }, -// expConflict: "block already exists", -// }, -// { -// name: "failure checking for complete block", -// tenantID: tenantID, -// blockID: blockID, -// path: "chunks/000001", -// body: chunkBodyContent, -// setUpBucketMock: func(bkt *bucket.ClientMock) { -// bkt.MockExists(metaPath, false, fmt.Errorf("test")) -// }, -// expInternalServerError: true, -// }, -// { -// name: "failure checking for in-flight meta file", -// tenantID: tenantID, -// blockID: blockID, -// path: "chunks/000001", -// body: chunkBodyContent, -// setUpBucketMock: func(bkt *bucket.ClientMock) { -// bkt.MockExists(metaPath, false, nil) -// setUpGet(bkt, path.Join(tenantID, blockID, uploadingMetaFilename), nil, fmt.Errorf("test")) -// }, -// expInternalServerError: true, -// }, -// { -// name: "missing in-flight meta file", -// tenantID: tenantID, -// blockID: blockID, -// path: "chunks/000001", -// body: chunkBodyContent, -// setUpBucketMock: func(bkt *bucket.ClientMock) { -// bkt.MockExists(metaPath, false, nil) -// setUpGet(bkt, path.Join(tenantID, blockID, uploadingMetaFilename), nil, bucket.ErrObjectDoesNotExist) -// }, -// expNotFound: "block upload not started", -// }, -// { -// name: "file upload fails", -// tenantID: tenantID, -// blockID: blockID, -// path: "chunks/000001", -// body: chunkBodyContent, -// setUpBucketMock: func(bkt *bucket.ClientMock) { -// bkt.MockExists(metaPath, false, nil) - -// b, err := json.Marshal(validMeta) -// setUpGet(bkt, path.Join(tenantID, blockID, uploadingMetaFilename), b, err) -// setUpGet(bkt, path.Join(tenantID, blockID, validationFilename), nil, bucket.ErrObjectDoesNotExist) - -// bkt.MockUpload(path.Join(tenantID, blockID, "chunks/000001"), fmt.Errorf("test")) -// }, -// expInternalServerError: true, -// }, -// { -// name: "invalid file size", -// tenantID: tenantID, -// blockID: blockID, -// path: "chunks/000001", -// body: chunkBodyContent + chunkBodyContent, -// setUpBucketMock: func(bkt *bucket.ClientMock) { -// bkt.MockExists(metaPath, false, nil) - -// b, err := json.Marshal(validMeta) -// setUpGet(bkt, path.Join(tenantID, blockID, uploadingMetaFilename), b, err) -// setUpGet(bkt, path.Join(tenantID, blockID, validationFilename), nil, bucket.ErrObjectDoesNotExist) -// }, -// expBadRequest: "file size doesn't match meta.json", -// }, -// { -// name: "unexpected file", -// tenantID: tenantID, -// blockID: blockID, -// path: "chunks/111111", -// body: chunkBodyContent, -// setUpBucketMock: func(bkt *bucket.ClientMock) { -// bkt.MockExists(metaPath, false, nil) - -// b, err := json.Marshal(validMeta) -// setUpGet(bkt, path.Join(tenantID, blockID, uploadingMetaFilename), b, err) -// setUpGet(bkt, path.Join(tenantID, blockID, validationFilename), nil, bucket.ErrObjectDoesNotExist) -// }, -// expBadRequest: "unexpected file", -// }, -// { -// name: "valid request", -// tenantID: tenantID, -// blockID: blockID, -// path: "chunks/000001", -// body: chunkBodyContent, -// setUpBucketMock: setupFnForValidRequest, -// verifyUpload: verifyFuncForValidRequest, -// }, -// { -// name: "valid request, with unknown content-length", -// tenantID: tenantID, -// blockID: blockID, -// path: "chunks/000001", -// body: chunkBodyContent, -// unknownContentLength: true, -// setUpBucketMock: setupFnForValidRequest, -// verifyUpload: verifyFuncForValidRequest, -// }, -// } -// for _, tc := range testCases { -// t.Run(tc.name, func(t *testing.T) { -// var bkt bucket.ClientMock -// if tc.setUpBucketMock != nil { -// tc.setUpBucketMock(&bkt) -// } - -// cfgProvider := newMockConfigProvider() -// cfgProvider.blockUploadEnabled[tc.tenantID] = !tc.disableBlockUpload -// c := &MultitenantCompactor{ -// logger: log.NewNopLogger(), -// bucketClient: &bkt, -// cfgProvider: cfgProvider, -// } -// var rdr io.Reader -// if tc.body != "" { -// rdr = strings.NewReader(tc.body) -// } -// r := httptest.NewRequest(http.MethodPost, fmt.Sprintf("/api/v1/upload/block/%s/files?path=%s", blockID, url.QueryEscape(tc.path)), rdr) -// if tc.tenantID != "" { -// r = r.WithContext(user.InjectOrgID(r.Context(), tenantID)) -// } -// if tc.blockID != "" { -// r = mux.SetURLVars(r, map[string]string{"block": tc.blockID}) -// } -// if tc.body != "" { -// r.ContentLength = int64(len(tc.body)) -// if tc.unknownContentLength { -// r.ContentLength = -1 -// } -// } -// w := httptest.NewRecorder() -// c.UploadBlockFile(w, r) - -// resp := w.Result() -// body, err := io.ReadAll(resp.Body) -// require.NoError(t, err) - -// switch { -// case tc.expBadRequest != "": -// assert.Equal(t, http.StatusBadRequest, resp.StatusCode) -// assert.Equal(t, fmt.Sprintf("%s\n", tc.expBadRequest), string(body)) -// case tc.expConflict != "": -// assert.Equal(t, http.StatusConflict, resp.StatusCode) -// assert.Equal(t, fmt.Sprintf("%s\n", tc.expConflict), string(body)) -// case tc.expNotFound != "": -// assert.Equal(t, http.StatusNotFound, resp.StatusCode) -// assert.Equal(t, fmt.Sprintf("%s\n", tc.expNotFound), string(body)) -// case tc.expInternalServerError: -// assert.Equal(t, http.StatusInternalServerError, resp.StatusCode) -// assert.Regexp(t, "internal server error \\(id [0-9a-f]{16}\\)\n", string(body)) -// default: -// assert.Equal(t, http.StatusOK, resp.StatusCode) -// assert.Empty(t, string(body)) -// } - -// bkt.AssertExpectations(t) - -// if tc.verifyUpload != nil { -// tc.verifyUpload(t, &bkt, tc.body) -// } -// }) -// } - -// type file struct { -// path string -// content string -// } - -// // Additional test cases using an in-memory bucket for state testing -// extraCases := []struct { -// name string -// files []file -// setUpBucket func(*testing.T, *objstore.InMemBucket) -// verifyBucket func(*testing.T, *objstore.InMemBucket, []file) -// }{ -// { -// name: "multiple sequential uploads of same file", -// files: []file{ -// { -// path: "chunks/000001", -// content: strings.Repeat("a", len(chunkBodyContent)), -// }, -// { -// path: "chunks/000001", -// content: strings.Repeat("b", len(chunkBodyContent)), -// }, -// }, -// setUpBucket: func(t *testing.T, bkt *objstore.InMemBucket) { -// marshalAndUploadJSON(t, bkt, uploadingMetaPath, validMeta) -// }, -// verifyBucket: func(t *testing.T, bkt *objstore.InMemBucket, files []file) { -// t.Helper() - -// ctx := context.Background() -// rdr, err := bkt.Get(ctx, path.Join(tenantID, blockID, files[1].path)) -// require.NoError(t, err) -// t.Cleanup(func() { -// _ = rdr.Close() -// }) - -// content, err := io.ReadAll(rdr) -// require.NoError(t, err) -// assert.Equal(t, files[1].content, string(content)) -// }, -// }, -// } -// for _, tc := range extraCases { -// t.Run(tc.name, func(t *testing.T) { -// bkt := objstore.NewInMemBucket() -// tc.setUpBucket(t, bkt) -// cfgProvider := newMockConfigProvider() -// cfgProvider.blockUploadEnabled[tenantID] = true -// c := &MultitenantCompactor{ -// logger: log.NewNopLogger(), -// bucketClient: bkt, -// cfgProvider: cfgProvider, -// } - -// for _, f := range tc.files { -// rdr := strings.NewReader(f.content) -// r := httptest.NewRequest(http.MethodPost, fmt.Sprintf("/api/v1/upload/block/%s/files?path=%s", blockID, url.QueryEscape(f.path)), rdr) -// urlVars := map[string]string{ -// "block": blockID, -// } -// r = mux.SetURLVars(r, urlVars) -// r = r.WithContext(user.InjectOrgID(r.Context(), tenantID)) -// w := httptest.NewRecorder() -// c.UploadBlockFile(w, r) - -// resp := w.Result() -// body, err := io.ReadAll(resp.Body) -// require.NoError(t, err) -// require.Equal(t, http.StatusOK, resp.StatusCode) -// require.Empty(t, body) -// } - -// tc.verifyBucket(t, bkt, tc.files) -// }) -// } -// } - -// func setUpGet(bkt *bucket.ClientMock, pth string, content []byte, err error) { -// bkt.On("Get", mock.Anything, pth).Return(func(_ context.Context, _ string) (io.ReadCloser, error) { -// return io.NopCloser(bytes.NewReader(content)), err -// }) -// } - -// // Test MultitenantCompactor.FinishBlockUpload -// func TestMultitenantCompactor_FinishBlockUpload(t *testing.T) { -// const tenantID = "test" -// const blockID = "01G3FZ0JWJYJC0ZM6Y9778P6KD" -// uploadingMetaPath := path.Join(tenantID, blockID, uploadingMetaFilename) -// metaPath := path.Join(tenantID, blockID, block.MetaFilename) -// injectedError := fmt.Errorf("injected error") -// validMeta := block.Meta{ -// BlockMeta: tsdb.BlockMeta{ -// Version: block.TSDBVersion1, -// ULID: ulid.MustParse(blockID), -// }, -// Thanos: block.ThanosMeta{ -// Labels: map[string]string{ -// mimir_tsdb.CompactorShardIDExternalLabel: "1_of_3", -// }, -// Files: []block.File{ -// { -// RelPath: "index", -// SizeBytes: 1, -// }, -// { -// RelPath: "chunks/000001", -// SizeBytes: 2, -// }, -// }, -// }, -// } - -// validSetup := func(t *testing.T, bkt objstore.Bucket) { -// err := marshalAndUploadToBucket(context.Background(), bkt, uploadingMetaPath, validMeta) -// require.NoError(t, err) -// for _, file := range validMeta.Thanos.Files { -// content := bytes.NewReader(make([]byte, file.SizeBytes)) -// err = bkt.Upload(context.Background(), path.Join(tenantID, blockID, file.RelPath), content) -// require.NoError(t, err) -// } -// } - -// testCases := []struct { -// name string -// tenantID string -// blockID string -// setUpBucket func(*testing.T, objstore.Bucket) -// errorInjector func(op bucket.Operation, name string) error -// disableBlockUpload bool -// enableValidation bool // should only be set to true for tests that fail before validation is started -// maxConcurrency int -// setConcurrency int64 -// expBadRequest string -// expConflict string -// expNotFound string -// expTooManyRequests bool -// expInternalServerError bool -// }{ -// { -// name: "without tenant ID", -// blockID: blockID, -// expBadRequest: "invalid tenant ID", -// }, -// { -// name: "without block ID", -// tenantID: tenantID, -// expBadRequest: "invalid block ID", -// }, -// { -// name: "invalid block ID", -// tenantID: tenantID, -// blockID: "1234", -// expBadRequest: "invalid block ID", -// }, -// { -// name: "block upload disabled", -// tenantID: tenantID, -// blockID: blockID, -// disableBlockUpload: true, -// expBadRequest: "block upload is disabled", -// }, -// { -// name: "complete block already exists", -// tenantID: tenantID, -// blockID: blockID, -// setUpBucket: func(t *testing.T, bkt objstore.Bucket) { -// err := marshalAndUploadToBucket(context.Background(), bkt, metaPath, validMeta) -// require.NoError(t, err) -// }, -// expConflict: "block already exists", -// }, -// { -// name: "checking for complete block fails", -// tenantID: tenantID, -// blockID: blockID, -// errorInjector: bucket.InjectErrorOn(bucket.OpExists, metaPath, injectedError), -// expInternalServerError: true, -// }, -// { -// name: "missing in-flight meta file", -// tenantID: tenantID, -// blockID: blockID, -// expNotFound: "block upload not started", -// }, -// { -// name: "downloading in-flight meta file fails", -// tenantID: tenantID, -// blockID: blockID, -// setUpBucket: func(t *testing.T, bkt objstore.Bucket) { -// err := marshalAndUploadToBucket(context.Background(), bkt, uploadingMetaPath, validMeta) -// require.NoError(t, err) -// }, -// errorInjector: bucket.InjectErrorOn(bucket.OpGet, uploadingMetaPath, injectedError), -// expInternalServerError: true, -// }, -// { -// name: "corrupt in-flight meta file", -// tenantID: tenantID, -// blockID: blockID, -// setUpBucket: func(t *testing.T, bkt objstore.Bucket) { -// err := bkt.Upload(context.Background(), uploadingMetaPath, bytes.NewReader([]byte("{"))) -// require.NoError(t, err) -// }, -// expInternalServerError: true, -// }, -// { -// name: "uploading meta file fails", -// tenantID: tenantID, -// blockID: blockID, -// setUpBucket: validSetup, -// errorInjector: bucket.InjectErrorOn(bucket.OpUpload, metaPath, injectedError), -// expInternalServerError: true, -// }, -// { -// name: "too many concurrent validations", -// tenantID: tenantID, -// blockID: blockID, -// setUpBucket: validSetup, -// enableValidation: true, -// maxConcurrency: 2, -// setConcurrency: 2, -// expTooManyRequests: true, -// }, -// } -// for _, tc := range testCases { -// t.Run(tc.name, func(t *testing.T) { -// bkt := objstore.NewInMemBucket() -// injectedBkt := bucket.ErrorInjectedBucketClient{ -// Bucket: bkt, -// Injector: tc.errorInjector, -// } -// if tc.setUpBucket != nil { -// tc.setUpBucket(t, bkt) -// } - -// cfgProvider := newMockConfigProvider() -// cfgProvider.blockUploadEnabled[tc.tenantID] = !tc.disableBlockUpload -// cfgProvider.blockUploadValidationEnabled[tc.tenantID] = tc.enableValidation -// c := &MultitenantCompactor{ -// logger: log.NewNopLogger(), -// bucketClient: &injectedBkt, -// cfgProvider: cfgProvider, -// } -// c.compactorCfg.MaxBlockUploadValidationConcurrency = tc.maxConcurrency -// if tc.setConcurrency > 0 { -// c.blockUploadValidations.Add(tc.setConcurrency) -// } - -// c.compactorCfg.DataDir = t.TempDir() - -// r := httptest.NewRequest(http.MethodPost, fmt.Sprintf("/api/v1/upload/block/%s/finish", tc.blockID), nil) -// if tc.tenantID != "" { -// r = r.WithContext(user.InjectOrgID(r.Context(), tenantID)) -// } -// if tc.blockID != "" { -// r = mux.SetURLVars(r, map[string]string{"block": tc.blockID}) -// } -// w := httptest.NewRecorder() -// c.FinishBlockUpload(w, r) - -// resp := w.Result() -// body, err := io.ReadAll(resp.Body) -// require.NoError(t, err) - -// switch { -// case tc.expBadRequest != "": -// assert.Equal(t, http.StatusBadRequest, resp.StatusCode) -// assert.Equal(t, fmt.Sprintf("%s\n", tc.expBadRequest), string(body)) -// case tc.expConflict != "": -// assert.Equal(t, http.StatusConflict, resp.StatusCode) -// assert.Equal(t, fmt.Sprintf("%s\n", tc.expConflict), string(body)) -// case tc.expNotFound != "": -// assert.Equal(t, http.StatusNotFound, resp.StatusCode) -// assert.Equal(t, fmt.Sprintf("%s\n", tc.expNotFound), string(body)) -// case tc.expInternalServerError: -// assert.Equal(t, http.StatusInternalServerError, resp.StatusCode) -// assert.Regexp(t, "internal server error \\(id [0-9a-f]{16}\\)\n", string(body)) -// case tc.expTooManyRequests: -// assert.Equal(t, http.StatusTooManyRequests, resp.StatusCode) -// assert.Equal(t, "too many block upload validations in progress, limit is 2\n", string(body)) -// default: -// assert.Equal(t, http.StatusOK, resp.StatusCode) -// assert.Empty(t, string(body)) -// exists, err := bkt.Exists(context.Background(), path.Join(tc.blockID, block.MetaFilename)) -// require.NoError(t, err) -// require.True(t, exists) -// } -// }) -// } -// } - -// func TestMultitenantCompactor_ValidateAndComplete(t *testing.T) { -// const tenantID = "test" -// const blockID = "01G3FZ0JWJYJC0ZM6Y9778P6KD" -// injectedError := fmt.Errorf("injected error") - -// uploadingMetaPath := path.Join(tenantID, blockID, uploadingMetaFilename) -// validationPath := path.Join(tenantID, blockID, validationFilename) -// metaPath := path.Join(tenantID, blockID, block.MetaFilename) - -// validationSucceeds := func(_ context.Context) error { return nil } - -// testCases := []struct { -// name string -// errorInjector func(op bucket.Operation, name string) error -// validation func(context.Context) error -// expectValidationFile bool -// expectErrorInValidationFile bool -// expectTempUploadingMeta bool -// expectMeta bool -// }{ -// { -// name: "validation fails", -// validation: func(_ context.Context) error { return injectedError }, -// expectValidationFile: true, -// expectErrorInValidationFile: true, -// expectTempUploadingMeta: true, -// expectMeta: false, -// }, -// { -// name: "validation fails, uploading error fails", -// errorInjector: bucket.InjectErrorOn(bucket.OpUpload, validationPath, injectedError), -// validation: func(_ context.Context) error { return injectedError }, -// expectValidationFile: true, -// expectErrorInValidationFile: false, -// expectTempUploadingMeta: true, -// expectMeta: false, -// }, -// { -// name: "uploading meta file fails", -// errorInjector: bucket.InjectErrorOn(bucket.OpUpload, metaPath, injectedError), -// validation: validationSucceeds, -// expectValidationFile: true, -// expectErrorInValidationFile: true, -// expectTempUploadingMeta: true, -// expectMeta: false, -// }, -// { -// name: "uploading meta file fails, uploading error fails", -// errorInjector: func(op bucket.Operation, target string) error { -// if op == bucket.OpUpload && (target == metaPath || target == validationPath) { -// return injectedError -// } -// return nil -// }, -// validation: validationSucceeds, -// expectValidationFile: true, -// expectErrorInValidationFile: false, -// expectTempUploadingMeta: true, -// expectMeta: false, -// }, -// { -// name: "removing in-flight meta file fails", -// errorInjector: bucket.InjectErrorOn(bucket.OpDelete, uploadingMetaPath, injectedError), -// validation: validationSucceeds, -// expectValidationFile: false, -// expectTempUploadingMeta: true, -// expectMeta: true, -// }, -// { -// name: "removing validation file fails", -// errorInjector: bucket.InjectErrorOn(bucket.OpDelete, validationPath, injectedError), -// validation: validationSucceeds, -// expectValidationFile: true, -// expectErrorInValidationFile: false, -// expectTempUploadingMeta: false, -// expectMeta: true, -// }, -// { -// name: "valid request", -// validation: validationSucceeds, -// expectValidationFile: false, -// expectTempUploadingMeta: false, -// expectMeta: true, -// }, -// } - -// for _, tc := range testCases { -// t.Run(tc.name, func(t *testing.T) { -// bkt := objstore.NewInMemBucket() -// var injectedBkt objstore.Bucket = bkt -// if tc.errorInjector != nil { -// injectedBkt = &bucket.ErrorInjectedBucketClient{ -// Bucket: bkt, -// Injector: tc.errorInjector, -// } -// } -// cfgProvider := newMockConfigProvider() -// c := &MultitenantCompactor{ -// logger: log.NewNopLogger(), -// bucketClient: injectedBkt, -// cfgProvider: cfgProvider, -// blockUploadBlocks: promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{tenantID}), -// blockUploadBytes: promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{tenantID}), -// blockUploadFiles: promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{tenantID}), -// } -// userBkt := bucket.NewUserBucketClient(tenantID, injectedBkt, cfgProvider) - -// meta := block.Meta{} -// marshalAndUploadJSON(t, bkt, uploadingMetaPath, meta) -// v := validationFile{} -// marshalAndUploadJSON(t, bkt, validationPath, v) - -// c.validateAndCompleteBlockUpload(log.NewNopLogger(), tenantID, userBkt, ulid.MustParse(blockID), &meta, tc.validation) - -// tempUploadingMetaExists, err := bkt.Exists(context.Background(), uploadingMetaPath) -// require.NoError(t, err) -// require.Equal(t, tempUploadingMetaExists, tc.expectTempUploadingMeta) - -// metaExists, err := bkt.Exists(context.Background(), metaPath) -// require.NoError(t, err) -// require.Equal(t, metaExists, tc.expectMeta) - -// if !tc.expectValidationFile { -// exists, err := bkt.Exists(context.Background(), validationPath) -// require.NoError(t, err) -// require.False(t, exists) -// return -// } - -// r, err := bkt.Get(context.Background(), validationPath) -// require.NoError(t, err) -// decoder := json.NewDecoder(r) -// err = decoder.Decode(&v) -// require.NoError(t, err) - -// if tc.expectErrorInValidationFile { -// require.NotEmpty(t, v.Error) -// } else { -// require.Empty(t, v.Error) -// } -// }) -// } -// } - -// func TestMultitenantCompactor_ValidateBlock(t *testing.T) { -// const tenantID = "test" -// ctx := context.Background() -// tmpDir := t.TempDir() -// bkt := objstore.NewInMemBucket() - -// type Missing uint8 -// const ( -// MissingMeta Missing = 1 << iota -// MissingIndex -// MissingChunks -// ) - -// validLabels := func() []labels.Labels { -// return []labels.Labels{ -// labels.FromStrings("a", "1"), -// labels.FromStrings("b", "2"), -// labels.FromStrings("c", "3"), -// } -// } - -// testCases := []struct { -// name string -// lbls func() []labels.Labels -// metaInject func(meta *block.Meta) -// indexInject func(fname string) -// chunkInject func(fname string) -// populateFileList bool -// maximumBlockSize int64 -// verifyChunks bool -// missing Missing -// expectError bool -// expectedMsg string -// }{ -// { -// name: "valid block", -// lbls: validLabels, -// verifyChunks: true, -// expectError: false, -// populateFileList: true, -// }, -// { -// name: "maximum block size exceeded", -// lbls: validLabels, -// populateFileList: true, -// maximumBlockSize: 1, -// expectError: true, -// expectedMsg: fmt.Sprintf(maxBlockUploadSizeBytesFormat, 1), -// }, -// { -// name: "missing meta file", -// lbls: validLabels, -// missing: MissingMeta, -// expectError: true, -// expectedMsg: "failed renaming while preparing block for validation", -// }, -// { -// name: "missing index file", -// lbls: validLabels, -// missing: MissingIndex, -// expectError: true, -// expectedMsg: "error validating block: open index file:", -// }, -// { -// name: "missing chunks file", -// lbls: validLabels, -// populateFileList: true, -// missing: MissingChunks, -// expectError: true, -// expectedMsg: "failed to stat chunks/", -// }, -// { -// name: "file size mismatch", -// lbls: validLabels, -// metaInject: func(meta *block.Meta) { -// require.Greater(t, len(meta.Thanos.Files), 0) -// meta.Thanos.Files[0].SizeBytes += 10 -// }, -// populateFileList: true, -// expectError: true, -// expectedMsg: "file size mismatch", -// }, -// { -// name: "empty index file", -// lbls: validLabels, -// indexInject: func(fname string) { -// require.NoError(t, os.Truncate(fname, 0)) -// }, -// expectError: true, -// expectedMsg: "error validating block: open index file: mmap, size 0: invalid argument", -// }, -// { -// name: "index file invalid magic number", -// lbls: validLabels, -// indexInject: func(fname string) { -// flipByteAt(t, fname, 0) // guaranteed to be a magic number byte -// }, -// expectError: true, -// expectedMsg: "error validating block: open index file: invalid magic number", -// }, -// { -// name: "out of order labels", -// lbls: func() []labels.Labels { -// b := labels.NewScratchBuilder(2) -// b.Add("d", "4") -// b.Add("a", "1") -// oooLabels := []labels.Labels{ -// b.Labels(), // Haven't called Sort(), so they will be out of order. -// labels.FromStrings("b", "2"), -// labels.FromStrings("c", "3"), -// } -// return oooLabels -// }, -// expectError: true, -// expectedMsg: "error validating block: index contains 1 postings with out of order labels", -// }, -// { -// name: "segment file invalid magic number", -// lbls: validLabels, -// chunkInject: func(fname string) { -// flipByteAt(t, fname, 0) // guaranteed to be a magic number byte -// }, -// verifyChunks: true, -// expectError: true, -// expectedMsg: "invalid magic number", -// }, -// { -// name: "segment file invalid checksum", -// lbls: validLabels, -// chunkInject: func(fname string) { -// flipByteAt(t, fname, 12) // guaranteed to be a data byte -// }, -// populateFileList: true, -// verifyChunks: true, -// expectError: true, -// expectedMsg: "checksum mismatch", -// }, -// { -// name: "empty segment file", -// lbls: validLabels, -// chunkInject: func(fname string) { -// require.NoError(t, os.Truncate(fname, 0)) -// }, -// verifyChunks: true, -// expectError: true, -// expectedMsg: "size 0: invalid argument", -// }, -// } - -// for _, tc := range testCases { -// t.Run(tc.name, func(t *testing.T) { -// // create a test block -// now := time.Now() -// blockID, err := block.CreateBlock(ctx, tmpDir, tc.lbls(), 300, now.Add(-2*time.Hour).UnixMilli(), now.UnixMilli(), labels.EmptyLabels()) -// require.NoError(t, err) -// testDir := filepath.Join(tmpDir, blockID.String()) -// meta, err := block.ReadMetaFromDir(testDir) -// require.NoError(t, err) -// if tc.populateFileList { -// stats, err := block.GatherFileStats(testDir) -// require.NoError(t, err) -// meta.Thanos.Files = stats -// } - -// // create a compactor -// cfgProvider := newMockConfigProvider() -// cfgProvider.blockUploadValidationEnabled[tenantID] = true -// cfgProvider.verifyChunks[tenantID] = tc.verifyChunks -// cfgProvider.blockUploadMaxBlockSizeBytes[tenantID] = tc.maximumBlockSize -// c := &MultitenantCompactor{ -// logger: log.NewNopLogger(), -// bucketClient: bkt, -// cfgProvider: cfgProvider, -// } - -// // upload the block -// require.NoError(t, block.Upload(ctx, log.NewNopLogger(), bkt, testDir, nil)) -// // remove meta.json as we will be uploading a new one with the uploading meta name -// require.NoError(t, bkt.Delete(ctx, path.Join(blockID.String(), block.MetaFilename))) - -// // handle meta file -// if tc.metaInject != nil { -// tc.metaInject(meta) -// } -// var metaBody bytes.Buffer -// require.NoError(t, meta.Write(&metaBody)) - -// // replace index file -// if tc.indexInject != nil { -// indexFile := filepath.Join(testDir, block.IndexFilename) -// indexObject := path.Join(blockID.String(), block.IndexFilename) -// require.NoError(t, bkt.Delete(ctx, indexObject)) -// tc.indexInject(indexFile) -// uploadLocalFileToBucket(ctx, t, bkt, indexFile, indexObject) -// } - -// // replace segment file -// if tc.chunkInject != nil { -// segmentFile := filepath.Join(testDir, block.ChunksDirname, "000001") -// segmentObject := path.Join(blockID.String(), block.ChunksDirname, "000001") -// require.NoError(t, bkt.Delete(ctx, segmentObject)) -// tc.chunkInject(segmentFile) -// uploadLocalFileToBucket(ctx, t, bkt, segmentFile, segmentObject) -// } - -// // delete any files that should be missing -// if tc.missing&MissingIndex != 0 { -// require.NoError(t, bkt.Delete(ctx, path.Join(blockID.String(), block.IndexFilename))) -// } - -// if tc.missing&MissingChunks != 0 { -// chunkDir := path.Join(blockID.String(), block.ChunksDirname) -// err := bkt.Iter(ctx, chunkDir, func(name string) error { -// require.NoError(t, bkt.Delete(ctx, name)) -// return nil -// }) -// require.NoError(t, err) -// } - -// // only upload renamed meta file if it is not meant to be missing -// if tc.missing&MissingMeta == 0 { -// // rename to uploading meta file as that is what validateBlock expects -// require.NoError(t, bkt.Upload(ctx, path.Join(blockID.String(), uploadingMetaFilename), &metaBody)) -// } - -// // validate the block -// err = c.validateBlock(ctx, c.logger, blockID, meta, bkt, tenantID) -// if tc.expectError { -// require.Error(t, err) -// require.Contains(t, err.Error(), tc.expectedMsg) -// } else { -// require.NoError(t, err) -// } -// }) -// } -// } - -// func TestMultitenantCompactor_PeriodicValidationUpdater(t *testing.T) { -// const tenantID = "test" -// const blockID = "01G3FZ0JWJYJC0ZM6Y9778P6KD" -// injectedError := fmt.Errorf("injected error") -// validationPath := path.Join(tenantID, blockID, validationFilename) - -// heartbeatInterval := 50 * time.Millisecond - -// validationExists := func(t *testing.T, bkt objstore.Bucket) bool { -// exists, err := bkt.Exists(context.Background(), validationPath) -// require.NoError(t, err) -// return exists -// } - -// testCases := []struct { -// name string -// errorInjector func(op bucket.Operation, name string) error -// cancelContext bool -// assertions func(t *testing.T, ctx context.Context, bkt objstore.Bucket) -// }{ -// { -// name: "updating validation file fails", -// errorInjector: bucket.InjectErrorOn(bucket.OpUpload, validationPath, injectedError), -// assertions: func(t *testing.T, ctx context.Context, bkt objstore.Bucket) { -// <-ctx.Done() -// require.True(t, errors.Is(context.Canceled, ctx.Err())) -// require.False(t, validationExists(t, bkt)) -// }, -// }, -// { -// name: "updating validation file succeeds", -// assertions: func(t *testing.T, ctx context.Context, bkt objstore.Bucket) { -// test.Poll(t, heartbeatInterval*2, true, func() interface{} { -// return validationExists(t, bkt) -// }) - -// v := validationFile{} -// r, err := bkt.Get(context.Background(), validationPath) -// require.NoError(t, err) -// decoder := json.NewDecoder(r) -// err = decoder.Decode(&v) -// require.NoError(t, err) -// require.NotEqual(t, 0, v.LastUpdate) -// require.Empty(t, v.Error) -// }, -// }, -// { -// name: "context cancelled before update", -// cancelContext: true, -// assertions: func(t *testing.T, ctx context.Context, bkt objstore.Bucket) { -// require.False(t, validationExists(t, bkt)) -// }, -// }, -// } - -// for _, tc := range testCases { -// t.Run(tc.name, func(t *testing.T) { -// bkt := objstore.NewInMemBucket() -// var injectedBkt objstore.Bucket = bkt -// if tc.errorInjector != nil { -// injectedBkt = &bucket.ErrorInjectedBucketClient{ -// Bucket: bkt, -// Injector: tc.errorInjector, -// } -// } - -// cfgProvider := newMockConfigProvider() -// c := &MultitenantCompactor{ -// logger: log.NewNopLogger(), -// bucketClient: injectedBkt, -// cfgProvider: cfgProvider, -// } -// userBkt := bucket.NewUserBucketClient(tenantID, injectedBkt, cfgProvider) -// ctx, cancel := context.WithCancel(context.Background()) - -// heartbeatInterval := heartbeatInterval -// if tc.cancelContext { -// cancel() -// heartbeatInterval = 1 * time.Hour // to avoid racing a heartbeat -// } - -// var wg sync.WaitGroup -// wg.Add(1) -// go func() { -// defer wg.Done() -// c.periodicValidationUpdater(ctx, log.NewNopLogger(), ulid.MustParse(blockID), userBkt, cancel, heartbeatInterval) -// }() - -// if !tc.cancelContext { -// time.Sleep(heartbeatInterval) -// } - -// tc.assertions(t, ctx, bkt) - -// cancel() -// wg.Wait() -// }) -// } -// } - -// func TestMultitenantCompactor_GetBlockUploadStateHandler(t *testing.T) { -// const ( -// tenantID = "tenant" -// blockID = "01G8X9GA8R6N8F75FW1J18G83N" -// ) - -// type testcase struct { -// setupBucket func(t *testing.T, bkt objstore.Bucket) -// disableBlockUpload bool -// expectedStatusCode int -// expectedBody string -// } - -// for name, tc := range map[string]testcase{ -// "block doesn't exist": { -// expectedStatusCode: http.StatusNotFound, -// expectedBody: "block doesn't exist", -// }, - -// "complete block": { -// setupBucket: func(t *testing.T, bkt objstore.Bucket) { -// marshalAndUploadJSON(t, bkt, path.Join(tenantID, blockID, block.MetaFilename), block.Meta{}) -// }, -// expectedStatusCode: http.StatusOK, -// expectedBody: `{"result":"complete"}`, -// }, - -// "upload in progress": { -// setupBucket: func(t *testing.T, bkt objstore.Bucket) { -// marshalAndUploadJSON(t, bkt, path.Join(tenantID, blockID, uploadingMetaFilename), block.Meta{}) -// }, -// expectedStatusCode: http.StatusOK, -// expectedBody: `{"result":"uploading"}`, -// }, - -// "validating": { -// setupBucket: func(t *testing.T, bkt objstore.Bucket) { -// marshalAndUploadJSON(t, bkt, path.Join(tenantID, blockID, uploadingMetaFilename), block.Meta{}) -// marshalAndUploadJSON(t, bkt, path.Join(tenantID, blockID, validationFilename), validationFile{LastUpdate: time.Now().UnixMilli()}) -// }, -// expectedStatusCode: http.StatusOK, -// expectedBody: `{"result":"validating"}`, -// }, - -// "validation failed": { -// setupBucket: func(t *testing.T, bkt objstore.Bucket) { -// marshalAndUploadJSON(t, bkt, path.Join(tenantID, blockID, uploadingMetaFilename), block.Meta{}) -// marshalAndUploadJSON(t, bkt, path.Join(tenantID, blockID, validationFilename), validationFile{LastUpdate: time.Now().UnixMilli(), Error: "error during validation"}) -// }, -// expectedStatusCode: http.StatusOK, -// expectedBody: `{"result":"failed","error":"error during validation"}`, -// }, - -// "stale validation file": { -// setupBucket: func(t *testing.T, bkt objstore.Bucket) { -// marshalAndUploadJSON(t, bkt, path.Join(tenantID, blockID, uploadingMetaFilename), block.Meta{}) -// marshalAndUploadJSON(t, bkt, path.Join(tenantID, blockID, validationFilename), validationFile{LastUpdate: time.Now().Add(-10 * time.Minute).UnixMilli()}) -// }, -// expectedStatusCode: http.StatusOK, -// expectedBody: `{"result":"uploading"}`, -// }, -// } { -// t.Run(name, func(t *testing.T) { -// bkt := objstore.NewInMemBucket() -// if tc.setupBucket != nil { -// tc.setupBucket(t, bkt) -// } - -// cfgProvider := newMockConfigProvider() -// cfgProvider.blockUploadEnabled[tenantID] = !tc.disableBlockUpload - -// c := &MultitenantCompactor{ -// logger: log.NewNopLogger(), -// bucketClient: bkt, -// cfgProvider: cfgProvider, -// } - -// r := httptest.NewRequest(http.MethodGet, fmt.Sprintf("/api/v1/upload/block/%s/check", blockID), nil) -// urlVars := map[string]string{"block": blockID} -// r = mux.SetURLVars(r, urlVars) -// r = r.WithContext(user.InjectOrgID(r.Context(), tenantID)) - -// w := httptest.NewRecorder() -// c.GetBlockUploadStateHandler(w, r) -// resp := w.Result() - -// body, err := io.ReadAll(resp.Body) - -// require.NoError(t, err) -// require.Equal(t, tc.expectedStatusCode, resp.StatusCode) -// require.Equal(t, tc.expectedBody, strings.TrimSpace(string(body))) -// }) -// } -// } - -// func TestMultitenantCompactor_ValidateMaximumBlockSize(t *testing.T) { -// const userID = "user" - -// type testCase struct { -// maximumBlockSize int64 -// fileSizes []int64 -// expectErr bool -// } - -// for name, tc := range map[string]testCase{ -// "no limit": { -// maximumBlockSize: 0, -// fileSizes: []int64{math.MaxInt64}, -// expectErr: false, -// }, -// "under limit": { -// maximumBlockSize: 4, -// fileSizes: []int64{1, 2}, -// expectErr: false, -// }, -// "under limit - zero size file included": { -// maximumBlockSize: 2, -// fileSizes: []int64{1, 0}, -// expectErr: false, -// }, -// "under limit - negative size file included": { -// maximumBlockSize: 2, -// fileSizes: []int64{2, -1}, -// expectErr: true, -// }, -// "exact limit": { -// maximumBlockSize: 3, -// fileSizes: []int64{1, 2}, -// expectErr: false, -// }, -// "over limit": { -// maximumBlockSize: 1, -// fileSizes: []int64{1, 1}, -// expectErr: true, -// }, -// "overflow": { -// maximumBlockSize: math.MaxInt64, -// fileSizes: []int64{math.MaxInt64, math.MaxInt64, math.MaxInt64}, -// expectErr: true, -// }, -// } { -// t.Run(name, func(t *testing.T) { -// files := make([]block.File, len(tc.fileSizes)) -// for i, size := range tc.fileSizes { -// files[i] = block.File{SizeBytes: size} -// } - -// cfgProvider := newMockConfigProvider() -// cfgProvider.blockUploadMaxBlockSizeBytes[userID] = tc.maximumBlockSize -// c := &MultitenantCompactor{ -// logger: log.NewNopLogger(), -// cfgProvider: cfgProvider, -// } - -// err := c.validateMaximumBlockSize(c.logger, files, userID) -// if tc.expectErr { -// require.Error(t, err) -// } else { -// require.NoError(t, err) -// } -// }) -// } -// } - -// func TestMultitenantCompactor_MarkBlockComplete(t *testing.T) { -// const tenantID = "test" -// const blockID = "01G3FZ0JWJYJC0ZM6Y9778P6KD" -// injectedError := fmt.Errorf("injected error") - -// uploadingMetaPath := path.Join(tenantID, blockID, uploadingMetaFilename) -// metaPath := path.Join(tenantID, blockID, block.MetaFilename) -// testCases := []struct { -// name string -// errorInjector func(op bucket.Operation, name string) error -// expectSuccess bool -// }{ -// { -// name: "marking block complete succeeds", -// expectSuccess: true, -// }, -// { -// name: "uploading meta file fails", -// errorInjector: bucket.InjectErrorOn(bucket.OpUpload, metaPath, injectedError), -// }, -// { -// name: "deleting uploading meta file fails", -// errorInjector: bucket.InjectErrorOn(bucket.OpDelete, uploadingMetaPath, injectedError), -// expectSuccess: true, -// }, -// } -// for _, tc := range testCases { -// t.Run(tc.name, func(t *testing.T) { -// bkt := objstore.NewInMemBucket() -// var injectedBkt objstore.Bucket = bkt -// if tc.errorInjector != nil { -// injectedBkt = &bucket.ErrorInjectedBucketClient{ -// Bucket: bkt, -// Injector: tc.errorInjector, -// } -// } -// cfgProvider := newMockConfigProvider() -// c := &MultitenantCompactor{ -// logger: log.NewNopLogger(), -// bucketClient: injectedBkt, -// cfgProvider: cfgProvider, -// blockUploadBlocks: promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{tenantID}), -// blockUploadBytes: promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{tenantID}), -// blockUploadFiles: promauto.With(nil).NewGaugeVec(prometheus.GaugeOpts{}, []string{tenantID}), -// } -// userBkt := bucket.NewUserBucketClient(tenantID, injectedBkt, cfgProvider) - -// meta := block.Meta{ -// Thanos: block.ThanosMeta{ -// Files: []block.File{ -// { -// RelPath: "chunks/000001", -// SizeBytes: 42, -// }, -// { -// RelPath: "index", -// SizeBytes: 17, -// }, -// { -// RelPath: "meta.json", -// }, -// }, -// }, -// } -// marshalAndUploadJSON(t, bkt, uploadingMetaPath, meta) - -// ctx := context.Background() -// err := c.markBlockComplete(ctx, log.NewNopLogger(), tenantID, userBkt, ulid.MustParse(blockID), &meta) -// if tc.expectSuccess { -// require.NoError(t, err) -// assert.Equal(t, 1.0, promtest.ToFloat64(c.blockUploadBlocks.WithLabelValues(tenantID))) -// assert.Equal(t, 59.0, promtest.ToFloat64(c.blockUploadBytes.WithLabelValues(tenantID))) -// assert.Equal(t, 3.0, promtest.ToFloat64(c.blockUploadFiles.WithLabelValues(tenantID))) -// } else { -// require.Error(t, err) -// assert.Equal(t, 0.0, promtest.ToFloat64(c.blockUploadBlocks.WithLabelValues(tenantID))) -// assert.Equal(t, 0.0, promtest.ToFloat64(c.blockUploadBytes.WithLabelValues(tenantID))) -// assert.Equal(t, 0.0, promtest.ToFloat64(c.blockUploadFiles.WithLabelValues(tenantID))) -// } -// }) -// } -// } - -// // marshalAndUploadJSON is a test helper for uploading a meta file to a certain path in a bucket. -// func marshalAndUploadJSON(t *testing.T, bkt objstore.Bucket, pth string, val interface{}) { -// t.Helper() -// err := marshalAndUploadToBucket(context.Background(), bkt, pth, val) -// require.NoError(t, err) -// } - -// func uploadLocalFileToBucket(ctx context.Context, t *testing.T, bkt objstore.Bucket, src, dst string) { -// t.Helper() -// fd, err := os.Open(src) -// require.NoError(t, err) -// defer func(fd *os.File) { -// err := fd.Close() -// require.NoError(t, err) -// }(fd) -// require.NoError(t, bkt.Upload(ctx, dst, fd)) -// } - -// // flipByteAt flips a byte at a given offset in a file. -// func flipByteAt(t *testing.T, fname string, offset int64) { -// fd, err := os.OpenFile(fname, os.O_RDWR, 0o644) -// require.NoError(t, err) -// defer func(fd *os.File) { -// err := fd.Close() -// require.NoError(t, err) -// }(fd) -// var b [1]byte -// _, err = fd.ReadAt(b[:], offset) -// require.NoError(t, err) -// // alter the byte -// b[0] = 0xff - b[0] -// _, err = fd.WriteAt(b[:], offset) -// require.NoError(t, err) -// } - -// func TestHexTimeNowNano(t *testing.T) { -// v := hexTimeNowNano() -// require.Len(t, v, 16, "Should have exactly 16 characters") - -// require.NotEqual(t, strings.Repeat("0", 16), v, "Should not be all zeros") -// time.Sleep(time.Nanosecond) -// require.NotEqual(t, v, hexTimeNowNano(), "Should generate a different one.") -// } From 4f881615665ac3edc978b596d22e77ff25075a79 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Wed, 13 Sep 2023 10:49:10 +0200 Subject: [PATCH 31/74] Fixes split_merge_compactor tests --- pkg/compactor/compactor_test.go | 1079 +++++++++---------- pkg/compactor/split_merge_compactor_test.go | 658 +++++------ pkg/phlaredb/block/block_test.go | 22 +- pkg/phlaredb/block/fetcher_test.go | 3 +- pkg/phlaredb/block/testutil/create_block.go | 4 +- pkg/phlaredb/validate_test.go | 5 +- pkg/pprof/testhelper/profile_builder.go | 2 + 7 files changed, 799 insertions(+), 974 deletions(-) diff --git a/pkg/compactor/compactor_test.go b/pkg/compactor/compactor_test.go index de94412ae2..7624a2d1a1 100644 --- a/pkg/compactor/compactor_test.go +++ b/pkg/compactor/compactor_test.go @@ -27,6 +27,8 @@ import ( "github.com/grafana/dskit/ring" "github.com/grafana/dskit/services" "github.com/grafana/dskit/test" + "github.com/grafana/pyroscope/pkg/phlaredb/block/testutil" + "github.com/grafana/pyroscope/pkg/pprof/testhelper" "github.com/grafana/regexp" "github.com/oklog/ulid" "github.com/pkg/errors" @@ -175,108 +177,108 @@ func TestMultitenantCompactor_ShouldDoNothingOnNoUserBlocks(t *testing.T) { }, removeIgnoredLogs(strings.Split(strings.TrimSpace(logs.String()), "\n"))) assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` - # TYPE pyroscope_compactor_runs_started_total counter - # HELP pyroscope_compactor_runs_started_total Total number of compaction runs started. - pyroscope_compactor_runs_started_total 1 - - # TYPE pyroscope_compactor_runs_completed_total counter - # HELP pyroscope_compactor_runs_completed_total Total number of compaction runs successfully completed. - pyroscope_compactor_runs_completed_total 1 - - # TYPE pyroscope_compactor_runs_failed_total counter - # HELP pyroscope_compactor_runs_failed_total Total number of compaction runs failed. - pyroscope_compactor_runs_failed_total{reason="error"} 0 - pyroscope_compactor_runs_failed_total{reason="shutdown"} 0 - - # HELP pyroscope_compactor_garbage_collection_duration_seconds Time it took to perform garbage collection iteration. - # TYPE pyroscope_compactor_garbage_collection_duration_seconds histogram - pyroscope_compactor_garbage_collection_duration_seconds_bucket{le="+Inf"} 0 - pyroscope_compactor_garbage_collection_duration_seconds_sum 0 - pyroscope_compactor_garbage_collection_duration_seconds_count 0 - - # HELP pyroscope_compactor_garbage_collection_failures_total Total number of failed garbage collection operations. - # TYPE pyroscope_compactor_garbage_collection_failures_total counter - pyroscope_compactor_garbage_collection_failures_total 0 - - # HELP pyroscope_compactor_garbage_collection_total Total number of garbage collection operations. - # TYPE pyroscope_compactor_garbage_collection_total counter - pyroscope_compactor_garbage_collection_total 0 - - # HELP pyroscope_compactor_meta_sync_duration_seconds Duration of the blocks metadata synchronization in seconds. - # TYPE pyroscope_compactor_meta_sync_duration_seconds histogram - pyroscope_compactor_meta_sync_duration_seconds_bucket{le="+Inf"} 0 - pyroscope_compactor_meta_sync_duration_seconds_sum 0 - pyroscope_compactor_meta_sync_duration_seconds_count 0 - - # HELP pyroscope_compactor_meta_sync_failures_total Total blocks metadata synchronization failures. - # TYPE pyroscope_compactor_meta_sync_failures_total counter - pyroscope_compactor_meta_sync_failures_total 0 - - # HELP pyroscope_compactor_meta_syncs_total Total blocks metadata synchronization attempts. - # TYPE pyroscope_compactor_meta_syncs_total counter - pyroscope_compactor_meta_syncs_total 0 - - # HELP pyroscope_compactor_group_compaction_runs_completed_total Total number of group completed compaction runs. This also includes compactor group runs that resulted with no compaction. - # TYPE pyroscope_compactor_group_compaction_runs_completed_total counter - pyroscope_compactor_group_compaction_runs_completed_total 0 - - # HELP pyroscope_compactor_group_compaction_runs_started_total Total number of group compaction attempts. - # TYPE pyroscope_compactor_group_compaction_runs_started_total counter - pyroscope_compactor_group_compaction_runs_started_total 0 - - # HELP pyroscope_compactor_group_compactions_failures_total Total number of failed group compactions. - # TYPE pyroscope_compactor_group_compactions_failures_total counter - pyroscope_compactor_group_compactions_failures_total 0 - - # HELP pyroscope_compactor_group_compactions_total Total number of group compaction attempts that resulted in new block(s). - # TYPE pyroscope_compactor_group_compactions_total counter - pyroscope_compactor_group_compactions_total 0 - - # TYPE pyroscope_compactor_block_cleanup_failures_total counter - # HELP pyroscope_compactor_block_cleanup_failures_total Total number of blocks failed to be deleted. - pyroscope_compactor_block_cleanup_failures_total 0 - - # HELP pyroscope_compactor_blocks_cleaned_total Total number of blocks deleted. - # TYPE pyroscope_compactor_blocks_cleaned_total counter - pyroscope_compactor_blocks_cleaned_total 0 - - # HELP pyroscope_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. - # TYPE pyroscope_compactor_blocks_marked_for_deletion_total counter - pyroscope_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 - pyroscope_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 - pyroscope_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 - - # TYPE pyroscope_compactor_block_cleanup_started_total counter - # HELP pyroscope_compactor_block_cleanup_started_total Total number of blocks cleanup runs started. - pyroscope_compactor_block_cleanup_started_total 1 - - # TYPE pyroscope_compactor_block_cleanup_completed_total counter - # HELP pyroscope_compactor_block_cleanup_completed_total Total number of blocks cleanup runs successfully completed. - pyroscope_compactor_block_cleanup_completed_total 1 - - # TYPE pyroscope_compactor_block_cleanup_failed_total counter - # HELP pyroscope_compactor_block_cleanup_failed_total Total number of blocks cleanup runs failed. - pyroscope_compactor_block_cleanup_failed_total 0 + # TYPE cortex_compactor_runs_started_total counter + # HELP cortex_compactor_runs_started_total Total number of compaction runs started. + cortex_compactor_runs_started_total 1 + + # TYPE cortex_compactor_runs_completed_total counter + # HELP cortex_compactor_runs_completed_total Total number of compaction runs successfully completed. + cortex_compactor_runs_completed_total 1 + + # TYPE cortex_compactor_runs_failed_total counter + # HELP cortex_compactor_runs_failed_total Total number of compaction runs failed. + cortex_compactor_runs_failed_total{reason="error"} 0 + cortex_compactor_runs_failed_total{reason="shutdown"} 0 + + # HELP cortex_compactor_garbage_collection_duration_seconds Time it took to perform garbage collection iteration. + # TYPE cortex_compactor_garbage_collection_duration_seconds histogram + cortex_compactor_garbage_collection_duration_seconds_bucket{le="+Inf"} 0 + cortex_compactor_garbage_collection_duration_seconds_sum 0 + cortex_compactor_garbage_collection_duration_seconds_count 0 + + # HELP cortex_compactor_garbage_collection_failures_total Total number of failed garbage collection operations. + # TYPE cortex_compactor_garbage_collection_failures_total counter + cortex_compactor_garbage_collection_failures_total 0 + + # HELP cortex_compactor_garbage_collection_total Total number of garbage collection operations. + # TYPE cortex_compactor_garbage_collection_total counter + cortex_compactor_garbage_collection_total 0 + + # HELP cortex_compactor_meta_sync_duration_seconds Duration of the blocks metadata synchronization in seconds. + # TYPE cortex_compactor_meta_sync_duration_seconds histogram + cortex_compactor_meta_sync_duration_seconds_bucket{le="+Inf"} 0 + cortex_compactor_meta_sync_duration_seconds_sum 0 + cortex_compactor_meta_sync_duration_seconds_count 0 + + # HELP cortex_compactor_meta_sync_failures_total Total blocks metadata synchronization failures. + # TYPE cortex_compactor_meta_sync_failures_total counter + cortex_compactor_meta_sync_failures_total 0 + + # HELP cortex_compactor_meta_syncs_total Total blocks metadata synchronization attempts. + # TYPE cortex_compactor_meta_syncs_total counter + cortex_compactor_meta_syncs_total 0 + + # HELP cortex_compactor_group_compaction_runs_completed_total Total number of group completed compaction runs. This also includes compactor group runs that resulted with no compaction. + # TYPE cortex_compactor_group_compaction_runs_completed_total counter + cortex_compactor_group_compaction_runs_completed_total 0 + + # HELP cortex_compactor_group_compaction_runs_started_total Total number of group compaction attempts. + # TYPE cortex_compactor_group_compaction_runs_started_total counter + cortex_compactor_group_compaction_runs_started_total 0 + + # HELP cortex_compactor_group_compactions_failures_total Total number of failed group compactions. + # TYPE cortex_compactor_group_compactions_failures_total counter + cortex_compactor_group_compactions_failures_total 0 + + # HELP cortex_compactor_group_compactions_total Total number of group compaction attempts that resulted in new block(s). + # TYPE cortex_compactor_group_compactions_total counter + cortex_compactor_group_compactions_total 0 + + # TYPE cortex_compactor_block_cleanup_failures_total counter + # HELP cortex_compactor_block_cleanup_failures_total Total number of blocks failed to be deleted. + cortex_compactor_block_cleanup_failures_total 0 + + # HELP cortex_compactor_blocks_cleaned_total Total number of blocks deleted. + # TYPE cortex_compactor_blocks_cleaned_total counter + cortex_compactor_blocks_cleaned_total 0 + + # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. + # TYPE cortex_compactor_blocks_marked_for_deletion_total counter + cortex_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 + cortex_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 + cortex_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 + + # TYPE cortex_compactor_block_cleanup_started_total counter + # HELP cortex_compactor_block_cleanup_started_total Total number of blocks cleanup runs started. + cortex_compactor_block_cleanup_started_total 1 + + # TYPE cortex_compactor_block_cleanup_completed_total counter + # HELP cortex_compactor_block_cleanup_completed_total Total number of blocks cleanup runs successfully completed. + cortex_compactor_block_cleanup_completed_total 1 + + # TYPE cortex_compactor_block_cleanup_failed_total counter + # HELP cortex_compactor_block_cleanup_failed_total Total number of blocks cleanup runs failed. + cortex_compactor_block_cleanup_failed_total 0 `), - "pyroscope_compactor_runs_started_total", - "pyroscope_compactor_runs_completed_total", - "pyroscope_compactor_runs_failed_total", - "pyroscope_compactor_garbage_collection_duration_seconds", - "pyroscope_compactor_garbage_collection_failures_total", - "pyroscope_compactor_garbage_collection_total", - "pyroscope_compactor_meta_sync_duration_seconds", - "pyroscope_compactor_meta_sync_failures_total", - "pyroscope_compactor_meta_syncs_total", - "pyroscope_compactor_group_compaction_runs_completed_total", - "pyroscope_compactor_group_compaction_runs_started_total", - "pyroscope_compactor_group_compactions_failures_total", - "pyroscope_compactor_group_compactions_total", - "pyroscope_compactor_block_cleanup_failures_total", - "pyroscope_compactor_blocks_cleaned_total", - "pyroscope_compactor_blocks_marked_for_deletion_total", - "pyroscope_compactor_block_cleanup_started_total", - "pyroscope_compactor_block_cleanup_completed_total", - "pyroscope_compactor_block_cleanup_failed_total", + "cortex_compactor_runs_started_total", + "cortex_compactor_runs_completed_total", + "cortex_compactor_runs_failed_total", + "cortex_compactor_garbage_collection_duration_seconds", + "cortex_compactor_garbage_collection_failures_total", + "cortex_compactor_garbage_collection_total", + "cortex_compactor_meta_sync_duration_seconds", + "cortex_compactor_meta_sync_failures_total", + "cortex_compactor_meta_syncs_total", + "cortex_compactor_group_compaction_runs_completed_total", + "cortex_compactor_group_compaction_runs_started_total", + "cortex_compactor_group_compactions_failures_total", + "cortex_compactor_group_compactions_total", + "cortex_compactor_block_cleanup_failures_total", + "cortex_compactor_blocks_cleaned_total", + "cortex_compactor_blocks_marked_for_deletion_total", + "cortex_compactor_block_cleanup_started_total", + "cortex_compactor_block_cleanup_completed_total", + "cortex_compactor_block_cleanup_failed_total", )) } @@ -314,108 +316,108 @@ func TestMultitenantCompactor_ShouldRetryCompactionOnFailureWhileDiscoveringUser }, removeIgnoredLogs(strings.Split(strings.TrimSpace(logs.String()), "\n"))) assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` - # TYPE pyroscope_compactor_runs_started_total counter - # HELP pyroscope_compactor_runs_started_total Total number of compaction runs started. - pyroscope_compactor_runs_started_total 1 - - # TYPE pyroscope_compactor_runs_completed_total counter - # HELP pyroscope_compactor_runs_completed_total Total number of compaction runs successfully completed. - pyroscope_compactor_runs_completed_total 0 - - # TYPE pyroscope_compactor_runs_failed_total counter - # HELP pyroscope_compactor_runs_failed_total Total number of compaction runs failed. - pyroscope_compactor_runs_failed_total{reason="error"} 1 - pyroscope_compactor_runs_failed_total{reason="shutdown"} 0 - - # HELP pyroscope_compactor_garbage_collection_duration_seconds Time it took to perform garbage collection iteration. - # TYPE pyroscope_compactor_garbage_collection_duration_seconds histogram - pyroscope_compactor_garbage_collection_duration_seconds_bucket{le="+Inf"} 0 - pyroscope_compactor_garbage_collection_duration_seconds_sum 0 - pyroscope_compactor_garbage_collection_duration_seconds_count 0 - - # HELP pyroscope_compactor_garbage_collection_failures_total Total number of failed garbage collection operations. - # TYPE pyroscope_compactor_garbage_collection_failures_total counter - pyroscope_compactor_garbage_collection_failures_total 0 - - # HELP pyroscope_compactor_garbage_collection_total Total number of garbage collection operations. - # TYPE pyroscope_compactor_garbage_collection_total counter - pyroscope_compactor_garbage_collection_total 0 - - # HELP pyroscope_compactor_meta_sync_duration_seconds Duration of the blocks metadata synchronization in seconds. - # TYPE pyroscope_compactor_meta_sync_duration_seconds histogram - pyroscope_compactor_meta_sync_duration_seconds_bucket{le="+Inf"} 0 - pyroscope_compactor_meta_sync_duration_seconds_sum 0 - pyroscope_compactor_meta_sync_duration_seconds_count 0 - - # HELP pyroscope_compactor_meta_sync_failures_total Total blocks metadata synchronization failures. - # TYPE pyroscope_compactor_meta_sync_failures_total counter - pyroscope_compactor_meta_sync_failures_total 0 - - # HELP pyroscope_compactor_meta_syncs_total Total blocks metadata synchronization attempts. - # TYPE pyroscope_compactor_meta_syncs_total counter - pyroscope_compactor_meta_syncs_total 0 - - # HELP pyroscope_compactor_group_compaction_runs_completed_total Total number of group completed compaction runs. This also includes compactor group runs that resulted with no compaction. - # TYPE pyroscope_compactor_group_compaction_runs_completed_total counter - pyroscope_compactor_group_compaction_runs_completed_total 0 - - # HELP pyroscope_compactor_group_compaction_runs_started_total Total number of group compaction attempts. - # TYPE pyroscope_compactor_group_compaction_runs_started_total counter - pyroscope_compactor_group_compaction_runs_started_total 0 - - # HELP pyroscope_compactor_group_compactions_failures_total Total number of failed group compactions. - # TYPE pyroscope_compactor_group_compactions_failures_total counter - pyroscope_compactor_group_compactions_failures_total 0 - - # HELP pyroscope_compactor_group_compactions_total Total number of group compaction attempts that resulted in new block(s). - # TYPE pyroscope_compactor_group_compactions_total counter - pyroscope_compactor_group_compactions_total 0 - - # TYPE pyroscope_compactor_block_cleanup_failures_total counter - # HELP pyroscope_compactor_block_cleanup_failures_total Total number of blocks failed to be deleted. - pyroscope_compactor_block_cleanup_failures_total 0 - - # HELP pyroscope_compactor_blocks_cleaned_total Total number of blocks deleted. - # TYPE pyroscope_compactor_blocks_cleaned_total counter - pyroscope_compactor_blocks_cleaned_total 0 - - # HELP pyroscope_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. - # TYPE pyroscope_compactor_blocks_marked_for_deletion_total counter - pyroscope_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 - pyroscope_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 - pyroscope_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 - - # TYPE pyroscope_compactor_block_cleanup_started_total counter - # HELP pyroscope_compactor_block_cleanup_started_total Total number of blocks cleanup runs started. - pyroscope_compactor_block_cleanup_started_total 1 - - # TYPE pyroscope_compactor_block_cleanup_completed_total counter - # HELP pyroscope_compactor_block_cleanup_completed_total Total number of blocks cleanup runs successfully completed. - pyroscope_compactor_block_cleanup_completed_total 0 - - # TYPE pyroscope_compactor_block_cleanup_failed_total counter - # HELP pyroscope_compactor_block_cleanup_failed_total Total number of blocks cleanup runs failed. - pyroscope_compactor_block_cleanup_failed_total 1 + # TYPE cortex_compactor_runs_started_total counter + # HELP cortex_compactor_runs_started_total Total number of compaction runs started. + cortex_compactor_runs_started_total 1 + + # TYPE cortex_compactor_runs_completed_total counter + # HELP cortex_compactor_runs_completed_total Total number of compaction runs successfully completed. + cortex_compactor_runs_completed_total 0 + + # TYPE cortex_compactor_runs_failed_total counter + # HELP cortex_compactor_runs_failed_total Total number of compaction runs failed. + cortex_compactor_runs_failed_total{reason="error"} 1 + cortex_compactor_runs_failed_total{reason="shutdown"} 0 + + # HELP cortex_compactor_garbage_collection_duration_seconds Time it took to perform garbage collection iteration. + # TYPE cortex_compactor_garbage_collection_duration_seconds histogram + cortex_compactor_garbage_collection_duration_seconds_bucket{le="+Inf"} 0 + cortex_compactor_garbage_collection_duration_seconds_sum 0 + cortex_compactor_garbage_collection_duration_seconds_count 0 + + # HELP cortex_compactor_garbage_collection_failures_total Total number of failed garbage collection operations. + # TYPE cortex_compactor_garbage_collection_failures_total counter + cortex_compactor_garbage_collection_failures_total 0 + + # HELP cortex_compactor_garbage_collection_total Total number of garbage collection operations. + # TYPE cortex_compactor_garbage_collection_total counter + cortex_compactor_garbage_collection_total 0 + + # HELP cortex_compactor_meta_sync_duration_seconds Duration of the blocks metadata synchronization in seconds. + # TYPE cortex_compactor_meta_sync_duration_seconds histogram + cortex_compactor_meta_sync_duration_seconds_bucket{le="+Inf"} 0 + cortex_compactor_meta_sync_duration_seconds_sum 0 + cortex_compactor_meta_sync_duration_seconds_count 0 + + # HELP cortex_compactor_meta_sync_failures_total Total blocks metadata synchronization failures. + # TYPE cortex_compactor_meta_sync_failures_total counter + cortex_compactor_meta_sync_failures_total 0 + + # HELP cortex_compactor_meta_syncs_total Total blocks metadata synchronization attempts. + # TYPE cortex_compactor_meta_syncs_total counter + cortex_compactor_meta_syncs_total 0 + + # HELP cortex_compactor_group_compaction_runs_completed_total Total number of group completed compaction runs. This also includes compactor group runs that resulted with no compaction. + # TYPE cortex_compactor_group_compaction_runs_completed_total counter + cortex_compactor_group_compaction_runs_completed_total 0 + + # HELP cortex_compactor_group_compaction_runs_started_total Total number of group compaction attempts. + # TYPE cortex_compactor_group_compaction_runs_started_total counter + cortex_compactor_group_compaction_runs_started_total 0 + + # HELP cortex_compactor_group_compactions_failures_total Total number of failed group compactions. + # TYPE cortex_compactor_group_compactions_failures_total counter + cortex_compactor_group_compactions_failures_total 0 + + # HELP cortex_compactor_group_compactions_total Total number of group compaction attempts that resulted in new block(s). + # TYPE cortex_compactor_group_compactions_total counter + cortex_compactor_group_compactions_total 0 + + # TYPE cortex_compactor_block_cleanup_failures_total counter + # HELP cortex_compactor_block_cleanup_failures_total Total number of blocks failed to be deleted. + cortex_compactor_block_cleanup_failures_total 0 + + # HELP cortex_compactor_blocks_cleaned_total Total number of blocks deleted. + # TYPE cortex_compactor_blocks_cleaned_total counter + cortex_compactor_blocks_cleaned_total 0 + + # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. + # TYPE cortex_compactor_blocks_marked_for_deletion_total counter + cortex_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 + cortex_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 + cortex_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 + + # TYPE cortex_compactor_block_cleanup_started_total counter + # HELP cortex_compactor_block_cleanup_started_total Total number of blocks cleanup runs started. + cortex_compactor_block_cleanup_started_total 1 + + # TYPE cortex_compactor_block_cleanup_completed_total counter + # HELP cortex_compactor_block_cleanup_completed_total Total number of blocks cleanup runs successfully completed. + cortex_compactor_block_cleanup_completed_total 0 + + # TYPE cortex_compactor_block_cleanup_failed_total counter + # HELP cortex_compactor_block_cleanup_failed_total Total number of blocks cleanup runs failed. + cortex_compactor_block_cleanup_failed_total 1 `), - "pyroscope_compactor_runs_started_total", - "pyroscope_compactor_runs_completed_total", - "pyroscope_compactor_runs_failed_total", - "pyroscope_compactor_garbage_collection_duration_seconds", - "pyroscope_compactor_garbage_collection_failures_total", - "pyroscope_compactor_garbage_collection_total", - "pyroscope_compactor_meta_sync_duration_seconds", - "pyroscope_compactor_meta_sync_failures_total", - "pyroscope_compactor_meta_syncs_total", - "pyroscope_compactor_group_compaction_runs_completed_total", - "pyroscope_compactor_group_compaction_runs_started_total", - "pyroscope_compactor_group_compactions_failures_total", - "pyroscope_compactor_group_compactions_total", - "pyroscope_compactor_block_cleanup_failures_total", - "pyroscope_compactor_blocks_cleaned_total", - "pyroscope_compactor_blocks_marked_for_deletion_total", - "pyroscope_compactor_block_cleanup_started_total", - "pyroscope_compactor_block_cleanup_completed_total", - "pyroscope_compactor_block_cleanup_failed_total", + "cortex_compactor_runs_started_total", + "cortex_compactor_runs_completed_total", + "cortex_compactor_runs_failed_total", + "cortex_compactor_garbage_collection_duration_seconds", + "cortex_compactor_garbage_collection_failures_total", + "cortex_compactor_garbage_collection_total", + "cortex_compactor_meta_sync_duration_seconds", + "cortex_compactor_meta_sync_failures_total", + "cortex_compactor_meta_syncs_total", + "cortex_compactor_group_compaction_runs_completed_total", + "cortex_compactor_group_compaction_runs_started_total", + "cortex_compactor_group_compactions_failures_total", + "cortex_compactor_group_compactions_total", + "cortex_compactor_block_cleanup_failures_total", + "cortex_compactor_blocks_cleaned_total", + "cortex_compactor_blocks_marked_for_deletion_total", + "cortex_compactor_block_cleanup_started_total", + "cortex_compactor_block_cleanup_completed_total", + "cortex_compactor_block_cleanup_failed_total", )) } @@ -449,22 +451,22 @@ func TestMultitenantCompactor_ShouldIncrementCompactionErrorIfFailedToCompactASi require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` - # TYPE pyroscope_compactor_runs_started_total counter - # HELP pyroscope_compactor_runs_started_total Total number of compaction runs started. - pyroscope_compactor_runs_started_total 1 - - # TYPE pyroscope_compactor_runs_completed_total counter - # HELP pyroscope_compactor_runs_completed_total Total number of compaction runs successfully completed. - pyroscope_compactor_runs_completed_total 0 - - # TYPE pyroscope_compactor_runs_failed_total counter - # HELP pyroscope_compactor_runs_failed_total Total number of compaction runs failed. - pyroscope_compactor_runs_failed_total{reason="error"} 1 - pyroscope_compactor_runs_failed_total{reason="shutdown"} 0 + # TYPE cortex_compactor_runs_started_total counter + # HELP cortex_compactor_runs_started_total Total number of compaction runs started. + cortex_compactor_runs_started_total 1 + + # TYPE cortex_compactor_runs_completed_total counter + # HELP cortex_compactor_runs_completed_total Total number of compaction runs successfully completed. + cortex_compactor_runs_completed_total 0 + + # TYPE cortex_compactor_runs_failed_total counter + # HELP cortex_compactor_runs_failed_total Total number of compaction runs failed. + cortex_compactor_runs_failed_total{reason="error"} 1 + cortex_compactor_runs_failed_total{reason="shutdown"} 0 `), - "pyroscope_compactor_runs_started_total", - "pyroscope_compactor_runs_completed_total", - "pyroscope_compactor_runs_failed_total", + "cortex_compactor_runs_started_total", + "cortex_compactor_runs_completed_total", + "cortex_compactor_runs_failed_total", )) } @@ -502,22 +504,22 @@ func TestMultitenantCompactor_ShouldIncrementCompactionShutdownIfTheContextIsCan require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` - # TYPE pyroscope_compactor_runs_started_total counter - # HELP pyroscope_compactor_runs_started_total Total number of compaction runs started. - pyroscope_compactor_runs_started_total 1 - - # TYPE pyroscope_compactor_runs_completed_total counter - # HELP pyroscope_compactor_runs_completed_total Total number of compaction runs successfully completed. - pyroscope_compactor_runs_completed_total 0 - - # TYPE pyroscope_compactor_runs_failed_total counter - # HELP pyroscope_compactor_runs_failed_total Total number of compaction runs failed. - pyroscope_compactor_runs_failed_total{reason="error"} 0 - pyroscope_compactor_runs_failed_total{reason="shutdown"} 1 + # TYPE cortex_compactor_runs_started_total counter + # HELP cortex_compactor_runs_started_total Total number of compaction runs started. + cortex_compactor_runs_started_total 1 + + # TYPE cortex_compactor_runs_completed_total counter + # HELP cortex_compactor_runs_completed_total Total number of compaction runs successfully completed. + cortex_compactor_runs_completed_total 0 + + # TYPE cortex_compactor_runs_failed_total counter + # HELP cortex_compactor_runs_failed_total Total number of compaction runs failed. + cortex_compactor_runs_failed_total{reason="error"} 0 + cortex_compactor_runs_failed_total{reason="shutdown"} 1 `), - "pyroscope_compactor_runs_started_total", - "pyroscope_compactor_runs_completed_total", - "pyroscope_compactor_runs_failed_total", + "cortex_compactor_runs_started_total", + "cortex_compactor_runs_completed_total", + "cortex_compactor_runs_failed_total", )) } @@ -600,68 +602,68 @@ func TestMultitenantCompactor_ShouldIterateOverUsersAndRunCompaction(t *testing. // Instead of testing for shipper metrics, we only check our metrics here. // Real shipper metrics are too variable to embed into a test. testedMetrics := []string{ - "pyroscope_compactor_runs_started_total", "pyroscope_compactor_runs_completed_total", "pyroscope_compactor_runs_failed_total", - "pyroscope_compactor_blocks_cleaned_total", "pyroscope_compactor_block_cleanup_failures_total", "pyroscope_compactor_blocks_marked_for_deletion_total", - "pyroscope_compactor_block_cleanup_started_total", "pyroscope_compactor_block_cleanup_completed_total", "pyroscope_compactor_block_cleanup_failed_total", - "pyroscope_compactor_group_compaction_runs_completed_total", "pyroscope_compactor_group_compaction_runs_started_total", - "pyroscope_compactor_group_compactions_failures_total", "pyroscope_compactor_group_compactions_total", + "cortex_compactor_runs_started_total", "cortex_compactor_runs_completed_total", "cortex_compactor_runs_failed_total", + "cortex_compactor_blocks_cleaned_total", "cortex_compactor_block_cleanup_failures_total", "cortex_compactor_blocks_marked_for_deletion_total", + "cortex_compactor_block_cleanup_started_total", "cortex_compactor_block_cleanup_completed_total", "cortex_compactor_block_cleanup_failed_total", + "cortex_compactor_group_compaction_runs_completed_total", "cortex_compactor_group_compaction_runs_started_total", + "cortex_compactor_group_compactions_failures_total", "cortex_compactor_group_compactions_total", } assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` - # TYPE pyroscope_compactor_runs_started_total counter - # HELP pyroscope_compactor_runs_started_total Total number of compaction runs started. - pyroscope_compactor_runs_started_total 1 - - # TYPE pyroscope_compactor_runs_completed_total counter - # HELP pyroscope_compactor_runs_completed_total Total number of compaction runs successfully completed. - pyroscope_compactor_runs_completed_total 1 - - # TYPE pyroscope_compactor_runs_failed_total counter - # HELP pyroscope_compactor_runs_failed_total Total number of compaction runs failed. - pyroscope_compactor_runs_failed_total{reason="error"} 0 - pyroscope_compactor_runs_failed_total{reason="shutdown"} 0 - - # HELP pyroscope_compactor_group_compaction_runs_completed_total Total number of group completed compaction runs. This also includes compactor group runs that resulted with no compaction. - # TYPE pyroscope_compactor_group_compaction_runs_completed_total counter - pyroscope_compactor_group_compaction_runs_completed_total 2 - - # HELP pyroscope_compactor_group_compaction_runs_started_total Total number of group compaction attempts. - # TYPE pyroscope_compactor_group_compaction_runs_started_total counter - pyroscope_compactor_group_compaction_runs_started_total 2 - - # HELP pyroscope_compactor_group_compactions_failures_total Total number of failed group compactions. - # TYPE pyroscope_compactor_group_compactions_failures_total counter - pyroscope_compactor_group_compactions_failures_total 0 - - # HELP pyroscope_compactor_group_compactions_total Total number of group compaction attempts that resulted in new block(s). - # TYPE pyroscope_compactor_group_compactions_total counter - pyroscope_compactor_group_compactions_total 0 - - # TYPE pyroscope_compactor_block_cleanup_failures_total counter - # HELP pyroscope_compactor_block_cleanup_failures_total Total number of blocks failed to be deleted. - pyroscope_compactor_block_cleanup_failures_total 0 - - # HELP pyroscope_compactor_blocks_cleaned_total Total number of blocks deleted. - # TYPE pyroscope_compactor_blocks_cleaned_total counter - pyroscope_compactor_blocks_cleaned_total 0 - - # HELP pyroscope_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. - # TYPE pyroscope_compactor_blocks_marked_for_deletion_total counter - pyroscope_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 - pyroscope_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 - pyroscope_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 - - # TYPE pyroscope_compactor_block_cleanup_started_total counter - # HELP pyroscope_compactor_block_cleanup_started_total Total number of blocks cleanup runs started. - pyroscope_compactor_block_cleanup_started_total 1 - - # TYPE pyroscope_compactor_block_cleanup_completed_total counter - # HELP pyroscope_compactor_block_cleanup_completed_total Total number of blocks cleanup runs successfully completed. - pyroscope_compactor_block_cleanup_completed_total 1 - - # TYPE pyroscope_compactor_block_cleanup_failed_total counter - # HELP pyroscope_compactor_block_cleanup_failed_total Total number of blocks cleanup runs failed. - pyroscope_compactor_block_cleanup_failed_total 0 + # TYPE cortex_compactor_runs_started_total counter + # HELP cortex_compactor_runs_started_total Total number of compaction runs started. + cortex_compactor_runs_started_total 1 + + # TYPE cortex_compactor_runs_completed_total counter + # HELP cortex_compactor_runs_completed_total Total number of compaction runs successfully completed. + cortex_compactor_runs_completed_total 1 + + # TYPE cortex_compactor_runs_failed_total counter + # HELP cortex_compactor_runs_failed_total Total number of compaction runs failed. + cortex_compactor_runs_failed_total{reason="error"} 0 + cortex_compactor_runs_failed_total{reason="shutdown"} 0 + + # HELP cortex_compactor_group_compaction_runs_completed_total Total number of group completed compaction runs. This also includes compactor group runs that resulted with no compaction. + # TYPE cortex_compactor_group_compaction_runs_completed_total counter + cortex_compactor_group_compaction_runs_completed_total 2 + + # HELP cortex_compactor_group_compaction_runs_started_total Total number of group compaction attempts. + # TYPE cortex_compactor_group_compaction_runs_started_total counter + cortex_compactor_group_compaction_runs_started_total 2 + + # HELP cortex_compactor_group_compactions_failures_total Total number of failed group compactions. + # TYPE cortex_compactor_group_compactions_failures_total counter + cortex_compactor_group_compactions_failures_total 0 + + # HELP cortex_compactor_group_compactions_total Total number of group compaction attempts that resulted in new block(s). + # TYPE cortex_compactor_group_compactions_total counter + cortex_compactor_group_compactions_total 0 + + # TYPE cortex_compactor_block_cleanup_failures_total counter + # HELP cortex_compactor_block_cleanup_failures_total Total number of blocks failed to be deleted. + cortex_compactor_block_cleanup_failures_total 0 + + # HELP cortex_compactor_blocks_cleaned_total Total number of blocks deleted. + # TYPE cortex_compactor_blocks_cleaned_total counter + cortex_compactor_blocks_cleaned_total 0 + + # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. + # TYPE cortex_compactor_blocks_marked_for_deletion_total counter + cortex_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 + cortex_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 + cortex_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 + + # TYPE cortex_compactor_block_cleanup_started_total counter + # HELP cortex_compactor_block_cleanup_started_total Total number of blocks cleanup runs started. + cortex_compactor_block_cleanup_started_total 1 + + # TYPE cortex_compactor_block_cleanup_completed_total counter + # HELP cortex_compactor_block_cleanup_completed_total Total number of blocks cleanup runs successfully completed. + cortex_compactor_block_cleanup_completed_total 1 + + # TYPE cortex_compactor_block_cleanup_failed_total counter + # HELP cortex_compactor_block_cleanup_failed_total Total number of blocks cleanup runs failed. + cortex_compactor_block_cleanup_failed_total 0 `), testedMetrics...)) } @@ -805,49 +807,49 @@ func TestMultitenantCompactor_ShouldNotCompactBlocksMarkedForDeletion(t *testing // Instead of testing for shipper metrics, we only check our metrics here. // Real shipper metrics are too variable to embed into a test. testedMetrics := []string{ - "pyroscope_compactor_runs_started_total", "pyroscope_compactor_runs_completed_total", "pyroscope_compactor_runs_failed_total", - "pyroscope_compactor_blocks_cleaned_total", "pyroscope_compactor_block_cleanup_failures_total", "pyroscope_compactor_blocks_marked_for_deletion_total", - "pyroscope_compactor_block_cleanup_started_total", "pyroscope_compactor_block_cleanup_completed_total", "pyroscope_compactor_block_cleanup_failed_total", + "cortex_compactor_runs_started_total", "cortex_compactor_runs_completed_total", "cortex_compactor_runs_failed_total", + "cortex_compactor_blocks_cleaned_total", "cortex_compactor_block_cleanup_failures_total", "cortex_compactor_blocks_marked_for_deletion_total", + "cortex_compactor_block_cleanup_started_total", "cortex_compactor_block_cleanup_completed_total", "cortex_compactor_block_cleanup_failed_total", } assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` - # TYPE pyroscope_compactor_runs_started_total counter - # HELP pyroscope_compactor_runs_started_total Total number of compaction runs started. - pyroscope_compactor_runs_started_total 1 - - # TYPE pyroscope_compactor_runs_completed_total counter - # HELP pyroscope_compactor_runs_completed_total Total number of compaction runs successfully completed. - pyroscope_compactor_runs_completed_total 1 - - # TYPE pyroscope_compactor_runs_failed_total counter - # HELP pyroscope_compactor_runs_failed_total Total number of compaction runs failed. - pyroscope_compactor_runs_failed_total{reason="error"} 0 - pyroscope_compactor_runs_failed_total{reason="shutdown"} 0 - - # TYPE pyroscope_compactor_block_cleanup_failures_total counter - # HELP pyroscope_compactor_block_cleanup_failures_total Total number of blocks failed to be deleted. - pyroscope_compactor_block_cleanup_failures_total 0 - - # HELP pyroscope_compactor_blocks_cleaned_total Total number of blocks deleted. - # TYPE pyroscope_compactor_blocks_cleaned_total counter - pyroscope_compactor_blocks_cleaned_total 1 - - # HELP pyroscope_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. - # TYPE pyroscope_compactor_blocks_marked_for_deletion_total counter - pyroscope_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 - pyroscope_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 - pyroscope_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 - - # TYPE pyroscope_compactor_block_cleanup_started_total counter - # HELP pyroscope_compactor_block_cleanup_started_total Total number of blocks cleanup runs started. - pyroscope_compactor_block_cleanup_started_total 1 - - # TYPE pyroscope_compactor_block_cleanup_completed_total counter - # HELP pyroscope_compactor_block_cleanup_completed_total Total number of blocks cleanup runs successfully completed. - pyroscope_compactor_block_cleanup_completed_total 1 - - # TYPE pyroscope_compactor_block_cleanup_failed_total counter - # HELP pyroscope_compactor_block_cleanup_failed_total Total number of blocks cleanup runs failed. - pyroscope_compactor_block_cleanup_failed_total 0 + # TYPE cortex_compactor_runs_started_total counter + # HELP cortex_compactor_runs_started_total Total number of compaction runs started. + cortex_compactor_runs_started_total 1 + + # TYPE cortex_compactor_runs_completed_total counter + # HELP cortex_compactor_runs_completed_total Total number of compaction runs successfully completed. + cortex_compactor_runs_completed_total 1 + + # TYPE cortex_compactor_runs_failed_total counter + # HELP cortex_compactor_runs_failed_total Total number of compaction runs failed. + cortex_compactor_runs_failed_total{reason="error"} 0 + cortex_compactor_runs_failed_total{reason="shutdown"} 0 + + # TYPE cortex_compactor_block_cleanup_failures_total counter + # HELP cortex_compactor_block_cleanup_failures_total Total number of blocks failed to be deleted. + cortex_compactor_block_cleanup_failures_total 0 + + # HELP cortex_compactor_blocks_cleaned_total Total number of blocks deleted. + # TYPE cortex_compactor_blocks_cleaned_total counter + cortex_compactor_blocks_cleaned_total 1 + + # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. + # TYPE cortex_compactor_blocks_marked_for_deletion_total counter + cortex_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 + cortex_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 + cortex_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 + + # TYPE cortex_compactor_block_cleanup_started_total counter + # HELP cortex_compactor_block_cleanup_started_total Total number of blocks cleanup runs started. + cortex_compactor_block_cleanup_started_total 1 + + # TYPE cortex_compactor_block_cleanup_completed_total counter + # HELP cortex_compactor_block_cleanup_completed_total Total number of blocks cleanup runs successfully completed. + cortex_compactor_block_cleanup_completed_total 1 + + # TYPE cortex_compactor_block_cleanup_failed_total counter + # HELP cortex_compactor_block_cleanup_failed_total Total number of blocks cleanup runs failed. + cortex_compactor_block_cleanup_failed_total 0 `), testedMetrics...)) } @@ -962,50 +964,50 @@ func TestMultitenantCompactor_ShouldNotCompactBlocksForUsersMarkedForDeletion(t // Instead of testing for shipper metrics, we only check our metrics here. // Real shipper metrics are too variable to embed into a test. testedMetrics := []string{ - "pyroscope_compactor_runs_started_total", "pyroscope_compactor_runs_completed_total", "pyroscope_compactor_runs_failed_total", - "pyroscope_compactor_blocks_cleaned_total", "pyroscope_compactor_block_cleanup_failures_total", "pyroscope_compactor_blocks_marked_for_deletion_total", - "pyroscope_compactor_block_cleanup_started_total", "pyroscope_compactor_block_cleanup_completed_total", "pyroscope_compactor_block_cleanup_failed_total", - "pyroscope_bucket_blocks_count", "pyroscope_bucket_blocks_marked_for_deletion_count", "pyroscope_bucket_index_last_successful_update_timestamp_seconds", + "cortex_compactor_runs_started_total", "cortex_compactor_runs_completed_total", "cortex_compactor_runs_failed_total", + "cortex_compactor_blocks_cleaned_total", "cortex_compactor_block_cleanup_failures_total", "cortex_compactor_blocks_marked_for_deletion_total", + "cortex_compactor_block_cleanup_started_total", "cortex_compactor_block_cleanup_completed_total", "cortex_compactor_block_cleanup_failed_total", + "cortex_bucket_blocks_count", "cortex_bucket_blocks_marked_for_deletion_count", "cortex_bucket_index_last_successful_update_timestamp_seconds", } assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` - # TYPE pyroscope_compactor_runs_started_total counter - # HELP pyroscope_compactor_runs_started_total Total number of compaction runs started. - pyroscope_compactor_runs_started_total 1 - - # TYPE pyroscope_compactor_runs_completed_total counter - # HELP pyroscope_compactor_runs_completed_total Total number of compaction runs successfully completed. - pyroscope_compactor_runs_completed_total 1 - - # TYPE pyroscope_compactor_runs_failed_total counter - # HELP pyroscope_compactor_runs_failed_total Total number of compaction runs failed. - pyroscope_compactor_runs_failed_total{reason="error"} 0 - pyroscope_compactor_runs_failed_total{reason="shutdown"} 0 - - # TYPE pyroscope_compactor_block_cleanup_failures_total counter - # HELP pyroscope_compactor_block_cleanup_failures_total Total number of blocks failed to be deleted. - pyroscope_compactor_block_cleanup_failures_total 0 - - # HELP pyroscope_compactor_blocks_cleaned_total Total number of blocks deleted. - # TYPE pyroscope_compactor_blocks_cleaned_total counter - pyroscope_compactor_blocks_cleaned_total 1 - - # HELP pyroscope_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. - # TYPE pyroscope_compactor_blocks_marked_for_deletion_total counter - pyroscope_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 - pyroscope_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 - pyroscope_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 - - # TYPE pyroscope_compactor_block_cleanup_started_total counter - # HELP pyroscope_compactor_block_cleanup_started_total Total number of blocks cleanup runs started. - pyroscope_compactor_block_cleanup_started_total 1 - - # TYPE pyroscope_compactor_block_cleanup_completed_total counter - # HELP pyroscope_compactor_block_cleanup_completed_total Total number of blocks cleanup runs successfully completed. - pyroscope_compactor_block_cleanup_completed_total 1 - - # TYPE pyroscope_compactor_block_cleanup_failed_total counter - # HELP pyroscope_compactor_block_cleanup_failed_total Total number of blocks cleanup runs failed. - pyroscope_compactor_block_cleanup_failed_total 0 + # TYPE cortex_compactor_runs_started_total counter + # HELP cortex_compactor_runs_started_total Total number of compaction runs started. + cortex_compactor_runs_started_total 1 + + # TYPE cortex_compactor_runs_completed_total counter + # HELP cortex_compactor_runs_completed_total Total number of compaction runs successfully completed. + cortex_compactor_runs_completed_total 1 + + # TYPE cortex_compactor_runs_failed_total counter + # HELP cortex_compactor_runs_failed_total Total number of compaction runs failed. + cortex_compactor_runs_failed_total{reason="error"} 0 + cortex_compactor_runs_failed_total{reason="shutdown"} 0 + + # TYPE cortex_compactor_block_cleanup_failures_total counter + # HELP cortex_compactor_block_cleanup_failures_total Total number of blocks failed to be deleted. + cortex_compactor_block_cleanup_failures_total 0 + + # HELP cortex_compactor_blocks_cleaned_total Total number of blocks deleted. + # TYPE cortex_compactor_blocks_cleaned_total counter + cortex_compactor_blocks_cleaned_total 1 + + # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. + # TYPE cortex_compactor_blocks_marked_for_deletion_total counter + cortex_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 + cortex_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 + cortex_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 + + # TYPE cortex_compactor_block_cleanup_started_total counter + # HELP cortex_compactor_block_cleanup_started_total Total number of blocks cleanup runs started. + cortex_compactor_block_cleanup_started_total 1 + + # TYPE cortex_compactor_block_cleanup_completed_total counter + # HELP cortex_compactor_block_cleanup_completed_total Total number of blocks cleanup runs successfully completed. + cortex_compactor_block_cleanup_completed_total 1 + + # TYPE cortex_compactor_block_cleanup_failed_total counter + # HELP cortex_compactor_block_cleanup_failed_total Total number of blocks cleanup runs failed. + cortex_compactor_block_cleanup_failed_total 0 `), testedMetrics...)) } @@ -1092,49 +1094,49 @@ func TestMultitenantCompactor_ShouldCompactAllUsersOnShardingEnabledButOnlyOneIn }, removeIgnoredLogs(strings.Split(strings.TrimSpace(logs.String()), "\n"))) assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` - # TYPE pyroscope_compactor_runs_started_total counter - # HELP pyroscope_compactor_runs_started_total Total number of compaction runs started. - pyroscope_compactor_runs_started_total 1 - - # TYPE pyroscope_compactor_runs_completed_total counter - # HELP pyroscope_compactor_runs_completed_total Total number of compaction runs successfully completed. - pyroscope_compactor_runs_completed_total 1 - - # TYPE pyroscope_compactor_runs_failed_total counter - # HELP pyroscope_compactor_runs_failed_total Total number of compaction runs failed. - pyroscope_compactor_runs_failed_total{reason="error"} 0 - pyroscope_compactor_runs_failed_total{reason="shutdown"} 0 - - # HELP pyroscope_compactor_group_compaction_runs_completed_total Total number of group completed compaction runs. This also includes compactor group runs that resulted with no compaction. - # TYPE pyroscope_compactor_group_compaction_runs_completed_total counter - pyroscope_compactor_group_compaction_runs_completed_total 2 - - # HELP pyroscope_compactor_group_compaction_runs_started_total Total number of group compaction attempts. - # TYPE pyroscope_compactor_group_compaction_runs_started_total counter - pyroscope_compactor_group_compaction_runs_started_total 2 - - # HELP pyroscope_compactor_group_compactions_failures_total Total number of failed group compactions. - # TYPE pyroscope_compactor_group_compactions_failures_total counter - pyroscope_compactor_group_compactions_failures_total 0 - - # HELP pyroscope_compactor_group_compactions_total Total number of group compaction attempts that resulted in new block(s). - # TYPE pyroscope_compactor_group_compactions_total counter - pyroscope_compactor_group_compactions_total 0 - - # HELP pyroscope_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. - # TYPE pyroscope_compactor_blocks_marked_for_deletion_total counter - pyroscope_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 - pyroscope_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 - pyroscope_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 + # TYPE cortex_compactor_runs_started_total counter + # HELP cortex_compactor_runs_started_total Total number of compaction runs started. + cortex_compactor_runs_started_total 1 + + # TYPE cortex_compactor_runs_completed_total counter + # HELP cortex_compactor_runs_completed_total Total number of compaction runs successfully completed. + cortex_compactor_runs_completed_total 1 + + # TYPE cortex_compactor_runs_failed_total counter + # HELP cortex_compactor_runs_failed_total Total number of compaction runs failed. + cortex_compactor_runs_failed_total{reason="error"} 0 + cortex_compactor_runs_failed_total{reason="shutdown"} 0 + + # HELP cortex_compactor_group_compaction_runs_completed_total Total number of group completed compaction runs. This also includes compactor group runs that resulted with no compaction. + # TYPE cortex_compactor_group_compaction_runs_completed_total counter + cortex_compactor_group_compaction_runs_completed_total 2 + + # HELP cortex_compactor_group_compaction_runs_started_total Total number of group compaction attempts. + # TYPE cortex_compactor_group_compaction_runs_started_total counter + cortex_compactor_group_compaction_runs_started_total 2 + + # HELP cortex_compactor_group_compactions_failures_total Total number of failed group compactions. + # TYPE cortex_compactor_group_compactions_failures_total counter + cortex_compactor_group_compactions_failures_total 0 + + # HELP cortex_compactor_group_compactions_total Total number of group compaction attempts that resulted in new block(s). + # TYPE cortex_compactor_group_compactions_total counter + cortex_compactor_group_compactions_total 0 + + # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. + # TYPE cortex_compactor_blocks_marked_for_deletion_total counter + cortex_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 + cortex_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 + cortex_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 `), - "pyroscope_compactor_runs_started_total", - "pyroscope_compactor_runs_completed_total", - "pyroscope_compactor_runs_failed_total", - "pyroscope_compactor_group_compaction_runs_completed_total", - "pyroscope_compactor_group_compaction_runs_started_total", - "pyroscope_compactor_group_compactions_failures_total", - "pyroscope_compactor_group_compactions_total", - "pyroscope_compactor_blocks_marked_for_deletion_total", + "cortex_compactor_runs_started_total", + "cortex_compactor_runs_completed_total", + "cortex_compactor_runs_failed_total", + "cortex_compactor_group_compaction_runs_completed_total", + "cortex_compactor_group_compaction_runs_started_total", + "cortex_compactor_group_compactions_failures_total", + "cortex_compactor_group_compactions_total", + "cortex_compactor_blocks_marked_for_deletion_total", )) } @@ -1404,49 +1406,49 @@ func TestMultitenantCompactor_ShouldSkipCompactionForJobsNoMoreOwnedAfterPlannin }, removeIgnoredLogs(strings.Split(strings.TrimSpace(logs.String()), "\n"))) assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` - # TYPE pyroscope_compactor_runs_started_total counter - # HELP pyroscope_compactor_runs_started_total Total number of compaction runs started. - pyroscope_compactor_runs_started_total 1 - - # TYPE pyroscope_compactor_runs_completed_total counter - # HELP pyroscope_compactor_runs_completed_total Total number of compaction runs successfully completed. - pyroscope_compactor_runs_completed_total 1 - - # TYPE pyroscope_compactor_runs_failed_total counter - # HELP pyroscope_compactor_runs_failed_total Total number of compaction runs failed. - pyroscope_compactor_runs_failed_total{reason="error"} 0 - pyroscope_compactor_runs_failed_total{reason="shutdown"} 0 - - # HELP pyroscope_compactor_group_compaction_runs_completed_total Total number of group completed compaction runs. This also includes compactor group runs that resulted with no compaction. - # TYPE pyroscope_compactor_group_compaction_runs_completed_total counter - pyroscope_compactor_group_compaction_runs_completed_total 1 - - # HELP pyroscope_compactor_group_compaction_runs_started_total Total number of group compaction attempts. - # TYPE pyroscope_compactor_group_compaction_runs_started_total counter - pyroscope_compactor_group_compaction_runs_started_total 1 - - # HELP pyroscope_compactor_group_compactions_failures_total Total number of failed group compactions. - # TYPE pyroscope_compactor_group_compactions_failures_total counter - pyroscope_compactor_group_compactions_failures_total 0 - - # HELP pyroscope_compactor_group_compactions_total Total number of group compaction attempts that resulted in new block(s). - # TYPE pyroscope_compactor_group_compactions_total counter - pyroscope_compactor_group_compactions_total 0 - - # HELP pyroscope_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. - # TYPE pyroscope_compactor_blocks_marked_for_deletion_total counter - pyroscope_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 - pyroscope_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 - pyroscope_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 + # TYPE cortex_compactor_runs_started_total counter + # HELP cortex_compactor_runs_started_total Total number of compaction runs started. + cortex_compactor_runs_started_total 1 + + # TYPE cortex_compactor_runs_completed_total counter + # HELP cortex_compactor_runs_completed_total Total number of compaction runs successfully completed. + cortex_compactor_runs_completed_total 1 + + # TYPE cortex_compactor_runs_failed_total counter + # HELP cortex_compactor_runs_failed_total Total number of compaction runs failed. + cortex_compactor_runs_failed_total{reason="error"} 0 + cortex_compactor_runs_failed_total{reason="shutdown"} 0 + + # HELP cortex_compactor_group_compaction_runs_completed_total Total number of group completed compaction runs. This also includes compactor group runs that resulted with no compaction. + # TYPE cortex_compactor_group_compaction_runs_completed_total counter + cortex_compactor_group_compaction_runs_completed_total 1 + + # HELP cortex_compactor_group_compaction_runs_started_total Total number of group compaction attempts. + # TYPE cortex_compactor_group_compaction_runs_started_total counter + cortex_compactor_group_compaction_runs_started_total 1 + + # HELP cortex_compactor_group_compactions_failures_total Total number of failed group compactions. + # TYPE cortex_compactor_group_compactions_failures_total counter + cortex_compactor_group_compactions_failures_total 0 + + # HELP cortex_compactor_group_compactions_total Total number of group compaction attempts that resulted in new block(s). + # TYPE cortex_compactor_group_compactions_total counter + cortex_compactor_group_compactions_total 0 + + # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. + # TYPE cortex_compactor_blocks_marked_for_deletion_total counter + cortex_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 + cortex_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 + cortex_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 `), - "pyroscope_compactor_runs_started_total", - "pyroscope_compactor_runs_completed_total", - "pyroscope_compactor_runs_failed_total", - "pyroscope_compactor_group_compaction_runs_completed_total", - "pyroscope_compactor_group_compaction_runs_started_total", - "pyroscope_compactor_group_compactions_failures_total", - "pyroscope_compactor_group_compactions_total", - "pyroscope_compactor_blocks_marked_for_deletion_total", + "cortex_compactor_runs_started_total", + "cortex_compactor_runs_completed_total", + "cortex_compactor_runs_failed_total", + "cortex_compactor_group_compaction_runs_completed_total", + "cortex_compactor_group_compaction_runs_started_total", + "cortex_compactor_group_compactions_failures_total", + "cortex_compactor_group_compactions_total", + "cortex_compactor_blocks_marked_for_deletion_total", )) } @@ -1514,94 +1516,63 @@ func TestMultitenantCompactor_ShouldSkipCompactionForJobsWithFirstLevelCompactio fmt.Sprintf(`level=info component=compactor user=user-2 msg="skipping compaction job because blocks in this job were uploaded too recently (within wait period)" groupKey=0@17241709254077376921-merge--1574776800000-1574784000000 waitPeriodNotElapsedFor="%s (min time: 1574776800000, max time: 1574784000000)"`, user2Meta2.ULID.String())) assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` - # TYPE pyroscope_compactor_runs_started_total counter - # HELP pyroscope_compactor_runs_started_total Total number of compaction runs started. - pyroscope_compactor_runs_started_total 1 - - # TYPE pyroscope_compactor_runs_completed_total counter - # HELP pyroscope_compactor_runs_completed_total Total number of compaction runs successfully completed. - pyroscope_compactor_runs_completed_total 1 - - # TYPE pyroscope_compactor_runs_failed_total counter - # HELP pyroscope_compactor_runs_failed_total Total number of compaction runs failed. - pyroscope_compactor_runs_failed_total{reason="error"} 0 - pyroscope_compactor_runs_failed_total{reason="shutdown"} 0 - - # HELP pyroscope_compactor_group_compaction_runs_completed_total Total number of group completed compaction runs. This also includes compactor group runs that resulted with no compaction. - # TYPE pyroscope_compactor_group_compaction_runs_completed_total counter - pyroscope_compactor_group_compaction_runs_completed_total 1 - - # HELP pyroscope_compactor_group_compaction_runs_started_total Total number of group compaction attempts. - # TYPE pyroscope_compactor_group_compaction_runs_started_total counter - pyroscope_compactor_group_compaction_runs_started_total 1 - - # HELP pyroscope_compactor_group_compactions_failures_total Total number of failed group compactions. - # TYPE pyroscope_compactor_group_compactions_failures_total counter - pyroscope_compactor_group_compactions_failures_total 0 - - # HELP pyroscope_compactor_group_compactions_total Total number of group compaction attempts that resulted in new block(s). - # TYPE pyroscope_compactor_group_compactions_total counter - pyroscope_compactor_group_compactions_total 0 - - # HELP pyroscope_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. - # TYPE pyroscope_compactor_blocks_marked_for_deletion_total counter - pyroscope_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 - pyroscope_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 - pyroscope_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 + # TYPE cortex_compactor_runs_started_total counter + # HELP cortex_compactor_runs_started_total Total number of compaction runs started. + cortex_compactor_runs_started_total 1 + + # TYPE cortex_compactor_runs_completed_total counter + # HELP cortex_compactor_runs_completed_total Total number of compaction runs successfully completed. + cortex_compactor_runs_completed_total 1 + + # TYPE cortex_compactor_runs_failed_total counter + # HELP cortex_compactor_runs_failed_total Total number of compaction runs failed. + cortex_compactor_runs_failed_total{reason="error"} 0 + cortex_compactor_runs_failed_total{reason="shutdown"} 0 + + # HELP cortex_compactor_group_compaction_runs_completed_total Total number of group completed compaction runs. This also includes compactor group runs that resulted with no compaction. + # TYPE cortex_compactor_group_compaction_runs_completed_total counter + cortex_compactor_group_compaction_runs_completed_total 1 + + # HELP cortex_compactor_group_compaction_runs_started_total Total number of group compaction attempts. + # TYPE cortex_compactor_group_compaction_runs_started_total counter + cortex_compactor_group_compaction_runs_started_total 1 + + # HELP cortex_compactor_group_compactions_failures_total Total number of failed group compactions. + # TYPE cortex_compactor_group_compactions_failures_total counter + cortex_compactor_group_compactions_failures_total 0 + + # HELP cortex_compactor_group_compactions_total Total number of group compaction attempts that resulted in new block(s). + # TYPE cortex_compactor_group_compactions_total counter + cortex_compactor_group_compactions_total 0 + + # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. + # TYPE cortex_compactor_blocks_marked_for_deletion_total counter + cortex_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 + cortex_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 + cortex_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 `), - "pyroscope_compactor_runs_started_total", - "pyroscope_compactor_runs_completed_total", - "pyroscope_compactor_runs_failed_total", - "pyroscope_compactor_group_compaction_runs_completed_total", - "pyroscope_compactor_group_compaction_runs_started_total", - "pyroscope_compactor_group_compactions_failures_total", - "pyroscope_compactor_group_compactions_total", - "pyroscope_compactor_blocks_marked_for_deletion_total", + "cortex_compactor_runs_started_total", + "cortex_compactor_runs_completed_total", + "cortex_compactor_runs_failed_total", + "cortex_compactor_group_compaction_runs_completed_total", + "cortex_compactor_group_compaction_runs_started_total", + "cortex_compactor_group_compactions_failures_total", + "cortex_compactor_group_compactions_total", + "cortex_compactor_blocks_marked_for_deletion_total", )) } -func createCustomTSDBBlock(t *testing.T, bkt objstore.Bucket, userID string, externalLabels map[string]string, appendFunc func(*tsdb.DB)) ulid.ULID { - // Create a temporary dir for TSDB. - tempDir := t.TempDir() +func createCustomBlock(t *testing.T, bkt objstore.Bucket, userID string, externalLabels map[string]string, generator func() []*testhelper.ProfileBuilder) ulid.ULID { + meta, dir := testutil.CreateBlock(t, generator) + blockLocalPath := filepath.Join(dir, meta.ULID.String()) - // Create a temporary dir for the snapshot. - snapshotDir := t.TempDir() - - // Create a new TSDB. - db, err := tsdb.Open(tempDir, nil, nil, &tsdb.Options{ - MinBlockDuration: int64(2 * 60 * 60 * 1000), // 2h period - MaxBlockDuration: int64(2 * 60 * 60 * 1000), // 2h period - RetentionDuration: int64(15 * 86400 * 1000), // 15 days - }, nil) - require.NoError(t, err) - - db.DisableCompactions() - - appendFunc(db) - - require.NoError(t, db.Compact()) - require.NoError(t, db.Snapshot(snapshotDir, true)) - - // Look for the created block (we expect one). - entries, err := os.ReadDir(snapshotDir) - require.NoError(t, err) - require.Len(t, entries, 1) - require.True(t, entries[0].IsDir()) - - blockID, err := ulid.Parse(entries[0].Name()) - require.NoError(t, err) - - // Inject Thanos external labels to the block. - meta := block.ThanosMeta{ - Labels: externalLabels, - Source: "test", - } - _, err = block.InjectThanosMeta(log.NewNopLogger(), filepath.Join(snapshotDir, blockID.String()), meta, nil) + meta.Source = "test" + meta.Labels = externalLabels + _, err := meta.WriteToFile(log.NewNopLogger(), blockLocalPath) require.NoError(t, err) // Copy the block files to the bucket. - srcRoot := filepath.Join(snapshotDir, blockID.String()) - require.NoError(t, filepath.Walk(srcRoot, func(file string, info os.FileInfo, err error) error { + require.NoError(t, filepath.Walk(blockLocalPath, func(file string, info os.FileInfo, err error) error { if err != nil { return err } @@ -1616,28 +1587,27 @@ func createCustomTSDBBlock(t *testing.T, bkt objstore.Bucket, userID string, ext } // Upload it to the bucket. - relPath, err := filepath.Rel(srcRoot, file) + relPath, err := filepath.Rel(blockLocalPath, file) if err != nil { return err } - return bkt.Upload(context.Background(), path.Join(userID, blockID.String(), relPath), bytes.NewReader(content)) + return bkt.Upload(context.Background(), path.Join(userID, meta.ULID.String(), relPath), bytes.NewReader(content)) })) - return blockID + return meta.ULID } func createTSDBBlock(t *testing.T, bkt objstore.Bucket, userID string, minT, maxT int64, numSeries int, externalLabels map[string]string) ulid.ULID { - return createCustomTSDBBlock(t, bkt, userID, externalLabels, func(db *tsdb.DB) { + return createCustomBlock(t, bkt, userID, externalLabels, func() []*testhelper.ProfileBuilder { + result := []*testhelper.ProfileBuilder{} appendSample := func(seriesID int, ts int64, value float64) { - lbls := labels.FromStrings("series_id", strconv.Itoa(seriesID)) - - app := db.Appender(context.Background()) - _, err := app.Append(0, lbls, ts, value) - require.NoError(t, err) - - err = app.Commit() - require.NoError(t, err) + profile := testhelper.NewProfileBuilder(ts*int64(time.Millisecond)). + CPUProfile(). + WithLabels( + "series_id", strconv.Itoa(seriesID), + ).ForStacktraceString("foo", "bar", "baz").AddSamples(1) + result = append(result, profile) } seriesID := 0 @@ -1653,6 +1623,7 @@ func createTSDBBlock(t *testing.T, bkt objstore.Bucket, userID string, minT, max // Guarantee a series with a sample at time maxT-1 appendSample(seriesID, maxT-1, float64(seriesID)) + return result }) } @@ -2242,11 +2213,11 @@ func TestMultitenantCompactor_OutOfOrderCompaction(t *testing.T) { require.Equal(t, block.NoCompactReason(block.OutOfOrderChunksNoCompactReason), m.Reason) assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` - # HELP pyroscope_compactor_blocks_marked_for_no_compaction_total Total number of blocks that were marked for no-compaction. - # TYPE pyroscope_compactor_blocks_marked_for_no_compaction_total counter - pyroscope_compactor_blocks_marked_for_no_compaction_total{reason="block-index-out-of-order-chunk"} 1 + # HELP cortex_compactor_blocks_marked_for_no_compaction_total Total number of blocks that were marked for no-compaction. + # TYPE cortex_compactor_blocks_marked_for_no_compaction_total counter + cortex_compactor_blocks_marked_for_no_compaction_total{reason="block-index-out-of-order-chunk"} 1 `), - "pyroscope_compactor_blocks_marked_for_no_compaction_total", + "cortex_compactor_blocks_marked_for_no_compaction_total", )) } diff --git a/pkg/compactor/split_merge_compactor_test.go b/pkg/compactor/split_merge_compactor_test.go index caee820b71..fa08ca0ef2 100644 --- a/pkg/compactor/split_merge_compactor_test.go +++ b/pkg/compactor/split_merge_compactor_test.go @@ -8,30 +8,29 @@ package compactor import ( "context" "os" - "path/filepath" "strconv" "strings" "testing" "time" "github.com/go-kit/log" - "github.com/grafana/dskit/flagext" "github.com/grafana/dskit/services" "github.com/grafana/dskit/test" "github.com/oklog/ulid" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/testutil" - "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/common/model" "github.com/prometheus/prometheus/tsdb" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" - "github.com/thanos-io/objstore" - "github.com/grafana/mimir/pkg/storage/bucket" - "github.com/grafana/mimir/pkg/storage/sharding" - mimir_tsdb "github.com/grafana/mimir/pkg/storage/tsdb" - "github.com/grafana/mimir/pkg/storage/tsdb/block" - util_test "github.com/grafana/mimir/pkg/util/test" + phlaremodel "github.com/grafana/pyroscope/pkg/model" + "github.com/grafana/pyroscope/pkg/objstore" + "github.com/grafana/pyroscope/pkg/objstore/client" + "github.com/grafana/pyroscope/pkg/objstore/providers/filesystem" + "github.com/grafana/pyroscope/pkg/phlaredb" + "github.com/grafana/pyroscope/pkg/phlaredb/block" + "github.com/grafana/pyroscope/pkg/phlaredb/sharding" ) func TestMultitenantCompactor_ShouldSupportSplitAndMergeCompactor(t *testing.T) { @@ -43,21 +42,21 @@ func TestMultitenantCompactor_ShouldSupportSplitAndMergeCompactor(t *testing.T) var ( blockRangeMillis = blockRange.Milliseconds() - compactionRanges = mimir_tsdb.DurationList{blockRange, 2 * blockRange, 4 * blockRange} + compactionRanges = DurationList{blockRange, 2 * blockRange, 4 * blockRange} ) externalLabels := func(shardID string) map[string]string { labels := map[string]string{} if shardID != "" { - labels[mimir_tsdb.CompactorShardIDExternalLabel] = shardID + labels[sharding.CompactorShardIDLabel] = shardID } return labels } externalLabelsWithTenantID := func(shardID string) map[string]string { labels := externalLabels(shardID) - labels[mimir_tsdb.DeprecatedTenantIDExternalLabel] = userID + labels["__org_id__"] = userID return labels } @@ -73,30 +72,24 @@ func TestMultitenantCompactor_ShouldSupportSplitAndMergeCompactor(t *testing.T) return []block.Meta{ { - BlockMeta: tsdb.BlockMeta{ - MinTime: 1 * blockRangeMillis, - MaxTime: 2 * blockRangeMillis, - Compaction: tsdb.BlockMetaCompaction{ - Sources: []ulid.ULID{block1, block2}, - }, - }, - Thanos: block.ThanosMeta{ - Labels: map[string]string{ - mimir_tsdb.CompactorShardIDExternalLabel: "1_of_2", - }, + MinTime: model.Time(1 * blockRangeMillis), + MaxTime: model.Time(2 * blockRangeMillis), + Compaction: tsdb.BlockMetaCompaction{ + Sources: []ulid.ULID{block1, block2}, + }, + + Labels: map[string]string{ + sharding.CompactorShardIDLabel: "1_of_2", }, }, { - BlockMeta: tsdb.BlockMeta{ - MinTime: 1 * blockRangeMillis, - MaxTime: 2 * blockRangeMillis, - Compaction: tsdb.BlockMetaCompaction{ - Sources: []ulid.ULID{block1, block2}, - }, - }, - Thanos: block.ThanosMeta{ - Labels: map[string]string{ - mimir_tsdb.CompactorShardIDExternalLabel: "2_of_2", - }, + MinTime: model.Time(1 * blockRangeMillis), + MaxTime: model.Time(2 * blockRangeMillis), + Compaction: tsdb.BlockMetaCompaction{ + Sources: []ulid.ULID{block1, block2}, + }, + + Labels: map[string]string{ + sharding.CompactorShardIDLabel: "2_of_2", }, }, } @@ -110,30 +103,24 @@ func TestMultitenantCompactor_ShouldSupportSplitAndMergeCompactor(t *testing.T) return []block.Meta{ { - BlockMeta: tsdb.BlockMeta{ - MinTime: 1 * blockRangeMillis, - MaxTime: 2 * blockRangeMillis, - Compaction: tsdb.BlockMetaCompaction{ - Sources: []ulid.ULID{block1, block2}, - }, - }, - Thanos: block.ThanosMeta{ - Labels: map[string]string{ - mimir_tsdb.CompactorShardIDExternalLabel: "1_of_2", - }, + MinTime: model.Time(1 * blockRangeMillis), + MaxTime: model.Time(2 * blockRangeMillis), + Compaction: tsdb.BlockMetaCompaction{ + Sources: []ulid.ULID{block1, block2}, + }, + + Labels: map[string]string{ + sharding.CompactorShardIDLabel: "1_of_2", }, }, { - BlockMeta: tsdb.BlockMeta{ - MinTime: 1 * blockRangeMillis, - MaxTime: 2 * blockRangeMillis, - Compaction: tsdb.BlockMetaCompaction{ - Sources: []ulid.ULID{block1, block2}, - }, - }, - Thanos: block.ThanosMeta{ - Labels: map[string]string{ - mimir_tsdb.CompactorShardIDExternalLabel: "2_of_2", - }, + MinTime: model.Time(1 * blockRangeMillis), + MaxTime: model.Time(2 * blockRangeMillis), + Compaction: tsdb.BlockMetaCompaction{ + Sources: []ulid.ULID{block1, block2}, + }, + + Labels: map[string]string{ + sharding.CompactorShardIDLabel: "2_of_2", }, }, } @@ -151,43 +138,34 @@ func TestMultitenantCompactor_ShouldSupportSplitAndMergeCompactor(t *testing.T) return []block.Meta{ { - BlockMeta: tsdb.BlockMeta{ - MinTime: 0, - MaxTime: (7 * time.Minute).Milliseconds(), - Compaction: tsdb.BlockMetaCompaction{ - Sources: []ulid.ULID{block1, block2}, - }, - }, - Thanos: block.ThanosMeta{ - Labels: map[string]string{ - mimir_tsdb.CompactorShardIDExternalLabel: "1_of_2", - }, + MinTime: 0, + MaxTime: model.Time((7 * time.Minute).Milliseconds()), + Compaction: tsdb.BlockMetaCompaction{ + Sources: []ulid.ULID{block1, block2}, + }, + + Labels: map[string]string{ + sharding.CompactorShardIDLabel: "1_of_2", }, }, { - BlockMeta: tsdb.BlockMeta{ - MinTime: 0, - MaxTime: (7 * time.Minute).Milliseconds(), - Compaction: tsdb.BlockMetaCompaction{ - Sources: []ulid.ULID{block1, block2}, - }, - }, - Thanos: block.ThanosMeta{ - Labels: map[string]string{ - mimir_tsdb.CompactorShardIDExternalLabel: "2_of_2", - }, + MinTime: 0, + MaxTime: model.Time((7 * time.Minute).Milliseconds()), + Compaction: tsdb.BlockMetaCompaction{ + Sources: []ulid.ULID{block1, block2}, + }, + + Labels: map[string]string{ + sharding.CompactorShardIDLabel: "2_of_2", }, }, { // Not compacted. - BlockMeta: tsdb.BlockMeta{ - MinTime: blockRangeMillis, - MaxTime: blockRangeMillis + time.Minute.Milliseconds(), - Compaction: tsdb.BlockMetaCompaction{ - Sources: []ulid.ULID{block3}, - }, - }, - Thanos: block.ThanosMeta{ - Labels: map[string]string{}, + MinTime: model.Time(blockRangeMillis), + MaxTime: model.Time(blockRangeMillis + time.Minute.Milliseconds()), + Compaction: tsdb.BlockMetaCompaction{ + Sources: []ulid.ULID{block3}, }, + + Labels: map[string]string{}, }, } }, @@ -204,43 +182,34 @@ func TestMultitenantCompactor_ShouldSupportSplitAndMergeCompactor(t *testing.T) return []block.Meta{ { - BlockMeta: tsdb.BlockMeta{ - MinTime: 0, - MaxTime: (10 * time.Minute).Milliseconds(), - Compaction: tsdb.BlockMetaCompaction{ - Sources: []ulid.ULID{block1, block2}, - }, - }, - Thanos: block.ThanosMeta{ - Labels: map[string]string{ - mimir_tsdb.CompactorShardIDExternalLabel: "1_of_2", - }, + MinTime: 0, + MaxTime: model.Time((10 * time.Minute).Milliseconds()), + Compaction: tsdb.BlockMetaCompaction{ + Sources: []ulid.ULID{block1, block2}, + }, + + Labels: map[string]string{ + sharding.CompactorShardIDLabel: "1_of_2", }, }, { - BlockMeta: tsdb.BlockMeta{ - MinTime: 0, - MaxTime: (10 * time.Minute).Milliseconds(), - Compaction: tsdb.BlockMetaCompaction{ - Sources: []ulid.ULID{block1, block2}, - }, - }, - Thanos: block.ThanosMeta{ - Labels: map[string]string{ - mimir_tsdb.CompactorShardIDExternalLabel: "2_of_2", - }, + MinTime: 0, + MaxTime: model.Time((10 * time.Minute).Milliseconds()), + Compaction: tsdb.BlockMetaCompaction{ + Sources: []ulid.ULID{block1, block2}, + }, + + Labels: map[string]string{ + sharding.CompactorShardIDLabel: "2_of_2", }, }, { // Not compacted. - BlockMeta: tsdb.BlockMeta{ - MinTime: blockRangeMillis, - MaxTime: blockRangeMillis + time.Minute.Milliseconds(), - Compaction: tsdb.BlockMetaCompaction{ - Sources: []ulid.ULID{block3}, - }, - }, - Thanos: block.ThanosMeta{ - Labels: map[string]string{}, + MinTime: model.Time(blockRangeMillis), + MaxTime: model.Time(blockRangeMillis + time.Minute.Milliseconds()), + Compaction: tsdb.BlockMetaCompaction{ + Sources: []ulid.ULID{block3}, }, + + Labels: map[string]string{}, }, } }, @@ -257,43 +226,34 @@ func TestMultitenantCompactor_ShouldSupportSplitAndMergeCompactor(t *testing.T) return []block.Meta{ { - BlockMeta: tsdb.BlockMeta{ - MinTime: 0, - MaxTime: (10 * time.Minute).Milliseconds(), - Compaction: tsdb.BlockMetaCompaction{ - Sources: []ulid.ULID{block1, block2}, - }, - }, - Thanos: block.ThanosMeta{ - Labels: map[string]string{ - mimir_tsdb.CompactorShardIDExternalLabel: "1_of_2", - }, + MinTime: 0, + MaxTime: model.Time((10 * time.Minute).Milliseconds()), + Compaction: tsdb.BlockMetaCompaction{ + Sources: []ulid.ULID{block1, block2}, + }, + + Labels: map[string]string{ + sharding.CompactorShardIDLabel: "1_of_2", }, }, { - BlockMeta: tsdb.BlockMeta{ - MinTime: 0, - MaxTime: (10 * time.Minute).Milliseconds(), - Compaction: tsdb.BlockMetaCompaction{ - Sources: []ulid.ULID{block1, block2}, - }, - }, - Thanos: block.ThanosMeta{ - Labels: map[string]string{ - mimir_tsdb.CompactorShardIDExternalLabel: "2_of_2", - }, + MinTime: 0, + MaxTime: model.Time((10 * time.Minute).Milliseconds()), + Compaction: tsdb.BlockMetaCompaction{ + Sources: []ulid.ULID{block1, block2}, + }, + + Labels: map[string]string{ + sharding.CompactorShardIDLabel: "2_of_2", }, }, { // Not compacted. - BlockMeta: tsdb.BlockMeta{ - MinTime: blockRangeMillis, - MaxTime: blockRangeMillis + time.Minute.Milliseconds(), - Compaction: tsdb.BlockMetaCompaction{ - Sources: []ulid.ULID{block3}, - }, - }, - Thanos: block.ThanosMeta{ - Labels: map[string]string{}, + MinTime: model.Time(blockRangeMillis), + MaxTime: model.Time(blockRangeMillis + time.Minute.Milliseconds()), + Compaction: tsdb.BlockMetaCompaction{ + Sources: []ulid.ULID{block3}, }, + + Labels: map[string]string{}, }, } }, @@ -324,61 +284,49 @@ func TestMultitenantCompactor_ShouldSupportSplitAndMergeCompactor(t *testing.T) // and then compacted with block1 in 2nd range. Finally, they've been compacted with // block4 and block5 in the 3rd range compaction (total levels: 4). { - BlockMeta: tsdb.BlockMeta{ - MinTime: 1, - MaxTime: 4 * blockRangeMillis, - Compaction: tsdb.BlockMetaCompaction{ - Sources: []ulid.ULID{block1a, block2, block3, block4a, block5a}, - }, - }, - Thanos: block.ThanosMeta{ - Labels: map[string]string{ - mimir_tsdb.CompactorShardIDExternalLabel: "1_of_2", - }, + MinTime: 1, + MaxTime: model.Time(4 * blockRangeMillis), + Compaction: tsdb.BlockMetaCompaction{ + Sources: []ulid.ULID{block1a, block2, block3, block4a, block5a}, + }, + + Labels: map[string]string{ + sharding.CompactorShardIDLabel: "1_of_2", }, }, { - BlockMeta: tsdb.BlockMeta{ - MinTime: 1, - MaxTime: 4 * blockRangeMillis, - Compaction: tsdb.BlockMetaCompaction{ - Sources: []ulid.ULID{block1b, block2, block3, block4b, block5b}, - }, - }, - Thanos: block.ThanosMeta{ - Labels: map[string]string{ - mimir_tsdb.CompactorShardIDExternalLabel: "2_of_2", - }, + MinTime: 1, + MaxTime: model.Time(4 * blockRangeMillis), + Compaction: tsdb.BlockMetaCompaction{ + Sources: []ulid.ULID{block1b, block2, block3, block4b, block5b}, + }, + + Labels: map[string]string{ + sharding.CompactorShardIDLabel: "2_of_2", }, }, // The two non-adjacent blocks block6 and block7 are split individually first and then merged // together in the 3rd range. { - BlockMeta: tsdb.BlockMeta{ - MinTime: 4 * blockRangeMillis, - MaxTime: 8 * blockRangeMillis, - Compaction: tsdb.BlockMetaCompaction{ - Sources: []ulid.ULID{block6, block7}, - }, - }, - Thanos: block.ThanosMeta{ - Labels: map[string]string{ - mimir_tsdb.CompactorShardIDExternalLabel: "1_of_2", - }, + MinTime: model.Time(4 * blockRangeMillis), + MaxTime: model.Time(8 * blockRangeMillis), + Compaction: tsdb.BlockMetaCompaction{ + Sources: []ulid.ULID{block6, block7}, + }, + + Labels: map[string]string{ + sharding.CompactorShardIDLabel: "1_of_2", }, }, { - BlockMeta: tsdb.BlockMeta{ - MinTime: 4 * blockRangeMillis, - MaxTime: 8 * blockRangeMillis, - Compaction: tsdb.BlockMetaCompaction{ - Sources: []ulid.ULID{block6, block7}, - }, - }, - Thanos: block.ThanosMeta{ - Labels: map[string]string{ - mimir_tsdb.CompactorShardIDExternalLabel: "2_of_2", - }, + MinTime: model.Time(4 * blockRangeMillis), + MaxTime: model.Time(8 * blockRangeMillis), + Compaction: tsdb.BlockMetaCompaction{ + Sources: []ulid.ULID{block6, block7}, + }, + + Labels: map[string]string{ + sharding.CompactorShardIDLabel: "2_of_2", }, }, } @@ -396,30 +344,24 @@ func TestMultitenantCompactor_ShouldSupportSplitAndMergeCompactor(t *testing.T) return []block.Meta{ { - BlockMeta: tsdb.BlockMeta{ - MinTime: 0, - MaxTime: (2 * time.Hour).Milliseconds(), - Compaction: tsdb.BlockMetaCompaction{ - Sources: []ulid.ULID{block1, block2, block3}, - }, - }, - Thanos: block.ThanosMeta{ - Labels: map[string]string{ - mimir_tsdb.CompactorShardIDExternalLabel: "1_of_2", - }, + MinTime: 0, + MaxTime: model.Time((2 * time.Hour).Milliseconds()), + Compaction: tsdb.BlockMetaCompaction{ + Sources: []ulid.ULID{block1, block2, block3}, + }, + + Labels: map[string]string{ + sharding.CompactorShardIDLabel: "1_of_2", }, }, { - BlockMeta: tsdb.BlockMeta{ - MinTime: 0, - MaxTime: (2 * time.Hour).Milliseconds(), - Compaction: tsdb.BlockMetaCompaction{ - Sources: []ulid.ULID{block1, block2, block3}, - }, - }, - Thanos: block.ThanosMeta{ - Labels: map[string]string{ - mimir_tsdb.CompactorShardIDExternalLabel: "2_of_2", - }, + MinTime: 0, + MaxTime: model.Time((2 * time.Hour).Milliseconds()), + Compaction: tsdb.BlockMetaCompaction{ + Sources: []ulid.ULID{block1, block2, block3}, + }, + + Labels: map[string]string{ + sharding.CompactorShardIDLabel: "2_of_2", }, }, } @@ -437,30 +379,24 @@ func TestMultitenantCompactor_ShouldSupportSplitAndMergeCompactor(t *testing.T) return []block.Meta{ { - BlockMeta: tsdb.BlockMeta{ - MinTime: 0, - MaxTime: (2 * time.Hour).Milliseconds(), - Compaction: tsdb.BlockMetaCompaction{ - Sources: []ulid.ULID{block1, block2, block3}, - }, - }, - Thanos: block.ThanosMeta{ - Labels: map[string]string{ - mimir_tsdb.CompactorShardIDExternalLabel: "1_of_2", - }, + MinTime: 0, + MaxTime: model.Time((2 * time.Hour).Milliseconds()), + Compaction: tsdb.BlockMetaCompaction{ + Sources: []ulid.ULID{block1, block2, block3}, + }, + + Labels: map[string]string{ + sharding.CompactorShardIDLabel: "1_of_2", }, }, { - BlockMeta: tsdb.BlockMeta{ - MinTime: 0, - MaxTime: (2 * time.Hour).Milliseconds(), - Compaction: tsdb.BlockMetaCompaction{ - Sources: []ulid.ULID{block1, block2, block3}, - }, - }, - Thanos: block.ThanosMeta{ - Labels: map[string]string{ - mimir_tsdb.CompactorShardIDExternalLabel: "2_of_2", - }, + MinTime: 0, + MaxTime: model.Time((2 * time.Hour).Milliseconds()), + Compaction: tsdb.BlockMetaCompaction{ + Sources: []ulid.ULID{block1, block2, block3}, + }, + + Labels: map[string]string{ + sharding.CompactorShardIDLabel: "2_of_2", }, }, } @@ -475,17 +411,14 @@ func TestMultitenantCompactor_ShouldSupportSplitAndMergeCompactor(t *testing.T) return []block.Meta{ { - BlockMeta: tsdb.BlockMeta{ - MinTime: (2 * blockRangeMillis) - 1, // Because there's only 1 sample with timestamp=maxT-1 - MaxTime: 2 * blockRangeMillis, - Compaction: tsdb.BlockMetaCompaction{ - Sources: []ulid.ULID{block1}, - }, - }, - Thanos: block.ThanosMeta{ - Labels: map[string]string{ - mimir_tsdb.CompactorShardIDExternalLabel: "1_of_2", - }, + MinTime: model.Time((2 * blockRangeMillis) - 1), // Because there's only 1 sample with timestamp=maxT-1 + MaxTime: model.Time(2 * blockRangeMillis), + Compaction: tsdb.BlockMetaCompaction{ + Sources: []ulid.ULID{block1}, + }, + + Labels: map[string]string{ + sharding.CompactorShardIDLabel: "1_of_2", }, }, } @@ -504,28 +437,22 @@ func TestMultitenantCompactor_ShouldSupportSplitAndMergeCompactor(t *testing.T) return []block.Meta{ // Compacted but not split. { - BlockMeta: tsdb.BlockMeta{ - MinTime: 0, - MaxTime: (10 * time.Minute).Milliseconds(), - Compaction: tsdb.BlockMetaCompaction{ - Sources: []ulid.ULID{block1, block2}, - }, - }, - Thanos: block.ThanosMeta{ - Labels: map[string]string{}, + MinTime: 0, + MaxTime: model.Time((10 * time.Minute).Milliseconds()), + Compaction: tsdb.BlockMetaCompaction{ + Sources: []ulid.ULID{block1, block2}, }, + + Labels: map[string]string{}, }, { // Not compacted. - BlockMeta: tsdb.BlockMeta{ - MinTime: blockRangeMillis, - MaxTime: blockRangeMillis + time.Minute.Milliseconds(), - Compaction: tsdb.BlockMetaCompaction{ - Sources: []ulid.ULID{block3}, - }, - }, - Thanos: block.ThanosMeta{ - Labels: map[string]string{}, + MinTime: model.Time(blockRangeMillis), + MaxTime: model.Time(blockRangeMillis + time.Minute.Milliseconds()), + Compaction: tsdb.BlockMetaCompaction{ + Sources: []ulid.ULID{block3}, }, + + Labels: map[string]string{}, }, } }, @@ -554,123 +481,46 @@ func TestMultitenantCompactor_ShouldSupportSplitAndMergeCompactor(t *testing.T) return []block.Meta{ // Block1 have been compacted with block4 and block5 in the 3rd range compaction. { - BlockMeta: tsdb.BlockMeta{ - MinTime: 1, - MaxTime: 4 * blockRangeMillis, - Compaction: tsdb.BlockMetaCompaction{ - Sources: []ulid.ULID{block1a, block4a, block5a}, - }, - }, - Thanos: block.ThanosMeta{ - Labels: map[string]string{ - mimir_tsdb.CompactorShardIDExternalLabel: "1_of_2", - }, + MinTime: 1, + MaxTime: model.Time(4 * blockRangeMillis), + Compaction: tsdb.BlockMetaCompaction{ + Sources: []ulid.ULID{block1a, block4a, block5a}, }, - }, - { - BlockMeta: tsdb.BlockMeta{ - MinTime: 1, - MaxTime: 4 * blockRangeMillis, - Compaction: tsdb.BlockMetaCompaction{ - Sources: []ulid.ULID{block1b, block4b, block5b}, - }, - }, - Thanos: block.ThanosMeta{ - Labels: map[string]string{ - mimir_tsdb.CompactorShardIDExternalLabel: "2_of_2", - }, + + Labels: map[string]string{ + sharding.CompactorShardIDLabel: "1_of_2", }, }, - // The two overlapping blocks (block2, block3) have been merged in the 1st range. { - BlockMeta: tsdb.BlockMeta{ - MinTime: blockRangeMillis, - MaxTime: 2 * blockRangeMillis, - Compaction: tsdb.BlockMetaCompaction{ - Sources: []ulid.ULID{block2, block3}, - }, + MinTime: 1, + MaxTime: model.Time(4 * blockRangeMillis), + Compaction: tsdb.BlockMetaCompaction{ + Sources: []ulid.ULID{block1b, block4b, block5b}, }, - Thanos: block.ThanosMeta{ - Labels: map[string]string{}, + + Labels: map[string]string{ + sharding.CompactorShardIDLabel: "2_of_2", }, }, - // The two non-adjacent blocks block6 and block7 are merged together in the 3rd range. + // The two overlapping blocks (block2, block3) have been merged in the 1st range. { - BlockMeta: tsdb.BlockMeta{ - MinTime: 4 * blockRangeMillis, - MaxTime: 8 * blockRangeMillis, - Compaction: tsdb.BlockMetaCompaction{ - Sources: []ulid.ULID{block6, block7}, - }, + MinTime: model.Time(blockRangeMillis), + MaxTime: model.Time(2 * blockRangeMillis), + Compaction: tsdb.BlockMetaCompaction{ + Sources: []ulid.ULID{block2, block3}, }, - Thanos: block.ThanosMeta{ - Labels: map[string]string{}, - }, - }, - } - }, - }, - "compaction on blocks containing native histograms": { - numShards: 2, - setup: func(t *testing.T, bkt objstore.Bucket) []block.Meta { - minT := blockRangeMillis - maxT := 2 * blockRangeMillis - - seriesID := 0 - - appendHistograms := func(db *tsdb.DB) { - db.EnableNativeHistograms() - - appendHistogram := func(seriesID int, ts int64) { - lbls := labels.FromStrings("series_id", strconv.Itoa(seriesID)) - - app := db.Appender(context.Background()) - _, err := app.AppendHistogram(0, lbls, ts, util_test.GenerateTestHistogram(seriesID), nil) - require.NoError(t, err) - err = app.Commit() - require.NoError(t, err) - } - - for ts := minT; ts < maxT; ts += (maxT - minT) / int64(numSeries-1) { - appendHistogram(seriesID, ts) - seriesID++ - } - - appendHistogram(seriesID, maxT-1) - } - - block1 := createCustomTSDBBlock(t, bkt, userID, externalLabels(""), appendHistograms) - block2 := createCustomTSDBBlock(t, bkt, userID, externalLabels(""), appendHistograms) - - return []block.Meta{ - { - BlockMeta: tsdb.BlockMeta{ - MinTime: 1 * blockRangeMillis, - MaxTime: 2 * blockRangeMillis, - Compaction: tsdb.BlockMetaCompaction{ - Sources: []ulid.ULID{block1, block2}, - }, - }, - Thanos: block.ThanosMeta{ - Labels: map[string]string{ - mimir_tsdb.CompactorShardIDExternalLabel: "1_of_2", - }, - }, + Labels: map[string]string{}, }, + // The two non-adjacent blocks block6 and block7 are merged together in the 3rd range. { - BlockMeta: tsdb.BlockMeta{ - MinTime: 1 * blockRangeMillis, - MaxTime: 2 * blockRangeMillis, - Compaction: tsdb.BlockMetaCompaction{ - Sources: []ulid.ULID{block1, block2}, - }, - }, - Thanos: block.ThanosMeta{ - Labels: map[string]string{ - mimir_tsdb.CompactorShardIDExternalLabel: "2_of_2", - }, + MinTime: model.Time(4 * blockRangeMillis), + MaxTime: model.Time(8 * blockRangeMillis), + Compaction: tsdb.BlockMetaCompaction{ + Sources: []ulid.ULID{block6, block7}, }, + + Labels: map[string]string{}, }, } }, @@ -683,10 +533,14 @@ func TestMultitenantCompactor_ShouldSupportSplitAndMergeCompactor(t *testing.T) storageDir := t.TempDir() fetcherDir := t.TempDir() - storageCfg := mimir_tsdb.BlocksStorageConfig{} - flagext.DefaultValues(&storageCfg) - storageCfg.Bucket.Backend = bucket.Filesystem - storageCfg.Bucket.Filesystem.Directory = storageDir + storageCfg := client.Config{ + StorageBackendConfig: client.StorageBackendConfig{ + Backend: client.Filesystem, + Filesystem: filesystem.Config{ + Directory: storageDir, + }, + }, + } compactorCfg := prepareConfig(t) compactorCfg.DataDir = workDir @@ -700,11 +554,13 @@ func TestMultitenantCompactor_ShouldSupportSplitAndMergeCompactor(t *testing.T) ctx := context.Background() // Create TSDB blocks in the storage and get the expected blocks. - bucketClient, err := bucket.NewClient(ctx, storageCfg.Bucket, "test", logger, nil) + bkt, err := client.NewBucket(context.Background(), storageCfg, "test") require.NoError(t, err) - expected := testData.setup(t, bucketClient) - c, err := NewMultitenantCompactor(compactorCfg, storageCfg, cfgProvider, logger, reg) + defer bkt.Close() + expected := testData.setup(t, bkt) + + c, err := NewMultitenantCompactor(compactorCfg, bkt, cfgProvider, logger, reg) require.NoError(t, err) require.NoError(t, services.StartAndAwaitRunning(context.Background(), c)) t.Cleanup(func() { @@ -721,7 +577,7 @@ func TestMultitenantCompactor_ShouldSupportSplitAndMergeCompactor(t *testing.T) }) // List back any (non deleted) block from the storage. - userBucket := bucket.NewUserBucketClient(userID, bucketClient, nil) + userBucket := objstore.NewUserBucketClient(userID, bkt, nil) fetcher, err := block.NewMetaFetcher(logger, 1, userBucket, @@ -743,7 +599,7 @@ func TestMultitenantCompactor_ShouldSupportSplitAndMergeCompactor(t *testing.T) assert.Equal(t, e.MinTime, actual[i].MinTime) assert.Equal(t, e.MaxTime, actual[i].MaxTime) assert.Equal(t, e.Compaction.Sources, actual[i].Compaction.Sources) - assert.Equal(t, e.Thanos.Labels, actual[i].Thanos.Labels) + assert.Equal(t, e.Labels, actual[i].Labels) } }) } @@ -759,7 +615,7 @@ func TestMultitenantCompactor_ShouldGuaranteeSeriesShardingConsistencyOverTheTim var ( blockRangeMillis = blockRange.Milliseconds() - compactionRanges = mimir_tsdb.DurationList{blockRange} + compactionRanges = DurationList{blockRange} // You should NEVER CHANGE the expected series here, otherwise it means you're introducing // a backward incompatible change. @@ -773,10 +629,14 @@ func TestMultitenantCompactor_ShouldGuaranteeSeriesShardingConsistencyOverTheTim storageDir := t.TempDir() fetcherDir := t.TempDir() - storageCfg := mimir_tsdb.BlocksStorageConfig{} - flagext.DefaultValues(&storageCfg) - storageCfg.Bucket.Backend = bucket.Filesystem - storageCfg.Bucket.Filesystem.Directory = storageDir + storageCfg := client.Config{ + StorageBackendConfig: client.StorageBackendConfig{ + Backend: client.Filesystem, + Filesystem: filesystem.Config{ + Directory: storageDir, + }, + }, + } compactorCfg := prepareConfig(t) compactorCfg.DataDir = workDir @@ -789,13 +649,13 @@ func TestMultitenantCompactor_ShouldGuaranteeSeriesShardingConsistencyOverTheTim reg := prometheus.NewPedanticRegistry() ctx := context.Background() - bucketClient, err := bucket.NewClient(ctx, storageCfg.Bucket, "test", logger, nil) + bucketClient, err := client.NewBucket(ctx, storageCfg, "test") require.NoError(t, err) // Create a TSDB block in the storage. blockID := createTSDBBlock(t, bucketClient, userID, blockRangeMillis, 2*blockRangeMillis, numSeries, nil) - c, err := NewMultitenantCompactor(compactorCfg, storageCfg, cfgProvider, logger, reg) + c, err := NewMultitenantCompactor(compactorCfg, bucketClient, cfgProvider, logger, reg) require.NoError(t, err) require.NoError(t, services.StartAndAwaitRunning(context.Background(), c)) t.Cleanup(func() { @@ -812,7 +672,7 @@ func TestMultitenantCompactor_ShouldGuaranteeSeriesShardingConsistencyOverTheTim }) // List back any (non deleted) block from the storage. - userBucket := bucket.NewUserBucketClient(userID, bucketClient, nil) + userBucket := objstore.NewUserBucketClient(userID, bucketClient, nil) fetcher, err := block.NewMetaFetcher(logger, 1, userBucket, @@ -834,34 +694,34 @@ func TestMultitenantCompactor_ShouldGuaranteeSeriesShardingConsistencyOverTheTim assert.Equal(t, blockRangeMillis, actualMeta.MinTime) assert.Equal(t, 2*blockRangeMillis, actualMeta.MaxTime) assert.Equal(t, []ulid.ULID{blockID}, actualMeta.Compaction.Sources) - assert.Equal(t, sharding.FormatShardIDLabelValue(uint64(idx), numShards), actualMeta.Thanos.Labels[mimir_tsdb.CompactorShardIDExternalLabel]) + assert.Equal(t, sharding.FormatShardIDLabelValue(uint64(idx), numShards), actualMeta.Labels[sharding.CompactorShardIDLabel]) } // Ensure each split block contains the right series, based on a series labels // hashing function which doesn't change over time. for _, actualMeta := range actualMetas { - expectedSeriesIDs := expectedSeriesIDByShard[actualMeta.Thanos.Labels[mimir_tsdb.CompactorShardIDExternalLabel]] + expectedSeriesIDs := expectedSeriesIDByShard[actualMeta.Labels[sharding.CompactorShardIDLabel]] - b, err := tsdb.OpenBlock(logger, filepath.Join(storageDir, userID, actualMeta.ULID.String()), nil) - require.NoError(t, err) - - indexReader, err := b.Index() - require.NoError(t, err) + b := phlaredb.NewSingleBlockQuerierFromMeta(ctx, bucketClient, actualMeta) + require.NoError(t, b.Open(ctx)) + indexReader := b.Index() // Find all series in the block. - postings, err := indexReader.PostingsForMatchers(false, labels.MustNewMatcher(labels.MatchRegexp, "series_id", ".+")) + postings, err := indexReader.Postings("series_id", nil) require.NoError(t, err) - builder := labels.NewScratchBuilder(1) + lbls := make(phlaremodel.Labels, 0, 6) + for postings.Next() { + _, err := indexReader.Series(postings.At(), &lbls, nil) // Symbolize the series labels. - require.NoError(t, indexReader.Series(postings.At(), &builder, nil)) + require.NoError(t, err) // Ensure the series below to the right shard. - seriesLabels := builder.Labels() + seriesLabels := lbls.Clone() seriesID, err := strconv.Atoi(seriesLabels.Get("series_id")) require.NoError(t, err) - assert.Contains(t, expectedSeriesIDs, seriesID, "series:", seriesLabels.String()) + assert.Contains(t, expectedSeriesIDs, seriesID, "series:", seriesLabels.ToPrometheusLabels().String()) } require.NoError(t, postings.Err()) diff --git a/pkg/phlaredb/block/block_test.go b/pkg/phlaredb/block/block_test.go index f5b17a67a9..94832767be 100644 --- a/pkg/phlaredb/block/block_test.go +++ b/pkg/phlaredb/block/block_test.go @@ -84,7 +84,7 @@ func TestDelete(t *testing.T) { bkt := objstore.NewInMemBucket() { - meta, dir, err := block_testutil.CreateBlock(t, func() []*testhelper.ProfileBuilder { + meta, dir := block_testutil.CreateBlock(t, func() []*testhelper.ProfileBuilder { return []*testhelper.ProfileBuilder{ testhelper.NewProfileBuilder(int64(1)). CPUProfile(). @@ -93,7 +93,6 @@ func TestDelete(t *testing.T) { ).ForStacktraceString("foo", "bar", "baz").AddSamples(1), } }) - require.NoError(t, err) require.NoError(t, block.Upload(ctx, log.NewNopLogger(), bkt, path.Join(dir, meta.ULID.String()))) require.Equal(t, 9, len(bkt.Objects())) @@ -105,7 +104,7 @@ func TestDelete(t *testing.T) { require.Equal(t, 0, len(bkt.Objects())) } { - b2, tmpDir, err := block_testutil.CreateBlock(t, func() []*testhelper.ProfileBuilder { + b2, tmpDir := block_testutil.CreateBlock(t, func() []*testhelper.ProfileBuilder { return []*testhelper.ProfileBuilder{ testhelper.NewProfileBuilder(int64(1)). CPUProfile(). @@ -114,7 +113,6 @@ func TestDelete(t *testing.T) { ).ForStacktraceString("foo", "bar", "baz").AddSamples(1), } }) - require.NoError(t, err) require.NoError(t, block.Upload(ctx, log.NewNopLogger(), bkt, path.Join(tmpDir, b2.ULID.String()))) require.Equal(t, 9, len(bkt.Objects())) @@ -129,7 +127,7 @@ func TestUpload(t *testing.T) { ctx := context.Background() bkt := objstore.NewInMemBucket() - b1, tmpDir, err := block_testutil.CreateBlock(t, func() []*testhelper.ProfileBuilder { + b1, tmpDir := block_testutil.CreateBlock(t, func() []*testhelper.ProfileBuilder { return []*testhelper.ProfileBuilder{ testhelper.NewProfileBuilder(int64(1)). CPUProfile(). @@ -139,7 +137,6 @@ func TestUpload(t *testing.T) { } }) - require.NoError(t, err) require.NoError(t, os.MkdirAll(path.Join(tmpDir, "test", b1.ULID.String()), os.ModePerm)) t.Run("wrong dir", func(t *testing.T) { @@ -221,7 +218,7 @@ func TestMarkForDeletion(t *testing.T) { } { t.Run(tcase.name, func(t *testing.T) { bkt := objstore.NewInMemBucket() - b1, tmpDir, err := block_testutil.CreateBlock(t, func() []*testhelper.ProfileBuilder { + b1, tmpDir := block_testutil.CreateBlock(t, func() []*testhelper.ProfileBuilder { return []*testhelper.ProfileBuilder{ testhelper.NewProfileBuilder(int64(1)). CPUProfile(). @@ -231,14 +228,13 @@ func TestMarkForDeletion(t *testing.T) { } }) id := b1.ULID - require.NoError(t, err) tcase.preUpload(t, id, bkt) require.NoError(t, block.Upload(ctx, log.NewNopLogger(), bkt, path.Join(tmpDir, id.String()))) c := promauto.With(nil).NewCounter(prometheus.CounterOpts{}) - err = block.MarkForDeletion(ctx, log.NewNopLogger(), bkt, id, "", c) + err := block.MarkForDeletion(ctx, log.NewNopLogger(), bkt, id, "", c) require.NoError(t, err) require.Equal(t, float64(tcase.blocksMarked), promtest.ToFloat64(c)) }) @@ -276,7 +272,7 @@ func TestMarkForNoCompact(t *testing.T) { } { t.Run(tcase.name, func(t *testing.T) { bkt := objstore.NewInMemBucket() - meta, tmpDir, err := block_testutil.CreateBlock(t, func() []*testhelper.ProfileBuilder { + meta, tmpDir := block_testutil.CreateBlock(t, func() []*testhelper.ProfileBuilder { return []*testhelper.ProfileBuilder{ testhelper.NewProfileBuilder(int64(1)). CPUProfile(). @@ -286,14 +282,13 @@ func TestMarkForNoCompact(t *testing.T) { } }) id := meta.ULID - require.NoError(t, err) tcase.preUpload(t, id, bkt) require.NoError(t, block.Upload(ctx, log.NewNopLogger(), bkt, path.Join(tmpDir, id.String()))) c := promauto.With(nil).NewCounter(prometheus.CounterOpts{}) - err = block.MarkForNoCompact(ctx, log.NewNopLogger(), bkt, id, block.ManualNoCompactReason, "", c) + err := block.MarkForNoCompact(ctx, log.NewNopLogger(), bkt, id, block.ManualNoCompactReason, "", c) require.NoError(t, err) require.Equal(t, float64(tcase.blocksMarked), promtest.ToFloat64(c)) }) @@ -306,7 +301,7 @@ func TestUploadCleanup(t *testing.T) { ctx := context.Background() bkt := objstore.NewInMemBucket() - meta, tmpDir, err := block_testutil.CreateBlock(t, func() []*testhelper.ProfileBuilder { + meta, tmpDir := block_testutil.CreateBlock(t, func() []*testhelper.ProfileBuilder { return []*testhelper.ProfileBuilder{ testhelper.NewProfileBuilder(int64(1)). CPUProfile(). @@ -316,7 +311,6 @@ func TestUploadCleanup(t *testing.T) { } }) b1 := meta.ULID - require.NoError(t, err) { errBkt := errBucket{Bucket: bkt, failSuffix: "/index.tsdb"} diff --git a/pkg/phlaredb/block/fetcher_test.go b/pkg/phlaredb/block/fetcher_test.go index c9794bdcc3..5c1b9d76dc 100644 --- a/pkg/phlaredb/block/fetcher_test.go +++ b/pkg/phlaredb/block/fetcher_test.go @@ -329,7 +329,7 @@ func TestMetaFetcher_ShouldNotIssueAnyAPICallToObjectStorageIfAllBlockMetasAreCa } func createTestBlock(t *testing.T) (blockID ulid.ULID, blockDir string) { - meta, dir, err := block_testutil.CreateBlock(t, func() []*testhelper.ProfileBuilder { + meta, dir := block_testutil.CreateBlock(t, func() []*testhelper.ProfileBuilder { return []*testhelper.ProfileBuilder{ testhelper.NewProfileBuilder(int64(1)). CPUProfile(). @@ -338,7 +338,6 @@ func createTestBlock(t *testing.T) (blockID ulid.ULID, blockDir string) { ).ForStacktraceString("foo", "bar", "baz").AddSamples(1), } }) - require.NoError(t, err) blockID = meta.ULID blockDir = filepath.Join(dir, blockID.String()) return diff --git a/pkg/phlaredb/block/testutil/create_block.go b/pkg/phlaredb/block/testutil/create_block.go index 920ce92036..882e16fa6b 100644 --- a/pkg/phlaredb/block/testutil/create_block.go +++ b/pkg/phlaredb/block/testutil/create_block.go @@ -25,7 +25,7 @@ func (n noLimit) Stop() {} // CreateBlock creates a block with the given profiles. // Returns the block metadata, the directory where the block is stored, and an error if any. -func CreateBlock(t testing.TB, generator func() []*testhelper.ProfileBuilder) (block.Meta, string, error) { +func CreateBlock(t testing.TB, generator func() []*testhelper.ProfileBuilder) (block.Meta, string) { t.Helper() dir := t.TempDir() ctx := context.Background() @@ -54,5 +54,5 @@ func CreateBlock(t testing.TB, generator func() []*testhelper.ProfileBuilder) (b meta = m } require.NotNil(t, meta) - return *meta, localDir, nil + return *meta, localDir } diff --git a/pkg/phlaredb/validate_test.go b/pkg/phlaredb/validate_test.go index cb5db77bea..c57d7a209e 100644 --- a/pkg/phlaredb/validate_test.go +++ b/pkg/phlaredb/validate_test.go @@ -14,7 +14,7 @@ import ( ) func Test_ValidateBlock(t *testing.T) { - meta, dir, err := testutil.CreateBlock(t, func() []*testhelper.ProfileBuilder { + meta, dir := testutil.CreateBlock(t, func() []*testhelper.ProfileBuilder { return []*testhelper.ProfileBuilder{ testhelper.NewProfileBuilder(int64(1)). CPUProfile(). @@ -23,9 +23,8 @@ func Test_ValidateBlock(t *testing.T) { ).ForStacktraceString("foo", "bar", "baz").AddSamples(1), } }) - require.NoError(t, err) - err = phlaredb.ValidateLocalBlock(context.Background(), path.Join(dir, meta.ULID.String())) + err := phlaredb.ValidateLocalBlock(context.Background(), path.Join(dir, meta.ULID.String())) require.NoError(t, err) t.Run("should error when a file is missing", func(t *testing.T) { os.Remove(path.Join(dir, meta.ULID.String(), block.IndexFilename)) diff --git a/pkg/pprof/testhelper/profile_builder.go b/pkg/pprof/testhelper/profile_builder.go index 8146c89af6..2235dc542c 100644 --- a/pkg/pprof/testhelper/profile_builder.go +++ b/pkg/pprof/testhelper/profile_builder.go @@ -23,6 +23,7 @@ type ProfileBuilder struct { externalFunctionID2LocationId map[uint32]uint64 } +// NewProfileBuilder creates a new ProfileBuilder with the given nanoseconds timestamp. func NewProfileBuilder(ts int64) *ProfileBuilder { return NewProfileBuilderWithLabels(ts, []*typesv1.LabelPair{ { @@ -32,6 +33,7 @@ func NewProfileBuilder(ts int64) *ProfileBuilder { }) } +// NewProfileBuilderWithLabels creates a new ProfileBuilder with the given nanoseconds timestamp and labels. func NewProfileBuilderWithLabels(ts int64, labels []*typesv1.LabelPair) *ProfileBuilder { profile := profilev1.ProfileFromVTPool() profile.TimeNanos = ts From 69532becd6b7579769ecc2765150d9a0934d1d3b Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Wed, 13 Sep 2023 10:55:42 +0200 Subject: [PATCH 32/74] Fixes split_merge_compactor tests --- pkg/compactor/compactor_test.go | 109 ++++++-------------------------- 1 file changed, 21 insertions(+), 88 deletions(-) diff --git a/pkg/compactor/compactor_test.go b/pkg/compactor/compactor_test.go index 7624a2d1a1..ce26dba383 100644 --- a/pkg/compactor/compactor_test.go +++ b/pkg/compactor/compactor_test.go @@ -27,9 +27,9 @@ import ( "github.com/grafana/dskit/ring" "github.com/grafana/dskit/services" "github.com/grafana/dskit/test" - "github.com/grafana/pyroscope/pkg/phlaredb/block/testutil" - "github.com/grafana/pyroscope/pkg/pprof/testhelper" - "github.com/grafana/regexp" + "github.com/grafana/loki/pkg/storage/bucket/filesystem" + "github.com/grafana/mimir/pkg/storage/bucket" + "github.com/grafana/mimir/pkg/storage/tsdb/block" "github.com/oklog/ulid" "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" @@ -46,11 +46,9 @@ import ( "github.com/thanos-io/objstore" "gopkg.in/yaml.v3" - "github.com/grafana/mimir/pkg/storage/bucket" - "github.com/grafana/mimir/pkg/storage/bucket/filesystem" - mimir_tsdb "github.com/grafana/mimir/pkg/storage/tsdb" - "github.com/grafana/mimir/pkg/storage/tsdb/block" - "github.com/grafana/mimir/pkg/util/validation" + "github.com/grafana/pyroscope/pkg/phlaredb/block/testutil" + "github.com/grafana/pyroscope/pkg/pprof/testhelper" + "github.com/grafana/pyroscope/pkg/validation" ) func TestConfig_ShouldSupportYamlConfig(t *testing.T) { @@ -65,7 +63,7 @@ compaction_retries: 123 cfg := Config{} flagext.DefaultValues(&cfg) assert.NoError(t, yaml.Unmarshal([]byte(yamlCfg), &cfg)) - assert.Equal(t, mimir_tsdb.DurationList{2 * time.Hour, 48 * time.Hour}, cfg.BlockRanges) + assert.Equal(t, DurationList{2 * time.Hour, 48 * time.Hour}, cfg.BlockRanges) assert.Equal(t, 123, cfg.BlockSyncConcurrency) assert.Equal(t, "/tmp", cfg.DataDir) assert.Equal(t, 15*time.Minute, cfg.CompactionInterval) @@ -84,7 +82,7 @@ func TestConfig_ShouldSupportCliFlags(t *testing.T) { "-compactor.compaction-retries=123", })) - assert.Equal(t, mimir_tsdb.DurationList{2 * time.Hour, 48 * time.Hour}, cfg.BlockRanges) + assert.Equal(t, DurationList{2 * time.Hour, 48 * time.Hour}, cfg.BlockRanges) assert.Equal(t, 123, cfg.BlockSyncConcurrency) assert.Equal(t, "/tmp", cfg.DataDir) assert.Equal(t, 15*time.Minute, cfg.CompactionInterval) @@ -429,7 +427,7 @@ func TestMultitenantCompactor_ShouldIncrementCompactionErrorIfFailedToCompactASi bucketClient.MockIter("", []string{userID}, nil) bucketClient.MockIter(userID+"/", []string{userID + "/01DTVP434PA9VFXSW2JKB3392D", userID + "/01DTW0ZCPDDNV4BV83Q2SV4QAZ"}, nil) bucketClient.MockIter(userID+"/markers/", nil, nil) - bucketClient.MockExists(path.Join(userID, mimir_tsdb.TenantDeletionMarkPath), false, nil) + bucketClient.MockExists(path.Join(userID, bucket.TenantDeletionMarkPath), false, nil) bucketClient.MockGet(userID+"/01DTVP434PA9VFXSW2JKB3392D/meta.json", mockBlockMetaJSON("01DTVP434PA9VFXSW2JKB3392D"), nil) bucketClient.MockGet(userID+"/01DTVP434PA9VFXSW2JKB3392D/deletion-mark.json", "", nil) bucketClient.MockGet(userID+"/01DTVP434PA9VFXSW2JKB3392D/no-compact-mark.json", "", nil) @@ -478,7 +476,7 @@ func TestMultitenantCompactor_ShouldIncrementCompactionShutdownIfTheContextIsCan bucketClient.MockIter("", []string{userID}, nil) bucketClient.MockIter(userID+"/", []string{userID + "/01DTVP434PA9VFXSW2JKB3392D", userID + "/01DTW0ZCPDDNV4BV83Q2SV4QAZ"}, nil) bucketClient.MockIter(userID+"/markers/", nil, nil) - bucketClient.MockExists(path.Join(userID, mimir_tsdb.TenantDeletionMarkPath), false, nil) + bucketClient.MockExists(path.Join(userID, bucket.TenantDeletionMarkPath), false, nil) bucketClient.MockGet(userID+"/01DTVP434PA9VFXSW2JKB3392D/meta.json", mockBlockMetaJSON("01DTVP434PA9VFXSW2JKB3392D"), nil) bucketClient.MockGet(userID+"/01DTVP434PA9VFXSW2JKB3392D/deletion-mark.json", "", nil) bucketClient.MockGet(userID+"/01DTVP434PA9VFXSW2JKB3392D/no-compact-mark.json", "", nil) @@ -529,8 +527,8 @@ func TestMultitenantCompactor_ShouldIterateOverUsersAndRunCompaction(t *testing. // Mock the bucket to contain two users, each one with two blocks (to make sure that grouper doesn't skip them). bucketClient := &bucket.ClientMock{} bucketClient.MockIter("", []string{"user-1", "user-2"}, nil) - bucketClient.MockExists(path.Join("user-1", mimir_tsdb.TenantDeletionMarkPath), false, nil) - bucketClient.MockExists(path.Join("user-2", mimir_tsdb.TenantDeletionMarkPath), false, nil) + bucketClient.MockExists(path.Join("user-1", bucket.TenantDeletionMarkPath), false, nil) + bucketClient.MockExists(path.Join("user-2", bucket.TenantDeletionMarkPath), false, nil) bucketClient.MockIter("user-1/", []string{"user-1/01DTVP434PA9VFXSW2JKB3392D", "user-1/01FS51A7GQ1RQWV35DBVYQM4KF"}, nil) bucketClient.MockIter("user-2/", []string{"user-2/01DTW0ZCPDDNV4BV83Q2SV4QAZ", "user-2/01FRSF035J26D6CGX7STCSD1KG"}, nil) bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", mockBlockMetaJSON("01DTVP434PA9VFXSW2JKB3392D"), nil) @@ -674,7 +672,7 @@ func TestMultitenantCompactor_ShouldStopCompactingTenantOnReachingMaxCompactionT // and since its planning will take longer than maxCompactionTime, we stop compactions early. bucketClient := &bucket.ClientMock{} bucketClient.MockIter("", []string{"user-1"}, nil) - bucketClient.MockExists(path.Join("user-1", mimir_tsdb.TenantDeletionMarkPath), false, nil) + bucketClient.MockExists(path.Join("user-1", bucket.TenantDeletionMarkPath), false, nil) bucketClient.MockIter("user-1/", []string{"user-1/01DTVP434PA9VFXSW2JKB3392D", "user-1/01FN3VCQV5X342W2ZKMQQXAZRX", "user-1/01FS51A7GQ1RQWV35DBVYQM4KF", "user-1/01FRQGQB7RWQ2TS0VWA82QTPXE"}, nil) bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", mockBlockMetaJSONWithTimeRangeAndLabels("01DTVP434PA9VFXSW2JKB3392D", 1574776800000, 1574784000000, map[string]string{"A": "B"}), nil) bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/deletion-mark.json", "", nil) @@ -745,7 +743,7 @@ func TestMultitenantCompactor_ShouldNotCompactBlocksMarkedForDeletion(t *testing bucketClient := &bucket.ClientMock{} bucketClient.MockIter("", []string{"user-1"}, nil) bucketClient.MockIter("user-1/", []string{"user-1/01DTVP434PA9VFXSW2JKB3392D", "user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ"}, nil) - bucketClient.MockExists(path.Join("user-1", mimir_tsdb.TenantDeletionMarkPath), false, nil) + bucketClient.MockExists(path.Join("user-1", bucket.TenantDeletionMarkPath), false, nil) // Block that has just been marked for deletion. It will not be deleted just yet, and it also will not be compacted. bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", mockBlockMetaJSON("01DTVP434PA9VFXSW2JKB3392D"), nil) @@ -863,7 +861,7 @@ func TestMultitenantCompactor_ShouldNotCompactBlocksMarkedForNoCompaction(t *tes bucketClient := &bucket.ClientMock{} bucketClient.MockIter("", []string{"user-1"}, nil) bucketClient.MockIter("user-1/", []string{"user-1/01DTVP434PA9VFXSW2JKB3392D"}, nil) - bucketClient.MockExists(path.Join("user-1", mimir_tsdb.TenantDeletionMarkPath), false, nil) + bucketClient.MockExists(path.Join("user-1", bucket.TenantDeletionMarkPath), false, nil) // Block that is marked for no compaction. It will be ignored. bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", mockBlockMetaJSON("01DTVP434PA9VFXSW2JKB3392D"), nil) @@ -917,8 +915,8 @@ func TestMultitenantCompactor_ShouldNotCompactBlocksForUsersMarkedForDeletion(t bucketClient := &bucket.ClientMock{} bucketClient.MockIter("", []string{"user-1"}, nil) bucketClient.MockIter("user-1/", []string{"user-1/01DTVP434PA9VFXSW2JKB3392D"}, nil) - bucketClient.MockGet(path.Join("user-1", mimir_tsdb.TenantDeletionMarkPath), `{"deletion_time": 1}`, nil) - bucketClient.MockUpload(path.Join("user-1", mimir_tsdb.TenantDeletionMarkPath), nil) + bucketClient.MockGet(path.Join("user-1", bucket.TenantDeletionMarkPath), `{"deletion_time": 1}`, nil) + bucketClient.MockUpload(path.Join("user-1", bucket.TenantDeletionMarkPath), nil) bucketClient.MockIter("user-1/01DTVP434PA9VFXSW2JKB3392D", []string{"user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", "user-1/01DTVP434PA9VFXSW2JKB3392D/index"}, nil) bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", mockBlockMetaJSON("01DTVP434PA9VFXSW2JKB3392D"), nil) @@ -1017,8 +1015,8 @@ func TestMultitenantCompactor_ShouldCompactAllUsersOnShardingEnabledButOnlyOneIn // Mock the bucket to contain two users, each one with one block. bucketClient := &bucket.ClientMock{} bucketClient.MockIter("", []string{"user-1", "user-2"}, nil) - bucketClient.MockExists(path.Join("user-1", mimir_tsdb.TenantDeletionMarkPath), false, nil) - bucketClient.MockExists(path.Join("user-2", mimir_tsdb.TenantDeletionMarkPath), false, nil) + bucketClient.MockExists(path.Join("user-1", bucket.TenantDeletionMarkPath), false, nil) + bucketClient.MockExists(path.Join("user-2", bucket.TenantDeletionMarkPath), false, nil) bucketClient.MockIter("user-1/", []string{"user-1/01DTVP434PA9VFXSW2JKB3392D", "user-1/01FSTQ95C8FS0ZAGTQS2EF1NEG"}, nil) bucketClient.MockIter("user-2/", []string{"user-2/01DTW0ZCPDDNV4BV83Q2SV4QAZ", "user-2/01FSV54G6QFQH1G9QE93G3B9TB"}, nil) bucketClient.MockIter("user-1/markers/", nil, nil) @@ -1157,7 +1155,7 @@ func TestMultitenantCompactor_ShouldCompactOnlyUsersOwnedByTheInstanceOnSharding for _, userID := range userIDs { bucketClient.MockIter(userID+"/", []string{userID + "/01DTVP434PA9VFXSW2JKB3392D"}, nil) bucketClient.MockIter(userID+"/markers/", nil, nil) - bucketClient.MockExists(path.Join(userID, mimir_tsdb.TenantDeletionMarkPath), false, nil) + bucketClient.MockExists(path.Join(userID, bucket.TenantDeletionMarkPath), false, nil) bucketClient.MockGet(userID+"/01DTVP434PA9VFXSW2JKB3392D/meta.json", mockBlockMetaJSON("01DTVP434PA9VFXSW2JKB3392D"), nil) bucketClient.MockGet(userID+"/01DTVP434PA9VFXSW2JKB3392D/deletion-mark.json", "", nil) bucketClient.MockGet(userID+"/01DTVP434PA9VFXSW2JKB3392D/no-compact-mark.json", "", nil) @@ -1330,7 +1328,7 @@ func TestMultitenantCompactor_ShouldSkipCompactionForJobsNoMoreOwnedAfterPlannin // for the splitting stage). bucketClient := &bucket.ClientMock{} bucketClient.MockIter("", []string{"user-1"}, nil) - bucketClient.MockExists(path.Join("user-1", mimir_tsdb.TenantDeletionMarkPath), false, nil) + bucketClient.MockExists(path.Join("user-1", bucket.TenantDeletionMarkPath), false, nil) bucketClient.MockIter("user-1/", []string{"user-1/01DTVP434PA9VFXSW2JK000001", "user-1/01DTVP434PA9VFXSW2JK000002"}, nil) bucketClient.MockIter("user-1/markers/", nil, nil) bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JK000001/meta.json", mockBlockMetaJSONWithTimeRange("01DTVP434PA9VFXSW2JK000001", 1574776800000, 1574784000000), nil) @@ -2156,71 +2154,6 @@ func stopServiceFn(t *testing.T, serv services.Service) func() { } } -func TestMultitenantCompactor_OutOfOrderCompaction(t *testing.T) { - // Generate a single block with out of order chunks. - specs := []*block.SeriesSpec{ - { - Labels: labels.FromStrings("case", "out_of_order"), - Chunks: []chunks.Meta{ - must(tsdbutil.ChunkFromSamples([]tsdbutil.Sample{newSample(20, 20, nil, nil), newSample(21, 21, nil, nil)})), - must(tsdbutil.ChunkFromSamples([]tsdbutil.Sample{newSample(10, 10, nil, nil), newSample(11, 11, nil, nil)})), - // Extend block to cover 2h. - must(tsdbutil.ChunkFromSamples([]tsdbutil.Sample{newSample(0, 0, nil, nil), newSample(2*time.Hour.Milliseconds()-1, 0, nil, nil)})), - }, - }, - } - - const user = "user" - - storageDir := t.TempDir() - // We need two blocks to start compaction. - meta1, err := block.GenerateBlockFromSpec(user, filepath.Join(storageDir, user), specs) - require.NoError(t, err) - meta2, err := block.GenerateBlockFromSpec(user, filepath.Join(storageDir, user), specs) - require.NoError(t, err) - - bkt, err := filesystem.NewBucketClient(filesystem.Config{Directory: storageDir}) - require.NoError(t, err) - - cfg := prepareConfig(t) - c, _, tsdbPlanner, logs, registry := prepare(t, cfg, bkt) - - tsdbPlanner.On("Plan", mock.Anything, mock.Anything).Return([]*block.Meta{meta1, meta2}, nil) - - // Start the compactor - require.NoError(t, services.StartAndAwaitRunning(context.Background(), c)) - - // Wait until a compaction run has been completed. - test.Poll(t, 10*time.Second, 1.0, func() interface{} { - return prom_testutil.ToFloat64(c.compactionRunsCompleted) - }) - - // Stop the compactor. - require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) - - // Verify that compactor has found block with out of order chunks, and this block is now marked for no-compaction. - r := regexp.MustCompile("level=info component=compactor user=user msg=\"block has been marked for no compaction\" block=([0-9A-Z]+)") - matches := r.FindStringSubmatch(logs.String()) - require.Len(t, matches, 2) // Entire string match + single group match. - - skippedBlock := matches[1] - require.True(t, skippedBlock == meta1.ULID.String() || skippedBlock == meta2.ULID.String()) - - m := &block.NoCompactMark{} - require.NoError(t, block.ReadMarker(context.Background(), log.NewNopLogger(), objstore.WithNoopInstr(bkt), path.Join(user, skippedBlock), m)) - require.Equal(t, skippedBlock, m.ID.String()) - require.NotZero(t, m.NoCompactTime) - require.Equal(t, block.NoCompactReason(block.OutOfOrderChunksNoCompactReason), m.Reason) - - assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` - # HELP cortex_compactor_blocks_marked_for_no_compaction_total Total number of blocks that were marked for no-compaction. - # TYPE cortex_compactor_blocks_marked_for_no_compaction_total counter - cortex_compactor_blocks_marked_for_no_compaction_total{reason="block-index-out-of-order-chunk"} 1 - `), - "cortex_compactor_blocks_marked_for_no_compaction_total", - )) -} - type sample struct { t int64 v float64 From e97e1fe74954b5b4fb9ca26df90d002c9a5db4aa Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Wed, 13 Sep 2023 11:57:18 +0200 Subject: [PATCH 33/74] Fixes more tests --- pkg/compactor/bucket_compactor.go | 49 +++++++++----------------- pkg/compactor/bucket_compactor_test.go | 10 ------ pkg/compactor/compactor_test.go | 6 +--- 3 files changed, 17 insertions(+), 48 deletions(-) diff --git a/pkg/compactor/bucket_compactor.go b/pkg/compactor/bucket_compactor.go index 67f691aed2..2b871b35de 100644 --- a/pkg/compactor/bucket_compactor.go +++ b/pkg/compactor/bucket_compactor.go @@ -24,6 +24,7 @@ import ( "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/tsdb" "go.uber.org/atomic" "github.com/grafana/pyroscope/pkg/objstore" @@ -31,7 +32,6 @@ import ( "github.com/grafana/pyroscope/pkg/objstore/providers/filesystem" "github.com/grafana/pyroscope/pkg/phlaredb" "github.com/grafana/pyroscope/pkg/phlaredb/block" - "github.com/grafana/pyroscope/pkg/phlaredb/sharding" "github.com/grafana/pyroscope/pkg/util" ) @@ -217,6 +217,14 @@ type Planner interface { Plan(ctx context.Context, metasByMinTime []*block.Meta) ([]*block.Meta, error) } +// Compactor provides compaction against an underlying storage of profiling data. +type Compactor interface { + // CompactWithSplitting merges and splits the source blocks into shardCount number of compacted blocks, + // and returns slice of block IDs. Position of returned block ID in the result slice corresponds to the shard index. + // If given compacted block has no series, corresponding block ID will be zero ULID value. + CompactWithSplitting(dest string, dirs []string, open []*tsdb.Block, shardCount uint64) (result []ulid.ULID, _ error) +} + // runCompactionJob plans and runs a single compaction against the provided job. The compacted result // is uploaded into the bucket the blocks were retrieved from. func (c *BucketCompactor) runCompactionJob(ctx context.Context, job *Job) (shouldRerun bool, compIDs []ulid.ULID, rerr error) { @@ -287,6 +295,7 @@ func (c *BucketCompactor) runCompactionJob(ctx context.Context, job *Job) (shoul compactionBegin := time.Now() // todo: move this to a separate function. + // localBucket, err := client.NewBucket(ctx, client.Config{ StorageBackendConfig: client.StorageBackendConfig{ Backend: client.Filesystem, @@ -361,13 +370,12 @@ func (c *BucketCompactor) runCompactionJob(ctx context.Context, job *Job) (shoul c.metrics.compactionBlocksVerificationFailed.Inc() } - blocksToUpload := convertCompactionResultToForEachJobs(compIDs, job.UseSplitting(), jobLogger) - err = concurrency.ForEachJob(ctx, len(blocksToUpload), c.blockSyncConcurrency, func(ctx context.Context, idx int) error { - blockToUpload := blocksToUpload[idx] + err = concurrency.ForEachJob(ctx, len(compIDs), c.blockSyncConcurrency, func(ctx context.Context, idx int) error { + ulidToUpload := compIDs[idx] uploadedBlocks.Inc() - bdir := filepath.Join(subDir, blockToUpload.ulid.String()) + bdir := filepath.Join(subDir, ulidToUpload.String()) newMeta, err := block.ReadMetaFromDir(bdir) if err != nil { @@ -385,11 +393,11 @@ func (c *BucketCompactor) runCompactionJob(ctx context.Context, job *Job) (shoul begin := time.Now() if err := block.Upload(ctx, jobLogger, c.bkt, bdir); err != nil { - return errors.Wrapf(err, "upload of %s failed", blockToUpload.ulid) + return errors.Wrapf(err, "upload of %s failed", ulidToUpload) } elapsed := time.Since(begin) - level.Info(jobLogger).Log("msg", "uploaded block", "result_block", blockToUpload.ulid, "duration", elapsed, "duration_ms", elapsed.Milliseconds(), "labels", labels.FromMap(newMeta.Labels)) + level.Info(jobLogger).Log("msg", "uploaded block", "result_block", ulidToUpload, "duration", elapsed, "duration_ms", elapsed.Milliseconds(), "labels", labels.FromMap(newMeta.Labels)) return nil }) if err != nil { @@ -456,27 +464,6 @@ func verifyCompactedBlocksTimeRanges(compIDs []ulid.ULID, sourceBlocksMinTime, s return nil } -// convertCompactionResultToForEachJobs filters out empty ULIDs. -// When handling result of split compactions, shard index is index in the slice returned by compaction. -func convertCompactionResultToForEachJobs(compactedBlocks []ulid.ULID, splitJob bool, jobLogger log.Logger) []ulidWithShardIndex { - result := make([]ulidWithShardIndex, 0, len(compactedBlocks)) - - for ix, id := range compactedBlocks { - // Skip if it's an empty block. - if id == (ulid.ULID{}) { - if splitJob { - level.Info(jobLogger).Log("msg", "compaction produced an empty block", "shard_id", sharding.FormatShardIDLabelValue(uint64(ix), uint64(len(compactedBlocks)))) - } else { - level.Info(jobLogger).Log("msg", "compaction produced an empty block") - } - continue - } - - result = append(result, ulidWithShardIndex{shardIndex: ix, ulid: id}) - } - return result -} - func deleteBlock(bkt objstore.Bucket, id ulid.ULID, bdir string, logger log.Logger, blocksMarkedForDeletion prometheus.Counter) error { if err := os.RemoveAll(bdir); err != nil { return errors.Wrapf(err, "remove old block dir %s", id) @@ -492,11 +479,6 @@ func deleteBlock(bkt objstore.Bucket, id ulid.ULID, bdir string, logger log.Logg return nil } -type ulidWithShardIndex struct { - ulid ulid.ULID - shardIndex int -} - // BucketCompactorMetrics holds the metrics tracked by BucketCompactor. type BucketCompactorMetrics struct { groupCompactionRunsStarted prometheus.Counter @@ -559,6 +541,7 @@ type BucketCompactor struct { sy *Syncer grouper Grouper planner Planner + comp Compactor compactDir string bkt objstore.Bucket concurrency int diff --git a/pkg/compactor/bucket_compactor_test.go b/pkg/compactor/bucket_compactor_test.go index cadce93354..8e8f5a7ea9 100644 --- a/pkg/compactor/bucket_compactor_test.go +++ b/pkg/compactor/bucket_compactor_test.go @@ -272,16 +272,6 @@ func TestNoCompactionMarkFilter(t *testing.T) { } } -func TestConvertCompactionResultToForEachJobs(t *testing.T) { - ulid1 := ulid.MustNew(1, nil) - ulid2 := ulid.MustNew(2, nil) - - res := convertCompactionResultToForEachJobs([]ulid.ULID{{}, ulid1, {}, ulid2, {}}, true, log.NewNopLogger()) - require.Len(t, res, 2) - require.Equal(t, ulidWithShardIndex{ulid: ulid1, shardIndex: 1}, res[0]) - require.Equal(t, ulidWithShardIndex{ulid: ulid2, shardIndex: 3}, res[1]) -} - func TestCompactedBlocksTimeRangeVerification(t *testing.T) { const ( sourceMinTime = 1000 diff --git a/pkg/compactor/compactor_test.go b/pkg/compactor/compactor_test.go index ce26dba383..4e78182af3 100644 --- a/pkg/compactor/compactor_test.go +++ b/pkg/compactor/compactor_test.go @@ -1755,15 +1755,11 @@ func prepareWithConfigProvider(t *testing.T, compactorCfg Config, bucketClient o logger := &componentLogger{component: "compactor", log: log.NewLogfmtLogger(logs)} registry := prometheus.NewRegistry() - bucketClientFactory := func(ctx context.Context) (objstore.Bucket, error) { - return bucketClient, nil - } - blocksCompactorFactory := func(ctx context.Context, cfg Config, logger log.Logger, reg prometheus.Registerer) (Compactor, Planner, error) { return tsdbCompactor, tsdbPlanner, nil } - c, err := newMultitenantCompactor(compactorCfg, storageCfg, limits, logger, registry, bucketClientFactory, splitAndMergeGrouperFactory, blocksCompactorFactory) + c, err := newMultitenantCompactor(compactorCfg, bucketClient, limits, logger, registry, splitAndMergeGrouperFactory, blocksCompactorFactory) require.NoError(t, err) return c, tsdbCompactor, tsdbPlanner, logs, registry From c461b89e594c8743c117a16dca59e6e8baddb80a Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Thu, 14 Sep 2023 13:26:43 +0200 Subject: [PATCH 34/74] fixing more tests --- pkg/compactor/blocks_cleaner_test.go | 74 +- pkg/compactor/bucket_compactor.go | 2 +- pkg/compactor/bucket_compactor_e2e_test.go | 1682 +++++++++---------- pkg/compactor/bucket_compactor_test.go | 86 +- pkg/compactor/compactor_test.go | 2 +- pkg/compactor/split_merge_compactor_test.go | 88 +- 6 files changed, 958 insertions(+), 976 deletions(-) diff --git a/pkg/compactor/blocks_cleaner_test.go b/pkg/compactor/blocks_cleaner_test.go index ffba250c63..8803d2e6c7 100644 --- a/pkg/compactor/blocks_cleaner_test.go +++ b/pkg/compactor/blocks_cleaner_test.go @@ -70,14 +70,14 @@ func testBlocksCleanerWithOptions(t *testing.T, options testBlocksCleanerOptions ctx := context.Background() now := time.Now() deletionDelay := 12 * time.Hour - block1 := createTSDBBlock(t, bucketClient, "user-1", 10, 20, 2, nil) - block2 := createTSDBBlock(t, bucketClient, "user-1", 20, 30, 2, nil) - block3 := createTSDBBlock(t, bucketClient, "user-1", 30, 40, 2, nil) + block1 := createDBBlock(t, bucketClient, "user-1", 10, 20, 2, nil) + block2 := createDBBlock(t, bucketClient, "user-1", 20, 30, 2, nil) + block3 := createDBBlock(t, bucketClient, "user-1", 30, 40, 2, nil) block4 := ulid.MustNew(4, rand.Reader) block5 := ulid.MustNew(5, rand.Reader) - block6 := createTSDBBlock(t, bucketClient, "user-1", 40, 50, 2, nil) - block7 := createTSDBBlock(t, bucketClient, "user-2", 10, 20, 2, nil) - block8 := createTSDBBlock(t, bucketClient, "user-2", 40, 50, 2, nil) + block6 := createDBBlock(t, bucketClient, "user-1", 40, 50, 2, nil) + block7 := createDBBlock(t, bucketClient, "user-2", 10, 20, 2, nil) + block8 := createDBBlock(t, bucketClient, "user-2", 40, 50, 2, nil) createDeletionMark(t, bucketClient, "user-1", block2, now.Add(-deletionDelay).Add(time.Hour)) // Block hasn't reached the deletion threshold yet. createDeletionMark(t, bucketClient, "user-1", block3, now.Add(-deletionDelay).Add(-time.Hour)) // Block reached the deletion threshold. createDeletionMark(t, bucketClient, "user-1", block4, now.Add(-deletionDelay).Add(time.Hour)) // Partial block hasn't reached the deletion threshold yet. @@ -87,8 +87,8 @@ func testBlocksCleanerWithOptions(t *testing.T, options testBlocksCleanerOptions // Blocks for user-3, marked for deletion. require.NoError(t, bucket.WriteTenantDeletionMark(context.Background(), bucketClient, "user-3", nil, bucket.NewTenantDeletionMark(time.Now()))) - block9 := createTSDBBlock(t, bucketClient, "user-3", 10, 30, 2, nil) - block10 := createTSDBBlock(t, bucketClient, "user-3", 30, 50, 2, nil) + block9 := createDBBlock(t, bucketClient, "user-3", 10, 30, 2, nil) + block10 := createDBBlock(t, bucketClient, "user-3", 30, 50, 2, nil) // User-4 with no more blocks, but couple of mark and debug files. Should be fully deleted. user4Mark := bucket.NewTenantDeletionMark(time.Now()) @@ -216,10 +216,10 @@ func TestBlocksCleaner_ShouldContinueOnBlockDeletionFailure(t *testing.T) { ctx := context.Background() now := time.Now() deletionDelay := 12 * time.Hour - block1 := createTSDBBlock(t, bucketClient, userID, 10, 20, 2, nil) - block2 := createTSDBBlock(t, bucketClient, userID, 20, 30, 2, nil) - block3 := createTSDBBlock(t, bucketClient, userID, 30, 40, 2, nil) - block4 := createTSDBBlock(t, bucketClient, userID, 40, 50, 2, nil) + block1 := createDBBlock(t, bucketClient, userID, 10, 20, 2, nil) + block2 := createDBBlock(t, bucketClient, userID, 20, 30, 2, nil) + block3 := createDBBlock(t, bucketClient, userID, 30, 40, 2, nil) + block4 := createDBBlock(t, bucketClient, userID, 40, 50, 2, nil) createDeletionMark(t, bucketClient, userID, block2, now.Add(-deletionDelay).Add(-time.Hour)) createDeletionMark(t, bucketClient, userID, block3, now.Add(-deletionDelay).Add(-time.Hour)) createDeletionMark(t, bucketClient, userID, block4, now.Add(-deletionDelay).Add(-time.Hour)) @@ -281,9 +281,9 @@ func TestBlocksCleaner_ShouldRebuildBucketIndexOnCorruptedOne(t *testing.T) { ctx := context.Background() now := time.Now() deletionDelay := 12 * time.Hour - block1 := createTSDBBlock(t, bucketClient, userID, 10, 20, 2, nil) - block2 := createTSDBBlock(t, bucketClient, userID, 20, 30, 2, nil) - block3 := createTSDBBlock(t, bucketClient, userID, 30, 40, 2, nil) + block1 := createDBBlock(t, bucketClient, userID, 10, 20, 2, nil) + block2 := createDBBlock(t, bucketClient, userID, 20, 30, 2, nil) + block3 := createDBBlock(t, bucketClient, userID, 30, 40, 2, nil) createDeletionMark(t, bucketClient, userID, block2, now.Add(-deletionDelay).Add(-time.Hour)) createDeletionMark(t, bucketClient, userID, block3, now.Add(-deletionDelay).Add(time.Hour)) @@ -335,9 +335,9 @@ func TestBlocksCleaner_ShouldRemoveMetricsForTenantsNotBelongingAnymoreToTheShar bucketClient = block.BucketWithGlobalMarkers(bucketClient) // Create blocks. - createTSDBBlock(t, bucketClient, "user-1", 10, 20, 2, nil) - createTSDBBlock(t, bucketClient, "user-1", 20, 30, 2, nil) - createTSDBBlock(t, bucketClient, "user-2", 30, 40, 2, nil) + createDBBlock(t, bucketClient, "user-1", 10, 20, 2, nil) + createDBBlock(t, bucketClient, "user-1", 20, 30, 2, nil) + createDBBlock(t, bucketClient, "user-2", 30, 40, 2, nil) cfg := BlocksCleanerConfig{ DeletionDelay: time.Hour, @@ -377,8 +377,8 @@ func TestBlocksCleaner_ShouldRemoveMetricsForTenantsNotBelongingAnymoreToTheShar cleaner.tenantsScanner = bucket.NewTenantsScanner(bucketClient, func(userID string) (bool, error) { return userID == "user-1", nil }, logger) // Create new blocks, to double check expected metrics have changed. - createTSDBBlock(t, bucketClient, "user-1", 40, 50, 2, nil) - createTSDBBlock(t, bucketClient, "user-2", 50, 60, 2, nil) + createDBBlock(t, bucketClient, "user-1", 40, 50, 2, nil) + createDBBlock(t, bucketClient, "user-2", 50, 60, 2, nil) require.NoError(t, cleaner.runCleanupWithErr(ctx)) @@ -404,8 +404,8 @@ func TestBlocksCleaner_ShouldNotCleanupUserThatDoesntBelongToShardAnymore(t *tes bucketClient = block.BucketWithGlobalMarkers(bucketClient) // Create blocks. - createTSDBBlock(t, bucketClient, "user-1", 10, 20, 2, nil) - createTSDBBlock(t, bucketClient, "user-2", 20, 30, 2, nil) + createDBBlock(t, bucketClient, "user-1", 10, 20, 2, nil) + createDBBlock(t, bucketClient, "user-2", 20, 30, 2, nil) cfg := BlocksCleanerConfig{ DeletionDelay: time.Hour, @@ -456,9 +456,9 @@ func TestBlocksCleaner_ListBlocksOutsideRetentionPeriod(t *testing.T) { ctx := context.Background() logger := log.NewNopLogger() - id1 := createTSDBBlock(t, bucketClient, "user-1", 5000, 6000, 2, nil) - id2 := createTSDBBlock(t, bucketClient, "user-1", 6000, 7000, 2, nil) - id3 := createTSDBBlock(t, bucketClient, "user-1", 7000, 8000, 2, nil) + id1 := createDBBlock(t, bucketClient, "user-1", 5000, 6000, 2, nil) + id2 := createDBBlock(t, bucketClient, "user-1", 6000, 7000, 2, nil) + id3 := createDBBlock(t, bucketClient, "user-1", 7000, 8000, 2, nil) w := bucketindex.NewUpdater(bucketClient, "user-1", nil, logger) idx, _, err := w.UpdateIndex(ctx, nil) @@ -516,10 +516,10 @@ func TestBlocksCleaner_ShouldRemoveBlocksOutsideRetentionPeriod(t *testing.T) { return time.Now().Add(time.Duration(hours)*time.Hour).Unix() * 1000 } - block1 := createTSDBBlock(t, bucketClient, "user-1", ts(-10), ts(-8), 2, nil) - block2 := createTSDBBlock(t, bucketClient, "user-1", ts(-8), ts(-6), 2, nil) - block3 := createTSDBBlock(t, bucketClient, "user-2", ts(-10), ts(-8), 2, nil) - block4 := createTSDBBlock(t, bucketClient, "user-2", ts(-8), ts(-6), 2, nil) + block1 := createDBBlock(t, bucketClient, "user-1", ts(-10), ts(-8), 2, nil) + block2 := createDBBlock(t, bucketClient, "user-1", ts(-8), ts(-6), 2, nil) + block3 := createDBBlock(t, bucketClient, "user-2", ts(-10), ts(-8), 2, nil) + block4 := createDBBlock(t, bucketClient, "user-2", ts(-8), ts(-6), 2, nil) cfg := BlocksCleanerConfig{ DeletionDelay: time.Hour, @@ -704,8 +704,8 @@ func TestBlocksCleaner_ShouldCleanUpFilesWhenNoMoreBlocksRemain(t *testing.T) { deletionDelay := 12 * time.Hour // Create two blocks and mark them for deletion at a time before the deletionDelay - block1 := createTSDBBlock(t, bucketClient, userID, 10, 20, 2, nil) - block2 := createTSDBBlock(t, bucketClient, userID, 20, 30, 2, nil) + block1 := createDBBlock(t, bucketClient, userID, 10, 20, 2, nil) + block2 := createDBBlock(t, bucketClient, userID, 20, 30, 2, nil) createDeletionMark(t, bucketClient, userID, block1, now.Add(-deletionDelay).Add(-time.Hour)) createDeletionMark(t, bucketClient, userID, block2, now.Add(-deletionDelay).Add(-time.Hour)) @@ -753,8 +753,8 @@ func TestBlocksCleaner_ShouldRemovePartialBlocksOutsideDelayPeriod(t *testing.T) return time.Now().Add(time.Duration(hours)*time.Hour).Unix() * 1000 } - block1 := createTSDBBlock(t, bucketClient, "user-1", ts(-10), ts(-8), 2, nil) - block2 := createTSDBBlock(t, bucketClient, "user-1", ts(-8), ts(-6), 2, nil) + block1 := createDBBlock(t, bucketClient, "user-1", ts(-10), ts(-8), 2, nil) + block2 := createDBBlock(t, bucketClient, "user-1", ts(-8), ts(-6), 2, nil) cfg := BlocksCleanerConfig{ DeletionDelay: time.Hour, @@ -823,8 +823,8 @@ func TestBlocksCleaner_ShouldNotRemovePartialBlocksInsideDelayPeriod(t *testing. return time.Now().Add(time.Duration(hours)*time.Hour).Unix() * 1000 } - block1 := createTSDBBlock(t, bucketClient, "user-1", ts(-10), ts(-8), 2, nil) - block2 := createTSDBBlock(t, bucketClient, "user-2", ts(-8), ts(-6), 2, nil) + block1 := createDBBlock(t, bucketClient, "user-1", ts(-10), ts(-8), 2, nil) + block2 := createDBBlock(t, bucketClient, "user-2", ts(-8), ts(-6), 2, nil) cfg := BlocksCleanerConfig{ DeletionDelay: time.Hour, @@ -910,7 +910,7 @@ func TestBlocksCleaner_ShouldNotRemovePartialBlocksIfConfiguredDelayIsInvalid(t } // Create a partial block. - block1 := createTSDBBlock(t, bucketClient, "user-1", ts(-10), ts(-8), 2, nil) + block1 := createDBBlock(t, bucketClient, "user-1", ts(-10), ts(-8), 2, nil) err := bucketClient.Delete(ctx, path.Join("user-1", block1.String(), block.MetaFilename)) require.NoError(t, err) @@ -962,7 +962,7 @@ func TestStalePartialBlockLastModifiedTime(t *testing.T) { const tenantId = "user" objectTime := time.Now().Add(-1 * time.Hour).Truncate(time.Second) // ignore milliseconds, as not all filesystems store them. - blockID := createTSDBBlock(t, b, tenantId, objectTime.UnixMilli(), time.Now().UnixMilli(), 2, nil) + blockID := createDBBlock(t, b, tenantId, objectTime.UnixMilli(), time.Now().UnixMilli(), 2, nil) for _, f := range []string{"meta.json", "index", "chunks/000001", "tombstones"} { require.NoError(t, os.Chtimes(filepath.Join(dir, tenantId, blockID.String(), filepath.FromSlash(f)), objectTime, objectTime)) } diff --git a/pkg/compactor/bucket_compactor.go b/pkg/compactor/bucket_compactor.go index 2b871b35de..56a88f932a 100644 --- a/pkg/compactor/bucket_compactor.go +++ b/pkg/compactor/bucket_compactor.go @@ -295,7 +295,7 @@ func (c *BucketCompactor) runCompactionJob(ctx context.Context, job *Job) (shoul compactionBegin := time.Now() // todo: move this to a separate function. - // + // Move to a compactor interface localBucket, err := client.NewBucket(ctx, client.Config{ StorageBackendConfig: client.StorageBackendConfig{ Backend: client.Filesystem, diff --git a/pkg/compactor/bucket_compactor_e2e_test.go b/pkg/compactor/bucket_compactor_e2e_test.go index 9c2018722a..db870d329c 100644 --- a/pkg/compactor/bucket_compactor_e2e_test.go +++ b/pkg/compactor/bucket_compactor_e2e_test.go @@ -5,844 +5,844 @@ package compactor -import ( - "bytes" - "context" - "encoding/json" - "fmt" - "math/rand" - "os" - "path" - "path/filepath" - "runtime" - "sort" - "strings" - "testing" - "time" - - "github.com/go-kit/log" - "github.com/grafana/dskit/runutil" - "github.com/oklog/ulid" - "github.com/pkg/errors" - "github.com/prometheus/client_golang/prometheus" - "github.com/prometheus/client_golang/prometheus/promauto" - promtest "github.com/prometheus/client_golang/prometheus/testutil" - "github.com/prometheus/prometheus/model/labels" - "github.com/prometheus/prometheus/storage" - "github.com/prometheus/prometheus/tsdb" - "github.com/prometheus/prometheus/tsdb/chunkenc" - "github.com/prometheus/prometheus/tsdb/chunks" - "github.com/prometheus/prometheus/tsdb/index" - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" - "github.com/thanos-io/objstore" - "github.com/thanos-io/objstore/providers/filesystem" - "golang.org/x/exp/slices" - "golang.org/x/sync/errgroup" - - "github.com/grafana/mimir/pkg/storage/tsdb/block" -) - -func TestSyncer_GarbageCollect_e2e(t *testing.T) { - foreachStore(t, func(t *testing.T, bkt objstore.Bucket) { - // Use bucket with global markers to make sure that our custom filters work correctly. - bkt = block.BucketWithGlobalMarkers(bkt) - - ctx, cancel := context.WithTimeout(context.Background(), 120*time.Second) - defer cancel() - - // Generate 10 source block metas and construct higher level blocks - // that are higher compactions of them. - var metas []*block.Meta - var ids []ulid.ULID - - for i := 0; i < 10; i++ { - var m block.Meta - - m.Version = 1 - m.ULID = ulid.MustNew(uint64(i), nil) - m.Compaction.Sources = []ulid.ULID{m.ULID} - m.Compaction.Level = 1 - m.MinTime = 0 - m.MaxTime = 2 * time.Hour.Milliseconds() - - ids = append(ids, m.ULID) - metas = append(metas, &m) - } - - var m1 block.Meta - m1.Version = 1 - m1.ULID = ulid.MustNew(100, nil) - m1.Compaction.Level = 2 - m1.Compaction.Sources = ids[:4] - m1.Thanos.Downsample.Resolution = 0 - - var m2 block.Meta - m2.Version = 1 - m2.ULID = ulid.MustNew(200, nil) - m2.Compaction.Level = 2 - m2.Compaction.Sources = ids[4:8] // last two source IDs is not part of a level 2 block. - m2.Thanos.Downsample.Resolution = 0 - - var m3 block.Meta - m3.Version = 1 - m3.ULID = ulid.MustNew(300, nil) - m3.Compaction.Level = 3 - m3.Compaction.Sources = ids[:9] // last source ID is not part of level 3 block. - m3.Thanos.Downsample.Resolution = 0 - m3.MinTime = 0 - m3.MaxTime = 2 * time.Hour.Milliseconds() - - var m4 block.Meta - m4.Version = 1 - m4.ULID = ulid.MustNew(400, nil) - m4.Compaction.Level = 2 - m4.Compaction.Sources = ids[9:] // covers the last block but is a different resolution. Must not trigger deletion. - m4.Thanos.Downsample.Resolution = 1000 - m4.MinTime = 0 - m4.MaxTime = 2 * time.Hour.Milliseconds() - - var m5 block.Meta - m5.Version = 1 - m5.ULID = ulid.MustNew(500, nil) - m5.Compaction.Level = 2 - m5.Compaction.Sources = ids[8:9] // built from block 8, but different resolution. Block 8 is already included in m3, can be deleted. - m5.Thanos.Downsample.Resolution = 1000 - m5.MinTime = 0 - m5.MaxTime = 2 * time.Hour.Milliseconds() - - // Create all blocks in the bucket. - for _, m := range append(metas, &m1, &m2, &m3, &m4, &m5) { - fmt.Println("create", m.ULID) - var buf bytes.Buffer - require.NoError(t, json.NewEncoder(&buf).Encode(&m)) - require.NoError(t, bkt.Upload(ctx, path.Join(m.ULID.String(), block.MetaFilename), &buf)) - } - - duplicateBlocksFilter := NewShardAwareDeduplicateFilter() - metaFetcher, err := block.NewMetaFetcher(nil, 32, objstore.WithNoopInstr(bkt), "", nil, []block.MetadataFilter{ - duplicateBlocksFilter, - }) - require.NoError(t, err) - - blocksMarkedForDeletion := promauto.With(nil).NewCounter(prometheus.CounterOpts{}) - sy, err := NewMetaSyncer(nil, nil, bkt, metaFetcher, duplicateBlocksFilter, blocksMarkedForDeletion) - require.NoError(t, err) - - // Do one initial synchronization with the bucket. - require.NoError(t, sy.SyncMetas(ctx)) - require.NoError(t, sy.GarbageCollect(ctx)) - - var rem []ulid.ULID - err = bkt.Iter(ctx, "", func(n string) error { - id, ok := block.IsBlockDir(n) - if !ok { - return nil - } - deletionMarkFile := path.Join(id.String(), block.DeletionMarkFilename) - - exists, err := bkt.Exists(ctx, deletionMarkFile) - if err != nil { - return err - } - if !exists { - rem = append(rem, id) - } - return nil - }) - require.NoError(t, err) - - sort.Slice(rem, func(i, j int) bool { - return rem[i].Compare(rem[j]) < 0 - }) - - // Only the level 3 block, the last source block in both resolutions should be left. - assert.Equal(t, []ulid.ULID{metas[9].ULID, m3.ULID, m4.ULID, m5.ULID}, rem) - - // After another sync the changes should also be reflected in the local groups. - require.NoError(t, sy.SyncMetas(ctx)) - require.NoError(t, sy.GarbageCollect(ctx)) - - // Only the level 3 block, the last source block in both resolutions should be left. - grouper := NewSplitAndMergeGrouper("user-1", []int64{2 * time.Hour.Milliseconds()}, 0, 0, log.NewNopLogger()) - groups, err := grouper.Groups(sy.Metas()) - require.NoError(t, err) - - assert.Equal(t, "0@17241709254077376921-merge--0-7200000", groups[0].Key()) - assert.Equal(t, []ulid.ULID{metas[9].ULID, m3.ULID}, groups[0].IDs()) - - assert.Equal(t, "1000@17241709254077376921-merge--0-7200000", groups[1].Key()) - assert.Equal(t, []ulid.ULID{m4.ULID, m5.ULID}, groups[1].IDs()) - }) -} - -func TestGroupCompactE2E(t *testing.T) { - foreachStore(t, func(t *testing.T, bkt objstore.Bucket) { - // Use bucket with global markers to make sure that our custom filters work correctly. - bkt = block.BucketWithGlobalMarkers(bkt) - - ctx, cancel := context.WithTimeout(context.Background(), 120*time.Second) - defer cancel() - - // Create fresh, empty directory for actual test. - dir := t.TempDir() - - // Start dir checker... we make sure that "dir" only contains group subdirectories during compaction, - // and not any block directories. Dir checker stops when context is canceled, or on first error, - // in which case error is logger and test is failed. (We cannot use Fatal or FailNow from a goroutine). - go func() { - for ctx.Err() == nil { - fs, err := os.ReadDir(dir) - if err != nil && !os.IsNotExist(err) { - t.Log("error while listing directory", dir) - t.Fail() - return - } - - for _, fi := range fs { - // Suffix used by Prometheus LeveledCompactor when doing compaction. - toCheck := strings.TrimSuffix(fi.Name(), ".tmp-for-creation") - - _, err := ulid.Parse(toCheck) - if err == nil { - t.Log("found block directory in main compaction directory", fi.Name()) - t.Fail() - return - } - } - - select { - case <-time.After(100 * time.Millisecond): - continue - case <-ctx.Done(): - return - } - } - }() - - logger := log.NewLogfmtLogger(os.Stderr) - - reg := prometheus.NewRegistry() - - duplicateBlocksFilter := NewShardAwareDeduplicateFilter() - noCompactMarkerFilter := NewNoCompactionMarkFilter(objstore.WithNoopInstr(bkt), true) - metaFetcher, err := block.NewMetaFetcher(nil, 32, objstore.WithNoopInstr(bkt), "", nil, []block.MetadataFilter{ - duplicateBlocksFilter, - noCompactMarkerFilter, - }) - require.NoError(t, err) - - blocksMarkedForDeletion := promauto.With(nil).NewCounter(prometheus.CounterOpts{}) - sy, err := NewMetaSyncer(nil, nil, bkt, metaFetcher, duplicateBlocksFilter, blocksMarkedForDeletion) - require.NoError(t, err) - - comp, err := tsdb.NewLeveledCompactor(ctx, reg, logger, []int64{1000, 3000}, nil, nil, true) - require.NoError(t, err) - - planner := NewSplitAndMergePlanner([]int64{1000, 3000}) - grouper := NewSplitAndMergeGrouper("user-1", []int64{1000, 3000}, 0, 0, logger) - metrics := NewBucketCompactorMetrics(blocksMarkedForDeletion, prometheus.NewPedanticRegistry()) - bComp, err := NewBucketCompactor(logger, sy, grouper, planner, comp, dir, bkt, 2, true, ownAllJobs, sortJobsByNewestBlocksFirst, 0, 4, metrics) - require.NoError(t, err) - - // Compaction on empty should not fail. - require.NoError(t, bComp.Compact(ctx, 0), 0) - assert.Equal(t, 0.0, promtest.ToFloat64(sy.metrics.blocksMarkedForDeletion)) - assert.Equal(t, 0.0, promtest.ToFloat64(sy.metrics.garbageCollectionFailures)) - assert.Equal(t, 0.0, promtest.ToFloat64(metrics.blocksMarkedForNoCompact)) - assert.Equal(t, 0.0, promtest.ToFloat64(metrics.groupCompactions)) - assert.Equal(t, 0.0, promtest.ToFloat64(metrics.groupCompactionRunsStarted)) - assert.Equal(t, 0.0, promtest.ToFloat64(metrics.groupCompactionRunsCompleted)) - assert.Equal(t, 0.0, promtest.ToFloat64(metrics.groupCompactionRunsFailed)) - - _, err = os.Stat(dir) - assert.True(t, os.IsNotExist(err), "dir %s should be remove after compaction.", dir) - - // Test label name with slash, regression: https://github.com/thanos-io/thanos/issues/1661. - extLabels := labels.FromStrings("e1", "1/weird") - extLabels2 := labels.FromStrings("e1", "1") - metas := createAndUpload(t, bkt, []blockgenSpec{ - { - numSamples: 100, mint: 500, maxt: 1000, extLset: extLabels, res: 124, - series: []labels.Labels{ - labels.FromStrings("a", "1"), - labels.FromStrings("a", "2", "b", "2"), - labels.FromStrings("a", "3"), - labels.FromStrings("a", "4"), - }, - }, - { - numSamples: 100, mint: 2000, maxt: 3000, extLset: extLabels, res: 124, - series: []labels.Labels{ - labels.FromStrings("a", "3"), - labels.FromStrings("a", "4"), - labels.FromStrings("a", "5"), - labels.FromStrings("a", "6"), - }, - }, - // Mix order to make sure compactor is able to deduct min time / max time. - // Currently TSDB does not produces empty blocks (see: https://github.com/prometheus/tsdb/pull/374). However before v2.7.0 it was - // so we still want to mimick this case as close as possible. - { - mint: 1000, maxt: 2000, extLset: extLabels, res: 124, - // Empty block. - }, - // Due to TSDB compaction delay (not compacting fresh block), we need one more block to be pushed to trigger compaction. - { - numSamples: 100, mint: 3000, maxt: 4000, extLset: extLabels, res: 124, - series: []labels.Labels{ - labels.FromStrings("a", "7"), - }, - }, - // Extra block for "distraction" for different resolution and one for different labels. - { - numSamples: 100, mint: 5000, maxt: 6000, extLset: labels.FromStrings("e1", "2"), res: 124, - series: []labels.Labels{ - labels.FromStrings("a", "7"), - }, - }, - // Extra block for "distraction" for different resolution and one for different labels. - { - numSamples: 100, mint: 4000, maxt: 5000, extLset: extLabels, res: 0, - series: []labels.Labels{ - labels.FromStrings("a", "7"), - }, - }, - // Second group (extLabels2). - { - numSamples: 100, mint: 2000, maxt: 3000, extLset: extLabels2, res: 124, - series: []labels.Labels{ - labels.FromStrings("a", "3"), - labels.FromStrings("a", "4"), - labels.FromStrings("a", "6"), - }, - }, - { - numSamples: 100, mint: 0, maxt: 1000, extLset: extLabels2, res: 124, - series: []labels.Labels{ - labels.FromStrings("a", "1"), - labels.FromStrings("a", "2", "b", "2"), - labels.FromStrings("a", "3"), - labels.FromStrings("a", "4"), - }, - }, - // Due to TSDB compaction delay (not compacting fresh block), we need one more block to be pushed to trigger compaction. - { - numSamples: 100, mint: 3000, maxt: 4000, extLset: extLabels2, res: 124, - series: []labels.Labels{ - labels.FromStrings("a", "7"), - }, - }, - }, []blockgenSpec{ - { - numSamples: 100, mint: 0, maxt: 499, extLset: extLabels, res: 124, - series: []labels.Labels{ - labels.FromStrings("a", "1"), - labels.FromStrings("a", "1", "b", "2"), - labels.FromStrings("a", "3"), - labels.FromStrings("a", "4"), - }, - }, - }) - - require.NoError(t, bComp.Compact(ctx, 0), 0) - assert.Equal(t, 5.0, promtest.ToFloat64(sy.metrics.blocksMarkedForDeletion)) - assert.Equal(t, 1.0, promtest.ToFloat64(metrics.blocksMarkedForNoCompact)) - assert.Equal(t, 0.0, promtest.ToFloat64(sy.metrics.garbageCollectionFailures)) - assert.Equal(t, 2.0, promtest.ToFloat64(metrics.groupCompactions)) - assert.Equal(t, 3.0, promtest.ToFloat64(metrics.groupCompactionRunsStarted)) - assert.Equal(t, 2.0, promtest.ToFloat64(metrics.groupCompactionRunsCompleted)) - assert.Equal(t, 1.0, promtest.ToFloat64(metrics.groupCompactionRunsFailed)) - - _, err = os.Stat(dir) - assert.True(t, os.IsNotExist(err), "dir %s should be remove after compaction.", dir) - - // Check object storage. All blocks that were included in new compacted one should be removed. New compacted ones - // are present and looks as expected. - nonCompactedExpected := map[ulid.ULID]bool{ - metas[3].ULID: false, - metas[4].ULID: false, - metas[5].ULID: false, - metas[8].ULID: false, - metas[9].ULID: false, - } - others := map[string]block.Meta{} - require.NoError(t, bkt.Iter(ctx, "", func(n string) error { - id, ok := block.IsBlockDir(n) - if !ok { - return nil - } - - if _, ok := nonCompactedExpected[id]; ok { - nonCompactedExpected[id] = true - return nil - } - - meta, err := block.DownloadMeta(ctx, logger, bkt, id) - if err != nil { - return err - } - - others[DefaultGroupKey(meta.Thanos)] = meta - return nil - })) - - for id, found := range nonCompactedExpected { - assert.True(t, found, "not found expected block %s", id.String()) - } - - // We expect two compacted blocks only outside of what we expected in `nonCompactedExpected`. - assert.Equal(t, 2, len(others)) - { - meta, ok := others[defaultGroupKey(124, extLabels)] - assert.True(t, ok, "meta not found") - - assert.Equal(t, int64(500), meta.MinTime) - assert.Equal(t, int64(3000), meta.MaxTime) - assert.Equal(t, uint64(6), meta.Stats.NumSeries) - assert.Equal(t, uint64(2*4*100), meta.Stats.NumSamples) // Only 2 times 4*100 because one block was empty. - assert.Equal(t, 2, meta.Compaction.Level) - assert.Equal(t, []ulid.ULID{metas[0].ULID, metas[1].ULID, metas[2].ULID}, meta.Compaction.Sources) - - // Check thanos meta. - assert.True(t, labels.Equal(extLabels, labels.FromMap(meta.Thanos.Labels)), "ext labels does not match") - assert.Equal(t, int64(124), meta.Thanos.Downsample.Resolution) - assert.True(t, len(meta.Thanos.SegmentFiles) > 0, "compacted blocks have segment files set") - } - { - meta, ok := others[defaultGroupKey(124, extLabels2)] - assert.True(t, ok, "meta not found") - - assert.Equal(t, int64(0), meta.MinTime) - assert.Equal(t, int64(3000), meta.MaxTime) - assert.Equal(t, uint64(5), meta.Stats.NumSeries) - assert.Equal(t, uint64(2*4*100-100), meta.Stats.NumSamples) - assert.Equal(t, 2, meta.Compaction.Level) - assert.Equal(t, []ulid.ULID{metas[6].ULID, metas[7].ULID}, meta.Compaction.Sources) - - // Check thanos meta. - assert.True(t, labels.Equal(extLabels2, labels.FromMap(meta.Thanos.Labels)), "ext labels does not match") - assert.Equal(t, int64(124), meta.Thanos.Downsample.Resolution) - assert.True(t, len(meta.Thanos.SegmentFiles) > 0, "compacted blocks have segment files set") - } - }) -} - -type blockgenSpec struct { - mint, maxt int64 - series []labels.Labels - numSamples int - extLset labels.Labels - res int64 -} - -func createAndUpload(t testing.TB, bkt objstore.Bucket, blocks []blockgenSpec, blocksWithOutOfOrderChunks []blockgenSpec) (metas []*block.Meta) { - prepareDir := t.TempDir() - - ctx, cancel := context.WithTimeout(context.Background(), 20*time.Second) - defer cancel() - - for _, b := range blocks { - id, meta := createBlock(ctx, t, prepareDir, b) - metas = append(metas, meta) - require.NoError(t, block.Upload(ctx, log.NewNopLogger(), bkt, filepath.Join(prepareDir, id.String()), nil)) - } - for _, b := range blocksWithOutOfOrderChunks { - id, meta := createBlock(ctx, t, prepareDir, b) - - err := putOutOfOrderIndex(filepath.Join(prepareDir, id.String()), b.mint, b.maxt) - require.NoError(t, err) - - metas = append(metas, meta) - require.NoError(t, block.Upload(ctx, log.NewNopLogger(), bkt, filepath.Join(prepareDir, id.String()), nil)) - } - - return metas -} - -func createBlock(ctx context.Context, t testing.TB, prepareDir string, b blockgenSpec) (id ulid.ULID, meta *block.Meta) { - var err error - if b.numSamples == 0 { - id, err = createEmptyBlock(prepareDir, b.mint, b.maxt, b.extLset, b.res) - } else { - id, err = createBlockWithOptions(ctx, prepareDir, b.series, b.numSamples, b.mint, b.maxt, b.extLset, b.res, false) - } - require.NoError(t, err) - - meta, err = block.ReadMetaFromDir(filepath.Join(prepareDir, id.String())) - require.NoError(t, err) - return -} - -// Regression test for Thanos issue #2459. -func TestGarbageCollectDoesntCreateEmptyBlocksWithDeletionMarksOnly(t *testing.T) { - logger := log.NewLogfmtLogger(os.Stderr) - - foreachStore(t, func(t *testing.T, bkt objstore.Bucket) { - // Use bucket with global markers to make sure that our custom filters work correctly. - bkt = block.BucketWithGlobalMarkers(bkt) - - ctx, cancel := context.WithTimeout(context.Background(), 120*time.Second) - defer cancel() - - // Generate two blocks, and then another block that covers both of them. - var metas []*block.Meta - var ids []ulid.ULID - - for i := 0; i < 2; i++ { - var m block.Meta - - m.Version = 1 - m.ULID = ulid.MustNew(uint64(i), nil) - m.Compaction.Sources = []ulid.ULID{m.ULID} - m.Compaction.Level = 1 - - ids = append(ids, m.ULID) - metas = append(metas, &m) - } - - var m1 block.Meta - m1.Version = 1 - m1.ULID = ulid.MustNew(100, nil) - m1.Compaction.Level = 2 - m1.Compaction.Sources = ids - m1.Thanos.Downsample.Resolution = 0 - - // Create all blocks in the bucket. - for _, m := range append(metas, &m1) { - fmt.Println("create", m.ULID) - var buf bytes.Buffer - require.NoError(t, json.NewEncoder(&buf).Encode(&m)) - require.NoError(t, bkt.Upload(ctx, path.Join(m.ULID.String(), block.MetaFilename), &buf)) - } - - blocksMarkedForDeletion := promauto.With(nil).NewCounter(prometheus.CounterOpts{}) - - duplicateBlocksFilter := NewShardAwareDeduplicateFilter() - metaFetcher, err := block.NewMetaFetcher(nil, 32, objstore.WithNoopInstr(bkt), "", nil, []block.MetadataFilter{ - duplicateBlocksFilter, - }) - require.NoError(t, err) - - sy, err := NewMetaSyncer(nil, nil, bkt, metaFetcher, duplicateBlocksFilter, blocksMarkedForDeletion) - require.NoError(t, err) - - // Do one initial synchronization with the bucket. - require.NoError(t, sy.SyncMetas(ctx)) - require.NoError(t, sy.GarbageCollect(ctx)) - - rem, err := listBlocksMarkedForDeletion(ctx, bkt) - require.NoError(t, err) - - sort.Slice(rem, func(i, j int) bool { - return rem[i].Compare(rem[j]) < 0 - }) - - assert.Equal(t, ids, rem) - - // Delete source blocks. - for _, id := range ids { - require.NoError(t, block.Delete(ctx, logger, bkt, id)) - } - - // After another garbage-collect, we should not find new blocks that are deleted with new deletion mark files. - require.NoError(t, sy.SyncMetas(ctx)) - require.NoError(t, sy.GarbageCollect(ctx)) - - rem, err = listBlocksMarkedForDeletion(ctx, bkt) - require.NoError(t, err) - assert.Equal(t, 0, len(rem)) - }) -} - -func listBlocksMarkedForDeletion(ctx context.Context, bkt objstore.Bucket) ([]ulid.ULID, error) { - var rem []ulid.ULID - err := bkt.Iter(ctx, "", func(n string) error { - id, ok := block.IsBlockDir(n) - if !ok { - return nil - } - deletionMarkFile := path.Join(id.String(), block.DeletionMarkFilename) - - exists, err := bkt.Exists(ctx, deletionMarkFile) - if err != nil { - return err - } - if exists { - rem = append(rem, id) - } - return nil - }) - return rem, err -} - -func foreachStore(t *testing.T, testFn func(t *testing.T, bkt objstore.Bucket)) { - t.Parallel() - - // Mandatory Inmem. Not parallel, to detect problem early. - if ok := t.Run("inmem", func(t *testing.T) { - testFn(t, objstore.NewInMemBucket()) - }); !ok { - return - } - - // Mandatory Filesystem. - t.Run("filesystem", func(t *testing.T) { - t.Parallel() - - dir := t.TempDir() - - b, err := filesystem.NewBucket(dir) - require.NoError(t, err) - testFn(t, b) - }) -} - -// createEmptyBlock produces empty block like it was the case before fix: https://github.com/prometheus/tsdb/pull/374. -// (Prometheus pre v2.7.0). -func createEmptyBlock(dir string, mint, maxt int64, extLset labels.Labels, resolution int64) (ulid.ULID, error) { - entropy := rand.New(rand.NewSource(time.Now().UnixNano())) - uid := ulid.MustNew(ulid.Now(), entropy) - - if err := os.Mkdir(path.Join(dir, uid.String()), os.ModePerm); err != nil { - return ulid.ULID{}, errors.Wrap(err, "close index") - } - - if err := os.Mkdir(path.Join(dir, uid.String(), "chunks"), os.ModePerm); err != nil { - return ulid.ULID{}, errors.Wrap(err, "close index") - } - - w, err := index.NewWriter(context.Background(), path.Join(dir, uid.String(), "index")) - if err != nil { - return ulid.ULID{}, errors.Wrap(err, "new index") - } - - if err := w.Close(); err != nil { - return ulid.ULID{}, errors.Wrap(err, "close index") - } - - m := tsdb.BlockMeta{ - Version: 1, - ULID: uid, - MinTime: mint, - MaxTime: maxt, - Compaction: tsdb.BlockMetaCompaction{ - Level: 1, - Sources: []ulid.ULID{uid}, - }, - } - b, err := json.Marshal(&m) - if err != nil { - return ulid.ULID{}, err - } - - if err := os.WriteFile(path.Join(dir, uid.String(), "meta.json"), b, os.ModePerm); err != nil { - return ulid.ULID{}, errors.Wrap(err, "saving meta.json") - } - - if _, err = block.InjectThanosMeta(log.NewNopLogger(), filepath.Join(dir, uid.String()), block.ThanosMeta{ - Labels: extLset.Map(), - Downsample: block.ThanosDownsample{Resolution: resolution}, - Source: block.TestSource, - }, nil); err != nil { - return ulid.ULID{}, errors.Wrap(err, "finalize block") - } - - return uid, nil -} - -func createBlockWithOptions( - ctx context.Context, - dir string, - series []labels.Labels, - numSamples int, - mint, maxt int64, - extLset labels.Labels, - resolution int64, - tombstones bool, -) (id ulid.ULID, err error) { - headOpts := tsdb.DefaultHeadOptions() - headOpts.ChunkDirRoot = filepath.Join(dir, "chunks") - headOpts.ChunkRange = 10000000000 - h, err := tsdb.NewHead(nil, nil, nil, nil, headOpts, nil) - if err != nil { - return id, errors.Wrap(err, "create head block") - } - defer func() { - runutil.CloseWithErrCapture(&err, h, "TSDB Head") - if e := os.RemoveAll(headOpts.ChunkDirRoot); e != nil { - err = errors.Wrap(e, "delete chunks dir") - } - }() - - var g errgroup.Group - timeStepSize := (maxt - mint) / int64(numSamples+1) - batchSize := len(series) / runtime.GOMAXPROCS(0) - - for len(series) > 0 { - l := batchSize - if len(series) < 1000 { - l = len(series) - } - batch := series[:l] - series = series[l:] - - g.Go(func() error { - t := mint - - for i := 0; i < numSamples; i++ { - app := h.Appender(ctx) - - for _, lset := range batch { - _, err := app.Append(0, lset, t, rand.Float64()) - if err != nil { - if rerr := app.Rollback(); rerr != nil { - err = errors.Wrapf(err, "rollback failed: %v", rerr) - } - - return errors.Wrap(err, "add sample") - } - } - if err := app.Commit(); err != nil { - return errors.Wrap(err, "commit") - } - t += timeStepSize - } - return nil - }) - } - if err := g.Wait(); err != nil { - return id, err - } - c, err := tsdb.NewLeveledCompactor(ctx, nil, log.NewNopLogger(), []int64{maxt - mint}, nil, nil, true) - if err != nil { - return id, errors.Wrap(err, "create compactor") - } - - id, err = c.Write(dir, h, mint, maxt, nil) - if err != nil { - return id, errors.Wrap(err, "write block") - } - - if id.Compare(ulid.ULID{}) == 0 { - return id, errors.Errorf("nothing to write, asked for %d samples", numSamples) - } - - blockDir := filepath.Join(dir, id.String()) - - if _, err = block.InjectThanosMeta(log.NewNopLogger(), blockDir, block.ThanosMeta{ - Labels: extLset.Map(), - Downsample: block.ThanosDownsample{Resolution: resolution}, - Source: block.TestSource, - Files: []block.File{}, - }, nil); err != nil { - return id, errors.Wrap(err, "finalize block") - } - - if !tombstones { - if err = os.Remove(filepath.Join(dir, id.String(), "tombstones")); err != nil { - return id, errors.Wrap(err, "remove tombstones") - } - } - - return id, nil -} - -var indexFilename = "index" - -type indexWriterSeries struct { - labels labels.Labels - chunks []chunks.Meta // series file offset of chunks -} - -type indexWriterSeriesSlice []*indexWriterSeries - -// putOutOfOrderIndex updates the index in blockDir with an index containing an out-of-order chunk -// copied from https://github.com/prometheus/prometheus/blob/b1ed4a0a663d0c62526312311c7529471abbc565/tsdb/index/index_test.go#L346 -func putOutOfOrderIndex(blockDir string, minTime int64, maxTime int64) error { - if minTime >= maxTime || minTime+4 >= maxTime { - return fmt.Errorf("minTime must be at least 4 less than maxTime to not create overlapping chunks") - } - - lbls := []labels.Labels{ - labels.FromStrings("lbl1", "1"), - } - - // Sort labels as the index writer expects series in sorted order. - sort.Sort(labels.Slice(lbls)) - - symbols := map[string]struct{}{} - for _, lset := range lbls { - lset.Range(func(l labels.Label) { - symbols[l.Name] = struct{}{} - symbols[l.Value] = struct{}{} - }) - } - - var input indexWriterSeriesSlice - - // Generate ChunkMetas for every label set. - for _, lset := range lbls { - var metas []chunks.Meta - // only need two chunks that are out-of-order - chk1 := chunks.Meta{ - MinTime: maxTime - 2, - MaxTime: maxTime - 1, - Ref: chunks.ChunkRef(rand.Uint64()), - Chunk: chunkenc.NewXORChunk(), - } - metas = append(metas, chk1) - chk2 := chunks.Meta{ - MinTime: minTime + 1, - MaxTime: minTime + 2, - Ref: chunks.ChunkRef(rand.Uint64()), - Chunk: chunkenc.NewXORChunk(), - } - metas = append(metas, chk2) - - input = append(input, &indexWriterSeries{ - labels: lset, - chunks: metas, - }) - } - - iw, err := index.NewWriter(context.Background(), filepath.Join(blockDir, indexFilename)) - if err != nil { - return err - } - - syms := []string{} - for s := range symbols { - syms = append(syms, s) - } - slices.Sort(syms) - for _, s := range syms { - if err := iw.AddSymbol(s); err != nil { - return err - } - } - - // Population procedure as done by compaction. - var ( - postings = index.NewMemPostings() - values = map[string]map[string]struct{}{} - ) - - for i, s := range input { - if err := iw.AddSeries(storage.SeriesRef(i), s.labels, s.chunks...); err != nil { - return err - } - - s.labels.Range(func(l labels.Label) { - valset, ok := values[l.Name] - if !ok { - valset = map[string]struct{}{} - values[l.Name] = valset - } - valset[l.Value] = struct{}{} - }) - postings.Add(storage.SeriesRef(i), s.labels) - } - - return iw.Close() -} +// import ( +// "bytes" +// "context" +// "encoding/json" +// "fmt" +// "math/rand" +// "os" +// "path" +// "path/filepath" +// "runtime" +// "sort" +// "strings" +// "testing" +// "time" + +// "github.com/go-kit/log" +// "github.com/grafana/dskit/runutil" +// "github.com/oklog/ulid" +// "github.com/pkg/errors" +// "github.com/prometheus/client_golang/prometheus" +// "github.com/prometheus/client_golang/prometheus/promauto" +// promtest "github.com/prometheus/client_golang/prometheus/testutil" +// "github.com/prometheus/prometheus/model/labels" +// "github.com/prometheus/prometheus/storage" +// "github.com/prometheus/prometheus/tsdb" +// "github.com/prometheus/prometheus/tsdb/chunkenc" +// "github.com/prometheus/prometheus/tsdb/chunks" +// "github.com/prometheus/prometheus/tsdb/index" +// "github.com/stretchr/testify/assert" +// "github.com/stretchr/testify/require" +// "github.com/thanos-io/objstore" +// "github.com/thanos-io/objstore/providers/filesystem" +// "golang.org/x/exp/slices" +// "golang.org/x/sync/errgroup" + +// "github.com/grafana/mimir/pkg/storage/tsdb/block" +// ) + +// func TestSyncer_GarbageCollect_e2e(t *testing.T) { +// foreachStore(t, func(t *testing.T, bkt objstore.Bucket) { +// // Use bucket with global markers to make sure that our custom filters work correctly. +// bkt = block.BucketWithGlobalMarkers(bkt) + +// ctx, cancel := context.WithTimeout(context.Background(), 120*time.Second) +// defer cancel() + +// // Generate 10 source block metas and construct higher level blocks +// // that are higher compactions of them. +// var metas []*block.Meta +// var ids []ulid.ULID + +// for i := 0; i < 10; i++ { +// var m block.Meta + +// m.Version = 1 +// m.ULID = ulid.MustNew(uint64(i), nil) +// m.Compaction.Sources = []ulid.ULID{m.ULID} +// m.Compaction.Level = 1 +// m.MinTime = 0 +// m.MaxTime = 2 * time.Hour.Milliseconds() + +// ids = append(ids, m.ULID) +// metas = append(metas, &m) +// } + +// var m1 block.Meta +// m1.Version = 1 +// m1.ULID = ulid.MustNew(100, nil) +// m1.Compaction.Level = 2 +// m1.Compaction.Sources = ids[:4] +// m1.Thanos.Downsample.Resolution = 0 + +// var m2 block.Meta +// m2.Version = 1 +// m2.ULID = ulid.MustNew(200, nil) +// m2.Compaction.Level = 2 +// m2.Compaction.Sources = ids[4:8] // last two source IDs is not part of a level 2 block. +// m2.Thanos.Downsample.Resolution = 0 + +// var m3 block.Meta +// m3.Version = 1 +// m3.ULID = ulid.MustNew(300, nil) +// m3.Compaction.Level = 3 +// m3.Compaction.Sources = ids[:9] // last source ID is not part of level 3 block. +// m3.Thanos.Downsample.Resolution = 0 +// m3.MinTime = 0 +// m3.MaxTime = 2 * time.Hour.Milliseconds() + +// var m4 block.Meta +// m4.Version = 1 +// m4.ULID = ulid.MustNew(400, nil) +// m4.Compaction.Level = 2 +// m4.Compaction.Sources = ids[9:] // covers the last block but is a different resolution. Must not trigger deletion. +// m4.Thanos.Downsample.Resolution = 1000 +// m4.MinTime = 0 +// m4.MaxTime = 2 * time.Hour.Milliseconds() + +// var m5 block.Meta +// m5.Version = 1 +// m5.ULID = ulid.MustNew(500, nil) +// m5.Compaction.Level = 2 +// m5.Compaction.Sources = ids[8:9] // built from block 8, but different resolution. Block 8 is already included in m3, can be deleted. +// m5.Thanos.Downsample.Resolution = 1000 +// m5.MinTime = 0 +// m5.MaxTime = 2 * time.Hour.Milliseconds() + +// // Create all blocks in the bucket. +// for _, m := range append(metas, &m1, &m2, &m3, &m4, &m5) { +// fmt.Println("create", m.ULID) +// var buf bytes.Buffer +// require.NoError(t, json.NewEncoder(&buf).Encode(&m)) +// require.NoError(t, bkt.Upload(ctx, path.Join(m.ULID.String(), block.MetaFilename), &buf)) +// } + +// duplicateBlocksFilter := NewShardAwareDeduplicateFilter() +// metaFetcher, err := block.NewMetaFetcher(nil, 32, objstore.WithNoopInstr(bkt), "", nil, []block.MetadataFilter{ +// duplicateBlocksFilter, +// }) +// require.NoError(t, err) + +// blocksMarkedForDeletion := promauto.With(nil).NewCounter(prometheus.CounterOpts{}) +// sy, err := NewMetaSyncer(nil, nil, bkt, metaFetcher, duplicateBlocksFilter, blocksMarkedForDeletion) +// require.NoError(t, err) + +// // Do one initial synchronization with the bucket. +// require.NoError(t, sy.SyncMetas(ctx)) +// require.NoError(t, sy.GarbageCollect(ctx)) + +// var rem []ulid.ULID +// err = bkt.Iter(ctx, "", func(n string) error { +// id, ok := block.IsBlockDir(n) +// if !ok { +// return nil +// } +// deletionMarkFile := path.Join(id.String(), block.DeletionMarkFilename) + +// exists, err := bkt.Exists(ctx, deletionMarkFile) +// if err != nil { +// return err +// } +// if !exists { +// rem = append(rem, id) +// } +// return nil +// }) +// require.NoError(t, err) + +// sort.Slice(rem, func(i, j int) bool { +// return rem[i].Compare(rem[j]) < 0 +// }) + +// // Only the level 3 block, the last source block in both resolutions should be left. +// assert.Equal(t, []ulid.ULID{metas[9].ULID, m3.ULID, m4.ULID, m5.ULID}, rem) + +// // After another sync the changes should also be reflected in the local groups. +// require.NoError(t, sy.SyncMetas(ctx)) +// require.NoError(t, sy.GarbageCollect(ctx)) + +// // Only the level 3 block, the last source block in both resolutions should be left. +// grouper := NewSplitAndMergeGrouper("user-1", []int64{2 * time.Hour.Milliseconds()}, 0, 0, log.NewNopLogger()) +// groups, err := grouper.Groups(sy.Metas()) +// require.NoError(t, err) + +// assert.Equal(t, "0@17241709254077376921-merge--0-7200000", groups[0].Key()) +// assert.Equal(t, []ulid.ULID{metas[9].ULID, m3.ULID}, groups[0].IDs()) + +// assert.Equal(t, "1000@17241709254077376921-merge--0-7200000", groups[1].Key()) +// assert.Equal(t, []ulid.ULID{m4.ULID, m5.ULID}, groups[1].IDs()) +// }) +// } + +// func TestGroupCompactE2E(t *testing.T) { +// foreachStore(t, func(t *testing.T, bkt objstore.Bucket) { +// // Use bucket with global markers to make sure that our custom filters work correctly. +// bkt = block.BucketWithGlobalMarkers(bkt) + +// ctx, cancel := context.WithTimeout(context.Background(), 120*time.Second) +// defer cancel() + +// // Create fresh, empty directory for actual test. +// dir := t.TempDir() + +// // Start dir checker... we make sure that "dir" only contains group subdirectories during compaction, +// // and not any block directories. Dir checker stops when context is canceled, or on first error, +// // in which case error is logger and test is failed. (We cannot use Fatal or FailNow from a goroutine). +// go func() { +// for ctx.Err() == nil { +// fs, err := os.ReadDir(dir) +// if err != nil && !os.IsNotExist(err) { +// t.Log("error while listing directory", dir) +// t.Fail() +// return +// } + +// for _, fi := range fs { +// // Suffix used by Prometheus LeveledCompactor when doing compaction. +// toCheck := strings.TrimSuffix(fi.Name(), ".tmp-for-creation") + +// _, err := ulid.Parse(toCheck) +// if err == nil { +// t.Log("found block directory in main compaction directory", fi.Name()) +// t.Fail() +// return +// } +// } + +// select { +// case <-time.After(100 * time.Millisecond): +// continue +// case <-ctx.Done(): +// return +// } +// } +// }() + +// logger := log.NewLogfmtLogger(os.Stderr) + +// reg := prometheus.NewRegistry() + +// duplicateBlocksFilter := NewShardAwareDeduplicateFilter() +// noCompactMarkerFilter := NewNoCompactionMarkFilter(objstore.WithNoopInstr(bkt), true) +// metaFetcher, err := block.NewMetaFetcher(nil, 32, objstore.WithNoopInstr(bkt), "", nil, []block.MetadataFilter{ +// duplicateBlocksFilter, +// noCompactMarkerFilter, +// }) +// require.NoError(t, err) + +// blocksMarkedForDeletion := promauto.With(nil).NewCounter(prometheus.CounterOpts{}) +// sy, err := NewMetaSyncer(nil, nil, bkt, metaFetcher, duplicateBlocksFilter, blocksMarkedForDeletion) +// require.NoError(t, err) + +// comp, err := tsdb.NewLeveledCompactor(ctx, reg, logger, []int64{1000, 3000}, nil, nil, true) +// require.NoError(t, err) + +// planner := NewSplitAndMergePlanner([]int64{1000, 3000}) +// grouper := NewSplitAndMergeGrouper("user-1", []int64{1000, 3000}, 0, 0, logger) +// metrics := NewBucketCompactorMetrics(blocksMarkedForDeletion, prometheus.NewPedanticRegistry()) +// bComp, err := NewBucketCompactor(logger, sy, grouper, planner, comp, dir, bkt, 2, true, ownAllJobs, sortJobsByNewestBlocksFirst, 0, 4, metrics) +// require.NoError(t, err) + +// // Compaction on empty should not fail. +// require.NoError(t, bComp.Compact(ctx, 0), 0) +// assert.Equal(t, 0.0, promtest.ToFloat64(sy.metrics.blocksMarkedForDeletion)) +// assert.Equal(t, 0.0, promtest.ToFloat64(sy.metrics.garbageCollectionFailures)) +// assert.Equal(t, 0.0, promtest.ToFloat64(metrics.blocksMarkedForNoCompact)) +// assert.Equal(t, 0.0, promtest.ToFloat64(metrics.groupCompactions)) +// assert.Equal(t, 0.0, promtest.ToFloat64(metrics.groupCompactionRunsStarted)) +// assert.Equal(t, 0.0, promtest.ToFloat64(metrics.groupCompactionRunsCompleted)) +// assert.Equal(t, 0.0, promtest.ToFloat64(metrics.groupCompactionRunsFailed)) + +// _, err = os.Stat(dir) +// assert.True(t, os.IsNotExist(err), "dir %s should be remove after compaction.", dir) + +// // Test label name with slash, regression: https://github.com/thanos-io/thanos/issues/1661. +// extLabels := labels.FromStrings("e1", "1/weird") +// extLabels2 := labels.FromStrings("e1", "1") +// metas := createAndUpload(t, bkt, []blockgenSpec{ +// { +// numSamples: 100, mint: 500, maxt: 1000, extLset: extLabels, res: 124, +// series: []labels.Labels{ +// labels.FromStrings("a", "1"), +// labels.FromStrings("a", "2", "b", "2"), +// labels.FromStrings("a", "3"), +// labels.FromStrings("a", "4"), +// }, +// }, +// { +// numSamples: 100, mint: 2000, maxt: 3000, extLset: extLabels, res: 124, +// series: []labels.Labels{ +// labels.FromStrings("a", "3"), +// labels.FromStrings("a", "4"), +// labels.FromStrings("a", "5"), +// labels.FromStrings("a", "6"), +// }, +// }, +// // Mix order to make sure compactor is able to deduct min time / max time. +// // Currently TSDB does not produces empty blocks (see: https://github.com/prometheus/tsdb/pull/374). However before v2.7.0 it was +// // so we still want to mimick this case as close as possible. +// { +// mint: 1000, maxt: 2000, extLset: extLabels, res: 124, +// // Empty block. +// }, +// // Due to TSDB compaction delay (not compacting fresh block), we need one more block to be pushed to trigger compaction. +// { +// numSamples: 100, mint: 3000, maxt: 4000, extLset: extLabels, res: 124, +// series: []labels.Labels{ +// labels.FromStrings("a", "7"), +// }, +// }, +// // Extra block for "distraction" for different resolution and one for different labels. +// { +// numSamples: 100, mint: 5000, maxt: 6000, extLset: labels.FromStrings("e1", "2"), res: 124, +// series: []labels.Labels{ +// labels.FromStrings("a", "7"), +// }, +// }, +// // Extra block for "distraction" for different resolution and one for different labels. +// { +// numSamples: 100, mint: 4000, maxt: 5000, extLset: extLabels, res: 0, +// series: []labels.Labels{ +// labels.FromStrings("a", "7"), +// }, +// }, +// // Second group (extLabels2). +// { +// numSamples: 100, mint: 2000, maxt: 3000, extLset: extLabels2, res: 124, +// series: []labels.Labels{ +// labels.FromStrings("a", "3"), +// labels.FromStrings("a", "4"), +// labels.FromStrings("a", "6"), +// }, +// }, +// { +// numSamples: 100, mint: 0, maxt: 1000, extLset: extLabels2, res: 124, +// series: []labels.Labels{ +// labels.FromStrings("a", "1"), +// labels.FromStrings("a", "2", "b", "2"), +// labels.FromStrings("a", "3"), +// labels.FromStrings("a", "4"), +// }, +// }, +// // Due to TSDB compaction delay (not compacting fresh block), we need one more block to be pushed to trigger compaction. +// { +// numSamples: 100, mint: 3000, maxt: 4000, extLset: extLabels2, res: 124, +// series: []labels.Labels{ +// labels.FromStrings("a", "7"), +// }, +// }, +// }, []blockgenSpec{ +// { +// numSamples: 100, mint: 0, maxt: 499, extLset: extLabels, res: 124, +// series: []labels.Labels{ +// labels.FromStrings("a", "1"), +// labels.FromStrings("a", "1", "b", "2"), +// labels.FromStrings("a", "3"), +// labels.FromStrings("a", "4"), +// }, +// }, +// }) + +// require.NoError(t, bComp.Compact(ctx, 0), 0) +// assert.Equal(t, 5.0, promtest.ToFloat64(sy.metrics.blocksMarkedForDeletion)) +// assert.Equal(t, 1.0, promtest.ToFloat64(metrics.blocksMarkedForNoCompact)) +// assert.Equal(t, 0.0, promtest.ToFloat64(sy.metrics.garbageCollectionFailures)) +// assert.Equal(t, 2.0, promtest.ToFloat64(metrics.groupCompactions)) +// assert.Equal(t, 3.0, promtest.ToFloat64(metrics.groupCompactionRunsStarted)) +// assert.Equal(t, 2.0, promtest.ToFloat64(metrics.groupCompactionRunsCompleted)) +// assert.Equal(t, 1.0, promtest.ToFloat64(metrics.groupCompactionRunsFailed)) + +// _, err = os.Stat(dir) +// assert.True(t, os.IsNotExist(err), "dir %s should be remove after compaction.", dir) + +// // Check object storage. All blocks that were included in new compacted one should be removed. New compacted ones +// // are present and looks as expected. +// nonCompactedExpected := map[ulid.ULID]bool{ +// metas[3].ULID: false, +// metas[4].ULID: false, +// metas[5].ULID: false, +// metas[8].ULID: false, +// metas[9].ULID: false, +// } +// others := map[string]block.Meta{} +// require.NoError(t, bkt.Iter(ctx, "", func(n string) error { +// id, ok := block.IsBlockDir(n) +// if !ok { +// return nil +// } + +// if _, ok := nonCompactedExpected[id]; ok { +// nonCompactedExpected[id] = true +// return nil +// } + +// meta, err := block.DownloadMeta(ctx, logger, bkt, id) +// if err != nil { +// return err +// } + +// others[DefaultGroupKey(meta.Thanos)] = meta +// return nil +// })) + +// for id, found := range nonCompactedExpected { +// assert.True(t, found, "not found expected block %s", id.String()) +// } + +// // We expect two compacted blocks only outside of what we expected in `nonCompactedExpected`. +// assert.Equal(t, 2, len(others)) +// { +// meta, ok := others[defaultGroupKey(124, extLabels)] +// assert.True(t, ok, "meta not found") + +// assert.Equal(t, int64(500), meta.MinTime) +// assert.Equal(t, int64(3000), meta.MaxTime) +// assert.Equal(t, uint64(6), meta.Stats.NumSeries) +// assert.Equal(t, uint64(2*4*100), meta.Stats.NumSamples) // Only 2 times 4*100 because one block was empty. +// assert.Equal(t, 2, meta.Compaction.Level) +// assert.Equal(t, []ulid.ULID{metas[0].ULID, metas[1].ULID, metas[2].ULID}, meta.Compaction.Sources) + +// // Check thanos meta. +// assert.True(t, labels.Equal(extLabels, labels.FromMap(meta.Thanos.Labels)), "ext labels does not match") +// assert.Equal(t, int64(124), meta.Thanos.Downsample.Resolution) +// assert.True(t, len(meta.Thanos.SegmentFiles) > 0, "compacted blocks have segment files set") +// } +// { +// meta, ok := others[defaultGroupKey(124, extLabels2)] +// assert.True(t, ok, "meta not found") + +// assert.Equal(t, int64(0), meta.MinTime) +// assert.Equal(t, int64(3000), meta.MaxTime) +// assert.Equal(t, uint64(5), meta.Stats.NumSeries) +// assert.Equal(t, uint64(2*4*100-100), meta.Stats.NumSamples) +// assert.Equal(t, 2, meta.Compaction.Level) +// assert.Equal(t, []ulid.ULID{metas[6].ULID, metas[7].ULID}, meta.Compaction.Sources) + +// // Check thanos meta. +// assert.True(t, labels.Equal(extLabels2, labels.FromMap(meta.Thanos.Labels)), "ext labels does not match") +// assert.Equal(t, int64(124), meta.Thanos.Downsample.Resolution) +// assert.True(t, len(meta.Thanos.SegmentFiles) > 0, "compacted blocks have segment files set") +// } +// }) +// } + +// type blockgenSpec struct { +// mint, maxt int64 +// series []labels.Labels +// numSamples int +// extLset labels.Labels +// res int64 +// } + +// func createAndUpload(t testing.TB, bkt objstore.Bucket, blocks []blockgenSpec, blocksWithOutOfOrderChunks []blockgenSpec) (metas []*block.Meta) { +// prepareDir := t.TempDir() + +// ctx, cancel := context.WithTimeout(context.Background(), 20*time.Second) +// defer cancel() + +// for _, b := range blocks { +// id, meta := createBlock(ctx, t, prepareDir, b) +// metas = append(metas, meta) +// require.NoError(t, block.Upload(ctx, log.NewNopLogger(), bkt, filepath.Join(prepareDir, id.String()), nil)) +// } +// for _, b := range blocksWithOutOfOrderChunks { +// id, meta := createBlock(ctx, t, prepareDir, b) + +// err := putOutOfOrderIndex(filepath.Join(prepareDir, id.String()), b.mint, b.maxt) +// require.NoError(t, err) + +// metas = append(metas, meta) +// require.NoError(t, block.Upload(ctx, log.NewNopLogger(), bkt, filepath.Join(prepareDir, id.String()), nil)) +// } + +// return metas +// } + +// func createBlock(ctx context.Context, t testing.TB, prepareDir string, b blockgenSpec) (id ulid.ULID, meta *block.Meta) { +// var err error +// if b.numSamples == 0 { +// id, err = createEmptyBlock(prepareDir, b.mint, b.maxt, b.extLset, b.res) +// } else { +// id, err = createBlockWithOptions(ctx, prepareDir, b.series, b.numSamples, b.mint, b.maxt, b.extLset, b.res, false) +// } +// require.NoError(t, err) + +// meta, err = block.ReadMetaFromDir(filepath.Join(prepareDir, id.String())) +// require.NoError(t, err) +// return +// } + +// // Regression test for Thanos issue #2459. +// func TestGarbageCollectDoesntCreateEmptyBlocksWithDeletionMarksOnly(t *testing.T) { +// logger := log.NewLogfmtLogger(os.Stderr) + +// foreachStore(t, func(t *testing.T, bkt objstore.Bucket) { +// // Use bucket with global markers to make sure that our custom filters work correctly. +// bkt = block.BucketWithGlobalMarkers(bkt) + +// ctx, cancel := context.WithTimeout(context.Background(), 120*time.Second) +// defer cancel() + +// // Generate two blocks, and then another block that covers both of them. +// var metas []*block.Meta +// var ids []ulid.ULID + +// for i := 0; i < 2; i++ { +// var m block.Meta + +// m.Version = 1 +// m.ULID = ulid.MustNew(uint64(i), nil) +// m.Compaction.Sources = []ulid.ULID{m.ULID} +// m.Compaction.Level = 1 + +// ids = append(ids, m.ULID) +// metas = append(metas, &m) +// } + +// var m1 block.Meta +// m1.Version = 1 +// m1.ULID = ulid.MustNew(100, nil) +// m1.Compaction.Level = 2 +// m1.Compaction.Sources = ids +// m1.Thanos.Downsample.Resolution = 0 + +// // Create all blocks in the bucket. +// for _, m := range append(metas, &m1) { +// fmt.Println("create", m.ULID) +// var buf bytes.Buffer +// require.NoError(t, json.NewEncoder(&buf).Encode(&m)) +// require.NoError(t, bkt.Upload(ctx, path.Join(m.ULID.String(), block.MetaFilename), &buf)) +// } + +// blocksMarkedForDeletion := promauto.With(nil).NewCounter(prometheus.CounterOpts{}) + +// duplicateBlocksFilter := NewShardAwareDeduplicateFilter() +// metaFetcher, err := block.NewMetaFetcher(nil, 32, objstore.WithNoopInstr(bkt), "", nil, []block.MetadataFilter{ +// duplicateBlocksFilter, +// }) +// require.NoError(t, err) + +// sy, err := NewMetaSyncer(nil, nil, bkt, metaFetcher, duplicateBlocksFilter, blocksMarkedForDeletion) +// require.NoError(t, err) + +// // Do one initial synchronization with the bucket. +// require.NoError(t, sy.SyncMetas(ctx)) +// require.NoError(t, sy.GarbageCollect(ctx)) + +// rem, err := listBlocksMarkedForDeletion(ctx, bkt) +// require.NoError(t, err) + +// sort.Slice(rem, func(i, j int) bool { +// return rem[i].Compare(rem[j]) < 0 +// }) + +// assert.Equal(t, ids, rem) + +// // Delete source blocks. +// for _, id := range ids { +// require.NoError(t, block.Delete(ctx, logger, bkt, id)) +// } + +// // After another garbage-collect, we should not find new blocks that are deleted with new deletion mark files. +// require.NoError(t, sy.SyncMetas(ctx)) +// require.NoError(t, sy.GarbageCollect(ctx)) + +// rem, err = listBlocksMarkedForDeletion(ctx, bkt) +// require.NoError(t, err) +// assert.Equal(t, 0, len(rem)) +// }) +// } + +// func listBlocksMarkedForDeletion(ctx context.Context, bkt objstore.Bucket) ([]ulid.ULID, error) { +// var rem []ulid.ULID +// err := bkt.Iter(ctx, "", func(n string) error { +// id, ok := block.IsBlockDir(n) +// if !ok { +// return nil +// } +// deletionMarkFile := path.Join(id.String(), block.DeletionMarkFilename) + +// exists, err := bkt.Exists(ctx, deletionMarkFile) +// if err != nil { +// return err +// } +// if exists { +// rem = append(rem, id) +// } +// return nil +// }) +// return rem, err +// } + +// func foreachStore(t *testing.T, testFn func(t *testing.T, bkt objstore.Bucket)) { +// t.Parallel() + +// // Mandatory Inmem. Not parallel, to detect problem early. +// if ok := t.Run("inmem", func(t *testing.T) { +// testFn(t, objstore.NewInMemBucket()) +// }); !ok { +// return +// } + +// // Mandatory Filesystem. +// t.Run("filesystem", func(t *testing.T) { +// t.Parallel() + +// dir := t.TempDir() + +// b, err := filesystem.NewBucket(dir) +// require.NoError(t, err) +// testFn(t, b) +// }) +// } + +// // createEmptyBlock produces empty block like it was the case before fix: https://github.com/prometheus/tsdb/pull/374. +// // (Prometheus pre v2.7.0). +// func createEmptyBlock(dir string, mint, maxt int64, extLset labels.Labels, resolution int64) (ulid.ULID, error) { +// entropy := rand.New(rand.NewSource(time.Now().UnixNano())) +// uid := ulid.MustNew(ulid.Now(), entropy) + +// if err := os.Mkdir(path.Join(dir, uid.String()), os.ModePerm); err != nil { +// return ulid.ULID{}, errors.Wrap(err, "close index") +// } + +// if err := os.Mkdir(path.Join(dir, uid.String(), "chunks"), os.ModePerm); err != nil { +// return ulid.ULID{}, errors.Wrap(err, "close index") +// } + +// w, err := index.NewWriter(context.Background(), path.Join(dir, uid.String(), "index")) +// if err != nil { +// return ulid.ULID{}, errors.Wrap(err, "new index") +// } + +// if err := w.Close(); err != nil { +// return ulid.ULID{}, errors.Wrap(err, "close index") +// } + +// m := tsdb.BlockMeta{ +// Version: 1, +// ULID: uid, +// MinTime: mint, +// MaxTime: maxt, +// Compaction: tsdb.BlockMetaCompaction{ +// Level: 1, +// Sources: []ulid.ULID{uid}, +// }, +// } +// b, err := json.Marshal(&m) +// if err != nil { +// return ulid.ULID{}, err +// } + +// if err := os.WriteFile(path.Join(dir, uid.String(), "meta.json"), b, os.ModePerm); err != nil { +// return ulid.ULID{}, errors.Wrap(err, "saving meta.json") +// } + +// if _, err = block.InjectThanosMeta(log.NewNopLogger(), filepath.Join(dir, uid.String()), block.ThanosMeta{ +// Labels: extLset.Map(), +// Downsample: block.ThanosDownsample{Resolution: resolution}, +// Source: block.TestSource, +// }, nil); err != nil { +// return ulid.ULID{}, errors.Wrap(err, "finalize block") +// } + +// return uid, nil +// } + +// func createBlockWithOptions( +// ctx context.Context, +// dir string, +// series []labels.Labels, +// numSamples int, +// mint, maxt int64, +// extLset labels.Labels, +// resolution int64, +// tombstones bool, +// ) (id ulid.ULID, err error) { +// headOpts := tsdb.DefaultHeadOptions() +// headOpts.ChunkDirRoot = filepath.Join(dir, "chunks") +// headOpts.ChunkRange = 10000000000 +// h, err := tsdb.NewHead(nil, nil, nil, nil, headOpts, nil) +// if err != nil { +// return id, errors.Wrap(err, "create head block") +// } +// defer func() { +// runutil.CloseWithErrCapture(&err, h, "TSDB Head") +// if e := os.RemoveAll(headOpts.ChunkDirRoot); e != nil { +// err = errors.Wrap(e, "delete chunks dir") +// } +// }() + +// var g errgroup.Group +// timeStepSize := (maxt - mint) / int64(numSamples+1) +// batchSize := len(series) / runtime.GOMAXPROCS(0) + +// for len(series) > 0 { +// l := batchSize +// if len(series) < 1000 { +// l = len(series) +// } +// batch := series[:l] +// series = series[l:] + +// g.Go(func() error { +// t := mint + +// for i := 0; i < numSamples; i++ { +// app := h.Appender(ctx) + +// for _, lset := range batch { +// _, err := app.Append(0, lset, t, rand.Float64()) +// if err != nil { +// if rerr := app.Rollback(); rerr != nil { +// err = errors.Wrapf(err, "rollback failed: %v", rerr) +// } + +// return errors.Wrap(err, "add sample") +// } +// } +// if err := app.Commit(); err != nil { +// return errors.Wrap(err, "commit") +// } +// t += timeStepSize +// } +// return nil +// }) +// } +// if err := g.Wait(); err != nil { +// return id, err +// } +// c, err := tsdb.NewLeveledCompactor(ctx, nil, log.NewNopLogger(), []int64{maxt - mint}, nil, nil, true) +// if err != nil { +// return id, errors.Wrap(err, "create compactor") +// } + +// id, err = c.Write(dir, h, mint, maxt, nil) +// if err != nil { +// return id, errors.Wrap(err, "write block") +// } + +// if id.Compare(ulid.ULID{}) == 0 { +// return id, errors.Errorf("nothing to write, asked for %d samples", numSamples) +// } + +// blockDir := filepath.Join(dir, id.String()) + +// if _, err = block.InjectThanosMeta(log.NewNopLogger(), blockDir, block.ThanosMeta{ +// Labels: extLset.Map(), +// Downsample: block.ThanosDownsample{Resolution: resolution}, +// Source: block.TestSource, +// Files: []block.File{}, +// }, nil); err != nil { +// return id, errors.Wrap(err, "finalize block") +// } + +// if !tombstones { +// if err = os.Remove(filepath.Join(dir, id.String(), "tombstones")); err != nil { +// return id, errors.Wrap(err, "remove tombstones") +// } +// } + +// return id, nil +// } + +// var indexFilename = "index" + +// type indexWriterSeries struct { +// labels labels.Labels +// chunks []chunks.Meta // series file offset of chunks +// } + +// type indexWriterSeriesSlice []*indexWriterSeries + +// // putOutOfOrderIndex updates the index in blockDir with an index containing an out-of-order chunk +// // copied from https://github.com/prometheus/prometheus/blob/b1ed4a0a663d0c62526312311c7529471abbc565/tsdb/index/index_test.go#L346 +// func putOutOfOrderIndex(blockDir string, minTime int64, maxTime int64) error { +// if minTime >= maxTime || minTime+4 >= maxTime { +// return fmt.Errorf("minTime must be at least 4 less than maxTime to not create overlapping chunks") +// } + +// lbls := []labels.Labels{ +// labels.FromStrings("lbl1", "1"), +// } + +// // Sort labels as the index writer expects series in sorted order. +// sort.Sort(labels.Slice(lbls)) + +// symbols := map[string]struct{}{} +// for _, lset := range lbls { +// lset.Range(func(l labels.Label) { +// symbols[l.Name] = struct{}{} +// symbols[l.Value] = struct{}{} +// }) +// } + +// var input indexWriterSeriesSlice + +// // Generate ChunkMetas for every label set. +// for _, lset := range lbls { +// var metas []chunks.Meta +// // only need two chunks that are out-of-order +// chk1 := chunks.Meta{ +// MinTime: maxTime - 2, +// MaxTime: maxTime - 1, +// Ref: chunks.ChunkRef(rand.Uint64()), +// Chunk: chunkenc.NewXORChunk(), +// } +// metas = append(metas, chk1) +// chk2 := chunks.Meta{ +// MinTime: minTime + 1, +// MaxTime: minTime + 2, +// Ref: chunks.ChunkRef(rand.Uint64()), +// Chunk: chunkenc.NewXORChunk(), +// } +// metas = append(metas, chk2) + +// input = append(input, &indexWriterSeries{ +// labels: lset, +// chunks: metas, +// }) +// } + +// iw, err := index.NewWriter(context.Background(), filepath.Join(blockDir, indexFilename)) +// if err != nil { +// return err +// } + +// syms := []string{} +// for s := range symbols { +// syms = append(syms, s) +// } +// slices.Sort(syms) +// for _, s := range syms { +// if err := iw.AddSymbol(s); err != nil { +// return err +// } +// } + +// // Population procedure as done by compaction. +// var ( +// postings = index.NewMemPostings() +// values = map[string]map[string]struct{}{} +// ) + +// for i, s := range input { +// if err := iw.AddSeries(storage.SeriesRef(i), s.labels, s.chunks...); err != nil { +// return err +// } + +// s.labels.Range(func(l labels.Label) { +// valset, ok := values[l.Name] +// if !ok { +// valset = map[string]struct{}{} +// values[l.Name] = valset +// } +// valset[l.Value] = struct{}{} +// }) +// postings.Add(storage.SeriesRef(i), s.labels) +// } + +// return iw.Close() +// } diff --git a/pkg/compactor/bucket_compactor_test.go b/pkg/compactor/bucket_compactor_test.go index 8e8f5a7ea9..5a3d02810c 100644 --- a/pkg/compactor/bucket_compactor_test.go +++ b/pkg/compactor/bucket_compactor_test.go @@ -8,6 +8,7 @@ package compactor import ( "context" "fmt" + "path/filepath" "strings" "testing" "time" @@ -18,42 +19,43 @@ import ( "github.com/prometheus/client_golang/prometheus/promauto" "github.com/prometheus/client_golang/prometheus/testutil" "github.com/prometheus/prometheus/model/labels" - "github.com/prometheus/prometheus/tsdb" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "github.com/thanos-io/objstore" - "github.com/grafana/mimir/pkg/storage/tsdb/block" - "github.com/grafana/mimir/pkg/util/extprom" + phlareobj "github.com/grafana/pyroscope/pkg/objstore" + objstore_testutil "github.com/grafana/pyroscope/pkg/objstore/testutil" + "github.com/grafana/pyroscope/pkg/phlaredb/block" + "github.com/grafana/pyroscope/pkg/util/extprom" ) func TestGroupKey(t *testing.T) { for _, tcase := range []struct { - input block.ThanosMeta + input block.Meta expected string }{ { - input: block.ThanosMeta{}, + input: block.Meta{}, expected: "0@17241709254077376921", }, { - input: block.ThanosMeta{ + input: block.Meta{ Labels: map[string]string{}, - Downsample: block.ThanosDownsample{Resolution: 0}, + Downsample: block.Downsample{Resolution: 0}, }, expected: "0@17241709254077376921", }, { - input: block.ThanosMeta{ + input: block.Meta{ Labels: map[string]string{"foo": "bar", "foo1": "bar2"}, - Downsample: block.ThanosDownsample{Resolution: 0}, + Downsample: block.Downsample{Resolution: 0}, }, expected: "0@2124638872457683483", }, { - input: block.ThanosMeta{ + input: block.Meta{ Labels: map[string]string{`foo/some..thing/some.thing/../`: `a_b_c/bar-something-a\metric/a\x`}, - Downsample: block.ThanosDownsample{Resolution: 0}, + Downsample: block.Downsample{Resolution: 0}, }, expected: "0@16590761456214576373", }, @@ -69,9 +71,9 @@ func TestGroupKey(t *testing.T) { func TestGroupMaxMinTime(t *testing.T) { g := &Job{ metasByMinTime: []*block.Meta{ - {BlockMeta: tsdb.BlockMeta{MinTime: 0, MaxTime: 10}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 1, MaxTime: 20}}, - {BlockMeta: tsdb.BlockMeta{MinTime: 2, MaxTime: 30}}, + {MinTime: 0, MaxTime: 10}, + {MinTime: 1, MaxTime: 20}, + {MinTime: 2, MaxTime: 30}, }, } @@ -120,7 +122,7 @@ func TestBucketCompactor_FilterOwnJobs(t *testing.T) { m := NewBucketCompactorMetrics(promauto.With(nil).NewCounter(prometheus.CounterOpts{}), nil) for testName, testCase := range tests { t.Run(testName, func(t *testing.T) { - bc, err := NewBucketCompactor(log.NewNopLogger(), nil, nil, nil, nil, "", nil, 2, false, testCase.ownJob, nil, 0, 4, m) + bc, err := NewBucketCompactor(log.NewNopLogger(), nil, nil, nil, "", nil, 2, testCase.ownJob, nil, 0, 1, 4, m) require.NoError(t, err) res, err := bc.filterOwnJobs(jobsFn()) @@ -134,29 +136,23 @@ func TestBucketCompactor_FilterOwnJobs(t *testing.T) { func TestBlockMaxTimeDeltas(t *testing.T) { j1 := NewJob("user", "key1", labels.EmptyLabels(), 0, false, 0, "") require.NoError(t, j1.AppendMeta(&block.Meta{ - BlockMeta: tsdb.BlockMeta{ - MinTime: 1500002700159, - MaxTime: 1500002800159, - }, + MinTime: 1500002700159, + MaxTime: 1500002800159, })) j2 := NewJob("user", "key2", labels.EmptyLabels(), 0, false, 0, "") require.NoError(t, j2.AppendMeta(&block.Meta{ - BlockMeta: tsdb.BlockMeta{ - MinTime: 1500002600159, - MaxTime: 1500002700159, - }, + MinTime: 1500002600159, + MaxTime: 1500002700159, })) require.NoError(t, j2.AppendMeta(&block.Meta{ - BlockMeta: tsdb.BlockMeta{ - MinTime: 1500002700159, - MaxTime: 1500002800159, - }, + MinTime: 1500002700159, + MaxTime: 1500002800159, })) metrics := NewBucketCompactorMetrics(promauto.With(nil).NewCounter(prometheus.CounterOpts{}), nil) now := time.UnixMilli(1500002900159) - bc, err := NewBucketCompactor(log.NewNopLogger(), nil, nil, nil, nil, "", nil, 2, false, nil, nil, 0, 4, metrics) + bc, err := NewBucketCompactor(log.NewNopLogger(), nil, nil, nil, "", nil, 2, nil, nil, 0, 0, 4, metrics) require.NoError(t, err) deltas := bc.blockMaxTimeDeltas(now, []*Job{j1, j2}) @@ -165,8 +161,9 @@ func TestBlockMaxTimeDeltas(t *testing.T) { func TestNoCompactionMarkFilter(t *testing.T) { ctx := context.Background() + // Use bucket with global markers to make sure that our custom filters work correctly. - bkt := block.BucketWithGlobalMarkers(objstore.NewInMemBucket()) + bkt := block.BucketWithGlobalMarkers(phlareobj.NewBucket(objstore.NewInMemBucket())) block1 := ulid.MustParse("01DTVP434PA9VFXSW2JK000001") // No mark file. block2 := ulid.MustParse("01DTVP434PA9VFXSW2JK000002") // Marked for no-compaction @@ -183,7 +180,7 @@ func TestNoCompactionMarkFilter(t *testing.T) { block5: blockMeta(block5.String(), 500, 600, nil), } - f := NewNoCompactionMarkFilter(objstore.WithNoopInstr(bkt), false) + f := NewNoCompactionMarkFilter(phlareobj.NewBucket(objstore.WithNoopInstr(bkt)), false) require.NoError(t, f.Filter(ctx, metas, synced)) require.Contains(t, metas, block1) @@ -204,7 +201,7 @@ func TestNoCompactionMarkFilter(t *testing.T) { block5: blockMeta(block5.String(), 500, 600, nil), } - f := NewNoCompactionMarkFilter(objstore.WithNoopInstr(bkt), true) + f := NewNoCompactionMarkFilter(phlareobj.NewBucket(objstore.WithNoopInstr(bkt)), true) require.NoError(t, f.Filter(ctx, metas, synced)) require.Contains(t, metas, block1) @@ -230,7 +227,7 @@ func TestNoCompactionMarkFilter(t *testing.T) { canceledCtx, cancel := context.WithCancel(context.Background()) cancel() - f := NewNoCompactionMarkFilter(objstore.WithNoopInstr(bkt), true) + f := NewNoCompactionMarkFilter(phlareobj.NewBucket(objstore.WithNoopInstr(bkt)), true) require.Error(t, f.Filter(canceledCtx, metas, synced)) require.Contains(t, metas, block1) @@ -247,7 +244,7 @@ func TestNoCompactionMarkFilter(t *testing.T) { block3: blockMeta(block3.String(), 300, 300, nil), // Has compaction marker with invalid version, but Filter doesn't check for that. } - f := NewNoCompactionMarkFilter(objstore.WithNoopInstr(bkt), true) + f := NewNoCompactionMarkFilter(phlareobj.NewBucket(objstore.WithNoopInstr(bkt)), true) err := f.Filter(ctx, metas, synced) require.NoError(t, err) require.Empty(t, metas) @@ -313,28 +310,13 @@ func TestCompactedBlocksTimeRangeVerification(t *testing.T) { testData := testData // Prevent loop variable being captured by func literal t.Run(testName, func(t *testing.T) { t.Parallel() + bucketClient, _ := objstore_testutil.NewFilesystemBucket(t, context.Background(), t.TempDir()) tempDir := t.TempDir() + compactedBlock1 := createDBBlock(t, bucketClient, "foo", testData.compactedBlockMinTime, testData.compactedBlockMinTime+500, 10, nil) + compactedBlock2 := createDBBlock(t, bucketClient, "foo", testData.compactedBlockMaxTime-500, testData.compactedBlockMaxTime, 10, nil) - compactedBlock1, err := block.CreateBlock( - context.Background(), tempDir, - []labels.Labels{ - labels.FromStrings("test", "foo", "a", "1"), - labels.FromStrings("test", "foo", "a", "2"), - labels.FromStrings("test", "foo", "a", "3"), - }, 10, testData.compactedBlockMinTime, testData.compactedBlockMinTime+500, labels.EmptyLabels()) - require.NoError(t, err) - - compactedBlock2, err := block.CreateBlock( - context.Background(), tempDir, - []labels.Labels{ - labels.FromStrings("test", "foo", "a", "1"), - labels.FromStrings("test", "foo", "a", "2"), - labels.FromStrings("test", "foo", "a", "3"), - }, 10, testData.compactedBlockMaxTime-500, testData.compactedBlockMaxTime, labels.EmptyLabels()) - require.NoError(t, err) - - err = verifyCompactedBlocksTimeRanges([]ulid.ULID{compactedBlock1, compactedBlock2}, sourceMinTime, sourceMaxTime, tempDir) + err := verifyCompactedBlocksTimeRanges([]ulid.ULID{compactedBlock1, compactedBlock2}, sourceMinTime, sourceMaxTime,filepath.Join(tempDir,"foo")) if testData.shouldErr { require.ErrorContains(t, err, testData.expectedErrMsg) } else { diff --git a/pkg/compactor/compactor_test.go b/pkg/compactor/compactor_test.go index 4e78182af3..0a00721090 100644 --- a/pkg/compactor/compactor_test.go +++ b/pkg/compactor/compactor_test.go @@ -1596,7 +1596,7 @@ func createCustomBlock(t *testing.T, bkt objstore.Bucket, userID string, externa return meta.ULID } -func createTSDBBlock(t *testing.T, bkt objstore.Bucket, userID string, minT, maxT int64, numSeries int, externalLabels map[string]string) ulid.ULID { +func createDBBlock(t *testing.T, bkt objstore.Bucket, userID string, minT, maxT int64, numSeries int, externalLabels map[string]string) ulid.ULID { return createCustomBlock(t, bkt, userID, externalLabels, func() []*testhelper.ProfileBuilder { result := []*testhelper.ProfileBuilder{} appendSample := func(seriesID int, ts int64, value float64) { diff --git a/pkg/compactor/split_merge_compactor_test.go b/pkg/compactor/split_merge_compactor_test.go index fa08ca0ef2..1b3025d09a 100644 --- a/pkg/compactor/split_merge_compactor_test.go +++ b/pkg/compactor/split_merge_compactor_test.go @@ -67,8 +67,8 @@ func TestMultitenantCompactor_ShouldSupportSplitAndMergeCompactor(t *testing.T) "overlapping blocks matching the 1st compaction range should be merged and split": { numShards: 2, setup: func(t *testing.T, bkt objstore.Bucket) []block.Meta { - block1 := createTSDBBlock(t, bkt, userID, blockRangeMillis, 2*blockRangeMillis, numSeries, externalLabels("")) - block2 := createTSDBBlock(t, bkt, userID, blockRangeMillis, 2*blockRangeMillis, numSeries, externalLabels("")) + block1 := createDBBlock(t, bkt, userID, blockRangeMillis, 2*blockRangeMillis, numSeries, externalLabels("")) + block2 := createDBBlock(t, bkt, userID, blockRangeMillis, 2*blockRangeMillis, numSeries, externalLabels("")) return []block.Meta{ { @@ -98,8 +98,8 @@ func TestMultitenantCompactor_ShouldSupportSplitAndMergeCompactor(t *testing.T) "overlapping blocks matching the 1st compaction range with mixed tenant ID labels should be merged and split": { numShards: 2, setup: func(t *testing.T, bkt objstore.Bucket) []block.Meta { - block1 := createTSDBBlock(t, bkt, userID, blockRangeMillis, 2*blockRangeMillis, numSeries, externalLabels("")) // Doesn't have __org_id__ label - block2 := createTSDBBlock(t, bkt, userID, blockRangeMillis, 2*blockRangeMillis, numSeries, externalLabelsWithTenantID("")) // Has __org_id__ label + block1 := createDBBlock(t, bkt, userID, blockRangeMillis, 2*blockRangeMillis, numSeries, externalLabels("")) // Doesn't have __org_id__ label + block2 := createDBBlock(t, bkt, userID, blockRangeMillis, 2*blockRangeMillis, numSeries, externalLabelsWithTenantID("")) // Has __org_id__ label return []block.Meta{ { @@ -129,12 +129,12 @@ func TestMultitenantCompactor_ShouldSupportSplitAndMergeCompactor(t *testing.T) "overlapping blocks matching the beginning of the 1st compaction range should be merged and split": { numShards: 2, setup: func(t *testing.T, bkt objstore.Bucket) []block.Meta { - block1 := createTSDBBlock(t, bkt, userID, 0, (5 * time.Minute).Milliseconds(), numSeries, externalLabels("")) - block2 := createTSDBBlock(t, bkt, userID, time.Minute.Milliseconds(), (7 * time.Minute).Milliseconds(), numSeries, externalLabels("")) + block1 := createDBBlock(t, bkt, userID, 0, (5 * time.Minute).Milliseconds(), numSeries, externalLabels("")) + block2 := createDBBlock(t, bkt, userID, time.Minute.Milliseconds(), (7 * time.Minute).Milliseconds(), numSeries, externalLabels("")) // Add another block as "most recent one" otherwise the previous blocks are not compacted // because the most recent blocks must cover the full range to be compacted. - block3 := createTSDBBlock(t, bkt, userID, blockRangeMillis, blockRangeMillis+time.Minute.Milliseconds(), numSeries, externalLabels("")) + block3 := createDBBlock(t, bkt, userID, blockRangeMillis, blockRangeMillis+time.Minute.Milliseconds(), numSeries, externalLabels("")) return []block.Meta{ { @@ -173,12 +173,12 @@ func TestMultitenantCompactor_ShouldSupportSplitAndMergeCompactor(t *testing.T) "non-overlapping blocks matching the beginning of the 1st compaction range (without gaps) should be merged and split": { numShards: 2, setup: func(t *testing.T, bkt objstore.Bucket) []block.Meta { - block1 := createTSDBBlock(t, bkt, userID, 0, (5 * time.Minute).Milliseconds(), numSeries, externalLabels("")) - block2 := createTSDBBlock(t, bkt, userID, (5 * time.Minute).Milliseconds(), (10 * time.Minute).Milliseconds(), numSeries, externalLabels("")) + block1 := createDBBlock(t, bkt, userID, 0, (5 * time.Minute).Milliseconds(), numSeries, externalLabels("")) + block2 := createDBBlock(t, bkt, userID, (5 * time.Minute).Milliseconds(), (10 * time.Minute).Milliseconds(), numSeries, externalLabels("")) // Add another block as "most recent one" otherwise the previous blocks are not compacted // because the most recent blocks must cover the full range to be compacted. - block3 := createTSDBBlock(t, bkt, userID, blockRangeMillis, blockRangeMillis+time.Minute.Milliseconds(), numSeries, externalLabels("")) + block3 := createDBBlock(t, bkt, userID, blockRangeMillis, blockRangeMillis+time.Minute.Milliseconds(), numSeries, externalLabels("")) return []block.Meta{ { @@ -217,12 +217,12 @@ func TestMultitenantCompactor_ShouldSupportSplitAndMergeCompactor(t *testing.T) "non-overlapping blocks matching the beginning of the 1st compaction range (with gaps) should be merged and split": { numShards: 2, setup: func(t *testing.T, bkt objstore.Bucket) []block.Meta { - block1 := createTSDBBlock(t, bkt, userID, 0, (5 * time.Minute).Milliseconds(), numSeries, externalLabels("")) - block2 := createTSDBBlock(t, bkt, userID, (7 * time.Minute).Milliseconds(), (10 * time.Minute).Milliseconds(), numSeries, externalLabels("")) + block1 := createDBBlock(t, bkt, userID, 0, (5 * time.Minute).Milliseconds(), numSeries, externalLabels("")) + block2 := createDBBlock(t, bkt, userID, (7 * time.Minute).Milliseconds(), (10 * time.Minute).Milliseconds(), numSeries, externalLabels("")) // Add another block as "most recent one" otherwise the previous blocks are not compacted // because the most recent blocks must cover the full range to be compacted. - block3 := createTSDBBlock(t, bkt, userID, blockRangeMillis, blockRangeMillis+time.Minute.Milliseconds(), numSeries, externalLabels("")) + block3 := createDBBlock(t, bkt, userID, blockRangeMillis, blockRangeMillis+time.Minute.Milliseconds(), numSeries, externalLabels("")) return []block.Meta{ { @@ -262,22 +262,22 @@ func TestMultitenantCompactor_ShouldSupportSplitAndMergeCompactor(t *testing.T) numShards: 2, setup: func(t *testing.T, bkt objstore.Bucket) []block.Meta { // Two split blocks in the 1st compaction range. - block1a := createTSDBBlock(t, bkt, userID, 1, blockRangeMillis, numSeries, externalLabels("1_of_2")) - block1b := createTSDBBlock(t, bkt, userID, 1, blockRangeMillis, numSeries, externalLabels("2_of_2")) + block1a := createDBBlock(t, bkt, userID, 1, blockRangeMillis, numSeries, externalLabels("1_of_2")) + block1b := createDBBlock(t, bkt, userID, 1, blockRangeMillis, numSeries, externalLabels("2_of_2")) // Two non-split overlapping blocks in the 1st compaction range. - block2 := createTSDBBlock(t, bkt, userID, blockRangeMillis, 2*blockRangeMillis, numSeries, externalLabels("")) - block3 := createTSDBBlock(t, bkt, userID, blockRangeMillis, 2*blockRangeMillis, numSeries, externalLabels("")) + block2 := createDBBlock(t, bkt, userID, blockRangeMillis, 2*blockRangeMillis, numSeries, externalLabels("")) + block3 := createDBBlock(t, bkt, userID, blockRangeMillis, 2*blockRangeMillis, numSeries, externalLabels("")) // Two split adjacent blocks in the 2nd compaction range. - block4a := createTSDBBlock(t, bkt, userID, 2*blockRangeMillis, 3*blockRangeMillis, numSeries, externalLabels("1_of_2")) - block4b := createTSDBBlock(t, bkt, userID, 2*blockRangeMillis, 3*blockRangeMillis, numSeries, externalLabels("2_of_2")) - block5a := createTSDBBlock(t, bkt, userID, 3*blockRangeMillis, 4*blockRangeMillis, numSeries, externalLabels("1_of_2")) - block5b := createTSDBBlock(t, bkt, userID, 3*blockRangeMillis, 4*blockRangeMillis, numSeries, externalLabels("2_of_2")) + block4a := createDBBlock(t, bkt, userID, 2*blockRangeMillis, 3*blockRangeMillis, numSeries, externalLabels("1_of_2")) + block4b := createDBBlock(t, bkt, userID, 2*blockRangeMillis, 3*blockRangeMillis, numSeries, externalLabels("2_of_2")) + block5a := createDBBlock(t, bkt, userID, 3*blockRangeMillis, 4*blockRangeMillis, numSeries, externalLabels("1_of_2")) + block5b := createDBBlock(t, bkt, userID, 3*blockRangeMillis, 4*blockRangeMillis, numSeries, externalLabels("2_of_2")) // Two non-adjacent non-split blocks in the 1st compaction range. - block6 := createTSDBBlock(t, bkt, userID, 4*blockRangeMillis, 5*blockRangeMillis, numSeries, externalLabels("")) - block7 := createTSDBBlock(t, bkt, userID, 7*blockRangeMillis, 8*blockRangeMillis, numSeries, externalLabels("")) + block6 := createDBBlock(t, bkt, userID, 4*blockRangeMillis, 5*blockRangeMillis, numSeries, externalLabels("")) + block7 := createDBBlock(t, bkt, userID, 7*blockRangeMillis, 8*blockRangeMillis, numSeries, externalLabels("")) return []block.Meta{ // The two overlapping blocks (block2, block3) have been merged and split in the 1st range, @@ -336,11 +336,11 @@ func TestMultitenantCompactor_ShouldSupportSplitAndMergeCompactor(t *testing.T) numShards: 2, setup: func(t *testing.T, bkt objstore.Bucket) []block.Meta { // Overlapping. - block1 := createTSDBBlock(t, bkt, userID, 0, (5 * time.Minute).Milliseconds(), numSeries, externalLabels("")) - block2 := createTSDBBlock(t, bkt, userID, time.Minute.Milliseconds(), (7 * time.Minute).Milliseconds(), numSeries, externalLabels("")) + block1 := createDBBlock(t, bkt, userID, 0, (5 * time.Minute).Milliseconds(), numSeries, externalLabels("")) + block2 := createDBBlock(t, bkt, userID, time.Minute.Milliseconds(), (7 * time.Minute).Milliseconds(), numSeries, externalLabels("")) // Not overlapping. - block3 := createTSDBBlock(t, bkt, userID, time.Hour.Milliseconds(), (2 * time.Hour).Milliseconds(), numSeries, externalLabels("")) + block3 := createDBBlock(t, bkt, userID, time.Hour.Milliseconds(), (2 * time.Hour).Milliseconds(), numSeries, externalLabels("")) return []block.Meta{ { @@ -371,11 +371,11 @@ func TestMultitenantCompactor_ShouldSupportSplitAndMergeCompactor(t *testing.T) numShards: 2, setup: func(t *testing.T, bkt objstore.Bucket) []block.Meta { // Overlapping. - block1 := createTSDBBlock(t, bkt, userID, 0, (5 * time.Minute).Milliseconds(), numSeries, externalLabels("")) // Without __org_id__ label - block2 := createTSDBBlock(t, bkt, userID, time.Minute.Milliseconds(), (7 * time.Minute).Milliseconds(), numSeries, externalLabelsWithTenantID("")) // With __org_id__ label + block1 := createDBBlock(t, bkt, userID, 0, (5 * time.Minute).Milliseconds(), numSeries, externalLabels("")) // Without __org_id__ label + block2 := createDBBlock(t, bkt, userID, time.Minute.Milliseconds(), (7 * time.Minute).Milliseconds(), numSeries, externalLabelsWithTenantID("")) // With __org_id__ label // Not overlapping. - block3 := createTSDBBlock(t, bkt, userID, time.Hour.Milliseconds(), (2 * time.Hour).Milliseconds(), numSeries, externalLabelsWithTenantID("")) // With __org_id__ label + block3 := createDBBlock(t, bkt, userID, time.Hour.Milliseconds(), (2 * time.Hour).Milliseconds(), numSeries, externalLabelsWithTenantID("")) // With __org_id__ label return []block.Meta{ { @@ -407,7 +407,7 @@ func TestMultitenantCompactor_ShouldSupportSplitAndMergeCompactor(t *testing.T) setup: func(t *testing.T, bkt objstore.Bucket) []block.Meta { // Generate a block with only 1 series. This block will be split into 1 split block only, // because the source block only has 1 series. - block1 := createTSDBBlock(t, bkt, userID, blockRangeMillis, 2*blockRangeMillis, 1, externalLabels("")) + block1 := createDBBlock(t, bkt, userID, blockRangeMillis, 2*blockRangeMillis, 1, externalLabels("")) return []block.Meta{ { @@ -427,12 +427,12 @@ func TestMultitenantCompactor_ShouldSupportSplitAndMergeCompactor(t *testing.T) "splitting should be disabled if configured shards = 0": { numShards: 0, setup: func(t *testing.T, bkt objstore.Bucket) []block.Meta { - block1 := createTSDBBlock(t, bkt, userID, 0, (5 * time.Minute).Milliseconds(), numSeries, externalLabels("")) - block2 := createTSDBBlock(t, bkt, userID, (5 * time.Minute).Milliseconds(), (10 * time.Minute).Milliseconds(), numSeries, externalLabels("")) + block1 := createDBBlock(t, bkt, userID, 0, (5 * time.Minute).Milliseconds(), numSeries, externalLabels("")) + block2 := createDBBlock(t, bkt, userID, (5 * time.Minute).Milliseconds(), (10 * time.Minute).Milliseconds(), numSeries, externalLabels("")) // Add another block as "most recent one" otherwise the previous blocks are not compacted // because the most recent blocks must cover the full range to be compacted. - block3 := createTSDBBlock(t, bkt, userID, blockRangeMillis, blockRangeMillis+time.Minute.Milliseconds(), numSeries, externalLabels("")) + block3 := createDBBlock(t, bkt, userID, blockRangeMillis, blockRangeMillis+time.Minute.Milliseconds(), numSeries, externalLabels("")) return []block.Meta{ // Compacted but not split. @@ -461,22 +461,22 @@ func TestMultitenantCompactor_ShouldSupportSplitAndMergeCompactor(t *testing.T) numShards: 0, setup: func(t *testing.T, bkt objstore.Bucket) []block.Meta { // Two split blocks in the 1st compaction range. - block1a := createTSDBBlock(t, bkt, userID, 1, blockRangeMillis, numSeries, externalLabels("1_of_2")) - block1b := createTSDBBlock(t, bkt, userID, 1, blockRangeMillis, numSeries, externalLabels("2_of_2")) + block1a := createDBBlock(t, bkt, userID, 1, blockRangeMillis, numSeries, externalLabels("1_of_2")) + block1b := createDBBlock(t, bkt, userID, 1, blockRangeMillis, numSeries, externalLabels("2_of_2")) // Two non-split overlapping blocks in the 1st compaction range. - block2 := createTSDBBlock(t, bkt, userID, blockRangeMillis, 2*blockRangeMillis, numSeries, externalLabels("")) - block3 := createTSDBBlock(t, bkt, userID, blockRangeMillis, 2*blockRangeMillis, numSeries, externalLabels("")) + block2 := createDBBlock(t, bkt, userID, blockRangeMillis, 2*blockRangeMillis, numSeries, externalLabels("")) + block3 := createDBBlock(t, bkt, userID, blockRangeMillis, 2*blockRangeMillis, numSeries, externalLabels("")) // Two split adjacent blocks in the 2nd compaction range. - block4a := createTSDBBlock(t, bkt, userID, 2*blockRangeMillis, 3*blockRangeMillis, numSeries, externalLabels("1_of_2")) - block4b := createTSDBBlock(t, bkt, userID, 2*blockRangeMillis, 3*blockRangeMillis, numSeries, externalLabels("2_of_2")) - block5a := createTSDBBlock(t, bkt, userID, 3*blockRangeMillis, 4*blockRangeMillis, numSeries, externalLabels("1_of_2")) - block5b := createTSDBBlock(t, bkt, userID, 3*blockRangeMillis, 4*blockRangeMillis, numSeries, externalLabels("2_of_2")) + block4a := createDBBlock(t, bkt, userID, 2*blockRangeMillis, 3*blockRangeMillis, numSeries, externalLabels("1_of_2")) + block4b := createDBBlock(t, bkt, userID, 2*blockRangeMillis, 3*blockRangeMillis, numSeries, externalLabels("2_of_2")) + block5a := createDBBlock(t, bkt, userID, 3*blockRangeMillis, 4*blockRangeMillis, numSeries, externalLabels("1_of_2")) + block5b := createDBBlock(t, bkt, userID, 3*blockRangeMillis, 4*blockRangeMillis, numSeries, externalLabels("2_of_2")) // Two non-adjacent non-split blocks in the 1st compaction range. - block6 := createTSDBBlock(t, bkt, userID, 4*blockRangeMillis, 5*blockRangeMillis, numSeries, externalLabels("")) - block7 := createTSDBBlock(t, bkt, userID, 7*blockRangeMillis, 8*blockRangeMillis, numSeries, externalLabels("")) + block6 := createDBBlock(t, bkt, userID, 4*blockRangeMillis, 5*blockRangeMillis, numSeries, externalLabels("")) + block7 := createDBBlock(t, bkt, userID, 7*blockRangeMillis, 8*blockRangeMillis, numSeries, externalLabels("")) return []block.Meta{ // Block1 have been compacted with block4 and block5 in the 3rd range compaction. @@ -653,7 +653,7 @@ func TestMultitenantCompactor_ShouldGuaranteeSeriesShardingConsistencyOverTheTim require.NoError(t, err) // Create a TSDB block in the storage. - blockID := createTSDBBlock(t, bucketClient, userID, blockRangeMillis, 2*blockRangeMillis, numSeries, nil) + blockID := createDBBlock(t, bucketClient, userID, blockRangeMillis, 2*blockRangeMillis, numSeries, nil) c, err := NewMultitenantCompactor(compactorCfg, bucketClient, cfgProvider, logger, reg) require.NoError(t, err) From 04b2a614647965615a20acaa405708b752d76046 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Thu, 14 Sep 2023 14:45:28 +0200 Subject: [PATCH 35/74] Fixes more tests --- pkg/compactor/compactor_test.go | 1066 +++++++++++++++---------------- 1 file changed, 533 insertions(+), 533 deletions(-) diff --git a/pkg/compactor/compactor_test.go b/pkg/compactor/compactor_test.go index 0a00721090..d41d093996 100644 --- a/pkg/compactor/compactor_test.go +++ b/pkg/compactor/compactor_test.go @@ -27,13 +27,11 @@ import ( "github.com/grafana/dskit/ring" "github.com/grafana/dskit/services" "github.com/grafana/dskit/test" - "github.com/grafana/loki/pkg/storage/bucket/filesystem" - "github.com/grafana/mimir/pkg/storage/bucket" - "github.com/grafana/mimir/pkg/storage/tsdb/block" "github.com/oklog/ulid" "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" prom_testutil "github.com/prometheus/client_golang/prometheus/testutil" + "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/histogram" "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/tsdb" @@ -46,7 +44,12 @@ import ( "github.com/thanos-io/objstore" "gopkg.in/yaml.v3" + pyroscope_objstore "github.com/grafana/pyroscope/pkg/objstore" + "github.com/grafana/pyroscope/pkg/objstore/providers/filesystem" + "github.com/grafana/pyroscope/pkg/phlaredb/block" "github.com/grafana/pyroscope/pkg/phlaredb/block/testutil" + "github.com/grafana/pyroscope/pkg/phlaredb/bucket" + "github.com/grafana/pyroscope/pkg/pprof/testhelper" "github.com/grafana/pyroscope/pkg/validation" ) @@ -100,13 +103,13 @@ func TestConfig_Validate(t *testing.T) { }, "should pass with only 1 block range period": { setup: func(cfg *Config) { - cfg.BlockRanges = mimir_tsdb.DurationList{time.Hour} + cfg.BlockRanges = DurationList{time.Hour} }, expected: "", }, "should fail with non divisible block range periods": { setup: func(cfg *Config) { - cfg.BlockRanges = mimir_tsdb.DurationList{2 * time.Hour, 12 * time.Hour, 24 * time.Hour, 30 * time.Hour} + cfg.BlockRanges = DurationList{2 * time.Hour, 12 * time.Hour, 24 * time.Hour, 30 * time.Hour} }, expected: errors.Errorf(errInvalidBlockRanges, 30*time.Hour, 24*time.Hour).Error(), }, @@ -175,108 +178,108 @@ func TestMultitenantCompactor_ShouldDoNothingOnNoUserBlocks(t *testing.T) { }, removeIgnoredLogs(strings.Split(strings.TrimSpace(logs.String()), "\n"))) assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` - # TYPE cortex_compactor_runs_started_total counter - # HELP cortex_compactor_runs_started_total Total number of compaction runs started. - cortex_compactor_runs_started_total 1 - - # TYPE cortex_compactor_runs_completed_total counter - # HELP cortex_compactor_runs_completed_total Total number of compaction runs successfully completed. - cortex_compactor_runs_completed_total 1 - - # TYPE cortex_compactor_runs_failed_total counter - # HELP cortex_compactor_runs_failed_total Total number of compaction runs failed. - cortex_compactor_runs_failed_total{reason="error"} 0 - cortex_compactor_runs_failed_total{reason="shutdown"} 0 - - # HELP cortex_compactor_garbage_collection_duration_seconds Time it took to perform garbage collection iteration. - # TYPE cortex_compactor_garbage_collection_duration_seconds histogram - cortex_compactor_garbage_collection_duration_seconds_bucket{le="+Inf"} 0 - cortex_compactor_garbage_collection_duration_seconds_sum 0 - cortex_compactor_garbage_collection_duration_seconds_count 0 - - # HELP cortex_compactor_garbage_collection_failures_total Total number of failed garbage collection operations. - # TYPE cortex_compactor_garbage_collection_failures_total counter - cortex_compactor_garbage_collection_failures_total 0 - - # HELP cortex_compactor_garbage_collection_total Total number of garbage collection operations. - # TYPE cortex_compactor_garbage_collection_total counter - cortex_compactor_garbage_collection_total 0 - - # HELP cortex_compactor_meta_sync_duration_seconds Duration of the blocks metadata synchronization in seconds. - # TYPE cortex_compactor_meta_sync_duration_seconds histogram - cortex_compactor_meta_sync_duration_seconds_bucket{le="+Inf"} 0 - cortex_compactor_meta_sync_duration_seconds_sum 0 - cortex_compactor_meta_sync_duration_seconds_count 0 - - # HELP cortex_compactor_meta_sync_failures_total Total blocks metadata synchronization failures. - # TYPE cortex_compactor_meta_sync_failures_total counter - cortex_compactor_meta_sync_failures_total 0 - - # HELP cortex_compactor_meta_syncs_total Total blocks metadata synchronization attempts. - # TYPE cortex_compactor_meta_syncs_total counter - cortex_compactor_meta_syncs_total 0 - - # HELP cortex_compactor_group_compaction_runs_completed_total Total number of group completed compaction runs. This also includes compactor group runs that resulted with no compaction. - # TYPE cortex_compactor_group_compaction_runs_completed_total counter - cortex_compactor_group_compaction_runs_completed_total 0 - - # HELP cortex_compactor_group_compaction_runs_started_total Total number of group compaction attempts. - # TYPE cortex_compactor_group_compaction_runs_started_total counter - cortex_compactor_group_compaction_runs_started_total 0 - - # HELP cortex_compactor_group_compactions_failures_total Total number of failed group compactions. - # TYPE cortex_compactor_group_compactions_failures_total counter - cortex_compactor_group_compactions_failures_total 0 - - # HELP cortex_compactor_group_compactions_total Total number of group compaction attempts that resulted in new block(s). - # TYPE cortex_compactor_group_compactions_total counter - cortex_compactor_group_compactions_total 0 - - # TYPE cortex_compactor_block_cleanup_failures_total counter - # HELP cortex_compactor_block_cleanup_failures_total Total number of blocks failed to be deleted. - cortex_compactor_block_cleanup_failures_total 0 - - # HELP cortex_compactor_blocks_cleaned_total Total number of blocks deleted. - # TYPE cortex_compactor_blocks_cleaned_total counter - cortex_compactor_blocks_cleaned_total 0 - - # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. - # TYPE cortex_compactor_blocks_marked_for_deletion_total counter - cortex_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 - cortex_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 - cortex_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 - - # TYPE cortex_compactor_block_cleanup_started_total counter - # HELP cortex_compactor_block_cleanup_started_total Total number of blocks cleanup runs started. - cortex_compactor_block_cleanup_started_total 1 - - # TYPE cortex_compactor_block_cleanup_completed_total counter - # HELP cortex_compactor_block_cleanup_completed_total Total number of blocks cleanup runs successfully completed. - cortex_compactor_block_cleanup_completed_total 1 - - # TYPE cortex_compactor_block_cleanup_failed_total counter - # HELP cortex_compactor_block_cleanup_failed_total Total number of blocks cleanup runs failed. - cortex_compactor_block_cleanup_failed_total 0 + # TYPE pyroscope_compactor_runs_started_total counter + # HELP pyroscope_compactor_runs_started_total Total number of compaction runs started. + pyroscope_compactor_runs_started_total 1 + + # TYPE pyroscope_compactor_runs_completed_total counter + # HELP pyroscope_compactor_runs_completed_total Total number of compaction runs successfully completed. + pyroscope_compactor_runs_completed_total 1 + + # TYPE pyroscope_compactor_runs_failed_total counter + # HELP pyroscope_compactor_runs_failed_total Total number of compaction runs failed. + pyroscope_compactor_runs_failed_total{reason="error"} 0 + pyroscope_compactor_runs_failed_total{reason="shutdown"} 0 + + # HELP pyroscope_compactor_garbage_collection_duration_seconds Time it took to perform garbage collection iteration. + # TYPE pyroscope_compactor_garbage_collection_duration_seconds histogram + pyroscope_compactor_garbage_collection_duration_seconds_bucket{le="+Inf"} 0 + pyroscope_compactor_garbage_collection_duration_seconds_sum 0 + pyroscope_compactor_garbage_collection_duration_seconds_count 0 + + # HELP pyroscope_compactor_garbage_collection_failures_total Total number of failed garbage collection operations. + # TYPE pyroscope_compactor_garbage_collection_failures_total counter + pyroscope_compactor_garbage_collection_failures_total 0 + + # HELP pyroscope_compactor_garbage_collection_total Total number of garbage collection operations. + # TYPE pyroscope_compactor_garbage_collection_total counter + pyroscope_compactor_garbage_collection_total 0 + + # HELP pyroscope_compactor_meta_sync_duration_seconds Duration of the blocks metadata synchronization in seconds. + # TYPE pyroscope_compactor_meta_sync_duration_seconds histogram + pyroscope_compactor_meta_sync_duration_seconds_bucket{le="+Inf"} 0 + pyroscope_compactor_meta_sync_duration_seconds_sum 0 + pyroscope_compactor_meta_sync_duration_seconds_count 0 + + # HELP pyroscope_compactor_meta_sync_failures_total Total blocks metadata synchronization failures. + # TYPE pyroscope_compactor_meta_sync_failures_total counter + pyroscope_compactor_meta_sync_failures_total 0 + + # HELP pyroscope_compactor_meta_syncs_total Total blocks metadata synchronization attempts. + # TYPE pyroscope_compactor_meta_syncs_total counter + pyroscope_compactor_meta_syncs_total 0 + + # HELP pyroscope_compactor_group_compaction_runs_completed_total Total number of group completed compaction runs. This also includes compactor group runs that resulted with no compaction. + # TYPE pyroscope_compactor_group_compaction_runs_completed_total counter + pyroscope_compactor_group_compaction_runs_completed_total 0 + + # HELP pyroscope_compactor_group_compaction_runs_started_total Total number of group compaction attempts. + # TYPE pyroscope_compactor_group_compaction_runs_started_total counter + pyroscope_compactor_group_compaction_runs_started_total 0 + + # HELP pyroscope_compactor_group_compactions_failures_total Total number of failed group compactions. + # TYPE pyroscope_compactor_group_compactions_failures_total counter + pyroscope_compactor_group_compactions_failures_total 0 + + # HELP pyroscope_compactor_group_compactions_total Total number of group compaction attempts that resulted in new block(s). + # TYPE pyroscope_compactor_group_compactions_total counter + pyroscope_compactor_group_compactions_total 0 + + # TYPE pyroscope_compactor_block_cleanup_failures_total counter + # HELP pyroscope_compactor_block_cleanup_failures_total Total number of blocks failed to be deleted. + pyroscope_compactor_block_cleanup_failures_total 0 + + # HELP pyroscope_compactor_blocks_cleaned_total Total number of blocks deleted. + # TYPE pyroscope_compactor_blocks_cleaned_total counter + pyroscope_compactor_blocks_cleaned_total 0 + + # HELP pyroscope_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. + # TYPE pyroscope_compactor_blocks_marked_for_deletion_total counter + pyroscope_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 + pyroscope_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 + pyroscope_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 + + # TYPE pyroscope_compactor_block_cleanup_started_total counter + # HELP pyroscope_compactor_block_cleanup_started_total Total number of blocks cleanup runs started. + pyroscope_compactor_block_cleanup_started_total 1 + + # TYPE pyroscope_compactor_block_cleanup_completed_total counter + # HELP pyroscope_compactor_block_cleanup_completed_total Total number of blocks cleanup runs successfully completed. + pyroscope_compactor_block_cleanup_completed_total 1 + + # TYPE pyroscope_compactor_block_cleanup_failed_total counter + # HELP pyroscope_compactor_block_cleanup_failed_total Total number of blocks cleanup runs failed. + pyroscope_compactor_block_cleanup_failed_total 0 `), - "cortex_compactor_runs_started_total", - "cortex_compactor_runs_completed_total", - "cortex_compactor_runs_failed_total", - "cortex_compactor_garbage_collection_duration_seconds", - "cortex_compactor_garbage_collection_failures_total", - "cortex_compactor_garbage_collection_total", - "cortex_compactor_meta_sync_duration_seconds", - "cortex_compactor_meta_sync_failures_total", - "cortex_compactor_meta_syncs_total", - "cortex_compactor_group_compaction_runs_completed_total", - "cortex_compactor_group_compaction_runs_started_total", - "cortex_compactor_group_compactions_failures_total", - "cortex_compactor_group_compactions_total", - "cortex_compactor_block_cleanup_failures_total", - "cortex_compactor_blocks_cleaned_total", - "cortex_compactor_blocks_marked_for_deletion_total", - "cortex_compactor_block_cleanup_started_total", - "cortex_compactor_block_cleanup_completed_total", - "cortex_compactor_block_cleanup_failed_total", + "pyroscope_compactor_runs_started_total", + "pyroscope_compactor_runs_completed_total", + "pyroscope_compactor_runs_failed_total", + "pyroscope_compactor_garbage_collection_duration_seconds", + "pyroscope_compactor_garbage_collection_failures_total", + "pyroscope_compactor_garbage_collection_total", + "pyroscope_compactor_meta_sync_duration_seconds", + "pyroscope_compactor_meta_sync_failures_total", + "pyroscope_compactor_meta_syncs_total", + "pyroscope_compactor_group_compaction_runs_completed_total", + "pyroscope_compactor_group_compaction_runs_started_total", + "pyroscope_compactor_group_compactions_failures_total", + "pyroscope_compactor_group_compactions_total", + "pyroscope_compactor_block_cleanup_failures_total", + "pyroscope_compactor_blocks_cleaned_total", + "pyroscope_compactor_blocks_marked_for_deletion_total", + "pyroscope_compactor_block_cleanup_started_total", + "pyroscope_compactor_block_cleanup_completed_total", + "pyroscope_compactor_block_cleanup_failed_total", )) } @@ -284,7 +287,7 @@ func TestMultitenantCompactor_ShouldRetryCompactionOnFailureWhileDiscoveringUser t.Parallel() // Fail to iterate over the bucket while discovering users. - bucketClient := &bucket.ClientMock{} + bucketClient := &pyroscope_objstore.ClientMock{} bucketClient.MockIter("", nil, errors.New("failed to iterate the bucket")) c, _, _, logs, registry := prepare(t, prepareConfig(t), bucketClient) @@ -314,108 +317,108 @@ func TestMultitenantCompactor_ShouldRetryCompactionOnFailureWhileDiscoveringUser }, removeIgnoredLogs(strings.Split(strings.TrimSpace(logs.String()), "\n"))) assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` - # TYPE cortex_compactor_runs_started_total counter - # HELP cortex_compactor_runs_started_total Total number of compaction runs started. - cortex_compactor_runs_started_total 1 - - # TYPE cortex_compactor_runs_completed_total counter - # HELP cortex_compactor_runs_completed_total Total number of compaction runs successfully completed. - cortex_compactor_runs_completed_total 0 - - # TYPE cortex_compactor_runs_failed_total counter - # HELP cortex_compactor_runs_failed_total Total number of compaction runs failed. - cortex_compactor_runs_failed_total{reason="error"} 1 - cortex_compactor_runs_failed_total{reason="shutdown"} 0 - - # HELP cortex_compactor_garbage_collection_duration_seconds Time it took to perform garbage collection iteration. - # TYPE cortex_compactor_garbage_collection_duration_seconds histogram - cortex_compactor_garbage_collection_duration_seconds_bucket{le="+Inf"} 0 - cortex_compactor_garbage_collection_duration_seconds_sum 0 - cortex_compactor_garbage_collection_duration_seconds_count 0 - - # HELP cortex_compactor_garbage_collection_failures_total Total number of failed garbage collection operations. - # TYPE cortex_compactor_garbage_collection_failures_total counter - cortex_compactor_garbage_collection_failures_total 0 - - # HELP cortex_compactor_garbage_collection_total Total number of garbage collection operations. - # TYPE cortex_compactor_garbage_collection_total counter - cortex_compactor_garbage_collection_total 0 - - # HELP cortex_compactor_meta_sync_duration_seconds Duration of the blocks metadata synchronization in seconds. - # TYPE cortex_compactor_meta_sync_duration_seconds histogram - cortex_compactor_meta_sync_duration_seconds_bucket{le="+Inf"} 0 - cortex_compactor_meta_sync_duration_seconds_sum 0 - cortex_compactor_meta_sync_duration_seconds_count 0 - - # HELP cortex_compactor_meta_sync_failures_total Total blocks metadata synchronization failures. - # TYPE cortex_compactor_meta_sync_failures_total counter - cortex_compactor_meta_sync_failures_total 0 - - # HELP cortex_compactor_meta_syncs_total Total blocks metadata synchronization attempts. - # TYPE cortex_compactor_meta_syncs_total counter - cortex_compactor_meta_syncs_total 0 - - # HELP cortex_compactor_group_compaction_runs_completed_total Total number of group completed compaction runs. This also includes compactor group runs that resulted with no compaction. - # TYPE cortex_compactor_group_compaction_runs_completed_total counter - cortex_compactor_group_compaction_runs_completed_total 0 - - # HELP cortex_compactor_group_compaction_runs_started_total Total number of group compaction attempts. - # TYPE cortex_compactor_group_compaction_runs_started_total counter - cortex_compactor_group_compaction_runs_started_total 0 - - # HELP cortex_compactor_group_compactions_failures_total Total number of failed group compactions. - # TYPE cortex_compactor_group_compactions_failures_total counter - cortex_compactor_group_compactions_failures_total 0 - - # HELP cortex_compactor_group_compactions_total Total number of group compaction attempts that resulted in new block(s). - # TYPE cortex_compactor_group_compactions_total counter - cortex_compactor_group_compactions_total 0 - - # TYPE cortex_compactor_block_cleanup_failures_total counter - # HELP cortex_compactor_block_cleanup_failures_total Total number of blocks failed to be deleted. - cortex_compactor_block_cleanup_failures_total 0 - - # HELP cortex_compactor_blocks_cleaned_total Total number of blocks deleted. - # TYPE cortex_compactor_blocks_cleaned_total counter - cortex_compactor_blocks_cleaned_total 0 - - # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. - # TYPE cortex_compactor_blocks_marked_for_deletion_total counter - cortex_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 - cortex_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 - cortex_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 - - # TYPE cortex_compactor_block_cleanup_started_total counter - # HELP cortex_compactor_block_cleanup_started_total Total number of blocks cleanup runs started. - cortex_compactor_block_cleanup_started_total 1 - - # TYPE cortex_compactor_block_cleanup_completed_total counter - # HELP cortex_compactor_block_cleanup_completed_total Total number of blocks cleanup runs successfully completed. - cortex_compactor_block_cleanup_completed_total 0 - - # TYPE cortex_compactor_block_cleanup_failed_total counter - # HELP cortex_compactor_block_cleanup_failed_total Total number of blocks cleanup runs failed. - cortex_compactor_block_cleanup_failed_total 1 + # TYPE pyroscope_compactor_runs_started_total counter + # HELP pyroscope_compactor_runs_started_total Total number of compaction runs started. + pyroscope_compactor_runs_started_total 1 + + # TYPE pyroscope_compactor_runs_completed_total counter + # HELP pyroscope_compactor_runs_completed_total Total number of compaction runs successfully completed. + pyroscope_compactor_runs_completed_total 0 + + # TYPE pyroscope_compactor_runs_failed_total counter + # HELP pyroscope_compactor_runs_failed_total Total number of compaction runs failed. + pyroscope_compactor_runs_failed_total{reason="error"} 1 + pyroscope_compactor_runs_failed_total{reason="shutdown"} 0 + + # HELP pyroscope_compactor_garbage_collection_duration_seconds Time it took to perform garbage collection iteration. + # TYPE pyroscope_compactor_garbage_collection_duration_seconds histogram + pyroscope_compactor_garbage_collection_duration_seconds_bucket{le="+Inf"} 0 + pyroscope_compactor_garbage_collection_duration_seconds_sum 0 + pyroscope_compactor_garbage_collection_duration_seconds_count 0 + + # HELP pyroscope_compactor_garbage_collection_failures_total Total number of failed garbage collection operations. + # TYPE pyroscope_compactor_garbage_collection_failures_total counter + pyroscope_compactor_garbage_collection_failures_total 0 + + # HELP pyroscope_compactor_garbage_collection_total Total number of garbage collection operations. + # TYPE pyroscope_compactor_garbage_collection_total counter + pyroscope_compactor_garbage_collection_total 0 + + # HELP pyroscope_compactor_meta_sync_duration_seconds Duration of the blocks metadata synchronization in seconds. + # TYPE pyroscope_compactor_meta_sync_duration_seconds histogram + pyroscope_compactor_meta_sync_duration_seconds_bucket{le="+Inf"} 0 + pyroscope_compactor_meta_sync_duration_seconds_sum 0 + pyroscope_compactor_meta_sync_duration_seconds_count 0 + + # HELP pyroscope_compactor_meta_sync_failures_total Total blocks metadata synchronization failures. + # TYPE pyroscope_compactor_meta_sync_failures_total counter + pyroscope_compactor_meta_sync_failures_total 0 + + # HELP pyroscope_compactor_meta_syncs_total Total blocks metadata synchronization attempts. + # TYPE pyroscope_compactor_meta_syncs_total counter + pyroscope_compactor_meta_syncs_total 0 + + # HELP pyroscope_compactor_group_compaction_runs_completed_total Total number of group completed compaction runs. This also includes compactor group runs that resulted with no compaction. + # TYPE pyroscope_compactor_group_compaction_runs_completed_total counter + pyroscope_compactor_group_compaction_runs_completed_total 0 + + # HELP pyroscope_compactor_group_compaction_runs_started_total Total number of group compaction attempts. + # TYPE pyroscope_compactor_group_compaction_runs_started_total counter + pyroscope_compactor_group_compaction_runs_started_total 0 + + # HELP pyroscope_compactor_group_compactions_failures_total Total number of failed group compactions. + # TYPE pyroscope_compactor_group_compactions_failures_total counter + pyroscope_compactor_group_compactions_failures_total 0 + + # HELP pyroscope_compactor_group_compactions_total Total number of group compaction attempts that resulted in new block(s). + # TYPE pyroscope_compactor_group_compactions_total counter + pyroscope_compactor_group_compactions_total 0 + + # TYPE pyroscope_compactor_block_cleanup_failures_total counter + # HELP pyroscope_compactor_block_cleanup_failures_total Total number of blocks failed to be deleted. + pyroscope_compactor_block_cleanup_failures_total 0 + + # HELP pyroscope_compactor_blocks_cleaned_total Total number of blocks deleted. + # TYPE pyroscope_compactor_blocks_cleaned_total counter + pyroscope_compactor_blocks_cleaned_total 0 + + # HELP pyroscope_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. + # TYPE pyroscope_compactor_blocks_marked_for_deletion_total counter + pyroscope_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 + pyroscope_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 + pyroscope_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 + + # TYPE pyroscope_compactor_block_cleanup_started_total counter + # HELP pyroscope_compactor_block_cleanup_started_total Total number of blocks cleanup runs started. + pyroscope_compactor_block_cleanup_started_total 1 + + # TYPE pyroscope_compactor_block_cleanup_completed_total counter + # HELP pyroscope_compactor_block_cleanup_completed_total Total number of blocks cleanup runs successfully completed. + pyroscope_compactor_block_cleanup_completed_total 0 + + # TYPE pyroscope_compactor_block_cleanup_failed_total counter + # HELP pyroscope_compactor_block_cleanup_failed_total Total number of blocks cleanup runs failed. + pyroscope_compactor_block_cleanup_failed_total 1 `), - "cortex_compactor_runs_started_total", - "cortex_compactor_runs_completed_total", - "cortex_compactor_runs_failed_total", - "cortex_compactor_garbage_collection_duration_seconds", - "cortex_compactor_garbage_collection_failures_total", - "cortex_compactor_garbage_collection_total", - "cortex_compactor_meta_sync_duration_seconds", - "cortex_compactor_meta_sync_failures_total", - "cortex_compactor_meta_syncs_total", - "cortex_compactor_group_compaction_runs_completed_total", - "cortex_compactor_group_compaction_runs_started_total", - "cortex_compactor_group_compactions_failures_total", - "cortex_compactor_group_compactions_total", - "cortex_compactor_block_cleanup_failures_total", - "cortex_compactor_blocks_cleaned_total", - "cortex_compactor_blocks_marked_for_deletion_total", - "cortex_compactor_block_cleanup_started_total", - "cortex_compactor_block_cleanup_completed_total", - "cortex_compactor_block_cleanup_failed_total", + "pyroscope_compactor_runs_started_total", + "pyroscope_compactor_runs_completed_total", + "pyroscope_compactor_runs_failed_total", + "pyroscope_compactor_garbage_collection_duration_seconds", + "pyroscope_compactor_garbage_collection_failures_total", + "pyroscope_compactor_garbage_collection_total", + "pyroscope_compactor_meta_sync_duration_seconds", + "pyroscope_compactor_meta_sync_failures_total", + "pyroscope_compactor_meta_syncs_total", + "pyroscope_compactor_group_compaction_runs_completed_total", + "pyroscope_compactor_group_compaction_runs_started_total", + "pyroscope_compactor_group_compactions_failures_total", + "pyroscope_compactor_group_compactions_total", + "pyroscope_compactor_block_cleanup_failures_total", + "pyroscope_compactor_blocks_cleaned_total", + "pyroscope_compactor_blocks_marked_for_deletion_total", + "pyroscope_compactor_block_cleanup_started_total", + "pyroscope_compactor_block_cleanup_completed_total", + "pyroscope_compactor_block_cleanup_failed_total", )) } @@ -423,7 +426,7 @@ func TestMultitenantCompactor_ShouldIncrementCompactionErrorIfFailedToCompactASi t.Parallel() userID := "test-user" - bucketClient := &bucket.ClientMock{} + bucketClient := &pyroscope_objstore.ClientMock{} bucketClient.MockIter("", []string{userID}, nil) bucketClient.MockIter(userID+"/", []string{userID + "/01DTVP434PA9VFXSW2JKB3392D", userID + "/01DTW0ZCPDDNV4BV83Q2SV4QAZ"}, nil) bucketClient.MockIter(userID+"/markers/", nil, nil) @@ -449,22 +452,22 @@ func TestMultitenantCompactor_ShouldIncrementCompactionErrorIfFailedToCompactASi require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` - # TYPE cortex_compactor_runs_started_total counter - # HELP cortex_compactor_runs_started_total Total number of compaction runs started. - cortex_compactor_runs_started_total 1 - - # TYPE cortex_compactor_runs_completed_total counter - # HELP cortex_compactor_runs_completed_total Total number of compaction runs successfully completed. - cortex_compactor_runs_completed_total 0 - - # TYPE cortex_compactor_runs_failed_total counter - # HELP cortex_compactor_runs_failed_total Total number of compaction runs failed. - cortex_compactor_runs_failed_total{reason="error"} 1 - cortex_compactor_runs_failed_total{reason="shutdown"} 0 + # TYPE pyroscope_compactor_runs_started_total counter + # HELP pyroscope_compactor_runs_started_total Total number of compaction runs started. + pyroscope_compactor_runs_started_total 1 + + # TYPE pyroscope_compactor_runs_completed_total counter + # HELP pyroscope_compactor_runs_completed_total Total number of compaction runs successfully completed. + pyroscope_compactor_runs_completed_total 0 + + # TYPE pyroscope_compactor_runs_failed_total counter + # HELP pyroscope_compactor_runs_failed_total Total number of compaction runs failed. + pyroscope_compactor_runs_failed_total{reason="error"} 1 + pyroscope_compactor_runs_failed_total{reason="shutdown"} 0 `), - "cortex_compactor_runs_started_total", - "cortex_compactor_runs_completed_total", - "cortex_compactor_runs_failed_total", + "pyroscope_compactor_runs_started_total", + "pyroscope_compactor_runs_completed_total", + "pyroscope_compactor_runs_failed_total", )) } @@ -472,7 +475,7 @@ func TestMultitenantCompactor_ShouldIncrementCompactionShutdownIfTheContextIsCan t.Parallel() userID := "test-user" - bucketClient := &bucket.ClientMock{} + bucketClient := &pyroscope_objstore.ClientMock{} bucketClient.MockIter("", []string{userID}, nil) bucketClient.MockIter(userID+"/", []string{userID + "/01DTVP434PA9VFXSW2JKB3392D", userID + "/01DTW0ZCPDDNV4BV83Q2SV4QAZ"}, nil) bucketClient.MockIter(userID+"/markers/", nil, nil) @@ -502,22 +505,22 @@ func TestMultitenantCompactor_ShouldIncrementCompactionShutdownIfTheContextIsCan require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` - # TYPE cortex_compactor_runs_started_total counter - # HELP cortex_compactor_runs_started_total Total number of compaction runs started. - cortex_compactor_runs_started_total 1 - - # TYPE cortex_compactor_runs_completed_total counter - # HELP cortex_compactor_runs_completed_total Total number of compaction runs successfully completed. - cortex_compactor_runs_completed_total 0 - - # TYPE cortex_compactor_runs_failed_total counter - # HELP cortex_compactor_runs_failed_total Total number of compaction runs failed. - cortex_compactor_runs_failed_total{reason="error"} 0 - cortex_compactor_runs_failed_total{reason="shutdown"} 1 + # TYPE pyroscope_compactor_runs_started_total counter + # HELP pyroscope_compactor_runs_started_total Total number of compaction runs started. + pyroscope_compactor_runs_started_total 1 + + # TYPE pyroscope_compactor_runs_completed_total counter + # HELP pyroscope_compactor_runs_completed_total Total number of compaction runs successfully completed. + pyroscope_compactor_runs_completed_total 0 + + # TYPE pyroscope_compactor_runs_failed_total counter + # HELP pyroscope_compactor_runs_failed_total Total number of compaction runs failed. + pyroscope_compactor_runs_failed_total{reason="error"} 0 + pyroscope_compactor_runs_failed_total{reason="shutdown"} 1 `), - "cortex_compactor_runs_started_total", - "cortex_compactor_runs_completed_total", - "cortex_compactor_runs_failed_total", + "pyroscope_compactor_runs_started_total", + "pyroscope_compactor_runs_completed_total", + "pyroscope_compactor_runs_failed_total", )) } @@ -525,7 +528,7 @@ func TestMultitenantCompactor_ShouldIterateOverUsersAndRunCompaction(t *testing. t.Parallel() // Mock the bucket to contain two users, each one with two blocks (to make sure that grouper doesn't skip them). - bucketClient := &bucket.ClientMock{} + bucketClient := &pyroscope_objstore.ClientMock{} bucketClient.MockIter("", []string{"user-1", "user-2"}, nil) bucketClient.MockExists(path.Join("user-1", bucket.TenantDeletionMarkPath), false, nil) bucketClient.MockExists(path.Join("user-2", bucket.TenantDeletionMarkPath), false, nil) @@ -600,68 +603,68 @@ func TestMultitenantCompactor_ShouldIterateOverUsersAndRunCompaction(t *testing. // Instead of testing for shipper metrics, we only check our metrics here. // Real shipper metrics are too variable to embed into a test. testedMetrics := []string{ - "cortex_compactor_runs_started_total", "cortex_compactor_runs_completed_total", "cortex_compactor_runs_failed_total", - "cortex_compactor_blocks_cleaned_total", "cortex_compactor_block_cleanup_failures_total", "cortex_compactor_blocks_marked_for_deletion_total", - "cortex_compactor_block_cleanup_started_total", "cortex_compactor_block_cleanup_completed_total", "cortex_compactor_block_cleanup_failed_total", - "cortex_compactor_group_compaction_runs_completed_total", "cortex_compactor_group_compaction_runs_started_total", - "cortex_compactor_group_compactions_failures_total", "cortex_compactor_group_compactions_total", + "pyroscope_compactor_runs_started_total", "pyroscope_compactor_runs_completed_total", "pyroscope_compactor_runs_failed_total", + "pyroscope_compactor_blocks_cleaned_total", "pyroscope_compactor_block_cleanup_failures_total", "pyroscope_compactor_blocks_marked_for_deletion_total", + "pyroscope_compactor_block_cleanup_started_total", "pyroscope_compactor_block_cleanup_completed_total", "pyroscope_compactor_block_cleanup_failed_total", + "pyroscope_compactor_group_compaction_runs_completed_total", "pyroscope_compactor_group_compaction_runs_started_total", + "pyroscope_compactor_group_compactions_failures_total", "pyroscope_compactor_group_compactions_total", } assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` - # TYPE cortex_compactor_runs_started_total counter - # HELP cortex_compactor_runs_started_total Total number of compaction runs started. - cortex_compactor_runs_started_total 1 - - # TYPE cortex_compactor_runs_completed_total counter - # HELP cortex_compactor_runs_completed_total Total number of compaction runs successfully completed. - cortex_compactor_runs_completed_total 1 - - # TYPE cortex_compactor_runs_failed_total counter - # HELP cortex_compactor_runs_failed_total Total number of compaction runs failed. - cortex_compactor_runs_failed_total{reason="error"} 0 - cortex_compactor_runs_failed_total{reason="shutdown"} 0 - - # HELP cortex_compactor_group_compaction_runs_completed_total Total number of group completed compaction runs. This also includes compactor group runs that resulted with no compaction. - # TYPE cortex_compactor_group_compaction_runs_completed_total counter - cortex_compactor_group_compaction_runs_completed_total 2 - - # HELP cortex_compactor_group_compaction_runs_started_total Total number of group compaction attempts. - # TYPE cortex_compactor_group_compaction_runs_started_total counter - cortex_compactor_group_compaction_runs_started_total 2 - - # HELP cortex_compactor_group_compactions_failures_total Total number of failed group compactions. - # TYPE cortex_compactor_group_compactions_failures_total counter - cortex_compactor_group_compactions_failures_total 0 - - # HELP cortex_compactor_group_compactions_total Total number of group compaction attempts that resulted in new block(s). - # TYPE cortex_compactor_group_compactions_total counter - cortex_compactor_group_compactions_total 0 - - # TYPE cortex_compactor_block_cleanup_failures_total counter - # HELP cortex_compactor_block_cleanup_failures_total Total number of blocks failed to be deleted. - cortex_compactor_block_cleanup_failures_total 0 - - # HELP cortex_compactor_blocks_cleaned_total Total number of blocks deleted. - # TYPE cortex_compactor_blocks_cleaned_total counter - cortex_compactor_blocks_cleaned_total 0 - - # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. - # TYPE cortex_compactor_blocks_marked_for_deletion_total counter - cortex_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 - cortex_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 - cortex_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 - - # TYPE cortex_compactor_block_cleanup_started_total counter - # HELP cortex_compactor_block_cleanup_started_total Total number of blocks cleanup runs started. - cortex_compactor_block_cleanup_started_total 1 - - # TYPE cortex_compactor_block_cleanup_completed_total counter - # HELP cortex_compactor_block_cleanup_completed_total Total number of blocks cleanup runs successfully completed. - cortex_compactor_block_cleanup_completed_total 1 - - # TYPE cortex_compactor_block_cleanup_failed_total counter - # HELP cortex_compactor_block_cleanup_failed_total Total number of blocks cleanup runs failed. - cortex_compactor_block_cleanup_failed_total 0 + # TYPE pyroscope_compactor_runs_started_total counter + # HELP pyroscope_compactor_runs_started_total Total number of compaction runs started. + pyroscope_compactor_runs_started_total 1 + + # TYPE pyroscope_compactor_runs_completed_total counter + # HELP pyroscope_compactor_runs_completed_total Total number of compaction runs successfully completed. + pyroscope_compactor_runs_completed_total 1 + + # TYPE pyroscope_compactor_runs_failed_total counter + # HELP pyroscope_compactor_runs_failed_total Total number of compaction runs failed. + pyroscope_compactor_runs_failed_total{reason="error"} 0 + pyroscope_compactor_runs_failed_total{reason="shutdown"} 0 + + # HELP pyroscope_compactor_group_compaction_runs_completed_total Total number of group completed compaction runs. This also includes compactor group runs that resulted with no compaction. + # TYPE pyroscope_compactor_group_compaction_runs_completed_total counter + pyroscope_compactor_group_compaction_runs_completed_total 2 + + # HELP pyroscope_compactor_group_compaction_runs_started_total Total number of group compaction attempts. + # TYPE pyroscope_compactor_group_compaction_runs_started_total counter + pyroscope_compactor_group_compaction_runs_started_total 2 + + # HELP pyroscope_compactor_group_compactions_failures_total Total number of failed group compactions. + # TYPE pyroscope_compactor_group_compactions_failures_total counter + pyroscope_compactor_group_compactions_failures_total 0 + + # HELP pyroscope_compactor_group_compactions_total Total number of group compaction attempts that resulted in new block(s). + # TYPE pyroscope_compactor_group_compactions_total counter + pyroscope_compactor_group_compactions_total 0 + + # TYPE pyroscope_compactor_block_cleanup_failures_total counter + # HELP pyroscope_compactor_block_cleanup_failures_total Total number of blocks failed to be deleted. + pyroscope_compactor_block_cleanup_failures_total 0 + + # HELP pyroscope_compactor_blocks_cleaned_total Total number of blocks deleted. + # TYPE pyroscope_compactor_blocks_cleaned_total counter + pyroscope_compactor_blocks_cleaned_total 0 + + # HELP pyroscope_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. + # TYPE pyroscope_compactor_blocks_marked_for_deletion_total counter + pyroscope_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 + pyroscope_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 + pyroscope_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 + + # TYPE pyroscope_compactor_block_cleanup_started_total counter + # HELP pyroscope_compactor_block_cleanup_started_total Total number of blocks cleanup runs started. + pyroscope_compactor_block_cleanup_started_total 1 + + # TYPE pyroscope_compactor_block_cleanup_completed_total counter + # HELP pyroscope_compactor_block_cleanup_completed_total Total number of blocks cleanup runs successfully completed. + pyroscope_compactor_block_cleanup_completed_total 1 + + # TYPE pyroscope_compactor_block_cleanup_failed_total counter + # HELP pyroscope_compactor_block_cleanup_failed_total Total number of blocks cleanup runs failed. + pyroscope_compactor_block_cleanup_failed_total 0 `), testedMetrics...)) } @@ -670,7 +673,7 @@ func TestMultitenantCompactor_ShouldStopCompactingTenantOnReachingMaxCompactionT // By using blocks with different labels, we get two compaction jobs. Only one of these jobs will be started, // and since its planning will take longer than maxCompactionTime, we stop compactions early. - bucketClient := &bucket.ClientMock{} + bucketClient := &pyroscope_objstore.ClientMock{} bucketClient.MockIter("", []string{"user-1"}, nil) bucketClient.MockExists(path.Join("user-1", bucket.TenantDeletionMarkPath), false, nil) bucketClient.MockIter("user-1/", []string{"user-1/01DTVP434PA9VFXSW2JKB3392D", "user-1/01FN3VCQV5X342W2ZKMQQXAZRX", "user-1/01FS51A7GQ1RQWV35DBVYQM4KF", "user-1/01FRQGQB7RWQ2TS0VWA82QTPXE"}, nil) @@ -740,7 +743,7 @@ func TestMultitenantCompactor_ShouldNotCompactBlocksMarkedForDeletion(t *testing cfg.DeletionDelay = 10 * time.Minute // Delete block after 10 minutes // Mock the bucket to contain two users, each one with one block. - bucketClient := &bucket.ClientMock{} + bucketClient := &pyroscope_objstore.ClientMock{} bucketClient.MockIter("", []string{"user-1"}, nil) bucketClient.MockIter("user-1/", []string{"user-1/01DTVP434PA9VFXSW2JKB3392D", "user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ"}, nil) bucketClient.MockExists(path.Join("user-1", bucket.TenantDeletionMarkPath), false, nil) @@ -805,49 +808,49 @@ func TestMultitenantCompactor_ShouldNotCompactBlocksMarkedForDeletion(t *testing // Instead of testing for shipper metrics, we only check our metrics here. // Real shipper metrics are too variable to embed into a test. testedMetrics := []string{ - "cortex_compactor_runs_started_total", "cortex_compactor_runs_completed_total", "cortex_compactor_runs_failed_total", - "cortex_compactor_blocks_cleaned_total", "cortex_compactor_block_cleanup_failures_total", "cortex_compactor_blocks_marked_for_deletion_total", - "cortex_compactor_block_cleanup_started_total", "cortex_compactor_block_cleanup_completed_total", "cortex_compactor_block_cleanup_failed_total", + "pyroscope_compactor_runs_started_total", "pyroscope_compactor_runs_completed_total", "pyroscope_compactor_runs_failed_total", + "pyroscope_compactor_blocks_cleaned_total", "pyroscope_compactor_block_cleanup_failures_total", "pyroscope_compactor_blocks_marked_for_deletion_total", + "pyroscope_compactor_block_cleanup_started_total", "pyroscope_compactor_block_cleanup_completed_total", "pyroscope_compactor_block_cleanup_failed_total", } assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` - # TYPE cortex_compactor_runs_started_total counter - # HELP cortex_compactor_runs_started_total Total number of compaction runs started. - cortex_compactor_runs_started_total 1 - - # TYPE cortex_compactor_runs_completed_total counter - # HELP cortex_compactor_runs_completed_total Total number of compaction runs successfully completed. - cortex_compactor_runs_completed_total 1 - - # TYPE cortex_compactor_runs_failed_total counter - # HELP cortex_compactor_runs_failed_total Total number of compaction runs failed. - cortex_compactor_runs_failed_total{reason="error"} 0 - cortex_compactor_runs_failed_total{reason="shutdown"} 0 - - # TYPE cortex_compactor_block_cleanup_failures_total counter - # HELP cortex_compactor_block_cleanup_failures_total Total number of blocks failed to be deleted. - cortex_compactor_block_cleanup_failures_total 0 - - # HELP cortex_compactor_blocks_cleaned_total Total number of blocks deleted. - # TYPE cortex_compactor_blocks_cleaned_total counter - cortex_compactor_blocks_cleaned_total 1 - - # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. - # TYPE cortex_compactor_blocks_marked_for_deletion_total counter - cortex_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 - cortex_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 - cortex_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 - - # TYPE cortex_compactor_block_cleanup_started_total counter - # HELP cortex_compactor_block_cleanup_started_total Total number of blocks cleanup runs started. - cortex_compactor_block_cleanup_started_total 1 - - # TYPE cortex_compactor_block_cleanup_completed_total counter - # HELP cortex_compactor_block_cleanup_completed_total Total number of blocks cleanup runs successfully completed. - cortex_compactor_block_cleanup_completed_total 1 - - # TYPE cortex_compactor_block_cleanup_failed_total counter - # HELP cortex_compactor_block_cleanup_failed_total Total number of blocks cleanup runs failed. - cortex_compactor_block_cleanup_failed_total 0 + # TYPE pyroscope_compactor_runs_started_total counter + # HELP pyroscope_compactor_runs_started_total Total number of compaction runs started. + pyroscope_compactor_runs_started_total 1 + + # TYPE pyroscope_compactor_runs_completed_total counter + # HELP pyroscope_compactor_runs_completed_total Total number of compaction runs successfully completed. + pyroscope_compactor_runs_completed_total 1 + + # TYPE pyroscope_compactor_runs_failed_total counter + # HELP pyroscope_compactor_runs_failed_total Total number of compaction runs failed. + pyroscope_compactor_runs_failed_total{reason="error"} 0 + pyroscope_compactor_runs_failed_total{reason="shutdown"} 0 + + # TYPE pyroscope_compactor_block_cleanup_failures_total counter + # HELP pyroscope_compactor_block_cleanup_failures_total Total number of blocks failed to be deleted. + pyroscope_compactor_block_cleanup_failures_total 0 + + # HELP pyroscope_compactor_blocks_cleaned_total Total number of blocks deleted. + # TYPE pyroscope_compactor_blocks_cleaned_total counter + pyroscope_compactor_blocks_cleaned_total 1 + + # HELP pyroscope_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. + # TYPE pyroscope_compactor_blocks_marked_for_deletion_total counter + pyroscope_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 + pyroscope_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 + pyroscope_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 + + # TYPE pyroscope_compactor_block_cleanup_started_total counter + # HELP pyroscope_compactor_block_cleanup_started_total Total number of blocks cleanup runs started. + pyroscope_compactor_block_cleanup_started_total 1 + + # TYPE pyroscope_compactor_block_cleanup_completed_total counter + # HELP pyroscope_compactor_block_cleanup_completed_total Total number of blocks cleanup runs successfully completed. + pyroscope_compactor_block_cleanup_completed_total 1 + + # TYPE pyroscope_compactor_block_cleanup_failed_total counter + # HELP pyroscope_compactor_block_cleanup_failed_total Total number of blocks cleanup runs failed. + pyroscope_compactor_block_cleanup_failed_total 0 `), testedMetrics...)) } @@ -858,7 +861,7 @@ func TestMultitenantCompactor_ShouldNotCompactBlocksMarkedForNoCompaction(t *tes cfg.DeletionDelay = 10 * time.Minute // Delete block after 10 minutes // Mock the bucket to contain one user with a block marked for no-compaction. - bucketClient := &bucket.ClientMock{} + bucketClient := &pyroscope_objstore.ClientMock{} bucketClient.MockIter("", []string{"user-1"}, nil) bucketClient.MockIter("user-1/", []string{"user-1/01DTVP434PA9VFXSW2JKB3392D"}, nil) bucketClient.MockExists(path.Join("user-1", bucket.TenantDeletionMarkPath), false, nil) @@ -912,7 +915,7 @@ func TestMultitenantCompactor_ShouldNotCompactBlocksForUsersMarkedForDeletion(t cfg.TenantCleanupDelay = 10 * time.Minute // To make sure it's not 0. // Mock the bucket to contain two users, each one with one block. - bucketClient := &bucket.ClientMock{} + bucketClient := &pyroscope_objstore.ClientMock{} bucketClient.MockIter("", []string{"user-1"}, nil) bucketClient.MockIter("user-1/", []string{"user-1/01DTVP434PA9VFXSW2JKB3392D"}, nil) bucketClient.MockGet(path.Join("user-1", bucket.TenantDeletionMarkPath), `{"deletion_time": 1}`, nil) @@ -962,50 +965,50 @@ func TestMultitenantCompactor_ShouldNotCompactBlocksForUsersMarkedForDeletion(t // Instead of testing for shipper metrics, we only check our metrics here. // Real shipper metrics are too variable to embed into a test. testedMetrics := []string{ - "cortex_compactor_runs_started_total", "cortex_compactor_runs_completed_total", "cortex_compactor_runs_failed_total", - "cortex_compactor_blocks_cleaned_total", "cortex_compactor_block_cleanup_failures_total", "cortex_compactor_blocks_marked_for_deletion_total", - "cortex_compactor_block_cleanup_started_total", "cortex_compactor_block_cleanup_completed_total", "cortex_compactor_block_cleanup_failed_total", - "cortex_bucket_blocks_count", "cortex_bucket_blocks_marked_for_deletion_count", "cortex_bucket_index_last_successful_update_timestamp_seconds", + "pyroscope_compactor_runs_started_total", "pyroscope_compactor_runs_completed_total", "pyroscope_compactor_runs_failed_total", + "pyroscope_compactor_blocks_cleaned_total", "pyroscope_compactor_block_cleanup_failures_total", "pyroscope_compactor_blocks_marked_for_deletion_total", + "pyroscope_compactor_block_cleanup_started_total", "pyroscope_compactor_block_cleanup_completed_total", "pyroscope_compactor_block_cleanup_failed_total", + "pyroscope_bucket_blocks_count", "pyroscope_bucket_blocks_marked_for_deletion_count", "pyroscope_bucket_index_last_successful_update_timestamp_seconds", } assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` - # TYPE cortex_compactor_runs_started_total counter - # HELP cortex_compactor_runs_started_total Total number of compaction runs started. - cortex_compactor_runs_started_total 1 - - # TYPE cortex_compactor_runs_completed_total counter - # HELP cortex_compactor_runs_completed_total Total number of compaction runs successfully completed. - cortex_compactor_runs_completed_total 1 - - # TYPE cortex_compactor_runs_failed_total counter - # HELP cortex_compactor_runs_failed_total Total number of compaction runs failed. - cortex_compactor_runs_failed_total{reason="error"} 0 - cortex_compactor_runs_failed_total{reason="shutdown"} 0 - - # TYPE cortex_compactor_block_cleanup_failures_total counter - # HELP cortex_compactor_block_cleanup_failures_total Total number of blocks failed to be deleted. - cortex_compactor_block_cleanup_failures_total 0 - - # HELP cortex_compactor_blocks_cleaned_total Total number of blocks deleted. - # TYPE cortex_compactor_blocks_cleaned_total counter - cortex_compactor_blocks_cleaned_total 1 - - # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. - # TYPE cortex_compactor_blocks_marked_for_deletion_total counter - cortex_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 - cortex_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 - cortex_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 - - # TYPE cortex_compactor_block_cleanup_started_total counter - # HELP cortex_compactor_block_cleanup_started_total Total number of blocks cleanup runs started. - cortex_compactor_block_cleanup_started_total 1 - - # TYPE cortex_compactor_block_cleanup_completed_total counter - # HELP cortex_compactor_block_cleanup_completed_total Total number of blocks cleanup runs successfully completed. - cortex_compactor_block_cleanup_completed_total 1 - - # TYPE cortex_compactor_block_cleanup_failed_total counter - # HELP cortex_compactor_block_cleanup_failed_total Total number of blocks cleanup runs failed. - cortex_compactor_block_cleanup_failed_total 0 + # TYPE pyroscope_compactor_runs_started_total counter + # HELP pyroscope_compactor_runs_started_total Total number of compaction runs started. + pyroscope_compactor_runs_started_total 1 + + # TYPE pyroscope_compactor_runs_completed_total counter + # HELP pyroscope_compactor_runs_completed_total Total number of compaction runs successfully completed. + pyroscope_compactor_runs_completed_total 1 + + # TYPE pyroscope_compactor_runs_failed_total counter + # HELP pyroscope_compactor_runs_failed_total Total number of compaction runs failed. + pyroscope_compactor_runs_failed_total{reason="error"} 0 + pyroscope_compactor_runs_failed_total{reason="shutdown"} 0 + + # TYPE pyroscope_compactor_block_cleanup_failures_total counter + # HELP pyroscope_compactor_block_cleanup_failures_total Total number of blocks failed to be deleted. + pyroscope_compactor_block_cleanup_failures_total 0 + + # HELP pyroscope_compactor_blocks_cleaned_total Total number of blocks deleted. + # TYPE pyroscope_compactor_blocks_cleaned_total counter + pyroscope_compactor_blocks_cleaned_total 1 + + # HELP pyroscope_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. + # TYPE pyroscope_compactor_blocks_marked_for_deletion_total counter + pyroscope_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 + pyroscope_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 + pyroscope_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 + + # TYPE pyroscope_compactor_block_cleanup_started_total counter + # HELP pyroscope_compactor_block_cleanup_started_total Total number of blocks cleanup runs started. + pyroscope_compactor_block_cleanup_started_total 1 + + # TYPE pyroscope_compactor_block_cleanup_completed_total counter + # HELP pyroscope_compactor_block_cleanup_completed_total Total number of blocks cleanup runs successfully completed. + pyroscope_compactor_block_cleanup_completed_total 1 + + # TYPE pyroscope_compactor_block_cleanup_failed_total counter + # HELP pyroscope_compactor_block_cleanup_failed_total Total number of blocks cleanup runs failed. + pyroscope_compactor_block_cleanup_failed_total 0 `), testedMetrics...)) } @@ -1013,7 +1016,7 @@ func TestMultitenantCompactor_ShouldCompactAllUsersOnShardingEnabledButOnlyOneIn t.Parallel() // Mock the bucket to contain two users, each one with one block. - bucketClient := &bucket.ClientMock{} + bucketClient := &pyroscope_objstore.ClientMock{} bucketClient.MockIter("", []string{"user-1", "user-2"}, nil) bucketClient.MockExists(path.Join("user-1", bucket.TenantDeletionMarkPath), false, nil) bucketClient.MockExists(path.Join("user-2", bucket.TenantDeletionMarkPath), false, nil) @@ -1092,49 +1095,49 @@ func TestMultitenantCompactor_ShouldCompactAllUsersOnShardingEnabledButOnlyOneIn }, removeIgnoredLogs(strings.Split(strings.TrimSpace(logs.String()), "\n"))) assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` - # TYPE cortex_compactor_runs_started_total counter - # HELP cortex_compactor_runs_started_total Total number of compaction runs started. - cortex_compactor_runs_started_total 1 - - # TYPE cortex_compactor_runs_completed_total counter - # HELP cortex_compactor_runs_completed_total Total number of compaction runs successfully completed. - cortex_compactor_runs_completed_total 1 - - # TYPE cortex_compactor_runs_failed_total counter - # HELP cortex_compactor_runs_failed_total Total number of compaction runs failed. - cortex_compactor_runs_failed_total{reason="error"} 0 - cortex_compactor_runs_failed_total{reason="shutdown"} 0 - - # HELP cortex_compactor_group_compaction_runs_completed_total Total number of group completed compaction runs. This also includes compactor group runs that resulted with no compaction. - # TYPE cortex_compactor_group_compaction_runs_completed_total counter - cortex_compactor_group_compaction_runs_completed_total 2 - - # HELP cortex_compactor_group_compaction_runs_started_total Total number of group compaction attempts. - # TYPE cortex_compactor_group_compaction_runs_started_total counter - cortex_compactor_group_compaction_runs_started_total 2 - - # HELP cortex_compactor_group_compactions_failures_total Total number of failed group compactions. - # TYPE cortex_compactor_group_compactions_failures_total counter - cortex_compactor_group_compactions_failures_total 0 - - # HELP cortex_compactor_group_compactions_total Total number of group compaction attempts that resulted in new block(s). - # TYPE cortex_compactor_group_compactions_total counter - cortex_compactor_group_compactions_total 0 - - # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. - # TYPE cortex_compactor_blocks_marked_for_deletion_total counter - cortex_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 - cortex_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 - cortex_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 + # TYPE pyroscope_compactor_runs_started_total counter + # HELP pyroscope_compactor_runs_started_total Total number of compaction runs started. + pyroscope_compactor_runs_started_total 1 + + # TYPE pyroscope_compactor_runs_completed_total counter + # HELP pyroscope_compactor_runs_completed_total Total number of compaction runs successfully completed. + pyroscope_compactor_runs_completed_total 1 + + # TYPE pyroscope_compactor_runs_failed_total counter + # HELP pyroscope_compactor_runs_failed_total Total number of compaction runs failed. + pyroscope_compactor_runs_failed_total{reason="error"} 0 + pyroscope_compactor_runs_failed_total{reason="shutdown"} 0 + + # HELP pyroscope_compactor_group_compaction_runs_completed_total Total number of group completed compaction runs. This also includes compactor group runs that resulted with no compaction. + # TYPE pyroscope_compactor_group_compaction_runs_completed_total counter + pyroscope_compactor_group_compaction_runs_completed_total 2 + + # HELP pyroscope_compactor_group_compaction_runs_started_total Total number of group compaction attempts. + # TYPE pyroscope_compactor_group_compaction_runs_started_total counter + pyroscope_compactor_group_compaction_runs_started_total 2 + + # HELP pyroscope_compactor_group_compactions_failures_total Total number of failed group compactions. + # TYPE pyroscope_compactor_group_compactions_failures_total counter + pyroscope_compactor_group_compactions_failures_total 0 + + # HELP pyroscope_compactor_group_compactions_total Total number of group compaction attempts that resulted in new block(s). + # TYPE pyroscope_compactor_group_compactions_total counter + pyroscope_compactor_group_compactions_total 0 + + # HELP pyroscope_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. + # TYPE pyroscope_compactor_blocks_marked_for_deletion_total counter + pyroscope_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 + pyroscope_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 + pyroscope_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 `), - "cortex_compactor_runs_started_total", - "cortex_compactor_runs_completed_total", - "cortex_compactor_runs_failed_total", - "cortex_compactor_group_compaction_runs_completed_total", - "cortex_compactor_group_compaction_runs_started_total", - "cortex_compactor_group_compactions_failures_total", - "cortex_compactor_group_compactions_total", - "cortex_compactor_blocks_marked_for_deletion_total", + "pyroscope_compactor_runs_started_total", + "pyroscope_compactor_runs_completed_total", + "pyroscope_compactor_runs_failed_total", + "pyroscope_compactor_group_compaction_runs_completed_total", + "pyroscope_compactor_group_compaction_runs_started_total", + "pyroscope_compactor_group_compactions_failures_total", + "pyroscope_compactor_group_compactions_total", + "pyroscope_compactor_blocks_marked_for_deletion_total", )) } @@ -1150,7 +1153,7 @@ func TestMultitenantCompactor_ShouldCompactOnlyUsersOwnedByTheInstanceOnSharding } // Mock the bucket to contain all users, each one with one block. - bucketClient := &bucket.ClientMock{} + bucketClient := &pyroscope_objstore.ClientMock{} bucketClient.MockIter("", userIDs, nil) for _, userID := range userIDs { bucketClient.MockIter(userID+"/", []string{userID + "/01DTVP434PA9VFXSW2JKB3392D"}, nil) @@ -1284,7 +1287,7 @@ func TestMultitenantCompactor_ShouldFailWithInvalidTSDBCompactOutput(t *testing. _, err = block.GenerateBlockFromSpec(user, filepath.Join(storageDir, user), sourceBlock3Spec) require.NoError(t, err) - bkt, err := filesystem.NewBucketClient(filesystem.Config{Directory: storageDir}) + bkt, err := filesystem.NewBucket(storageDir) require.NoError(t, err) cfg := prepareConfig(t) @@ -1326,7 +1329,7 @@ func TestMultitenantCompactor_ShouldSkipCompactionForJobsNoMoreOwnedAfterPlannin // Mock the bucket to contain one user with two non-overlapping blocks (we expect two compaction jobs to be scheduled // for the splitting stage). - bucketClient := &bucket.ClientMock{} + bucketClient := &pyroscope_objstore.ClientMock{} bucketClient.MockIter("", []string{"user-1"}, nil) bucketClient.MockExists(path.Join("user-1", bucket.TenantDeletionMarkPath), false, nil) bucketClient.MockIter("user-1/", []string{"user-1/01DTVP434PA9VFXSW2JK000001", "user-1/01DTVP434PA9VFXSW2JK000002"}, nil) @@ -1404,49 +1407,49 @@ func TestMultitenantCompactor_ShouldSkipCompactionForJobsNoMoreOwnedAfterPlannin }, removeIgnoredLogs(strings.Split(strings.TrimSpace(logs.String()), "\n"))) assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` - # TYPE cortex_compactor_runs_started_total counter - # HELP cortex_compactor_runs_started_total Total number of compaction runs started. - cortex_compactor_runs_started_total 1 - - # TYPE cortex_compactor_runs_completed_total counter - # HELP cortex_compactor_runs_completed_total Total number of compaction runs successfully completed. - cortex_compactor_runs_completed_total 1 - - # TYPE cortex_compactor_runs_failed_total counter - # HELP cortex_compactor_runs_failed_total Total number of compaction runs failed. - cortex_compactor_runs_failed_total{reason="error"} 0 - cortex_compactor_runs_failed_total{reason="shutdown"} 0 - - # HELP cortex_compactor_group_compaction_runs_completed_total Total number of group completed compaction runs. This also includes compactor group runs that resulted with no compaction. - # TYPE cortex_compactor_group_compaction_runs_completed_total counter - cortex_compactor_group_compaction_runs_completed_total 1 - - # HELP cortex_compactor_group_compaction_runs_started_total Total number of group compaction attempts. - # TYPE cortex_compactor_group_compaction_runs_started_total counter - cortex_compactor_group_compaction_runs_started_total 1 - - # HELP cortex_compactor_group_compactions_failures_total Total number of failed group compactions. - # TYPE cortex_compactor_group_compactions_failures_total counter - cortex_compactor_group_compactions_failures_total 0 - - # HELP cortex_compactor_group_compactions_total Total number of group compaction attempts that resulted in new block(s). - # TYPE cortex_compactor_group_compactions_total counter - cortex_compactor_group_compactions_total 0 - - # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. - # TYPE cortex_compactor_blocks_marked_for_deletion_total counter - cortex_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 - cortex_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 - cortex_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 + # TYPE pyroscope_compactor_runs_started_total counter + # HELP pyroscope_compactor_runs_started_total Total number of compaction runs started. + pyroscope_compactor_runs_started_total 1 + + # TYPE pyroscope_compactor_runs_completed_total counter + # HELP pyroscope_compactor_runs_completed_total Total number of compaction runs successfully completed. + pyroscope_compactor_runs_completed_total 1 + + # TYPE pyroscope_compactor_runs_failed_total counter + # HELP pyroscope_compactor_runs_failed_total Total number of compaction runs failed. + pyroscope_compactor_runs_failed_total{reason="error"} 0 + pyroscope_compactor_runs_failed_total{reason="shutdown"} 0 + + # HELP pyroscope_compactor_group_compaction_runs_completed_total Total number of group completed compaction runs. This also includes compactor group runs that resulted with no compaction. + # TYPE pyroscope_compactor_group_compaction_runs_completed_total counter + pyroscope_compactor_group_compaction_runs_completed_total 1 + + # HELP pyroscope_compactor_group_compaction_runs_started_total Total number of group compaction attempts. + # TYPE pyroscope_compactor_group_compaction_runs_started_total counter + pyroscope_compactor_group_compaction_runs_started_total 1 + + # HELP pyroscope_compactor_group_compactions_failures_total Total number of failed group compactions. + # TYPE pyroscope_compactor_group_compactions_failures_total counter + pyroscope_compactor_group_compactions_failures_total 0 + + # HELP pyroscope_compactor_group_compactions_total Total number of group compaction attempts that resulted in new block(s). + # TYPE pyroscope_compactor_group_compactions_total counter + pyroscope_compactor_group_compactions_total 0 + + # HELP pyroscope_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. + # TYPE pyroscope_compactor_blocks_marked_for_deletion_total counter + pyroscope_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 + pyroscope_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 + pyroscope_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 `), - "cortex_compactor_runs_started_total", - "cortex_compactor_runs_completed_total", - "cortex_compactor_runs_failed_total", - "cortex_compactor_group_compaction_runs_completed_total", - "cortex_compactor_group_compaction_runs_started_total", - "cortex_compactor_group_compactions_failures_total", - "cortex_compactor_group_compactions_total", - "cortex_compactor_blocks_marked_for_deletion_total", + "pyroscope_compactor_runs_started_total", + "pyroscope_compactor_runs_completed_total", + "pyroscope_compactor_runs_failed_total", + "pyroscope_compactor_group_compaction_runs_completed_total", + "pyroscope_compactor_group_compaction_runs_started_total", + "pyroscope_compactor_group_compactions_failures_total", + "pyroscope_compactor_group_compactions_total", + "pyroscope_compactor_blocks_marked_for_deletion_total", )) } @@ -1454,7 +1457,7 @@ func TestMultitenantCompactor_ShouldSkipCompactionForJobsWithFirstLevelCompactio t.Parallel() storageDir := t.TempDir() - bucketClient, err := filesystem.NewBucketClient(filesystem.Config{Directory: storageDir}) + bucketClient, err := filesystem.NewBucket(storageDir) require.NoError(t, err) // Mock two tenants, each with 2 overlapping blocks. @@ -1477,7 +1480,7 @@ func TestMultitenantCompactor_ShouldSkipCompactionForJobsWithFirstLevelCompactio // Mock the last modified timestamp returned for each of the block's meta.json. const waitPeriod = 10 * time.Minute - bucketClient = &bucketWithMockedAttributes{ + mockClient := &bucketWithMockedAttributes{ Bucket: bucketClient, customAttributes: map[string]objstore.ObjectAttributes{ path.Join("user-1", user1Meta1.ULID.String(), block.MetaFilename): {LastModified: time.Now().Add(-20 * time.Minute)}, @@ -1489,7 +1492,7 @@ func TestMultitenantCompactor_ShouldSkipCompactionForJobsWithFirstLevelCompactio cfg := prepareConfig(t) cfg.CompactionWaitPeriod = waitPeriod - c, _, tsdbPlanner, logs, registry := prepare(t, cfg, bucketClient) + c, _, tsdbPlanner, logs, registry := prepare(t, cfg, mockClient) // Mock the planner as if there's no compaction to do, in order to simplify tests. tsdbPlanner.On("Plan", mock.Anything, mock.Anything).Return([]*block.Meta{}, nil) @@ -1514,49 +1517,49 @@ func TestMultitenantCompactor_ShouldSkipCompactionForJobsWithFirstLevelCompactio fmt.Sprintf(`level=info component=compactor user=user-2 msg="skipping compaction job because blocks in this job were uploaded too recently (within wait period)" groupKey=0@17241709254077376921-merge--1574776800000-1574784000000 waitPeriodNotElapsedFor="%s (min time: 1574776800000, max time: 1574784000000)"`, user2Meta2.ULID.String())) assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` - # TYPE cortex_compactor_runs_started_total counter - # HELP cortex_compactor_runs_started_total Total number of compaction runs started. - cortex_compactor_runs_started_total 1 - - # TYPE cortex_compactor_runs_completed_total counter - # HELP cortex_compactor_runs_completed_total Total number of compaction runs successfully completed. - cortex_compactor_runs_completed_total 1 - - # TYPE cortex_compactor_runs_failed_total counter - # HELP cortex_compactor_runs_failed_total Total number of compaction runs failed. - cortex_compactor_runs_failed_total{reason="error"} 0 - cortex_compactor_runs_failed_total{reason="shutdown"} 0 - - # HELP cortex_compactor_group_compaction_runs_completed_total Total number of group completed compaction runs. This also includes compactor group runs that resulted with no compaction. - # TYPE cortex_compactor_group_compaction_runs_completed_total counter - cortex_compactor_group_compaction_runs_completed_total 1 - - # HELP cortex_compactor_group_compaction_runs_started_total Total number of group compaction attempts. - # TYPE cortex_compactor_group_compaction_runs_started_total counter - cortex_compactor_group_compaction_runs_started_total 1 - - # HELP cortex_compactor_group_compactions_failures_total Total number of failed group compactions. - # TYPE cortex_compactor_group_compactions_failures_total counter - cortex_compactor_group_compactions_failures_total 0 - - # HELP cortex_compactor_group_compactions_total Total number of group compaction attempts that resulted in new block(s). - # TYPE cortex_compactor_group_compactions_total counter - cortex_compactor_group_compactions_total 0 - - # HELP cortex_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. - # TYPE cortex_compactor_blocks_marked_for_deletion_total counter - cortex_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 - cortex_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 - cortex_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 + # TYPE pyroscope_compactor_runs_started_total counter + # HELP pyroscope_compactor_runs_started_total Total number of compaction runs started. + pyroscope_compactor_runs_started_total 1 + + # TYPE pyroscope_compactor_runs_completed_total counter + # HELP pyroscope_compactor_runs_completed_total Total number of compaction runs successfully completed. + pyroscope_compactor_runs_completed_total 1 + + # TYPE pyroscope_compactor_runs_failed_total counter + # HELP pyroscope_compactor_runs_failed_total Total number of compaction runs failed. + pyroscope_compactor_runs_failed_total{reason="error"} 0 + pyroscope_compactor_runs_failed_total{reason="shutdown"} 0 + + # HELP pyroscope_compactor_group_compaction_runs_completed_total Total number of group completed compaction runs. This also includes compactor group runs that resulted with no compaction. + # TYPE pyroscope_compactor_group_compaction_runs_completed_total counter + pyroscope_compactor_group_compaction_runs_completed_total 1 + + # HELP pyroscope_compactor_group_compaction_runs_started_total Total number of group compaction attempts. + # TYPE pyroscope_compactor_group_compaction_runs_started_total counter + pyroscope_compactor_group_compaction_runs_started_total 1 + + # HELP pyroscope_compactor_group_compactions_failures_total Total number of failed group compactions. + # TYPE pyroscope_compactor_group_compactions_failures_total counter + pyroscope_compactor_group_compactions_failures_total 0 + + # HELP pyroscope_compactor_group_compactions_total Total number of group compaction attempts that resulted in new block(s). + # TYPE pyroscope_compactor_group_compactions_total counter + pyroscope_compactor_group_compactions_total 0 + + # HELP pyroscope_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. + # TYPE pyroscope_compactor_blocks_marked_for_deletion_total counter + pyroscope_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 + pyroscope_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 + pyroscope_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 `), - "cortex_compactor_runs_started_total", - "cortex_compactor_runs_completed_total", - "cortex_compactor_runs_failed_total", - "cortex_compactor_group_compaction_runs_completed_total", - "cortex_compactor_group_compaction_runs_started_total", - "cortex_compactor_group_compactions_failures_total", - "cortex_compactor_group_compactions_total", - "cortex_compactor_blocks_marked_for_deletion_total", + "pyroscope_compactor_runs_started_total", + "pyroscope_compactor_runs_completed_total", + "pyroscope_compactor_runs_failed_total", + "pyroscope_compactor_group_compaction_runs_completed_total", + "pyroscope_compactor_group_compaction_runs_started_total", + "pyroscope_compactor_group_compactions_failures_total", + "pyroscope_compactor_group_compactions_total", + "pyroscope_compactor_blocks_marked_for_deletion_total", )) } @@ -1858,19 +1861,16 @@ func mockBlockMetaJSONWithTimeRangeAndLabels(id string, mint, maxt int64, lbls m func blockMeta(id string, mint, maxt int64, lbls map[string]string) *block.Meta { return &block.Meta{ - BlockMeta: tsdb.BlockMeta{ - Version: 1, - ULID: ulid.MustParse(id), - MinTime: mint, - MaxTime: maxt, - Compaction: tsdb.BlockMetaCompaction{ - Level: 1, - Sources: []ulid.ULID{ulid.MustParse(id)}, - }, - }, - Thanos: block.ThanosMeta{ - Labels: lbls, + Version: 1, + ULID: ulid.MustParse(id), + MinTime: model.Time(mint), + MaxTime: model.Time(maxt), + Compaction: tsdb.BlockMetaCompaction{ + Level: 1, + Sources: []ulid.ULID{ulid.MustParse(id)}, }, + + Labels: lbls, } } @@ -1983,7 +1983,7 @@ func TestMultitenantCompactor_ShouldFailCompactionOnTimeout(t *testing.T) { t.Parallel() // Mock the bucket - bucketClient := &bucket.ClientMock{} + bucketClient := &pyroscope_objstore.ClientMock{} bucketClient.MockIter("", []string{}, nil) ringStore, closer := consul.NewInMemoryClient(ring.GetCodec(), log.NewNopLogger(), nil) From 66b0b37b8192fadeacd0849877bb206c43fb4f35 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Thu, 14 Sep 2023 14:52:34 +0200 Subject: [PATCH 36/74] Add limits for the compactor --- pkg/validation/limits.go | 71 ++++++++++++++++++++++++++++++++++++++++ 1 file changed, 71 insertions(+) diff --git a/pkg/validation/limits.go b/pkg/validation/limits.go index 8713af7ff4..bf8378196b 100644 --- a/pkg/validation/limits.go +++ b/pkg/validation/limits.go @@ -3,8 +3,10 @@ package validation import ( "encoding/json" "flag" + "fmt" "time" + "github.com/grafana/pyroscope/pkg/phlaredb/block" "github.com/pkg/errors" "github.com/prometheus/common/model" "gopkg.in/yaml.v3" @@ -56,6 +58,19 @@ type Limits struct { // Query frontend. QuerySplitDuration model.Duration `yaml:"split_queries_by_interval" json:"split_queries_by_interval"` + + // Compactor. + CompactorBlocksRetentionPeriod model.Duration `yaml:"compactor_blocks_retention_period" json:"compactor_blocks_retention_period"` + CompactorSplitAndMergeShards int `yaml:"compactor_split_and_merge_shards" json:"compactor_split_and_merge_shards"` + CompactorSplitGroups int `yaml:"compactor_split_groups" json:"compactor_split_groups"` + CompactorTenantShardSize int `yaml:"compactor_tenant_shard_size" json:"compactor_tenant_shard_size"` + CompactorPartialBlockDeletionDelay model.Duration `yaml:"compactor_partial_block_deletion_delay" json:"compactor_partial_block_deletion_delay"` + + // This config doesn't have a CLI flag registered here because they're registered in + // their own original config struct. + S3SSEType string `yaml:"s3_sse_type" json:"s3_sse_type" doc:"nocli|description=S3 server-side encryption type. Required to enable server-side encryption overrides for a specific tenant. If not set, the default S3 client settings are used."` + S3SSEKMSKeyID string `yaml:"s3_sse_kms_key_id" json:"s3_sse_kms_key_id" doc:"nocli|description=S3 server-side encryption KMS Key ID. Ignored if the SSE type override is not set."` + S3SSEKMSEncryptionContext string `yaml:"s3_sse_kms_encryption_context" json:"s3_sse_kms_encryption_context" doc:"nocli|description=S3 server-side encryption KMS encryption context. If unset and the key ID override is set, the encryption context will not be provided to S3. Ignored if the SSE type override is not set."` } // LimitError are errors that do not comply with the limits specified. @@ -97,6 +112,13 @@ func (l *Limits) RegisterFlags(f *flag.FlagSet) { f.IntVar(&l.MaxProfileStacktraceSampleLabels, "validation.max-profile-stacktrace-sample-labels", 100, "Maximum number of labels in a profile sample. 0 to disable.") f.IntVar(&l.MaxProfileStacktraceDepth, "validation.max-profile-stacktrace-depth", 1000, "Maximum depth of a profile stacktrace. Profiles are not rejected instead stacktraces are truncated. 0 to disable.") f.IntVar(&l.MaxProfileSymbolValueLength, "validation.max-profile-symbol-value-length", 65535, "Maximum length of a profile symbol value (labels, function names and filenames, etc...). Profiles are not rejected instead symbol values are truncated. 0 to disable.") + + f.Var(&l.CompactorBlocksRetentionPeriod, "compactor.blocks-retention-period", "Delete blocks containing samples older than the specified retention period. 0 to disable.") + f.IntVar(&l.CompactorSplitAndMergeShards, "compactor.split-and-merge-shards", 0, "The number of shards to use when splitting blocks. 0 to disable splitting.") + f.IntVar(&l.CompactorSplitGroups, "compactor.split-groups", 1, "Number of groups that blocks for splitting should be grouped into. Each group of blocks is then split separately. Number of output split shards is controlled by -compactor.split-and-merge-shards.") + f.IntVar(&l.CompactorTenantShardSize, "compactor.compactor-tenant-shard-size", 0, "Max number of compactors that can compact blocks for single tenant. 0 to disable the limit and use all compactors.") + _ = l.CompactorPartialBlockDeletionDelay.Set("1d") + f.Var(&l.CompactorPartialBlockDeletionDelay, "compactor.partial-block-deletion-delay", fmt.Sprintf("If a partial block (unfinished block without %s file) hasn't been modified for this time, it will be marked for deletion. The minimum accepted value is %s: a lower value will be ignored and the feature disabled. 0 to disable.", block.MetaFilename, MinCompactorPartialBlockDeletionDelay.String())) } // UnmarshalYAML implements the yaml.Unmarshaler interface. @@ -261,6 +283,55 @@ func (o *Overrides) QuerySplitDuration(tenantID string) time.Duration { return time.Duration(o.getOverridesForTenant(tenantID).QuerySplitDuration) } +// CompactorTenantShardSize returns number of compactors that this user can use. 0 = all compactors. +func (o *Overrides) CompactorTenantShardSize(userID string) int { + return o.getOverridesForTenant(userID).CompactorTenantShardSize +} + +// CompactorBlocksRetentionPeriod returns the retention period for a given user. +func (o *Overrides) CompactorBlocksRetentionPeriod(userID string) time.Duration { + return time.Duration(o.getOverridesForTenant(userID).CompactorBlocksRetentionPeriod) +} + +// CompactorSplitAndMergeShards returns the number of shards to use when splitting blocks. +func (o *Overrides) CompactorSplitAndMergeShards(userID string) int { + return o.getOverridesForTenant(userID).CompactorSplitAndMergeShards +} + +// CompactorSplitGroups returns the number of groups that blocks for splitting should be grouped into. +func (o *Overrides) CompactorSplitGroups(userID string) int { + return o.getOverridesForTenant(userID).CompactorSplitGroups +} + +// CompactorPartialBlockDeletionDelay returns the partial block deletion delay time period for a given user, +// and whether the configured value was valid. If the value wasn't valid, the returned delay is the default one +// and the caller is responsible to warn the Mimir operator about it. +func (o *Overrides) CompactorPartialBlockDeletionDelay(userID string) (delay time.Duration, valid bool) { + delay = time.Duration(o.getOverridesForTenant(userID).CompactorPartialBlockDeletionDelay) + + // Forcefully disable partial blocks deletion if the configured delay is too low. + if delay > 0 && delay < MinCompactorPartialBlockDeletionDelay { + return 0, false + } + + return delay, true +} + +// S3SSEType returns the per-tenant S3 SSE type. +func (o *Overrides) S3SSEType(user string) string { + return o.getOverridesForTenant(user).S3SSEType +} + +// S3SSEKMSKeyID returns the per-tenant S3 KMS-SSE key id. +func (o *Overrides) S3SSEKMSKeyID(user string) string { + return o.getOverridesForTenant(user).S3SSEKMSKeyID +} + +// S3SSEKMSEncryptionContext returns the per-tenant S3 KMS-SSE encryption context. +func (o *Overrides) S3SSEKMSEncryptionContext(user string) string { + return o.getOverridesForTenant(user).S3SSEKMSEncryptionContext +} + // MaxQueriersPerTenant returns the limit to the number of queriers that can be used // Shuffle sharding will be used to distribute queries across queriers. // 0 means no limit. Currently disabled. From 53e514780176d4496da4109a00781fc4eaf9f8ef Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Thu, 14 Sep 2023 14:55:47 +0200 Subject: [PATCH 37/74] Removes unused limits and code --- pkg/compactor/compactor.go | 12 ------------ pkg/compactor/compactor_test.go | 3 --- 2 files changed, 15 deletions(-) diff --git a/pkg/compactor/compactor.go b/pkg/compactor/compactor.go index 8693e1a7ed..5e7aacb25d 100644 --- a/pkg/compactor/compactor.go +++ b/pkg/compactor/compactor.go @@ -193,18 +193,6 @@ type ConfigProvider interface { // and whether the configured value was valid. If the value wasn't valid, the returned delay is the default one // and the caller is responsible to warn the Mimir operator about it. CompactorPartialBlockDeletionDelay(userID string) (delay time.Duration, valid bool) - - // CompactorBlockUploadEnabled returns whether block upload is enabled for a given tenant. - CompactorBlockUploadEnabled(tenantID string) bool - - // CompactorBlockUploadValidationEnabled returns whether block upload validation is enabled for a given tenant. - CompactorBlockUploadValidationEnabled(tenantID string) bool - - // CompactorBlockUploadVerifyChunks returns whether chunk verification is enabled for a given tenant. - CompactorBlockUploadVerifyChunks(tenantID string) bool - - // CompactorBlockUploadMaxBlockSizeBytes returns the maximum size in bytes of a block that is allowed to be uploaded or validated for a given user. - CompactorBlockUploadMaxBlockSizeBytes(userID string) int64 } // MultitenantCompactor is a multi-tenant TSDB blocks compactor based on Thanos. diff --git a/pkg/compactor/compactor_test.go b/pkg/compactor/compactor_test.go index d41d093996..a00eeaae82 100644 --- a/pkg/compactor/compactor_test.go +++ b/pkg/compactor/compactor_test.go @@ -1744,9 +1744,6 @@ func prepare(t *testing.T, compactorCfg Config, bucketClient objstore.Bucket) (* } func prepareWithConfigProvider(t *testing.T, compactorCfg Config, bucketClient objstore.Bucket, limits ConfigProvider) (*MultitenantCompactor, *tsdbCompactorMock, *tsdbPlannerMock, *concurrency.SyncBuffer, prometheus.Gatherer) { - storageCfg := mimir_tsdb.BlocksStorageConfig{} - flagext.DefaultValues(&storageCfg) - // Create a temporary directory for compactor data. dataDir := t.TempDir() From 37df96118e8e2e29160986818c994a78e5088298 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Thu, 14 Sep 2023 15:53:32 +0200 Subject: [PATCH 38/74] Switches to Compactor interface for mocking --- pkg/compactor/bucket_compactor.go | 113 +++--- pkg/compactor/bucket_compactor_test.go | 6 +- pkg/compactor/compactor.go | 72 ++-- pkg/compactor/compactor_test.go | 465 ++++++++++++------------- pkg/compactor/split_merge_compactor.go | 7 + 5 files changed, 331 insertions(+), 332 deletions(-) diff --git a/pkg/compactor/bucket_compactor.go b/pkg/compactor/bucket_compactor.go index 56a88f932a..3c86a7f255 100644 --- a/pkg/compactor/bucket_compactor.go +++ b/pkg/compactor/bucket_compactor.go @@ -24,7 +24,6 @@ import ( "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" "github.com/prometheus/prometheus/model/labels" - "github.com/prometheus/prometheus/tsdb" "go.uber.org/atomic" "github.com/grafana/pyroscope/pkg/objstore" @@ -220,9 +219,63 @@ type Planner interface { // Compactor provides compaction against an underlying storage of profiling data. type Compactor interface { // CompactWithSplitting merges and splits the source blocks into shardCount number of compacted blocks, - // and returns slice of block IDs. Position of returned block ID in the result slice corresponds to the shard index. - // If given compacted block has no series, corresponding block ID will be zero ULID value. - CompactWithSplitting(dest string, dirs []string, open []*tsdb.Block, shardCount uint64) (result []ulid.ULID, _ error) + // and returns slice of block IDs. + // If given compacted block has no series, corresponding block ID will not be returned. + CompactWithSplitting(ctx context.Context, dst string, dirs []string, shardCount uint64) (result []ulid.ULID, _ error) +} + +type BlockCompactor struct { + blockOpenConcurrency int + logger log.Logger +} + +func (c *BlockCompactor) CompactWithSplitting(ctx context.Context, dest string, dirs []string, shardCount uint64) ([]ulid.ULID, error) { + localBucket, err := client.NewBucket(ctx, client.Config{ + StorageBackendConfig: client.StorageBackendConfig{ + Backend: client.Filesystem, + Filesystem: filesystem.Config{Directory: dest}, + }, + }, "local-compactor") + if err != nil { + return nil, errors.Wrap(err, "create local bucket") + } + defer localBucket.Close() + + readers := make([]phlaredb.BlockReader, len(dirs)) + defer func() { + for _, b := range readers { + if b != nil { + if err := b.Close(); err != nil { + level.Warn(c.logger).Log("msg", "failed to close block", "err", err) + } + } + } + }() + err = concurrency.ForEachJob(ctx, len(readers), c.blockOpenConcurrency, func(ctx context.Context, idx int) error { + dir := dirs[idx] + meta, err := block.ReadMetaFromDir(dir) + if err != nil { + return errors.Wrapf(err, "failed to read meta the block dir %s", dir) + } + b := phlaredb.NewSingleBlockQuerierFromMeta(ctx, localBucket, meta) + if err := b.Open(ctx); err != nil { + return errors.Wrapf(err, "open block %s", meta.ULID) + } + readers[idx] = b + return nil + }) + if err != nil { + return nil, err + } + metas, err := phlaredb.CompactWithSplitting(ctx, readers, shardCount, dest) + if err != nil { + return nil, errors.Wrapf(err, "compact blocks %v", dirs) + } + result := make([]ulid.ULID, len(metas)) + for i := range metas { + result[i] = metas[i].ULID + } + return result, nil } // runCompactionJob plans and runs a single compaction against the provided job. The compacted result @@ -291,59 +344,19 @@ func (c *BucketCompactor) runCompactionJob(ctx context.Context, job *Job) (shoul for ix, meta := range toCompact { blocksToCompactDirs[ix] = filepath.Join(subDir, meta.ULID.String()) } - - compactionBegin := time.Now() - - // todo: move this to a separate function. - // Move to a compactor interface - localBucket, err := client.NewBucket(ctx, client.Config{ - StorageBackendConfig: client.StorageBackendConfig{ - Backend: client.Filesystem, - Filesystem: filesystem.Config{Directory: subDir}, - }, - }, "local-compactor") - if err != nil { - return false, nil, errors.Wrap(err, "create local bucket") - } - defer localBucket.Close() - - src := make([]phlaredb.BlockReader, len(toCompact)) - defer func() { - for _, b := range src { - if b != nil { - if err := b.Close(); err != nil { - level.Warn(jobLogger).Log("msg", "failed to close block", "err", err) - } - } - } - }() - err = concurrency.ForEachJob(ctx, len(src), c.blockOpenConcurrency, func(ctx context.Context, idx int) error { - meta := toCompact[idx] - b := phlaredb.NewSingleBlockQuerierFromMeta(ctx, localBucket, meta) - if err := b.Open(ctx); err != nil { - return errors.Wrapf(err, "open block %s", meta.ULID) - } - src[idx] = b - return nil - }) - if err != nil { - return false, nil, err - } elapsed := time.Since(downloadBegin) level.Info(jobLogger).Log("msg", "downloaded and verified blocks; compacting blocks", "blocks", len(blocksToCompactDirs), "plan", fmt.Sprintf("%v", blocksToCompactDirs), "duration", elapsed, "duration_ms", elapsed.Milliseconds()) - var out []block.Meta + compactionBegin := time.Now() + if job.UseSplitting() { - out, err = phlaredb.CompactWithSplitting(ctx, src, uint64(job.SplittingShards()), subDir) + compIDs, err = c.comp.CompactWithSplitting(ctx, subDir, blocksToCompactDirs, uint64(job.SplittingShards())) } else { - out, err = phlaredb.CompactWithSplitting(ctx, src, 1, subDir) + compIDs, err = c.comp.CompactWithSplitting(ctx, subDir, blocksToCompactDirs, 1) } if err != nil { return false, nil, errors.Wrapf(err, "compact blocks %v", blocksToCompactDirs) } - for _, o := range out { - compIDs = append(compIDs, o.ULID) - } if !hasNonZeroULIDs(compIDs) { // Prometheus compactor found that the compacted block would have no samples. @@ -559,6 +572,7 @@ func NewBucketCompactor( sy *Syncer, grouper Grouper, planner Planner, + comp Compactor, compactDir string, bkt objstore.Bucket, concurrency int, @@ -566,7 +580,6 @@ func NewBucketCompactor( sortJobs JobsOrderFunc, waitPeriod time.Duration, blockSyncConcurrency int, - blockOpenConcurrency int, metrics *BucketCompactorMetrics, ) (*BucketCompactor, error) { if concurrency <= 0 { @@ -577,6 +590,7 @@ func NewBucketCompactor( sy: sy, grouper: grouper, planner: planner, + comp: comp, compactDir: compactDir, bkt: bkt, concurrency: concurrency, @@ -584,7 +598,6 @@ func NewBucketCompactor( sortJobs: sortJobs, waitPeriod: waitPeriod, blockSyncConcurrency: blockSyncConcurrency, - blockOpenConcurrency: blockOpenConcurrency, metrics: metrics, }, nil } diff --git a/pkg/compactor/bucket_compactor_test.go b/pkg/compactor/bucket_compactor_test.go index 5a3d02810c..0b3d1489e3 100644 --- a/pkg/compactor/bucket_compactor_test.go +++ b/pkg/compactor/bucket_compactor_test.go @@ -122,7 +122,7 @@ func TestBucketCompactor_FilterOwnJobs(t *testing.T) { m := NewBucketCompactorMetrics(promauto.With(nil).NewCounter(prometheus.CounterOpts{}), nil) for testName, testCase := range tests { t.Run(testName, func(t *testing.T) { - bc, err := NewBucketCompactor(log.NewNopLogger(), nil, nil, nil, "", nil, 2, testCase.ownJob, nil, 0, 1, 4, m) + bc, err := NewBucketCompactor(log.NewNopLogger(), nil, nil, nil, nil, "", nil, 0, nil, nil, 0, 4, m) require.NoError(t, err) res, err := bc.filterOwnJobs(jobsFn()) @@ -152,7 +152,7 @@ func TestBlockMaxTimeDeltas(t *testing.T) { metrics := NewBucketCompactorMetrics(promauto.With(nil).NewCounter(prometheus.CounterOpts{}), nil) now := time.UnixMilli(1500002900159) - bc, err := NewBucketCompactor(log.NewNopLogger(), nil, nil, nil, "", nil, 2, nil, nil, 0, 0, 4, metrics) + bc, err := NewBucketCompactor(log.NewNopLogger(), nil, nil, nil, nil, "", nil, 2, nil, nil, 0, 4, metrics) require.NoError(t, err) deltas := bc.blockMaxTimeDeltas(now, []*Job{j1, j2}) @@ -316,7 +316,7 @@ func TestCompactedBlocksTimeRangeVerification(t *testing.T) { compactedBlock1 := createDBBlock(t, bucketClient, "foo", testData.compactedBlockMinTime, testData.compactedBlockMinTime+500, 10, nil) compactedBlock2 := createDBBlock(t, bucketClient, "foo", testData.compactedBlockMaxTime-500, testData.compactedBlockMaxTime, 10, nil) - err := verifyCompactedBlocksTimeRanges([]ulid.ULID{compactedBlock1, compactedBlock2}, sourceMinTime, sourceMaxTime,filepath.Join(tempDir,"foo")) + err := verifyCompactedBlocksTimeRanges([]ulid.ULID{compactedBlock1, compactedBlock2}, sourceMinTime, sourceMaxTime, filepath.Join(tempDir, "foo")) if testData.shouldErr { require.ErrorContains(t, err, testData.expectedErrMsg) } else { diff --git a/pkg/compactor/compactor.go b/pkg/compactor/compactor.go index 5e7aacb25d..8094fbbdcc 100644 --- a/pkg/compactor/compactor.go +++ b/pkg/compactor/compactor.go @@ -69,6 +69,14 @@ type BlocksGrouperFactory func( reg prometheus.Registerer, ) Grouper +// BlocksCompactorFactory builds and returns the compactor and planner to use to compact a tenant's blocks. +type BlocksCompactorFactory func( + ctx context.Context, + cfg Config, + logger log.Logger, + reg prometheus.Registerer, +) (Compactor, Planner, error) + // Config holds the MultitenantCompactor config. type Config struct { BlockRanges DurationList `yaml:"block_ranges" category:"advanced"` @@ -87,10 +95,8 @@ type Config struct { NoBlocksFileCleanupEnabled bool `yaml:"no_blocks_file_cleanup_enabled" category:"experimental"` // Compactor concurrency options - MaxOpeningBlocksConcurrency int `yaml:"max_opening_blocks_concurrency" category:"advanced"` // Number of goroutines opening blocks before compaction. - MaxClosingBlocksConcurrency int `yaml:"max_closing_blocks_concurrency" category:"advanced"` // Max number of blocks that can be closed concurrently during split compaction. Note that closing of newly compacted block uses a lot of memory for writing index. - SymbolsFlushersConcurrency int `yaml:"symbols_flushers_concurrency" category:"advanced"` // Number of symbols flushers used when doing split compaction. - MaxBlockUploadValidationConcurrency int `yaml:"max_block_upload_validation_concurrency" category:"advanced"` // Max number of uploaded blocks that can be validated concurrently. + MaxOpeningBlocksConcurrency int `yaml:"max_opening_blocks_concurrency" category:"advanced"` // Number of goroutines opening blocks before compaction. + // MaxClosingBlocksConcurrency int `yaml:"max_closing_blocks_concurrency" category:"advanced"` // Max number of blocks that can be closed concurrently during split compaction. Note that closing of newly compacted block uses a lot of memory for writing index. EnabledTenants flagext.StringSliceCSV `yaml:"enabled_tenants" category:"advanced"` DisabledTenants flagext.StringSliceCSV `yaml:"disabled_tenants" category:"advanced"` @@ -107,7 +113,8 @@ type Config struct { retryMaxBackoff time.Duration `yaml:"-"` // Allow downstream projects to customise the blocks compactor. - BlocksGrouperFactory BlocksGrouperFactory `yaml:"-"` + BlocksGrouperFactory BlocksGrouperFactory `yaml:"-"` + BlocksCompactorFactory BlocksCompactorFactory `yaml:"-"` } // RegisterFlags registers the MultitenantCompactor flags. @@ -137,9 +144,7 @@ func (cfg *Config) RegisterFlags(f *flag.FlagSet, logger log.Logger) { f.BoolVar(&cfg.NoBlocksFileCleanupEnabled, "compactor.no-blocks-file-cleanup-enabled", false, "If enabled, will delete the bucket-index, markers and debug files in the tenant bucket when there are no blocks left in the index.") // compactor concurrency options f.IntVar(&cfg.MaxOpeningBlocksConcurrency, "compactor.max-opening-blocks-concurrency", 1, "Number of goroutines opening blocks before compaction.") - f.IntVar(&cfg.MaxClosingBlocksConcurrency, "compactor.max-closing-blocks-concurrency", 1, "Max number of blocks that can be closed concurrently during split compaction. Note that closing of newly compacted block uses a lot of memory for writing index.") - f.IntVar(&cfg.SymbolsFlushersConcurrency, "compactor.symbols-flushers-concurrency", 1, "Number of symbols flushers used when doing split compaction.") - f.IntVar(&cfg.MaxBlockUploadValidationConcurrency, "compactor.max-block-upload-validation-concurrency", 1, "Max number of uploaded blocks that can be validated concurrently. 0 = no limit.") + // f.IntVar(&cfg.MaxClosingBlocksConcurrency, "compactor.max-closing-blocks-concurrency", 1, "Max number of blocks that can be closed concurrently during split compaction. Note that closing of newly compacted block uses a lot of memory for writing index.") f.Var(&cfg.EnabledTenants, "compactor.enabled-tenants", "Comma separated list of tenants that can be compacted. If specified, only these tenants will be compacted by compactor, otherwise all tenants can be compacted. Subject to sharding.") f.Var(&cfg.DisabledTenants, "compactor.disabled-tenants", "Comma separated list of tenants that cannot be compacted by this compactor. If specified, and compactor would normally pick given tenant for compaction (via -compactor.enabled-tenants or sharding), it will be ignored instead.") @@ -156,15 +161,9 @@ func (cfg *Config) Validate() error { if cfg.MaxOpeningBlocksConcurrency < 1 { return errInvalidMaxOpeningBlocksConcurrency } - if cfg.MaxClosingBlocksConcurrency < 1 { - return errInvalidMaxClosingBlocksConcurrency - } - if cfg.SymbolsFlushersConcurrency < 1 { - return errInvalidSymbolFlushersConcurrency - } - if cfg.MaxBlockUploadValidationConcurrency < 0 { - return errInvalidMaxBlockUploadValidationConcurrency - } + // if cfg.MaxClosingBlocksConcurrency < 1 { + // return errInvalidMaxClosingBlocksConcurrency + // } if !util.StringsContain(CompactionOrders, cfg.CompactionJobsOrder) { return errInvalidCompactionOrder } @@ -207,13 +206,15 @@ type MultitenantCompactor struct { // Functions that creates bucket client, grouper, planner and compactor using the context. // Useful for injecting mock objects from tests. - blocksGrouperFactory BlocksGrouperFactory + blocksGrouperFactory BlocksGrouperFactory + blocksCompactorFactory BlocksCompactorFactory // Blocks cleaner is responsible to hard delete blocks marked for deletion. blocksCleaner *BlocksCleaner // Underlying compactor and planner used to compact TSDB blocks. - blocksPlanner Planner + blocksPlanner Planner + blocksCompactor Compactor // Client used to run operations on the bucket storing blocks. bucketClient objstore.Bucket @@ -256,13 +257,14 @@ type MultitenantCompactor struct { // NewMultitenantCompactor makes a new MultitenantCompactor. func NewMultitenantCompactor(compactorCfg Config, bucketClient objstore.Bucket, cfgProvider ConfigProvider, logger log.Logger, registerer prometheus.Registerer) (*MultitenantCompactor, error) { // Configure the compactor and grouper factories only if they weren't already set by a downstream project. - if compactorCfg.BlocksGrouperFactory == nil { + if compactorCfg.BlocksGrouperFactory == nil || compactorCfg.BlocksCompactorFactory == nil { configureSplitAndMergeCompactor(&compactorCfg) } blocksGrouperFactory := compactorCfg.BlocksGrouperFactory + blocksCompactorFactory := compactorCfg.BlocksCompactorFactory - mimirCompactor, err := newMultitenantCompactor(compactorCfg, bucketClient, cfgProvider, logger, registerer, blocksGrouperFactory) + mimirCompactor, err := newMultitenantCompactor(compactorCfg, bucketClient, cfgProvider, logger, registerer, blocksGrouperFactory, blocksCompactorFactory) if err != nil { return nil, errors.Wrap(err, "failed to create blocks compactor") } @@ -277,18 +279,18 @@ func newMultitenantCompactor( logger log.Logger, registerer prometheus.Registerer, blocksGrouperFactory BlocksGrouperFactory, + blocksCompactorFactory BlocksCompactorFactory, ) (*MultitenantCompactor, error) { c := &MultitenantCompactor{ - compactorCfg: compactorCfg, - cfgProvider: cfgProvider, - parentLogger: logger, - logger: log.With(logger, "component", "compactor"), - registerer: registerer, - syncerMetrics: newAggregatedSyncerMetrics(registerer), - bucketClient: bucketClient, - blocksGrouperFactory: blocksGrouperFactory, - blocksPlanner: NewSplitAndMergePlanner(compactorCfg.BlockRanges.ToMilliseconds()), - + compactorCfg: compactorCfg, + cfgProvider: cfgProvider, + parentLogger: logger, + logger: log.With(logger, "component", "compactor"), + registerer: registerer, + syncerMetrics: newAggregatedSyncerMetrics(registerer), + bucketClient: bucketClient, + blocksGrouperFactory: blocksGrouperFactory, + blocksCompactorFactory: blocksCompactorFactory, compactionRunsStarted: promauto.With(registerer).NewCounter(prometheus.CounterOpts{ Name: "pyroscope_compactor_runs_started_total", Help: "Total number of compaction runs started.", @@ -383,6 +385,12 @@ func newMultitenantCompactor( func (c *MultitenantCompactor) starting(ctx context.Context) error { var err error + // Create blocks compactor dependencies. + c.blocksCompactor, c.blocksPlanner, err = c.blocksCompactorFactory(ctx, c.compactorCfg, c.logger, c.registerer) + if err != nil { + return errors.Wrap(err, "failed to initialize compactor dependencies") + } + // Wrap the bucket client to write block deletion marks in the global location too. c.bucketClient = block.BucketWithGlobalMarkers(c.bucketClient) @@ -709,6 +717,7 @@ func (c *MultitenantCompactor) compactUser(ctx context.Context, userID string) e syncer, c.blocksGrouperFactory(ctx, c.compactorCfg, c.cfgProvider, userID, userLogger, reg), c.blocksPlanner, + c.blocksCompactor, path.Join(c.compactorCfg.DataDir, "compact"), userBucket, c.compactorCfg.CompactionConcurrency, @@ -716,7 +725,6 @@ func (c *MultitenantCompactor) compactUser(ctx context.Context, userID string) e c.jobsOrder, c.compactorCfg.CompactionWaitPeriod, c.compactorCfg.BlockSyncConcurrency, - c.compactorCfg.MaxOpeningBlocksConcurrency, c.bucketCompactorMetrics, ) if err != nil { diff --git a/pkg/compactor/compactor_test.go b/pkg/compactor/compactor_test.go index a00eeaae82..57c0f98266 100644 --- a/pkg/compactor/compactor_test.go +++ b/pkg/compactor/compactor_test.go @@ -33,10 +33,8 @@ import ( prom_testutil "github.com/prometheus/client_golang/prometheus/testutil" "github.com/prometheus/common/model" "github.com/prometheus/prometheus/model/histogram" - "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/tsdb" "github.com/prometheus/prometheus/tsdb/chunkenc" - "github.com/prometheus/prometheus/tsdb/chunks" "github.com/prometheus/prometheus/tsdb/tsdbutil" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/mock" @@ -45,11 +43,9 @@ import ( "gopkg.in/yaml.v3" pyroscope_objstore "github.com/grafana/pyroscope/pkg/objstore" - "github.com/grafana/pyroscope/pkg/objstore/providers/filesystem" "github.com/grafana/pyroscope/pkg/phlaredb/block" "github.com/grafana/pyroscope/pkg/phlaredb/block/testutil" "github.com/grafana/pyroscope/pkg/phlaredb/bucket" - "github.com/grafana/pyroscope/pkg/pprof/testhelper" "github.com/grafana/pyroscope/pkg/validation" ) @@ -123,14 +119,6 @@ func TestConfig_Validate(t *testing.T) { setup: func(cfg *Config) { cfg.MaxOpeningBlocksConcurrency = 0 }, expected: errInvalidMaxOpeningBlocksConcurrency.Error(), }, - "should fail on invalid value of max-closing-blocks-concurrency": { - setup: func(cfg *Config) { cfg.MaxClosingBlocksConcurrency = 0 }, - expected: errInvalidMaxClosingBlocksConcurrency.Error(), - }, - "should fail on invalid value of symbols-flushers-concurrency": { - setup: func(cfg *Config) { cfg.SymbolsFlushersConcurrency = 0 }, - expected: errInvalidSymbolFlushersConcurrency.Error(), - }, } for testName, testData := range tests { @@ -152,7 +140,7 @@ func TestMultitenantCompactor_ShouldDoNothingOnNoUserBlocks(t *testing.T) { t.Parallel() // No user blocks stored in the bucket. - bucketClient := &bucket.ClientMock{} + bucketClient := &pyroscope_objstore.ClientMock{} bucketClient.MockIter("", []string{}, nil) cfg := prepareConfig(t) c, _, _, logs, registry := prepare(t, cfg, bucketClient) @@ -1224,105 +1212,105 @@ func TestMultitenantCompactor_ShouldCompactOnlyUsersOwnedByTheInstanceOnSharding } } -func TestMultitenantCompactor_ShouldFailWithInvalidTSDBCompactOutput(t *testing.T) { - const user = "user-1" - - // Two blocks with overlapping time range - sourceBlock1Spec := []*block.SeriesSpec{ - { - Labels: labels.FromStrings("case", "source_spec_1"), - Chunks: []chunks.Meta{ - must(tsdbutil.ChunkFromSamples([]tsdbutil.Sample{ - newSample(1000, 1000, nil, nil), - newSample(2000, 2000, nil, nil), - })), - }, - }, - } - - sourceBlock2Spec := []*block.SeriesSpec{ - { - Labels: labels.FromStrings("case", "source_spec_2"), - Chunks: []chunks.Meta{ - must(tsdbutil.ChunkFromSamples([]tsdbutil.Sample{ - newSample(1500, 1500, nil, nil), - newSample(2500, 2500, nil, nil), - })), - }, - }, - } - - // Block with sufficient time range so compaction job gets triggered - sourceBlock3Spec := []*block.SeriesSpec{ - { - Labels: labels.FromStrings("case", "source_spec_3"), - Chunks: []chunks.Meta{ - must(tsdbutil.ChunkFromSamples([]tsdbutil.Sample{ - newSample(0, 0, nil, nil), - newSample(2*time.Hour.Milliseconds()-1, 0, nil, nil), - })), - }, - }, - } - - // Compacted block not containing minTime/maxTime from source blocks - compactedBlockSpec := []*block.SeriesSpec{ - { - Labels: labels.FromStrings("case", "compacted_spec"), - Chunks: []chunks.Meta{ - must(tsdbutil.ChunkFromSamples([]tsdbutil.Sample{ - newSample(1250, 1250, nil, nil), - newSample(2250, 2250, nil, nil), - })), - }, - }, - } - - storageDir := t.TempDir() - - meta1, err := block.GenerateBlockFromSpec(user, filepath.Join(storageDir, user), sourceBlock1Spec) - require.NoError(t, err) - meta2, err := block.GenerateBlockFromSpec(user, filepath.Join(storageDir, user), sourceBlock2Spec) - require.NoError(t, err) - _, err = block.GenerateBlockFromSpec(user, filepath.Join(storageDir, user), sourceBlock3Spec) - require.NoError(t, err) - - bkt, err := filesystem.NewBucket(storageDir) - require.NoError(t, err) - - cfg := prepareConfig(t) - cfg.CompactionRetries = 1 // No need to retry as we're testing for failure - c, tsdbCompactor, tsdbPlanner, logs, _ := prepare(t, cfg, bkt) - - tsdbPlanner.On("Plan", mock.Anything, mock.Anything).Return([]*block.Meta{meta1, meta2}, nil).Once() - tsdbPlanner.On("Plan", mock.Anything, mock.Anything).Return([]*block.Meta{}, nil).Once() - mockCall := tsdbCompactor.On("Compact", mock.Anything, mock.Anything, mock.Anything) - mockCall.RunFn = func(args mock.Arguments) { - dir := args.Get(0).(string) - - compactedMeta, err := block.GenerateBlockFromSpec(user, dir, compactedBlockSpec) - require.NoError(t, err) - f, err := os.OpenFile(filepath.Join(dir, compactedMeta.ULID.String(), "tombstones"), os.O_RDONLY|os.O_CREATE, 0o666) - require.NoError(t, err) - defer f.Close() - - mockCall.ReturnArguments = mock.Arguments{compactedMeta.ULID, nil} - } - - // Start the compactor - require.NoError(t, services.StartAndAwaitRunning(context.Background(), c)) - - // Compaction block verification should fail due to invalid output block - test.Poll(t, 5*time.Second, 1.0, func() interface{} { - return prom_testutil.ToFloat64(c.bucketCompactorMetrics.compactionBlocksVerificationFailed) - }) - - // Stop the compactor. - require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) - - // Check logs for compacted block verification failure - assert.Contains(t, logs.String(), "compacted block(s) do not contain minTime 1000 and maxTime 2501 from the source blocks") -} +// func TestMultitenantCompactor_ShouldFailWithInvalidTSDBCompactOutput(t *testing.T) { +// const user = "user-1" + +// // Two blocks with overlapping time range +// sourceBlock1Spec := []*block.SeriesSpec{ +// { +// Labels: labels.FromStrings("case", "source_spec_1"), +// Chunks: []chunks.Meta{ +// must(tsdbutil.ChunkFromSamples([]tsdbutil.Sample{ +// newSample(1000, 1000, nil, nil), +// newSample(2000, 2000, nil, nil), +// })), +// }, +// }, +// } + +// sourceBlock2Spec := []*block.SeriesSpec{ +// { +// Labels: labels.FromStrings("case", "source_spec_2"), +// Chunks: []chunks.Meta{ +// must(tsdbutil.ChunkFromSamples([]tsdbutil.Sample{ +// newSample(1500, 1500, nil, nil), +// newSample(2500, 2500, nil, nil), +// })), +// }, +// }, +// } + +// // Block with sufficient time range so compaction job gets triggered +// sourceBlock3Spec := []*block.SeriesSpec{ +// { +// Labels: labels.FromStrings("case", "source_spec_3"), +// Chunks: []chunks.Meta{ +// must(tsdbutil.ChunkFromSamples([]tsdbutil.Sample{ +// newSample(0, 0, nil, nil), +// newSample(2*time.Hour.Milliseconds()-1, 0, nil, nil), +// })), +// }, +// }, +// } + +// // Compacted block not containing minTime/maxTime from source blocks +// compactedBlockSpec := []*block.SeriesSpec{ +// { +// Labels: labels.FromStrings("case", "compacted_spec"), +// Chunks: []chunks.Meta{ +// must(tsdbutil.ChunkFromSamples([]tsdbutil.Sample{ +// newSample(1250, 1250, nil, nil), +// newSample(2250, 2250, nil, nil), +// })), +// }, +// }, +// } + +// storageDir := t.TempDir() + +// meta1, err := block.GenerateBlockFromSpec(user, filepath.Join(storageDir, user), sourceBlock1Spec) +// require.NoError(t, err) +// meta2, err := block.GenerateBlockFromSpec(user, filepath.Join(storageDir, user), sourceBlock2Spec) +// require.NoError(t, err) +// _, err = block.GenerateBlockFromSpec(user, filepath.Join(storageDir, user), sourceBlock3Spec) +// require.NoError(t, err) + +// bkt, err := filesystem.NewBucket(storageDir) +// require.NoError(t, err) + +// cfg := prepareConfig(t) +// cfg.CompactionRetries = 1 // No need to retry as we're testing for failure +// c, tsdbCompactor, tsdbPlanner, logs, _ := prepare(t, cfg, bkt) + +// tsdbPlanner.On("Plan", mock.Anything, mock.Anything).Return([]*block.Meta{meta1, meta2}, nil).Once() +// tsdbPlanner.On("Plan", mock.Anything, mock.Anything).Return([]*block.Meta{}, nil).Once() +// mockCall := tsdbCompactor.On("Compact", mock.Anything, mock.Anything, mock.Anything) +// mockCall.RunFn = func(args mock.Arguments) { +// dir := args.Get(0).(string) + +// compactedMeta, err := block.GenerateBlockFromSpec(user, dir, compactedBlockSpec) +// require.NoError(t, err) +// f, err := os.OpenFile(filepath.Join(dir, compactedMeta.ULID.String(), "tombstones"), os.O_RDONLY|os.O_CREATE, 0o666) +// require.NoError(t, err) +// defer f.Close() + +// mockCall.ReturnArguments = mock.Arguments{compactedMeta.ULID, nil} +// } + +// // Start the compactor +// require.NoError(t, services.StartAndAwaitRunning(context.Background(), c)) + +// // Compaction block verification should fail due to invalid output block +// test.Poll(t, 5*time.Second, 1.0, func() interface{} { +// return prom_testutil.ToFloat64(c.bucketCompactorMetrics.compactionBlocksVerificationFailed) +// }) + +// // Stop the compactor. +// require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) + +// // Check logs for compacted block verification failure +// assert.Contains(t, logs.String(), "compacted block(s) do not contain minTime 1000 and maxTime 2501 from the source blocks") +// } func TestMultitenantCompactor_ShouldSkipCompactionForJobsNoMoreOwnedAfterPlanning(t *testing.T) { t.Parallel() @@ -1453,115 +1441,115 @@ func TestMultitenantCompactor_ShouldSkipCompactionForJobsNoMoreOwnedAfterPlannin )) } -func TestMultitenantCompactor_ShouldSkipCompactionForJobsWithFirstLevelCompactionBlocksAndWaitPeriodNotElapsed(t *testing.T) { - t.Parallel() - - storageDir := t.TempDir() - bucketClient, err := filesystem.NewBucket(storageDir) - require.NoError(t, err) - - // Mock two tenants, each with 2 overlapping blocks. - spec := []*block.SeriesSpec{{ - Labels: labels.FromStrings(labels.MetricName, "series_1"), - Chunks: []chunks.Meta{must(tsdbutil.ChunkFromSamples([]tsdbutil.Sample{ - newSample(1574776800000, 0, nil, nil), - newSample(1574783999999, 0, nil, nil), - }))}, - }} - - user1Meta1, err := block.GenerateBlockFromSpec("user-1", filepath.Join(storageDir, "user-1"), spec) - require.NoError(t, err) - user1Meta2, err := block.GenerateBlockFromSpec("user-1", filepath.Join(storageDir, "user-1"), spec) - require.NoError(t, err) - user2Meta1, err := block.GenerateBlockFromSpec("user-2", filepath.Join(storageDir, "user-2"), spec) - require.NoError(t, err) - user2Meta2, err := block.GenerateBlockFromSpec("user-2", filepath.Join(storageDir, "user-2"), spec) - require.NoError(t, err) - - // Mock the last modified timestamp returned for each of the block's meta.json. - const waitPeriod = 10 * time.Minute - mockClient := &bucketWithMockedAttributes{ - Bucket: bucketClient, - customAttributes: map[string]objstore.ObjectAttributes{ - path.Join("user-1", user1Meta1.ULID.String(), block.MetaFilename): {LastModified: time.Now().Add(-20 * time.Minute)}, - path.Join("user-1", user1Meta2.ULID.String(), block.MetaFilename): {LastModified: time.Now().Add(-20 * time.Minute)}, - path.Join("user-2", user2Meta1.ULID.String(), block.MetaFilename): {LastModified: time.Now().Add(-20 * time.Minute)}, - path.Join("user-2", user2Meta2.ULID.String(), block.MetaFilename): {LastModified: time.Now().Add(-5 * time.Minute)}, - }, - } - - cfg := prepareConfig(t) - cfg.CompactionWaitPeriod = waitPeriod - c, _, tsdbPlanner, logs, registry := prepare(t, cfg, mockClient) - - // Mock the planner as if there's no compaction to do, in order to simplify tests. - tsdbPlanner.On("Plan", mock.Anything, mock.Anything).Return([]*block.Meta{}, nil) - - require.NoError(t, services.StartAndAwaitRunning(context.Background(), c)) - - // Compactor doesn't wait for blocks cleaner to finish, but our test checks for cleaner metrics. - require.NoError(t, c.blocksCleaner.AwaitRunning(context.Background())) - - // Wait until a run has completed. - test.Poll(t, 5*time.Second, 1.0, func() interface{} { - return prom_testutil.ToFloat64(c.compactionRunsCompleted) - }) - - require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) - - // We expect only 1 compaction job has been expected, while the 2nd has been skipped. - tsdbPlanner.AssertNumberOfCalls(t, "Plan", 1) - - // Ensure the skipped compaction job is the expected one. - assert.Contains(t, strings.Split(strings.TrimSpace(logs.String()), "\n"), - fmt.Sprintf(`level=info component=compactor user=user-2 msg="skipping compaction job because blocks in this job were uploaded too recently (within wait period)" groupKey=0@17241709254077376921-merge--1574776800000-1574784000000 waitPeriodNotElapsedFor="%s (min time: 1574776800000, max time: 1574784000000)"`, user2Meta2.ULID.String())) - - assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` - # TYPE pyroscope_compactor_runs_started_total counter - # HELP pyroscope_compactor_runs_started_total Total number of compaction runs started. - pyroscope_compactor_runs_started_total 1 - - # TYPE pyroscope_compactor_runs_completed_total counter - # HELP pyroscope_compactor_runs_completed_total Total number of compaction runs successfully completed. - pyroscope_compactor_runs_completed_total 1 - - # TYPE pyroscope_compactor_runs_failed_total counter - # HELP pyroscope_compactor_runs_failed_total Total number of compaction runs failed. - pyroscope_compactor_runs_failed_total{reason="error"} 0 - pyroscope_compactor_runs_failed_total{reason="shutdown"} 0 - - # HELP pyroscope_compactor_group_compaction_runs_completed_total Total number of group completed compaction runs. This also includes compactor group runs that resulted with no compaction. - # TYPE pyroscope_compactor_group_compaction_runs_completed_total counter - pyroscope_compactor_group_compaction_runs_completed_total 1 - - # HELP pyroscope_compactor_group_compaction_runs_started_total Total number of group compaction attempts. - # TYPE pyroscope_compactor_group_compaction_runs_started_total counter - pyroscope_compactor_group_compaction_runs_started_total 1 - - # HELP pyroscope_compactor_group_compactions_failures_total Total number of failed group compactions. - # TYPE pyroscope_compactor_group_compactions_failures_total counter - pyroscope_compactor_group_compactions_failures_total 0 - - # HELP pyroscope_compactor_group_compactions_total Total number of group compaction attempts that resulted in new block(s). - # TYPE pyroscope_compactor_group_compactions_total counter - pyroscope_compactor_group_compactions_total 0 - - # HELP pyroscope_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. - # TYPE pyroscope_compactor_blocks_marked_for_deletion_total counter - pyroscope_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 - pyroscope_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 - pyroscope_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 - `), - "pyroscope_compactor_runs_started_total", - "pyroscope_compactor_runs_completed_total", - "pyroscope_compactor_runs_failed_total", - "pyroscope_compactor_group_compaction_runs_completed_total", - "pyroscope_compactor_group_compaction_runs_started_total", - "pyroscope_compactor_group_compactions_failures_total", - "pyroscope_compactor_group_compactions_total", - "pyroscope_compactor_blocks_marked_for_deletion_total", - )) -} +// func TestMultitenantCompactor_ShouldSkipCompactionForJobsWithFirstLevelCompactionBlocksAndWaitPeriodNotElapsed(t *testing.T) { +// t.Parallel() + +// storageDir := t.TempDir() +// bucketClient, err := filesystem.NewBucket(storageDir) +// require.NoError(t, err) + +// // Mock two tenants, each with 2 overlapping blocks. +// spec := []*block.SeriesSpec{{ +// Labels: labels.FromStrings(labels.MetricName, "series_1"), +// Chunks: []chunks.Meta{must(tsdbutil.ChunkFromSamples([]tsdbutil.Sample{ +// newSample(1574776800000, 0, nil, nil), +// newSample(1574783999999, 0, nil, nil), +// }))}, +// }} + +// user1Meta1, err := block.GenerateBlockFromSpec("user-1", filepath.Join(storageDir, "user-1"), spec) +// require.NoError(t, err) +// user1Meta2, err := block.GenerateBlockFromSpec("user-1", filepath.Join(storageDir, "user-1"), spec) +// require.NoError(t, err) +// user2Meta1, err := block.GenerateBlockFromSpec("user-2", filepath.Join(storageDir, "user-2"), spec) +// require.NoError(t, err) +// user2Meta2, err := block.GenerateBlockFromSpec("user-2", filepath.Join(storageDir, "user-2"), spec) +// require.NoError(t, err) + +// // Mock the last modified timestamp returned for each of the block's meta.json. +// const waitPeriod = 10 * time.Minute +// mockClient := &bucketWithMockedAttributes{ +// Bucket: bucketClient, +// customAttributes: map[string]objstore.ObjectAttributes{ +// path.Join("user-1", user1Meta1.ULID.String(), block.MetaFilename): {LastModified: time.Now().Add(-20 * time.Minute)}, +// path.Join("user-1", user1Meta2.ULID.String(), block.MetaFilename): {LastModified: time.Now().Add(-20 * time.Minute)}, +// path.Join("user-2", user2Meta1.ULID.String(), block.MetaFilename): {LastModified: time.Now().Add(-20 * time.Minute)}, +// path.Join("user-2", user2Meta2.ULID.String(), block.MetaFilename): {LastModified: time.Now().Add(-5 * time.Minute)}, +// }, +// } + +// cfg := prepareConfig(t) +// cfg.CompactionWaitPeriod = waitPeriod +// c, _, tsdbPlanner, logs, registry := prepare(t, cfg, mockClient) + +// // Mock the planner as if there's no compaction to do, in order to simplify tests. +// tsdbPlanner.On("Plan", mock.Anything, mock.Anything).Return([]*block.Meta{}, nil) + +// require.NoError(t, services.StartAndAwaitRunning(context.Background(), c)) + +// // Compactor doesn't wait for blocks cleaner to finish, but our test checks for cleaner metrics. +// require.NoError(t, c.blocksCleaner.AwaitRunning(context.Background())) + +// // Wait until a run has completed. +// test.Poll(t, 5*time.Second, 1.0, func() interface{} { +// return prom_testutil.ToFloat64(c.compactionRunsCompleted) +// }) + +// require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) + +// // We expect only 1 compaction job has been expected, while the 2nd has been skipped. +// tsdbPlanner.AssertNumberOfCalls(t, "Plan", 1) + +// // Ensure the skipped compaction job is the expected one. +// assert.Contains(t, strings.Split(strings.TrimSpace(logs.String()), "\n"), +// fmt.Sprintf(`level=info component=compactor user=user-2 msg="skipping compaction job because blocks in this job were uploaded too recently (within wait period)" groupKey=0@17241709254077376921-merge--1574776800000-1574784000000 waitPeriodNotElapsedFor="%s (min time: 1574776800000, max time: 1574784000000)"`, user2Meta2.ULID.String())) + +// assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` +// # TYPE pyroscope_compactor_runs_started_total counter +// # HELP pyroscope_compactor_runs_started_total Total number of compaction runs started. +// pyroscope_compactor_runs_started_total 1 + +// # TYPE pyroscope_compactor_runs_completed_total counter +// # HELP pyroscope_compactor_runs_completed_total Total number of compaction runs successfully completed. +// pyroscope_compactor_runs_completed_total 1 + +// # TYPE pyroscope_compactor_runs_failed_total counter +// # HELP pyroscope_compactor_runs_failed_total Total number of compaction runs failed. +// pyroscope_compactor_runs_failed_total{reason="error"} 0 +// pyroscope_compactor_runs_failed_total{reason="shutdown"} 0 + +// # HELP pyroscope_compactor_group_compaction_runs_completed_total Total number of group completed compaction runs. This also includes compactor group runs that resulted with no compaction. +// # TYPE pyroscope_compactor_group_compaction_runs_completed_total counter +// pyroscope_compactor_group_compaction_runs_completed_total 1 + +// # HELP pyroscope_compactor_group_compaction_runs_started_total Total number of group compaction attempts. +// # TYPE pyroscope_compactor_group_compaction_runs_started_total counter +// pyroscope_compactor_group_compaction_runs_started_total 1 + +// # HELP pyroscope_compactor_group_compactions_failures_total Total number of failed group compactions. +// # TYPE pyroscope_compactor_group_compactions_failures_total counter +// pyroscope_compactor_group_compactions_failures_total 0 + +// # HELP pyroscope_compactor_group_compactions_total Total number of group compaction attempts that resulted in new block(s). +// # TYPE pyroscope_compactor_group_compactions_total counter +// pyroscope_compactor_group_compactions_total 0 + +// # HELP pyroscope_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. +// # TYPE pyroscope_compactor_blocks_marked_for_deletion_total counter +// pyroscope_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 +// pyroscope_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 +// pyroscope_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 +// `), +// "pyroscope_compactor_runs_started_total", +// "pyroscope_compactor_runs_completed_total", +// "pyroscope_compactor_runs_failed_total", +// "pyroscope_compactor_group_compaction_runs_completed_total", +// "pyroscope_compactor_group_compaction_runs_started_total", +// "pyroscope_compactor_group_compactions_failures_total", +// "pyroscope_compactor_group_compactions_total", +// "pyroscope_compactor_blocks_marked_for_deletion_total", +// )) +// } func createCustomBlock(t *testing.T, bkt objstore.Bucket, userID string, externalLabels map[string]string, generator func() []*testhelper.ProfileBuilder) ulid.ULID { meta, dir := testutil.CreateBlock(t, generator) @@ -1711,9 +1699,7 @@ func prepareConfig(t *testing.T) Config { compactorCfg.retryMaxBackoff = 0 // Use settings that ensure things will be done concurrently, verifying ordering assumptions. - // Helps to expose bugs such as https://github.com/prometheus/prometheus/pull/10108 compactorCfg.MaxOpeningBlocksConcurrency = 3 - compactorCfg.MaxClosingBlocksConcurrency = 3 // Do not wait for ring stability by default, in order to speed up tests. compactorCfg.ShardingRing.WaitStabilityMinDuration = 0 @@ -1734,7 +1720,7 @@ func prepareConfig(t *testing.T) Config { return compactorCfg } -func prepare(t *testing.T, compactorCfg Config, bucketClient objstore.Bucket) (*MultitenantCompactor, *tsdbCompactorMock, *tsdbPlannerMock, *concurrency.SyncBuffer, prometheus.Gatherer) { +func prepare(t *testing.T, compactorCfg Config, bucketClient objstore.Bucket) (*MultitenantCompactor, *blockCompactorMock, *tsdbPlannerMock, *concurrency.SyncBuffer, prometheus.Gatherer) { var limits validation.Limits flagext.DefaultValues(&limits) overrides, err := validation.NewOverrides(limits, nil) @@ -1743,26 +1729,26 @@ func prepare(t *testing.T, compactorCfg Config, bucketClient objstore.Bucket) (* return prepareWithConfigProvider(t, compactorCfg, bucketClient, overrides) } -func prepareWithConfigProvider(t *testing.T, compactorCfg Config, bucketClient objstore.Bucket, limits ConfigProvider) (*MultitenantCompactor, *tsdbCompactorMock, *tsdbPlannerMock, *concurrency.SyncBuffer, prometheus.Gatherer) { +func prepareWithConfigProvider(t *testing.T, compactorCfg Config, bucketClient objstore.Bucket, limits ConfigProvider) (*MultitenantCompactor, *blockCompactorMock, *tsdbPlannerMock, *concurrency.SyncBuffer, prometheus.Gatherer) { // Create a temporary directory for compactor data. dataDir := t.TempDir() compactorCfg.DataDir = dataDir - tsdbCompactor := &tsdbCompactorMock{} + blockCompactor := &blockCompactorMock{} tsdbPlanner := &tsdbPlannerMock{} logs := &concurrency.SyncBuffer{} logger := &componentLogger{component: "compactor", log: log.NewLogfmtLogger(logs)} registry := prometheus.NewRegistry() blocksCompactorFactory := func(ctx context.Context, cfg Config, logger log.Logger, reg prometheus.Registerer) (Compactor, Planner, error) { - return tsdbCompactor, tsdbPlanner, nil + return blockCompactor, tsdbPlanner, nil } - c, err := newMultitenantCompactor(compactorCfg, bucketClient, limits, logger, registry, splitAndMergeGrouperFactory, blocksCompactorFactory) + c, err := newMultitenantCompactor(compactorCfg, pyroscope_objstore.NewBucket(bucketClient), limits, logger, registry, splitAndMergeGrouperFactory, blocksCompactorFactory) require.NoError(t, err) - return c, tsdbCompactor, tsdbPlanner, logs, registry + return c, blockCompactor, tsdbPlanner, logs, registry } type componentLogger struct { @@ -1807,27 +1793,12 @@ func (c *componentLogger) Log(keyvals ...interface{}) error { return nil } -type tsdbCompactorMock struct { +type blockCompactorMock struct { mock.Mock } -func (m *tsdbCompactorMock) Plan(dir string) ([]string, error) { - args := m.Called(dir) - return args.Get(0).([]string), args.Error(1) -} - -func (m *tsdbCompactorMock) Write(dest string, b tsdb.BlockReader, mint, maxt int64, parent *tsdb.BlockMeta) (ulid.ULID, error) { - args := m.Called(dest, b, mint, maxt, parent) - return args.Get(0).(ulid.ULID), args.Error(1) -} - -func (m *tsdbCompactorMock) Compact(dest string, dirs []string, open []*tsdb.Block) (ulid.ULID, error) { - args := m.Called(dest, dirs, open) - return args.Get(0).(ulid.ULID), args.Error(1) -} - -func (m *tsdbCompactorMock) CompactWithSplitting(dest string, dirs []string, open []*tsdb.Block, shardCount uint64) (result []ulid.ULID, _ error) { - args := m.Called(dest, dirs, open, shardCount) +func (m *blockCompactorMock) CompactWithSplitting(ctx context.Context, dest string, dirs []string, shardCount uint64) (result []ulid.ULID, _ error) { + args := m.Called(ctx, dest, dirs, shardCount) return args.Get(0).([]ulid.ULID), args.Error(1) } diff --git a/pkg/compactor/split_merge_compactor.go b/pkg/compactor/split_merge_compactor.go index 1995ca2e7a..3975f36def 100644 --- a/pkg/compactor/split_merge_compactor.go +++ b/pkg/compactor/split_merge_compactor.go @@ -20,8 +20,15 @@ func splitAndMergeGrouperFactory(_ context.Context, cfg Config, cfgProvider Conf logger) } +func splitAndMergeCompactorFactory(ctx context.Context, cfg Config, logger log.Logger, reg prometheus.Registerer) (Compactor, Planner, error) { + return &BlockCompactor{ + blockOpenConcurrency: cfg.MaxOpeningBlocksConcurrency, + logger: logger, + }, NewSplitAndMergePlanner(cfg.BlockRanges.ToMilliseconds()), nil +} // configureSplitAndMergeCompactor updates the provided configuration injecting the split-and-merge compactor. func configureSplitAndMergeCompactor(cfg *Config) { cfg.BlocksGrouperFactory = splitAndMergeGrouperFactory + cfg.BlocksCompactorFactory = splitAndMergeCompactorFactory } From 091ee650b614ef3b6ed3e851af46fa9cc2ad2b42 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Thu, 14 Sep 2023 17:07:09 +0200 Subject: [PATCH 39/74] Fixes blocks_cleaner_test.go --- pkg/compactor/blocks_cleaner_test.go | 17 ++++++++++------- pkg/compactor/compactor.go | 13 +++++-------- pkg/compactor/compactor_test.go | 2 +- 3 files changed, 16 insertions(+), 16 deletions(-) diff --git a/pkg/compactor/blocks_cleaner_test.go b/pkg/compactor/blocks_cleaner_test.go index 8803d2e6c7..aaa78761c4 100644 --- a/pkg/compactor/blocks_cleaner_test.go +++ b/pkg/compactor/blocks_cleaner_test.go @@ -131,12 +131,12 @@ func testBlocksCleanerWithOptions(t *testing.T, options testBlocksCleanerOptions {path: path.Join("user-1", block5.String(), block.DeletionMarkFilename), expectedExists: false}, {path: path.Join("user-1", block.DeletionMarkFilepath(block5)), expectedExists: false}, // Should not delete a partial block without deletion mark. - {path: path.Join("user-1", block6.String(), "index"), expectedExists: true}, + {path: path.Join("user-1", block6.String(), block.IndexFilename), expectedExists: true}, // Should completely delete blocks for user-3, marked for deletion {path: path.Join("user-3", block9.String(), block.MetaFilename), expectedExists: false}, - {path: path.Join("user-3", block9.String(), "index"), expectedExists: false}, + {path: path.Join("user-3", block9.String(), block.IndexFilename), expectedExists: false}, {path: path.Join("user-3", block10.String(), block.MetaFilename), expectedExists: false}, - {path: path.Join("user-3", block10.String(), "index"), expectedExists: false}, + {path: path.Join("user-3", block10.String(), block.IndexFilename), expectedExists: false}, // Tenant deletion mark is not removed. {path: path.Join("user-3", bucket.TenantDeletionMarkPath), expectedExists: true}, // User-4 is removed fully. @@ -963,16 +963,19 @@ func TestStalePartialBlockLastModifiedTime(t *testing.T) { objectTime := time.Now().Add(-1 * time.Hour).Truncate(time.Second) // ignore milliseconds, as not all filesystems store them. blockID := createDBBlock(t, b, tenantId, objectTime.UnixMilli(), time.Now().UnixMilli(), 2, nil) - for _, f := range []string{"meta.json", "index", "chunks/000001", "tombstones"} { - require.NoError(t, os.Chtimes(filepath.Join(dir, tenantId, blockID.String(), filepath.FromSlash(f)), objectTime, objectTime)) - } + err := filepath.Walk(filepath.Join(dir, tenantId, blockID.String()), func(path string, info os.FileInfo, err error) error { + require.NoError(t, err) + require.NoError(t, os.Chtimes(path, objectTime, objectTime)) + return nil + }) + require.NoError(t, err) userBucket := objstore.NewUserBucketClient(tenantId, b, nil) emptyBlockID := ulid.ULID{} require.NotEqual(t, blockID, emptyBlockID) empty := true - err := userBucket.Iter(context.Background(), emptyBlockID.String(), func(_ string) error { + err = userBucket.Iter(context.Background(), emptyBlockID.String(), func(_ string) error { empty = false return nil }) diff --git a/pkg/compactor/compactor.go b/pkg/compactor/compactor.go index 8094fbbdcc..1b5445962c 100644 --- a/pkg/compactor/compactor.go +++ b/pkg/compactor/compactor.go @@ -45,18 +45,15 @@ const ( ) const ( - blocksMarkedForDeletionName = "gitcompactor_blocks_marked_for_deletion_total" + blocksMarkedForDeletionName = "pyroscope_compactor_blocks_marked_for_deletion_total" blocksMarkedForDeletionHelp = "Total number of blocks marked for deletion in compactor." ) var ( - errInvalidBlockRanges = "compactor block range periods should be divisible by the previous one, but %s is not divisible by %s" - errInvalidCompactionOrder = fmt.Errorf("unsupported compaction order (supported values: %s)", strings.Join(CompactionOrders, ", ")) - errInvalidMaxOpeningBlocksConcurrency = fmt.Errorf("invalid max-opening-blocks-concurrency value, must be positive") - errInvalidMaxClosingBlocksConcurrency = fmt.Errorf("invalid max-closing-blocks-concurrency value, must be positive") - errInvalidSymbolFlushersConcurrency = fmt.Errorf("invalid symbols-flushers-concurrency value, must be positive") - errInvalidMaxBlockUploadValidationConcurrency = fmt.Errorf("invalid max-block-upload-validation-concurrency value, can't be negative") - RingOp = ring.NewOp([]ring.InstanceState{ring.ACTIVE}, nil) + errInvalidBlockRanges = "compactor block range periods should be divisible by the previous one, but %s is not divisible by %s" + errInvalidCompactionOrder = fmt.Errorf("unsupported compaction order (supported values: %s)", strings.Join(CompactionOrders, ", ")) + errInvalidMaxOpeningBlocksConcurrency = fmt.Errorf("invalid max-opening-blocks-concurrency value, must be positive") + RingOp = ring.NewOp([]ring.InstanceState{ring.ACTIVE}, nil) ) // BlocksGrouperFactory builds and returns the grouper to use to compact a tenant's blocks. diff --git a/pkg/compactor/compactor_test.go b/pkg/compactor/compactor_test.go index 57c0f98266..b3319ae22d 100644 --- a/pkg/compactor/compactor_test.go +++ b/pkg/compactor/compactor_test.go @@ -1604,7 +1604,7 @@ func createDBBlock(t *testing.T, bkt objstore.Bucket, userID string, minT, maxT // Append a sample for each series, spreading it between minT and maxT-1 (both included). // Since we append one more series below, here we create N-1 series. if numSeries > 1 { - for ts := minT; ts < maxT; ts += (maxT - minT) / int64(numSeries-1) { + for ts := minT; ts <= maxT; ts += (maxT - minT) / int64(numSeries-1) { appendSample(seriesID, ts, float64(seriesID)) seriesID++ } From 40a43688a0ad62810fb788487fbfb06e08cd3993 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Fri, 15 Sep 2023 00:10:01 +0200 Subject: [PATCH 40/74] Removes tombstones removal --- pkg/compactor/bucket_compactor.go | 4 ---- 1 file changed, 4 deletions(-) diff --git a/pkg/compactor/bucket_compactor.go b/pkg/compactor/bucket_compactor.go index 3c86a7f255..bcd832d3fa 100644 --- a/pkg/compactor/bucket_compactor.go +++ b/pkg/compactor/bucket_compactor.go @@ -395,10 +395,6 @@ func (c *BucketCompactor) runCompactionJob(ctx context.Context, job *Job) (shoul return errors.Wrapf(err, "failed to read meta the block dir %s", bdir) } - if err = os.Remove(filepath.Join(bdir, "tombstones")); err != nil { - return errors.Wrap(err, "remove tombstones") - } - // Ensure the compacted block is valid. if err := phlaredb.ValidateLocalBlock(ctx, bdir); err != nil { return errors.Wrapf(err, "invalid result block %s", bdir) From c8dad64c27352dce1b869b6eb647a280432d4911 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Sun, 24 Sep 2023 19:08:47 +1000 Subject: [PATCH 41/74] Fixes the compactor shard label not being saved --- pkg/phlaredb/compact.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/pkg/phlaredb/compact.go b/pkg/phlaredb/compact.go index aeadd09176..6db1cab0cc 100644 --- a/pkg/phlaredb/compact.go +++ b/pkg/phlaredb/compact.go @@ -74,6 +74,9 @@ func CompactWithSplitting(ctx context.Context, src []BlockReader, shardsCount ui for i := range writers { meta := outMeta.Clone() meta.ULID = ulid.MustNew(outBlocksTime, rand.Reader) + if shardsCount > 1 { + meta.Labels[sharding.CompactorShardIDLabel] = sharding.FormatShardIDLabelValue(uint64(i), shardsCount) + } writers[i], err = newBlockWriter(dst, meta) if err != nil { return nil, fmt.Errorf("create block writer: %w", err) @@ -107,11 +110,8 @@ func CompactWithSplitting(ctx context.Context, src []BlockReader, shardsCount ui } out := make([]block.Meta, 0, len(writers)) - for shard, w := range writers { + for _, w := range writers { if w.meta.Stats.NumSamples > 0 { - if shardsCount > 1 { - w.meta.Labels[sharding.CompactorShardIDLabel] = sharding.FormatShardIDLabelValue(uint64(shard), shardsCount) - } out = append(out, *w.meta) } } From cc3c2ea3ead0b499b20d36933c9ebc5e9d0b263e Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Thu, 28 Sep 2023 05:28:37 +0800 Subject: [PATCH 42/74] Fixes most compactor tests. --- pkg/compactor/bucket_compactor_test.go | 7 +- pkg/compactor/compactor.go | 16 +- pkg/compactor/compactor_test.go | 448 ++++++++------------ pkg/compactor/split_merge_compactor_test.go | 87 +--- pkg/compactor/split_merge_grouper.go | 2 +- pkg/compactor/split_merge_grouper_test.go | 87 ++-- pkg/compactor/split_merge_job.go | 2 +- pkg/compactor/split_merge_job_test.go | 8 +- pkg/compactor/tenant_deletion_api.go | 4 +- pkg/frontend/frontend.go | 2 +- pkg/phlaredb/block/metadata.go | 2 +- pkg/phlaredb/bucket/tenant_scanner.go | 4 +- pkg/phlaredb/bucketindex/loader.go | 10 +- pkg/phlaredb/compact.go | 17 +- pkg/phlaredb/compact_test.go | 6 +- pkg/phlaredb/head_test.go | 1 + pkg/storegateway/bucket_stores.go | 4 +- 17 files changed, 261 insertions(+), 446 deletions(-) diff --git a/pkg/compactor/bucket_compactor_test.go b/pkg/compactor/bucket_compactor_test.go index 0b3d1489e3..4943109b5f 100644 --- a/pkg/compactor/bucket_compactor_test.go +++ b/pkg/compactor/bucket_compactor_test.go @@ -122,7 +122,7 @@ func TestBucketCompactor_FilterOwnJobs(t *testing.T) { m := NewBucketCompactorMetrics(promauto.With(nil).NewCounter(prometheus.CounterOpts{}), nil) for testName, testCase := range tests { t.Run(testName, func(t *testing.T) { - bc, err := NewBucketCompactor(log.NewNopLogger(), nil, nil, nil, nil, "", nil, 0, nil, nil, 0, 4, m) + bc, err := NewBucketCompactor(log.NewNopLogger(), nil, nil, nil, nil, "", nil, 2, testCase.ownJob, nil, 0, 4, m) require.NoError(t, err) res, err := bc.filterOwnJobs(jobsFn()) @@ -310,9 +310,10 @@ func TestCompactedBlocksTimeRangeVerification(t *testing.T) { testData := testData // Prevent loop variable being captured by func literal t.Run(testName, func(t *testing.T) { t.Parallel() - bucketClient, _ := objstore_testutil.NewFilesystemBucket(t, context.Background(), t.TempDir()) - tempDir := t.TempDir() + + bucketClient, _ := objstore_testutil.NewFilesystemBucket(t, context.Background(), tempDir) + compactedBlock1 := createDBBlock(t, bucketClient, "foo", testData.compactedBlockMinTime, testData.compactedBlockMinTime+500, 10, nil) compactedBlock2 := createDBBlock(t, bucketClient, "foo", testData.compactedBlockMaxTime-500, testData.compactedBlockMaxTime, 10, nil) diff --git a/pkg/compactor/compactor.go b/pkg/compactor/compactor.go index 1b5445962c..7bf5c04b6f 100644 --- a/pkg/compactor/compactor.go +++ b/pkg/compactor/compactor.go @@ -576,11 +576,11 @@ func (c *MultitenantCompactor) compactUsers(ctx context.Context) { // Ensure the user ID belongs to our shard. if owned, err := c.shardingStrategy.compactorOwnUser(userID); err != nil { c.compactionRunSkippedTenants.Inc() - level.Warn(c.logger).Log("msg", "unable to check if user is owned by this shard", "user", userID, "err", err) + level.Warn(c.logger).Log("msg", "unable to check if user is owned by this shard", "tenant", userID, "err", err) continue } else if !owned { c.compactionRunSkippedTenants.Inc() - level.Debug(c.logger).Log("msg", "skipping user because it is not owned by this shard", "user", userID) + level.Debug(c.logger).Log("msg", "skipping user because it is not owned by this shard", "tenant", userID) continue } @@ -588,32 +588,32 @@ func (c *MultitenantCompactor) compactUsers(ctx context.Context) { if markedForDeletion, err := bucket.TenantDeletionMarkExists(ctx, c.bucketClient, userID); err != nil { c.compactionRunSkippedTenants.Inc() - level.Warn(c.logger).Log("msg", "unable to check if user is marked for deletion", "user", userID, "err", err) + level.Warn(c.logger).Log("msg", "unable to check if user is marked for deletion", "tenant", userID, "err", err) continue } else if markedForDeletion { c.compactionRunSkippedTenants.Inc() - level.Debug(c.logger).Log("msg", "skipping user because it is marked for deletion", "user", userID) + level.Debug(c.logger).Log("msg", "skipping user because it is marked for deletion", "tenant", userID) continue } - level.Info(c.logger).Log("msg", "starting compaction of user blocks", "user", userID) + level.Info(c.logger).Log("msg", "starting compaction of user blocks", "tenant", userID) if err = c.compactUserWithRetries(ctx, userID); err != nil { switch { case errors.Is(err, context.Canceled): // We don't want to count shutdowns as failed compactions because we will pick up with the rest of the compaction after the restart. - level.Info(c.logger).Log("msg", "compaction for user was interrupted by a shutdown", "user", userID) + level.Info(c.logger).Log("msg", "compaction for user was interrupted by a shutdown", "tenant", userID) return default: c.compactionRunFailedTenants.Inc() compactionErrorCount++ - level.Error(c.logger).Log("msg", "failed to compact user blocks", "user", userID, "err", err) + level.Error(c.logger).Log("msg", "failed to compact user blocks", "tenant", userID, "err", err) } continue } c.compactionRunSucceededTenants.Inc() - level.Info(c.logger).Log("msg", "successfully compacted user blocks", "user", userID) + level.Info(c.logger).Log("msg", "successfully compacted user blocks", "tenant", userID) } // Delete local files for unowned tenants, if there are any. This cleans up diff --git a/pkg/compactor/compactor_test.go b/pkg/compactor/compactor_test.go index b3319ae22d..7143a9b2a7 100644 --- a/pkg/compactor/compactor_test.go +++ b/pkg/compactor/compactor_test.go @@ -43,6 +43,7 @@ import ( "gopkg.in/yaml.v3" pyroscope_objstore "github.com/grafana/pyroscope/pkg/objstore" + "github.com/grafana/pyroscope/pkg/objstore/providers/filesystem" "github.com/grafana/pyroscope/pkg/phlaredb/block" "github.com/grafana/pyroscope/pkg/phlaredb/block/testutil" "github.com/grafana/pyroscope/pkg/phlaredb/bucket" @@ -570,22 +571,22 @@ func TestMultitenantCompactor_ShouldIterateOverUsersAndRunCompaction(t *testing. `level=info component=compactor msg="compactor is ACTIVE in the ring"`, `level=info component=compactor msg="discovering users from bucket"`, `level=info component=compactor msg="discovered users from bucket" users=2`, - `level=info component=compactor msg="starting compaction of user blocks" user=user-1`, - `level=info component=compactor user=user-1 msg="start sync of metas"`, - `level=info component=compactor user=user-1 msg="start of GC"`, - `level=debug component=compactor user=user-1 msg="grouper found a compactable blocks group" groupKey=0@17241709254077376921-merge--1574776800000-1574784000000 job="stage: merge, range start: 1574776800000, range end: 1574784000000, shard: , blocks: 01DTVP434PA9VFXSW2JKB3392D (min time: 2019-11-26 14:00:00 +0000 UTC, max time: 2019-11-26 16:00:00 +0000 UTC),01FS51A7GQ1RQWV35DBVYQM4KF (min time: 2019-11-26 14:00:00 +0000 UTC, max time: 2019-11-26 16:00:00 +0000 UTC)"`, - `level=info component=compactor user=user-1 msg="start of compactions"`, - `level=info component=compactor user=user-1 groupKey=0@17241709254077376921-merge--1574776800000-1574784000000 msg="compaction job succeeded"`, - `level=info component=compactor user=user-1 msg="compaction iterations done"`, - `level=info component=compactor msg="successfully compacted user blocks" user=user-1`, - `level=info component=compactor msg="starting compaction of user blocks" user=user-2`, - `level=info component=compactor user=user-2 msg="start sync of metas"`, - `level=info component=compactor user=user-2 msg="start of GC"`, - `level=debug component=compactor user=user-2 msg="grouper found a compactable blocks group" groupKey=0@17241709254077376921-merge--1574776800000-1574784000000 job="stage: merge, range start: 1574776800000, range end: 1574784000000, shard: , blocks: 01DTW0ZCPDDNV4BV83Q2SV4QAZ (min time: 2019-11-26 14:00:00 +0000 UTC, max time: 2019-11-26 16:00:00 +0000 UTC),01FRSF035J26D6CGX7STCSD1KG (min time: 2019-11-26 14:00:00 +0000 UTC, max time: 2019-11-26 16:00:00 +0000 UTC)"`, - `level=info component=compactor user=user-2 msg="start of compactions"`, - `level=info component=compactor user=user-2 groupKey=0@17241709254077376921-merge--1574776800000-1574784000000 msg="compaction job succeeded"`, - `level=info component=compactor user=user-2 msg="compaction iterations done"`, - `level=info component=compactor msg="successfully compacted user blocks" user=user-2`, + `level=info component=compactor msg="starting compaction of user blocks" tenant=user-1`, + `level=info component=compactor tenant=user-1 msg="start sync of metas"`, + `level=info component=compactor tenant=user-1 msg="start of GC"`, + `level=debug component=compactor tenant=user-1 msg="grouper found a compactable blocks group" groupKey=0@17241709254077376921-merge--1574776800000-1574784000000 job="stage: merge, range start: 1574776800000, range end: 1574784000000, shard: , blocks: 01DTVP434PA9VFXSW2JKB3392D (min time: 2019-11-26 14:00:00 +0000 UTC, max time: 2019-11-26 16:00:00 +0000 UTC),01FS51A7GQ1RQWV35DBVYQM4KF (min time: 2019-11-26 14:00:00 +0000 UTC, max time: 2019-11-26 16:00:00 +0000 UTC)"`, + `level=info component=compactor tenant=user-1 msg="start of compactions"`, + `level=info component=compactor tenant=user-1 groupKey=0@17241709254077376921-merge--1574776800000-1574784000000 msg="compaction job succeeded"`, + `level=info component=compactor tenant=user-1 msg="compaction iterations done"`, + `level=info component=compactor msg="successfully compacted user blocks" tenant=user-1`, + `level=info component=compactor msg="starting compaction of user blocks" tenant=user-2`, + `level=info component=compactor tenant=user-2 msg="start sync of metas"`, + `level=info component=compactor tenant=user-2 msg="start of GC"`, + `level=debug component=compactor tenant=user-2 msg="grouper found a compactable blocks group" groupKey=0@17241709254077376921-merge--1574776800000-1574784000000 job="stage: merge, range start: 1574776800000, range end: 1574784000000, shard: , blocks: 01DTW0ZCPDDNV4BV83Q2SV4QAZ (min time: 2019-11-26 14:00:00 +0000 UTC, max time: 2019-11-26 16:00:00 +0000 UTC),01FRSF035J26D6CGX7STCSD1KG (min time: 2019-11-26 14:00:00 +0000 UTC, max time: 2019-11-26 16:00:00 +0000 UTC)"`, + `level=info component=compactor tenant=user-2 msg="start of compactions"`, + `level=info component=compactor tenant=user-2 groupKey=0@17241709254077376921-merge--1574776800000-1574784000000 msg="compaction job succeeded"`, + `level=info component=compactor tenant=user-2 msg="compaction iterations done"`, + `level=info component=compactor msg="successfully compacted user blocks" tenant=user-2`, }, removeIgnoredLogs(strings.Split(strings.TrimSpace(logs.String()), "\n"))) // Instead of testing for shipper metrics, we only check our metrics here. @@ -711,16 +712,16 @@ func TestMultitenantCompactor_ShouldStopCompactingTenantOnReachingMaxCompactionT `level=info component=compactor msg="compactor is ACTIVE in the ring"`, `level=info component=compactor msg="discovering users from bucket"`, `level=info component=compactor msg="discovered users from bucket" users=1`, - `level=info component=compactor msg="starting compaction of user blocks" user=user-1`, - `level=info component=compactor user=user-1 msg="start sync of metas"`, - `level=info component=compactor user=user-1 msg="start of GC"`, - `level=debug component=compactor user=user-1 msg="grouper found a compactable blocks group" groupKey=0@12695595599644216241-merge--1574776800000-1574784000000 job="stage: merge, range start: 1574776800000, range end: 1574784000000, shard: , blocks: 01FN3VCQV5X342W2ZKMQQXAZRX (min time: 2019-11-26 14:00:00 +0000 UTC, max time: 2019-11-26 16:00:00 +0000 UTC),01FRQGQB7RWQ2TS0VWA82QTPXE (min time: 2019-11-26 14:00:00 +0000 UTC, max time: 2019-11-26 16:00:00 +0000 UTC)"`, - `level=debug component=compactor user=user-1 msg="grouper found a compactable blocks group" groupKey=0@414047632870839233-merge--1574776800000-1574784000000 job="stage: merge, range start: 1574776800000, range end: 1574784000000, shard: , blocks: 01DTVP434PA9VFXSW2JKB3392D (min time: 2019-11-26 14:00:00 +0000 UTC, max time: 2019-11-26 16:00:00 +0000 UTC),01FS51A7GQ1RQWV35DBVYQM4KF (min time: 2019-11-26 14:00:00 +0000 UTC, max time: 2019-11-26 16:00:00 +0000 UTC)"`, - `level=info component=compactor user=user-1 msg="start of compactions"`, - `level=info component=compactor user=user-1 msg="max compaction time reached, no more compactions will be started"`, - `level=info component=compactor user=user-1 groupKey=0@12695595599644216241-merge--1574776800000-1574784000000 msg="compaction job succeeded"`, - `level=info component=compactor user=user-1 msg="compaction iterations done"`, - `level=info component=compactor msg="successfully compacted user blocks" user=user-1`, + `level=info component=compactor msg="starting compaction of user blocks" tenant=user-1`, + `level=info component=compactor tenant=user-1 msg="start sync of metas"`, + `level=info component=compactor tenant=user-1 msg="start of GC"`, + `level=debug component=compactor tenant=user-1 msg="grouper found a compactable blocks group" groupKey=0@12695595599644216241-merge--1574776800000-1574784000000 job="stage: merge, range start: 1574776800000, range end: 1574784000000, shard: , blocks: 01FN3VCQV5X342W2ZKMQQXAZRX (min time: 2019-11-26 14:00:00 +0000 UTC, max time: 2019-11-26 16:00:00 +0000 UTC),01FRQGQB7RWQ2TS0VWA82QTPXE (min time: 2019-11-26 14:00:00 +0000 UTC, max time: 2019-11-26 16:00:00 +0000 UTC)"`, + `level=debug component=compactor tenant=user-1 msg="grouper found a compactable blocks group" groupKey=0@414047632870839233-merge--1574776800000-1574784000000 job="stage: merge, range start: 1574776800000, range end: 1574784000000, shard: , blocks: 01DTVP434PA9VFXSW2JKB3392D (min time: 2019-11-26 14:00:00 +0000 UTC, max time: 2019-11-26 16:00:00 +0000 UTC),01FS51A7GQ1RQWV35DBVYQM4KF (min time: 2019-11-26 14:00:00 +0000 UTC, max time: 2019-11-26 16:00:00 +0000 UTC)"`, + `level=info component=compactor tenant=user-1 msg="start of compactions"`, + `level=info component=compactor tenant=user-1 msg="max compaction time reached, no more compactions will be started"`, + `level=info component=compactor tenant=user-1 groupKey=0@12695595599644216241-merge--1574776800000-1574784000000 msg="compaction job succeeded"`, + `level=info component=compactor tenant=user-1 msg="compaction iterations done"`, + `level=info component=compactor msg="successfully compacted user blocks" tenant=user-1`, }, removeIgnoredLogs(strings.Split(strings.TrimSpace(logs.String()), "\n"))) } @@ -785,12 +786,12 @@ func TestMultitenantCompactor_ShouldNotCompactBlocksMarkedForDeletion(t *testing `level=info component=compactor msg="compactor is ACTIVE in the ring"`, `level=info component=compactor msg="discovering users from bucket"`, `level=info component=compactor msg="discovered users from bucket" users=1`, - `level=info component=compactor msg="starting compaction of user blocks" user=user-1`, - `level=info component=compactor user=user-1 msg="start sync of metas"`, - `level=info component=compactor user=user-1 msg="start of GC"`, - `level=info component=compactor user=user-1 msg="start of compactions"`, - `level=info component=compactor user=user-1 msg="compaction iterations done"`, - `level=info component=compactor msg="successfully compacted user blocks" user=user-1`, + `level=info component=compactor msg="starting compaction of user blocks" tenant=user-1`, + `level=info component=compactor tenant=user-1 msg="start sync of metas"`, + `level=info component=compactor tenant=user-1 msg="start of GC"`, + `level=info component=compactor tenant=user-1 msg="start of compactions"`, + `level=info component=compactor tenant=user-1 msg="compaction iterations done"`, + `level=info component=compactor msg="successfully compacted user blocks" tenant=user-1`, }, removeIgnoredLogs(strings.Split(strings.TrimSpace(logs.String()), "\n"))) // Instead of testing for shipper metrics, we only check our metrics here. @@ -886,12 +887,12 @@ func TestMultitenantCompactor_ShouldNotCompactBlocksMarkedForNoCompaction(t *tes `level=info component=compactor msg="compactor is ACTIVE in the ring"`, `level=info component=compactor msg="discovering users from bucket"`, `level=info component=compactor msg="discovered users from bucket" users=1`, - `level=info component=compactor msg="starting compaction of user blocks" user=user-1`, - `level=info component=compactor user=user-1 msg="start sync of metas"`, - `level=info component=compactor user=user-1 msg="start of GC"`, - `level=info component=compactor user=user-1 msg="start of compactions"`, - `level=info component=compactor user=user-1 msg="compaction iterations done"`, - `level=info component=compactor msg="successfully compacted user blocks" user=user-1`, + `level=info component=compactor msg="starting compaction of user blocks" tenant=user-1`, + `level=info component=compactor tenant=user-1 msg="start sync of metas"`, + `level=info component=compactor tenant=user-1 msg="start of GC"`, + `level=info component=compactor tenant=user-1 msg="start of compactions"`, + `level=info component=compactor tenant=user-1 msg="compaction iterations done"`, + `level=info component=compactor msg="successfully compacted user blocks" tenant=user-1`, }, removeIgnoredLogs(strings.Split(strings.TrimSpace(logs.String()), "\n"))) } @@ -947,7 +948,7 @@ func TestMultitenantCompactor_ShouldNotCompactBlocksForUsersMarkedForDeletion(t `level=info component=compactor msg="compactor is ACTIVE in the ring"`, `level=info component=compactor msg="discovering users from bucket"`, `level=info component=compactor msg="discovered users from bucket" users=1`, - `level=debug component=compactor msg="skipping user because it is marked for deletion" user=user-1`, + `level=debug component=compactor msg="skipping user because it is marked for deletion" tenant=user-1`, }, removeIgnoredLogs(strings.Split(strings.TrimSpace(logs.String()), "\n"))) // Instead of testing for shipper metrics, we only check our metrics here. @@ -1064,22 +1065,22 @@ func TestMultitenantCompactor_ShouldCompactAllUsersOnShardingEnabledButOnlyOneIn `level=info component=compactor msg="compactor is ACTIVE in the ring"`, `level=info component=compactor msg="discovering users from bucket"`, `level=info component=compactor msg="discovered users from bucket" users=2`, - `level=info component=compactor msg="starting compaction of user blocks" user=user-1`, - `level=info component=compactor user=user-1 msg="start sync of metas"`, - `level=info component=compactor user=user-1 msg="start of GC"`, - `level=debug component=compactor user=user-1 msg="grouper found a compactable blocks group" groupKey=0@17241709254077376921-merge--1574776800000-1574784000000 job="stage: merge, range start: 1574776800000, range end: 1574784000000, shard: , blocks: 01DTVP434PA9VFXSW2JKB3392D (min time: 2019-11-26 14:00:00 +0000 UTC, max time: 2019-11-26 16:00:00 +0000 UTC),01FSTQ95C8FS0ZAGTQS2EF1NEG (min time: 2019-11-26 14:00:00 +0000 UTC, max time: 2019-11-26 16:00:00 +0000 UTC)"`, - `level=info component=compactor user=user-1 msg="start of compactions"`, - `level=info component=compactor user=user-1 groupKey=0@17241709254077376921-merge--1574776800000-1574784000000 msg="compaction job succeeded"`, - `level=info component=compactor user=user-1 msg="compaction iterations done"`, - `level=info component=compactor msg="successfully compacted user blocks" user=user-1`, - `level=info component=compactor msg="starting compaction of user blocks" user=user-2`, - `level=info component=compactor user=user-2 msg="start sync of metas"`, - `level=info component=compactor user=user-2 msg="start of GC"`, - `level=debug component=compactor user=user-2 msg="grouper found a compactable blocks group" groupKey=0@17241709254077376921-merge--1574776800000-1574784000000 job="stage: merge, range start: 1574776800000, range end: 1574784000000, shard: , blocks: 01DTW0ZCPDDNV4BV83Q2SV4QAZ (min time: 2019-11-26 14:00:00 +0000 UTC, max time: 2019-11-26 16:00:00 +0000 UTC),01FSV54G6QFQH1G9QE93G3B9TB (min time: 2019-11-26 14:00:00 +0000 UTC, max time: 2019-11-26 16:00:00 +0000 UTC)"`, - `level=info component=compactor user=user-2 msg="start of compactions"`, - `level=info component=compactor user=user-2 groupKey=0@17241709254077376921-merge--1574776800000-1574784000000 msg="compaction job succeeded"`, - `level=info component=compactor user=user-2 msg="compaction iterations done"`, - `level=info component=compactor msg="successfully compacted user blocks" user=user-2`, + `level=info component=compactor msg="starting compaction of user blocks" tenant=user-1`, + `level=info component=compactor tenant=user-1 msg="start sync of metas"`, + `level=info component=compactor tenant=user-1 msg="start of GC"`, + `level=debug component=compactor tenant=user-1 msg="grouper found a compactable blocks group" groupKey=0@17241709254077376921-merge--1574776800000-1574784000000 job="stage: merge, range start: 1574776800000, range end: 1574784000000, shard: , blocks: 01DTVP434PA9VFXSW2JKB3392D (min time: 2019-11-26 14:00:00 +0000 UTC, max time: 2019-11-26 16:00:00 +0000 UTC),01FSTQ95C8FS0ZAGTQS2EF1NEG (min time: 2019-11-26 14:00:00 +0000 UTC, max time: 2019-11-26 16:00:00 +0000 UTC)"`, + `level=info component=compactor tenant=user-1 msg="start of compactions"`, + `level=info component=compactor tenant=user-1 groupKey=0@17241709254077376921-merge--1574776800000-1574784000000 msg="compaction job succeeded"`, + `level=info component=compactor tenant=user-1 msg="compaction iterations done"`, + `level=info component=compactor msg="successfully compacted user blocks" tenant=user-1`, + `level=info component=compactor msg="starting compaction of user blocks" tenant=user-2`, + `level=info component=compactor tenant=user-2 msg="start sync of metas"`, + `level=info component=compactor tenant=user-2 msg="start of GC"`, + `level=debug component=compactor tenant=user-2 msg="grouper found a compactable blocks group" groupKey=0@17241709254077376921-merge--1574776800000-1574784000000 job="stage: merge, range start: 1574776800000, range end: 1574784000000, shard: , blocks: 01DTW0ZCPDDNV4BV83Q2SV4QAZ (min time: 2019-11-26 14:00:00 +0000 UTC, max time: 2019-11-26 16:00:00 +0000 UTC),01FSV54G6QFQH1G9QE93G3B9TB (min time: 2019-11-26 14:00:00 +0000 UTC, max time: 2019-11-26 16:00:00 +0000 UTC)"`, + `level=info component=compactor tenant=user-2 msg="start of compactions"`, + `level=info component=compactor tenant=user-2 groupKey=0@17241709254077376921-merge--1574776800000-1574784000000 msg="compaction job succeeded"`, + `level=info component=compactor tenant=user-2 msg="compaction iterations done"`, + `level=info component=compactor msg="successfully compacted user blocks" tenant=user-2`, }, removeIgnoredLogs(strings.Split(strings.TrimSpace(logs.String()), "\n"))) assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` @@ -1208,110 +1209,10 @@ func TestMultitenantCompactor_ShouldCompactOnlyUsersOwnedByTheInstanceOnSharding for _, userID := range userIDs { _, l, err := findCompactorByUserID(compactors, logs, userID) require.NoError(t, err) - assert.Contains(t, l.String(), fmt.Sprintf(`level=info component=compactor msg="successfully compacted user blocks" user=%s`, userID)) + assert.Contains(t, l.String(), fmt.Sprintf(`level=info component=compactor msg="successfully compacted user blocks" tenant=%s`, userID)) } } -// func TestMultitenantCompactor_ShouldFailWithInvalidTSDBCompactOutput(t *testing.T) { -// const user = "user-1" - -// // Two blocks with overlapping time range -// sourceBlock1Spec := []*block.SeriesSpec{ -// { -// Labels: labels.FromStrings("case", "source_spec_1"), -// Chunks: []chunks.Meta{ -// must(tsdbutil.ChunkFromSamples([]tsdbutil.Sample{ -// newSample(1000, 1000, nil, nil), -// newSample(2000, 2000, nil, nil), -// })), -// }, -// }, -// } - -// sourceBlock2Spec := []*block.SeriesSpec{ -// { -// Labels: labels.FromStrings("case", "source_spec_2"), -// Chunks: []chunks.Meta{ -// must(tsdbutil.ChunkFromSamples([]tsdbutil.Sample{ -// newSample(1500, 1500, nil, nil), -// newSample(2500, 2500, nil, nil), -// })), -// }, -// }, -// } - -// // Block with sufficient time range so compaction job gets triggered -// sourceBlock3Spec := []*block.SeriesSpec{ -// { -// Labels: labels.FromStrings("case", "source_spec_3"), -// Chunks: []chunks.Meta{ -// must(tsdbutil.ChunkFromSamples([]tsdbutil.Sample{ -// newSample(0, 0, nil, nil), -// newSample(2*time.Hour.Milliseconds()-1, 0, nil, nil), -// })), -// }, -// }, -// } - -// // Compacted block not containing minTime/maxTime from source blocks -// compactedBlockSpec := []*block.SeriesSpec{ -// { -// Labels: labels.FromStrings("case", "compacted_spec"), -// Chunks: []chunks.Meta{ -// must(tsdbutil.ChunkFromSamples([]tsdbutil.Sample{ -// newSample(1250, 1250, nil, nil), -// newSample(2250, 2250, nil, nil), -// })), -// }, -// }, -// } - -// storageDir := t.TempDir() - -// meta1, err := block.GenerateBlockFromSpec(user, filepath.Join(storageDir, user), sourceBlock1Spec) -// require.NoError(t, err) -// meta2, err := block.GenerateBlockFromSpec(user, filepath.Join(storageDir, user), sourceBlock2Spec) -// require.NoError(t, err) -// _, err = block.GenerateBlockFromSpec(user, filepath.Join(storageDir, user), sourceBlock3Spec) -// require.NoError(t, err) - -// bkt, err := filesystem.NewBucket(storageDir) -// require.NoError(t, err) - -// cfg := prepareConfig(t) -// cfg.CompactionRetries = 1 // No need to retry as we're testing for failure -// c, tsdbCompactor, tsdbPlanner, logs, _ := prepare(t, cfg, bkt) - -// tsdbPlanner.On("Plan", mock.Anything, mock.Anything).Return([]*block.Meta{meta1, meta2}, nil).Once() -// tsdbPlanner.On("Plan", mock.Anything, mock.Anything).Return([]*block.Meta{}, nil).Once() -// mockCall := tsdbCompactor.On("Compact", mock.Anything, mock.Anything, mock.Anything) -// mockCall.RunFn = func(args mock.Arguments) { -// dir := args.Get(0).(string) - -// compactedMeta, err := block.GenerateBlockFromSpec(user, dir, compactedBlockSpec) -// require.NoError(t, err) -// f, err := os.OpenFile(filepath.Join(dir, compactedMeta.ULID.String(), "tombstones"), os.O_RDONLY|os.O_CREATE, 0o666) -// require.NoError(t, err) -// defer f.Close() - -// mockCall.ReturnArguments = mock.Arguments{compactedMeta.ULID, nil} -// } - -// // Start the compactor -// require.NoError(t, services.StartAndAwaitRunning(context.Background(), c)) - -// // Compaction block verification should fail due to invalid output block -// test.Poll(t, 5*time.Second, 1.0, func() interface{} { -// return prom_testutil.ToFloat64(c.bucketCompactorMetrics.compactionBlocksVerificationFailed) -// }) - -// // Stop the compactor. -// require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) - -// // Check logs for compacted block verification failure -// assert.Contains(t, logs.String(), "compacted block(s) do not contain minTime 1000 and maxTime 2501 from the source blocks") -// } - func TestMultitenantCompactor_ShouldSkipCompactionForJobsNoMoreOwnedAfterPlanning(t *testing.T) { t.Parallel() @@ -1380,18 +1281,18 @@ func TestMultitenantCompactor_ShouldSkipCompactionForJobsNoMoreOwnedAfterPlannin `level=info component=compactor msg="compactor is ACTIVE in the ring"`, `level=info component=compactor msg="discovering users from bucket"`, `level=info component=compactor msg="discovered users from bucket" users=1`, - `level=info component=compactor msg="starting compaction of user blocks" user=user-1`, - `level=info component=compactor user=user-1 msg="start sync of metas"`, - `level=info component=compactor user=user-1 msg="start of GC"`, - `level=info component=compactor user=user-1 msg="start of compactions"`, - `level=debug component=compactor user=user-1 msg="grouper found a compactable blocks group" groupKey=0@17241709254077376921-split-4_of_4-1574776800000-1574784000000 job="stage: split, range start: 1574776800000, range end: 1574784000000, shard: 4_of_4, blocks: 01DTVP434PA9VFXSW2JK000001 (min time: 2019-11-26 14:00:00 +0000 UTC, max time: 2019-11-26 16:00:00 +0000 UTC)"`, - `level=debug component=compactor user=user-1 msg="grouper found a compactable blocks group" groupKey=0@17241709254077376921-split-1_of_4-1574863200000-1574870400000 job="stage: split, range start: 1574863200000, range end: 1574870400000, shard: 1_of_4, blocks: 01DTVP434PA9VFXSW2JK000002 (min time: 2019-11-27 14:00:00 +0000 UTC, max time: 2019-11-27 16:00:00 +0000 UTC)"`, + `level=info component=compactor msg="starting compaction of user blocks" tenant=user-1`, + `level=info component=compactor tenant=user-1 msg="start sync of metas"`, + `level=info component=compactor tenant=user-1 msg="start of GC"`, + `level=info component=compactor tenant=user-1 msg="start of compactions"`, + `level=debug component=compactor tenant=user-1 msg="grouper found a compactable blocks group" groupKey=0@17241709254077376921-split-4_of_4-1574776800000-1574784000000 job="stage: split, range start: 1574776800000, range end: 1574784000000, shard: 4_of_4, blocks: 01DTVP434PA9VFXSW2JK000001 (min time: 2019-11-26 14:00:00 +0000 UTC, max time: 2019-11-26 16:00:00 +0000 UTC)"`, + `level=debug component=compactor tenant=user-1 msg="grouper found a compactable blocks group" groupKey=0@17241709254077376921-split-1_of_4-1574863200000-1574870400000 job="stage: split, range start: 1574863200000, range end: 1574870400000, shard: 1_of_4, blocks: 01DTVP434PA9VFXSW2JK000002 (min time: 2019-11-27 14:00:00 +0000 UTC, max time: 2019-11-27 16:00:00 +0000 UTC)"`, // The ownership check is failing because, to keep this test simple, we've just switched // the instance state to LEAVING and there are no other instances in the ring. - `level=info component=compactor user=user-1 groupKey=0@17241709254077376921-split-4_of_4-1574776800000-1574784000000 msg="compaction job succeeded"`, - `level=info component=compactor user=user-1 msg="skipped compaction because unable to check whether the job is owned by the compactor instance" groupKey=0@17241709254077376921-split-1_of_4-1574863200000-1574870400000 err="at least 1 live replicas required, could only find 0 - unhealthy instances: 1.2.3.4:0"`, - `level=info component=compactor user=user-1 msg="compaction iterations done"`, - `level=info component=compactor msg="successfully compacted user blocks" user=user-1`, + `level=info component=compactor tenant=user-1 groupKey=0@17241709254077376921-split-4_of_4-1574776800000-1574784000000 msg="compaction job succeeded"`, + `level=info component=compactor tenant=user-1 msg="skipped compaction because unable to check whether the job is owned by the compactor instance" groupKey=0@17241709254077376921-split-1_of_4-1574863200000-1574870400000 err="at least 1 live replicas required, could only find 0 - unhealthy instances: 1.2.3.4:0"`, + `level=info component=compactor tenant=user-1 msg="compaction iterations done"`, + `level=info component=compactor msg="successfully compacted user blocks" tenant=user-1`, }, removeIgnoredLogs(strings.Split(strings.TrimSpace(logs.String()), "\n"))) assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` @@ -1441,122 +1342,110 @@ func TestMultitenantCompactor_ShouldSkipCompactionForJobsNoMoreOwnedAfterPlannin )) } -// func TestMultitenantCompactor_ShouldSkipCompactionForJobsWithFirstLevelCompactionBlocksAndWaitPeriodNotElapsed(t *testing.T) { -// t.Parallel() - -// storageDir := t.TempDir() -// bucketClient, err := filesystem.NewBucket(storageDir) -// require.NoError(t, err) - -// // Mock two tenants, each with 2 overlapping blocks. -// spec := []*block.SeriesSpec{{ -// Labels: labels.FromStrings(labels.MetricName, "series_1"), -// Chunks: []chunks.Meta{must(tsdbutil.ChunkFromSamples([]tsdbutil.Sample{ -// newSample(1574776800000, 0, nil, nil), -// newSample(1574783999999, 0, nil, nil), -// }))}, -// }} - -// user1Meta1, err := block.GenerateBlockFromSpec("user-1", filepath.Join(storageDir, "user-1"), spec) -// require.NoError(t, err) -// user1Meta2, err := block.GenerateBlockFromSpec("user-1", filepath.Join(storageDir, "user-1"), spec) -// require.NoError(t, err) -// user2Meta1, err := block.GenerateBlockFromSpec("user-2", filepath.Join(storageDir, "user-2"), spec) -// require.NoError(t, err) -// user2Meta2, err := block.GenerateBlockFromSpec("user-2", filepath.Join(storageDir, "user-2"), spec) -// require.NoError(t, err) - -// // Mock the last modified timestamp returned for each of the block's meta.json. -// const waitPeriod = 10 * time.Minute -// mockClient := &bucketWithMockedAttributes{ -// Bucket: bucketClient, -// customAttributes: map[string]objstore.ObjectAttributes{ -// path.Join("user-1", user1Meta1.ULID.String(), block.MetaFilename): {LastModified: time.Now().Add(-20 * time.Minute)}, -// path.Join("user-1", user1Meta2.ULID.String(), block.MetaFilename): {LastModified: time.Now().Add(-20 * time.Minute)}, -// path.Join("user-2", user2Meta1.ULID.String(), block.MetaFilename): {LastModified: time.Now().Add(-20 * time.Minute)}, -// path.Join("user-2", user2Meta2.ULID.String(), block.MetaFilename): {LastModified: time.Now().Add(-5 * time.Minute)}, -// }, -// } - -// cfg := prepareConfig(t) -// cfg.CompactionWaitPeriod = waitPeriod -// c, _, tsdbPlanner, logs, registry := prepare(t, cfg, mockClient) - -// // Mock the planner as if there's no compaction to do, in order to simplify tests. -// tsdbPlanner.On("Plan", mock.Anything, mock.Anything).Return([]*block.Meta{}, nil) - -// require.NoError(t, services.StartAndAwaitRunning(context.Background(), c)) - -// // Compactor doesn't wait for blocks cleaner to finish, but our test checks for cleaner metrics. -// require.NoError(t, c.blocksCleaner.AwaitRunning(context.Background())) - -// // Wait until a run has completed. -// test.Poll(t, 5*time.Second, 1.0, func() interface{} { -// return prom_testutil.ToFloat64(c.compactionRunsCompleted) -// }) - -// require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) - -// // We expect only 1 compaction job has been expected, while the 2nd has been skipped. -// tsdbPlanner.AssertNumberOfCalls(t, "Plan", 1) - -// // Ensure the skipped compaction job is the expected one. -// assert.Contains(t, strings.Split(strings.TrimSpace(logs.String()), "\n"), -// fmt.Sprintf(`level=info component=compactor user=user-2 msg="skipping compaction job because blocks in this job were uploaded too recently (within wait period)" groupKey=0@17241709254077376921-merge--1574776800000-1574784000000 waitPeriodNotElapsedFor="%s (min time: 1574776800000, max time: 1574784000000)"`, user2Meta2.ULID.String())) - -// assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` -// # TYPE pyroscope_compactor_runs_started_total counter -// # HELP pyroscope_compactor_runs_started_total Total number of compaction runs started. -// pyroscope_compactor_runs_started_total 1 - -// # TYPE pyroscope_compactor_runs_completed_total counter -// # HELP pyroscope_compactor_runs_completed_total Total number of compaction runs successfully completed. -// pyroscope_compactor_runs_completed_total 1 - -// # TYPE pyroscope_compactor_runs_failed_total counter -// # HELP pyroscope_compactor_runs_failed_total Total number of compaction runs failed. -// pyroscope_compactor_runs_failed_total{reason="error"} 0 -// pyroscope_compactor_runs_failed_total{reason="shutdown"} 0 - -// # HELP pyroscope_compactor_group_compaction_runs_completed_total Total number of group completed compaction runs. This also includes compactor group runs that resulted with no compaction. -// # TYPE pyroscope_compactor_group_compaction_runs_completed_total counter -// pyroscope_compactor_group_compaction_runs_completed_total 1 - -// # HELP pyroscope_compactor_group_compaction_runs_started_total Total number of group compaction attempts. -// # TYPE pyroscope_compactor_group_compaction_runs_started_total counter -// pyroscope_compactor_group_compaction_runs_started_total 1 - -// # HELP pyroscope_compactor_group_compactions_failures_total Total number of failed group compactions. -// # TYPE pyroscope_compactor_group_compactions_failures_total counter -// pyroscope_compactor_group_compactions_failures_total 0 - -// # HELP pyroscope_compactor_group_compactions_total Total number of group compaction attempts that resulted in new block(s). -// # TYPE pyroscope_compactor_group_compactions_total counter -// pyroscope_compactor_group_compactions_total 0 - -// # HELP pyroscope_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. -// # TYPE pyroscope_compactor_blocks_marked_for_deletion_total counter -// pyroscope_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 -// pyroscope_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 -// pyroscope_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 -// `), -// "pyroscope_compactor_runs_started_total", -// "pyroscope_compactor_runs_completed_total", -// "pyroscope_compactor_runs_failed_total", -// "pyroscope_compactor_group_compaction_runs_completed_total", -// "pyroscope_compactor_group_compaction_runs_started_total", -// "pyroscope_compactor_group_compactions_failures_total", -// "pyroscope_compactor_group_compactions_total", -// "pyroscope_compactor_blocks_marked_for_deletion_total", -// )) -// } +func TestMultitenantCompactor_ShouldSkipCompactionForJobsWithFirstLevelCompactionBlocksAndWaitPeriodNotElapsed(t *testing.T) { + t.Parallel() + + storageDir := t.TempDir() + bucketClient, err := filesystem.NewBucket(storageDir) + require.NoError(t, err) + user1Meta1 := createDBBlock(t, bucketClient, "user-1", 0, (2 * time.Hour).Milliseconds(), 10, nil) + user1Meta2 := createDBBlock(t, bucketClient, "user-1", 0, (2 * time.Hour).Milliseconds(), 10, nil) + user2Meta1 := createDBBlock(t, bucketClient, "user-2", 0, (2 * time.Hour).Milliseconds(), 10, nil) + user2Meta2 := createDBBlock(t, bucketClient, "user-2", 0, (2 * time.Hour).Milliseconds(), 10, nil) + + // Mock the last modified timestamp returned for each of the block's meta.json. + const waitPeriod = 10 * time.Minute + mockClient := &bucketWithMockedAttributes{ + Bucket: bucketClient, + customAttributes: map[string]objstore.ObjectAttributes{ + path.Join("user-1", user1Meta1.String(), block.MetaFilename): {LastModified: time.Now().Add(-20 * time.Minute)}, + path.Join("user-1", user1Meta2.String(), block.MetaFilename): {LastModified: time.Now().Add(-20 * time.Minute)}, + path.Join("user-2", user2Meta1.String(), block.MetaFilename): {LastModified: time.Now().Add(-20 * time.Minute)}, + path.Join("user-2", user2Meta2.String(), block.MetaFilename): {LastModified: time.Now().Add(-5 * time.Minute)}, + }, + } + + cfg := prepareConfig(t) + cfg.CompactionWaitPeriod = waitPeriod + c, _, tsdbPlanner, logs, registry := prepare(t, cfg, mockClient) + + // Mock the planner as if there's no compaction to do, in order to simplify tests. + tsdbPlanner.On("Plan", mock.Anything, mock.Anything).Return([]*block.Meta{}, nil) + + require.NoError(t, services.StartAndAwaitRunning(context.Background(), c)) + + // Compactor doesn't wait for blocks cleaner to finish, but our test checks for cleaner metrics. + require.NoError(t, c.blocksCleaner.AwaitRunning(context.Background())) + + // Wait until a run has completed. + test.Poll(t, 5*time.Second, 1.0, func() interface{} { + return prom_testutil.ToFloat64(c.compactionRunsCompleted) + }) + + require.NoError(t, services.StopAndAwaitTerminated(context.Background(), c)) + + // We expect only 1 compaction job has been expected, while the 2nd has been skipped. + tsdbPlanner.AssertNumberOfCalls(t, "Plan", 1) + + // Ensure the skipped compaction job is the expected one. + assert.Contains(t, strings.Split(strings.TrimSpace(logs.String()), "\n"), + fmt.Sprintf(`level=info component=compactor tenant=user-2 msg="skipping compaction job because blocks in this job were uploaded too recently (within wait period)" groupKey=0@17241709254077376921-merge--0-7200000 waitPeriodNotElapsedFor="%s (min time: 1970-01-01T08:00:00+08:00, max time: 1970-01-01T10:00:00+08:00)"`, user2Meta2.String())) + + assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` + # TYPE pyroscope_compactor_runs_started_total counter + # HELP pyroscope_compactor_runs_started_total Total number of compaction runs started. + pyroscope_compactor_runs_started_total 1 + + # TYPE pyroscope_compactor_runs_completed_total counter + # HELP pyroscope_compactor_runs_completed_total Total number of compaction runs successfully completed. + pyroscope_compactor_runs_completed_total 1 + + # TYPE pyroscope_compactor_runs_failed_total counter + # HELP pyroscope_compactor_runs_failed_total Total number of compaction runs failed. + pyroscope_compactor_runs_failed_total{reason="error"} 0 + pyroscope_compactor_runs_failed_total{reason="shutdown"} 0 + + # HELP pyroscope_compactor_group_compaction_runs_completed_total Total number of group completed compaction runs. This also includes compactor group runs that resulted with no compaction. + # TYPE pyroscope_compactor_group_compaction_runs_completed_total counter + pyroscope_compactor_group_compaction_runs_completed_total 1 + + # HELP pyroscope_compactor_group_compaction_runs_started_total Total number of group compaction attempts. + # TYPE pyroscope_compactor_group_compaction_runs_started_total counter + pyroscope_compactor_group_compaction_runs_started_total 1 + + # HELP pyroscope_compactor_group_compactions_failures_total Total number of failed group compactions. + # TYPE pyroscope_compactor_group_compactions_failures_total counter + pyroscope_compactor_group_compactions_failures_total 0 + + # HELP pyroscope_compactor_group_compactions_total Total number of group compaction attempts that resulted in new block(s). + # TYPE pyroscope_compactor_group_compactions_total counter + pyroscope_compactor_group_compactions_total 0 + + # HELP pyroscope_compactor_blocks_marked_for_deletion_total Total number of blocks marked for deletion in compactor. + # TYPE pyroscope_compactor_blocks_marked_for_deletion_total counter + pyroscope_compactor_blocks_marked_for_deletion_total{reason="compaction"} 0 + pyroscope_compactor_blocks_marked_for_deletion_total{reason="partial"} 0 + pyroscope_compactor_blocks_marked_for_deletion_total{reason="retention"} 0 + `), + "pyroscope_compactor_runs_started_total", + "pyroscope_compactor_runs_completed_total", + "pyroscope_compactor_runs_failed_total", + "pyroscope_compactor_group_compaction_runs_completed_total", + "pyroscope_compactor_group_compaction_runs_started_total", + "pyroscope_compactor_group_compactions_failures_total", + "pyroscope_compactor_group_compactions_total", + "pyroscope_compactor_blocks_marked_for_deletion_total", + )) +} func createCustomBlock(t *testing.T, bkt objstore.Bucket, userID string, externalLabels map[string]string, generator func() []*testhelper.ProfileBuilder) ulid.ULID { meta, dir := testutil.CreateBlock(t, generator) blockLocalPath := filepath.Join(dir, meta.ULID.String()) meta.Source = "test" - meta.Labels = externalLabels + for k, v := range externalLabels { + meta.Labels[k] = v + } _, err := meta.WriteToFile(log.NewNopLogger(), blockLocalPath) require.NoError(t, err) @@ -1608,10 +1497,11 @@ func createDBBlock(t *testing.T, bkt objstore.Bucket, userID string, minT, maxT appendSample(seriesID, ts, float64(seriesID)) seriesID++ } + } else { + appendSample(seriesID, minT, float64(seriesID)) } - - // Guarantee a series with a sample at time maxT-1 - appendSample(seriesID, maxT-1, float64(seriesID)) + // Guarantee a series with a sample at time maxT + appendSample(seriesID, maxT, float64(seriesID)) return result }) } diff --git a/pkg/compactor/split_merge_compactor_test.go b/pkg/compactor/split_merge_compactor_test.go index 1b3025d09a..b92ee229c4 100644 --- a/pkg/compactor/split_merge_compactor_test.go +++ b/pkg/compactor/split_merge_compactor_test.go @@ -54,12 +54,6 @@ func TestMultitenantCompactor_ShouldSupportSplitAndMergeCompactor(t *testing.T) return labels } - externalLabelsWithTenantID := func(shardID string) map[string]string { - labels := externalLabels(shardID) - labels["__org_id__"] = userID - return labels - } - tests := map[string]struct { numShards int setup func(t *testing.T, bkt objstore.Bucket) []block.Meta @@ -95,37 +89,6 @@ func TestMultitenantCompactor_ShouldSupportSplitAndMergeCompactor(t *testing.T) } }, }, - "overlapping blocks matching the 1st compaction range with mixed tenant ID labels should be merged and split": { - numShards: 2, - setup: func(t *testing.T, bkt objstore.Bucket) []block.Meta { - block1 := createDBBlock(t, bkt, userID, blockRangeMillis, 2*blockRangeMillis, numSeries, externalLabels("")) // Doesn't have __org_id__ label - block2 := createDBBlock(t, bkt, userID, blockRangeMillis, 2*blockRangeMillis, numSeries, externalLabelsWithTenantID("")) // Has __org_id__ label - - return []block.Meta{ - { - MinTime: model.Time(1 * blockRangeMillis), - MaxTime: model.Time(2 * blockRangeMillis), - Compaction: tsdb.BlockMetaCompaction{ - Sources: []ulid.ULID{block1, block2}, - }, - - Labels: map[string]string{ - sharding.CompactorShardIDLabel: "1_of_2", - }, - }, { - MinTime: model.Time(1 * blockRangeMillis), - MaxTime: model.Time(2 * blockRangeMillis), - Compaction: tsdb.BlockMetaCompaction{ - Sources: []ulid.ULID{block1, block2}, - }, - - Labels: map[string]string{ - sharding.CompactorShardIDLabel: "2_of_2", - }, - }, - } - }, - }, "overlapping blocks matching the beginning of the 1st compaction range should be merged and split": { numShards: 2, setup: func(t *testing.T, bkt objstore.Bucket) []block.Meta { @@ -276,7 +239,7 @@ func TestMultitenantCompactor_ShouldSupportSplitAndMergeCompactor(t *testing.T) block5b := createDBBlock(t, bkt, userID, 3*blockRangeMillis, 4*blockRangeMillis, numSeries, externalLabels("2_of_2")) // Two non-adjacent non-split blocks in the 1st compaction range. - block6 := createDBBlock(t, bkt, userID, 4*blockRangeMillis, 5*blockRangeMillis, numSeries, externalLabels("")) + block6 := createDBBlock(t, bkt, userID, 4*blockRangeMillis+1, 5*blockRangeMillis, numSeries, externalLabels("")) block7 := createDBBlock(t, bkt, userID, 7*blockRangeMillis, 8*blockRangeMillis, numSeries, externalLabels("")) return []block.Meta{ @@ -308,7 +271,7 @@ func TestMultitenantCompactor_ShouldSupportSplitAndMergeCompactor(t *testing.T) // The two non-adjacent blocks block6 and block7 are split individually first and then merged // together in the 3rd range. { - MinTime: model.Time(4 * blockRangeMillis), + MinTime: model.Time(4*blockRangeMillis + 1), MaxTime: model.Time(8 * blockRangeMillis), Compaction: tsdb.BlockMetaCompaction{ Sources: []ulid.ULID{block6, block7}, @@ -319,7 +282,7 @@ func TestMultitenantCompactor_ShouldSupportSplitAndMergeCompactor(t *testing.T) }, }, { - MinTime: model.Time(4 * blockRangeMillis), + MinTime: model.Time(4*blockRangeMillis + 1), MaxTime: model.Time(8 * blockRangeMillis), Compaction: tsdb.BlockMetaCompaction{ Sources: []ulid.ULID{block6, block7}, @@ -367,41 +330,6 @@ func TestMultitenantCompactor_ShouldSupportSplitAndMergeCompactor(t *testing.T) } }, }, - "overlapping and non-overlapping blocks within the same range and mixed tenant ID label should be split and compacted together": { - numShards: 2, - setup: func(t *testing.T, bkt objstore.Bucket) []block.Meta { - // Overlapping. - block1 := createDBBlock(t, bkt, userID, 0, (5 * time.Minute).Milliseconds(), numSeries, externalLabels("")) // Without __org_id__ label - block2 := createDBBlock(t, bkt, userID, time.Minute.Milliseconds(), (7 * time.Minute).Milliseconds(), numSeries, externalLabelsWithTenantID("")) // With __org_id__ label - - // Not overlapping. - block3 := createDBBlock(t, bkt, userID, time.Hour.Milliseconds(), (2 * time.Hour).Milliseconds(), numSeries, externalLabelsWithTenantID("")) // With __org_id__ label - - return []block.Meta{ - { - MinTime: 0, - MaxTime: model.Time((2 * time.Hour).Milliseconds()), - Compaction: tsdb.BlockMetaCompaction{ - Sources: []ulid.ULID{block1, block2, block3}, - }, - - Labels: map[string]string{ - sharding.CompactorShardIDLabel: "1_of_2", - }, - }, { - MinTime: 0, - MaxTime: model.Time((2 * time.Hour).Milliseconds()), - Compaction: tsdb.BlockMetaCompaction{ - Sources: []ulid.ULID{block1, block2, block3}, - }, - - Labels: map[string]string{ - sharding.CompactorShardIDLabel: "2_of_2", - }, - }, - } - }, - }, "should correctly handle empty blocks generated in the splitting stage": { numShards: 2, setup: func(t *testing.T, bkt objstore.Bucket) []block.Meta { @@ -411,7 +339,7 @@ func TestMultitenantCompactor_ShouldSupportSplitAndMergeCompactor(t *testing.T) return []block.Meta{ { - MinTime: model.Time((2 * blockRangeMillis) - 1), // Because there's only 1 sample with timestamp=maxT-1 + MinTime: model.Time(blockRangeMillis), // Because there's only 1 sample with timestamp=maxT-1 MaxTime: model.Time(2 * blockRangeMillis), Compaction: tsdb.BlockMetaCompaction{ Sources: []ulid.ULID{block1}, @@ -596,6 +524,7 @@ func TestMultitenantCompactor_ShouldSupportSplitAndMergeCompactor(t *testing.T) // Compare actual blocks with the expected ones. require.Len(t, actual, len(expected)) for i, e := range expected { + delete(actual[i].Labels, block.HostnameLabel) assert.Equal(t, e.MinTime, actual[i].MinTime) assert.Equal(t, e.MaxTime, actual[i].MaxTime) assert.Equal(t, e.Compaction.Sources, actual[i].Compaction.Sources) @@ -691,8 +620,8 @@ func TestMultitenantCompactor_ShouldGuaranteeSeriesShardingConsistencyOverTheTim // Ensure the input block has been split. require.Len(t, actualMetas, numShards) for idx, actualMeta := range actualMetas { - assert.Equal(t, blockRangeMillis, actualMeta.MinTime) - assert.Equal(t, 2*blockRangeMillis, actualMeta.MaxTime) + assert.Equal(t, model.Time(blockRangeMillis), actualMeta.MinTime) + assert.Equal(t, model.Time(2*blockRangeMillis), actualMeta.MaxTime) assert.Equal(t, []ulid.ULID{blockID}, actualMeta.Compaction.Sources) assert.Equal(t, sharding.FormatShardIDLabelValue(uint64(idx), numShards), actualMeta.Labels[sharding.CompactorShardIDLabel]) } @@ -702,7 +631,7 @@ func TestMultitenantCompactor_ShouldGuaranteeSeriesShardingConsistencyOverTheTim for _, actualMeta := range actualMetas { expectedSeriesIDs := expectedSeriesIDByShard[actualMeta.Labels[sharding.CompactorShardIDLabel]] - b := phlaredb.NewSingleBlockQuerierFromMeta(ctx, bucketClient, actualMeta) + b := phlaredb.NewSingleBlockQuerierFromMeta(ctx, userBucket, actualMeta) require.NoError(t, b.Open(ctx)) indexReader := b.Index() diff --git a/pkg/compactor/split_merge_grouper.go b/pkg/compactor/split_merge_grouper.go index d3df772a67..4899dbb732 100644 --- a/pkg/compactor/split_merge_grouper.go +++ b/pkg/compactor/split_merge_grouper.go @@ -304,7 +304,7 @@ func groupBlocksByRange(blocks []*block.Meta, tr int64) []blocksGroup { for ; i < len(blocks); i++ { // If the block does not start within this group, then we should break the iteration // and move it to the next group. - if int64(blocks[i].MinTime) >= group.rangeEnd { + if int64(blocks[i].MinTime) > group.rangeEnd { break } diff --git a/pkg/compactor/split_merge_grouper_test.go b/pkg/compactor/split_merge_grouper_test.go index 05f1e331f2..cf34b2e12b 100644 --- a/pkg/compactor/split_merge_grouper_test.go +++ b/pkg/compactor/split_merge_grouper_test.go @@ -107,8 +107,8 @@ func TestPlanCompaction(t *testing.T) { {ULID: block1, MinTime: 0, MaxTime: 10}, {ULID: block2, MinTime: 0, MaxTime: 10}, // 1st level range [10, 20] - {ULID: block3, MinTime: 10, MaxTime: 20}, - {ULID: block4, MinTime: 10, MaxTime: 20}, + {ULID: block3, MinTime: 11, MaxTime: 20}, + {ULID: block4, MinTime: 11, MaxTime: 20}, }, expected: []*job{ {userID: userID, stage: stageSplit, shardID: "1_of_1", blocksGroup: blocksGroup{ @@ -123,8 +123,8 @@ func TestPlanCompaction(t *testing.T) { rangeStart: 10, rangeEnd: 20, blocks: []*block.Meta{ - {ULID: block3, MinTime: 10, MaxTime: 20}, - {ULID: block4, MinTime: 10, MaxTime: 20}, + {ULID: block3, MinTime: 11, MaxTime: 20}, + {ULID: block4, MinTime: 11, MaxTime: 20}, }, }}, }, @@ -138,8 +138,8 @@ func TestPlanCompaction(t *testing.T) { {ULID: block1, MinTime: 0, MaxTime: 10}, {ULID: block2, MinTime: 0, MaxTime: 10}, // 1st level range [10, 20] - {ULID: block3, MinTime: 10, MaxTime: 20}, - {ULID: block4, MinTime: 10, MaxTime: 20}, + {ULID: block3, MinTime: 11, MaxTime: 20}, + {ULID: block4, MinTime: 11, MaxTime: 20}, }, expected: []*job{ {userID: userID, stage: stageSplit, shardID: "1_of_1", blocksGroup: blocksGroup{ @@ -154,8 +154,8 @@ func TestPlanCompaction(t *testing.T) { rangeStart: 10, rangeEnd: 20, blocks: []*block.Meta{ - {ULID: block3, MinTime: 10, MaxTime: 20}, - {ULID: block4, MinTime: 10, MaxTime: 20}, + {ULID: block3, MinTime: 11, MaxTime: 20}, + {ULID: block4, MinTime: 11, MaxTime: 20}, }, }}, }, @@ -169,8 +169,8 @@ func TestPlanCompaction(t *testing.T) { {ULID: block1, MinTime: 0, MaxTime: 10}, {ULID: block2, MinTime: 0, MaxTime: 10}, // 1st level range [10, 20] - {ULID: block3, MinTime: 10, MaxTime: 20}, - {ULID: block4, MinTime: 10, MaxTime: 20}, + {ULID: block3, MinTime: 11, MaxTime: 20}, + {ULID: block4, MinTime: 11, MaxTime: 20}, }, expected: []*job{ {userID: userID, stage: stageSplit, shardID: "1_of_2", blocksGroup: blocksGroup{ @@ -191,14 +191,14 @@ func TestPlanCompaction(t *testing.T) { rangeStart: 10, rangeEnd: 20, blocks: []*block.Meta{ - {ULID: block3, MinTime: 10, MaxTime: 20}, + {ULID: block3, MinTime: 11, MaxTime: 20}, }, }}, {userID: userID, stage: stageSplit, shardID: "2_of_2", blocksGroup: blocksGroup{ rangeStart: 10, rangeEnd: 20, blocks: []*block.Meta{ - {ULID: block4, MinTime: 10, MaxTime: 20}, + {ULID: block4, MinTime: 11, MaxTime: 20}, }, }}, }, @@ -211,8 +211,8 @@ func TestPlanCompaction(t *testing.T) { {ULID: block1, MinTime: 0, MaxTime: 10}, {ULID: block2, MinTime: 0, MaxTime: 10}, // 1st level range [10, 20] - {ULID: block3, MinTime: 10, MaxTime: 20}, - {ULID: block4, MinTime: 10, MaxTime: 20}, + {ULID: block3, MinTime: 11, MaxTime: 20}, + {ULID: block4, MinTime: 11, MaxTime: 20}, }, expected: []*job{ {userID: userID, stage: stageMerge, blocksGroup: blocksGroup{ @@ -227,8 +227,8 @@ func TestPlanCompaction(t *testing.T) { rangeStart: 10, rangeEnd: 20, blocks: []*block.Meta{ - {ULID: block3, MinTime: 10, MaxTime: 20}, - {ULID: block4, MinTime: 10, MaxTime: 20}, + {ULID: block3, MinTime: 11, MaxTime: 20}, + {ULID: block4, MinTime: 11, MaxTime: 20}, }, }}, }, @@ -243,7 +243,7 @@ func TestPlanCompaction(t *testing.T) { {ULID: block3, MinTime: 0, MaxTime: 10, Labels: map[string]string{sharding.CompactorShardIDLabel: "2_of_2"}}, {ULID: block4, MinTime: 10, MaxTime: 20, Labels: map[string]string{sharding.CompactorShardIDLabel: "2_of_2"}}, // 2nd level range [20, 40] - {ULID: block5, MinTime: 20, MaxTime: 30, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_2"}}, + {ULID: block5, MinTime: 21, MaxTime: 30, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_2"}}, {ULID: block6, MinTime: 30, MaxTime: 40, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_2"}}, }, expected: []*job{ @@ -267,7 +267,7 @@ func TestPlanCompaction(t *testing.T) { rangeStart: 20, rangeEnd: 40, blocks: []*block.Meta{ - {ULID: block5, MinTime: 20, MaxTime: 30, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_2"}}, + {ULID: block5, MinTime: 21, MaxTime: 30, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_2"}}, {ULID: block6, MinTime: 30, MaxTime: 40, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_2"}}, }, }}, @@ -283,8 +283,8 @@ func TestPlanCompaction(t *testing.T) { {ULID: block3, MinTime: 0, MaxTime: 10, Labels: map[string]string{sharding.CompactorShardIDLabel: "2_of_2"}}, {ULID: block4, MinTime: 10, MaxTime: 20, Labels: map[string]string{sharding.CompactorShardIDLabel: "2_of_2"}}, // 2nd level range [20, 40] - {ULID: block5, MinTime: 20, MaxTime: 40}, - {ULID: block6, MinTime: 20, MaxTime: 40, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_2"}}, + {ULID: block5, MinTime: 21, MaxTime: 40}, + {ULID: block6, MinTime: 21, MaxTime: 40, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_2"}}, }, expected: []*job{ {userID: userID, stage: stageMerge, shardID: "1_of_2", blocksGroup: blocksGroup{ @@ -317,15 +317,15 @@ func TestPlanCompaction(t *testing.T) { {ULID: block10, MinTime: 0, MaxTime: 10, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, {ULID: block3, MinTime: 10, MaxTime: 20, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, // To be compacted on 2nd level range [20, 40] - {ULID: block4, MinTime: 20, MaxTime: 30, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, + {ULID: block4, MinTime: 21, MaxTime: 30, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, {ULID: block5, MinTime: 30, MaxTime: 40, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, // Already compacted on 2nd level range [40, 60] - {ULID: block6, MinTime: 40, MaxTime: 60, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, + {ULID: block6, MinTime: 41, MaxTime: 60, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, // Not compacted on 2nd level because the range [60, 80] // has other 1st level range groups to be compacted first - {ULID: block7, MinTime: 60, MaxTime: 70, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, + {ULID: block7, MinTime: 61, MaxTime: 70, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, // To be compacted on 1st level range [70, 80] - {ULID: block8, MinTime: 70, MaxTime: 80, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, + {ULID: block8, MinTime: 71, MaxTime: 80, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, {ULID: block9, MinTime: 75, MaxTime: 80, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, }, expected: []*job{ @@ -341,7 +341,7 @@ func TestPlanCompaction(t *testing.T) { rangeStart: 70, rangeEnd: 80, blocks: []*block.Meta{ - {ULID: block8, MinTime: 70, MaxTime: 80, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, + {ULID: block8, MinTime: 71, MaxTime: 80, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, {ULID: block9, MinTime: 75, MaxTime: 80, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, }, }}, @@ -349,7 +349,7 @@ func TestPlanCompaction(t *testing.T) { rangeStart: 20, rangeEnd: 40, blocks: []*block.Meta{ - {ULID: block4, MinTime: 20, MaxTime: 30, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, + {ULID: block4, MinTime: 21, MaxTime: 30, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, {ULID: block5, MinTime: 30, MaxTime: 40, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, }, }}, @@ -371,16 +371,16 @@ func TestPlanCompaction(t *testing.T) { blocks: []*block.Meta{ {ULID: block1, MinTime: 0, MaxTime: 40, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, {ULID: block2, MinTime: 40, MaxTime: 70, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, - {ULID: block3, MinTime: 80, MaxTime: 120, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, - {ULID: block4, MinTime: 80, MaxTime: 120, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, + {ULID: block3, MinTime: 81, MaxTime: 120, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, + {ULID: block4, MinTime: 81, MaxTime: 120, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, }, expected: []*job{ {userID: userID, stage: stageMerge, shardID: "1_of_1", blocksGroup: blocksGroup{ rangeStart: 80, rangeEnd: 120, blocks: []*block.Meta{ - {ULID: block3, MinTime: 80, MaxTime: 120, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, - {ULID: block4, MinTime: 80, MaxTime: 120, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, + {ULID: block3, MinTime: 81, MaxTime: 120, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, + {ULID: block4, MinTime: 81, MaxTime: 120, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, }, }}, }, @@ -391,7 +391,7 @@ func TestPlanCompaction(t *testing.T) { blocks: []*block.Meta{ {ULID: block1, MinTime: 10, MaxTime: 20}, {ULID: block2, MinTime: 10, MaxTime: 30}, // This block spans across two 1st level ranges. - {ULID: block3, MinTime: 20, MaxTime: 30}, + {ULID: block3, MinTime: 21, MaxTime: 30}, {ULID: block4, MinTime: 30, MaxTime: 40}, }, expected: []*job{ @@ -406,7 +406,7 @@ func TestPlanCompaction(t *testing.T) { rangeStart: 20, rangeEnd: 40, blocks: []*block.Meta{ - {ULID: block3, MinTime: 20, MaxTime: 30}, + {ULID: block3, MinTime: 21, MaxTime: 30}, {ULID: block4, MinTime: 30, MaxTime: 40}, }, }}, @@ -439,16 +439,16 @@ func TestPlanCompaction(t *testing.T) { {ULID: block1, MinTime: 0, MaxTime: 40, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, {ULID: block2, MinTime: 30, MaxTime: 150, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, // This block is larger then the largest compaction range. {ULID: block3, MinTime: 40, MaxTime: 70, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, - {ULID: block4, MinTime: 80, MaxTime: 120, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, - {ULID: block5, MinTime: 80, MaxTime: 120, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, + {ULID: block4, MinTime: 81, MaxTime: 120, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, + {ULID: block5, MinTime: 81, MaxTime: 120, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, }, expected: []*job{ {userID: userID, stage: stageMerge, shardID: "1_of_1", blocksGroup: blocksGroup{ rangeStart: 80, rangeEnd: 120, blocks: []*block.Meta{ - {ULID: block4, MinTime: 80, MaxTime: 120, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, - {ULID: block5, MinTime: 80, MaxTime: 120, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, + {ULID: block4, MinTime: 81, MaxTime: 120, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, + {ULID: block5, MinTime: 81, MaxTime: 120, Labels: map[string]string{sharding.CompactorShardIDLabel: "1_of_1"}}, }, }}, }, @@ -718,14 +718,18 @@ func TestGroupBlocksByRange(t *testing.T) { timeRange: 20, blocks: []*block.Meta{ {MinTime: 10, MaxTime: 15}, - {MinTime: 40, MaxTime: 60}, + {MinTime: 21, MaxTime: 40}, + {MinTime: 41, MaxTime: 60}, }, expected: []blocksGroup{ {rangeStart: 0, rangeEnd: 20, blocks: []*block.Meta{ {MinTime: 10, MaxTime: 15}, }}, + {rangeStart: 20, rangeEnd: 40, blocks: []*block.Meta{ + {MinTime: 21, MaxTime: 40}, + }}, {rangeStart: 40, rangeEnd: 60, blocks: []*block.Meta{ - {MinTime: 40, MaxTime: 60}, + {MinTime: 41, MaxTime: 60}, }}, }, }, @@ -754,7 +758,7 @@ func TestGroupBlocksByRange(t *testing.T) { {MinTime: 0, MaxTime: 20}, {MinTime: 0, MaxTime: 40}, // This block is larger then the range. {MinTime: 10, MaxTime: 20}, - {MinTime: 20, MaxTime: 30}, + {MinTime: 21, MaxTime: 30}, }, expected: []blocksGroup{ {rangeStart: 0, rangeEnd: 20, blocks: []*block.Meta{ @@ -762,7 +766,7 @@ func TestGroupBlocksByRange(t *testing.T) { {MinTime: 10, MaxTime: 20}, }}, {rangeStart: 20, rangeEnd: 40, blocks: []*block.Meta{ - {MinTime: 20, MaxTime: 30}, + {MinTime: 21, MaxTime: 30}, }}, }, }, @@ -787,7 +791,8 @@ func TestGroupBlocksByRange(t *testing.T) { for testName, testData := range tests { t.Run(testName, func(t *testing.T) { - assert.Equal(t, testData.expected, groupBlocksByRange(testData.blocks, testData.timeRange)) + actual := groupBlocksByRange(testData.blocks, testData.timeRange) + assert.Equal(t, testData.expected, actual) }) } } diff --git a/pkg/compactor/split_merge_job.go b/pkg/compactor/split_merge_job.go index 2033c4cf7a..c282baa2e0 100644 --- a/pkg/compactor/split_merge_job.go +++ b/pkg/compactor/split_merge_job.go @@ -97,7 +97,7 @@ func (j *job) String() string { // blocksGroup holds a group of blocks within the same time range. type blocksGroup struct { rangeStart int64 // Included. - rangeEnd int64 // Excluded. + rangeEnd int64 // Included. blocks []*block.Meta // Sorted by MinTime. } diff --git a/pkg/compactor/split_merge_job_test.go b/pkg/compactor/split_merge_job_test.go index 3b894d5723..c99e1b4a16 100644 --- a/pkg/compactor/split_merge_job_test.go +++ b/pkg/compactor/split_merge_job_test.go @@ -79,7 +79,7 @@ func TestJob_conflicts(t *testing.T) { stage: stageMerge, shardID: "1_of_2", blocksGroup: blocksGroup{ - rangeStart: 20, + rangeStart: 21, rangeEnd: 30, blocks: []*block.Meta{withShardIDLabel(block3, "1_of_2"), withShardIDLabel(block4, "1_of_2")}, }, @@ -163,7 +163,7 @@ func TestJob_conflicts(t *testing.T) { stage: stageMerge, shardID: "1_of_2", blocksGroup: blocksGroup{ - rangeStart: 20, + rangeStart: 21, rangeEnd: 40, blocks: []*block.Meta{withShardIDLabel(block3, "1_of_2"), withShardIDLabel(block4, "1_of_2")}, }, @@ -190,6 +190,10 @@ func TestBlocksGroup_overlaps(t *testing.T) { first: blocksGroup{rangeStart: 10, rangeEnd: 20}, second: blocksGroup{rangeStart: 20, rangeEnd: 30}, expected: false, + }, { + first: blocksGroup{rangeStart: 10, rangeEnd: 20}, + second: blocksGroup{rangeStart: 21, rangeEnd: 30}, + expected: false, }, { first: blocksGroup{rangeStart: 10, rangeEnd: 20}, second: blocksGroup{rangeStart: 19, rangeEnd: 30}, diff --git a/pkg/compactor/tenant_deletion_api.go b/pkg/compactor/tenant_deletion_api.go index 470e04fda7..489092945e 100644 --- a/pkg/compactor/tenant_deletion_api.go +++ b/pkg/compactor/tenant_deletion_api.go @@ -33,13 +33,13 @@ package compactor // err = mimir_tsdb.WriteTenantDeletionMark(r.Context(), c.bucketClient, userID, c.cfgProvider, mimir_tsdb.NewTenantDeletionMark(time.Now())) // if err != nil { -// level.Error(c.logger).Log("msg", "failed to write tenant deletion mark", "user", userID, "err", err) +// level.Error(c.logger).Log("msg", "failed to write tenant deletion mark", "tenant", userID, "err", err) // http.Error(w, err.Error(), http.StatusInternalServerError) // return // } -// level.Info(c.logger).Log("msg", "tenant deletion mark in blocks storage created", "user", userID) +// level.Info(c.logger).Log("msg", "tenant deletion mark in blocks storage created", "tenant", userID) // w.WriteHeader(http.StatusOK) // } diff --git a/pkg/frontend/frontend.go b/pkg/frontend/frontend.go index 6610487744..8a1ba4419c 100644 --- a/pkg/frontend/frontend.go +++ b/pkg/frontend/frontend.go @@ -292,7 +292,7 @@ func (f *Frontend) QueryResult(ctx context.Context, r *connect.Request[frontendp case req.response <- qrReq: // Should always be possible, unless QueryResult is called multiple times with the same queryID. default: - level.Warn(f.log).Log("msg", "failed to write query result to the response channel", "queryID", qrReq.QueryID, "user", userID) + level.Warn(f.log).Log("msg", "failed to write query result to the response channel", "queryID", qrReq.QueryID, "tenant", userID) } } diff --git a/pkg/phlaredb/block/metadata.go b/pkg/phlaredb/block/metadata.go index f55f63c02c..0cdede23b1 100644 --- a/pkg/phlaredb/block/metadata.go +++ b/pkg/phlaredb/block/metadata.go @@ -121,7 +121,7 @@ type Meta struct { Version MetaVersion `json:"version"` // Labels are the external labels identifying the producer as well as tenant. - Labels map[string]string `json:"labels,omitempty"` + Labels map[string]string `json:"labels"` // Source is a real upload source of the block. Source SourceType `json:"source,omitempty"` diff --git a/pkg/phlaredb/bucket/tenant_scanner.go b/pkg/phlaredb/bucket/tenant_scanner.go index 65e17927f2..413ae466fa 100644 --- a/pkg/phlaredb/bucket/tenant_scanner.go +++ b/pkg/phlaredb/bucket/tenant_scanner.go @@ -52,7 +52,7 @@ func (s *TenantsScanner) ScanTenants(ctx context.Context) (users, markedForDelet // Check if it's owned by this instance. owned, err := s.isOwned(userID) if err != nil { - level.Warn(s.logger).Log("msg", "unable to check if user is owned by this shard", "user", userID, "err", err) + level.Warn(s.logger).Log("msg", "unable to check if user is owned by this shard", "tenant", userID, "err", err) } else if !owned { users = append(users[:ix], users[ix+1:]...) continue @@ -60,7 +60,7 @@ func (s *TenantsScanner) ScanTenants(ctx context.Context) (users, markedForDelet deletionMarkExists, err := TenantDeletionMarkExists(ctx, s.bucketClient, userID) if err != nil { - level.Warn(s.logger).Log("msg", "unable to check if user is marked for deletion", "user", userID, "err", err) + level.Warn(s.logger).Log("msg", "unable to check if user is marked for deletion", "tenant", userID, "err", err) } else if deletionMarkExists { users = append(users[:ix], users[ix+1:]...) markedForDeletion = append(markedForDeletion, userID) diff --git a/pkg/phlaredb/bucketindex/loader.go b/pkg/phlaredb/bucketindex/loader.go index ed68f2065c..0faa5ed00e 100644 --- a/pkg/phlaredb/bucketindex/loader.go +++ b/pkg/phlaredb/bucketindex/loader.go @@ -112,12 +112,12 @@ func (l *Loader) GetIndex(ctx context.Context, userID string) (*Index, error) { l.cacheIndex(userID, nil, err) if errors.Is(err, ErrIndexNotFound) { - level.Warn(l.logger).Log("msg", "bucket index not found", "user", userID) + level.Warn(l.logger).Log("msg", "bucket index not found", "tenant", userID) } else { // We don't track ErrIndexNotFound as failure because it's a legit case (eg. a tenant just // started to remote write and its blocks haven't uploaded to storage yet). l.loadFailures.Inc() - level.Error(l.logger).Log("msg", "unable to load bucket index", "user", userID, "err", err) + level.Error(l.logger).Log("msg", "unable to load bucket index", "tenant", userID, "err", err) } return nil, err @@ -128,7 +128,7 @@ func (l *Loader) GetIndex(ctx context.Context, userID string) (*Index, error) { elapsedTime := time.Since(startTime) l.loadDuration.Observe(elapsedTime.Seconds()) - level.Info(l.logger).Log("msg", "loaded bucket index", "user", userID, "duration", elapsedTime) + level.Info(l.logger).Log("msg", "loaded bucket index", "tenant", userID, "duration", elapsedTime) return idx, nil } @@ -193,7 +193,7 @@ func (l *Loader) updateCachedIndex(ctx context.Context, userID string) { idx, err := ReadIndex(ctx, l.bkt, userID, l.cfgProvider, l.logger) if err != nil && !errors.Is(err, ErrIndexNotFound) { l.loadFailures.Inc() - level.Warn(l.logger).Log("msg", "unable to update bucket index", "user", userID, "err", err) + level.Warn(l.logger).Log("msg", "unable to update bucket index", "tenant", userID, "err", err) return } @@ -214,7 +214,7 @@ func (l *Loader) deleteCachedIndex(userID string) { delete(l.indexes, userID) l.indexesMx.Unlock() - level.Info(l.logger).Log("msg", "unloaded bucket index", "user", userID, "reason", "idle") + level.Info(l.logger).Log("msg", "unloaded bucket index", "tenant", userID, "reason", "idle") } func (l *Loader) countLoadedIndexesMetric() float64 { diff --git a/pkg/phlaredb/compact.go b/pkg/phlaredb/compact.go index 6db1cab0cc..daa5e58e4b 100644 --- a/pkg/phlaredb/compact.go +++ b/pkg/phlaredb/compact.go @@ -75,6 +75,9 @@ func CompactWithSplitting(ctx context.Context, src []BlockReader, shardsCount ui meta := outMeta.Clone() meta.ULID = ulid.MustNew(outBlocksTime, rand.Reader) if shardsCount > 1 { + if meta.Labels == nil { + meta.Labels = make(map[string]string) + } meta.Labels[sharding.CompactorShardIDLabel] = sharding.FormatShardIDLabelValue(uint64(i), shardsCount) } writers[i], err = newBlockWriter(dst, meta) @@ -131,7 +134,6 @@ type blockWriter struct { path string meta *block.Meta totalProfiles uint64 - min, max int64 } func newBlockWriter(dst string, meta *block.Meta) (*blockWriter, error) { @@ -152,8 +154,6 @@ func newBlockWriter(dst string, meta *block.Meta) (*blockWriter, error) { profilesWriter: profileWriter, path: blockPath, meta: meta, - min: math.MaxInt64, - max: math.MinInt64, }, nil } @@ -171,12 +171,6 @@ func (bw *blockWriter) WriteRow(r profileRow) error { return err } bw.totalProfiles++ - if r.timeNanos < bw.min { - bw.min = r.timeNanos - } - if r.timeNanos > bw.max { - bw.max = r.timeNanos - } return nil } @@ -199,8 +193,6 @@ func (bw *blockWriter) Close(ctx context.Context) error { bw.meta.Stats.NumSeries = bw.indexRewriter.NumSeries() bw.meta.Stats.NumSamples = bw.symbolsRewriter.NumSamples() bw.meta.Compaction.Deletable = bw.totalProfiles == 0 - bw.meta.MinTime = model.TimeFromUnixNano(bw.min) - bw.meta.MaxTime = model.TimeFromUnixNano(bw.max) if _, err := bw.meta.WriteToFile(util.Logger, bw.path); err != nil { return err } @@ -420,9 +412,6 @@ func compactMetas(src ...block.Meta) block.Meta { labels[k] = v } } - if hostname, err := os.Hostname(); err == nil { - labels[block.HostnameLabel] = hostname - } meta.Source = block.CompactorSource meta.Compaction = tsdb.BlockMetaCompaction{ Deletable: false, diff --git a/pkg/phlaredb/compact_test.go b/pkg/phlaredb/compact_test.go index 040e98d829..0b5cce65d9 100644 --- a/pkg/phlaredb/compact_test.go +++ b/pkg/phlaredb/compact_test.go @@ -119,8 +119,7 @@ func TestCompactWithSplitting(t *testing.T) { require.Equal(t, "7_of_16", compacted[2].Labels[sharding.CompactorShardIDLabel]) require.Equal(t, "14_of_16", compacted[3].Labels[sharding.CompactorShardIDLabel]) - // The series b should span from 11 to 20 and not 1 to 20. - require.Equal(t, model.TimeFromUnix(11), compacted[1].MinTime) + require.Equal(t, model.TimeFromUnix(1), compacted[1].MinTime) require.Equal(t, model.TimeFromUnix(20), compacted[1].MaxTime) // We first verify we have all series and timestamps across querying all blocks. @@ -620,9 +619,6 @@ func TestCompactMetas(t *testing.T) { }, }...) labels := map[string]string{"foo": "bar", "bar": "buzz"} - if hostname, err := os.Hostname(); err == nil { - labels[block.HostnameLabel] = hostname - } require.Equal(t, model.TimeFromUnix(0), actual.MinTime) require.Equal(t, model.TimeFromUnix(200), actual.MaxTime) require.Equal(t, tsdb.BlockMetaCompaction{ diff --git a/pkg/phlaredb/head_test.go b/pkg/phlaredb/head_test.go index 3b0f4a32a1..23d177ebfe 100644 --- a/pkg/phlaredb/head_test.go +++ b/pkg/phlaredb/head_test.go @@ -292,6 +292,7 @@ func TestHeadFlush(t *testing.T) { NumSeries: 8, NumProfiles: 11, }, + Labels: map[string]string{}, Files: []block.File{ { RelPath: "index.tsdb", diff --git a/pkg/storegateway/bucket_stores.go b/pkg/storegateway/bucket_stores.go index bf327c9b55..5f8179e701 100644 --- a/pkg/storegateway/bucket_stores.go +++ b/pkg/storegateway/bucket_stores.go @@ -354,9 +354,9 @@ func (bs *BucketStores) closeBucketStoreAndDeleteLocalFilesForExcludedTenants(in case errors.Is(err, errBucketStoreNotFound): // This is OK, nothing was closed. case err == nil: - level.Info(bs.logger).Log("msg", "closed bucket store for user", "user", userID) + level.Info(bs.logger).Log("msg", "closed bucket store for user", "tenant", userID) default: - level.Warn(bs.logger).Log("msg", "failed to close bucket store for user", "user", userID, "err", err) + level.Warn(bs.logger).Log("msg", "failed to close bucket store for user", "tenant", userID, "err", err) } userSyncDir := bs.syncDirForUser(userID) From 0a1d6d984157ddbe0ad70b6045278faff81d007d Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Fri, 29 Sep 2023 09:14:45 +0200 Subject: [PATCH 43/74] Implement e2e tests --- pkg/compactor/bucket_compactor.go | 15 - pkg/compactor/bucket_compactor_e2e_test.go | 1134 +++++--------------- pkg/compactor/tenant_deletion_api.go | 98 -- pkg/compactor/tenant_deletion_api_test.go | 100 -- 4 files changed, 293 insertions(+), 1054 deletions(-) delete mode 100644 pkg/compactor/tenant_deletion_api.go delete mode 100644 pkg/compactor/tenant_deletion_api_test.go diff --git a/pkg/compactor/bucket_compactor.go b/pkg/compactor/bucket_compactor.go index bcd832d3fa..c33bac107b 100644 --- a/pkg/compactor/bucket_compactor.go +++ b/pkg/compactor/bucket_compactor.go @@ -358,20 +358,6 @@ func (c *BucketCompactor) runCompactionJob(ctx context.Context, job *Job) (shoul return false, nil, errors.Wrapf(err, "compact blocks %v", blocksToCompactDirs) } - if !hasNonZeroULIDs(compIDs) { - // Prometheus compactor found that the compacted block would have no samples. - level.Info(jobLogger).Log("msg", "compacted block would have no samples, deleting source blocks", "blocks", fmt.Sprintf("%v", blocksToCompactDirs)) - for _, meta := range toCompact { - if meta.Stats.NumSamples == 0 { - if err := deleteBlock(c.bkt, meta.ULID, filepath.Join(subDir, meta.ULID.String()), jobLogger, c.metrics.blocksMarkedForDeletion); err != nil { - level.Warn(jobLogger).Log("msg", "failed to mark for deletion an empty block found during compaction", "block", meta.ULID, "err", err) - } - } - } - // Even though this block was empty, there may be more work to do. - return true, nil, nil - } - elapsed = time.Since(compactionBegin) level.Info(jobLogger).Log("msg", "compacted blocks", "new", fmt.Sprintf("%v", compIDs), "blocks", fmt.Sprintf("%v", blocksToCompactDirs), "duration", elapsed, "duration_ms", elapsed.Milliseconds()) @@ -558,7 +544,6 @@ type BucketCompactor struct { sortJobs JobsOrderFunc waitPeriod time.Duration blockSyncConcurrency int - blockOpenConcurrency int metrics *BucketCompactorMetrics } diff --git a/pkg/compactor/bucket_compactor_e2e_test.go b/pkg/compactor/bucket_compactor_e2e_test.go index db870d329c..b21c8c0fe9 100644 --- a/pkg/compactor/bucket_compactor_e2e_test.go +++ b/pkg/compactor/bucket_compactor_e2e_test.go @@ -5,844 +5,296 @@ package compactor -// import ( -// "bytes" -// "context" -// "encoding/json" -// "fmt" -// "math/rand" -// "os" -// "path" -// "path/filepath" -// "runtime" -// "sort" -// "strings" -// "testing" -// "time" - -// "github.com/go-kit/log" -// "github.com/grafana/dskit/runutil" -// "github.com/oklog/ulid" -// "github.com/pkg/errors" -// "github.com/prometheus/client_golang/prometheus" -// "github.com/prometheus/client_golang/prometheus/promauto" -// promtest "github.com/prometheus/client_golang/prometheus/testutil" -// "github.com/prometheus/prometheus/model/labels" -// "github.com/prometheus/prometheus/storage" -// "github.com/prometheus/prometheus/tsdb" -// "github.com/prometheus/prometheus/tsdb/chunkenc" -// "github.com/prometheus/prometheus/tsdb/chunks" -// "github.com/prometheus/prometheus/tsdb/index" -// "github.com/stretchr/testify/assert" -// "github.com/stretchr/testify/require" -// "github.com/thanos-io/objstore" -// "github.com/thanos-io/objstore/providers/filesystem" -// "golang.org/x/exp/slices" -// "golang.org/x/sync/errgroup" - -// "github.com/grafana/mimir/pkg/storage/tsdb/block" -// ) - -// func TestSyncer_GarbageCollect_e2e(t *testing.T) { -// foreachStore(t, func(t *testing.T, bkt objstore.Bucket) { -// // Use bucket with global markers to make sure that our custom filters work correctly. -// bkt = block.BucketWithGlobalMarkers(bkt) - -// ctx, cancel := context.WithTimeout(context.Background(), 120*time.Second) -// defer cancel() - -// // Generate 10 source block metas and construct higher level blocks -// // that are higher compactions of them. -// var metas []*block.Meta -// var ids []ulid.ULID - -// for i := 0; i < 10; i++ { -// var m block.Meta - -// m.Version = 1 -// m.ULID = ulid.MustNew(uint64(i), nil) -// m.Compaction.Sources = []ulid.ULID{m.ULID} -// m.Compaction.Level = 1 -// m.MinTime = 0 -// m.MaxTime = 2 * time.Hour.Milliseconds() - -// ids = append(ids, m.ULID) -// metas = append(metas, &m) -// } - -// var m1 block.Meta -// m1.Version = 1 -// m1.ULID = ulid.MustNew(100, nil) -// m1.Compaction.Level = 2 -// m1.Compaction.Sources = ids[:4] -// m1.Thanos.Downsample.Resolution = 0 - -// var m2 block.Meta -// m2.Version = 1 -// m2.ULID = ulid.MustNew(200, nil) -// m2.Compaction.Level = 2 -// m2.Compaction.Sources = ids[4:8] // last two source IDs is not part of a level 2 block. -// m2.Thanos.Downsample.Resolution = 0 - -// var m3 block.Meta -// m3.Version = 1 -// m3.ULID = ulid.MustNew(300, nil) -// m3.Compaction.Level = 3 -// m3.Compaction.Sources = ids[:9] // last source ID is not part of level 3 block. -// m3.Thanos.Downsample.Resolution = 0 -// m3.MinTime = 0 -// m3.MaxTime = 2 * time.Hour.Milliseconds() - -// var m4 block.Meta -// m4.Version = 1 -// m4.ULID = ulid.MustNew(400, nil) -// m4.Compaction.Level = 2 -// m4.Compaction.Sources = ids[9:] // covers the last block but is a different resolution. Must not trigger deletion. -// m4.Thanos.Downsample.Resolution = 1000 -// m4.MinTime = 0 -// m4.MaxTime = 2 * time.Hour.Milliseconds() - -// var m5 block.Meta -// m5.Version = 1 -// m5.ULID = ulid.MustNew(500, nil) -// m5.Compaction.Level = 2 -// m5.Compaction.Sources = ids[8:9] // built from block 8, but different resolution. Block 8 is already included in m3, can be deleted. -// m5.Thanos.Downsample.Resolution = 1000 -// m5.MinTime = 0 -// m5.MaxTime = 2 * time.Hour.Milliseconds() - -// // Create all blocks in the bucket. -// for _, m := range append(metas, &m1, &m2, &m3, &m4, &m5) { -// fmt.Println("create", m.ULID) -// var buf bytes.Buffer -// require.NoError(t, json.NewEncoder(&buf).Encode(&m)) -// require.NoError(t, bkt.Upload(ctx, path.Join(m.ULID.String(), block.MetaFilename), &buf)) -// } - -// duplicateBlocksFilter := NewShardAwareDeduplicateFilter() -// metaFetcher, err := block.NewMetaFetcher(nil, 32, objstore.WithNoopInstr(bkt), "", nil, []block.MetadataFilter{ -// duplicateBlocksFilter, -// }) -// require.NoError(t, err) - -// blocksMarkedForDeletion := promauto.With(nil).NewCounter(prometheus.CounterOpts{}) -// sy, err := NewMetaSyncer(nil, nil, bkt, metaFetcher, duplicateBlocksFilter, blocksMarkedForDeletion) -// require.NoError(t, err) - -// // Do one initial synchronization with the bucket. -// require.NoError(t, sy.SyncMetas(ctx)) -// require.NoError(t, sy.GarbageCollect(ctx)) - -// var rem []ulid.ULID -// err = bkt.Iter(ctx, "", func(n string) error { -// id, ok := block.IsBlockDir(n) -// if !ok { -// return nil -// } -// deletionMarkFile := path.Join(id.String(), block.DeletionMarkFilename) - -// exists, err := bkt.Exists(ctx, deletionMarkFile) -// if err != nil { -// return err -// } -// if !exists { -// rem = append(rem, id) -// } -// return nil -// }) -// require.NoError(t, err) - -// sort.Slice(rem, func(i, j int) bool { -// return rem[i].Compare(rem[j]) < 0 -// }) - -// // Only the level 3 block, the last source block in both resolutions should be left. -// assert.Equal(t, []ulid.ULID{metas[9].ULID, m3.ULID, m4.ULID, m5.ULID}, rem) - -// // After another sync the changes should also be reflected in the local groups. -// require.NoError(t, sy.SyncMetas(ctx)) -// require.NoError(t, sy.GarbageCollect(ctx)) - -// // Only the level 3 block, the last source block in both resolutions should be left. -// grouper := NewSplitAndMergeGrouper("user-1", []int64{2 * time.Hour.Milliseconds()}, 0, 0, log.NewNopLogger()) -// groups, err := grouper.Groups(sy.Metas()) -// require.NoError(t, err) - -// assert.Equal(t, "0@17241709254077376921-merge--0-7200000", groups[0].Key()) -// assert.Equal(t, []ulid.ULID{metas[9].ULID, m3.ULID}, groups[0].IDs()) - -// assert.Equal(t, "1000@17241709254077376921-merge--0-7200000", groups[1].Key()) -// assert.Equal(t, []ulid.ULID{m4.ULID, m5.ULID}, groups[1].IDs()) -// }) -// } - -// func TestGroupCompactE2E(t *testing.T) { -// foreachStore(t, func(t *testing.T, bkt objstore.Bucket) { -// // Use bucket with global markers to make sure that our custom filters work correctly. -// bkt = block.BucketWithGlobalMarkers(bkt) - -// ctx, cancel := context.WithTimeout(context.Background(), 120*time.Second) -// defer cancel() - -// // Create fresh, empty directory for actual test. -// dir := t.TempDir() - -// // Start dir checker... we make sure that "dir" only contains group subdirectories during compaction, -// // and not any block directories. Dir checker stops when context is canceled, or on first error, -// // in which case error is logger and test is failed. (We cannot use Fatal or FailNow from a goroutine). -// go func() { -// for ctx.Err() == nil { -// fs, err := os.ReadDir(dir) -// if err != nil && !os.IsNotExist(err) { -// t.Log("error while listing directory", dir) -// t.Fail() -// return -// } - -// for _, fi := range fs { -// // Suffix used by Prometheus LeveledCompactor when doing compaction. -// toCheck := strings.TrimSuffix(fi.Name(), ".tmp-for-creation") - -// _, err := ulid.Parse(toCheck) -// if err == nil { -// t.Log("found block directory in main compaction directory", fi.Name()) -// t.Fail() -// return -// } -// } - -// select { -// case <-time.After(100 * time.Millisecond): -// continue -// case <-ctx.Done(): -// return -// } -// } -// }() - -// logger := log.NewLogfmtLogger(os.Stderr) - -// reg := prometheus.NewRegistry() - -// duplicateBlocksFilter := NewShardAwareDeduplicateFilter() -// noCompactMarkerFilter := NewNoCompactionMarkFilter(objstore.WithNoopInstr(bkt), true) -// metaFetcher, err := block.NewMetaFetcher(nil, 32, objstore.WithNoopInstr(bkt), "", nil, []block.MetadataFilter{ -// duplicateBlocksFilter, -// noCompactMarkerFilter, -// }) -// require.NoError(t, err) - -// blocksMarkedForDeletion := promauto.With(nil).NewCounter(prometheus.CounterOpts{}) -// sy, err := NewMetaSyncer(nil, nil, bkt, metaFetcher, duplicateBlocksFilter, blocksMarkedForDeletion) -// require.NoError(t, err) - -// comp, err := tsdb.NewLeveledCompactor(ctx, reg, logger, []int64{1000, 3000}, nil, nil, true) -// require.NoError(t, err) - -// planner := NewSplitAndMergePlanner([]int64{1000, 3000}) -// grouper := NewSplitAndMergeGrouper("user-1", []int64{1000, 3000}, 0, 0, logger) -// metrics := NewBucketCompactorMetrics(blocksMarkedForDeletion, prometheus.NewPedanticRegistry()) -// bComp, err := NewBucketCompactor(logger, sy, grouper, planner, comp, dir, bkt, 2, true, ownAllJobs, sortJobsByNewestBlocksFirst, 0, 4, metrics) -// require.NoError(t, err) - -// // Compaction on empty should not fail. -// require.NoError(t, bComp.Compact(ctx, 0), 0) -// assert.Equal(t, 0.0, promtest.ToFloat64(sy.metrics.blocksMarkedForDeletion)) -// assert.Equal(t, 0.0, promtest.ToFloat64(sy.metrics.garbageCollectionFailures)) -// assert.Equal(t, 0.0, promtest.ToFloat64(metrics.blocksMarkedForNoCompact)) -// assert.Equal(t, 0.0, promtest.ToFloat64(metrics.groupCompactions)) -// assert.Equal(t, 0.0, promtest.ToFloat64(metrics.groupCompactionRunsStarted)) -// assert.Equal(t, 0.0, promtest.ToFloat64(metrics.groupCompactionRunsCompleted)) -// assert.Equal(t, 0.0, promtest.ToFloat64(metrics.groupCompactionRunsFailed)) - -// _, err = os.Stat(dir) -// assert.True(t, os.IsNotExist(err), "dir %s should be remove after compaction.", dir) - -// // Test label name with slash, regression: https://github.com/thanos-io/thanos/issues/1661. -// extLabels := labels.FromStrings("e1", "1/weird") -// extLabels2 := labels.FromStrings("e1", "1") -// metas := createAndUpload(t, bkt, []blockgenSpec{ -// { -// numSamples: 100, mint: 500, maxt: 1000, extLset: extLabels, res: 124, -// series: []labels.Labels{ -// labels.FromStrings("a", "1"), -// labels.FromStrings("a", "2", "b", "2"), -// labels.FromStrings("a", "3"), -// labels.FromStrings("a", "4"), -// }, -// }, -// { -// numSamples: 100, mint: 2000, maxt: 3000, extLset: extLabels, res: 124, -// series: []labels.Labels{ -// labels.FromStrings("a", "3"), -// labels.FromStrings("a", "4"), -// labels.FromStrings("a", "5"), -// labels.FromStrings("a", "6"), -// }, -// }, -// // Mix order to make sure compactor is able to deduct min time / max time. -// // Currently TSDB does not produces empty blocks (see: https://github.com/prometheus/tsdb/pull/374). However before v2.7.0 it was -// // so we still want to mimick this case as close as possible. -// { -// mint: 1000, maxt: 2000, extLset: extLabels, res: 124, -// // Empty block. -// }, -// // Due to TSDB compaction delay (not compacting fresh block), we need one more block to be pushed to trigger compaction. -// { -// numSamples: 100, mint: 3000, maxt: 4000, extLset: extLabels, res: 124, -// series: []labels.Labels{ -// labels.FromStrings("a", "7"), -// }, -// }, -// // Extra block for "distraction" for different resolution and one for different labels. -// { -// numSamples: 100, mint: 5000, maxt: 6000, extLset: labels.FromStrings("e1", "2"), res: 124, -// series: []labels.Labels{ -// labels.FromStrings("a", "7"), -// }, -// }, -// // Extra block for "distraction" for different resolution and one for different labels. -// { -// numSamples: 100, mint: 4000, maxt: 5000, extLset: extLabels, res: 0, -// series: []labels.Labels{ -// labels.FromStrings("a", "7"), -// }, -// }, -// // Second group (extLabels2). -// { -// numSamples: 100, mint: 2000, maxt: 3000, extLset: extLabels2, res: 124, -// series: []labels.Labels{ -// labels.FromStrings("a", "3"), -// labels.FromStrings("a", "4"), -// labels.FromStrings("a", "6"), -// }, -// }, -// { -// numSamples: 100, mint: 0, maxt: 1000, extLset: extLabels2, res: 124, -// series: []labels.Labels{ -// labels.FromStrings("a", "1"), -// labels.FromStrings("a", "2", "b", "2"), -// labels.FromStrings("a", "3"), -// labels.FromStrings("a", "4"), -// }, -// }, -// // Due to TSDB compaction delay (not compacting fresh block), we need one more block to be pushed to trigger compaction. -// { -// numSamples: 100, mint: 3000, maxt: 4000, extLset: extLabels2, res: 124, -// series: []labels.Labels{ -// labels.FromStrings("a", "7"), -// }, -// }, -// }, []blockgenSpec{ -// { -// numSamples: 100, mint: 0, maxt: 499, extLset: extLabels, res: 124, -// series: []labels.Labels{ -// labels.FromStrings("a", "1"), -// labels.FromStrings("a", "1", "b", "2"), -// labels.FromStrings("a", "3"), -// labels.FromStrings("a", "4"), -// }, -// }, -// }) - -// require.NoError(t, bComp.Compact(ctx, 0), 0) -// assert.Equal(t, 5.0, promtest.ToFloat64(sy.metrics.blocksMarkedForDeletion)) -// assert.Equal(t, 1.0, promtest.ToFloat64(metrics.blocksMarkedForNoCompact)) -// assert.Equal(t, 0.0, promtest.ToFloat64(sy.metrics.garbageCollectionFailures)) -// assert.Equal(t, 2.0, promtest.ToFloat64(metrics.groupCompactions)) -// assert.Equal(t, 3.0, promtest.ToFloat64(metrics.groupCompactionRunsStarted)) -// assert.Equal(t, 2.0, promtest.ToFloat64(metrics.groupCompactionRunsCompleted)) -// assert.Equal(t, 1.0, promtest.ToFloat64(metrics.groupCompactionRunsFailed)) - -// _, err = os.Stat(dir) -// assert.True(t, os.IsNotExist(err), "dir %s should be remove after compaction.", dir) - -// // Check object storage. All blocks that were included in new compacted one should be removed. New compacted ones -// // are present and looks as expected. -// nonCompactedExpected := map[ulid.ULID]bool{ -// metas[3].ULID: false, -// metas[4].ULID: false, -// metas[5].ULID: false, -// metas[8].ULID: false, -// metas[9].ULID: false, -// } -// others := map[string]block.Meta{} -// require.NoError(t, bkt.Iter(ctx, "", func(n string) error { -// id, ok := block.IsBlockDir(n) -// if !ok { -// return nil -// } - -// if _, ok := nonCompactedExpected[id]; ok { -// nonCompactedExpected[id] = true -// return nil -// } - -// meta, err := block.DownloadMeta(ctx, logger, bkt, id) -// if err != nil { -// return err -// } - -// others[DefaultGroupKey(meta.Thanos)] = meta -// return nil -// })) - -// for id, found := range nonCompactedExpected { -// assert.True(t, found, "not found expected block %s", id.String()) -// } - -// // We expect two compacted blocks only outside of what we expected in `nonCompactedExpected`. -// assert.Equal(t, 2, len(others)) -// { -// meta, ok := others[defaultGroupKey(124, extLabels)] -// assert.True(t, ok, "meta not found") - -// assert.Equal(t, int64(500), meta.MinTime) -// assert.Equal(t, int64(3000), meta.MaxTime) -// assert.Equal(t, uint64(6), meta.Stats.NumSeries) -// assert.Equal(t, uint64(2*4*100), meta.Stats.NumSamples) // Only 2 times 4*100 because one block was empty. -// assert.Equal(t, 2, meta.Compaction.Level) -// assert.Equal(t, []ulid.ULID{metas[0].ULID, metas[1].ULID, metas[2].ULID}, meta.Compaction.Sources) - -// // Check thanos meta. -// assert.True(t, labels.Equal(extLabels, labels.FromMap(meta.Thanos.Labels)), "ext labels does not match") -// assert.Equal(t, int64(124), meta.Thanos.Downsample.Resolution) -// assert.True(t, len(meta.Thanos.SegmentFiles) > 0, "compacted blocks have segment files set") -// } -// { -// meta, ok := others[defaultGroupKey(124, extLabels2)] -// assert.True(t, ok, "meta not found") - -// assert.Equal(t, int64(0), meta.MinTime) -// assert.Equal(t, int64(3000), meta.MaxTime) -// assert.Equal(t, uint64(5), meta.Stats.NumSeries) -// assert.Equal(t, uint64(2*4*100-100), meta.Stats.NumSamples) -// assert.Equal(t, 2, meta.Compaction.Level) -// assert.Equal(t, []ulid.ULID{metas[6].ULID, metas[7].ULID}, meta.Compaction.Sources) - -// // Check thanos meta. -// assert.True(t, labels.Equal(extLabels2, labels.FromMap(meta.Thanos.Labels)), "ext labels does not match") -// assert.Equal(t, int64(124), meta.Thanos.Downsample.Resolution) -// assert.True(t, len(meta.Thanos.SegmentFiles) > 0, "compacted blocks have segment files set") -// } -// }) -// } - -// type blockgenSpec struct { -// mint, maxt int64 -// series []labels.Labels -// numSamples int -// extLset labels.Labels -// res int64 -// } - -// func createAndUpload(t testing.TB, bkt objstore.Bucket, blocks []blockgenSpec, blocksWithOutOfOrderChunks []blockgenSpec) (metas []*block.Meta) { -// prepareDir := t.TempDir() - -// ctx, cancel := context.WithTimeout(context.Background(), 20*time.Second) -// defer cancel() - -// for _, b := range blocks { -// id, meta := createBlock(ctx, t, prepareDir, b) -// metas = append(metas, meta) -// require.NoError(t, block.Upload(ctx, log.NewNopLogger(), bkt, filepath.Join(prepareDir, id.String()), nil)) -// } -// for _, b := range blocksWithOutOfOrderChunks { -// id, meta := createBlock(ctx, t, prepareDir, b) - -// err := putOutOfOrderIndex(filepath.Join(prepareDir, id.String()), b.mint, b.maxt) -// require.NoError(t, err) - -// metas = append(metas, meta) -// require.NoError(t, block.Upload(ctx, log.NewNopLogger(), bkt, filepath.Join(prepareDir, id.String()), nil)) -// } - -// return metas -// } - -// func createBlock(ctx context.Context, t testing.TB, prepareDir string, b blockgenSpec) (id ulid.ULID, meta *block.Meta) { -// var err error -// if b.numSamples == 0 { -// id, err = createEmptyBlock(prepareDir, b.mint, b.maxt, b.extLset, b.res) -// } else { -// id, err = createBlockWithOptions(ctx, prepareDir, b.series, b.numSamples, b.mint, b.maxt, b.extLset, b.res, false) -// } -// require.NoError(t, err) - -// meta, err = block.ReadMetaFromDir(filepath.Join(prepareDir, id.String())) -// require.NoError(t, err) -// return -// } - -// // Regression test for Thanos issue #2459. -// func TestGarbageCollectDoesntCreateEmptyBlocksWithDeletionMarksOnly(t *testing.T) { -// logger := log.NewLogfmtLogger(os.Stderr) - -// foreachStore(t, func(t *testing.T, bkt objstore.Bucket) { -// // Use bucket with global markers to make sure that our custom filters work correctly. -// bkt = block.BucketWithGlobalMarkers(bkt) - -// ctx, cancel := context.WithTimeout(context.Background(), 120*time.Second) -// defer cancel() - -// // Generate two blocks, and then another block that covers both of them. -// var metas []*block.Meta -// var ids []ulid.ULID - -// for i := 0; i < 2; i++ { -// var m block.Meta - -// m.Version = 1 -// m.ULID = ulid.MustNew(uint64(i), nil) -// m.Compaction.Sources = []ulid.ULID{m.ULID} -// m.Compaction.Level = 1 - -// ids = append(ids, m.ULID) -// metas = append(metas, &m) -// } - -// var m1 block.Meta -// m1.Version = 1 -// m1.ULID = ulid.MustNew(100, nil) -// m1.Compaction.Level = 2 -// m1.Compaction.Sources = ids -// m1.Thanos.Downsample.Resolution = 0 - -// // Create all blocks in the bucket. -// for _, m := range append(metas, &m1) { -// fmt.Println("create", m.ULID) -// var buf bytes.Buffer -// require.NoError(t, json.NewEncoder(&buf).Encode(&m)) -// require.NoError(t, bkt.Upload(ctx, path.Join(m.ULID.String(), block.MetaFilename), &buf)) -// } - -// blocksMarkedForDeletion := promauto.With(nil).NewCounter(prometheus.CounterOpts{}) - -// duplicateBlocksFilter := NewShardAwareDeduplicateFilter() -// metaFetcher, err := block.NewMetaFetcher(nil, 32, objstore.WithNoopInstr(bkt), "", nil, []block.MetadataFilter{ -// duplicateBlocksFilter, -// }) -// require.NoError(t, err) - -// sy, err := NewMetaSyncer(nil, nil, bkt, metaFetcher, duplicateBlocksFilter, blocksMarkedForDeletion) -// require.NoError(t, err) - -// // Do one initial synchronization with the bucket. -// require.NoError(t, sy.SyncMetas(ctx)) -// require.NoError(t, sy.GarbageCollect(ctx)) - -// rem, err := listBlocksMarkedForDeletion(ctx, bkt) -// require.NoError(t, err) - -// sort.Slice(rem, func(i, j int) bool { -// return rem[i].Compare(rem[j]) < 0 -// }) - -// assert.Equal(t, ids, rem) - -// // Delete source blocks. -// for _, id := range ids { -// require.NoError(t, block.Delete(ctx, logger, bkt, id)) -// } - -// // After another garbage-collect, we should not find new blocks that are deleted with new deletion mark files. -// require.NoError(t, sy.SyncMetas(ctx)) -// require.NoError(t, sy.GarbageCollect(ctx)) - -// rem, err = listBlocksMarkedForDeletion(ctx, bkt) -// require.NoError(t, err) -// assert.Equal(t, 0, len(rem)) -// }) -// } - -// func listBlocksMarkedForDeletion(ctx context.Context, bkt objstore.Bucket) ([]ulid.ULID, error) { -// var rem []ulid.ULID -// err := bkt.Iter(ctx, "", func(n string) error { -// id, ok := block.IsBlockDir(n) -// if !ok { -// return nil -// } -// deletionMarkFile := path.Join(id.String(), block.DeletionMarkFilename) - -// exists, err := bkt.Exists(ctx, deletionMarkFile) -// if err != nil { -// return err -// } -// if exists { -// rem = append(rem, id) -// } -// return nil -// }) -// return rem, err -// } - -// func foreachStore(t *testing.T, testFn func(t *testing.T, bkt objstore.Bucket)) { -// t.Parallel() - -// // Mandatory Inmem. Not parallel, to detect problem early. -// if ok := t.Run("inmem", func(t *testing.T) { -// testFn(t, objstore.NewInMemBucket()) -// }); !ok { -// return -// } - -// // Mandatory Filesystem. -// t.Run("filesystem", func(t *testing.T) { -// t.Parallel() - -// dir := t.TempDir() - -// b, err := filesystem.NewBucket(dir) -// require.NoError(t, err) -// testFn(t, b) -// }) -// } - -// // createEmptyBlock produces empty block like it was the case before fix: https://github.com/prometheus/tsdb/pull/374. -// // (Prometheus pre v2.7.0). -// func createEmptyBlock(dir string, mint, maxt int64, extLset labels.Labels, resolution int64) (ulid.ULID, error) { -// entropy := rand.New(rand.NewSource(time.Now().UnixNano())) -// uid := ulid.MustNew(ulid.Now(), entropy) - -// if err := os.Mkdir(path.Join(dir, uid.String()), os.ModePerm); err != nil { -// return ulid.ULID{}, errors.Wrap(err, "close index") -// } - -// if err := os.Mkdir(path.Join(dir, uid.String(), "chunks"), os.ModePerm); err != nil { -// return ulid.ULID{}, errors.Wrap(err, "close index") -// } - -// w, err := index.NewWriter(context.Background(), path.Join(dir, uid.String(), "index")) -// if err != nil { -// return ulid.ULID{}, errors.Wrap(err, "new index") -// } - -// if err := w.Close(); err != nil { -// return ulid.ULID{}, errors.Wrap(err, "close index") -// } - -// m := tsdb.BlockMeta{ -// Version: 1, -// ULID: uid, -// MinTime: mint, -// MaxTime: maxt, -// Compaction: tsdb.BlockMetaCompaction{ -// Level: 1, -// Sources: []ulid.ULID{uid}, -// }, -// } -// b, err := json.Marshal(&m) -// if err != nil { -// return ulid.ULID{}, err -// } - -// if err := os.WriteFile(path.Join(dir, uid.String(), "meta.json"), b, os.ModePerm); err != nil { -// return ulid.ULID{}, errors.Wrap(err, "saving meta.json") -// } - -// if _, err = block.InjectThanosMeta(log.NewNopLogger(), filepath.Join(dir, uid.String()), block.ThanosMeta{ -// Labels: extLset.Map(), -// Downsample: block.ThanosDownsample{Resolution: resolution}, -// Source: block.TestSource, -// }, nil); err != nil { -// return ulid.ULID{}, errors.Wrap(err, "finalize block") -// } - -// return uid, nil -// } - -// func createBlockWithOptions( -// ctx context.Context, -// dir string, -// series []labels.Labels, -// numSamples int, -// mint, maxt int64, -// extLset labels.Labels, -// resolution int64, -// tombstones bool, -// ) (id ulid.ULID, err error) { -// headOpts := tsdb.DefaultHeadOptions() -// headOpts.ChunkDirRoot = filepath.Join(dir, "chunks") -// headOpts.ChunkRange = 10000000000 -// h, err := tsdb.NewHead(nil, nil, nil, nil, headOpts, nil) -// if err != nil { -// return id, errors.Wrap(err, "create head block") -// } -// defer func() { -// runutil.CloseWithErrCapture(&err, h, "TSDB Head") -// if e := os.RemoveAll(headOpts.ChunkDirRoot); e != nil { -// err = errors.Wrap(e, "delete chunks dir") -// } -// }() - -// var g errgroup.Group -// timeStepSize := (maxt - mint) / int64(numSamples+1) -// batchSize := len(series) / runtime.GOMAXPROCS(0) - -// for len(series) > 0 { -// l := batchSize -// if len(series) < 1000 { -// l = len(series) -// } -// batch := series[:l] -// series = series[l:] - -// g.Go(func() error { -// t := mint - -// for i := 0; i < numSamples; i++ { -// app := h.Appender(ctx) - -// for _, lset := range batch { -// _, err := app.Append(0, lset, t, rand.Float64()) -// if err != nil { -// if rerr := app.Rollback(); rerr != nil { -// err = errors.Wrapf(err, "rollback failed: %v", rerr) -// } - -// return errors.Wrap(err, "add sample") -// } -// } -// if err := app.Commit(); err != nil { -// return errors.Wrap(err, "commit") -// } -// t += timeStepSize -// } -// return nil -// }) -// } -// if err := g.Wait(); err != nil { -// return id, err -// } -// c, err := tsdb.NewLeveledCompactor(ctx, nil, log.NewNopLogger(), []int64{maxt - mint}, nil, nil, true) -// if err != nil { -// return id, errors.Wrap(err, "create compactor") -// } - -// id, err = c.Write(dir, h, mint, maxt, nil) -// if err != nil { -// return id, errors.Wrap(err, "write block") -// } - -// if id.Compare(ulid.ULID{}) == 0 { -// return id, errors.Errorf("nothing to write, asked for %d samples", numSamples) -// } - -// blockDir := filepath.Join(dir, id.String()) - -// if _, err = block.InjectThanosMeta(log.NewNopLogger(), blockDir, block.ThanosMeta{ -// Labels: extLset.Map(), -// Downsample: block.ThanosDownsample{Resolution: resolution}, -// Source: block.TestSource, -// Files: []block.File{}, -// }, nil); err != nil { -// return id, errors.Wrap(err, "finalize block") -// } - -// if !tombstones { -// if err = os.Remove(filepath.Join(dir, id.String(), "tombstones")); err != nil { -// return id, errors.Wrap(err, "remove tombstones") -// } -// } - -// return id, nil -// } - -// var indexFilename = "index" - -// type indexWriterSeries struct { -// labels labels.Labels -// chunks []chunks.Meta // series file offset of chunks -// } - -// type indexWriterSeriesSlice []*indexWriterSeries - -// // putOutOfOrderIndex updates the index in blockDir with an index containing an out-of-order chunk -// // copied from https://github.com/prometheus/prometheus/blob/b1ed4a0a663d0c62526312311c7529471abbc565/tsdb/index/index_test.go#L346 -// func putOutOfOrderIndex(blockDir string, minTime int64, maxTime int64) error { -// if minTime >= maxTime || minTime+4 >= maxTime { -// return fmt.Errorf("minTime must be at least 4 less than maxTime to not create overlapping chunks") -// } - -// lbls := []labels.Labels{ -// labels.FromStrings("lbl1", "1"), -// } - -// // Sort labels as the index writer expects series in sorted order. -// sort.Sort(labels.Slice(lbls)) - -// symbols := map[string]struct{}{} -// for _, lset := range lbls { -// lset.Range(func(l labels.Label) { -// symbols[l.Name] = struct{}{} -// symbols[l.Value] = struct{}{} -// }) -// } - -// var input indexWriterSeriesSlice - -// // Generate ChunkMetas for every label set. -// for _, lset := range lbls { -// var metas []chunks.Meta -// // only need two chunks that are out-of-order -// chk1 := chunks.Meta{ -// MinTime: maxTime - 2, -// MaxTime: maxTime - 1, -// Ref: chunks.ChunkRef(rand.Uint64()), -// Chunk: chunkenc.NewXORChunk(), -// } -// metas = append(metas, chk1) -// chk2 := chunks.Meta{ -// MinTime: minTime + 1, -// MaxTime: minTime + 2, -// Ref: chunks.ChunkRef(rand.Uint64()), -// Chunk: chunkenc.NewXORChunk(), -// } -// metas = append(metas, chk2) - -// input = append(input, &indexWriterSeries{ -// labels: lset, -// chunks: metas, -// }) -// } - -// iw, err := index.NewWriter(context.Background(), filepath.Join(blockDir, indexFilename)) -// if err != nil { -// return err -// } - -// syms := []string{} -// for s := range symbols { -// syms = append(syms, s) -// } -// slices.Sort(syms) -// for _, s := range syms { -// if err := iw.AddSymbol(s); err != nil { -// return err -// } -// } - -// // Population procedure as done by compaction. -// var ( -// postings = index.NewMemPostings() -// values = map[string]map[string]struct{}{} -// ) - -// for i, s := range input { -// if err := iw.AddSeries(storage.SeriesRef(i), s.labels, s.chunks...); err != nil { -// return err -// } - -// s.labels.Range(func(l labels.Label) { -// valset, ok := values[l.Name] -// if !ok { -// valset = map[string]struct{}{} -// values[l.Name] = valset -// } -// valset[l.Value] = struct{}{} -// }) -// postings.Add(storage.SeriesRef(i), s.labels) -// } - -// return iw.Close() -// } +import ( + "bytes" + "context" + "encoding/json" + "fmt" + "os" + "path" + "sort" + "strings" + "testing" + "time" + + "github.com/go-kit/log" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" + promtest "github.com/prometheus/client_golang/prometheus/testutil" + "github.com/prometheus/common/model" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/thanos-io/objstore" + + phlareobj "github.com/grafana/pyroscope/pkg/objstore" + "github.com/grafana/pyroscope/pkg/objstore/providers/filesystem" + "github.com/grafana/pyroscope/pkg/phlaredb/block" + "github.com/oklog/ulid" +) + +func TestSyncer_GarbageCollect_e2e(t *testing.T) { + foreachStore(t, func(t *testing.T, bkt phlareobj.Bucket) { + // Use bucket with global markers to make sure that our custom filters work correctly. + bkt = block.BucketWithGlobalMarkers(bkt) + + ctx, cancel := context.WithTimeout(context.Background(), 120*time.Second) + defer cancel() + + // Generate 10 source block metas and construct higher level blocks + // that are higher compactions of them. + var metas []*block.Meta + var ids []ulid.ULID + + for i := 0; i < 10; i++ { + var m block.Meta + + m.Version = 1 + m.ULID = ulid.MustNew(uint64(i), nil) + m.Compaction.Sources = []ulid.ULID{m.ULID} + m.Compaction.Level = 1 + m.MinTime = 0 + m.MaxTime = model.Time(2 * time.Hour.Milliseconds()) + + ids = append(ids, m.ULID) + metas = append(metas, &m) + } + + var m1 block.Meta + m1.Version = 1 + m1.ULID = ulid.MustNew(100, nil) + m1.Compaction.Level = 2 + m1.Compaction.Sources = ids[:4] + m1.Downsample.Resolution = 0 + + var m2 block.Meta + m2.Version = 1 + m2.ULID = ulid.MustNew(200, nil) + m2.Compaction.Level = 2 + m2.Compaction.Sources = ids[4:8] // last two source IDs is not part of a level 2 block. + m2.Downsample.Resolution = 0 + + var m3 block.Meta + m3.Version = 1 + m3.ULID = ulid.MustNew(300, nil) + m3.Compaction.Level = 3 + m3.Compaction.Sources = ids[:9] // last source ID is not part of level 3 block. + m3.Downsample.Resolution = 0 + m3.MinTime = 0 + m3.MaxTime = model.Time(2 * time.Hour.Milliseconds()) + + var m4 block.Meta + m4.Version = 1 + m4.ULID = ulid.MustNew(400, nil) + m4.Compaction.Level = 2 + m4.Compaction.Sources = ids[9:] // covers the last block but is a different resolution. Must not trigger deletion. + m4.Downsample.Resolution = 1000 + m4.MinTime = 0 + m4.MaxTime = model.Time(2 * time.Hour.Milliseconds()) + + var m5 block.Meta + m5.Version = 1 + m5.ULID = ulid.MustNew(500, nil) + m5.Compaction.Level = 2 + m5.Compaction.Sources = ids[8:9] // built from block 8, but different resolution. Block 8 is already included in m3, can be deleted. + m5.Downsample.Resolution = 1000 + m5.MinTime = 0 + m5.MaxTime = model.Time(2 * time.Hour.Milliseconds()) + + // Create all blocks in the bucket. + for _, m := range append(metas, &m1, &m2, &m3, &m4, &m5) { + fmt.Println("create", m.ULID) + var buf bytes.Buffer + require.NoError(t, json.NewEncoder(&buf).Encode(&m)) + require.NoError(t, bkt.Upload(ctx, path.Join(m.ULID.String(), block.MetaFilename), &buf)) + } + + duplicateBlocksFilter := NewShardAwareDeduplicateFilter() + metaFetcher, err := block.NewMetaFetcher(nil, 32, bkt, "", nil, []block.MetadataFilter{ + duplicateBlocksFilter, + }) + require.NoError(t, err) + + blocksMarkedForDeletion := promauto.With(nil).NewCounter(prometheus.CounterOpts{}) + sy, err := NewMetaSyncer(nil, nil, bkt, metaFetcher, duplicateBlocksFilter, blocksMarkedForDeletion) + require.NoError(t, err) + + // Do one initial synchronization with the bucket. + require.NoError(t, sy.SyncMetas(ctx)) + require.NoError(t, sy.GarbageCollect(ctx)) + + var rem []ulid.ULID + err = bkt.Iter(ctx, "", func(n string) error { + id, ok := block.IsBlockDir(n) + if !ok { + return nil + } + deletionMarkFile := path.Join(id.String(), block.DeletionMarkFilename) + + exists, err := bkt.Exists(ctx, deletionMarkFile) + if err != nil { + return err + } + if !exists { + rem = append(rem, id) + } + return nil + }) + require.NoError(t, err) + + sort.Slice(rem, func(i, j int) bool { + return rem[i].Compare(rem[j]) < 0 + }) + + // Only the level 3 block, the last source block in both resolutions should be left. + assert.Equal(t, []ulid.ULID{metas[9].ULID, m3.ULID, m4.ULID, m5.ULID}, rem) + + // After another sync the changes should also be reflected in the local groups. + require.NoError(t, sy.SyncMetas(ctx)) + require.NoError(t, sy.GarbageCollect(ctx)) + + // Only the level 3 block, the last source block in both resolutions should be left. + grouper := NewSplitAndMergeGrouper("user-1", []int64{2 * time.Hour.Milliseconds()}, 0, 0, log.NewNopLogger()) + groups, err := grouper.Groups(sy.Metas()) + require.NoError(t, err) + + assert.Equal(t, "0@17241709254077376921-merge--0-7200000", groups[0].Key()) + assert.Equal(t, []ulid.ULID{metas[9].ULID, m3.ULID}, groups[0].IDs()) + + assert.Equal(t, "1000@17241709254077376921-merge--0-7200000", groups[1].Key()) + assert.Equal(t, []ulid.ULID{m4.ULID, m5.ULID}, groups[1].IDs()) + }) +} + +func TestGroupCompactE2E(t *testing.T) { + foreachStore(t, func(t *testing.T, bkt phlareobj.Bucket) { + // Use bucket with global markers to make sure that our custom filters work correctly. + bkt = block.BucketWithGlobalMarkers(bkt) + + ctx, cancel := context.WithTimeout(context.Background(), 120*time.Second) + defer cancel() + + // Create fresh, empty directory for actual test. + dir := t.TempDir() + + // Start dir checker... we make sure that "dir" only contains group subdirectories during compaction, + // and not any block directories. Dir checker stops when context is canceled, or on first error, + // in which case error is logger and test is failed. (We cannot use Fatal or FailNow from a goroutine). + go func() { + for ctx.Err() == nil { + fs, err := os.ReadDir(dir) + if err != nil && !os.IsNotExist(err) { + t.Log("error while listing directory", dir) + t.Fail() + return + } + + for _, fi := range fs { + // Suffix used by Prometheus LeveledCompactor when doing compaction. + toCheck := strings.TrimSuffix(fi.Name(), ".tmp-for-creation") + + _, err := ulid.Parse(toCheck) + if err == nil { + t.Log("found block directory in main compaction directory", fi.Name()) + t.Fail() + return + } + } + + select { + case <-time.After(100 * time.Millisecond): + continue + case <-ctx.Done(): + return + } + } + }() + + logger := log.NewLogfmtLogger(os.Stderr) + + duplicateBlocksFilter := NewShardAwareDeduplicateFilter() + noCompactMarkerFilter := NewNoCompactionMarkFilter(bkt, true) + metaFetcher, err := block.NewMetaFetcher(nil, 32, bkt, "", nil, []block.MetadataFilter{ + duplicateBlocksFilter, + noCompactMarkerFilter, + }) + require.NoError(t, err) + + blocksMarkedForDeletion := promauto.With(nil).NewCounter(prometheus.CounterOpts{}) + sy, err := NewMetaSyncer(nil, nil, bkt, metaFetcher, duplicateBlocksFilter, blocksMarkedForDeletion) + require.NoError(t, err) + + planner := NewSplitAndMergePlanner([]int64{1000, 3000}) + grouper := NewSplitAndMergeGrouper("user-1", []int64{1000, 3000}, 0, 0, logger) + metrics := NewBucketCompactorMetrics(blocksMarkedForDeletion, prometheus.NewPedanticRegistry()) + bComp, err := NewBucketCompactor(logger, sy, grouper, planner, &BlockCompactor{ + blockOpenConcurrency: 100, + logger: logger, + }, dir, bkt, 2, ownAllJobs, sortJobsByNewestBlocksFirst, 0, 4, metrics) + require.NoError(t, err) + + // Compaction on empty should not fail. + require.NoError(t, bComp.Compact(ctx, 0), 0) + assert.Equal(t, 0.0, promtest.ToFloat64(sy.metrics.blocksMarkedForDeletion)) + assert.Equal(t, 0.0, promtest.ToFloat64(sy.metrics.garbageCollectionFailures)) + assert.Equal(t, 0.0, promtest.ToFloat64(metrics.blocksMarkedForNoCompact)) + assert.Equal(t, 0.0, promtest.ToFloat64(metrics.groupCompactions)) + assert.Equal(t, 0.0, promtest.ToFloat64(metrics.groupCompactionRunsStarted)) + assert.Equal(t, 0.0, promtest.ToFloat64(metrics.groupCompactionRunsCompleted)) + assert.Equal(t, 0.0, promtest.ToFloat64(metrics.groupCompactionRunsFailed)) + + _, err = os.Stat(dir) + assert.True(t, os.IsNotExist(err), "dir %s should be remove after compaction.", dir) + + m1 := createDBBlock(t, bkt, "", 500, 1000, 10, nil) + m2 := createDBBlock(t, bkt, "", 500, 1000, 10, nil) + + m3 := createDBBlock(t, bkt, "", 1001, 2000, 10, nil) + m4 := createDBBlock(t, bkt, "", 1001, 3000, 10, nil) + + require.NoError(t, bComp.Compact(ctx, 0), 0) + assert.Equal(t, 6.0, promtest.ToFloat64(sy.metrics.blocksMarkedForDeletion)) + assert.Equal(t, 0.0, promtest.ToFloat64(metrics.blocksMarkedForNoCompact)) + assert.Equal(t, 0.0, promtest.ToFloat64(sy.metrics.garbageCollectionFailures)) + assert.Equal(t, 3.0, promtest.ToFloat64(metrics.groupCompactions)) + assert.Equal(t, 3.0, promtest.ToFloat64(metrics.groupCompactionRunsStarted)) + assert.Equal(t, 3.0, promtest.ToFloat64(metrics.groupCompactionRunsCompleted)) + assert.Equal(t, 0.0, promtest.ToFloat64(metrics.groupCompactionRunsFailed)) + + _, err = os.Stat(dir) + assert.True(t, os.IsNotExist(err), "dir %s should be remove after compaction.", dir) + + metas, _, err := metaFetcher.FetchWithoutMarkedForDeletion(context.Background()) + require.NoError(t, err) + require.Len(t, metas, 1) + var meta block.Meta + for _, m := range metas { + meta = *m + } + require.Equal(t, []ulid.ULID{m1, m2, m3, m4}, meta.Compaction.Sources) + require.Equal(t, 3, meta.Compaction.Level) + require.Equal(t, model.Time(500), meta.MinTime) + require.Equal(t, model.Time(3000), meta.MaxTime) + }) +} + +func foreachStore(t *testing.T, testFn func(t *testing.T, bkt phlareobj.Bucket)) { + t.Parallel() + + // Mandatory Inmem. Not parallel, to detect problem early. + if ok := t.Run("inmem", func(t *testing.T) { + testFn(t, phlareobj.NewBucket(objstore.NewInMemBucket())) + }); !ok { + return + } + + // Mandatory Filesystem. + t.Run("filesystem", func(t *testing.T) { + t.Parallel() + + dir := t.TempDir() + + b, err := filesystem.NewBucket(dir) + require.NoError(t, err) + testFn(t, b) + }) +} diff --git a/pkg/compactor/tenant_deletion_api.go b/pkg/compactor/tenant_deletion_api.go deleted file mode 100644 index 489092945e..0000000000 --- a/pkg/compactor/tenant_deletion_api.go +++ /dev/null @@ -1,98 +0,0 @@ -// SPDX-License-Identifier: AGPL-3.0-only -// Provenance-includes-location: https://github.com/grafana/mimir/blob/main/pkg/compactor/tenant_deletion_api.go -// Provenance-includes-license: Apache-2.0 -// Provenance-includes-copyright: The Cortex Authors. - -package compactor - -// import ( -// "context" -// "net/http" -// "strings" -// "time" - -// "github.com/go-kit/log/level" -// "github.com/grafana/dskit/tenant" -// "github.com/oklog/ulid" -// "github.com/pkg/errors" - -// "github.com/grafana/mimir/pkg/storage/bucket" -// mimir_tsdb "github.com/grafana/mimir/pkg/storage/tsdb" -// "github.com/grafana/mimir/pkg/util" -// ) - -// func (c *MultitenantCompactor) DeleteTenant(w http.ResponseWriter, r *http.Request) { -// ctx := r.Context() -// userID, err := tenant.TenantID(ctx) -// if err != nil { -// // When Mimir is running, it uses Auth Middleware for checking X-Scope-OrgID and injecting tenant into context. -// // Auth Middleware sends http.StatusUnauthorized if X-Scope-OrgID is missing, so we do too here, for consistency. -// http.Error(w, err.Error(), http.StatusUnauthorized) -// return -// } - -// err = mimir_tsdb.WriteTenantDeletionMark(r.Context(), c.bucketClient, userID, c.cfgProvider, mimir_tsdb.NewTenantDeletionMark(time.Now())) -// if err != nil { -// level.Error(c.logger).Log("msg", "failed to write tenant deletion mark", "tenant", userID, "err", err) - -// http.Error(w, err.Error(), http.StatusInternalServerError) -// return -// } - -// level.Info(c.logger).Log("msg", "tenant deletion mark in blocks storage created", "tenant", userID) - -// w.WriteHeader(http.StatusOK) -// } - -// type DeleteTenantStatusResponse struct { -// TenantID string `json:"tenant_id"` -// BlocksDeleted bool `json:"blocks_deleted"` -// } - -// func (c *MultitenantCompactor) DeleteTenantStatus(w http.ResponseWriter, r *http.Request) { -// ctx := r.Context() -// userID, err := tenant.TenantID(ctx) -// if err != nil { -// http.Error(w, err.Error(), http.StatusBadRequest) -// return -// } - -// result := DeleteTenantStatusResponse{} -// result.TenantID = userID -// result.BlocksDeleted, err = c.isBlocksForUserDeleted(ctx, userID) -// if err != nil { -// http.Error(w, err.Error(), http.StatusInternalServerError) -// return -// } - -// util.WriteJSONResponse(w, result) -// } - -// func (c *MultitenantCompactor) isBlocksForUserDeleted(ctx context.Context, userID string) (bool, error) { -// errBlockFound := errors.New("block found") - -// userBucket := bucket.NewUserBucketClient(userID, c.bucketClient, c.cfgProvider) -// err := userBucket.Iter(ctx, "", func(s string) error { -// s = strings.TrimSuffix(s, "/") - -// _, err := ulid.Parse(s) -// if err != nil { -// // not block, keep looking -// return nil -// } - -// // Used as shortcut to stop iteration. -// return errBlockFound -// }) - -// if errors.Is(err, errBlockFound) { -// return false, nil -// } - -// if err != nil { -// return false, err -// } - -// // No blocks found, all good. -// return true, nil -// } diff --git a/pkg/compactor/tenant_deletion_api_test.go b/pkg/compactor/tenant_deletion_api_test.go deleted file mode 100644 index cf961e0c47..0000000000 --- a/pkg/compactor/tenant_deletion_api_test.go +++ /dev/null @@ -1,100 +0,0 @@ -// SPDX-License-Identifier: AGPL-3.0-only -// Provenance-includes-location: https://github.com/grafana/mimir/blob/main/pkg/compactor/tenant_deletion_api_test.go -// Provenance-includes-license: Apache-2.0 -// Provenance-includes-copyright: The Cortex Authors. -package compactor - -// import ( -// "bytes" -// "context" -// "net/http" -// "net/http/httptest" -// "path" -// "testing" - -// "github.com/grafana/dskit/services" -// "github.com/grafana/dskit/user" -// "github.com/stretchr/testify/require" -// "github.com/thanos-io/objstore" - -// "github.com/grafana/mimir/pkg/storage/tsdb" -// ) - -// func TestDeleteTenant(t *testing.T) { -// bkt := objstore.NewInMemBucket() -// cfg := prepareConfig(t) -// c, _, _, _, _ := prepare(t, cfg, bkt) -// require.NoError(t, services.StartAndAwaitRunning(context.Background(), c)) -// t.Cleanup(stopServiceFn(t, c)) - -// { -// resp := httptest.NewRecorder() -// c.DeleteTenant(resp, &http.Request{}) -// require.Equal(t, http.StatusUnauthorized, resp.Code) -// } - -// { -// ctx := context.Background() -// ctx = user.InjectOrgID(ctx, "fake") - -// req := &http.Request{} -// resp := httptest.NewRecorder() -// c.DeleteTenant(resp, req.WithContext(ctx)) - -// require.Equal(t, http.StatusOK, resp.Code) -// objs := bkt.Objects() -// require.NotNil(t, objs[path.Join("fake", tsdb.TenantDeletionMarkPath)]) -// } -// } - -// func TestDeleteTenantStatus(t *testing.T) { -// const username = "user" - -// for name, tc := range map[string]struct { -// objects map[string][]byte -// expectedBlocksDeleted bool -// }{ -// "empty": { -// objects: nil, -// expectedBlocksDeleted: true, -// }, - -// "no user objects": { -// objects: map[string][]byte{ -// "different-user/01EQK4QKFHVSZYVJ908Y7HH9E0/meta.json": []byte("data"), -// }, -// expectedBlocksDeleted: true, -// }, - -// "non-block files": { -// objects: map[string][]byte{ -// "user/deletion-mark.json": []byte("data"), -// }, -// expectedBlocksDeleted: true, -// }, - -// "block files": { -// objects: map[string][]byte{ -// "user/01EQK4QKFHVSZYVJ908Y7HH9E0/meta.json": []byte("data"), -// }, -// expectedBlocksDeleted: false, -// }, -// } { -// t.Run(name, func(t *testing.T) { -// bkt := objstore.NewInMemBucket() -// // "upload" objects -// for objName, data := range tc.objects { -// require.NoError(t, bkt.Upload(context.Background(), objName, bytes.NewReader(data))) -// } - -// cfg := prepareConfig(t) -// c, _, _, _, _ := prepare(t, cfg, bkt) -// require.NoError(t, services.StartAndAwaitRunning(context.Background(), c)) -// t.Cleanup(stopServiceFn(t, c)) - -// res, err := c.isBlocksForUserDeleted(context.Background(), username) -// require.NoError(t, err) -// require.Equal(t, tc.expectedBlocksDeleted, res) -// }) -// } -// } From d2226c106b6656973d46a96a7af949fae5fdc258 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Fri, 29 Sep 2023 09:28:16 +0200 Subject: [PATCH 44/74] lint --- pkg/compactor/bucket_compactor_e2e_test.go | 2 +- pkg/compactor/compactor_test.go | 61 ++++++---------------- pkg/phlaredb/validate.go | 1 + pkg/phlaredb/validate_test.go | 3 +- pkg/validation/limits.go | 3 +- 5 files changed, 21 insertions(+), 49 deletions(-) diff --git a/pkg/compactor/bucket_compactor_e2e_test.go b/pkg/compactor/bucket_compactor_e2e_test.go index b21c8c0fe9..8e70563f33 100644 --- a/pkg/compactor/bucket_compactor_e2e_test.go +++ b/pkg/compactor/bucket_compactor_e2e_test.go @@ -18,6 +18,7 @@ import ( "time" "github.com/go-kit/log" + "github.com/oklog/ulid" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" promtest "github.com/prometheus/client_golang/prometheus/testutil" @@ -29,7 +30,6 @@ import ( phlareobj "github.com/grafana/pyroscope/pkg/objstore" "github.com/grafana/pyroscope/pkg/objstore/providers/filesystem" "github.com/grafana/pyroscope/pkg/phlaredb/block" - "github.com/oklog/ulid" ) func TestSyncer_GarbageCollect_e2e(t *testing.T) { diff --git a/pkg/compactor/compactor_test.go b/pkg/compactor/compactor_test.go index 7143a9b2a7..6791a46b7e 100644 --- a/pkg/compactor/compactor_test.go +++ b/pkg/compactor/compactor_test.go @@ -32,10 +32,7 @@ import ( "github.com/prometheus/client_golang/prometheus" prom_testutil "github.com/prometheus/client_golang/prometheus/testutil" "github.com/prometheus/common/model" - "github.com/prometheus/prometheus/model/histogram" "github.com/prometheus/prometheus/tsdb" - "github.com/prometheus/prometheus/tsdb/chunkenc" - "github.com/prometheus/prometheus/tsdb/tsdbutil" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/mock" "github.com/stretchr/testify/require" @@ -51,6 +48,11 @@ import ( "github.com/grafana/pyroscope/pkg/validation" ) +const ( + instanceID = "compactor-1" + addr = "1.2.3.4" +) + func TestConfig_ShouldSupportYamlConfig(t *testing.T) { yamlCfg := ` block_ranges: [2h, 48h] @@ -1034,8 +1036,8 @@ func TestMultitenantCompactor_ShouldCompactAllUsersOnShardingEnabledButOnlyOneIn t.Cleanup(func() { assert.NoError(t, closer.Close()) }) cfg := prepareConfig(t) - cfg.ShardingRing.Common.InstanceID = "compactor-1" - cfg.ShardingRing.Common.InstanceAddr = "1.2.3.4" + cfg.ShardingRing.Common.InstanceID = instanceID + cfg.ShardingRing.Common.InstanceAddr = addr cfg.ShardingRing.Common.KVStore.Mock = ringStore c, _, tsdbPlanner, logs, registry := prepare(t, cfg, bucketClient) @@ -1237,8 +1239,8 @@ func TestMultitenantCompactor_ShouldSkipCompactionForJobsNoMoreOwnedAfterPlannin cfg := prepareConfig(t) cfg.CompactionConcurrency = 1 - cfg.ShardingRing.Common.InstanceID = "compactor-1" - cfg.ShardingRing.Common.InstanceAddr = "1.2.3.4" + cfg.ShardingRing.Common.InstanceID = instanceID + cfg.ShardingRing.Common.InstanceAddr = addr cfg.ShardingRing.Common.KVStore.Mock = ringStore limits := newMockConfigProvider() @@ -1479,7 +1481,7 @@ func createCustomBlock(t *testing.T, bkt objstore.Bucket, userID string, externa func createDBBlock(t *testing.T, bkt objstore.Bucket, userID string, minT, maxT int64, numSeries int, externalLabels map[string]string) ulid.ULID { return createCustomBlock(t, bkt, userID, externalLabels, func() []*testhelper.ProfileBuilder { result := []*testhelper.ProfileBuilder{} - appendSample := func(seriesID int, ts int64, value float64) { + appendSample := func(seriesID int, ts int64) { profile := testhelper.NewProfileBuilder(ts*int64(time.Millisecond)). CPUProfile(). WithLabels( @@ -1494,14 +1496,14 @@ func createDBBlock(t *testing.T, bkt objstore.Bucket, userID string, minT, maxT // Since we append one more series below, here we create N-1 series. if numSeries > 1 { for ts := minT; ts <= maxT; ts += (maxT - minT) / int64(numSeries-1) { - appendSample(seriesID, ts, float64(seriesID)) + appendSample(seriesID, ts) seriesID++ } } else { - appendSample(seriesID, minT, float64(seriesID)) + appendSample(seriesID, minT) } // Guarantee a series with a sample at time maxT - appendSample(seriesID, maxT, float64(seriesID)) + appendSample(seriesID, maxT) return result }) } @@ -1848,8 +1850,8 @@ func TestMultitenantCompactor_ShouldFailCompactionOnTimeout(t *testing.T) { t.Cleanup(func() { assert.NoError(t, closer.Close()) }) cfg := prepareConfig(t) - cfg.ShardingRing.Common.InstanceID = "compactor-1" - cfg.ShardingRing.Common.InstanceAddr = "1.2.3.4" + cfg.ShardingRing.Common.InstanceID = instanceID + cfg.ShardingRing.Common.InstanceAddr = addr cfg.ShardingRing.Common.KVStore.Mock = ringStore // Set ObservePeriod to longer than the timeout period to mock a timeout while waiting on ring to become ACTIVE @@ -2008,32 +2010,6 @@ func stopServiceFn(t *testing.T, serv services.Service) func() { } } -type sample struct { - t int64 - v float64 - h *histogram.Histogram - fh *histogram.FloatHistogram -} - -func newSample(t int64, v float64, h *histogram.Histogram, fh *histogram.FloatHistogram) tsdbutil.Sample { - return sample{t, v, h, fh} -} -func (s sample) T() int64 { return s.t } -func (s sample) F() float64 { return s.v } -func (s sample) H() *histogram.Histogram { return s.h } -func (s sample) FH() *histogram.FloatHistogram { return s.fh } - -func (s sample) Type() chunkenc.ValueType { - switch { - case s.h != nil: - return chunkenc.ValHistogram - case s.fh != nil: - return chunkenc.ValFloatHistogram - default: - return chunkenc.ValFloat - } -} - type bucketWithMockedAttributes struct { objstore.Bucket @@ -2047,10 +2023,3 @@ func (b *bucketWithMockedAttributes) Attributes(ctx context.Context, name string return b.Bucket.Attributes(ctx, name) } - -func must[T any](v T, err error) T { - if err != nil { - panic(err) - } - return v -} diff --git a/pkg/phlaredb/validate.go b/pkg/phlaredb/validate.go index 9c671dc0eb..1b4206df09 100644 --- a/pkg/phlaredb/validate.go +++ b/pkg/phlaredb/validate.go @@ -5,6 +5,7 @@ import ( "path" "github.com/grafana/dskit/runutil" + "github.com/grafana/pyroscope/pkg/objstore/client" "github.com/grafana/pyroscope/pkg/objstore/providers/filesystem" "github.com/grafana/pyroscope/pkg/phlaredb/block" diff --git a/pkg/phlaredb/validate_test.go b/pkg/phlaredb/validate_test.go index c57d7a209e..1237e979d9 100644 --- a/pkg/phlaredb/validate_test.go +++ b/pkg/phlaredb/validate_test.go @@ -6,11 +6,12 @@ import ( "path" "testing" + "github.com/stretchr/testify/require" + "github.com/grafana/pyroscope/pkg/phlaredb" "github.com/grafana/pyroscope/pkg/phlaredb/block" "github.com/grafana/pyroscope/pkg/phlaredb/block/testutil" "github.com/grafana/pyroscope/pkg/pprof/testhelper" - "github.com/stretchr/testify/require" ) func Test_ValidateBlock(t *testing.T) { diff --git a/pkg/validation/limits.go b/pkg/validation/limits.go index bf8378196b..4c956fd7df 100644 --- a/pkg/validation/limits.go +++ b/pkg/validation/limits.go @@ -6,10 +6,11 @@ import ( "fmt" "time" - "github.com/grafana/pyroscope/pkg/phlaredb/block" "github.com/pkg/errors" "github.com/prometheus/common/model" "gopkg.in/yaml.v3" + + "github.com/grafana/pyroscope/pkg/phlaredb/block" ) const ( From 19ddfccfbb517220a5f42461f312ca80b6f7d517 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Fri, 29 Sep 2023 14:42:51 +0200 Subject: [PATCH 45/74] Fixes a race detected when generating ULID --- pkg/phlaredb/block/metadata.go | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/pkg/phlaredb/block/metadata.go b/pkg/phlaredb/block/metadata.go index f55f63c02c..29e5075f0c 100644 --- a/pkg/phlaredb/block/metadata.go +++ b/pkg/phlaredb/block/metadata.go @@ -1,11 +1,11 @@ package block import ( + "crypto/rand" "encoding/json" "fmt" "io" "math" - "math/rand" "os" "path/filepath" "time" @@ -168,10 +168,8 @@ func (m *Meta) Clone() *Meta { return &clone } -var ulidEntropy = rand.New(rand.NewSource(time.Now().UnixNano())) - func generateULID() ulid.ULID { - return ulid.MustNew(ulid.Timestamp(time.Now()), ulidEntropy) + return ulid.MustNew(ulid.Timestamp(time.Now()), rand.Reader) } func NewMeta() *Meta { From 237cf4f19434b9ee5ec65649b006f988f036c4b6 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Fri, 29 Sep 2023 16:20:22 +0200 Subject: [PATCH 46/74] Add the compactor compenent --- cmd/pyroscope/help-all.txt.tmpl | 118 ++++++ cmd/pyroscope/help.txt.tmpl | 26 ++ .../index.md | 338 ++++++++++++++++++ pkg/api/api.go | 9 + pkg/compactor/compactor.go | 2 +- pkg/phlare/modules.go | 18 +- pkg/phlare/phlare.go | 27 +- 7 files changed, 526 insertions(+), 12 deletions(-) diff --git a/cmd/pyroscope/help-all.txt.tmpl b/cmd/pyroscope/help-all.txt.tmpl index 31d67fb139..f866c9d757 100644 --- a/cmd/pyroscope/help-all.txt.tmpl +++ b/cmd/pyroscope/help-all.txt.tmpl @@ -11,6 +11,124 @@ Usage of ./pyroscope: How frequently to scan the bucket, or to refresh the bucket index (if enabled), in order to look for changes (new blocks shipped by ingesters and blocks deleted by retention or compaction). (default 15m0s) -blocks-storage.bucket-store.tenant-sync-concurrency int Maximum number of concurrent tenants synching blocks. (default 10) + -compactor.block-ranges value + List of compaction time ranges. (default 4h0m0s) + -compactor.block-sync-concurrency int + Number of Go routines to use when downloading blocks for compaction and uploading resulting blocks. (default 8) + -compactor.blocks-retention-period duration + Delete blocks containing samples older than the specified retention period. 0 to disable. + -compactor.cleanup-concurrency int + Max number of tenants for which blocks cleanup and maintenance should run concurrently. (default 20) + -compactor.cleanup-interval duration + How frequently compactor should run blocks cleanup and maintenance, as well as update the bucket index. (default 15m0s) + -compactor.compaction-concurrency int + Max number of concurrent compactions running. (default 1) + -compactor.compaction-interval duration + The frequency at which the compaction runs (default 1h0m0s) + -compactor.compaction-jobs-order string + The sorting to use when deciding which compaction jobs should run first for a given tenant. Supported values are: smallest-range-oldest-blocks-first, newest-blocks-first. (default "smallest-range-oldest-blocks-first") + -compactor.compaction-retries int + How many times to retry a failed compaction within a single compaction run. (default 3) + -compactor.compactor-tenant-shard-size int + Max number of compactors that can compact blocks for single tenant. 0 to disable the limit and use all compactors. + -compactor.data-dir string + Directory to temporarily store blocks during compaction. This directory is not required to be persisted between restarts. (default "./data-compactor/") + -compactor.deletion-delay duration + Time before a block marked for deletion is deleted from bucket. If not 0, blocks will be marked for deletion and compactor component will permanently delete blocks marked for deletion from the bucket. If 0, blocks will be deleted straight away. Note that deleting blocks immediately can cause query failures. (default 12h0m0s) + -compactor.disabled-tenants comma-separated-list-of-strings + Comma separated list of tenants that cannot be compacted by this compactor. If specified, and compactor would normally pick given tenant for compaction (via -compactor.enabled-tenants or sharding), it will be ignored instead. + -compactor.enabled-tenants comma-separated-list-of-strings + Comma separated list of tenants that can be compacted. If specified, only these tenants will be compacted by compactor, otherwise all tenants can be compacted. Subject to sharding. + -compactor.first-level-compaction-wait-period duration + How long the compactor waits before compacting first-level blocks that are uploaded by the ingesters. This configuration option allows for the reduction of cases where the compactor begins to compact blocks before all ingesters have uploaded their blocks to the storage. (default 25m0s) + -compactor.max-compaction-time duration + Max time for starting compactions for a single tenant. After this time no new compactions for the tenant are started before next compaction cycle. This can help in multi-tenant environments to avoid single tenant using all compaction time, but also in single-tenant environments to force new discovery of blocks more often. 0 = disabled. (default 1h0m0s) + -compactor.max-opening-blocks-concurrency int + Number of goroutines opening blocks before compaction. (default 1) + -compactor.meta-sync-concurrency int + Number of Go routines to use when syncing block meta files from the long term storage. (default 20) + -compactor.no-blocks-file-cleanup-enabled + [experimental] If enabled, will delete the bucket-index, markers and debug files in the tenant bucket when there are no blocks left in the index. + -compactor.partial-block-deletion-delay duration + If a partial block (unfinished block without meta.json file) hasn't been modified for this time, it will be marked for deletion. The minimum accepted value is 4h0m0s: a lower value will be ignored and the feature disabled. 0 to disable. (default 1d) + -compactor.ring.consul.acl-token string + ACL Token used to interact with Consul. + -compactor.ring.consul.cas-retry-delay duration + Maximum duration to wait before retrying a Compare And Swap (CAS) operation. (default 1s) + -compactor.ring.consul.client-timeout duration + HTTP timeout when talking to Consul (default 20s) + -compactor.ring.consul.consistent-reads + Enable consistent reads to Consul. + -compactor.ring.consul.hostname string + Hostname and port of Consul. (default "localhost:8500") + -compactor.ring.consul.watch-burst-size int + Burst size used in rate limit. Values less than 1 are treated as 1. (default 1) + -compactor.ring.consul.watch-rate-limit float + Rate limit when watching key or prefix in Consul, in requests per second. 0 disables the rate limit. (default 1) + -compactor.ring.etcd.dial-timeout duration + The dial timeout for the etcd connection. (default 10s) + -compactor.ring.etcd.endpoints string + The etcd endpoints to connect to. + -compactor.ring.etcd.max-retries int + The maximum number of retries to do for failed ops. (default 10) + -compactor.ring.etcd.password string + Etcd password. + -compactor.ring.etcd.tls-ca-path string + Path to the CA certificates to validate server certificate against. If not set, the host's root CA certificates are used. + -compactor.ring.etcd.tls-cert-path string + Path to the client certificate, which will be used for authenticating with the server. Also requires the key path to be configured. + -compactor.ring.etcd.tls-cipher-suites string + Override the default cipher suite list (separated by commas). + -compactor.ring.etcd.tls-enabled + Enable TLS. + -compactor.ring.etcd.tls-insecure-skip-verify + Skip validating server certificate. + -compactor.ring.etcd.tls-key-path string + Path to the key for the client certificate. Also requires the client certificate to be configured. + -compactor.ring.etcd.tls-min-version string + Override the default minimum TLS version. Allowed values: VersionTLS10, VersionTLS11, VersionTLS12, VersionTLS13 + -compactor.ring.etcd.tls-server-name string + Override the expected name on the server certificate. + -compactor.ring.etcd.username string + Etcd username. + -compactor.ring.heartbeat-period duration + Period at which to heartbeat to the ring. 0 = disabled. (default 15s) + -compactor.ring.heartbeat-timeout duration + The heartbeat timeout after which compactors are considered unhealthy within the ring. 0 = never (timeout disabled). (default 1m0s) + -compactor.ring.instance-addr string + IP address to advertise in the ring. Default is auto-detected. + -compactor.ring.instance-enable-ipv6 + Enable using a IPv6 instance address. (default false) + -compactor.ring.instance-id string + Instance ID to register in the ring. (default "") + -compactor.ring.instance-interface-names string + List of network interface names to look up when finding the instance IP address. (default []) + -compactor.ring.instance-port int + Port to advertise in the ring (defaults to -server.http-listen-port). + -compactor.ring.multi.mirror-enabled + Mirror writes to secondary store. + -compactor.ring.multi.mirror-timeout duration + Timeout for storing value to secondary store. (default 2s) + -compactor.ring.multi.primary string + Primary backend storage used by multi-client. + -compactor.ring.multi.secondary string + Secondary backend storage used by multi-client. + -compactor.ring.prefix string + The prefix for the keys in the store. Should end with a /. (default "collectors/") + -compactor.ring.store string + Backend storage to use for the ring. Supported values are: consul, etcd, inmemory, memberlist, multi. (default "memberlist") + -compactor.ring.wait-active-instance-timeout duration + Timeout for waiting on compactor to become ACTIVE in the ring. (default 10m0s) + -compactor.ring.wait-stability-max-duration duration + Maximum time to wait for ring stability at startup. If the compactor ring keeps changing after this period of time, the compactor will start anyway. (default 5m0s) + -compactor.ring.wait-stability-min-duration duration + Minimum time to wait for ring stability at startup. 0 to disable. + -compactor.split-and-merge-shards int + The number of shards to use when splitting blocks. 0 to disable splitting. + -compactor.split-groups int + Number of groups that blocks for splitting should be grouped into. Each group of blocks is then split separately. Number of output split shards is controlled by -compactor.split-and-merge-shards. (default 1) + -compactor.tenant-cleanup-delay duration + For tenants marked for deletion, this is time between deleting of last block, and doing final cleanup (marker files, debug files) of the tenant. (default 6h0m0s) -config.expand-env Expands ${var} in config according to the values of the environment variables. -config.file string diff --git a/cmd/pyroscope/help.txt.tmpl b/cmd/pyroscope/help.txt.tmpl index b3160d21a3..4b7c0edcf7 100644 --- a/cmd/pyroscope/help.txt.tmpl +++ b/cmd/pyroscope/help.txt.tmpl @@ -5,6 +5,32 @@ Usage of ./pyroscope: When set to true, incoming HTTP requests must specify tenant ID in HTTP X-Scope-OrgId header. When set to false, tenant ID anonymous is used instead. -blocks-storage.bucket-store.sync-dir string Directory to store synchronized pyroscope block headers. This directory is not required to be persisted between restarts, but it's highly recommended in order to improve the store-gateway startup time. (default "./data/pyroscope-sync/") + -compactor.blocks-retention-period duration + Delete blocks containing samples older than the specified retention period. 0 to disable. + -compactor.compactor-tenant-shard-size int + Max number of compactors that can compact blocks for single tenant. 0 to disable the limit and use all compactors. + -compactor.data-dir string + Directory to temporarily store blocks during compaction. This directory is not required to be persisted between restarts. (default "./data-compactor/") + -compactor.first-level-compaction-wait-period duration + How long the compactor waits before compacting first-level blocks that are uploaded by the ingesters. This configuration option allows for the reduction of cases where the compactor begins to compact blocks before all ingesters have uploaded their blocks to the storage. (default 25m0s) + -compactor.partial-block-deletion-delay duration + If a partial block (unfinished block without meta.json file) hasn't been modified for this time, it will be marked for deletion. The minimum accepted value is 4h0m0s: a lower value will be ignored and the feature disabled. 0 to disable. (default 1d) + -compactor.ring.consul.hostname string + Hostname and port of Consul. (default "localhost:8500") + -compactor.ring.etcd.endpoints string + The etcd endpoints to connect to. + -compactor.ring.etcd.password string + Etcd password. + -compactor.ring.etcd.username string + Etcd username. + -compactor.ring.instance-interface-names string + List of network interface names to look up when finding the instance IP address. (default []) + -compactor.ring.store string + Backend storage to use for the ring. Supported values are: consul, etcd, inmemory, memberlist, multi. (default "memberlist") + -compactor.split-and-merge-shards int + The number of shards to use when splitting blocks. 0 to disable splitting. + -compactor.split-groups int + Number of groups that blocks for splitting should be grouped into. Each group of blocks is then split separately. Number of output split shards is controlled by -compactor.split-and-merge-shards. (default 1) -config.expand-env Expands ${var} in config according to the values of the environment variables. -config.file string diff --git a/docs/sources/configure-server/reference-configuration-parameters/index.md b/docs/sources/configure-server/reference-configuration-parameters/index.md index 43512c414d..9cc34524d0 100644 --- a/docs/sources/configure-server/reference-configuration-parameters/index.md +++ b/docs/sources/configure-server/reference-configuration-parameters/index.md @@ -180,6 +180,57 @@ limits: # CLI flag: -querier.split-queries-by-interval [split_queries_by_interval: | default = 0s] + # Delete blocks containing samples older than the specified retention period. + # 0 to disable. + # CLI flag: -compactor.blocks-retention-period + [compactor_blocks_retention_period: | default = 0s] + + # The number of shards to use when splitting blocks. 0 to disable splitting. + # CLI flag: -compactor.split-and-merge-shards + [compactor_split_and_merge_shards: | default = 0] + + # Number of groups that blocks for splitting should be grouped into. Each + # group of blocks is then split separately. Number of output split shards is + # controlled by -compactor.split-and-merge-shards. + # CLI flag: -compactor.split-groups + [compactor_split_groups: | default = 1] + + # Max number of compactors that can compact blocks for single tenant. 0 to + # disable the limit and use all compactors. + # CLI flag: -compactor.compactor-tenant-shard-size + [compactor_tenant_shard_size: | default = 0] + + # If a partial block (unfinished block without meta.json file) hasn't been + # modified for this time, it will be marked for deletion. The minimum accepted + # value is 4h0m0s: a lower value will be ignored and the feature disabled. 0 + # to disable. + # CLI flag: -compactor.partial-block-deletion-delay + [compactor_partial_block_deletion_delay: | default = 1d] + + # S3 server-side encryption type. Required to enable server-side encryption + # overrides for a specific tenant. If not set, the default S3 client settings + # are used. + [s3_sse_type: | default = ""] + + # S3 server-side encryption KMS Key ID. Ignored if the SSE type override is + # not set. + [s3_sse_kms_key_id: | default = ""] + + # S3 server-side encryption KMS encryption context. If unset and the key ID + # override is set, the encryption context will not be provided to S3. Ignored + # if the SSE type override is not set. + [s3_sse_kms_encryption_context: | default = ""] + + # This limits how far into the past profiling data can be ingested. This limit + # is enforced in the distributor. 0 to disable, defaults to 1h. + # CLI flag: -validation.reject-older-than + [reject_older_than: | default = 1h] + + # This limits how far into the future profiling data can be ingested. This + # limit is enforced in the distributor. 0 to disable, defaults to 10m. + # CLI flag: -validation.reject-newer-than + [reject_newer_than: | default = 10m] + # The query_scheduler block configures the query-scheduler. [query_scheduler: ] @@ -458,6 +509,293 @@ runtime_config: # CLI flag: -runtime-config.file [file: | default = ""] +compactor: + # List of compaction time ranges. + # CLI flag: -compactor.block-ranges + [block_ranges: | default = 4h0m0s] + + # Number of Go routines to use when downloading blocks for compaction and + # uploading resulting blocks. + # CLI flag: -compactor.block-sync-concurrency + [block_sync_concurrency: | default = 8] + + # Number of Go routines to use when syncing block meta files from the long + # term storage. + # CLI flag: -compactor.meta-sync-concurrency + [meta_sync_concurrency: | default = 20] + + # Directory to temporarily store blocks during compaction. This directory is + # not required to be persisted between restarts. + # CLI flag: -compactor.data-dir + [data_dir: | default = "./data-compactor/"] + + # The frequency at which the compaction runs + # CLI flag: -compactor.compaction-interval + [compaction_interval: | default = 1h] + + # How many times to retry a failed compaction within a single compaction run. + # CLI flag: -compactor.compaction-retries + [compaction_retries: | default = 3] + + # Max number of concurrent compactions running. + # CLI flag: -compactor.compaction-concurrency + [compaction_concurrency: | default = 1] + + # How long the compactor waits before compacting first-level blocks that are + # uploaded by the ingesters. This configuration option allows for the + # reduction of cases where the compactor begins to compact blocks before all + # ingesters have uploaded their blocks to the storage. + # CLI flag: -compactor.first-level-compaction-wait-period + [first_level_compaction_wait_period: | default = 25m] + + # How frequently compactor should run blocks cleanup and maintenance, as well + # as update the bucket index. + # CLI flag: -compactor.cleanup-interval + [cleanup_interval: | default = 15m] + + # Max number of tenants for which blocks cleanup and maintenance should run + # concurrently. + # CLI flag: -compactor.cleanup-concurrency + [cleanup_concurrency: | default = 20] + + # Time before a block marked for deletion is deleted from bucket. If not 0, + # blocks will be marked for deletion and compactor component will permanently + # delete blocks marked for deletion from the bucket. If 0, blocks will be + # deleted straight away. Note that deleting blocks immediately can cause query + # failures. + # CLI flag: -compactor.deletion-delay + [deletion_delay: | default = 12h] + + # For tenants marked for deletion, this is time between deleting of last + # block, and doing final cleanup (marker files, debug files) of the tenant. + # CLI flag: -compactor.tenant-cleanup-delay + [tenant_cleanup_delay: | default = 6h] + + # Max time for starting compactions for a single tenant. After this time no + # new compactions for the tenant are started before next compaction cycle. + # This can help in multi-tenant environments to avoid single tenant using all + # compaction time, but also in single-tenant environments to force new + # discovery of blocks more often. 0 = disabled. + # CLI flag: -compactor.max-compaction-time + [max_compaction_time: | default = 1h] + + # If enabled, will delete the bucket-index, markers and debug files in the + # tenant bucket when there are no blocks left in the index. + # CLI flag: -compactor.no-blocks-file-cleanup-enabled + [no_blocks_file_cleanup_enabled: | default = false] + + # Number of goroutines opening blocks before compaction. + # CLI flag: -compactor.max-opening-blocks-concurrency + [max_opening_blocks_concurrency: | default = 1] + + # Comma separated list of tenants that can be compacted. If specified, only + # these tenants will be compacted by compactor, otherwise all tenants can be + # compacted. Subject to sharding. + # CLI flag: -compactor.enabled-tenants + [enabled_tenants: | default = ""] + + # Comma separated list of tenants that cannot be compacted by this compactor. + # If specified, and compactor would normally pick given tenant for compaction + # (via -compactor.enabled-tenants or sharding), it will be ignored instead. + # CLI flag: -compactor.disabled-tenants + [disabled_tenants: | default = ""] + + sharding_ring: + # The key-value store used to share the hash ring across multiple instances. + kvstore: + # Backend storage to use for the ring. Supported values are: consul, etcd, + # inmemory, memberlist, multi. + # CLI flag: -compactor.ring.store + [store: | default = "memberlist"] + + # The prefix for the keys in the store. Should end with a /. + # CLI flag: -compactor.ring.prefix + [prefix: | default = "collectors/"] + + consul: + # Hostname and port of Consul. + # CLI flag: -compactor.ring.consul.hostname + [host: | default = "localhost:8500"] + + # ACL Token used to interact with Consul. + # CLI flag: -compactor.ring.consul.acl-token + [acl_token: | default = ""] + + # HTTP timeout when talking to Consul + # CLI flag: -compactor.ring.consul.client-timeout + [http_client_timeout: | default = 20s] + + # Enable consistent reads to Consul. + # CLI flag: -compactor.ring.consul.consistent-reads + [consistent_reads: | default = false] + + # Rate limit when watching key or prefix in Consul, in requests per + # second. 0 disables the rate limit. + # CLI flag: -compactor.ring.consul.watch-rate-limit + [watch_rate_limit: | default = 1] + + # Burst size used in rate limit. Values less than 1 are treated as 1. + # CLI flag: -compactor.ring.consul.watch-burst-size + [watch_burst_size: | default = 1] + + # Maximum duration to wait before retrying a Compare And Swap (CAS) + # operation. + # CLI flag: -compactor.ring.consul.cas-retry-delay + [cas_retry_delay: | default = 1s] + + etcd: + # The etcd endpoints to connect to. + # CLI flag: -compactor.ring.etcd.endpoints + [endpoints: | default = []] + + # The dial timeout for the etcd connection. + # CLI flag: -compactor.ring.etcd.dial-timeout + [dial_timeout: | default = 10s] + + # The maximum number of retries to do for failed ops. + # CLI flag: -compactor.ring.etcd.max-retries + [max_retries: | default = 10] + + # Enable TLS. + # CLI flag: -compactor.ring.etcd.tls-enabled + [tls_enabled: | default = false] + + # Path to the client certificate, which will be used for authenticating + # with the server. Also requires the key path to be configured. + # CLI flag: -compactor.ring.etcd.tls-cert-path + [tls_cert_path: | default = ""] + + # Path to the key for the client certificate. Also requires the client + # certificate to be configured. + # CLI flag: -compactor.ring.etcd.tls-key-path + [tls_key_path: | default = ""] + + # Path to the CA certificates to validate server certificate against. If + # not set, the host's root CA certificates are used. + # CLI flag: -compactor.ring.etcd.tls-ca-path + [tls_ca_path: | default = ""] + + # Override the expected name on the server certificate. + # CLI flag: -compactor.ring.etcd.tls-server-name + [tls_server_name: | default = ""] + + # Skip validating server certificate. + # CLI flag: -compactor.ring.etcd.tls-insecure-skip-verify + [tls_insecure_skip_verify: | default = false] + + # Override the default cipher suite list (separated by commas). Allowed + # values: + # + # Secure Ciphers: + # - TLS_RSA_WITH_AES_128_CBC_SHA + # - TLS_RSA_WITH_AES_256_CBC_SHA + # - TLS_RSA_WITH_AES_128_GCM_SHA256 + # - TLS_RSA_WITH_AES_256_GCM_SHA384 + # - TLS_AES_128_GCM_SHA256 + # - TLS_AES_256_GCM_SHA384 + # - TLS_CHACHA20_POLY1305_SHA256 + # - TLS_ECDHE_ECDSA_WITH_AES_128_CBC_SHA + # - TLS_ECDHE_ECDSA_WITH_AES_256_CBC_SHA + # - TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA + # - TLS_ECDHE_RSA_WITH_AES_256_CBC_SHA + # - TLS_ECDHE_ECDSA_WITH_AES_128_GCM_SHA256 + # - TLS_ECDHE_ECDSA_WITH_AES_256_GCM_SHA384 + # - TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256 + # - TLS_ECDHE_RSA_WITH_AES_256_GCM_SHA384 + # - TLS_ECDHE_RSA_WITH_CHACHA20_POLY1305_SHA256 + # - TLS_ECDHE_ECDSA_WITH_CHACHA20_POLY1305_SHA256 + # + # Insecure Ciphers: + # - TLS_RSA_WITH_RC4_128_SHA + # - TLS_RSA_WITH_3DES_EDE_CBC_SHA + # - TLS_RSA_WITH_AES_128_CBC_SHA256 + # - TLS_ECDHE_ECDSA_WITH_RC4_128_SHA + # - TLS_ECDHE_RSA_WITH_RC4_128_SHA + # - TLS_ECDHE_RSA_WITH_3DES_EDE_CBC_SHA + # - TLS_ECDHE_ECDSA_WITH_AES_128_CBC_SHA256 + # - TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA256 + # CLI flag: -compactor.ring.etcd.tls-cipher-suites + [tls_cipher_suites: | default = ""] + + # Override the default minimum TLS version. Allowed values: + # VersionTLS10, VersionTLS11, VersionTLS12, VersionTLS13 + # CLI flag: -compactor.ring.etcd.tls-min-version + [tls_min_version: | default = ""] + + # Etcd username. + # CLI flag: -compactor.ring.etcd.username + [username: | default = ""] + + # Etcd password. + # CLI flag: -compactor.ring.etcd.password + [password: | default = ""] + + multi: + # Primary backend storage used by multi-client. + # CLI flag: -compactor.ring.multi.primary + [primary: | default = ""] + + # Secondary backend storage used by multi-client. + # CLI flag: -compactor.ring.multi.secondary + [secondary: | default = ""] + + # Mirror writes to secondary store. + # CLI flag: -compactor.ring.multi.mirror-enabled + [mirror_enabled: | default = false] + + # Timeout for storing value to secondary store. + # CLI flag: -compactor.ring.multi.mirror-timeout + [mirror_timeout: | default = 2s] + + # Period at which to heartbeat to the ring. 0 = disabled. + # CLI flag: -compactor.ring.heartbeat-period + [heartbeat_period: | default = 15s] + + # The heartbeat timeout after which compactors are considered unhealthy + # within the ring. 0 = never (timeout disabled). + # CLI flag: -compactor.ring.heartbeat-timeout + [heartbeat_timeout: | default = 1m] + + # Instance ID to register in the ring. + # CLI flag: -compactor.ring.instance-id + [instance_id: | default = ""] + + # List of network interface names to look up when finding the instance IP + # address. + # CLI flag: -compactor.ring.instance-interface-names + [instance_interface_names: | default = []] + + # Port to advertise in the ring (defaults to -server.http-listen-port). + # CLI flag: -compactor.ring.instance-port + [instance_port: | default = 0] + + # IP address to advertise in the ring. Default is auto-detected. + # CLI flag: -compactor.ring.instance-addr + [instance_addr: | default = ""] + + # Enable using a IPv6 instance address. (default false) + # CLI flag: -compactor.ring.instance-enable-ipv6 + [instance_enable_ipv6: | default = false] + + # Minimum time to wait for ring stability at startup. 0 to disable. + # CLI flag: -compactor.ring.wait-stability-min-duration + [wait_stability_min_duration: | default = 0s] + + # Maximum time to wait for ring stability at startup. If the compactor ring + # keeps changing after this period of time, the compactor will start anyway. + # CLI flag: -compactor.ring.wait-stability-max-duration + [wait_stability_max_duration: | default = 5m] + + # Timeout for waiting on compactor to become ACTIVE in the ring. + # CLI flag: -compactor.ring.wait-active-instance-timeout + [wait_active_instance_timeout: | default = 10m] + + # The sorting to use when deciding which compaction jobs should run first for + # a given tenant. Supported values are: smallest-range-oldest-blocks-first, + # newest-blocks-first. + # CLI flag: -compactor.compaction-jobs-order + [compaction_jobs_order: | default = "smallest-range-oldest-blocks-first"] + storage: # Backend storage to use. Supported backends are: s3, gcs, azure, swift, # filesystem, cos. diff --git a/pkg/api/api.go b/pkg/api/api.go index 77dbd28917..66cdae7637 100644 --- a/pkg/api/api.go +++ b/pkg/api/api.go @@ -30,6 +30,7 @@ import ( statusv1 "github.com/grafana/pyroscope/api/gen/proto/go/status/v1" "github.com/grafana/pyroscope/api/gen/proto/go/storegateway/v1/storegatewayv1connect" "github.com/grafana/pyroscope/api/openapiv2" + "github.com/grafana/pyroscope/pkg/compactor" "github.com/grafana/pyroscope/pkg/distributor" "github.com/grafana/pyroscope/pkg/frontend" "github.com/grafana/pyroscope/pkg/frontend/frontendpb/frontendpbconnect" @@ -252,6 +253,14 @@ func (a *API) RegisterStoreGateway(svc *storegateway.StoreGateway) { a.RegisterRoute("/store-gateway/tenant/{tenant}/blocks", http.HandlerFunc(svc.BlocksHandler), false, true, "GET") } +// RegisterCompactor registers routes associated with the compactor. +func (a *API) RegisterCompactor(c *compactor.MultitenantCompactor) { + a.indexPage.AddLinks(defaultWeight, "Compactor", []IndexPageLink{ + {Desc: "Ring status", Path: "/compactor/ring"}, + }) + a.RegisterRoute("/compactor/ring", http.HandlerFunc(c.RingHandler), false, true, "GET", "POST") +} + // RegisterQueryFrontend registers the endpoints associated with the query frontend. func (a *API) RegisterQueryFrontend(frontendSvc *frontend.Frontend) { frontendpbconnect.RegisterFrontendForQuerierHandler(a.server.HTTP, frontendSvc, a.grpcAuthMiddleware) diff --git a/pkg/compactor/compactor.go b/pkg/compactor/compactor.go index 7bf5c04b6f..92370d70cd 100644 --- a/pkg/compactor/compactor.go +++ b/pkg/compactor/compactor.go @@ -118,7 +118,7 @@ type Config struct { func (cfg *Config) RegisterFlags(f *flag.FlagSet, logger log.Logger) { cfg.ShardingRing.RegisterFlags(f, logger) - cfg.BlockRanges = DurationList{2 * time.Hour, 12 * time.Hour, 24 * time.Hour} + cfg.BlockRanges = DurationList{4 * time.Hour} cfg.retryMinBackoff = 10 * time.Second cfg.retryMaxBackoff = time.Minute diff --git a/pkg/phlare/modules.go b/pkg/phlare/modules.go index c16a0dafd2..6cf2f32ebb 100644 --- a/pkg/phlare/modules.go +++ b/pkg/phlare/modules.go @@ -31,6 +31,7 @@ import ( "gopkg.in/yaml.v3" statusv1 "github.com/grafana/pyroscope/api/gen/proto/go/status/v1" + "github.com/grafana/pyroscope/pkg/compactor" "github.com/grafana/pyroscope/pkg/distributor" "github.com/grafana/pyroscope/pkg/frontend" "github.com/grafana/pyroscope/pkg/ingester" @@ -67,9 +68,9 @@ const ( RuntimeConfig string = "runtime-config" Overrides string = "overrides" OverridesExporter string = "overrides-exporter" + Compactor string = "compactor" // QueryFrontendTripperware string = "query-frontend-tripperware" - // Compactor string = "compactor" // IndexGateway string = "index-gateway" // IndexGatewayRing string = "index-gateway-ring" ) @@ -167,6 +168,19 @@ func (f *Phlare) initQueryScheduler() (services.Service, error) { return s, nil } +func (f *Phlare) initCompactor() (serv services.Service, err error) { + f.Cfg.Compactor.ShardingRing.Common.ListenPort = f.Cfg.Server.HTTPListenPort + + f.Compactor, err = compactor.NewMultitenantCompactor(f.Cfg.Compactor, f.storageBucket, f.Overrides, log.With(f.logger, "component", "compactor"), f.reg) + if err != nil { + return + } + + // Expose HTTP endpoints. + f.API.RegisterCompactor(f.Compactor) + return f.Compactor, nil +} + // setupWorkerTimeout sets the max loop duration for the querier worker and frontend worker // to 90% of the read or write http timeout, whichever is smaller. // This is to ensure that the worker doesn't timeout before the http handler and that the connection @@ -284,7 +298,7 @@ func (f *Phlare) initMemberlistKV() (services.Service, error) { f.Cfg.QueryScheduler.ServiceDiscovery.SchedulerRing.KVStore.MemberlistKV = f.MemberlistKV.GetMemberlistKV f.Cfg.OverridesExporter.Ring.Ring.KVStore.MemberlistKV = f.MemberlistKV.GetMemberlistKV f.Cfg.StoreGateway.ShardingRing.Ring.KVStore.MemberlistKV = f.MemberlistKV.GetMemberlistKV - + f.Cfg.Compactor.ShardingRing.Common.KVStore.MemberlistKV = f.MemberlistKV.GetMemberlistKV f.Cfg.Frontend.QuerySchedulerDiscovery = f.Cfg.QueryScheduler.ServiceDiscovery f.Cfg.Worker.QuerySchedulerDiscovery = f.Cfg.QueryScheduler.ServiceDiscovery diff --git a/pkg/phlare/phlare.go b/pkg/phlare/phlare.go index aa1d744455..a02c9c9cb4 100644 --- a/pkg/phlare/phlare.go +++ b/pkg/phlare/phlare.go @@ -36,6 +36,7 @@ import ( "github.com/grafana/pyroscope/pkg/api" "github.com/grafana/pyroscope/pkg/cfg" + "github.com/grafana/pyroscope/pkg/compactor" "github.com/grafana/pyroscope/pkg/distributor" "github.com/grafana/pyroscope/pkg/frontend" "github.com/grafana/pyroscope/pkg/ingester" @@ -74,6 +75,7 @@ type Config struct { Tracing tracing.Config `yaml:"tracing"` OverridesExporter exporter.Config `yaml:"overrides_exporter" doc:"hidden"` RuntimeConfig runtimeconfig.Config `yaml:"runtime_config"` + Compactor compactor.Config `yaml:"compactor"` Storage StorageConfig `yaml:"storage"` SelfProfiling SelfProfilingConfig `yaml:"self_profiling,omitempty"` @@ -138,6 +140,7 @@ func (c *Config) RegisterFlagsWithContext(ctx context.Context, f *flag.FlagSet) c.RuntimeConfig.RegisterFlags(f) c.Analytics.RegisterFlags(f) c.LimitsConfig.RegisterFlags(f) + c.Compactor.RegisterFlags(f, log.NewLogfmtLogger(os.Stderr)) c.API.RegisterFlags(f) } @@ -173,6 +176,9 @@ func (c *Config) Validate() error { if len(c.Target) == 0 { return errors.New("no modules specified") } + if err := c.Compactor.Validate(); err != nil { + return err + } return c.Ingester.Validate() } @@ -184,6 +190,7 @@ func (c *Config) ApplyDynamicConfig() cfg.Source { c.Worker.QuerySchedulerDiscovery.SchedulerRing.KVStore.Store = c.Ingester.LifecyclerConfig.RingConfig.KVStore.Store c.QueryScheduler.ServiceDiscovery.SchedulerRing.KVStore.Store = c.Ingester.LifecyclerConfig.RingConfig.KVStore.Store c.StoreGateway.ShardingRing.Ring.KVStore.Store = c.Ingester.LifecyclerConfig.RingConfig.KVStore.Store + c.Compactor.ShardingRing.Common.KVStore.Store = c.Ingester.LifecyclerConfig.RingConfig.KVStore.Store return func(dst cfg.Cloneable) error { return nil @@ -214,6 +221,7 @@ type Phlare struct { usageReport *usagestats.Reporter RuntimeConfig *runtimeconfig.Manager Overrides *validation.Overrides + Compactor *compactor.MultitenantCompactor TenantLimits validation.TenantLimits @@ -279,21 +287,22 @@ func (f *Phlare) setupModuleManager() error { mm.RegisterModule(UsageReport, f.initUsageReport) mm.RegisterModule(QueryFrontend, f.initQueryFrontend) mm.RegisterModule(QueryScheduler, f.initQueryScheduler) + mm.RegisterModule(Compactor, f.initCompactor) mm.RegisterModule(All, nil) // Add dependencies deps := map[string][]string{ All: {Ingester, Distributor, QueryScheduler, QueryFrontend, Querier, StoreGateway}, - Server: {GRPCGateway}, - API: {Server}, - Distributor: {Overrides, Ring, API, UsageReport}, - Querier: {Overrides, API, MemberlistKV, Ring, UsageReport}, - QueryFrontend: {OverridesExporter, API, MemberlistKV, UsageReport}, - QueryScheduler: {Overrides, API, MemberlistKV, UsageReport}, - Ingester: {Overrides, API, MemberlistKV, Storage, UsageReport}, - StoreGateway: {API, Storage, Overrides, MemberlistKV, UsageReport}, - + Server: {GRPCGateway}, + API: {Server}, + Distributor: {Overrides, Ring, API, UsageReport}, + Querier: {Overrides, API, MemberlistKV, Ring, UsageReport}, + QueryFrontend: {OverridesExporter, API, MemberlistKV, UsageReport}, + QueryScheduler: {Overrides, API, MemberlistKV, UsageReport}, + Ingester: {Overrides, API, MemberlistKV, Storage, UsageReport}, + StoreGateway: {API, Storage, Overrides, MemberlistKV, UsageReport}, + Compactor: {API, Storage, Overrides, MemberlistKV, UsageReport}, UsageReport: {Storage, MemberlistKV}, Overrides: {RuntimeConfig}, OverridesExporter: {Overrides, MemberlistKV}, From dd2612e3f4d9933a23f27b1b7e024ce6d406bf6e Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Mon, 2 Oct 2023 09:13:33 +0200 Subject: [PATCH 47/74] Add the compactor helm component --- .../pyroscope/rendered/micro-services.yaml | 165 ++++++++++++++++++ .../helm/pyroscope/values-micro-services.yaml | 9 + .../jsonnet/values-micro-services.json | 13 ++ 3 files changed, 187 insertions(+) diff --git a/operations/pyroscope/helm/pyroscope/rendered/micro-services.yaml b/operations/pyroscope/helm/pyroscope/rendered/micro-services.yaml index e039611b5f..0a879f3583 100644 --- a/operations/pyroscope/helm/pyroscope/rendered/micro-services.yaml +++ b/operations/pyroscope/helm/pyroscope/rendered/micro-services.yaml @@ -2,6 +2,27 @@ # Source: pyroscope/templates/deployments-statefulsets.yaml apiVersion: policy/v1 kind: PodDisruptionBudget +metadata: + name: pyroscope-dev-compactor + namespace: default + labels: + helm.sh/chart: pyroscope-1.0.3 + app.kubernetes.io/name: pyroscope + app.kubernetes.io/instance: pyroscope-dev + app.kubernetes.io/version: "1.0.0" + app.kubernetes.io/managed-by: Helm + app.kubernetes.io/component: "compactor" +spec: + maxUnavailable: 1 + selector: + matchLabels: + app.kubernetes.io/name: pyroscope + app.kubernetes.io/instance: pyroscope-dev + app.kubernetes.io/component: "compactor" +--- +# Source: pyroscope/templates/deployments-statefulsets.yaml +apiVersion: policy/v1 +kind: PodDisruptionBudget metadata: name: pyroscope-dev-distributor namespace: default @@ -1633,6 +1654,57 @@ spec: # Source: pyroscope/templates/services.yaml apiVersion: v1 kind: Service +metadata: + name: pyroscope-dev-compactor + namespace: default + labels: + helm.sh/chart: pyroscope-1.0.3 + app.kubernetes.io/name: pyroscope + app.kubernetes.io/instance: pyroscope-dev + app.kubernetes.io/version: "1.0.0" + app.kubernetes.io/managed-by: Helm + app.kubernetes.io/component: "compactor" +spec: + type: ClusterIP + ports: + - port: 4040 + targetPort: http2 + protocol: TCP + name: http2 + selector: + app.kubernetes.io/name: pyroscope + app.kubernetes.io/instance: pyroscope-dev + app.kubernetes.io/component: "compactor" +--- +# Source: pyroscope/templates/services.yaml +apiVersion: v1 +kind: Service +metadata: + name: pyroscope-dev-compactor-headless + namespace: default + labels: + helm.sh/chart: pyroscope-1.0.3 + app.kubernetes.io/name: pyroscope + app.kubernetes.io/instance: pyroscope-dev + app.kubernetes.io/version: "1.0.0" + app.kubernetes.io/managed-by: Helm + app.kubernetes.io/component: "compactor" +spec: + type: ClusterIP + clusterIP: None + ports: + - port: 4040 + targetPort: http2 + protocol: TCP + name: http2 + selector: + app.kubernetes.io/name: pyroscope + app.kubernetes.io/instance: pyroscope-dev + app.kubernetes.io/component: "compactor" +--- +# Source: pyroscope/templates/services.yaml +apiVersion: v1 +kind: Service metadata: name: pyroscope-dev-distributor namespace: default @@ -2480,6 +2552,99 @@ spec: # Source: pyroscope/templates/deployments-statefulsets.yaml apiVersion: apps/v1 kind: StatefulSet +metadata: + name: pyroscope-dev-compactor + namespace: default + labels: + helm.sh/chart: pyroscope-1.0.3 + app.kubernetes.io/name: pyroscope + app.kubernetes.io/instance: pyroscope-dev + app.kubernetes.io/version: "1.0.0" + app.kubernetes.io/managed-by: Helm + app.kubernetes.io/component: "compactor" +spec: + serviceName: pyroscope-dev-compactor-headless + podManagementPolicy: Parallel + replicas: 3 + selector: + matchLabels: + app.kubernetes.io/name: pyroscope + app.kubernetes.io/instance: pyroscope-dev + app.kubernetes.io/component: "compactor" + template: + metadata: + annotations: + checksum/config: 027360c7de6747514d4ecfafd152f3a01b26a673d9cce1ef8697a8c67d6d48d9 + profiles.grafana.com/cpu.port_name: http2 + profiles.grafana.com/cpu.scrape: "true" + profiles.grafana.com/goroutine.port_name: http2 + profiles.grafana.com/goroutine.scrape: "true" + profiles.grafana.com/memory.port_name: http2 + profiles.grafana.com/memory.scrape: "true" + labels: + app.kubernetes.io/name: pyroscope + app.kubernetes.io/instance: pyroscope-dev + app.kubernetes.io/component: "compactor" + name: "compactor" + spec: + serviceAccountName: pyroscope-dev + securityContext: + fsGroup: 10001 + runAsNonRoot: true + runAsUser: 10001 + containers: + - name: "compactor" + securityContext: + {} + image: "grafana/pyroscope:1.0.0" + imagePullPolicy: IfNotPresent + args: + - "-target=compactor" + - "-self-profiling.disable-push=true" + - "-server.http-listen-port=4040" + - "-memberlist.cluster-label=default-pyroscope-dev" + - "-memberlist.join=dns+pyroscope-dev-memberlist.default.svc.cluster.local.:7946" + - "-config.file=/etc/pyroscope/config.yaml" + - "-runtime-config.file=/etc/pyroscope/overrides/overrides.yaml" + - "-log.level=debug" + ports: + - name: http2 + containerPort: 4040 + protocol: TCP + - name: memberlist + containerPort: 7946 + protocol: TCP + readinessProbe: + httpGet: + path: /ready + port: http2 + volumeMounts: + - name: config + mountPath: /etc/pyroscope/config.yaml + subPath: config.yaml + - name: overrides-config + mountPath: /etc/pyroscope/overrides/ + - name: data + mountPath: /data + resources: + limits: + memory: 16Gi + requests: + cpu: 1 + memory: 8Gi + volumes: + - name: config + configMap: + name: pyroscope-dev-config + - name: overrides-config + configMap: + name: pyroscope-dev-overrides-config + - name: data + emptyDir: {} +--- +# Source: pyroscope/templates/deployments-statefulsets.yaml +apiVersion: apps/v1 +kind: StatefulSet metadata: name: pyroscope-dev-ingester namespace: default diff --git a/operations/pyroscope/helm/pyroscope/values-micro-services.yaml b/operations/pyroscope/helm/pyroscope/values-micro-services.yaml index c4ceb00aeb..2fe2cfe845 100644 --- a/operations/pyroscope/helm/pyroscope/values-micro-services.yaml +++ b/operations/pyroscope/helm/pyroscope/values-micro-services.yaml @@ -50,6 +50,15 @@ pyroscope: requests: memory: 8Gi cpu: 1 + compactor: + kind: StatefulSet + replicaCount: 3 + resources: + limits: + memory: 16Gi + requests: + memory: 8Gi + cpu: 1 store-gateway: kind: StatefulSet replicaCount: 3 diff --git a/operations/pyroscope/jsonnet/values-micro-services.json b/operations/pyroscope/jsonnet/values-micro-services.json index f644883a16..d764597ca1 100644 --- a/operations/pyroscope/jsonnet/values-micro-services.json +++ b/operations/pyroscope/jsonnet/values-micro-services.json @@ -4,6 +4,19 @@ }, "pyroscope": { "components": { + "compactor": { + "kind": "StatefulSet", + "replicaCount": 3, + "resources": { + "limits": { + "memory": "16Gi" + }, + "requests": { + "cpu": 1, + "memory": "8Gi" + } + } + }, "distributor": { "kind": "Deployment", "replicaCount": 2, From d6d000c9784a8b9412873def476575b1f0cc5a69 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Mon, 2 Oct 2023 13:31:49 +0200 Subject: [PATCH 48/74] Fixes tests after the range change --- pkg/compactor/compactor_test.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/pkg/compactor/compactor_test.go b/pkg/compactor/compactor_test.go index 6791a46b7e..c35fe9ee3d 100644 --- a/pkg/compactor/compactor_test.go +++ b/pkg/compactor/compactor_test.go @@ -1391,7 +1391,7 @@ func TestMultitenantCompactor_ShouldSkipCompactionForJobsWithFirstLevelCompactio // Ensure the skipped compaction job is the expected one. assert.Contains(t, strings.Split(strings.TrimSpace(logs.String()), "\n"), - fmt.Sprintf(`level=info component=compactor tenant=user-2 msg="skipping compaction job because blocks in this job were uploaded too recently (within wait period)" groupKey=0@17241709254077376921-merge--0-7200000 waitPeriodNotElapsedFor="%s (min time: 1970-01-01T08:00:00+08:00, max time: 1970-01-01T10:00:00+08:00)"`, user2Meta2.String())) + fmt.Sprintf(`level=info component=compactor tenant=user-2 msg="skipping compaction job because blocks in this job were uploaded too recently (within wait period)" groupKey=0@17241709254077376921-merge--0-7200000 waitPeriodNotElapsedFor="%s (min time: 1970-01-01T01:00:00+01:00, max time: 1970-01-01T03:00:00+01:00)"`, user2Meta2.String())) assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` # TYPE pyroscope_compactor_runs_started_total counter @@ -1586,6 +1586,8 @@ func removeIgnoredLogs(input []string) []string { func prepareConfig(t *testing.T) Config { compactorCfg := Config{} flagext.DefaultValues(&compactorCfg) + // Use multiple range for testing. + compactorCfg.BlockRanges = DurationList{2 * time.Hour, 12 * time.Hour, 24 * time.Hour} compactorCfg.retryMinBackoff = 0 compactorCfg.retryMaxBackoff = 0 From 22ccdcd61c28bbbcbbd23d20782329752f032829 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Tue, 3 Oct 2023 14:54:43 +0200 Subject: [PATCH 49/74] change: store-gateway uses the bucketindex Changes the store-gateway to use the bucketindex instead of iterating through all blocks. This also tweak block query config. --- cmd/pyroscope/help-all.txt.tmpl | 8 +- pkg/phlaredb/bucketindex/index.go | 14 ++ pkg/querier/querier.go | 2 +- pkg/storegateway/block_filter.go | 85 ++++++-- pkg/storegateway/bucket.go | 62 ++---- .../bucket_index_metadata_fetcher.go | 121 +++++++++++ .../bucket_index_metadata_fetcher_test.go | 198 ++++++++++++++++++ pkg/storegateway/bucket_stores.go | 47 ++++- pkg/storegateway/gateway.go | 1 + 9 files changed, 463 insertions(+), 75 deletions(-) create mode 100644 pkg/storegateway/bucket_index_metadata_fetcher.go create mode 100644 pkg/storegateway/bucket_index_metadata_fetcher_test.go diff --git a/cmd/pyroscope/help-all.txt.tmpl b/cmd/pyroscope/help-all.txt.tmpl index f866c9d757..03a240f1ea 100644 --- a/cmd/pyroscope/help-all.txt.tmpl +++ b/cmd/pyroscope/help-all.txt.tmpl @@ -4,7 +4,11 @@ Usage of ./pyroscope: -auth.multitenancy-enabled When set to true, incoming HTTP requests must specify tenant ID in HTTP X-Scope-OrgId header. When set to false, tenant ID anonymous is used instead. -blocks-storage.bucket-store.ignore-blocks-within duration - Blocks with minimum time within this duration are ignored, and not loaded by store-gateway. Useful when used together with -querier.query-store-after to prevent loading young blocks, because there are usually many of them (depending on number of ingesters) and they are not yet compacted. Negative values or 0 disable the filter. (default 2h0m0s) + Blocks with minimum time within this duration are ignored, and not loaded by store-gateway. Useful when used together with -querier.query-store-after to prevent loading young blocks, because there are usually many of them (depending on number of ingesters) and they are not yet compacted. Negative values or 0 disable the filter. (default 10h0m0s) + -blocks-storage.bucket-store.ignore-deletion-marks-delay duration + Duration after which the blocks marked for deletion will be filtered out while fetching blocks. The idea of ignore-deletion-marks-delay is to ignore blocks that are marked for deletion with some delay. This ensures store can still serve blocks that are meant to be deleted but do not have a replacement yet. (default 1h0m0s) + -blocks-storage.bucket-store.meta-sync-concurrency int + Number of Go routines to use when syncing block meta files from object storage per tenant. (default 20) -blocks-storage.bucket-store.sync-dir string Directory to store synchronized pyroscope block headers. This directory is not required to be persisted between restarts, but it's highly recommended in order to improve the store-gateway startup time. (default "./data/pyroscope-sync/") -blocks-storage.bucket-store.sync-interval duration @@ -524,7 +528,7 @@ Usage of ./pyroscope: -querier.max-query-parallelism int Maximum number of queries that will be scheduled in parallel by the frontend. -querier.query-store-after duration - The time after which a metric should be queried from storage and not just ingesters. 0 means all queries are sent to store. If this option is enabled, the time range of the query sent to the store-gateway will be manipulated to ensure the query end is not more recent than 'now - query-store-after'. (default 4h0m0s) + The time after which a metric should be queried from storage and not just ingesters. 0 means all queries are sent to store. If this option is enabled, the time range of the query sent to the store-gateway will be manipulated to ensure the query end is not more recent than 'now - query-store-after'. (default 12h0m0s) -querier.split-queries-by-interval duration Split queries by a time interval and execute in parallel. The value 0 disables splitting by time -query-frontend.grpc-client-config.backoff-max-period duration diff --git a/pkg/phlaredb/bucketindex/index.go b/pkg/phlaredb/bucketindex/index.go index 6ed88373a2..ac4923b683 100644 --- a/pkg/phlaredb/bucketindex/index.go +++ b/pkg/phlaredb/bucketindex/index.go @@ -102,6 +102,20 @@ func (m *Block) String() string { return fmt.Sprintf("%s (min time: %s max time: %s, compactor shard: %s)", m.ID, minT.String(), maxT.String(), shard) } +// Meta returns a block meta based on the known information in the index. +// The returned meta doesn't include all original meta.json data but only a subset +// of it. +func (m *Block) Meta() *block.Meta { + return &block.Meta{ + ULID: m.ID, + MinTime: m.MinTime, + MaxTime: m.MaxTime, + Labels: map[string]string{ + sharding.CompactorShardIDLabel: m.CompactorShardID, + }, + } +} + func BlockFromMeta(meta block.Meta) *Block { return &Block{ ID: meta.ULID, diff --git a/pkg/querier/querier.go b/pkg/querier/querier.go index 26f0dd7156..41d14ca01b 100644 --- a/pkg/querier/querier.go +++ b/pkg/querier/querier.go @@ -43,7 +43,7 @@ type Config struct { // RegisterFlags registers distributor-related flags. func (cfg *Config) RegisterFlags(fs *flag.FlagSet) { cfg.PoolConfig.RegisterFlagsWithPrefix("querier", fs) - fs.DurationVar(&cfg.QueryStoreAfter, "querier.query-store-after", 4*time.Hour, "The time after which a metric should be queried from storage and not just ingesters. 0 means all queries are sent to store. If this option is enabled, the time range of the query sent to the store-gateway will be manipulated to ensure the query end is not more recent than 'now - query-store-after'.") + fs.DurationVar(&cfg.QueryStoreAfter, "querier.query-store-after", 12*time.Hour, "The time after which a metric should be queried from storage and not just ingesters. 0 means all queries are sent to store. If this option is enabled, the time range of the query sent to the store-gateway will be manipulated to ensure the query end is not more recent than 'now - query-store-after'.") } type Querier struct { diff --git a/pkg/storegateway/block_filter.go b/pkg/storegateway/block_filter.go index 78780584a4..58f6162e01 100644 --- a/pkg/storegateway/block_filter.go +++ b/pkg/storegateway/block_filter.go @@ -9,10 +9,11 @@ import ( "github.com/grafana/dskit/ring" "github.com/oklog/ulid" "github.com/pkg/errors" - "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/prometheus/model/timestamp" + "github.com/grafana/pyroscope/pkg/objstore" "github.com/grafana/pyroscope/pkg/phlaredb/block" + "github.com/grafana/pyroscope/pkg/phlaredb/bucketindex" ) const ( @@ -21,11 +22,6 @@ const ( var errStoreGatewayUnhealthy = errors.New("store-gateway is unhealthy in the ring") -// GaugeVec hides something like a Prometheus GaugeVec or an extprom.TxGaugeVec. -type GaugeVec interface { - WithLabelValues(lvs ...string) prometheus.Gauge -} - type ShardingStrategy interface { // FilterUsers whose blocks should be loaded by the store-gateway. Returns the list of user IDs // that should be synced by the store-gateway. @@ -34,11 +30,7 @@ type ShardingStrategy interface { // FilterBlocks filters metas in-place keeping only blocks that should be loaded by the store-gateway. // The provided loaded map contains blocks which have been previously returned by this function and // are now loaded or loading in the store-gateway. - FilterBlocks(ctx context.Context, userID string, metas map[ulid.ULID]*block.Meta, loaded map[ulid.ULID]struct{}, synced GaugeVec) error -} - -type BlockMetaFilter interface { - Filter(ctx context.Context, metas map[ulid.ULID]*block.Meta, synced GaugeVec) error + FilterBlocks(ctx context.Context, userID string, metas map[ulid.ULID]*block.Meta, loaded map[ulid.ULID]struct{}, synced block.GaugeVec) error } type shardingMetadataFilterAdapter struct { @@ -96,7 +88,7 @@ func (s *ShuffleShardingStrategy) FilterUsers(_ context.Context, userIDs []strin } // FilterBlocks implements ShardingStrategy. -func (s *ShuffleShardingStrategy) FilterBlocks(_ context.Context, userID string, metas map[ulid.ULID]*block.Meta, loaded map[ulid.ULID]struct{}, synced GaugeVec) error { +func (s *ShuffleShardingStrategy) FilterBlocks(_ context.Context, userID string, metas map[ulid.ULID]*block.Meta, loaded map[ulid.ULID]struct{}, synced block.GaugeVec) error { // As a protection, ensure the store-gateway instance is healthy in the ring. If it's unhealthy because it's failing // to heartbeat or get updates from the ring, or even removed from the ring because of the auto-forget feature, then // keep the previously loaded blocks. @@ -179,7 +171,7 @@ func GetShuffleShardingSubring(ring *ring.Ring, userID string, limits ShardingLi return ring.ShuffleShard(userID, shardSize) } -func NewShardingMetadataFilterAdapter(userID string, strategy ShardingStrategy) BlockMetaFilter { +func NewShardingMetadataFilterAdapter(userID string, strategy ShardingStrategy) block.MetadataFilter { return &shardingMetadataFilterAdapter{ userID: userID, strategy: strategy, @@ -189,7 +181,7 @@ func NewShardingMetadataFilterAdapter(userID string, strategy ShardingStrategy) // Filter implements block.MetadataFilter. // This function is NOT safe for use by multiple goroutines concurrently. -func (a *shardingMetadataFilterAdapter) Filter(ctx context.Context, metas map[ulid.ULID]*block.Meta, synced GaugeVec) error { +func (a *shardingMetadataFilterAdapter) Filter(ctx context.Context, metas map[ulid.ULID]*block.Meta, synced block.GaugeVec) error { if err := a.strategy.FilterBlocks(ctx, a.userID, metas, a.lastBlocks, synced); err != nil { return err } @@ -214,7 +206,7 @@ func newMinTimeMetaFilter(limit time.Duration) *minTimeMetaFilter { return &minTimeMetaFilter{limit: limit} } -func (f *minTimeMetaFilter) Filter(_ context.Context, metas map[ulid.ULID]*block.Meta, synced GaugeVec) error { +func (f *minTimeMetaFilter) Filter(_ context.Context, metas map[ulid.ULID]*block.Meta, synced block.GaugeVec) error { if f.limit <= 0 { return nil } @@ -231,3 +223,66 @@ func (f *minTimeMetaFilter) Filter(_ context.Context, metas map[ulid.ULID]*block } return nil } + +type MetadataFilterWithBucketIndex interface { + // FilterWithBucketIndex is like Thanos MetadataFilter.Filter() but it provides in input the bucket index too. + FilterWithBucketIndex(ctx context.Context, metas map[ulid.ULID]*block.Meta, idx *bucketindex.Index, synced block.GaugeVec) error +} + +// IgnoreDeletionMarkFilter is like the Thanos IgnoreDeletionMarkFilter, but it also implements +// the MetadataFilterWithBucketIndex interface. +type IgnoreDeletionMarkFilter struct { + upstream *block.IgnoreDeletionMarkFilter + + delay time.Duration + deletionMarkMap map[ulid.ULID]*block.DeletionMark +} + +// NewIgnoreDeletionMarkFilter creates IgnoreDeletionMarkFilter. +func NewIgnoreDeletionMarkFilter(logger log.Logger, bkt objstore.BucketReader, delay time.Duration, concurrency int) *IgnoreDeletionMarkFilter { + return &IgnoreDeletionMarkFilter{ + upstream: block.NewIgnoreDeletionMarkFilter(logger, bkt, delay, concurrency), + delay: delay, + } +} + +// DeletionMarkBlocks returns blocks that were marked for deletion. +func (f *IgnoreDeletionMarkFilter) DeletionMarkBlocks() map[ulid.ULID]*block.DeletionMark { + // If the cached deletion marks exist it means the filter function was called with the bucket + // index, so it's safe to return it. + if f.deletionMarkMap != nil { + return f.deletionMarkMap + } + + return f.upstream.DeletionMarkBlocks() +} + +// Filter implements block.MetadataFilter. +func (f *IgnoreDeletionMarkFilter) Filter(ctx context.Context, metas map[ulid.ULID]*block.Meta, synced block.GaugeVec) error { + return f.upstream.Filter(ctx, metas, synced) +} + +// FilterWithBucketIndex implements MetadataFilterWithBucketIndex. +func (f *IgnoreDeletionMarkFilter) FilterWithBucketIndex(_ context.Context, metas map[ulid.ULID]*block.Meta, idx *bucketindex.Index, synced block.GaugeVec) error { + // Build a map of block deletion marks + marks := make(map[ulid.ULID]*block.DeletionMark, len(idx.BlockDeletionMarks)) + for _, mark := range idx.BlockDeletionMarks { + marks[mark.ID] = mark.BlockDeletionMark() + } + + // Keep it cached. + f.deletionMarkMap = marks + + for _, mark := range marks { + if _, ok := metas[mark.ID]; !ok { + continue + } + + if time.Since(time.Unix(mark.DeletionTime, 0)).Seconds() > f.delay.Seconds() { + synced.WithLabelValues(block.MarkedForDeletionMeta).Inc() + delete(metas, mark.ID) + } + } + + return nil +} diff --git a/pkg/storegateway/bucket.go b/pkg/storegateway/bucket.go index e8b480442c..8be24cbc55 100644 --- a/pkg/storegateway/bucket.go +++ b/pkg/storegateway/bucket.go @@ -14,7 +14,6 @@ import ( "github.com/oklog/ulid" "github.com/pkg/errors" "github.com/prometheus/common/model" - "github.com/samber/lo" phlareobj "github.com/grafana/pyroscope/pkg/objstore" "github.com/grafana/pyroscope/pkg/phlaredb" @@ -30,7 +29,9 @@ type BucketStoreStats struct { } type BucketStore struct { - bucket phlareobj.Bucket + bucket phlareobj.Bucket + fetcher block.MetadataFetcher + tenantID, syncDir string logger log.Logger @@ -39,18 +40,17 @@ type BucketStore struct { blocks map[ulid.ULID]*Block blockSet *bucketBlockSet - filters []BlockMetaFilter metrics *Metrics stats BucketStoreStats } -func NewBucketStore(bucket phlareobj.Bucket, tenantID string, syncDir string, filters []BlockMetaFilter, logger log.Logger, Metrics *Metrics) (*BucketStore, error) { +func NewBucketStore(bucket phlareobj.Bucket, fetcher block.MetadataFetcher, tenantID string, syncDir string, logger log.Logger, Metrics *Metrics) (*BucketStore, error) { s := &BucketStore{ + fetcher: fetcher, bucket: phlareobj.NewPrefixedBucket(bucket, tenantID+"/phlaredb"), tenantID: tenantID, syncDir: syncDir, logger: logger, - filters: filters, blockSet: newBucketBlockSet(), blocks: map[ulid.ULID]*Block{}, metrics: Metrics, @@ -100,7 +100,8 @@ func (s *BucketStore) getBlock(id ulid.ULID) *Block { } func (s *BucketStore) SyncBlocks(ctx context.Context) error { - metas, metaFetchErr := s.fetchBlocksMeta(ctx) + // TODO sounds like we should get the meta this is just a list of ids + metas, _, metaFetchErr := s.fetcher.Fetch(ctx) // For partial view allow adding new blocks at least. if metaFetchErr != nil && metas == nil { return metaFetchErr @@ -175,6 +176,15 @@ func (bs *BucketStore) addBlock(ctx context.Context, meta *block.Meta) (err erro b, err := func() (*Block, error) { bs.blocksMx.Lock() defer bs.blocksMx.Unlock() + // fetch the meta from the bucket + r, err := bs.bucket.Get(ctx, path.Join(meta.ULID.String(), block.MetaFilename)) + if err != nil { + return nil, errors.Wrap(err, "get meta") + } + meta, err := block.Read(r) + if err != nil { + return nil, errors.Wrap(err, "read meta") + } b, err := bs.createBlock(ctx, meta) if err != nil { return nil, errors.Wrap(err, "load block from disk") @@ -254,46 +264,6 @@ func (s *BucketStore) RemoveBlocksAndClose() error { return nil } -func (s *BucketStore) fetchBlocksMeta(ctx context.Context) (map[ulid.ULID]*block.Meta, error) { - var ( - to = time.Now() - from = to.Add(-time.Hour * 24 * 31) // todo make this configurable - ) - - var ( - metas []*block.Meta - mtx sync.Mutex - ) - - start := time.Now() - level.Debug(s.logger).Log("msg", "fetching blocks meta", "from", from, "to", to) - defer func() { - level.Debug(s.logger).Log("msg", "fetched blocks meta", "total", len(metas), "elapsed", time.Since(start)) - }() - if err := block.IterBlockMetas(ctx, s.bucket, from, to, func(m *block.Meta) { - mtx.Lock() - defer mtx.Unlock() - metas = append(metas, m) - }); err != nil { - return nil, errors.Wrap(err, "iter block metas") - } - - metaMap := lo.SliceToMap(metas, func(item *block.Meta) (ulid.ULID, *block.Meta) { - return item.ULID, item - }) - if len(metaMap) == 0 { - return nil, nil - } - for _, filter := range s.filters { - // NOTE: filter can update synced metric accordingly to the reason of the exclude. - // todo: wire up the filter with the metrics. - if err := filter.Filter(ctx, metaMap, s.metrics.Synced); err != nil { - return nil, errors.Wrap(err, "filter metas") - } - } - return metaMap, nil -} - // bucketBlockSet holds all blocks. type bucketBlockSet struct { mtx sync.RWMutex diff --git a/pkg/storegateway/bucket_index_metadata_fetcher.go b/pkg/storegateway/bucket_index_metadata_fetcher.go new file mode 100644 index 0000000000..cd9b36befe --- /dev/null +++ b/pkg/storegateway/bucket_index_metadata_fetcher.go @@ -0,0 +1,121 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/cortexproject/cortex/blob/master/pkg/storegateway/bucket_index_metadata_fetcher.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Cortex Authors. + +package storegateway + +import ( + "context" + "time" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/oklog/ulid" + "github.com/pkg/errors" + "github.com/prometheus/client_golang/prometheus" + + "github.com/grafana/pyroscope/pkg/objstore" + "github.com/grafana/pyroscope/pkg/phlaredb/block" + "github.com/grafana/pyroscope/pkg/phlaredb/bucketindex" +) + +const ( + corruptedBucketIndex = "corrupted-bucket-index" + noBucketIndex = "no-bucket-index" +) + +// BucketIndexMetadataFetcher is a Thanos MetadataFetcher implementation leveraging on the Mimir bucket index. +type BucketIndexMetadataFetcher struct { + userID string + bkt objstore.Bucket + cfgProvider objstore.TenantConfigProvider + logger log.Logger + filters []block.MetadataFilter + metrics *block.FetcherMetrics +} + +func NewBucketIndexMetadataFetcher( + userID string, + bkt objstore.Bucket, + cfgProvider objstore.TenantConfigProvider, + logger log.Logger, + reg prometheus.Registerer, + filters []block.MetadataFilter, +) *BucketIndexMetadataFetcher { + return &BucketIndexMetadataFetcher{ + userID: userID, + bkt: bkt, + cfgProvider: cfgProvider, + logger: logger, + filters: filters, + metrics: block.NewFetcherMetrics(reg, [][]string{{corruptedBucketIndex}, {noBucketIndex}, {minTimeExcludedMeta}}), + } +} + +// Fetch implements block.MetadataFetcher. Not goroutine-safe. +func (f *BucketIndexMetadataFetcher) Fetch(ctx context.Context) (metas map[ulid.ULID]*block.Meta, partial map[ulid.ULID]error, err error) { + f.metrics.ResetTx() + + start := time.Now() + defer func() { + f.metrics.SyncDuration.Observe(time.Since(start).Seconds()) + if err != nil { + f.metrics.SyncFailures.Inc() + } + }() + f.metrics.Syncs.Inc() + + // Fetch the bucket index. + idx, err := bucketindex.ReadIndex(ctx, f.bkt, f.userID, f.cfgProvider, f.logger) + if errors.Is(err, bucketindex.ErrIndexNotFound) { + // This is a legit case happening when the first blocks of a tenant have recently been uploaded by ingesters + // and their bucket index has not been created yet. + f.metrics.Synced.WithLabelValues(noBucketIndex).Set(1) + f.metrics.Submit() + + return nil, nil, nil + } + if errors.Is(err, bucketindex.ErrIndexCorrupted) { + // In case a single tenant bucket index is corrupted, we don't want the store-gateway to fail at startup + // because unable to fetch blocks metadata. We'll act as if the tenant has no bucket index, but the query + // will fail anyway in the querier (the querier fails in the querier if bucket index is corrupted). + level.Error(f.logger).Log("msg", "corrupted bucket index found", "user", f.userID, "err", err) + f.metrics.Synced.WithLabelValues(corruptedBucketIndex).Set(1) + f.metrics.Submit() + + return nil, nil, nil + } + if err != nil { + f.metrics.Synced.WithLabelValues(block.FailedMeta).Set(1) + f.metrics.Submit() + + return nil, nil, errors.Wrapf(err, "read bucket index") + } + + // Build block metas out of the index. + metas = make(map[ulid.ULID]*block.Meta, len(idx.Blocks)) + for _, b := range idx.Blocks { + metas[b.ID] = b.Meta() + } + + for _, filter := range f.filters { + var err error + + // NOTE: filter can update synced metric accordingly to the reason of the exclude. + if customFilter, ok := filter.(MetadataFilterWithBucketIndex); ok { + err = customFilter.FilterWithBucketIndex(ctx, metas, idx, f.metrics.Synced) + } else { + err = filter.Filter(ctx, metas, f.metrics.Synced) + } + + if err != nil { + return nil, nil, errors.Wrap(err, "filter metas") + } + } + + f.metrics.Synced.WithLabelValues(block.LoadedMeta).Set(float64(len(metas))) + f.metrics.Submit() + + return metas, nil, nil +} diff --git a/pkg/storegateway/bucket_index_metadata_fetcher_test.go b/pkg/storegateway/bucket_index_metadata_fetcher_test.go new file mode 100644 index 0000000000..041f885a2c --- /dev/null +++ b/pkg/storegateway/bucket_index_metadata_fetcher_test.go @@ -0,0 +1,198 @@ +// SPDX-License-Identifier: AGPL-3.0-only +// Provenance-includes-location: https://github.com/cortexproject/cortex/blob/master/pkg/storegateway/bucket_index_metadata_fetcher_test.go +// Provenance-includes-license: Apache-2.0 +// Provenance-includes-copyright: The Cortex Authors. + +package storegateway + +import ( + "bytes" + "context" + "path" + "strings" + "testing" + "time" + + "github.com/go-kit/log" + "github.com/grafana/dskit/concurrency" + "github.com/oklog/ulid" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/testutil" + "github.com/prometheus/common/model" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + + "github.com/grafana/pyroscope/pkg/objstore" + objstore_testutil "github.com/grafana/pyroscope/pkg/objstore/testutil" + "github.com/grafana/pyroscope/pkg/phlaredb/block" + "github.com/grafana/pyroscope/pkg/phlaredb/bucketindex" +) + +func TestBucketIndexMetadataFetcher_Fetch(t *testing.T) { + const userID = "user-1" + + ctx := context.Background() + + bkt, _ := objstore_testutil.NewFilesystemBucket(t, ctx, t.TempDir()) + reg := prometheus.NewPedanticRegistry() + now := time.Now() + logs := &concurrency.SyncBuffer{} + logger := log.NewLogfmtLogger(logs) + + // Create a bucket index. + block1 := &bucketindex.Block{ID: ulid.MustNew(1, nil)} + block2 := &bucketindex.Block{ID: ulid.MustNew(2, nil)} + block3 := &bucketindex.Block{ID: ulid.MustNew(3, nil)} + block4 := &bucketindex.Block{ID: ulid.MustNew(4, nil), MinTime: model.Time(now.Add(-30 * time.Minute).UnixMilli())} // Has most-recent data, to be ignored by minTimeMetaFilter. + + mark1 := &bucketindex.BlockDeletionMark{ID: block1.ID, DeletionTime: now.Add(-time.Hour).Unix()} // Below the ignore delay threshold. + mark2 := &bucketindex.BlockDeletionMark{ID: block2.ID, DeletionTime: now.Add(-3 * time.Hour).Unix()} // Above the ignore delay threshold. + + require.NoError(t, bucketindex.WriteIndex(ctx, bkt, userID, nil, &bucketindex.Index{ + Version: bucketindex.IndexVersion1, + Blocks: bucketindex.Blocks{block1, block2, block3, block4}, + BlockDeletionMarks: bucketindex.BlockDeletionMarks{mark1, mark2}, + UpdatedAt: now.Unix(), + })) + + // Create a metadata fetcher with filters. + filters := []block.MetadataFilter{ + NewIgnoreDeletionMarkFilter(logger, objstore.NewUserBucketClient(userID, bkt, nil), 2*time.Hour, 1), + newMinTimeMetaFilter(1 * time.Hour), + } + + fetcher := NewBucketIndexMetadataFetcher(userID, bkt, nil, logger, reg, filters) + metas, partials, err := fetcher.Fetch(ctx) + require.NoError(t, err) + assert.Equal(t, map[ulid.ULID]*block.Meta{ + block1.ID: block1.Meta(), + block3.ID: block3.Meta(), + }, metas) + assert.Empty(t, partials) + assert.Empty(t, logs) + + assert.NoError(t, testutil.GatherAndCompare(reg, bytes.NewBufferString(` + # HELP blocks_meta_sync_failures_total Total blocks metadata synchronization failures + # TYPE blocks_meta_sync_failures_total counter + blocks_meta_sync_failures_total 0 + + # HELP blocks_meta_synced Number of block metadata synced + # TYPE blocks_meta_synced gauge + blocks_meta_synced{state="corrupted-bucket-index"} 0 + blocks_meta_synced{state="corrupted-meta-json"} 0 + blocks_meta_synced{state="duplicate"} 0 + blocks_meta_synced{state="failed"} 0 + blocks_meta_synced{state="label-excluded"} 0 + blocks_meta_synced{state="loaded"} 2 + blocks_meta_synced{state="marked-for-deletion"} 1 + blocks_meta_synced{state="marked-for-no-compact"} 0 + blocks_meta_synced{state="no-bucket-index"} 0 + blocks_meta_synced{state="no-meta-json"} 0 + blocks_meta_synced{state="time-excluded"} 0 + blocks_meta_synced{state="min-time-excluded"} 1 + + # HELP blocks_meta_syncs_total Total blocks metadata synchronization attempts + # TYPE blocks_meta_syncs_total counter + blocks_meta_syncs_total 1 + `), + "blocks_meta_sync_failures_total", + "blocks_meta_synced", + "blocks_meta_syncs_total", + )) +} + +func TestBucketIndexMetadataFetcher_Fetch_NoBucketIndex(t *testing.T) { + const userID = "user-1" + + ctx := context.Background() + bkt, _ := objstore_testutil.NewFilesystemBucket(t, ctx, t.TempDir()) + reg := prometheus.NewPedanticRegistry() + logs := &concurrency.SyncBuffer{} + logger := log.NewLogfmtLogger(logs) + + fetcher := NewBucketIndexMetadataFetcher(userID, bkt, nil, logger, reg, nil) + metas, partials, err := fetcher.Fetch(ctx) + require.NoError(t, err) + assert.Empty(t, metas) + assert.Empty(t, partials) + assert.Empty(t, logs) + + assert.NoError(t, testutil.GatherAndCompare(reg, bytes.NewBufferString(` + # HELP blocks_meta_sync_failures_total Total blocks metadata synchronization failures + # TYPE blocks_meta_sync_failures_total counter + blocks_meta_sync_failures_total 0 + + # HELP blocks_meta_synced Number of block metadata synced + # TYPE blocks_meta_synced gauge + blocks_meta_synced{state="corrupted-bucket-index"} 0 + blocks_meta_synced{state="corrupted-meta-json"} 0 + blocks_meta_synced{state="duplicate"} 0 + blocks_meta_synced{state="failed"} 0 + blocks_meta_synced{state="label-excluded"} 0 + blocks_meta_synced{state="loaded"} 0 + blocks_meta_synced{state="marked-for-deletion"} 0 + blocks_meta_synced{state="marked-for-no-compact"} 0 + blocks_meta_synced{state="no-bucket-index"} 1 + blocks_meta_synced{state="no-meta-json"} 0 + blocks_meta_synced{state="time-excluded"} 0 + blocks_meta_synced{state="min-time-excluded"} 0 + + # HELP blocks_meta_syncs_total Total blocks metadata synchronization attempts + # TYPE blocks_meta_syncs_total counter + blocks_meta_syncs_total 1 + `), + "blocks_meta_sync_failures_total", + "blocks_meta_synced", + "blocks_meta_syncs_total", + )) +} + +func TestBucketIndexMetadataFetcher_Fetch_CorruptedBucketIndex(t *testing.T) { + const userID = "user-1" + + ctx := context.Background() + + bkt, _ := objstore_testutil.NewFilesystemBucket(t, ctx, t.TempDir()) + reg := prometheus.NewPedanticRegistry() + logs := &concurrency.SyncBuffer{} + logger := log.NewLogfmtLogger(logs) + + // Upload a corrupted bucket index. + require.NoError(t, bkt.Upload(ctx, path.Join(userID, bucketindex.IndexCompressedFilename), strings.NewReader("invalid}!"))) + + fetcher := NewBucketIndexMetadataFetcher(userID, bkt, nil, logger, reg, nil) + metas, partials, err := fetcher.Fetch(ctx) + require.NoError(t, err) + assert.Empty(t, metas) + assert.Empty(t, partials) + assert.Regexp(t, "corrupted bucket index found", logs) + + assert.NoError(t, testutil.GatherAndCompare(reg, bytes.NewBufferString(` + # HELP blocks_meta_sync_failures_total Total blocks metadata synchronization failures + # TYPE blocks_meta_sync_failures_total counter + blocks_meta_sync_failures_total 0 + + # HELP blocks_meta_synced Number of block metadata synced + # TYPE blocks_meta_synced gauge + blocks_meta_synced{state="corrupted-bucket-index"} 1 + blocks_meta_synced{state="corrupted-meta-json"} 0 + blocks_meta_synced{state="duplicate"} 0 + blocks_meta_synced{state="failed"} 0 + blocks_meta_synced{state="label-excluded"} 0 + blocks_meta_synced{state="loaded"} 0 + blocks_meta_synced{state="marked-for-deletion"} 0 + blocks_meta_synced{state="marked-for-no-compact"} 0 + blocks_meta_synced{state="no-bucket-index"} 0 + blocks_meta_synced{state="no-meta-json"} 0 + blocks_meta_synced{state="time-excluded"} 0 + blocks_meta_synced{state="min-time-excluded"} 0 + + # HELP blocks_meta_syncs_total Total blocks metadata synchronization attempts + # TYPE blocks_meta_syncs_total counter + blocks_meta_syncs_total 1 + `), + "blocks_meta_sync_failures_total", + "blocks_meta_synced", + "blocks_meta_syncs_total", + )) +} diff --git a/pkg/storegateway/bucket_stores.go b/pkg/storegateway/bucket_stores.go index 5f8179e701..656880a4f7 100644 --- a/pkg/storegateway/bucket_stores.go +++ b/pkg/storegateway/bucket_stores.go @@ -17,6 +17,7 @@ import ( "github.com/prometheus/client_golang/prometheus/promauto" phlareobj "github.com/grafana/pyroscope/pkg/objstore" + "github.com/grafana/pyroscope/pkg/phlaredb/block" "github.com/grafana/pyroscope/pkg/phlaredb/bucket" "github.com/grafana/pyroscope/pkg/util" ) @@ -24,10 +25,12 @@ import ( var errBucketStoreNotFound = errors.New("bucket store not found") type BucketStoreConfig struct { - SyncDir string `yaml:"sync_dir"` - SyncInterval time.Duration `yaml:"sync_interval" category:"advanced"` - TenantSyncConcurrency int `yaml:"tenant_sync_concurrency" category:"advanced"` - IgnoreBlocksWithin time.Duration `yaml:"ignore_blocks_within" category:"advanced"` + SyncDir string `yaml:"sync_dir"` + SyncInterval time.Duration `yaml:"sync_interval" category:"advanced"` + TenantSyncConcurrency int `yaml:"tenant_sync_concurrency" category:"advanced"` + IgnoreBlocksWithin time.Duration `yaml:"ignore_blocks_within" category:"advanced"` + MetaSyncConcurrency int `yaml:"meta_sync_concurrency" category:"advanced"` + IgnoreDeletionMarksDelay time.Duration `yaml:"ignore_deletion_mark_delay" category:"advanced"` } // RegisterFlags registers the BucketStore flags @@ -41,7 +44,7 @@ func (cfg *BucketStoreConfig) RegisterFlags(f *flag.FlagSet, logger log.Logger) f.StringVar(&cfg.SyncDir, "blocks-storage.bucket-store.sync-dir", "./data/pyroscope-sync/", "Directory to store synchronized pyroscope block headers. This directory is not required to be persisted between restarts, but it's highly recommended in order to improve the store-gateway startup time.") f.DurationVar(&cfg.SyncInterval, "blocks-storage.bucket-store.sync-interval", 15*time.Minute, "How frequently to scan the bucket, or to refresh the bucket index (if enabled), in order to look for changes (new blocks shipped by ingesters and blocks deleted by retention or compaction).") f.IntVar(&cfg.TenantSyncConcurrency, "blocks-storage.bucket-store.tenant-sync-concurrency", 10, "Maximum number of concurrent tenants synching blocks.") - f.DurationVar(&cfg.IgnoreBlocksWithin, "blocks-storage.bucket-store.ignore-blocks-within", 2*time.Hour, "Blocks with minimum time within this duration are ignored, and not loaded by store-gateway. Useful when used together with -querier.query-store-after to prevent loading young blocks, because there are usually many of them (depending on number of ingesters) and they are not yet compacted. Negative values or 0 disable the filter.") + f.DurationVar(&cfg.IgnoreBlocksWithin, "blocks-storage.bucket-store.ignore-blocks-within", 10*time.Hour, "Blocks with minimum time within this duration are ignored, and not loaded by store-gateway. Useful when used together with -querier.query-store-after to prevent loading young blocks, because there are usually many of them (depending on number of ingesters) and they are not yet compacted. Negative values or 0 disable the filter.") // f.Uint64Var(&cfg.MaxChunkPoolBytes, "blocks-storage.bucket-store.max-chunk-pool-bytes", uint64(2*units.Gibibyte), "Max size - in bytes - of a chunks pool, used to reduce memory allocations. The pool is shared across all tenants. 0 to disable the limit.") // f.IntVar(&cfg.ChunkPoolMinBucketSizeBytes, "blocks-storage.bucket-store.chunk-pool-min-bucket-size-bytes", ChunkPoolDefaultMinBucketSize, "Size - in bytes - of the smallest chunks pool bucket.") @@ -49,10 +52,10 @@ func (cfg *BucketStoreConfig) RegisterFlags(f *flag.FlagSet, logger log.Logger) // f.Uint64Var(&cfg.SeriesHashCacheMaxBytes, "blocks-storage.bucket-store.series-hash-cache-max-size-bytes", uint64(1*units.Gibibyte), "Max size - in bytes - of the in-memory series hash cache. The cache is shared across all tenants and it's used only when query sharding is enabled.") // f.IntVar(&cfg.MaxConcurrent, "blocks-storage.bucket-store.max-concurrent", 100, "Max number of concurrent queries to execute against the long-term storage. The limit is shared across all tenants.") // f.IntVar(&cfg.BlockSyncConcurrency, "blocks-storage.bucket-store.block-sync-concurrency", 20, "Maximum number of concurrent blocks synching per tenant.") - // f.IntVar(&cfg.MetaSyncConcurrency, "blocks-storage.bucket-store.meta-sync-concurrency", 20, "Number of Go routines to use when syncing block meta files from object storage per tenant.") + f.IntVar(&cfg.MetaSyncConcurrency, "blocks-storage.bucket-store.meta-sync-concurrency", 20, "Number of Go routines to use when syncing block meta files from object storage per tenant.") // f.DurationVar(&cfg.DeprecatedConsistencyDelay, consistencyDelayFlag, 0, "Minimum age of a block before it's being read. Set it to safe value (e.g 30m) if your object storage is eventually consistent. GCS and S3 are (roughly) strongly consistent.") - // f.DurationVar(&cfg.IgnoreDeletionMarksDelay, "blocks-storage.bucket-store.ignore-deletion-marks-delay", time.Hour*1, "Duration after which the blocks marked for deletion will be filtered out while fetching blocks. "+ - // "The idea of ignore-deletion-marks-delay is to ignore blocks that are marked for deletion with some delay. This ensures store can still serve blocks that are meant to be deleted but do not have a replacement yet.") + f.DurationVar(&cfg.IgnoreDeletionMarksDelay, "blocks-storage.bucket-store.ignore-deletion-marks-delay", time.Hour*1, "Duration after which the blocks marked for deletion will be filtered out while fetching blocks. "+ + "The idea of ignore-deletion-marks-delay is to ignore blocks that are marked for deletion with some delay. This ensures store can still serve blocks that are meant to be deleted but do not have a replacement yet.") // f.IntVar(&cfg.PostingOffsetsInMemSampling, "blocks-storage.bucket-store.posting-offsets-in-mem-sampling", DefaultPostingOffsetInMemorySampling, "Controls what is the ratio of postings offsets that the store will hold in memory.") // f.BoolVar(&cfg.IndexHeaderLazyLoadingEnabled, "blocks-storage.bucket-store.index-header-lazy-loading-enabled", true, "If enabled, store-gateway will lazy load an index-header only once required by a query.") // f.DurationVar(&cfg.IndexHeaderLazyLoadingIdleTimeout, "blocks-storage.bucket-store.index-header-lazy-loading-idle-timeout", 60*time.Minute, "If index-header lazy loading is enabled and this setting is > 0, the store-gateway will offload unused index-headers after 'idle timeout' inactivity.") @@ -85,6 +88,18 @@ func (cfg *BucketStoreConfig) Validate(logger log.Logger) error { return nil } +type BucketIndexConfig struct { + UpdateOnErrorInterval time.Duration `yaml:"update_on_error_interval" category:"advanced"` + IdleTimeout time.Duration `yaml:"idle_timeout" category:"advanced"` + MaxStalePeriod time.Duration `yaml:"max_stale_period" category:"advanced"` +} + +func (cfg *BucketIndexConfig) RegisterFlagsWithPrefix(f *flag.FlagSet, prefix string) { + f.DurationVar(&cfg.UpdateOnErrorInterval, prefix+"update-on-error-interval", time.Minute, "How frequently a bucket index, which previously failed to load, should be tried to load again. This option is used only by querier.") + f.DurationVar(&cfg.IdleTimeout, prefix+"idle-timeout", time.Hour, "How long a unused bucket index should be cached. Once this timeout expires, the unused bucket index is removed from the in-memory cache. This option is used only by querier.") + f.DurationVar(&cfg.MaxStalePeriod, prefix+"max-stale-period", time.Hour, "The maximum allowed age of a bucket index (last updated) before queries start failing because the bucket index is too old. The bucket index is periodically updated by the compactor, and this check is enforced in the querier (at query time).") +} + type BucketStores struct { storageBucket phlareobj.Bucket cfg BucketStoreConfig @@ -307,17 +322,27 @@ func (bs *BucketStores) getOrCreateStore(userID string) (*BucketStore, error) { level.Info(userLogger).Log("msg", "creating user bucket store") // The sharding strategy filter MUST be before the ones we create here (order matters). - filters := []BlockMetaFilter{ + filters := []block.MetadataFilter{ NewShardingMetadataFilterAdapter(userID, bs.shardingStrategy), - // block.NewConsistencyDelayMetaFilter(userLogger, u.cfg.BucketStore.DeprecatedConsistencyDelay, fetcherReg), newMinTimeMetaFilter(bs.cfg.IgnoreBlocksWithin), + NewIgnoreDeletionMarkFilter(userLogger, bs.storageBucket, bs.cfg.IgnoreDeletionMarksDelay, bs.cfg.MetaSyncConcurrency), } + fetcherReg := prometheus.NewRegistry() + + fetcher := NewBucketIndexMetadataFetcher( + userID, + bs.storageBucket, + bs.limits, + bs.logger, + fetcherReg, + filters, + ) s, err := NewBucketStore( bs.storageBucket, + fetcher, userID, bs.syncDirForUser(userID), - filters, userLogger, bs.metrics, ) diff --git a/pkg/storegateway/gateway.go b/pkg/storegateway/gateway.go index ab0e172cad..d7230163e3 100644 --- a/pkg/storegateway/gateway.go +++ b/pkg/storegateway/gateway.go @@ -34,6 +34,7 @@ var errInvalidTenantShardSize = errors.New("invalid tenant shard size, the value type Limits interface { ShardingLimits + phlareobj.TenantConfigProvider } // ShardingLimits is the interface that should be implemented by the limits provider, From ab041e3623ffa88d7fc0409f2ab569630e16a526 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Tue, 3 Oct 2023 14:59:04 +0200 Subject: [PATCH 50/74] Fixes helm to not use a disk --- .../reference-configuration-parameters/index.md | 17 +++++++++++++++-- .../helm/pyroscope/values-micro-services.yaml | 2 ++ .../jsonnet/values-micro-services.json | 3 +++ 3 files changed, 20 insertions(+), 2 deletions(-) diff --git a/docs/sources/configure-server/reference-configuration-parameters/index.md b/docs/sources/configure-server/reference-configuration-parameters/index.md index 9cc34524d0..b07e5b7425 100644 --- a/docs/sources/configure-server/reference-configuration-parameters/index.md +++ b/docs/sources/configure-server/reference-configuration-parameters/index.md @@ -476,7 +476,20 @@ store_gateway: # are usually many of them (depending on number of ingesters) and they are # not yet compacted. Negative values or 0 disable the filter. # CLI flag: -blocks-storage.bucket-store.ignore-blocks-within - [ignore_blocks_within: | default = 2h] + [ignore_blocks_within: | default = 10h] + + # Number of Go routines to use when syncing block meta files from object + # storage per tenant. + # CLI flag: -blocks-storage.bucket-store.meta-sync-concurrency + [meta_sync_concurrency: | default = 20] + + # Duration after which the blocks marked for deletion will be filtered out + # while fetching blocks. The idea of ignore-deletion-marks-delay is to + # ignore blocks that are marked for deletion with some delay. This ensures + # store can still serve blocks that are meant to be deleted but do not have + # a replacement yet. + # CLI flag: -blocks-storage.bucket-store.ignore-deletion-marks-delay + [ignore_deletion_mark_delay: | default = 1h] # The memberlist block configures the Gossip memberlist. [memberlist: ] @@ -1424,7 +1437,7 @@ pool_config: # the time range of the query sent to the store-gateway will be manipulated to # ensure the query end is not more recent than 'now - query-store-after'. # CLI flag: -querier.query-store-after -[query_store_after: | default = 4h] +[query_store_after: | default = 12h] ``` ### query_frontend diff --git a/operations/pyroscope/helm/pyroscope/values-micro-services.yaml b/operations/pyroscope/helm/pyroscope/values-micro-services.yaml index 2fe2cfe845..fcfd81e769 100644 --- a/operations/pyroscope/helm/pyroscope/values-micro-services.yaml +++ b/operations/pyroscope/helm/pyroscope/values-micro-services.yaml @@ -53,6 +53,8 @@ pyroscope: compactor: kind: StatefulSet replicaCount: 3 + persistence: + enabled: false resources: limits: memory: 16Gi diff --git a/operations/pyroscope/jsonnet/values-micro-services.json b/operations/pyroscope/jsonnet/values-micro-services.json index d764597ca1..60a487d783 100644 --- a/operations/pyroscope/jsonnet/values-micro-services.json +++ b/operations/pyroscope/jsonnet/values-micro-services.json @@ -6,6 +6,9 @@ "components": { "compactor": { "kind": "StatefulSet", + "persistence": { + "enabled": false + }, "replicaCount": 3, "resources": { "limits": { From 17159a668fab4ec448214f2fb6e4f028cbd4c3b0 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Tue, 3 Oct 2023 19:42:42 +0200 Subject: [PATCH 51/74] Use stable time format for logs --- pkg/compactor/compactor_test.go | 16 ++++++++-------- pkg/compactor/split_merge_job.go | 3 ++- 2 files changed, 10 insertions(+), 9 deletions(-) diff --git a/pkg/compactor/compactor_test.go b/pkg/compactor/compactor_test.go index c35fe9ee3d..4fe68b8468 100644 --- a/pkg/compactor/compactor_test.go +++ b/pkg/compactor/compactor_test.go @@ -576,7 +576,7 @@ func TestMultitenantCompactor_ShouldIterateOverUsersAndRunCompaction(t *testing. `level=info component=compactor msg="starting compaction of user blocks" tenant=user-1`, `level=info component=compactor tenant=user-1 msg="start sync of metas"`, `level=info component=compactor tenant=user-1 msg="start of GC"`, - `level=debug component=compactor tenant=user-1 msg="grouper found a compactable blocks group" groupKey=0@17241709254077376921-merge--1574776800000-1574784000000 job="stage: merge, range start: 1574776800000, range end: 1574784000000, shard: , blocks: 01DTVP434PA9VFXSW2JKB3392D (min time: 2019-11-26 14:00:00 +0000 UTC, max time: 2019-11-26 16:00:00 +0000 UTC),01FS51A7GQ1RQWV35DBVYQM4KF (min time: 2019-11-26 14:00:00 +0000 UTC, max time: 2019-11-26 16:00:00 +0000 UTC)"`, + `level=debug component=compactor tenant=user-1 msg="grouper found a compactable blocks group" groupKey=0@17241709254077376921-merge--1574776800000-1574784000000 job="stage: merge, range start: 1574776800000, range end: 1574784000000, shard: , blocks: 01DTVP434PA9VFXSW2JKB3392D (min time: 2019-11-26T14:00:00Z, max time: 2019-11-26T16:00:00Z),01FS51A7GQ1RQWV35DBVYQM4KF (min time: 2019-11-26T14:00:00Z, max time: 2019-11-26T16:00:00Z)"`, `level=info component=compactor tenant=user-1 msg="start of compactions"`, `level=info component=compactor tenant=user-1 groupKey=0@17241709254077376921-merge--1574776800000-1574784000000 msg="compaction job succeeded"`, `level=info component=compactor tenant=user-1 msg="compaction iterations done"`, @@ -584,7 +584,7 @@ func TestMultitenantCompactor_ShouldIterateOverUsersAndRunCompaction(t *testing. `level=info component=compactor msg="starting compaction of user blocks" tenant=user-2`, `level=info component=compactor tenant=user-2 msg="start sync of metas"`, `level=info component=compactor tenant=user-2 msg="start of GC"`, - `level=debug component=compactor tenant=user-2 msg="grouper found a compactable blocks group" groupKey=0@17241709254077376921-merge--1574776800000-1574784000000 job="stage: merge, range start: 1574776800000, range end: 1574784000000, shard: , blocks: 01DTW0ZCPDDNV4BV83Q2SV4QAZ (min time: 2019-11-26 14:00:00 +0000 UTC, max time: 2019-11-26 16:00:00 +0000 UTC),01FRSF035J26D6CGX7STCSD1KG (min time: 2019-11-26 14:00:00 +0000 UTC, max time: 2019-11-26 16:00:00 +0000 UTC)"`, + `level=debug component=compactor tenant=user-2 msg="grouper found a compactable blocks group" groupKey=0@17241709254077376921-merge--1574776800000-1574784000000 job="stage: merge, range start: 1574776800000, range end: 1574784000000, shard: , blocks: 01DTW0ZCPDDNV4BV83Q2SV4QAZ (min time: 2019-11-26T14:00:00Z, max time: 2019-11-26T16:00:00Z),01FRSF035J26D6CGX7STCSD1KG (min time: 2019-11-26T14:00:00Z, max time: 2019-11-26T16:00:00Z)"`, `level=info component=compactor tenant=user-2 msg="start of compactions"`, `level=info component=compactor tenant=user-2 groupKey=0@17241709254077376921-merge--1574776800000-1574784000000 msg="compaction job succeeded"`, `level=info component=compactor tenant=user-2 msg="compaction iterations done"`, @@ -717,8 +717,8 @@ func TestMultitenantCompactor_ShouldStopCompactingTenantOnReachingMaxCompactionT `level=info component=compactor msg="starting compaction of user blocks" tenant=user-1`, `level=info component=compactor tenant=user-1 msg="start sync of metas"`, `level=info component=compactor tenant=user-1 msg="start of GC"`, - `level=debug component=compactor tenant=user-1 msg="grouper found a compactable blocks group" groupKey=0@12695595599644216241-merge--1574776800000-1574784000000 job="stage: merge, range start: 1574776800000, range end: 1574784000000, shard: , blocks: 01FN3VCQV5X342W2ZKMQQXAZRX (min time: 2019-11-26 14:00:00 +0000 UTC, max time: 2019-11-26 16:00:00 +0000 UTC),01FRQGQB7RWQ2TS0VWA82QTPXE (min time: 2019-11-26 14:00:00 +0000 UTC, max time: 2019-11-26 16:00:00 +0000 UTC)"`, - `level=debug component=compactor tenant=user-1 msg="grouper found a compactable blocks group" groupKey=0@414047632870839233-merge--1574776800000-1574784000000 job="stage: merge, range start: 1574776800000, range end: 1574784000000, shard: , blocks: 01DTVP434PA9VFXSW2JKB3392D (min time: 2019-11-26 14:00:00 +0000 UTC, max time: 2019-11-26 16:00:00 +0000 UTC),01FS51A7GQ1RQWV35DBVYQM4KF (min time: 2019-11-26 14:00:00 +0000 UTC, max time: 2019-11-26 16:00:00 +0000 UTC)"`, + `level=debug component=compactor tenant=user-1 msg="grouper found a compactable blocks group" groupKey=0@12695595599644216241-merge--1574776800000-1574784000000 job="stage: merge, range start: 1574776800000, range end: 1574784000000, shard: , blocks: 01FN3VCQV5X342W2ZKMQQXAZRX (min time: 2019-11-26T14:00:00Z, max time: 2019-11-26T16:00:00Z),01FRQGQB7RWQ2TS0VWA82QTPXE (min time: 2019-11-26T14:00:00Z, max time: 2019-11-26T16:00:00Z)"`, + `level=debug component=compactor tenant=user-1 msg="grouper found a compactable blocks group" groupKey=0@414047632870839233-merge--1574776800000-1574784000000 job="stage: merge, range start: 1574776800000, range end: 1574784000000, shard: , blocks: 01DTVP434PA9VFXSW2JKB3392D (min time: 2019-11-26T14:00:00Z, max time: 2019-11-26T16:00:00Z),01FS51A7GQ1RQWV35DBVYQM4KF (min time: 2019-11-26T14:00:00Z, max time: 2019-11-26T16:00:00Z)"`, `level=info component=compactor tenant=user-1 msg="start of compactions"`, `level=info component=compactor tenant=user-1 msg="max compaction time reached, no more compactions will be started"`, `level=info component=compactor tenant=user-1 groupKey=0@12695595599644216241-merge--1574776800000-1574784000000 msg="compaction job succeeded"`, @@ -1070,7 +1070,7 @@ func TestMultitenantCompactor_ShouldCompactAllUsersOnShardingEnabledButOnlyOneIn `level=info component=compactor msg="starting compaction of user blocks" tenant=user-1`, `level=info component=compactor tenant=user-1 msg="start sync of metas"`, `level=info component=compactor tenant=user-1 msg="start of GC"`, - `level=debug component=compactor tenant=user-1 msg="grouper found a compactable blocks group" groupKey=0@17241709254077376921-merge--1574776800000-1574784000000 job="stage: merge, range start: 1574776800000, range end: 1574784000000, shard: , blocks: 01DTVP434PA9VFXSW2JKB3392D (min time: 2019-11-26 14:00:00 +0000 UTC, max time: 2019-11-26 16:00:00 +0000 UTC),01FSTQ95C8FS0ZAGTQS2EF1NEG (min time: 2019-11-26 14:00:00 +0000 UTC, max time: 2019-11-26 16:00:00 +0000 UTC)"`, + `level=debug component=compactor tenant=user-1 msg="grouper found a compactable blocks group" groupKey=0@17241709254077376921-merge--1574776800000-1574784000000 job="stage: merge, range start: 1574776800000, range end: 1574784000000, shard: , blocks: 01DTVP434PA9VFXSW2JKB3392D (min time: 2019-11-26T14:00:00Z, max time: 2019-11-26T16:00:00Z),01FSTQ95C8FS0ZAGTQS2EF1NEG (min time: 2019-11-26T14:00:00Z, max time: 2019-11-26T16:00:00Z)"`, `level=info component=compactor tenant=user-1 msg="start of compactions"`, `level=info component=compactor tenant=user-1 groupKey=0@17241709254077376921-merge--1574776800000-1574784000000 msg="compaction job succeeded"`, `level=info component=compactor tenant=user-1 msg="compaction iterations done"`, @@ -1078,7 +1078,7 @@ func TestMultitenantCompactor_ShouldCompactAllUsersOnShardingEnabledButOnlyOneIn `level=info component=compactor msg="starting compaction of user blocks" tenant=user-2`, `level=info component=compactor tenant=user-2 msg="start sync of metas"`, `level=info component=compactor tenant=user-2 msg="start of GC"`, - `level=debug component=compactor tenant=user-2 msg="grouper found a compactable blocks group" groupKey=0@17241709254077376921-merge--1574776800000-1574784000000 job="stage: merge, range start: 1574776800000, range end: 1574784000000, shard: , blocks: 01DTW0ZCPDDNV4BV83Q2SV4QAZ (min time: 2019-11-26 14:00:00 +0000 UTC, max time: 2019-11-26 16:00:00 +0000 UTC),01FSV54G6QFQH1G9QE93G3B9TB (min time: 2019-11-26 14:00:00 +0000 UTC, max time: 2019-11-26 16:00:00 +0000 UTC)"`, + `level=debug component=compactor tenant=user-2 msg="grouper found a compactable blocks group" groupKey=0@17241709254077376921-merge--1574776800000-1574784000000 job="stage: merge, range start: 1574776800000, range end: 1574784000000, shard: , blocks: 01DTW0ZCPDDNV4BV83Q2SV4QAZ (min time: 2019-11-26T14:00:00Z, max time: 2019-11-26T16:00:00Z),01FSV54G6QFQH1G9QE93G3B9TB (min time: 2019-11-26T14:00:00Z, max time: 2019-11-26T16:00:00Z)"`, `level=info component=compactor tenant=user-2 msg="start of compactions"`, `level=info component=compactor tenant=user-2 groupKey=0@17241709254077376921-merge--1574776800000-1574784000000 msg="compaction job succeeded"`, `level=info component=compactor tenant=user-2 msg="compaction iterations done"`, @@ -1287,8 +1287,8 @@ func TestMultitenantCompactor_ShouldSkipCompactionForJobsNoMoreOwnedAfterPlannin `level=info component=compactor tenant=user-1 msg="start sync of metas"`, `level=info component=compactor tenant=user-1 msg="start of GC"`, `level=info component=compactor tenant=user-1 msg="start of compactions"`, - `level=debug component=compactor tenant=user-1 msg="grouper found a compactable blocks group" groupKey=0@17241709254077376921-split-4_of_4-1574776800000-1574784000000 job="stage: split, range start: 1574776800000, range end: 1574784000000, shard: 4_of_4, blocks: 01DTVP434PA9VFXSW2JK000001 (min time: 2019-11-26 14:00:00 +0000 UTC, max time: 2019-11-26 16:00:00 +0000 UTC)"`, - `level=debug component=compactor tenant=user-1 msg="grouper found a compactable blocks group" groupKey=0@17241709254077376921-split-1_of_4-1574863200000-1574870400000 job="stage: split, range start: 1574863200000, range end: 1574870400000, shard: 1_of_4, blocks: 01DTVP434PA9VFXSW2JK000002 (min time: 2019-11-27 14:00:00 +0000 UTC, max time: 2019-11-27 16:00:00 +0000 UTC)"`, + `level=debug component=compactor tenant=user-1 msg="grouper found a compactable blocks group" groupKey=0@17241709254077376921-split-4_of_4-1574776800000-1574784000000 job="stage: split, range start: 1574776800000, range end: 1574784000000, shard: 4_of_4, blocks: 01DTVP434PA9VFXSW2JK000001 (min time: 2019-11-26T14:00:00Z, max time: 2019-11-26T16:00:00Z)"`, + `level=debug component=compactor tenant=user-1 msg="grouper found a compactable blocks group" groupKey=0@17241709254077376921-split-1_of_4-1574863200000-1574870400000 job="stage: split, range start: 1574863200000, range end: 1574870400000, shard: 1_of_4, blocks: 01DTVP434PA9VFXSW2JK000002 (min time: 2019-11-27T14:00:00Z, max time: 2019-11-27T16:00:00Z)"`, // The ownership check is failing because, to keep this test simple, we've just switched // the instance state to LEAVING and there are no other instances in the ring. `level=info component=compactor tenant=user-1 groupKey=0@17241709254077376921-split-4_of_4-1574776800000-1574784000000 msg="compaction job succeeded"`, diff --git a/pkg/compactor/split_merge_job.go b/pkg/compactor/split_merge_job.go index c282baa2e0..c756f4b319 100644 --- a/pkg/compactor/split_merge_job.go +++ b/pkg/compactor/split_merge_job.go @@ -8,6 +8,7 @@ package compactor import ( "fmt" "strings" + "time" "github.com/prometheus/prometheus/model/labels" "golang.org/x/exp/slices" @@ -84,7 +85,7 @@ func (j *job) String() string { for _, block := range j.blocks { minT := block.MinTime.Time().UTC() maxT := block.MaxTime.Time().UTC() - blocks = append(blocks, fmt.Sprintf("%s (min time: %s, max time: %s)", block.ULID.String(), minT.String(), maxT.String())) + blocks = append(blocks, fmt.Sprintf("%s (min time: %s, max time: %s)", block.ULID.String(), minT.Format(time.RFC3339), maxT.Format(time.RFC3339))) } // Keep the output stable for tests. From 2db23f02407efc35bdde2b424eb81bd0526f4efd Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Tue, 3 Oct 2023 20:06:10 +0200 Subject: [PATCH 52/74] Fixes time format issue --- pkg/compactor/compactor_test.go | 2 +- pkg/compactor/split_merge_job.go | 2 +- pkg/phlaredb/block/metadata.go | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/pkg/compactor/compactor_test.go b/pkg/compactor/compactor_test.go index 4fe68b8468..ac5c2ccbb8 100644 --- a/pkg/compactor/compactor_test.go +++ b/pkg/compactor/compactor_test.go @@ -1391,7 +1391,7 @@ func TestMultitenantCompactor_ShouldSkipCompactionForJobsWithFirstLevelCompactio // Ensure the skipped compaction job is the expected one. assert.Contains(t, strings.Split(strings.TrimSpace(logs.String()), "\n"), - fmt.Sprintf(`level=info component=compactor tenant=user-2 msg="skipping compaction job because blocks in this job were uploaded too recently (within wait period)" groupKey=0@17241709254077376921-merge--0-7200000 waitPeriodNotElapsedFor="%s (min time: 1970-01-01T01:00:00+01:00, max time: 1970-01-01T03:00:00+01:00)"`, user2Meta2.String())) + fmt.Sprintf(`level=info component=compactor tenant=user-2 msg="skipping compaction job because blocks in this job were uploaded too recently (within wait period)" groupKey=0@17241709254077376921-merge--0-7200000 waitPeriodNotElapsedFor="%s (min time: 1970-01-01T00:00:00Z, max time: 1970-01-01T02:00:00Z)"`, user2Meta2.String())) assert.NoError(t, prom_testutil.GatherAndCompare(registry, strings.NewReader(` # TYPE pyroscope_compactor_runs_started_total counter diff --git a/pkg/compactor/split_merge_job.go b/pkg/compactor/split_merge_job.go index c756f4b319..ca98402c43 100644 --- a/pkg/compactor/split_merge_job.go +++ b/pkg/compactor/split_merge_job.go @@ -85,7 +85,7 @@ func (j *job) String() string { for _, block := range j.blocks { minT := block.MinTime.Time().UTC() maxT := block.MaxTime.Time().UTC() - blocks = append(blocks, fmt.Sprintf("%s (min time: %s, max time: %s)", block.ULID.String(), minT.Format(time.RFC3339), maxT.Format(time.RFC3339))) + blocks = append(blocks, fmt.Sprintf("%s (min time: %s, max time: %s)", block.ULID.String(), minT.Format(time.RFC3339Nano), maxT.Format(time.RFC3339Nano))) } // Keep the output stable for tests. diff --git a/pkg/phlaredb/block/metadata.go b/pkg/phlaredb/block/metadata.go index a282491135..d3dea51217 100644 --- a/pkg/phlaredb/block/metadata.go +++ b/pkg/phlaredb/block/metadata.go @@ -151,8 +151,8 @@ func (m *Meta) String() string { return fmt.Sprintf( "%s (min time: %s, max time: %s)", m.ULID, - m.MinTime.Time().Format(time.RFC3339Nano), - m.MaxTime.Time().Format(time.RFC3339Nano), + m.MinTime.Time().UTC().Format(time.RFC3339Nano), + m.MaxTime.Time().UTC().Format(time.RFC3339Nano), ) } From 1518b3f4796be1cdee57790f57564f83203d15e4 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Wed, 4 Oct 2023 08:47:33 +0200 Subject: [PATCH 53/74] Changes the default compactor dir --- cmd/pyroscope/help-all.txt.tmpl | 2 +- cmd/pyroscope/help.txt.tmpl | 2 +- pkg/compactor/compactor.go | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/cmd/pyroscope/help-all.txt.tmpl b/cmd/pyroscope/help-all.txt.tmpl index 03a240f1ea..ae0b52c903 100644 --- a/cmd/pyroscope/help-all.txt.tmpl +++ b/cmd/pyroscope/help-all.txt.tmpl @@ -36,7 +36,7 @@ Usage of ./pyroscope: -compactor.compactor-tenant-shard-size int Max number of compactors that can compact blocks for single tenant. 0 to disable the limit and use all compactors. -compactor.data-dir string - Directory to temporarily store blocks during compaction. This directory is not required to be persisted between restarts. (default "./data-compactor/") + Directory to temporarily store blocks during compaction. This directory is not required to be persisted between restarts. (default "/data") -compactor.deletion-delay duration Time before a block marked for deletion is deleted from bucket. If not 0, blocks will be marked for deletion and compactor component will permanently delete blocks marked for deletion from the bucket. If 0, blocks will be deleted straight away. Note that deleting blocks immediately can cause query failures. (default 12h0m0s) -compactor.disabled-tenants comma-separated-list-of-strings diff --git a/cmd/pyroscope/help.txt.tmpl b/cmd/pyroscope/help.txt.tmpl index 4b7c0edcf7..360c6a83cf 100644 --- a/cmd/pyroscope/help.txt.tmpl +++ b/cmd/pyroscope/help.txt.tmpl @@ -10,7 +10,7 @@ Usage of ./pyroscope: -compactor.compactor-tenant-shard-size int Max number of compactors that can compact blocks for single tenant. 0 to disable the limit and use all compactors. -compactor.data-dir string - Directory to temporarily store blocks during compaction. This directory is not required to be persisted between restarts. (default "./data-compactor/") + Directory to temporarily store blocks during compaction. This directory is not required to be persisted between restarts. (default "/data") -compactor.first-level-compaction-wait-period duration How long the compactor waits before compacting first-level blocks that are uploaded by the ingesters. This configuration option allows for the reduction of cases where the compactor begins to compact blocks before all ingesters have uploaded their blocks to the storage. (default 25m0s) -compactor.partial-block-deletion-delay duration diff --git a/pkg/compactor/compactor.go b/pkg/compactor/compactor.go index 92370d70cd..7f11984f5e 100644 --- a/pkg/compactor/compactor.go +++ b/pkg/compactor/compactor.go @@ -125,7 +125,7 @@ func (cfg *Config) RegisterFlags(f *flag.FlagSet, logger log.Logger) { f.Var(&cfg.BlockRanges, "compactor.block-ranges", "List of compaction time ranges.") f.IntVar(&cfg.BlockSyncConcurrency, "compactor.block-sync-concurrency", 8, "Number of Go routines to use when downloading blocks for compaction and uploading resulting blocks.") f.IntVar(&cfg.MetaSyncConcurrency, "compactor.meta-sync-concurrency", 20, "Number of Go routines to use when syncing block meta files from the long term storage.") - f.StringVar(&cfg.DataDir, "compactor.data-dir", "./data-compactor/", "Directory to temporarily store blocks during compaction. This directory is not required to be persisted between restarts.") + f.StringVar(&cfg.DataDir, "compactor.data-dir", "/data", "Directory to temporarily store blocks during compaction. This directory is not required to be persisted between restarts.") f.DurationVar(&cfg.CompactionInterval, "compactor.compaction-interval", time.Hour, "The frequency at which the compaction runs") f.DurationVar(&cfg.MaxCompactionTime, "compactor.max-compaction-time", time.Hour, "Max time for starting compactions for a single tenant. After this time no new compactions for the tenant are started before next compaction cycle. This can help in multi-tenant environments to avoid single tenant using all compaction time, but also in single-tenant environments to force new discovery of blocks more often. 0 = disabled.") f.IntVar(&cfg.CompactionRetries, "compactor.compaction-retries", 3, "How many times to retry a failed compaction within a single compaction run.") From c7b5a3632e0185c1fd877fce383b32170d17ed86 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Thu, 5 Oct 2023 07:55:42 +0200 Subject: [PATCH 54/74] Fixes panic while rewriting symbols. --- pkg/phlaredb/symdb/rewriter.go | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/pkg/phlaredb/symdb/rewriter.go b/pkg/phlaredb/symdb/rewriter.go index 8d34b52f36..98f64bdf2f 100644 --- a/pkg/phlaredb/symdb/rewriter.go +++ b/pkg/phlaredb/symdb/rewriter.go @@ -305,6 +305,11 @@ func (t *lookupTable[T]) reset() { // for future resolve, and returned values is the marked // index to unresolved. func (t *lookupTable[T]) tryLookup(x uint32) uint32 { + // todo(ctovena): this is a hack to make sure we don't have any out of bounds errors + // see https://github.com/grafana/pyroscope/issues/2488 + if x >= uint32(len(t.resolved)) { + t.grow(int(x + 1)) + } if v := t.resolved[x]; v != 0 { if v&marker > 0 { return v // Already marked for resolve. From 26e546a45e9a0dac242eff30362a1fcd45025611 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Thu, 5 Oct 2023 11:25:47 +0200 Subject: [PATCH 55/74] Fixes mark global path and add 8h compaction range --- cmd/pyroscope/help-all.txt.tmpl | 2 +- pkg/compactor/blocks_cleaner.go | 4 +- pkg/compactor/blocks_cleaner_test.go | 84 ++--- pkg/compactor/bucket_compactor.go | 7 + pkg/compactor/bucket_compactor_e2e_test.go | 19 +- pkg/compactor/bucket_compactor_test.go | 2 +- pkg/compactor/compactor.go | 4 +- pkg/compactor/compactor_test.go | 296 +++++++++--------- pkg/compactor/split_merge_compactor_test.go | 4 +- pkg/ingester/instance.go | 2 +- pkg/objstore/user_bucket_client.go | 8 +- .../global_markers_bucket_client_test.go | 2 +- pkg/phlaredb/block/testutil/mock_block.go | 8 +- pkg/phlaredb/bucket/tenant_deletion_mark.go | 6 +- .../bucket/tenant_deletion_mark_test.go | 6 +- pkg/phlaredb/bucketindex/loader_test.go | 6 +- pkg/phlaredb/bucketindex/storage.go | 6 +- pkg/phlaredb/bucketindex/storage_test.go | 2 +- pkg/phlaredb/bucketindex/updater.go | 2 +- pkg/phlaredb/bucketindex/updater_test.go | 10 +- pkg/storegateway/bucket.go | 2 +- .../bucket_index_metadata_fetcher_test.go | 4 +- 22 files changed, 247 insertions(+), 239 deletions(-) diff --git a/cmd/pyroscope/help-all.txt.tmpl b/cmd/pyroscope/help-all.txt.tmpl index ae0b52c903..d2723004fe 100644 --- a/cmd/pyroscope/help-all.txt.tmpl +++ b/cmd/pyroscope/help-all.txt.tmpl @@ -16,7 +16,7 @@ Usage of ./pyroscope: -blocks-storage.bucket-store.tenant-sync-concurrency int Maximum number of concurrent tenants synching blocks. (default 10) -compactor.block-ranges value - List of compaction time ranges. (default 4h0m0s) + List of compaction time ranges. (default 4h0m0s,8h0m0s) -compactor.block-sync-concurrency int Number of Go routines to use when downloading blocks for compaction and uploading resulting blocks. (default 8) -compactor.blocks-retention-period duration diff --git a/pkg/compactor/blocks_cleaner.go b/pkg/compactor/blocks_cleaner.go index bd70c4e049..97e8e59479 100644 --- a/pkg/compactor/blocks_cleaner.go +++ b/pkg/compactor/blocks_cleaner.go @@ -274,7 +274,7 @@ func (c *BlocksCleaner) deleteRemainingData(ctx context.Context, userBucket objs // deleteUserMarkedForDeletion removes blocks and remaining data for tenant marked for deletion. func (c *BlocksCleaner) deleteUserMarkedForDeletion(ctx context.Context, userID string, userLogger log.Logger) error { - userBucket := objstore.NewUserBucketClient(userID, c.bucketClient, c.cfgProvider) + userBucket := objstore.NewTenantBucketClient(userID, c.bucketClient, c.cfgProvider) level.Info(userLogger).Log("msg", "deleting blocks for tenant marked for deletion") @@ -367,7 +367,7 @@ func (c *BlocksCleaner) deleteUserMarkedForDeletion(ctx context.Context, userID } func (c *BlocksCleaner) cleanUser(ctx context.Context, userID string, userLogger log.Logger) (returnErr error) { - userBucket := objstore.NewUserBucketClient(userID, c.bucketClient, c.cfgProvider) + userBucket := objstore.NewTenantBucketClient(userID, c.bucketClient, c.cfgProvider) startTime := time.Now() level.Info(userLogger).Log("msg", "started blocks cleanup and maintenance") diff --git a/pkg/compactor/blocks_cleaner_test.go b/pkg/compactor/blocks_cleaner_test.go index aaa78761c4..bc06a54538 100644 --- a/pkg/compactor/blocks_cleaner_test.go +++ b/pkg/compactor/blocks_cleaner_test.go @@ -78,12 +78,12 @@ func testBlocksCleanerWithOptions(t *testing.T, options testBlocksCleanerOptions block6 := createDBBlock(t, bucketClient, "user-1", 40, 50, 2, nil) block7 := createDBBlock(t, bucketClient, "user-2", 10, 20, 2, nil) block8 := createDBBlock(t, bucketClient, "user-2", 40, 50, 2, nil) - createDeletionMark(t, bucketClient, "user-1", block2, now.Add(-deletionDelay).Add(time.Hour)) // Block hasn't reached the deletion threshold yet. - createDeletionMark(t, bucketClient, "user-1", block3, now.Add(-deletionDelay).Add(-time.Hour)) // Block reached the deletion threshold. - createDeletionMark(t, bucketClient, "user-1", block4, now.Add(-deletionDelay).Add(time.Hour)) // Partial block hasn't reached the deletion threshold yet. - createDeletionMark(t, bucketClient, "user-1", block5, now.Add(-deletionDelay).Add(-time.Hour)) // Partial block reached the deletion threshold. - require.NoError(t, bucketClient.Delete(ctx, path.Join("user-1", block6.String(), block.MetaFilename))) // Partial block without deletion mark. - createDeletionMark(t, bucketClient, "user-2", block7, now.Add(-deletionDelay).Add(-time.Hour)) // Block reached the deletion threshold. + createDeletionMark(t, bucketClient, "user-1", block2, now.Add(-deletionDelay).Add(time.Hour)) // Block hasn't reached the deletion threshold yet. + createDeletionMark(t, bucketClient, "user-1", block3, now.Add(-deletionDelay).Add(-time.Hour)) // Block reached the deletion threshold. + createDeletionMark(t, bucketClient, "user-1", block4, now.Add(-deletionDelay).Add(time.Hour)) // Partial block hasn't reached the deletion threshold yet. + createDeletionMark(t, bucketClient, "user-1", block5, now.Add(-deletionDelay).Add(-time.Hour)) // Partial block reached the deletion threshold. + require.NoError(t, bucketClient.Delete(ctx, path.Join("user-1", "phlaredb", block6.String(), block.MetaFilename))) // Partial block without deletion mark. + createDeletionMark(t, bucketClient, "user-2", block7, now.Add(-deletionDelay).Add(-time.Hour)) // Block reached the deletion threshold. // Blocks for user-3, marked for deletion. require.NoError(t, bucket.WriteTenantDeletionMark(context.Background(), bucketClient, "user-3", nil, bucket.NewTenantDeletionMark(time.Now()))) @@ -117,30 +117,30 @@ func testBlocksCleanerWithOptions(t *testing.T, options testBlocksCleanerOptions }{ // Check the storage to ensure only the block which has reached the deletion threshold // has been effectively deleted. - {path: path.Join("user-1", block1.String(), block.MetaFilename), expectedExists: true}, - {path: path.Join("user-1", block3.String(), block.MetaFilename), expectedExists: false}, - {path: path.Join("user-2", block7.String(), block.MetaFilename), expectedExists: false}, - {path: path.Join("user-2", block8.String(), block.MetaFilename), expectedExists: true}, + {path: path.Join("user-1", "phlaredb/", block1.String(), block.MetaFilename), expectedExists: true}, + {path: path.Join("user-1", "phlaredb/", block3.String(), block.MetaFilename), expectedExists: false}, + {path: path.Join("user-2", "phlaredb/", block7.String(), block.MetaFilename), expectedExists: false}, + {path: path.Join("user-2", "phlaredb/", block8.String(), block.MetaFilename), expectedExists: true}, // Should not delete a block with deletion mark who hasn't reached the deletion threshold yet. - {path: path.Join("user-1", block2.String(), block.MetaFilename), expectedExists: true}, - {path: path.Join("user-1", block.DeletionMarkFilepath(block2)), expectedExists: true}, + {path: path.Join("user-1", "phlaredb/", block2.String(), block.MetaFilename), expectedExists: true}, + {path: path.Join("user-1", "phlaredb/", block.DeletionMarkFilepath(block2)), expectedExists: true}, // Should delete a partial block with deletion mark who hasn't reached the deletion threshold yet. - {path: path.Join("user-1", block4.String(), block.DeletionMarkFilename), expectedExists: false}, - {path: path.Join("user-1", block.DeletionMarkFilepath(block4)), expectedExists: false}, + {path: path.Join("user-1", "phlaredb/", block4.String(), block.DeletionMarkFilename), expectedExists: false}, + {path: path.Join("user-1", "phlaredb/", block.DeletionMarkFilepath(block4)), expectedExists: false}, // Should delete a partial block with deletion mark who has reached the deletion threshold. - {path: path.Join("user-1", block5.String(), block.DeletionMarkFilename), expectedExists: false}, - {path: path.Join("user-1", block.DeletionMarkFilepath(block5)), expectedExists: false}, + {path: path.Join("user-1", "phlaredb/", block5.String(), block.DeletionMarkFilename), expectedExists: false}, + {path: path.Join("user-1", "phlaredb/", block.DeletionMarkFilepath(block5)), expectedExists: false}, // Should not delete a partial block without deletion mark. - {path: path.Join("user-1", block6.String(), block.IndexFilename), expectedExists: true}, + {path: path.Join("user-1", "phlaredb/", block6.String(), block.IndexFilename), expectedExists: true}, // Should completely delete blocks for user-3, marked for deletion - {path: path.Join("user-3", block9.String(), block.MetaFilename), expectedExists: false}, - {path: path.Join("user-3", block9.String(), block.IndexFilename), expectedExists: false}, - {path: path.Join("user-3", block10.String(), block.MetaFilename), expectedExists: false}, - {path: path.Join("user-3", block10.String(), block.IndexFilename), expectedExists: false}, + {path: path.Join("user-3", "phlaredb/", block9.String(), block.MetaFilename), expectedExists: false}, + {path: path.Join("user-3", "phlaredb/", block9.String(), block.IndexFilename), expectedExists: false}, + {path: path.Join("user-3", "phlaredb/", block10.String(), block.MetaFilename), expectedExists: false}, + {path: path.Join("user-3", "phlaredb/", block10.String(), block.IndexFilename), expectedExists: false}, // Tenant deletion mark is not removed. - {path: path.Join("user-3", bucket.TenantDeletionMarkPath), expectedExists: true}, + {path: path.Join("user-3", "phlaredb/", bucket.TenantDeletionMarkPath), expectedExists: true}, // User-4 is removed fully. - {path: path.Join("user-4", bucket.TenantDeletionMarkPath), expectedExists: options.user4FilesExist}, + {path: path.Join("user-4", "phlaredb/", bucket.TenantDeletionMarkPath), expectedExists: options.user4FilesExist}, } { exists, err := bucketClient.Exists(ctx, tc.path) require.NoError(t, err) @@ -227,7 +227,7 @@ func TestBlocksCleaner_ShouldContinueOnBlockDeletionFailure(t *testing.T) { // To emulate a failure deleting a block, we wrap the bucket client in a mocked one. bucketClient = &mockBucketFailure{ Bucket: bucketClient, - DeleteFailures: []string{path.Join(userID, block3.String(), block.MetaFilename)}, + DeleteFailures: []string{path.Join(userID, "phlaredb/", block3.String(), block.MetaFilename)}, } cfg := BlocksCleanerConfig{ @@ -248,10 +248,10 @@ func TestBlocksCleaner_ShouldContinueOnBlockDeletionFailure(t *testing.T) { path string expectedExists bool }{ - {path: path.Join(userID, block1.String(), block.MetaFilename), expectedExists: true}, - {path: path.Join(userID, block2.String(), block.MetaFilename), expectedExists: false}, - {path: path.Join(userID, block3.String(), block.MetaFilename), expectedExists: true}, - {path: path.Join(userID, block4.String(), block.MetaFilename), expectedExists: false}, + {path: path.Join(userID, "phlaredb/", block1.String(), block.MetaFilename), expectedExists: true}, + {path: path.Join(userID, "phlaredb/", block2.String(), block.MetaFilename), expectedExists: false}, + {path: path.Join(userID, "phlaredb/", block3.String(), block.MetaFilename), expectedExists: true}, + {path: path.Join(userID, "phlaredb/", block4.String(), block.MetaFilename), expectedExists: false}, } { exists, err := bucketClient.Exists(ctx, tc.path) require.NoError(t, err) @@ -288,7 +288,7 @@ func TestBlocksCleaner_ShouldRebuildBucketIndexOnCorruptedOne(t *testing.T) { createDeletionMark(t, bucketClient, userID, block3, now.Add(-deletionDelay).Add(time.Hour)) // Write a corrupted bucket index. - require.NoError(t, bucketClient.Upload(ctx, path.Join(userID, bucketindex.IndexCompressedFilename), strings.NewReader("invalid!}"))) + require.NoError(t, bucketClient.Upload(ctx, path.Join(userID, "phlaredb/", bucketindex.IndexCompressedFilename), strings.NewReader("invalid!}"))) cfg := BlocksCleanerConfig{ DeletionDelay: deletionDelay, @@ -308,9 +308,9 @@ func TestBlocksCleaner_ShouldRebuildBucketIndexOnCorruptedOne(t *testing.T) { path string expectedExists bool }{ - {path: path.Join(userID, block1.String(), block.MetaFilename), expectedExists: true}, - {path: path.Join(userID, block2.String(), block.MetaFilename), expectedExists: false}, - {path: path.Join(userID, block3.String(), block.MetaFilename), expectedExists: true}, + {path: path.Join(userID, "phlaredb/", block1.String(), block.MetaFilename), expectedExists: true}, + {path: path.Join(userID, "phlaredb/", block2.String(), block.MetaFilename), expectedExists: false}, + {path: path.Join(userID, "phlaredb/", block3.String(), block.MetaFilename), expectedExists: true}, } { exists, err := bucketClient.Exists(ctx, tc.path) require.NoError(t, err) @@ -536,7 +536,7 @@ func TestBlocksCleaner_ShouldRemoveBlocksOutsideRetentionPeriod(t *testing.T) { cleaner := NewBlocksCleaner(cfg, bucketClient, bucket.AllTenants, cfgProvider, logger, reg) assertBlockExists := func(user string, blockID ulid.ULID, expectExists bool) { - exists, err := bucketClient.Exists(ctx, path.Join(user, blockID.String(), block.MetaFilename)) + exists, err := bucketClient.Exists(ctx, path.Join(user, "phlaredb/", blockID.String(), block.MetaFilename)) require.NoError(t, err) assert.Equal(t, expectExists, exists) } @@ -685,11 +685,11 @@ func TestBlocksCleaner_ShouldRemoveBlocksOutsideRetentionPeriod(t *testing.T) { } func checkBlock(t *testing.T, user string, bucketClient objstore.Bucket, blockID ulid.ULID, metaJSONExists bool, markedForDeletion bool) { - exists, err := bucketClient.Exists(context.Background(), path.Join(user, blockID.String(), block.MetaFilename)) + exists, err := bucketClient.Exists(context.Background(), path.Join(user, "phlaredb/", blockID.String(), block.MetaFilename)) require.NoError(t, err) require.Equal(t, metaJSONExists, exists) - exists, err = bucketClient.Exists(context.Background(), path.Join(user, blockID.String(), block.DeletionMarkFilename)) + exists, err = bucketClient.Exists(context.Background(), path.Join(user, "phlaredb/", blockID.String(), block.DeletionMarkFilename)) require.NoError(t, err) require.Equal(t, markedForDeletion, exists) } @@ -716,7 +716,7 @@ func TestBlocksCleaner_ShouldCleanUpFilesWhenNoMoreBlocksRemain(t *testing.T) { // Create a deletion mark within the deletionDelay period that won't correspond to any block randomULID := ulid.MustNew(ulid.Now(), rand.Reader) createDeletionMark(t, bucketClient, userID, randomULID, now.Add(-deletionDelay).Add(time.Hour)) - blockDeletionMarkFile := path.Join(userID, block.DeletionMarkFilepath(randomULID)) + blockDeletionMarkFile := path.Join(userID, "phlaredb/", block.DeletionMarkFilepath(randomULID)) exists, err := bucketClient.Exists(ctx, blockDeletionMarkFile) require.NoError(t, err) assert.True(t, exists) @@ -771,7 +771,7 @@ func TestBlocksCleaner_ShouldRemovePartialBlocksOutsideDelayPeriod(t *testing.T) cleaner := NewBlocksCleaner(cfg, bucketClient, bucket.AllTenants, cfgProvider, logger, reg) makeBlockPartial := func(user string, blockID ulid.ULID) { - err := bucketClient.Delete(ctx, path.Join(user, blockID.String(), block.MetaFilename)) + err := bucketClient.Delete(ctx, path.Join(user, "phlaredb/", blockID.String(), block.MetaFilename)) require.NoError(t, err) } @@ -841,12 +841,12 @@ func TestBlocksCleaner_ShouldNotRemovePartialBlocksInsideDelayPeriod(t *testing. cleaner := NewBlocksCleaner(cfg, bucketClient, bucket.AllTenants, cfgProvider, logger, reg) makeBlockPartial := func(user string, blockID ulid.ULID) { - err := bucketClient.Delete(ctx, path.Join(user, blockID.String(), block.MetaFilename)) + err := bucketClient.Delete(ctx, path.Join(user, "phlaredb/", blockID.String(), block.MetaFilename)) require.NoError(t, err) } corruptMeta := func(user string, blockID ulid.ULID) { - err := bucketClient.Upload(ctx, path.Join(user, blockID.String(), block.MetaFilename), strings.NewReader("corrupted file contents")) + err := bucketClient.Upload(ctx, path.Join(user, "phlaredb/", blockID.String(), block.MetaFilename), strings.NewReader("corrupted file contents")) require.NoError(t, err) } @@ -911,7 +911,7 @@ func TestBlocksCleaner_ShouldNotRemovePartialBlocksIfConfiguredDelayIsInvalid(t // Create a partial block. block1 := createDBBlock(t, bucketClient, "user-1", ts(-10), ts(-8), 2, nil) - err := bucketClient.Delete(ctx, path.Join("user-1", block1.String(), block.MetaFilename)) + err := bucketClient.Delete(ctx, path.Join("user-1", "phlaredb/", block1.String(), block.MetaFilename)) require.NoError(t, err) cfg := BlocksCleanerConfig{ @@ -963,14 +963,14 @@ func TestStalePartialBlockLastModifiedTime(t *testing.T) { objectTime := time.Now().Add(-1 * time.Hour).Truncate(time.Second) // ignore milliseconds, as not all filesystems store them. blockID := createDBBlock(t, b, tenantId, objectTime.UnixMilli(), time.Now().UnixMilli(), 2, nil) - err := filepath.Walk(filepath.Join(dir, tenantId, blockID.String()), func(path string, info os.FileInfo, err error) error { + err := filepath.Walk(filepath.Join(dir, tenantId, "phlaredb/", blockID.String()), func(path string, info os.FileInfo, err error) error { require.NoError(t, err) require.NoError(t, os.Chtimes(path, objectTime, objectTime)) return nil }) require.NoError(t, err) - userBucket := objstore.NewUserBucketClient(tenantId, b, nil) + userBucket := objstore.NewTenantBucketClient(tenantId, b, nil) emptyBlockID := ulid.ULID{} require.NotEqual(t, blockID, emptyBlockID) diff --git a/pkg/compactor/bucket_compactor.go b/pkg/compactor/bucket_compactor.go index c33bac107b..9f03641998 100644 --- a/pkg/compactor/bucket_compactor.go +++ b/pkg/compactor/bucket_compactor.go @@ -11,6 +11,7 @@ import ( "os" "path" "path/filepath" + "strings" "sync" "time" @@ -230,6 +231,12 @@ type BlockCompactor struct { } func (c *BlockCompactor) CompactWithSplitting(ctx context.Context, dest string, dirs []string, shardCount uint64) ([]ulid.ULID, error) { + defer func() { + if err := recover(); err != nil { + level.Error(c.logger).Log("msg", "panic during compaction", "err", err, "dirs", strings.Join(dirs, ",")) + panic(err) + } + }() localBucket, err := client.NewBucket(ctx, client.Config{ StorageBackendConfig: client.StorageBackendConfig{ Backend: client.Filesystem, diff --git a/pkg/compactor/bucket_compactor_e2e_test.go b/pkg/compactor/bucket_compactor_e2e_test.go index 8e70563f33..683a87da54 100644 --- a/pkg/compactor/bucket_compactor_e2e_test.go +++ b/pkg/compactor/bucket_compactor_e2e_test.go @@ -167,8 +167,9 @@ func TestSyncer_GarbageCollect_e2e(t *testing.T) { func TestGroupCompactE2E(t *testing.T) { foreachStore(t, func(t *testing.T, bkt phlareobj.Bucket) { + userbkt := phlareobj.NewTenantBucketClient("user-1", bkt, nil).(phlareobj.Bucket) // Use bucket with global markers to make sure that our custom filters work correctly. - bkt = block.BucketWithGlobalMarkers(bkt) + userbkt = block.BucketWithGlobalMarkers(userbkt) ctx, cancel := context.WithTimeout(context.Background(), 120*time.Second) defer cancel() @@ -212,15 +213,15 @@ func TestGroupCompactE2E(t *testing.T) { logger := log.NewLogfmtLogger(os.Stderr) duplicateBlocksFilter := NewShardAwareDeduplicateFilter() - noCompactMarkerFilter := NewNoCompactionMarkFilter(bkt, true) - metaFetcher, err := block.NewMetaFetcher(nil, 32, bkt, "", nil, []block.MetadataFilter{ + noCompactMarkerFilter := NewNoCompactionMarkFilter(userbkt, true) + metaFetcher, err := block.NewMetaFetcher(nil, 32, userbkt, "", nil, []block.MetadataFilter{ duplicateBlocksFilter, noCompactMarkerFilter, }) require.NoError(t, err) blocksMarkedForDeletion := promauto.With(nil).NewCounter(prometheus.CounterOpts{}) - sy, err := NewMetaSyncer(nil, nil, bkt, metaFetcher, duplicateBlocksFilter, blocksMarkedForDeletion) + sy, err := NewMetaSyncer(nil, nil, userbkt, metaFetcher, duplicateBlocksFilter, blocksMarkedForDeletion) require.NoError(t, err) planner := NewSplitAndMergePlanner([]int64{1000, 3000}) @@ -229,7 +230,7 @@ func TestGroupCompactE2E(t *testing.T) { bComp, err := NewBucketCompactor(logger, sy, grouper, planner, &BlockCompactor{ blockOpenConcurrency: 100, logger: logger, - }, dir, bkt, 2, ownAllJobs, sortJobsByNewestBlocksFirst, 0, 4, metrics) + }, dir, userbkt, 2, ownAllJobs, sortJobsByNewestBlocksFirst, 0, 4, metrics) require.NoError(t, err) // Compaction on empty should not fail. @@ -245,11 +246,11 @@ func TestGroupCompactE2E(t *testing.T) { _, err = os.Stat(dir) assert.True(t, os.IsNotExist(err), "dir %s should be remove after compaction.", dir) - m1 := createDBBlock(t, bkt, "", 500, 1000, 10, nil) - m2 := createDBBlock(t, bkt, "", 500, 1000, 10, nil) + m1 := createDBBlock(t, bkt, "user-1", 500, 1000, 10, nil) + m2 := createDBBlock(t, bkt, "user-1", 500, 1000, 10, nil) - m3 := createDBBlock(t, bkt, "", 1001, 2000, 10, nil) - m4 := createDBBlock(t, bkt, "", 1001, 3000, 10, nil) + m3 := createDBBlock(t, bkt, "user-1", 1001, 2000, 10, nil) + m4 := createDBBlock(t, bkt, "user-1", 1001, 3000, 10, nil) require.NoError(t, bComp.Compact(ctx, 0), 0) assert.Equal(t, 6.0, promtest.ToFloat64(sy.metrics.blocksMarkedForDeletion)) diff --git a/pkg/compactor/bucket_compactor_test.go b/pkg/compactor/bucket_compactor_test.go index 4943109b5f..4468925db9 100644 --- a/pkg/compactor/bucket_compactor_test.go +++ b/pkg/compactor/bucket_compactor_test.go @@ -317,7 +317,7 @@ func TestCompactedBlocksTimeRangeVerification(t *testing.T) { compactedBlock1 := createDBBlock(t, bucketClient, "foo", testData.compactedBlockMinTime, testData.compactedBlockMinTime+500, 10, nil) compactedBlock2 := createDBBlock(t, bucketClient, "foo", testData.compactedBlockMaxTime-500, testData.compactedBlockMaxTime, 10, nil) - err := verifyCompactedBlocksTimeRanges([]ulid.ULID{compactedBlock1, compactedBlock2}, sourceMinTime, sourceMaxTime, filepath.Join(tempDir, "foo")) + err := verifyCompactedBlocksTimeRanges([]ulid.ULID{compactedBlock1, compactedBlock2}, sourceMinTime, sourceMaxTime, filepath.Join(tempDir, "foo", "phlaredb/")) if testData.shouldErr { require.ErrorContains(t, err, testData.expectedErrMsg) } else { diff --git a/pkg/compactor/compactor.go b/pkg/compactor/compactor.go index 7f11984f5e..44571cdf32 100644 --- a/pkg/compactor/compactor.go +++ b/pkg/compactor/compactor.go @@ -118,7 +118,7 @@ type Config struct { func (cfg *Config) RegisterFlags(f *flag.FlagSet, logger log.Logger) { cfg.ShardingRing.RegisterFlags(f, logger) - cfg.BlockRanges = DurationList{4 * time.Hour} + cfg.BlockRanges = DurationList{4 * time.Hour, 8 * time.Hour} cfg.retryMinBackoff = 10 * time.Second cfg.retryMaxBackoff = time.Minute @@ -668,7 +668,7 @@ func (c *MultitenantCompactor) compactUserWithRetries(ctx context.Context, userI } func (c *MultitenantCompactor) compactUser(ctx context.Context, userID string) error { - userBucket := objstore.NewUserBucketClient(userID, c.bucketClient, c.cfgProvider) + userBucket := objstore.NewTenantBucketClient(userID, c.bucketClient, c.cfgProvider) reg := prometheus.NewRegistry() defer c.syncerMetrics.gatherThanosSyncerMetrics(reg) diff --git a/pkg/compactor/compactor_test.go b/pkg/compactor/compactor_test.go index ac5c2ccbb8..33aeaab295 100644 --- a/pkg/compactor/compactor_test.go +++ b/pkg/compactor/compactor_test.go @@ -419,17 +419,17 @@ func TestMultitenantCompactor_ShouldIncrementCompactionErrorIfFailedToCompactASi userID := "test-user" bucketClient := &pyroscope_objstore.ClientMock{} bucketClient.MockIter("", []string{userID}, nil) - bucketClient.MockIter(userID+"/", []string{userID + "/01DTVP434PA9VFXSW2JKB3392D", userID + "/01DTW0ZCPDDNV4BV83Q2SV4QAZ"}, nil) - bucketClient.MockIter(userID+"/markers/", nil, nil) - bucketClient.MockExists(path.Join(userID, bucket.TenantDeletionMarkPath), false, nil) - bucketClient.MockGet(userID+"/01DTVP434PA9VFXSW2JKB3392D/meta.json", mockBlockMetaJSON("01DTVP434PA9VFXSW2JKB3392D"), nil) - bucketClient.MockGet(userID+"/01DTVP434PA9VFXSW2JKB3392D/deletion-mark.json", "", nil) - bucketClient.MockGet(userID+"/01DTVP434PA9VFXSW2JKB3392D/no-compact-mark.json", "", nil) - bucketClient.MockGet(userID+"/01DTW0ZCPDDNV4BV83Q2SV4QAZ/meta.json", mockBlockMetaJSON("01DTW0ZCPDDNV4BV83Q2SV4QAZ"), nil) - bucketClient.MockGet(userID+"/01DTW0ZCPDDNV4BV83Q2SV4QAZ/deletion-mark.json", "", nil) - bucketClient.MockGet(userID+"/01DTW0ZCPDDNV4BV83Q2SV4QAZ/no-compact-mark.json", "", nil) - bucketClient.MockGet(userID+"/bucket-index.json.gz", "", nil) - bucketClient.MockUpload(userID+"/bucket-index.json.gz", nil) + bucketClient.MockIter(userID+"/phlaredb/", []string{userID + "/phlaredb/01DTVP434PA9VFXSW2JKB3392D", userID + "/phlaredb/01DTW0ZCPDDNV4BV83Q2SV4QAZ"}, nil) + bucketClient.MockIter(userID+"/phlaredb/markers/", nil, nil) + bucketClient.MockExists(path.Join(userID, "phlaredb", bucket.TenantDeletionMarkPath), false, nil) + bucketClient.MockGet(userID+"/phlaredb/01DTVP434PA9VFXSW2JKB3392D/meta.json", mockBlockMetaJSON("01DTVP434PA9VFXSW2JKB3392D"), nil) + bucketClient.MockGet(userID+"/phlaredb/01DTVP434PA9VFXSW2JKB3392D/deletion-mark.json", "", nil) + bucketClient.MockGet(userID+"/phlaredb/01DTVP434PA9VFXSW2JKB3392D/no-compact-mark.json", "", nil) + bucketClient.MockGet(userID+"/phlaredb/01DTW0ZCPDDNV4BV83Q2SV4QAZ/meta.json", mockBlockMetaJSON("01DTW0ZCPDDNV4BV83Q2SV4QAZ"), nil) + bucketClient.MockGet(userID+"/phlaredb/01DTW0ZCPDDNV4BV83Q2SV4QAZ/deletion-mark.json", "", nil) + bucketClient.MockGet(userID+"/phlaredb/01DTW0ZCPDDNV4BV83Q2SV4QAZ/no-compact-mark.json", "", nil) + bucketClient.MockGet(userID+"/phlaredb/bucket-index.json.gz", "", nil) + bucketClient.MockUpload(userID+"/phlaredb/bucket-index.json.gz", nil) c, _, tsdbPlannerMock, _, registry := prepare(t, prepareConfig(t), bucketClient) tsdbPlannerMock.On("Plan", mock.Anything, mock.Anything).Return([]*block.Meta{}, errors.New("Failed to plan")) @@ -468,17 +468,17 @@ func TestMultitenantCompactor_ShouldIncrementCompactionShutdownIfTheContextIsCan userID := "test-user" bucketClient := &pyroscope_objstore.ClientMock{} bucketClient.MockIter("", []string{userID}, nil) - bucketClient.MockIter(userID+"/", []string{userID + "/01DTVP434PA9VFXSW2JKB3392D", userID + "/01DTW0ZCPDDNV4BV83Q2SV4QAZ"}, nil) - bucketClient.MockIter(userID+"/markers/", nil, nil) - bucketClient.MockExists(path.Join(userID, bucket.TenantDeletionMarkPath), false, nil) - bucketClient.MockGet(userID+"/01DTVP434PA9VFXSW2JKB3392D/meta.json", mockBlockMetaJSON("01DTVP434PA9VFXSW2JKB3392D"), nil) - bucketClient.MockGet(userID+"/01DTVP434PA9VFXSW2JKB3392D/deletion-mark.json", "", nil) - bucketClient.MockGet(userID+"/01DTVP434PA9VFXSW2JKB3392D/no-compact-mark.json", "", nil) - bucketClient.MockGet(userID+"/01DTW0ZCPDDNV4BV83Q2SV4QAZ/meta.json", mockBlockMetaJSON("01DTW0ZCPDDNV4BV83Q2SV4QAZ"), nil) - bucketClient.MockGet(userID+"/01DTW0ZCPDDNV4BV83Q2SV4QAZ/deletion-mark.json", "", nil) - bucketClient.MockGet(userID+"/01DTW0ZCPDDNV4BV83Q2SV4QAZ/no-compact-mark.json", "", nil) - bucketClient.MockGet(userID+"/bucket-index.json.gz", "", nil) - bucketClient.MockUpload(userID+"/bucket-index.json.gz", nil) + bucketClient.MockIter(userID+"/phlaredb/markers/", nil, nil) + bucketClient.MockIter(userID+"/phlaredb/", []string{userID + "/phlaredb/01DTVP434PA9VFXSW2JKB3392D", userID + "/phlaredb/01DTW0ZCPDDNV4BV83Q2SV4QAZ"}, nil) + bucketClient.MockExists(path.Join(userID, "phlaredb/", bucket.TenantDeletionMarkPath), false, nil) + bucketClient.MockGet(userID+"/phlaredb/01DTVP434PA9VFXSW2JKB3392D/meta.json", mockBlockMetaJSON("01DTVP434PA9VFXSW2JKB3392D"), nil) + bucketClient.MockGet(userID+"/phlaredb/01DTVP434PA9VFXSW2JKB3392D/deletion-mark.json", "", nil) + bucketClient.MockGet(userID+"/phlaredb/01DTVP434PA9VFXSW2JKB3392D/no-compact-mark.json", "", nil) + bucketClient.MockGet(userID+"/phlaredb/01DTW0ZCPDDNV4BV83Q2SV4QAZ/meta.json", mockBlockMetaJSON("01DTW0ZCPDDNV4BV83Q2SV4QAZ"), nil) + bucketClient.MockGet(userID+"/phlaredb/01DTW0ZCPDDNV4BV83Q2SV4QAZ/deletion-mark.json", "", nil) + bucketClient.MockGet(userID+"/phlaredb/01DTW0ZCPDDNV4BV83Q2SV4QAZ/no-compact-mark.json", "", nil) + bucketClient.MockGet(userID+"/phlaredb/bucket-index.json.gz", "", nil) + bucketClient.MockUpload(userID+"/phlaredb/bucket-index.json.gz", nil) c, _, tsdbPlannerMock, logs, registry := prepare(t, prepareConfig(t), bucketClient) t.Cleanup(func() { @@ -521,29 +521,29 @@ func TestMultitenantCompactor_ShouldIterateOverUsersAndRunCompaction(t *testing. // Mock the bucket to contain two users, each one with two blocks (to make sure that grouper doesn't skip them). bucketClient := &pyroscope_objstore.ClientMock{} bucketClient.MockIter("", []string{"user-1", "user-2"}, nil) - bucketClient.MockExists(path.Join("user-1", bucket.TenantDeletionMarkPath), false, nil) - bucketClient.MockExists(path.Join("user-2", bucket.TenantDeletionMarkPath), false, nil) - bucketClient.MockIter("user-1/", []string{"user-1/01DTVP434PA9VFXSW2JKB3392D", "user-1/01FS51A7GQ1RQWV35DBVYQM4KF"}, nil) - bucketClient.MockIter("user-2/", []string{"user-2/01DTW0ZCPDDNV4BV83Q2SV4QAZ", "user-2/01FRSF035J26D6CGX7STCSD1KG"}, nil) - bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", mockBlockMetaJSON("01DTVP434PA9VFXSW2JKB3392D"), nil) - bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/deletion-mark.json", "", nil) - bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/no-compact-mark.json", "", nil) - bucketClient.MockGet("user-1/01FS51A7GQ1RQWV35DBVYQM4KF/meta.json", mockBlockMetaJSON("01FS51A7GQ1RQWV35DBVYQM4KF"), nil) - bucketClient.MockGet("user-1/01FS51A7GQ1RQWV35DBVYQM4KF/deletion-mark.json", "", nil) - bucketClient.MockGet("user-1/01FS51A7GQ1RQWV35DBVYQM4KF/no-compact-mark.json", "", nil) - - bucketClient.MockGet("user-2/01DTW0ZCPDDNV4BV83Q2SV4QAZ/meta.json", mockBlockMetaJSON("01DTW0ZCPDDNV4BV83Q2SV4QAZ"), nil) - bucketClient.MockGet("user-2/01DTW0ZCPDDNV4BV83Q2SV4QAZ/deletion-mark.json", "", nil) - bucketClient.MockGet("user-2/01DTW0ZCPDDNV4BV83Q2SV4QAZ/no-compact-mark.json", "", nil) - bucketClient.MockGet("user-2/01FRSF035J26D6CGX7STCSD1KG/meta.json", mockBlockMetaJSON("01FRSF035J26D6CGX7STCSD1KG"), nil) - bucketClient.MockGet("user-2/01FRSF035J26D6CGX7STCSD1KG/deletion-mark.json", "", nil) - bucketClient.MockGet("user-2/01FRSF035J26D6CGX7STCSD1KG/no-compact-mark.json", "", nil) - bucketClient.MockGet("user-1/bucket-index.json.gz", "", nil) - bucketClient.MockGet("user-2/bucket-index.json.gz", "", nil) - bucketClient.MockIter("user-1/markers/", nil, nil) - bucketClient.MockIter("user-2/markers/", nil, nil) - bucketClient.MockUpload("user-1/bucket-index.json.gz", nil) - bucketClient.MockUpload("user-2/bucket-index.json.gz", nil) + bucketClient.MockExists(path.Join("user-1", "phlaredb/", bucket.TenantDeletionMarkPath), false, nil) + bucketClient.MockExists(path.Join("user-2", "phlaredb/", bucket.TenantDeletionMarkPath), false, nil) + bucketClient.MockIter("user-1/phlaredb/", []string{"user-1/phlaredb/01DTVP434PA9VFXSW2JKB3392D", "user-1/phlaredb/01FS51A7GQ1RQWV35DBVYQM4KF"}, nil) + bucketClient.MockIter("user-2/phlaredb/", []string{"user-2/phlaredb/01DTW0ZCPDDNV4BV83Q2SV4QAZ", "user-2/phlaredb/01FRSF035J26D6CGX7STCSD1KG"}, nil) + bucketClient.MockGet("user-1/phlaredb/01DTVP434PA9VFXSW2JKB3392D/meta.json", mockBlockMetaJSON("01DTVP434PA9VFXSW2JKB3392D"), nil) + bucketClient.MockGet("user-1/phlaredb/01DTVP434PA9VFXSW2JKB3392D/deletion-mark.json", "", nil) + bucketClient.MockGet("user-1/phlaredb/01DTVP434PA9VFXSW2JKB3392D/no-compact-mark.json", "", nil) + bucketClient.MockGet("user-1/phlaredb/01FS51A7GQ1RQWV35DBVYQM4KF/meta.json", mockBlockMetaJSON("01FS51A7GQ1RQWV35DBVYQM4KF"), nil) + bucketClient.MockGet("user-1/phlaredb/01FS51A7GQ1RQWV35DBVYQM4KF/deletion-mark.json", "", nil) + bucketClient.MockGet("user-1/phlaredb/01FS51A7GQ1RQWV35DBVYQM4KF/no-compact-mark.json", "", nil) + + bucketClient.MockGet("user-2/phlaredb/01DTW0ZCPDDNV4BV83Q2SV4QAZ/meta.json", mockBlockMetaJSON("01DTW0ZCPDDNV4BV83Q2SV4QAZ"), nil) + bucketClient.MockGet("user-2/phlaredb/01DTW0ZCPDDNV4BV83Q2SV4QAZ/deletion-mark.json", "", nil) + bucketClient.MockGet("user-2/phlaredb/01DTW0ZCPDDNV4BV83Q2SV4QAZ/no-compact-mark.json", "", nil) + bucketClient.MockGet("user-2/phlaredb/01FRSF035J26D6CGX7STCSD1KG/meta.json", mockBlockMetaJSON("01FRSF035J26D6CGX7STCSD1KG"), nil) + bucketClient.MockGet("user-2/phlaredb/01FRSF035J26D6CGX7STCSD1KG/deletion-mark.json", "", nil) + bucketClient.MockGet("user-2/phlaredb/01FRSF035J26D6CGX7STCSD1KG/no-compact-mark.json", "", nil) + bucketClient.MockGet("user-1/phlaredb/bucket-index.json.gz", "", nil) + bucketClient.MockGet("user-2/phlaredb/bucket-index.json.gz", "", nil) + bucketClient.MockIter("user-1/phlaredb/markers/", nil, nil) + bucketClient.MockIter("user-2/phlaredb/markers/", nil, nil) + bucketClient.MockUpload("user-1/phlaredb/bucket-index.json.gz", nil) + bucketClient.MockUpload("user-2/phlaredb/bucket-index.json.gz", nil) c, _, tsdbPlanner, logs, registry := prepare(t, prepareConfig(t), bucketClient) @@ -666,23 +666,23 @@ func TestMultitenantCompactor_ShouldStopCompactingTenantOnReachingMaxCompactionT // and since its planning will take longer than maxCompactionTime, we stop compactions early. bucketClient := &pyroscope_objstore.ClientMock{} bucketClient.MockIter("", []string{"user-1"}, nil) - bucketClient.MockExists(path.Join("user-1", bucket.TenantDeletionMarkPath), false, nil) - bucketClient.MockIter("user-1/", []string{"user-1/01DTVP434PA9VFXSW2JKB3392D", "user-1/01FN3VCQV5X342W2ZKMQQXAZRX", "user-1/01FS51A7GQ1RQWV35DBVYQM4KF", "user-1/01FRQGQB7RWQ2TS0VWA82QTPXE"}, nil) - bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", mockBlockMetaJSONWithTimeRangeAndLabels("01DTVP434PA9VFXSW2JKB3392D", 1574776800000, 1574784000000, map[string]string{"A": "B"}), nil) - bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/deletion-mark.json", "", nil) - bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/no-compact-mark.json", "", nil) - bucketClient.MockGet("user-1/01FS51A7GQ1RQWV35DBVYQM4KF/meta.json", mockBlockMetaJSONWithTimeRangeAndLabels("01FS51A7GQ1RQWV35DBVYQM4KF", 1574776800000, 1574784000000, map[string]string{"A": "B"}), nil) - bucketClient.MockGet("user-1/01FS51A7GQ1RQWV35DBVYQM4KF/deletion-mark.json", "", nil) - bucketClient.MockGet("user-1/01FS51A7GQ1RQWV35DBVYQM4KF/no-compact-mark.json", "", nil) - bucketClient.MockGet("user-1/01FN3VCQV5X342W2ZKMQQXAZRX/meta.json", mockBlockMetaJSONWithTimeRangeAndLabels("01FN3VCQV5X342W2ZKMQQXAZRX", 1574776800000, 1574784000000, map[string]string{"C": "D"}), nil) - bucketClient.MockGet("user-1/01FN3VCQV5X342W2ZKMQQXAZRX/deletion-mark.json", "", nil) - bucketClient.MockGet("user-1/01FN3VCQV5X342W2ZKMQQXAZRX/no-compact-mark.json", "", nil) - bucketClient.MockGet("user-1/01FRQGQB7RWQ2TS0VWA82QTPXE/meta.json", mockBlockMetaJSONWithTimeRangeAndLabels("01FRQGQB7RWQ2TS0VWA82QTPXE", 1574776800000, 1574784000000, map[string]string{"C": "D"}), nil) - bucketClient.MockGet("user-1/01FRQGQB7RWQ2TS0VWA82QTPXE/deletion-mark.json", "", nil) - bucketClient.MockGet("user-1/01FRQGQB7RWQ2TS0VWA82QTPXE/no-compact-mark.json", "", nil) - bucketClient.MockGet("user-1/bucket-index.json.gz", "", nil) - bucketClient.MockIter("user-1/markers/", nil, nil) - bucketClient.MockUpload("user-1/bucket-index.json.gz", nil) + bucketClient.MockExists(path.Join("user-1", "phlaredb/", bucket.TenantDeletionMarkPath), false, nil) + bucketClient.MockIter("user-1/phlaredb/", []string{"user-1/phlaredb/01DTVP434PA9VFXSW2JKB3392D", "user-1/phlaredb/01FN3VCQV5X342W2ZKMQQXAZRX", "user-1/phlaredb/01FS51A7GQ1RQWV35DBVYQM4KF", "user-1/phlaredb/01FRQGQB7RWQ2TS0VWA82QTPXE"}, nil) + bucketClient.MockGet("user-1/phlaredb/01DTVP434PA9VFXSW2JKB3392D/meta.json", mockBlockMetaJSONWithTimeRangeAndLabels("01DTVP434PA9VFXSW2JKB3392D", 1574776800000, 1574784000000, map[string]string{"A": "B"}), nil) + bucketClient.MockGet("user-1/phlaredb/01DTVP434PA9VFXSW2JKB3392D/deletion-mark.json", "", nil) + bucketClient.MockGet("user-1/phlaredb/01DTVP434PA9VFXSW2JKB3392D/no-compact-mark.json", "", nil) + bucketClient.MockGet("user-1/phlaredb/01FS51A7GQ1RQWV35DBVYQM4KF/meta.json", mockBlockMetaJSONWithTimeRangeAndLabels("01FS51A7GQ1RQWV35DBVYQM4KF", 1574776800000, 1574784000000, map[string]string{"A": "B"}), nil) + bucketClient.MockGet("user-1/phlaredb/01FS51A7GQ1RQWV35DBVYQM4KF/deletion-mark.json", "", nil) + bucketClient.MockGet("user-1/phlaredb/01FS51A7GQ1RQWV35DBVYQM4KF/no-compact-mark.json", "", nil) + bucketClient.MockGet("user-1/phlaredb/01FN3VCQV5X342W2ZKMQQXAZRX/meta.json", mockBlockMetaJSONWithTimeRangeAndLabels("01FN3VCQV5X342W2ZKMQQXAZRX", 1574776800000, 1574784000000, map[string]string{"C": "D"}), nil) + bucketClient.MockGet("user-1/phlaredb/01FN3VCQV5X342W2ZKMQQXAZRX/deletion-mark.json", "", nil) + bucketClient.MockGet("user-1/phlaredb/01FN3VCQV5X342W2ZKMQQXAZRX/no-compact-mark.json", "", nil) + bucketClient.MockGet("user-1/phlaredb/01FRQGQB7RWQ2TS0VWA82QTPXE/meta.json", mockBlockMetaJSONWithTimeRangeAndLabels("01FRQGQB7RWQ2TS0VWA82QTPXE", 1574776800000, 1574784000000, map[string]string{"C": "D"}), nil) + bucketClient.MockGet("user-1/phlaredb/01FRQGQB7RWQ2TS0VWA82QTPXE/deletion-mark.json", "", nil) + bucketClient.MockGet("user-1/phlaredb/01FRQGQB7RWQ2TS0VWA82QTPXE/no-compact-mark.json", "", nil) + bucketClient.MockGet("user-1/phlaredb/bucket-index.json.gz", "", nil) + bucketClient.MockIter("user-1/phlaredb/markers/", nil, nil) + bucketClient.MockUpload("user-1/phlaredb/bucket-index.json.gz", nil) cfg := prepareConfig(t) cfg.MaxCompactionTime = 500 * time.Millisecond // Enough time to start one compaction. We will make it last longer than this. @@ -736,35 +736,35 @@ func TestMultitenantCompactor_ShouldNotCompactBlocksMarkedForDeletion(t *testing // Mock the bucket to contain two users, each one with one block. bucketClient := &pyroscope_objstore.ClientMock{} bucketClient.MockIter("", []string{"user-1"}, nil) - bucketClient.MockIter("user-1/", []string{"user-1/01DTVP434PA9VFXSW2JKB3392D", "user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ"}, nil) - bucketClient.MockExists(path.Join("user-1", bucket.TenantDeletionMarkPath), false, nil) + bucketClient.MockIter("user-1/phlaredb/", []string{"user-1/phlaredb/01DTVP434PA9VFXSW2JKB3392D", "user-1/phlaredb/01DTW0ZCPDDNV4BV83Q2SV4QAZ"}, nil) + bucketClient.MockExists(path.Join("user-1", "phlaredb/", bucket.TenantDeletionMarkPath), false, nil) // Block that has just been marked for deletion. It will not be deleted just yet, and it also will not be compacted. - bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", mockBlockMetaJSON("01DTVP434PA9VFXSW2JKB3392D"), nil) - bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/deletion-mark.json", mockDeletionMarkJSON("01DTVP434PA9VFXSW2JKB3392D", time.Now()), nil) - bucketClient.MockGet("user-1/markers/01DTVP434PA9VFXSW2JKB3392D-deletion-mark.json", mockDeletionMarkJSON("01DTVP434PA9VFXSW2JKB3392D", time.Now()), nil) + bucketClient.MockGet("user-1/phlaredb/01DTVP434PA9VFXSW2JKB3392D/meta.json", mockBlockMetaJSON("01DTVP434PA9VFXSW2JKB3392D"), nil) + bucketClient.MockGet("user-1/phlaredb/01DTVP434PA9VFXSW2JKB3392D/deletion-mark.json", mockDeletionMarkJSON("01DTVP434PA9VFXSW2JKB3392D", time.Now()), nil) + bucketClient.MockGet("user-1/phlaredb/markers/01DTVP434PA9VFXSW2JKB3392D-deletion-mark.json", mockDeletionMarkJSON("01DTVP434PA9VFXSW2JKB3392D", time.Now()), nil) // This block will be deleted by cleaner. - bucketClient.MockGet("user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ/meta.json", mockBlockMetaJSON("01DTW0ZCPDDNV4BV83Q2SV4QAZ"), nil) - bucketClient.MockGet("user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ/deletion-mark.json", mockDeletionMarkJSON("01DTW0ZCPDDNV4BV83Q2SV4QAZ", time.Now().Add(-cfg.DeletionDelay)), nil) - bucketClient.MockGet("user-1/markers/01DTW0ZCPDDNV4BV83Q2SV4QAZ-deletion-mark.json", mockDeletionMarkJSON("01DTW0ZCPDDNV4BV83Q2SV4QAZ", time.Now().Add(-cfg.DeletionDelay)), nil) + bucketClient.MockGet("user-1/phlaredb/01DTW0ZCPDDNV4BV83Q2SV4QAZ/meta.json", mockBlockMetaJSON("01DTW0ZCPDDNV4BV83Q2SV4QAZ"), nil) + bucketClient.MockGet("user-1/phlaredb/01DTW0ZCPDDNV4BV83Q2SV4QAZ/deletion-mark.json", mockDeletionMarkJSON("01DTW0ZCPDDNV4BV83Q2SV4QAZ", time.Now().Add(-cfg.DeletionDelay)), nil) + bucketClient.MockGet("user-1/phlaredb/markers/01DTW0ZCPDDNV4BV83Q2SV4QAZ-deletion-mark.json", mockDeletionMarkJSON("01DTW0ZCPDDNV4BV83Q2SV4QAZ", time.Now().Add(-cfg.DeletionDelay)), nil) - bucketClient.MockIter("user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ", []string{ - "user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ/meta.json", - "user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ/deletion-mark.json", + bucketClient.MockIter("user-1/phlaredb/01DTW0ZCPDDNV4BV83Q2SV4QAZ", []string{ + "user-1/phlaredb/01DTW0ZCPDDNV4BV83Q2SV4QAZ/meta.json", + "user-1/phlaredb/01DTW0ZCPDDNV4BV83Q2SV4QAZ/deletion-mark.json", }, nil) - bucketClient.MockIter("user-1/markers/", []string{ - "user-1/markers/01DTVP434PA9VFXSW2JKB3392D-deletion-mark.json", - "user-1/markers/01DTW0ZCPDDNV4BV83Q2SV4QAZ-deletion-mark.json", + bucketClient.MockIter("user-1/phlaredb/markers/", []string{ + "user-1/phlaredb/markers/01DTVP434PA9VFXSW2JKB3392D-deletion-mark.json", + "user-1/phlaredb/markers/01DTW0ZCPDDNV4BV83Q2SV4QAZ-deletion-mark.json", }, nil) - bucketClient.MockDelete("user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ/meta.json", nil) - bucketClient.MockDelete("user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ/deletion-mark.json", nil) - bucketClient.MockDelete("user-1/markers/01DTW0ZCPDDNV4BV83Q2SV4QAZ-deletion-mark.json", nil) - bucketClient.MockDelete("user-1/01DTW0ZCPDDNV4BV83Q2SV4QAZ", nil) - bucketClient.MockGet("user-1/bucket-index.json.gz", "", nil) - bucketClient.MockUpload("user-1/bucket-index.json.gz", nil) + bucketClient.MockDelete("user-1/phlaredb/01DTW0ZCPDDNV4BV83Q2SV4QAZ/meta.json", nil) + bucketClient.MockDelete("user-1/phlaredb/01DTW0ZCPDDNV4BV83Q2SV4QAZ/deletion-mark.json", nil) + bucketClient.MockDelete("user-1/phlaredb/markers/01DTW0ZCPDDNV4BV83Q2SV4QAZ-deletion-mark.json", nil) + bucketClient.MockDelete("user-1/phlaredb/01DTW0ZCPDDNV4BV83Q2SV4QAZ", nil) + bucketClient.MockGet("user-1/phlaredb/bucket-index.json.gz", "", nil) + bucketClient.MockUpload("user-1/phlaredb/bucket-index.json.gz", nil) c, _, tsdbPlanner, logs, registry := prepare(t, cfg, bucketClient) @@ -854,18 +854,18 @@ func TestMultitenantCompactor_ShouldNotCompactBlocksMarkedForNoCompaction(t *tes // Mock the bucket to contain one user with a block marked for no-compaction. bucketClient := &pyroscope_objstore.ClientMock{} bucketClient.MockIter("", []string{"user-1"}, nil) - bucketClient.MockIter("user-1/", []string{"user-1/01DTVP434PA9VFXSW2JKB3392D"}, nil) - bucketClient.MockExists(path.Join("user-1", bucket.TenantDeletionMarkPath), false, nil) + bucketClient.MockIter("user-1/phlaredb/", []string{"user-1/phlaredb/01DTVP434PA9VFXSW2JKB3392D"}, nil) + bucketClient.MockExists(path.Join("user-1", "phlaredb/", bucket.TenantDeletionMarkPath), false, nil) // Block that is marked for no compaction. It will be ignored. - bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", mockBlockMetaJSON("01DTVP434PA9VFXSW2JKB3392D"), nil) - bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/deletion-mark.json", "", nil) - bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/no-compact-mark.json", `{"id":"01DTVP434PA9VFXSW2JKB3392D","version":1,"details":"details","no_compact_time":1637757932,"reason":"reason"}`, nil) + bucketClient.MockGet("user-1/phlaredb/01DTVP434PA9VFXSW2JKB3392D/meta.json", mockBlockMetaJSON("01DTVP434PA9VFXSW2JKB3392D"), nil) + bucketClient.MockGet("user-1/phlaredb/01DTVP434PA9VFXSW2JKB3392D/deletion-mark.json", "", nil) + bucketClient.MockGet("user-1/phlaredb/01DTVP434PA9VFXSW2JKB3392D/no-compact-mark.json", `{"id":"01DTVP434PA9VFXSW2JKB3392D","version":1,"details":"details","no_compact_time":1637757932,"reason":"reason"}`, nil) - bucketClient.MockIter("user-1/markers/", []string{"user-1/markers/01DTVP434PA9VFXSW2JKB3392D-no-compact-mark.json"}, nil) + bucketClient.MockIter("user-1/phlaredb/markers/", []string{"user-1/markers/01DTVP434PA9VFXSW2JKB3392D-no-compact-mark.json"}, nil) - bucketClient.MockGet("user-1/bucket-index.json.gz", "", nil) - bucketClient.MockUpload("user-1/bucket-index.json.gz", nil) + bucketClient.MockGet("user-1/phlaredb/bucket-index.json.gz", "", nil) + bucketClient.MockUpload("user-1/phlaredb/bucket-index.json.gz", nil) c, _, tsdbPlanner, logs, _ := prepare(t, cfg, bucketClient) @@ -908,19 +908,19 @@ func TestMultitenantCompactor_ShouldNotCompactBlocksForUsersMarkedForDeletion(t // Mock the bucket to contain two users, each one with one block. bucketClient := &pyroscope_objstore.ClientMock{} bucketClient.MockIter("", []string{"user-1"}, nil) - bucketClient.MockIter("user-1/", []string{"user-1/01DTVP434PA9VFXSW2JKB3392D"}, nil) - bucketClient.MockGet(path.Join("user-1", bucket.TenantDeletionMarkPath), `{"deletion_time": 1}`, nil) - bucketClient.MockUpload(path.Join("user-1", bucket.TenantDeletionMarkPath), nil) + bucketClient.MockIter("user-1/phlaredb/", []string{"user-1/phlaredb/01DTVP434PA9VFXSW2JKB3392D"}, nil) + bucketClient.MockGet(path.Join("user-1", "phlaredb/", bucket.TenantDeletionMarkPath), `{"deletion_time": 1}`, nil) + bucketClient.MockUpload(path.Join("user-1", "phlaredb/", bucket.TenantDeletionMarkPath), nil) - bucketClient.MockIter("user-1/01DTVP434PA9VFXSW2JKB3392D", []string{"user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", "user-1/01DTVP434PA9VFXSW2JKB3392D/index"}, nil) - bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", mockBlockMetaJSON("01DTVP434PA9VFXSW2JKB3392D"), nil) - bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/index", "some index content", nil) - bucketClient.MockExists("user-1/01DTVP434PA9VFXSW2JKB3392D/deletion-mark.json", false, nil) - bucketClient.MockExists("user-1/markers/01DTVP434PA9VFXSW2JKB3392D-deletion-mark.json", false, nil) + bucketClient.MockIter("user-1/phlaredb/01DTVP434PA9VFXSW2JKB3392D", []string{"user-1/phlaredb/01DTVP434PA9VFXSW2JKB3392D/meta.json", "user-1/phlaredb/01DTVP434PA9VFXSW2JKB3392D/index"}, nil) + bucketClient.MockGet("user-1/phlaredb/01DTVP434PA9VFXSW2JKB3392D/meta.json", mockBlockMetaJSON("01DTVP434PA9VFXSW2JKB3392D"), nil) + bucketClient.MockGet("user-1/phlaredb/01DTVP434PA9VFXSW2JKB3392D/index", "some index content", nil) + bucketClient.MockExists("user-1/phlaredb/01DTVP434PA9VFXSW2JKB3392D/deletion-mark.json", false, nil) + bucketClient.MockExists("user-1/phlaredb/markers/01DTVP434PA9VFXSW2JKB3392D-deletion-mark.json", false, nil) - bucketClient.MockDelete("user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", nil) - bucketClient.MockDelete("user-1/01DTVP434PA9VFXSW2JKB3392D/index", nil) - bucketClient.MockDelete("user-1/bucket-index.json.gz", nil) + bucketClient.MockDelete("user-1/phlaredb/01DTVP434PA9VFXSW2JKB3392D/meta.json", nil) + bucketClient.MockDelete("user-1/phlaredb/01DTVP434PA9VFXSW2JKB3392D/index", nil) + bucketClient.MockDelete("user-1/phlaredb/bucket-index.json.gz", nil) c, _, tsdbPlanner, logs, registry := prepare(t, cfg, bucketClient) @@ -1009,28 +1009,28 @@ func TestMultitenantCompactor_ShouldCompactAllUsersOnShardingEnabledButOnlyOneIn // Mock the bucket to contain two users, each one with one block. bucketClient := &pyroscope_objstore.ClientMock{} bucketClient.MockIter("", []string{"user-1", "user-2"}, nil) - bucketClient.MockExists(path.Join("user-1", bucket.TenantDeletionMarkPath), false, nil) - bucketClient.MockExists(path.Join("user-2", bucket.TenantDeletionMarkPath), false, nil) - bucketClient.MockIter("user-1/", []string{"user-1/01DTVP434PA9VFXSW2JKB3392D", "user-1/01FSTQ95C8FS0ZAGTQS2EF1NEG"}, nil) - bucketClient.MockIter("user-2/", []string{"user-2/01DTW0ZCPDDNV4BV83Q2SV4QAZ", "user-2/01FSV54G6QFQH1G9QE93G3B9TB"}, nil) - bucketClient.MockIter("user-1/markers/", nil, nil) - bucketClient.MockIter("user-2/markers/", nil, nil) - bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/meta.json", mockBlockMetaJSON("01DTVP434PA9VFXSW2JKB3392D"), nil) - bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/deletion-mark.json", "", nil) - bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JKB3392D/no-compact-mark.json", "", nil) - bucketClient.MockGet("user-1/01FSTQ95C8FS0ZAGTQS2EF1NEG/meta.json", mockBlockMetaJSON("01FSTQ95C8FS0ZAGTQS2EF1NEG"), nil) - bucketClient.MockGet("user-1/01FSTQ95C8FS0ZAGTQS2EF1NEG/deletion-mark.json", "", nil) - bucketClient.MockGet("user-1/01FSTQ95C8FS0ZAGTQS2EF1NEG/no-compact-mark.json", "", nil) - bucketClient.MockGet("user-2/01DTW0ZCPDDNV4BV83Q2SV4QAZ/meta.json", mockBlockMetaJSON("01DTW0ZCPDDNV4BV83Q2SV4QAZ"), nil) - bucketClient.MockGet("user-2/01DTW0ZCPDDNV4BV83Q2SV4QAZ/deletion-mark.json", "", nil) - bucketClient.MockGet("user-2/01DTW0ZCPDDNV4BV83Q2SV4QAZ/no-compact-mark.json", "", nil) - bucketClient.MockGet("user-2/01FSV54G6QFQH1G9QE93G3B9TB/meta.json", mockBlockMetaJSON("01FSV54G6QFQH1G9QE93G3B9TB"), nil) - bucketClient.MockGet("user-2/01FSV54G6QFQH1G9QE93G3B9TB/deletion-mark.json", "", nil) - bucketClient.MockGet("user-2/01FSV54G6QFQH1G9QE93G3B9TB/no-compact-mark.json", "", nil) - bucketClient.MockGet("user-1/bucket-index.json.gz", "", nil) - bucketClient.MockGet("user-2/bucket-index.json.gz", "", nil) - bucketClient.MockUpload("user-1/bucket-index.json.gz", nil) - bucketClient.MockUpload("user-2/bucket-index.json.gz", nil) + bucketClient.MockExists(path.Join("user-1", "phlaredb", bucket.TenantDeletionMarkPath), false, nil) + bucketClient.MockExists(path.Join("user-2", "phlaredb", bucket.TenantDeletionMarkPath), false, nil) + bucketClient.MockIter("user-1/phlaredb/", []string{"user-1/phlaredb/01DTVP434PA9VFXSW2JKB3392D", "user-1/phlaredb/01FSTQ95C8FS0ZAGTQS2EF1NEG"}, nil) + bucketClient.MockIter("user-2/phlaredb/", []string{"user-2/phlaredb/01DTW0ZCPDDNV4BV83Q2SV4QAZ", "user-2/phlaredb/01FSV54G6QFQH1G9QE93G3B9TB"}, nil) + bucketClient.MockIter("user-1/phlaredb/markers/", nil, nil) + bucketClient.MockIter("user-2/phlaredb/markers/", nil, nil) + bucketClient.MockGet("user-1/phlaredb/01DTVP434PA9VFXSW2JKB3392D/meta.json", mockBlockMetaJSON("01DTVP434PA9VFXSW2JKB3392D"), nil) + bucketClient.MockGet("user-1/phlaredb/01DTVP434PA9VFXSW2JKB3392D/deletion-mark.json", "", nil) + bucketClient.MockGet("user-1/phlaredb/01DTVP434PA9VFXSW2JKB3392D/no-compact-mark.json", "", nil) + bucketClient.MockGet("user-1/phlaredb/01FSTQ95C8FS0ZAGTQS2EF1NEG/meta.json", mockBlockMetaJSON("01FSTQ95C8FS0ZAGTQS2EF1NEG"), nil) + bucketClient.MockGet("user-1/phlaredb/01FSTQ95C8FS0ZAGTQS2EF1NEG/deletion-mark.json", "", nil) + bucketClient.MockGet("user-1/phlaredb/01FSTQ95C8FS0ZAGTQS2EF1NEG/no-compact-mark.json", "", nil) + bucketClient.MockGet("user-2/phlaredb/01DTW0ZCPDDNV4BV83Q2SV4QAZ/meta.json", mockBlockMetaJSON("01DTW0ZCPDDNV4BV83Q2SV4QAZ"), nil) + bucketClient.MockGet("user-2/phlaredb/01DTW0ZCPDDNV4BV83Q2SV4QAZ/deletion-mark.json", "", nil) + bucketClient.MockGet("user-2/phlaredb/01DTW0ZCPDDNV4BV83Q2SV4QAZ/no-compact-mark.json", "", nil) + bucketClient.MockGet("user-2/phlaredb/01FSV54G6QFQH1G9QE93G3B9TB/meta.json", mockBlockMetaJSON("01FSV54G6QFQH1G9QE93G3B9TB"), nil) + bucketClient.MockGet("user-2/phlaredb/01FSV54G6QFQH1G9QE93G3B9TB/deletion-mark.json", "", nil) + bucketClient.MockGet("user-2/phlaredb/01FSV54G6QFQH1G9QE93G3B9TB/no-compact-mark.json", "", nil) + bucketClient.MockGet("user-1/phlaredb/bucket-index.json.gz", "", nil) + bucketClient.MockGet("user-2/phlaredb/bucket-index.json.gz", "", nil) + bucketClient.MockUpload("user-1/phlaredb/bucket-index.json.gz", nil) + bucketClient.MockUpload("user-2/phlaredb/bucket-index.json.gz", nil) ringStore, closer := consul.NewInMemoryClient(ring.GetCodec(), log.NewNopLogger(), nil) t.Cleanup(func() { assert.NoError(t, closer.Close()) }) @@ -1147,14 +1147,14 @@ func TestMultitenantCompactor_ShouldCompactOnlyUsersOwnedByTheInstanceOnSharding bucketClient := &pyroscope_objstore.ClientMock{} bucketClient.MockIter("", userIDs, nil) for _, userID := range userIDs { - bucketClient.MockIter(userID+"/", []string{userID + "/01DTVP434PA9VFXSW2JKB3392D"}, nil) - bucketClient.MockIter(userID+"/markers/", nil, nil) - bucketClient.MockExists(path.Join(userID, bucket.TenantDeletionMarkPath), false, nil) - bucketClient.MockGet(userID+"/01DTVP434PA9VFXSW2JKB3392D/meta.json", mockBlockMetaJSON("01DTVP434PA9VFXSW2JKB3392D"), nil) - bucketClient.MockGet(userID+"/01DTVP434PA9VFXSW2JKB3392D/deletion-mark.json", "", nil) - bucketClient.MockGet(userID+"/01DTVP434PA9VFXSW2JKB3392D/no-compact-mark.json", "", nil) - bucketClient.MockGet(userID+"/bucket-index.json.gz", "", nil) - bucketClient.MockUpload(userID+"/bucket-index.json.gz", nil) + bucketClient.MockIter(userID+"/phlaredb/", []string{userID + "/phlaredb/01DTVP434PA9VFXSW2JKB3392D"}, nil) + bucketClient.MockIter(userID+"/phlaredb/markers/", nil, nil) + bucketClient.MockExists(path.Join(userID, "phlaredb/", bucket.TenantDeletionMarkPath), false, nil) + bucketClient.MockGet(userID+"/phlaredb/01DTVP434PA9VFXSW2JKB3392D/meta.json", mockBlockMetaJSON("01DTVP434PA9VFXSW2JKB3392D"), nil) + bucketClient.MockGet(userID+"/phlaredb/01DTVP434PA9VFXSW2JKB3392D/deletion-mark.json", "", nil) + bucketClient.MockGet(userID+"/phlaredb/01DTVP434PA9VFXSW2JKB3392D/no-compact-mark.json", "", nil) + bucketClient.MockGet(userID+"/phlaredb/bucket-index.json.gz", "", nil) + bucketClient.MockUpload(userID+"/phlaredb/bucket-index.json.gz", nil) } // Create a shared KV Store @@ -1222,17 +1222,17 @@ func TestMultitenantCompactor_ShouldSkipCompactionForJobsNoMoreOwnedAfterPlannin // for the splitting stage). bucketClient := &pyroscope_objstore.ClientMock{} bucketClient.MockIter("", []string{"user-1"}, nil) - bucketClient.MockExists(path.Join("user-1", bucket.TenantDeletionMarkPath), false, nil) - bucketClient.MockIter("user-1/", []string{"user-1/01DTVP434PA9VFXSW2JK000001", "user-1/01DTVP434PA9VFXSW2JK000002"}, nil) - bucketClient.MockIter("user-1/markers/", nil, nil) - bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JK000001/meta.json", mockBlockMetaJSONWithTimeRange("01DTVP434PA9VFXSW2JK000001", 1574776800000, 1574784000000), nil) - bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JK000001/deletion-mark.json", "", nil) - bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JK000001/no-compact-mark.json", "", nil) - bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JK000002/meta.json", mockBlockMetaJSONWithTimeRange("01DTVP434PA9VFXSW2JK000002", 1574863200000, 1574870400000), nil) - bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JK000002/deletion-mark.json", "", nil) - bucketClient.MockGet("user-1/01DTVP434PA9VFXSW2JK000002/no-compact-mark.json", "", nil) - bucketClient.MockGet("user-1/bucket-index.json.gz", "", nil) - bucketClient.MockUpload("user-1/bucket-index.json.gz", nil) + bucketClient.MockExists(path.Join("user-1", "phlaredb", bucket.TenantDeletionMarkPath), false, nil) + bucketClient.MockIter("user-1/phlaredb/", []string{"user-1/phlaredb/01DTVP434PA9VFXSW2JK000001", "user-1/phlaredb/01DTVP434PA9VFXSW2JK000002"}, nil) + bucketClient.MockIter("user-1/phlaredb/markers/", nil, nil) + bucketClient.MockGet("user-1/phlaredb/01DTVP434PA9VFXSW2JK000001/meta.json", mockBlockMetaJSONWithTimeRange("01DTVP434PA9VFXSW2JK000001", 1574776800000, 1574784000000), nil) + bucketClient.MockGet("user-1/phlaredb/01DTVP434PA9VFXSW2JK000001/deletion-mark.json", "", nil) + bucketClient.MockGet("user-1/phlaredb/01DTVP434PA9VFXSW2JK000001/no-compact-mark.json", "", nil) + bucketClient.MockGet("user-1/phlaredb/01DTVP434PA9VFXSW2JK000002/meta.json", mockBlockMetaJSONWithTimeRange("01DTVP434PA9VFXSW2JK000002", 1574863200000, 1574870400000), nil) + bucketClient.MockGet("user-1/phlaredb/01DTVP434PA9VFXSW2JK000002/deletion-mark.json", "", nil) + bucketClient.MockGet("user-1/phlaredb/01DTVP434PA9VFXSW2JK000002/no-compact-mark.json", "", nil) + bucketClient.MockGet("user-1/phlaredb/bucket-index.json.gz", "", nil) + bucketClient.MockUpload("user-1/phlaredb/bucket-index.json.gz", nil) ringStore, closer := consul.NewInMemoryClient(ring.GetCodec(), log.NewNopLogger(), nil) t.Cleanup(func() { assert.NoError(t, closer.Close()) }) @@ -1360,10 +1360,10 @@ func TestMultitenantCompactor_ShouldSkipCompactionForJobsWithFirstLevelCompactio mockClient := &bucketWithMockedAttributes{ Bucket: bucketClient, customAttributes: map[string]objstore.ObjectAttributes{ - path.Join("user-1", user1Meta1.String(), block.MetaFilename): {LastModified: time.Now().Add(-20 * time.Minute)}, - path.Join("user-1", user1Meta2.String(), block.MetaFilename): {LastModified: time.Now().Add(-20 * time.Minute)}, - path.Join("user-2", user2Meta1.String(), block.MetaFilename): {LastModified: time.Now().Add(-20 * time.Minute)}, - path.Join("user-2", user2Meta2.String(), block.MetaFilename): {LastModified: time.Now().Add(-5 * time.Minute)}, + path.Join("user-1", "phlaredb", user1Meta1.String(), block.MetaFilename): {LastModified: time.Now().Add(-20 * time.Minute)}, + path.Join("user-1", "phlaredb", user1Meta2.String(), block.MetaFilename): {LastModified: time.Now().Add(-20 * time.Minute)}, + path.Join("user-2", "phlaredb", user2Meta1.String(), block.MetaFilename): {LastModified: time.Now().Add(-20 * time.Minute)}, + path.Join("user-2", "phlaredb", user2Meta2.String(), block.MetaFilename): {LastModified: time.Now().Add(-5 * time.Minute)}, }, } @@ -1472,7 +1472,7 @@ func createCustomBlock(t *testing.T, bkt objstore.Bucket, userID string, externa return err } - return bkt.Upload(context.Background(), path.Join(userID, meta.ULID.String(), relPath), bytes.NewReader(content)) + return bkt.Upload(context.Background(), path.Join(userID, "phlaredb/", meta.ULID.String(), relPath), bytes.NewReader(content)) })) return meta.ULID @@ -1510,7 +1510,7 @@ func createDBBlock(t *testing.T, bkt objstore.Bucket, userID string, minT, maxT func createDeletionMark(t *testing.T, bkt objstore.Bucket, userID string, blockID ulid.ULID, deletionTime time.Time) { content := mockDeletionMarkJSON(blockID.String(), deletionTime) - blockPath := path.Join(userID, blockID.String()) + blockPath := path.Join(userID, "phlaredb/", blockID.String()) markPath := path.Join(blockPath, block.DeletionMarkFilename) require.NoError(t, bkt.Upload(context.Background(), markPath, strings.NewReader(content))) diff --git a/pkg/compactor/split_merge_compactor_test.go b/pkg/compactor/split_merge_compactor_test.go index b92ee229c4..26a24a0f4a 100644 --- a/pkg/compactor/split_merge_compactor_test.go +++ b/pkg/compactor/split_merge_compactor_test.go @@ -505,7 +505,7 @@ func TestMultitenantCompactor_ShouldSupportSplitAndMergeCompactor(t *testing.T) }) // List back any (non deleted) block from the storage. - userBucket := objstore.NewUserBucketClient(userID, bkt, nil) + userBucket := objstore.NewTenantBucketClient(userID, bkt, nil) fetcher, err := block.NewMetaFetcher(logger, 1, userBucket, @@ -601,7 +601,7 @@ func TestMultitenantCompactor_ShouldGuaranteeSeriesShardingConsistencyOverTheTim }) // List back any (non deleted) block from the storage. - userBucket := objstore.NewUserBucketClient(userID, bucketClient, nil) + userBucket := objstore.NewTenantBucketClient(userID, bucketClient, nil) fetcher, err := block.NewMetaFetcher(logger, 1, userBucket, diff --git a/pkg/ingester/instance.go b/pkg/ingester/instance.go index 9e44f638a5..ccb8036f95 100644 --- a/pkg/ingester/instance.go +++ b/pkg/ingester/instance.go @@ -51,7 +51,7 @@ func newInstance(phlarectx context.Context, cfg phlaredb.Config, tenantID string inst.logger, inst.reg, db, - phlareobj.NewPrefixedBucket(storageBucket, tenantID+"/phlaredb"), + phlareobj.NewTenantBucketClient(tenantID, storageBucket, nil), block.IngesterSource, false, false, diff --git a/pkg/objstore/user_bucket_client.go b/pkg/objstore/user_bucket_client.go index 18605244d0..d89ad31f50 100644 --- a/pkg/objstore/user_bucket_client.go +++ b/pkg/objstore/user_bucket_client.go @@ -5,12 +5,12 @@ package objstore -// NewUserBucketClient returns a bucket client to use to access the storage on behalf of the provided user. +// NewTenantBucketClient returns a bucket client to use to access the storage on behalf of the provided user. // The cfgProvider can be nil. -func NewUserBucketClient(userID string, bucket Bucket, cfgProvider TenantConfigProvider) InstrumentedBucket { +func NewTenantBucketClient(tenantID string, bucket Bucket, cfgProvider TenantConfigProvider) InstrumentedBucket { // Inject the user/tenant prefix. - bucket = NewPrefixedBucket(bucket, userID) + bucket = NewPrefixedBucket(bucket, tenantID+"/phlaredb") // Inject the SSE config. - return NewSSEBucketClient(userID, bucket, cfgProvider) + return NewSSEBucketClient(tenantID, bucket, cfgProvider) } diff --git a/pkg/phlaredb/block/global_markers_bucket_client_test.go b/pkg/phlaredb/block/global_markers_bucket_client_test.go index 763da95efa..d6212fdf63 100644 --- a/pkg/phlaredb/block/global_markers_bucket_client_test.go +++ b/pkg/phlaredb/block/global_markers_bucket_client_test.go @@ -274,7 +274,7 @@ func TestBucketWithGlobalMarkers_ShouldWorkCorrectlyWithBucketMetrics(t *testing // user prefix. bkt, _ := objstore_testutil.NewFilesystemBucket(t, ctx, t.TempDir()) bkt = BucketWithGlobalMarkers(bkt) - userBkt := objstore.NewUserBucketClient("user-1", bkt, nil) + userBkt := objstore.NewTenantBucketClient("user-1", bkt, nil) reader, err := userBkt.Get(ctx, "does-not-exist") require.Error(t, err) diff --git a/pkg/phlaredb/block/testutil/mock_block.go b/pkg/phlaredb/block/testutil/mock_block.go index 3919ad5c6e..b04b07343c 100644 --- a/pkg/phlaredb/block/testutil/mock_block.go +++ b/pkg/phlaredb/block/testutil/mock_block.go @@ -48,11 +48,11 @@ func MockStorageBlockWithExtLabels(t testing.TB, bucket objstore.Bucket, userID require.NoError(t, err, "failed to marshal mocked block meta") metaContentReader := strings.NewReader(string(metaContent)) - metaPath := fmt.Sprintf("%s/%s/meta.json", userID, id.String()) + metaPath := fmt.Sprintf("%s/phlaredb/%s/meta.json", userID, id.String()) require.NoError(t, bucket.Upload(context.Background(), metaPath, metaContentReader)) // Upload an empty index, just to make sure the meta.json is not the only object in the block location. - indexPath := fmt.Sprintf("%s/%s/index", userID, id.String()) + indexPath := fmt.Sprintf("%s/phlaredb/%s/index", userID, id.String()) require.NoError(t, bucket.Upload(context.Background(), indexPath, strings.NewReader(""))) return meta @@ -69,7 +69,7 @@ func MockStorageDeletionMark(t testing.TB, bucket objstore.Bucket, userID string require.NoError(t, err, "failed to marshal mocked deletion mark") markContentReader := strings.NewReader(string(markContent)) - markPath := fmt.Sprintf("%s/%s/%s", userID, meta.ULID.String(), block.DeletionMarkFilename) + markPath := fmt.Sprintf("%s/phlaredb/%s/%s", userID, meta.ULID.String(), block.DeletionMarkFilename) require.NoError(t, bucket.Upload(context.Background(), markPath, markContentReader)) return &mark @@ -88,7 +88,7 @@ func MockNoCompactMark(t testing.TB, bucket objstore.Bucket, userID string, meta require.NoError(t, err, "failed to marshal mocked no-compact mark") markContentReader := strings.NewReader(string(markContent)) - markPath := fmt.Sprintf("%s/%s/%s", userID, meta.ULID.String(), block.NoCompactMarkFilename) + markPath := fmt.Sprintf("%s/phlaredb/%s/%s", userID, meta.ULID.String(), block.NoCompactMarkFilename) require.NoError(t, bucket.Upload(context.Background(), markPath, markContentReader)) return &mark diff --git a/pkg/phlaredb/bucket/tenant_deletion_mark.go b/pkg/phlaredb/bucket/tenant_deletion_mark.go index 635b74cf15..086ac688dc 100644 --- a/pkg/phlaredb/bucket/tenant_deletion_mark.go +++ b/pkg/phlaredb/bucket/tenant_deletion_mark.go @@ -36,14 +36,14 @@ func NewTenantDeletionMark(deletionTime time.Time) *TenantDeletionMark { // Checks for deletion mark for tenant. Errors other than "object not found" are returned. func TenantDeletionMarkExists(ctx context.Context, bkt objstore.BucketReader, userID string) (bool, error) { - markerFile := path.Join(userID, TenantDeletionMarkPath) + markerFile := path.Join(userID, "phlaredb/", TenantDeletionMarkPath) return bkt.Exists(ctx, markerFile) } // Uploads deletion mark to the tenant location in the bucket. func WriteTenantDeletionMark(ctx context.Context, bkt objstore.Bucket, userID string, cfgProvider objstore.TenantConfigProvider, mark *TenantDeletionMark) error { - bkt = objstore.NewUserBucketClient(userID, bkt, cfgProvider) + bkt = objstore.NewTenantBucketClient(userID, bkt, cfgProvider) data, err := json.Marshal(mark) if err != nil { @@ -55,7 +55,7 @@ func WriteTenantDeletionMark(ctx context.Context, bkt objstore.Bucket, userID st // Returns tenant deletion mark for given user, if it exists. If it doesn't exist, returns nil mark, and no error. func ReadTenantDeletionMark(ctx context.Context, bkt objstore.BucketReader, userID string) (*TenantDeletionMark, error) { - markerFile := path.Join(userID, TenantDeletionMarkPath) + markerFile := path.Join(userID, "phlaredb/", TenantDeletionMarkPath) r, err := bkt.Get(ctx, markerFile) if err != nil { diff --git a/pkg/phlaredb/bucket/tenant_deletion_mark_test.go b/pkg/phlaredb/bucket/tenant_deletion_mark_test.go index 5ee5d35574..f7c2cf9e4a 100644 --- a/pkg/phlaredb/bucket/tenant_deletion_mark_test.go +++ b/pkg/phlaredb/bucket/tenant_deletion_mark_test.go @@ -30,15 +30,15 @@ func TestTenantDeletionMarkExists(t *testing.T) { "mark doesn't exist": { objects: map[string][]byte{ - "user/01EQK4QKFHVSZYVJ908Y7HH9E0/meta.json": []byte("data"), + "user/phlaredb/01EQK4QKFHVSZYVJ908Y7HH9E0/meta.json": []byte("data"), }, exists: false, }, "mark exists": { objects: map[string][]byte{ - "user/01EQK4QKFHVSZYVJ908Y7HH9E0/meta.json": []byte("data"), - "user/" + TenantDeletionMarkPath: []byte("data"), + "user/phlaredb/01EQK4QKFHVSZYVJ908Y7HH9E0/meta.json": []byte("data"), + "user/phlaredb/" + TenantDeletionMarkPath: []byte("data"), }, exists: true, }, diff --git a/pkg/phlaredb/bucketindex/loader_test.go b/pkg/phlaredb/bucketindex/loader_test.go index 65eb8cc773..ea81d1f6cf 100644 --- a/pkg/phlaredb/bucketindex/loader_test.go +++ b/pkg/phlaredb/bucketindex/loader_test.go @@ -103,7 +103,7 @@ func TestLoader_GetIndex_ShouldCacheError(t *testing.T) { }) // Write a corrupted index. - require.NoError(t, bkt.Upload(ctx, path.Join("user-1", IndexCompressedFilename), strings.NewReader("invalid!}"))) + require.NoError(t, bkt.Upload(ctx, path.Join("user-1", "phlaredb/", IndexCompressedFilename), strings.NewReader("invalid!}"))) // Request the index multiple times. for i := 0; i < 10; i++ { @@ -236,7 +236,7 @@ func TestLoader_ShouldUpdateIndexInBackgroundOnPreviousLoadFailure(t *testing.T) bkt, _ := objstore_testutil.NewFilesystemBucket(t, ctx, t.TempDir()) // Write a corrupted index. - require.NoError(t, bkt.Upload(ctx, path.Join("user-1", IndexCompressedFilename), strings.NewReader("invalid!}"))) + require.NoError(t, bkt.Upload(ctx, path.Join("user-1", "phlaredb/", IndexCompressedFilename), strings.NewReader("invalid!}"))) // Create the loader. cfg := LoaderConfig{ @@ -374,7 +374,7 @@ func TestLoader_ShouldNotCacheCriticalErrorOnBackgroundUpdates(t *testing.T) { assert.Equal(t, idx, actualIdx) // Write a corrupted index. - require.NoError(t, bkt.Upload(ctx, path.Join("user-1", IndexCompressedFilename), strings.NewReader("invalid!}"))) + require.NoError(t, bkt.Upload(ctx, path.Join("user-1", "phlaredb/", IndexCompressedFilename), strings.NewReader("invalid!}"))) // Wait until the first failure has been tracked. test.Poll(t, 3*time.Second, true, func() interface{} { diff --git a/pkg/phlaredb/bucketindex/storage.go b/pkg/phlaredb/bucketindex/storage.go index 397f1b5770..4e7fd480fe 100644 --- a/pkg/phlaredb/bucketindex/storage.go +++ b/pkg/phlaredb/bucketindex/storage.go @@ -31,7 +31,7 @@ func ReadIndex(ctx context.Context, bkt objstore.Bucket, userID string, cfgProvi ctx, cancel := context.WithTimeout(ctx, time.Minute) defer cancel() - userBkt := objstore.NewUserBucketClient(userID, bkt, cfgProvider) + userBkt := objstore.NewTenantBucketClient(userID, bkt, cfgProvider) // Get the bucket index. reader, err := userBkt.WithExpectedErrs(userBkt.IsObjNotFoundErr).Get(ctx, IndexCompressedFilename) @@ -62,7 +62,7 @@ func ReadIndex(ctx context.Context, bkt objstore.Bucket, userID string, cfgProvi // WriteIndex uploads the provided index to the storage. func WriteIndex(ctx context.Context, bkt objstore.Bucket, userID string, cfgProvider objstore.TenantConfigProvider, idx *Index) error { - bkt = objstore.NewUserBucketClient(userID, bkt, cfgProvider) + bkt = objstore.NewTenantBucketClient(userID, bkt, cfgProvider) // Marshal the index. content, err := json.Marshal(idx) @@ -93,7 +93,7 @@ func WriteIndex(ctx context.Context, bkt objstore.Bucket, userID string, cfgProv // DeleteIndex deletes the bucket index from the storage. No error is returned if the index // does not exist. func DeleteIndex(ctx context.Context, bkt objstore.Bucket, userID string, cfgProvider objstore.TenantConfigProvider) error { - bkt = objstore.NewUserBucketClient(userID, bkt, cfgProvider) + bkt = objstore.NewTenantBucketClient(userID, bkt, cfgProvider) err := bkt.Delete(ctx, IndexCompressedFilename) if err != nil && !bkt.IsObjNotFoundErr(err) { diff --git a/pkg/phlaredb/bucketindex/storage_test.go b/pkg/phlaredb/bucketindex/storage_test.go index a04a618af1..915cd9526b 100644 --- a/pkg/phlaredb/bucketindex/storage_test.go +++ b/pkg/phlaredb/bucketindex/storage_test.go @@ -36,7 +36,7 @@ func TestReadIndex_ShouldReturnErrorIfIndexIsCorrupted(t *testing.T) { bkt, _ := objstore_testutil.NewFilesystemBucket(t, ctx, t.TempDir()) // Write a corrupted index. - require.NoError(t, bkt.Upload(ctx, path.Join(userID, IndexCompressedFilename), strings.NewReader("invalid!}"))) + require.NoError(t, bkt.Upload(ctx, path.Join(userID, "phlaredb/", IndexCompressedFilename), strings.NewReader("invalid!}"))) idx, err := ReadIndex(ctx, bkt, userID, nil, log.NewNopLogger()) require.Equal(t, ErrIndexCorrupted, err) diff --git a/pkg/phlaredb/bucketindex/updater.go b/pkg/phlaredb/bucketindex/updater.go index 36ef8586be..ff7f6797d0 100644 --- a/pkg/phlaredb/bucketindex/updater.go +++ b/pkg/phlaredb/bucketindex/updater.go @@ -37,7 +37,7 @@ type Updater struct { func NewUpdater(bkt objstore.Bucket, userID string, cfgProvider objstore.TenantConfigProvider, logger log.Logger) *Updater { return &Updater{ - bkt: objstore.NewUserBucketClient(userID, bkt, cfgProvider), + bkt: objstore.NewTenantBucketClient(userID, bkt, cfgProvider), logger: logger, } } diff --git a/pkg/phlaredb/bucketindex/updater_test.go b/pkg/phlaredb/bucketindex/updater_test.go index a73481695e..de0c4686b0 100644 --- a/pkg/phlaredb/bucketindex/updater_test.go +++ b/pkg/phlaredb/bucketindex/updater_test.go @@ -58,7 +58,7 @@ func TestUpdater_UpdateIndex(t *testing.T) { []*block.DeletionMark{block2Mark, block4Mark}) // Hard delete a block and update the index. - require.NoError(t, block.Delete(ctx, log.NewNopLogger(), objstore.NewUserBucketClient(userID, bkt, nil), block2.ULID)) + require.NoError(t, block.Delete(ctx, log.NewNopLogger(), objstore.NewTenantBucketClient(userID, bkt, nil), block2.ULID)) returnedIdx, _, err = w.UpdateIndex(ctx, returnedIdx) require.NoError(t, err) @@ -86,7 +86,7 @@ func TestUpdater_UpdateIndex_ShouldSkipPartialBlocks(t *testing.T) { block_testutil.MockNoCompactMark(t, bkt, userID, block3) // Delete a block's meta.json to simulate a partial block. - require.NoError(t, bkt.Delete(ctx, path.Join(userID, block3.ULID.String(), block.MetaFilename))) + require.NoError(t, bkt.Delete(ctx, path.Join(userID, "phlaredb/", block3.ULID.String(), block.MetaFilename))) w := NewUpdater(bkt, userID, nil, logger) idx, partials, err := w.UpdateIndex(ctx, nil) @@ -115,7 +115,7 @@ func TestUpdater_UpdateIndex_ShouldSkipBlocksWithCorruptedMeta(t *testing.T) { block2Mark := block_testutil.MockStorageDeletionMark(t, bkt, userID, block2) // Overwrite a block's meta.json with invalid data. - require.NoError(t, bkt.Upload(ctx, path.Join(userID, block3.ULID.String(), block.MetaFilename), bytes.NewReader([]byte("invalid!}")))) + require.NoError(t, bkt.Upload(ctx, path.Join(userID, "phlaredb/", block3.ULID.String(), block.MetaFilename), bytes.NewReader([]byte("invalid!}")))) w := NewUpdater(bkt, userID, nil, logger) idx, partials, err := w.UpdateIndex(ctx, nil) @@ -144,7 +144,7 @@ func TestUpdater_UpdateIndex_ShouldSkipCorruptedDeletionMarks(t *testing.T) { block2Mark := block_testutil.MockStorageDeletionMark(t, bkt, userID, block2) // Overwrite a block's deletion-mark.json with invalid data. - require.NoError(t, bkt.Upload(ctx, path.Join(userID, block2Mark.ID.String(), block.DeletionMarkFilename), bytes.NewReader([]byte("invalid!}")))) + require.NoError(t, bkt.Upload(ctx, path.Join(userID, "phlaredb/", block2Mark.ID.String(), block.DeletionMarkFilename), bytes.NewReader([]byte("invalid!}")))) w := NewUpdater(bkt, userID, nil, logger) idx, partials, err := w.UpdateIndex(ctx, nil) @@ -228,7 +228,7 @@ func TestUpdater_UpdateIndexFromVersion1ToVersion2(t *testing.T) { } func getBlockUploadedAt(t testing.TB, bkt objstore.Bucket, userID string, blockID ulid.ULID) int64 { - metaFile := path.Join(userID, blockID.String(), block.MetaFilename) + metaFile := path.Join(userID, "phlaredb/", blockID.String(), block.MetaFilename) attrs, err := bkt.Attributes(context.Background(), metaFile) require.NoError(t, err) diff --git a/pkg/storegateway/bucket.go b/pkg/storegateway/bucket.go index 8be24cbc55..d6513055a4 100644 --- a/pkg/storegateway/bucket.go +++ b/pkg/storegateway/bucket.go @@ -47,7 +47,7 @@ type BucketStore struct { func NewBucketStore(bucket phlareobj.Bucket, fetcher block.MetadataFetcher, tenantID string, syncDir string, logger log.Logger, Metrics *Metrics) (*BucketStore, error) { s := &BucketStore{ fetcher: fetcher, - bucket: phlareobj.NewPrefixedBucket(bucket, tenantID+"/phlaredb"), + bucket: phlareobj.NewTenantBucketClient(tenantID, bucket, nil), tenantID: tenantID, syncDir: syncDir, logger: logger, diff --git a/pkg/storegateway/bucket_index_metadata_fetcher_test.go b/pkg/storegateway/bucket_index_metadata_fetcher_test.go index 041f885a2c..592a98d47f 100644 --- a/pkg/storegateway/bucket_index_metadata_fetcher_test.go +++ b/pkg/storegateway/bucket_index_metadata_fetcher_test.go @@ -57,7 +57,7 @@ func TestBucketIndexMetadataFetcher_Fetch(t *testing.T) { // Create a metadata fetcher with filters. filters := []block.MetadataFilter{ - NewIgnoreDeletionMarkFilter(logger, objstore.NewUserBucketClient(userID, bkt, nil), 2*time.Hour, 1), + NewIgnoreDeletionMarkFilter(logger, objstore.NewTenantBucketClient(userID, bkt, nil), 2*time.Hour, 1), newMinTimeMetaFilter(1 * time.Hour), } @@ -158,7 +158,7 @@ func TestBucketIndexMetadataFetcher_Fetch_CorruptedBucketIndex(t *testing.T) { logger := log.NewLogfmtLogger(logs) // Upload a corrupted bucket index. - require.NoError(t, bkt.Upload(ctx, path.Join(userID, bucketindex.IndexCompressedFilename), strings.NewReader("invalid}!"))) + require.NoError(t, bkt.Upload(ctx, path.Join(userID, "phlaredb/", bucketindex.IndexCompressedFilename), strings.NewReader("invalid}!"))) fetcher := NewBucketIndexMetadataFetcher(userID, bkt, nil, logger, reg, nil) metas, partials, err := fetcher.Fetch(ctx) From ac099cae7dfee531bd0f90bb2d4f3e079475a328 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Thu, 5 Oct 2023 17:27:26 +0200 Subject: [PATCH 56/74] Fixes store-gateway local cache. --- .../global_markers_bucket_client_test.go | 24 +++++++++++++++++++ pkg/storegateway/block.go | 20 ++++++++++++---- pkg/storegateway/bucket.go | 10 +------- pkg/storegateway/gateway_blocks_http.go | 4 +++- 4 files changed, 44 insertions(+), 14 deletions(-) diff --git a/pkg/phlaredb/block/global_markers_bucket_client_test.go b/pkg/phlaredb/block/global_markers_bucket_client_test.go index d6212fdf63..b836fa2984 100644 --- a/pkg/phlaredb/block/global_markers_bucket_client_test.go +++ b/pkg/phlaredb/block/global_markers_bucket_client_test.go @@ -8,10 +8,12 @@ package block import ( "bytes" "context" + "os" "path" "strings" "testing" + "github.com/go-kit/log" "github.com/oklog/ulid" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/testutil" @@ -336,3 +338,25 @@ func TestBucketWithGlobalMarkers_ShouldWorkCorrectlyWithBucketMetrics(t *testing "objstore_bucket_operation_failures_total", )) } + +func TestPhlareDBGlobalMarker(t *testing.T) { + // Create a mocked block deletion mark in the global location. + bkt, _ := objstore_testutil.NewFilesystemBucket(t, context.Background(), t.TempDir()) + bkt = BucketWithGlobalMarkers(bkt) + + bkt = objstore.NewTenantBucketClient("foo-1", bkt, nil) + + id := generateULID() + + err := MarkForDeletion(context.Background(), log.NewLogfmtLogger(os.Stderr), bkt, id, "foo", prometheus.NewCounter(prometheus.CounterOpts{})) + require.NoError(t, err) + + ok, err := bkt.Exists(context.Background(), DeletionMarkFilepath(id)) + + require.NoError(t, err) + require.True(t, ok) + + ok, err = bkt.Exists(context.Background(), path.Join(id.String(), DeletionMarkFilename)) + require.NoError(t, err) + require.True(t, ok) +} diff --git a/pkg/storegateway/block.go b/pkg/storegateway/block.go index 468defd2ff..c2f96feebd 100644 --- a/pkg/storegateway/block.go +++ b/pkg/storegateway/block.go @@ -3,6 +3,7 @@ package storegateway import ( "context" "os" + "path" "path/filepath" "github.com/go-kit/log" @@ -33,18 +34,29 @@ func (bs *BucketStore) createBlock(ctx context.Context, meta *block.Meta) (*Bloc } metaPath := filepath.Join(blockLocalPath, block.MetaFilename) if _, err := os.Stat(metaPath); errors.Is(err, os.ErrNotExist) { + // fetch the meta from the bucket + r, err := bs.bucket.Get(ctx, path.Join(meta.ULID.String(), block.MetaFilename)) + if err != nil { + return nil, errors.Wrap(err, "get meta") + } + meta, err := block.Read(r) + if err != nil { + return nil, errors.Wrap(err, "read meta") + } // add meta.json if it does not exist if _, err := meta.WriteToFile(bs.logger, blockLocalPath); err != nil { return nil, errors.Wrap(err, "write meta.json") } } else { // read meta.json if it exists and validate it - if diskMeta, _, err := block.MetaFromDir(blockLocalPath); err != nil { - if meta.String() != diskMeta.String() { - return nil, errors.Wrap(err, "meta.json does not match") - } + diskMeta, _, err := block.MetaFromDir(blockLocalPath) + if err != nil { return nil, errors.Wrap(err, "read meta.json") } + if meta.ULID.String() != diskMeta.ULID.String() { + return nil, errors.Wrap(err, "meta.json does not match") + } + meta = diskMeta } return &Block{ diff --git a/pkg/storegateway/bucket.go b/pkg/storegateway/bucket.go index d6513055a4..ef3e60fd75 100644 --- a/pkg/storegateway/bucket.go +++ b/pkg/storegateway/bucket.go @@ -176,15 +176,7 @@ func (bs *BucketStore) addBlock(ctx context.Context, meta *block.Meta) (err erro b, err := func() (*Block, error) { bs.blocksMx.Lock() defer bs.blocksMx.Unlock() - // fetch the meta from the bucket - r, err := bs.bucket.Get(ctx, path.Join(meta.ULID.String(), block.MetaFilename)) - if err != nil { - return nil, errors.Wrap(err, "get meta") - } - meta, err := block.Read(r) - if err != nil { - return nil, errors.Wrap(err, "read meta") - } + b, err := bs.createBlock(ctx, meta) if err != nil { return nil, errors.Wrap(err, "load block from disk") diff --git a/pkg/storegateway/gateway_blocks_http.go b/pkg/storegateway/gateway_blocks_http.go index 86ac54e61f..a5f105d02c 100644 --- a/pkg/storegateway/gateway_blocks_http.go +++ b/pkg/storegateway/gateway_blocks_http.go @@ -13,6 +13,7 @@ import ( "github.com/dustin/go-humanize" "github.com/gorilla/mux" + "github.com/prometheus/prometheus/model/labels" "github.com/grafana/pyroscope/pkg/phlaredb/block" "github.com/grafana/pyroscope/pkg/util" @@ -99,7 +100,7 @@ func (s *StoreGateway) BlocksHandler(w http.ResponseWriter, req *http.Request) { } var sources []string for _, pb := range m.Compaction.Sources { - sources = append(parents, pb.String()) + sources = append(sources, pb.String()) } var blockSplitID *uint32 if splitCount > 0 { @@ -124,6 +125,7 @@ func (s *StoreGateway) BlocksHandler(w http.ResponseWriter, req *http.Request) { Stats: m.Stats, Sources: sources, Parents: parents, + Labels: labels.FromMap(m.Labels).String(), }) richMetas = append(richMetas, richMeta{ From 9860bbffde15ff37b190251a8d2eb6f4a4e5f3c0 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Fri, 6 Oct 2023 09:12:17 +0200 Subject: [PATCH 57/74] Fixes store-gateway stale meta --- pkg/phlaredb/block_querier.go | 2 +- pkg/storegateway/block.go | 14 +++++++++++--- 2 files changed, 12 insertions(+), 4 deletions(-) diff --git a/pkg/phlaredb/block_querier.go b/pkg/phlaredb/block_querier.go index 30672dc1b5..19f9826f0e 100644 --- a/pkg/phlaredb/block_querier.go +++ b/pkg/phlaredb/block_querier.go @@ -1001,7 +1001,7 @@ func (q *singleBlockQuerier) openFiles(ctx context.Context) error { case block.MetaVersion3: q.symbols, err = symdb.Open(ctx, q.bucket, q.meta) default: - panic(fmt.Errorf("unsupported block version %d", q.meta.Version)) + panic(fmt.Errorf("unsupported block version %d id %s", q.meta.Version, q.meta.ULID.String())) } return err })) diff --git a/pkg/storegateway/block.go b/pkg/storegateway/block.go index c2f96feebd..c7279b3dad 100644 --- a/pkg/storegateway/block.go +++ b/pkg/storegateway/block.go @@ -33,6 +33,7 @@ func (bs *BucketStore) createBlock(ctx context.Context, meta *block.Meta) (*Bloc } } metaPath := filepath.Join(blockLocalPath, block.MetaFilename) + var outMeta *block.Meta if _, err := os.Stat(metaPath); errors.Is(err, os.ErrNotExist) { // fetch the meta from the bucket r, err := bs.bucket.Get(ctx, path.Join(meta.ULID.String(), block.MetaFilename)) @@ -47,21 +48,28 @@ func (bs *BucketStore) createBlock(ctx context.Context, meta *block.Meta) (*Bloc if _, err := meta.WriteToFile(bs.logger, blockLocalPath); err != nil { return nil, errors.Wrap(err, "write meta.json") } + outMeta = meta.Clone() } else { // read meta.json if it exists and validate it diskMeta, _, err := block.MetaFromDir(blockLocalPath) if err != nil { return nil, errors.Wrap(err, "read meta.json") } + if meta.ULID.String() != diskMeta.ULID.String() { return nil, errors.Wrap(err, "meta.json does not match") } - meta = diskMeta + outMeta = diskMeta.Clone() + + } + + if outMeta.Version == 0 || len(outMeta.Files) == 0 { + return nil, errors.New("meta.json is empty") } return &Block{ - meta: meta, + meta: outMeta, logger: bs.logger, - BlockCloser: phlaredb.NewSingleBlockQuerierFromMeta(ctx, bs.bucket, meta), + BlockCloser: phlaredb.NewSingleBlockQuerierFromMeta(ctx, bs.bucket, outMeta), }, nil } From 4a1aa95c4a71cc14980358bb37916d5bcdc5dee4 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Fri, 6 Oct 2023 16:41:52 +0200 Subject: [PATCH 58/74] Add compaction metrics --- pkg/compactor/bucket_compactor.go | 95 ++++++++++++++++++++++ pkg/compactor/bucket_compactor_e2e_test.go | 1 + pkg/compactor/split_merge_compactor.go | 1 + 3 files changed, 97 insertions(+) diff --git a/pkg/compactor/bucket_compactor.go b/pkg/compactor/bucket_compactor.go index 9f03641998..203ae052fb 100644 --- a/pkg/compactor/bucket_compactor.go +++ b/pkg/compactor/bucket_compactor.go @@ -228,6 +228,73 @@ type Compactor interface { type BlockCompactor struct { blockOpenConcurrency int logger log.Logger + metrics *CompactorMetrics +} + +type CompactorMetrics struct { + Ran *prometheus.CounterVec + InProgress *prometheus.GaugeVec + OverlappingBlocks prometheus.Counter + Duration *prometheus.HistogramVec + Size *prometheus.HistogramVec + Samples *prometheus.HistogramVec + Range *prometheus.HistogramVec + Split *prometheus.HistogramVec +} + +func newCompactorMetrics(r prometheus.Registerer) *CompactorMetrics { + m := &CompactorMetrics{} + + m.Ran = prometheus.NewCounterVec(prometheus.CounterOpts{ + Name: "pyroscope_compactions_total", + Help: "Total number of compactions that were executed per level.", + }, []string{"level"}) + m.InProgress = prometheus.NewGaugeVec(prometheus.GaugeOpts{ + Name: "pyroscope_compactions_current", + Help: "The amount of compaction in progress per level", + }, []string{"level"}) + m.OverlappingBlocks = prometheus.NewCounter(prometheus.CounterOpts{ + Name: "pyroscope_vertical_compactions_total", + Help: "Total number of compactions done on overlapping blocks.", + }) + m.Duration = prometheus.NewHistogramVec(prometheus.HistogramOpts{ + Name: "pyroscope_compaction_duration_seconds", + Help: "Duration of compaction runs", + Buckets: prometheus.ExponentialBuckets(1, 2, 14), + }, []string{"level"}) + m.Size = prometheus.NewHistogramVec(prometheus.HistogramOpts{ + Name: "pyroscope_compaction_size_bytes", + Help: "Final block size after compaction by level", + Buckets: prometheus.ExponentialBuckets(32, 1.5, 12), + }, []string{"level"}) + m.Samples = prometheus.NewHistogramVec(prometheus.HistogramOpts{ + Name: "pyroscope_compaction_samples", + Help: "Final number of samples after compaction by level", + Buckets: prometheus.ExponentialBuckets(4, 1.5, 12), + }, []string{"level"}) + m.Range = prometheus.NewHistogramVec(prometheus.HistogramOpts{ + Name: "pyroscope_compaction_range_seconds", + Help: "Final time range after compaction by level.", + Buckets: prometheus.ExponentialBuckets(100, 4, 10), + }, []string{"level"}) + m.Split = prometheus.NewHistogramVec(prometheus.HistogramOpts{ + Name: "pyroscope_compaction_splits", + Help: "Compaction split factor by level.", + Buckets: []float64{1, 2, 4, 8, 16, 32, 64}, + }, []string{"level"}) + + if r != nil { + r.MustRegister( + m.Ran, + m.InProgress, + m.OverlappingBlocks, + m.Duration, + m.Range, + m.Samples, + m.Size, + ) + } + return m } func (c *BlockCompactor) CompactWithSplitting(ctx context.Context, dest string, dirs []string, shardCount uint64) ([]ulid.ULID, error) { @@ -258,6 +325,25 @@ func (c *BlockCompactor) CompactWithSplitting(ctx context.Context, dest string, } } }() + currentLevel := 0 + for _, r := range readers { + lvl := r.Meta().Compaction.Level + if lvl > currentLevel { + currentLevel = lvl + } + } + currentLevel++ + + start := time.Now() + defer func() { + c.metrics.Duration.WithLabelValues(fmt.Sprintf("%d", currentLevel)).Observe(time.Since(start).Seconds()) + c.metrics.InProgress.WithLabelValues(fmt.Sprintf("%d", currentLevel)).Dec() + }() + c.metrics.InProgress.WithLabelValues(fmt.Sprintf("%d", currentLevel)).Inc() + c.metrics.Ran.WithLabelValues(fmt.Sprintf("%d", currentLevel)).Inc() + c.metrics.Split.WithLabelValues(fmt.Sprintf("%d", currentLevel)).Observe(float64(shardCount)) + + // Open all blocks err = concurrency.ForEachJob(ctx, len(readers), c.blockOpenConcurrency, func(ctx context.Context, idx int) error { dir := dirs[idx] meta, err := block.ReadMetaFromDir(dir) @@ -278,6 +364,15 @@ func (c *BlockCompactor) CompactWithSplitting(ctx context.Context, dest string, if err != nil { return nil, errors.Wrapf(err, "compact blocks %v", dirs) } + for _, m := range metas { + c.metrics.Range.WithLabelValues(fmt.Sprintf("%d", currentLevel)).Observe(float64(m.MaxTime-m.MinTime) / 1000) + c.metrics.Samples.WithLabelValues(fmt.Sprintf("%d", currentLevel)).Observe(float64(m.Stats.NumSamples)) + size := float64(0) + for _, f := range m.Files { + size += float64(f.SizeBytes) + } + c.metrics.Size.WithLabelValues(fmt.Sprintf("%d", currentLevel)).Observe(size) + } result := make([]ulid.ULID, len(metas)) for i := range metas { result[i] = metas[i].ULID diff --git a/pkg/compactor/bucket_compactor_e2e_test.go b/pkg/compactor/bucket_compactor_e2e_test.go index 683a87da54..3e0c74c3cb 100644 --- a/pkg/compactor/bucket_compactor_e2e_test.go +++ b/pkg/compactor/bucket_compactor_e2e_test.go @@ -230,6 +230,7 @@ func TestGroupCompactE2E(t *testing.T) { bComp, err := NewBucketCompactor(logger, sy, grouper, planner, &BlockCompactor{ blockOpenConcurrency: 100, logger: logger, + metrics: newCompactorMetrics(nil), }, dir, userbkt, 2, ownAllJobs, sortJobsByNewestBlocksFirst, 0, 4, metrics) require.NoError(t, err) diff --git a/pkg/compactor/split_merge_compactor.go b/pkg/compactor/split_merge_compactor.go index 3975f36def..ffe58d2d2f 100644 --- a/pkg/compactor/split_merge_compactor.go +++ b/pkg/compactor/split_merge_compactor.go @@ -24,6 +24,7 @@ func splitAndMergeCompactorFactory(ctx context.Context, cfg Config, logger log.L return &BlockCompactor{ blockOpenConcurrency: cfg.MaxOpeningBlocksConcurrency, logger: logger, + metrics: newCompactorMetrics(reg), }, NewSplitAndMergePlanner(cfg.BlockRanges.ToMilliseconds()), nil } From 4a1ccf773be5d1668b6daefa850985ac5348d355 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Mon, 9 Oct 2023 10:55:25 +0200 Subject: [PATCH 59/74] Fixes panic in compaction metrics --- pkg/compactor/bucket_compactor.go | 35 ++++++++++++++++--------------- 1 file changed, 18 insertions(+), 17 deletions(-) diff --git a/pkg/compactor/bucket_compactor.go b/pkg/compactor/bucket_compactor.go index 203ae052fb..9a1450ac62 100644 --- a/pkg/compactor/bucket_compactor.go +++ b/pkg/compactor/bucket_compactor.go @@ -325,23 +325,6 @@ func (c *BlockCompactor) CompactWithSplitting(ctx context.Context, dest string, } } }() - currentLevel := 0 - for _, r := range readers { - lvl := r.Meta().Compaction.Level - if lvl > currentLevel { - currentLevel = lvl - } - } - currentLevel++ - - start := time.Now() - defer func() { - c.metrics.Duration.WithLabelValues(fmt.Sprintf("%d", currentLevel)).Observe(time.Since(start).Seconds()) - c.metrics.InProgress.WithLabelValues(fmt.Sprintf("%d", currentLevel)).Dec() - }() - c.metrics.InProgress.WithLabelValues(fmt.Sprintf("%d", currentLevel)).Inc() - c.metrics.Ran.WithLabelValues(fmt.Sprintf("%d", currentLevel)).Inc() - c.metrics.Split.WithLabelValues(fmt.Sprintf("%d", currentLevel)).Observe(float64(shardCount)) // Open all blocks err = concurrency.ForEachJob(ctx, len(readers), c.blockOpenConcurrency, func(ctx context.Context, idx int) error { @@ -360,6 +343,24 @@ func (c *BlockCompactor) CompactWithSplitting(ctx context.Context, dest string, if err != nil { return nil, err } + currentLevel := 0 + for _, r := range readers { + lvl := r.Meta().Compaction.Level + if lvl > currentLevel { + currentLevel = lvl + } + } + currentLevel++ + + start := time.Now() + defer func() { + c.metrics.Duration.WithLabelValues(fmt.Sprintf("%d", currentLevel)).Observe(time.Since(start).Seconds()) + c.metrics.InProgress.WithLabelValues(fmt.Sprintf("%d", currentLevel)).Dec() + }() + c.metrics.InProgress.WithLabelValues(fmt.Sprintf("%d", currentLevel)).Inc() + c.metrics.Ran.WithLabelValues(fmt.Sprintf("%d", currentLevel)).Inc() + c.metrics.Split.WithLabelValues(fmt.Sprintf("%d", currentLevel)).Observe(float64(shardCount)) + metas, err := phlaredb.CompactWithSplitting(ctx, readers, shardCount, dest) if err != nil { return nil, errors.Wrapf(err, "compact blocks %v", dirs) From 4576fed6e5281ad919fc36e774cdb6e02c005836 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Mon, 9 Oct 2023 10:58:45 +0200 Subject: [PATCH 60/74] Discard __hostname__ labels from block for compaction planning --- pkg/compactor/job.go | 2 +- pkg/compactor/split_merge_grouper.go | 14 +++++++++----- pkg/compactor/split_merge_job.go | 4 ++-- 3 files changed, 12 insertions(+), 8 deletions(-) diff --git a/pkg/compactor/job.go b/pkg/compactor/job.go index 2ac15690b9..cf6ef6a3bf 100644 --- a/pkg/compactor/job.go +++ b/pkg/compactor/job.go @@ -60,7 +60,7 @@ func (job *Job) Key() string { // AppendMeta the block with the given meta to the job. func (job *Job) AppendMeta(meta *block.Meta) error { - if !labels.Equal(job.labels, labels.FromMap(meta.Labels)) { + if !labels.Equal(labelsWithout(job.labels.Map(), block.HostnameLabel), labelsWithout(meta.Labels, block.HostnameLabel)) { return errors.New("block and group labels do not match") } if job.resolution != meta.Downsample.Resolution { diff --git a/pkg/compactor/split_merge_grouper.go b/pkg/compactor/split_merge_grouper.go index 4899dbb732..c90d841b15 100644 --- a/pkg/compactor/split_merge_grouper.go +++ b/pkg/compactor/split_merge_grouper.go @@ -362,16 +362,20 @@ func getMaxTime(blocks []*block.Meta) int64 { // defaultGroupKeyWithoutShardID returns the default group key excluding ShardIDLabelName // when computing it. func defaultGroupKeyWithoutShardID(meta *block.Meta) string { - return defaultGroupKey(meta.Downsample.Resolution, labelsWithoutShard(meta.Labels)) + return defaultGroupKey(meta.Downsample.Resolution, labelsWithout(meta.Labels, sharding.CompactorShardIDLabel, block.HostnameLabel)) } -// Return labels built from base, but without any label with name equal to sharding.CompactorShardIDExternalLabel. -func labelsWithoutShard(base map[string]string) labels.Labels { +// labelsWithout returns a copy of the input labels without the given labels. +func labelsWithout(base map[string]string, without ...string) labels.Labels { b := labels.NewScratchBuilder(len(base)) +Outer: for k, v := range base { - if k != sharding.CompactorShardIDLabel { - b.Add(k, v) + for _, w := range without { + if k == w { + continue Outer + } } + b.Add(k, v) } b.Sort() return b.Labels() diff --git a/pkg/compactor/split_merge_job.go b/pkg/compactor/split_merge_job.go index ca98402c43..a5ff15218d 100644 --- a/pkg/compactor/split_merge_job.go +++ b/pkg/compactor/split_merge_job.go @@ -59,8 +59,8 @@ func (j *job) conflicts(other *job) bool { // are never merged together, so they can't conflict. Since all blocks within the same job are expected to have the same // downsample resolution and external labels, we just check the 1st block of each job. if len(j.blocks) > 0 && len(other.blocks) > 0 { - myLabels := labelsWithoutShard(j.blocksGroup.blocks[0].Labels) - otherLabels := labelsWithoutShard(other.blocksGroup.blocks[0].Labels) + myLabels := labelsWithout(j.blocksGroup.blocks[0].Labels, block.HostnameLabel, sharding.CompactorShardIDLabel) + otherLabels := labelsWithout(other.blocksGroup.blocks[0].Labels, block.HostnameLabel, sharding.CompactorShardIDLabel) if !labels.Equal(myLabels, otherLabels) { return false } From 7f99a2723906ead955e2124847bdea07f703a0ff Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Mon, 9 Oct 2023 10:59:46 +0200 Subject: [PATCH 61/74] Improves parquet profile file reader --- pkg/phlaredb/block_querier.go | 4 ++-- pkg/phlaredb/compact.go | 20 ++++++++++++++------ 2 files changed, 16 insertions(+), 8 deletions(-) diff --git a/pkg/phlaredb/block_querier.go b/pkg/phlaredb/block_querier.go index 196d42d85b..f6c7c0ca44 100644 --- a/pkg/phlaredb/block_querier.go +++ b/pkg/phlaredb/block_querier.go @@ -317,8 +317,8 @@ func NewSingleBlockQuerierFromMeta(phlarectx context.Context, bucketReader phlar return q } -func (b *singleBlockQuerier) Profiles() []parquet.RowGroup { - return b.profiles.file.RowGroups() +func (b *singleBlockQuerier) Profiles() parquet.Rows { + return parquet.NewReader(b.profiles.file.File) } func (b *singleBlockQuerier) Index() IndexReader { diff --git a/pkg/phlaredb/compact.go b/pkg/phlaredb/compact.go index daa5e58e4b..b6095e8aed 100644 --- a/pkg/phlaredb/compact.go +++ b/pkg/phlaredb/compact.go @@ -4,12 +4,15 @@ import ( "context" "crypto/rand" "fmt" + "io" "io/fs" "math" "os" "path/filepath" "sort" + "github.com/grafana/dskit/multierror" + "github.com/grafana/dskit/runutil" "github.com/oklog/ulid" "github.com/parquet-go/parquet-go" "github.com/pkg/errors" @@ -17,9 +20,6 @@ import ( "github.com/prometheus/prometheus/storage" "github.com/prometheus/prometheus/tsdb" - "github.com/grafana/dskit/multierror" - "github.com/grafana/dskit/runutil" - "github.com/grafana/pyroscope/pkg/iter" phlaremodel "github.com/grafana/pyroscope/pkg/model" phlareparquet "github.com/grafana/pyroscope/pkg/parquet" @@ -34,7 +34,7 @@ import ( type BlockReader interface { Meta() block.Meta - Profiles() []parquet.RowGroup + Profiles() parquet.Rows Index() IndexReader Symbols() symdb.SymbolsReader Close() error @@ -443,6 +443,7 @@ type profileRow struct { type profileRowIterator struct { profiles iter.Iterator[parquet.Row] blockReader BlockReader + closer io.Closer index IndexReader allPostings index.Postings err error @@ -459,10 +460,11 @@ func newProfileRowIterator(s BlockReader) (*profileRowIterator, error) { return nil, err } // todo close once https://github.com/grafana/pyroscope/issues/2172 is done. - reader := parquet.MultiRowGroup(s.Profiles()...).Rows() + reader := s.Profiles() return &profileRowIterator{ profiles: phlareparquet.NewBufferedRowReaderIterator(reader, 1024), blockReader: s, + closer: reader, index: s.Index(), allPostings: allPostings, currentSeriesIdx: math.MaxUint32, @@ -513,7 +515,13 @@ func (p *profileRowIterator) Err() error { } func (p *profileRowIterator) Close() error { - return p.profiles.Close() + err := p.profiles.Close() + if p.closer != nil { + if err := p.closer.Close(); err != nil { + return err + } + } + return err } func newMergeRowProfileIterator(src []BlockReader) (iter.Iterator[profileRow], error) { From 33a9fc0eb02ed986582d0d6da42f3b106c10b3ff Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Mon, 9 Oct 2023 15:38:54 +0200 Subject: [PATCH 62/74] Fixes more broken tests --- pkg/compactor/bucket_compactor_e2e_test.go | 8 +++--- pkg/compactor/split_merge_compactor_test.go | 6 ++--- pkg/compactor/split_merge_grouper_test.go | 28 +++++++++++++++++++++ pkg/phlaredb/bucket/tenant_scanner_test.go | 12 ++++----- 4 files changed, 41 insertions(+), 13 deletions(-) diff --git a/pkg/compactor/bucket_compactor_e2e_test.go b/pkg/compactor/bucket_compactor_e2e_test.go index 3e0c74c3cb..e196a4cb9e 100644 --- a/pkg/compactor/bucket_compactor_e2e_test.go +++ b/pkg/compactor/bucket_compactor_e2e_test.go @@ -254,12 +254,12 @@ func TestGroupCompactE2E(t *testing.T) { m4 := createDBBlock(t, bkt, "user-1", 1001, 3000, 10, nil) require.NoError(t, bComp.Compact(ctx, 0), 0) - assert.Equal(t, 6.0, promtest.ToFloat64(sy.metrics.blocksMarkedForDeletion)) + assert.Equal(t, 5.0, promtest.ToFloat64(sy.metrics.blocksMarkedForDeletion)) assert.Equal(t, 0.0, promtest.ToFloat64(metrics.blocksMarkedForNoCompact)) assert.Equal(t, 0.0, promtest.ToFloat64(sy.metrics.garbageCollectionFailures)) - assert.Equal(t, 3.0, promtest.ToFloat64(metrics.groupCompactions)) - assert.Equal(t, 3.0, promtest.ToFloat64(metrics.groupCompactionRunsStarted)) - assert.Equal(t, 3.0, promtest.ToFloat64(metrics.groupCompactionRunsCompleted)) + assert.Equal(t, 2.0, promtest.ToFloat64(metrics.groupCompactions)) + assert.Equal(t, 2.0, promtest.ToFloat64(metrics.groupCompactionRunsStarted)) + assert.Equal(t, 2.0, promtest.ToFloat64(metrics.groupCompactionRunsCompleted)) assert.Equal(t, 0.0, promtest.ToFloat64(metrics.groupCompactionRunsFailed)) _, err = os.Stat(dir) diff --git a/pkg/compactor/split_merge_compactor_test.go b/pkg/compactor/split_merge_compactor_test.go index 26a24a0f4a..3ecb231bba 100644 --- a/pkg/compactor/split_merge_compactor_test.go +++ b/pkg/compactor/split_merge_compactor_test.go @@ -396,14 +396,14 @@ func TestMultitenantCompactor_ShouldSupportSplitAndMergeCompactor(t *testing.T) block2 := createDBBlock(t, bkt, userID, blockRangeMillis, 2*blockRangeMillis, numSeries, externalLabels("")) block3 := createDBBlock(t, bkt, userID, blockRangeMillis, 2*blockRangeMillis, numSeries, externalLabels("")) - // Two split adjacent blocks in the 2nd compaction range. + // // Two split adjacent blocks in the 2nd compaction range. block4a := createDBBlock(t, bkt, userID, 2*blockRangeMillis, 3*blockRangeMillis, numSeries, externalLabels("1_of_2")) block4b := createDBBlock(t, bkt, userID, 2*blockRangeMillis, 3*blockRangeMillis, numSeries, externalLabels("2_of_2")) block5a := createDBBlock(t, bkt, userID, 3*blockRangeMillis, 4*blockRangeMillis, numSeries, externalLabels("1_of_2")) block5b := createDBBlock(t, bkt, userID, 3*blockRangeMillis, 4*blockRangeMillis, numSeries, externalLabels("2_of_2")) // Two non-adjacent non-split blocks in the 1st compaction range. - block6 := createDBBlock(t, bkt, userID, 4*blockRangeMillis, 5*blockRangeMillis, numSeries, externalLabels("")) + block6 := createDBBlock(t, bkt, userID, 4*blockRangeMillis+1, 5*blockRangeMillis, numSeries, externalLabels("")) block7 := createDBBlock(t, bkt, userID, 7*blockRangeMillis, 8*blockRangeMillis, numSeries, externalLabels("")) return []block.Meta{ @@ -442,7 +442,7 @@ func TestMultitenantCompactor_ShouldSupportSplitAndMergeCompactor(t *testing.T) }, // The two non-adjacent blocks block6 and block7 are merged together in the 3rd range. { - MinTime: model.Time(4 * blockRangeMillis), + MinTime: model.Time(4*blockRangeMillis) + 1, MaxTime: model.Time(8 * blockRangeMillis), Compaction: tsdb.BlockMetaCompaction{ Sources: []ulid.ULID{block6, block7}, diff --git a/pkg/compactor/split_merge_grouper_test.go b/pkg/compactor/split_merge_grouper_test.go index cf34b2e12b..96e8586586 100644 --- a/pkg/compactor/split_merge_grouper_test.go +++ b/pkg/compactor/split_merge_grouper_test.go @@ -7,8 +7,10 @@ package compactor import ( "testing" + "time" "github.com/oklog/ulid" + "github.com/prometheus/common/model" "github.com/stretchr/testify/assert" "github.com/grafana/pyroscope/pkg/phlaredb/block" @@ -693,6 +695,7 @@ func TestGroupBlocksByShardID(t *testing.T) { } func TestGroupBlocksByRange(t *testing.T) { + blockRange := 2 * time.Hour.Milliseconds() tests := map[string]struct { timeRange int64 blocks []*block.Meta @@ -787,6 +790,31 @@ func TestGroupBlocksByRange(t *testing.T) { }}, }, }, + "2 different range": { + timeRange: 4 * blockRange, + blocks: []*block.Meta{ + {MinTime: model.Time(blockRange), MaxTime: model.Time(2 * blockRange)}, + {MinTime: model.Time(blockRange), MaxTime: model.Time(2 * blockRange)}, + {MinTime: model.Time(4*blockRange) + 1, MaxTime: model.Time(5 * blockRange)}, + {MinTime: model.Time(7 * blockRange), MaxTime: model.Time(8 * blockRange)}, + }, + expected: []blocksGroup{ + { + rangeStart: 0, rangeEnd: 4 * blockRange, + blocks: []*block.Meta{ + {MinTime: model.Time(blockRange), MaxTime: model.Time(2 * blockRange)}, + {MinTime: model.Time(blockRange), MaxTime: model.Time(2 * blockRange)}, + }, + }, + { + rangeStart: 4 * blockRange, rangeEnd: 8 * blockRange, + blocks: []*block.Meta{ + {MinTime: model.Time(4*blockRange) + 1, MaxTime: model.Time(5 * blockRange)}, + {MinTime: model.Time(7 * blockRange), MaxTime: model.Time(8 * blockRange)}, + }, + }, + }, + }, } for testName, testData := range tests { diff --git a/pkg/phlaredb/bucket/tenant_scanner_test.go b/pkg/phlaredb/bucket/tenant_scanner_test.go index ffc41d657e..dac967af96 100644 --- a/pkg/phlaredb/bucket/tenant_scanner_test.go +++ b/pkg/phlaredb/bucket/tenant_scanner_test.go @@ -21,8 +21,8 @@ import ( func TestUsersScanner_ScanUsers_ShouldReturnedOwnedUsersOnly(t *testing.T) { bucketClient := &objstore.ClientMock{} bucketClient.MockIter("", []string{"user-1", "user-2", "user-3", "user-4"}, nil) - bucketClient.MockExists(path.Join("user-1", TenantDeletionMarkPath), false, nil) - bucketClient.MockExists(path.Join("user-3", TenantDeletionMarkPath), true, nil) + bucketClient.MockExists(path.Join("user-1", "phlaredb/", TenantDeletionMarkPath), false, nil) + bucketClient.MockExists(path.Join("user-3", "phlaredb/", TenantDeletionMarkPath), true, nil) isOwned := func(userID string) (bool, error) { return userID == "user-1" || userID == "user-3", nil @@ -40,8 +40,8 @@ func TestUsersScanner_ScanUsers_ShouldReturnUsersForWhichOwnerCheckOrTenantDelet bucketClient := &objstore.ClientMock{} bucketClient.MockIter("", expected, nil) - bucketClient.MockExists(path.Join("user-1", TenantDeletionMarkPath), false, nil) - bucketClient.MockExists(path.Join("user-2", TenantDeletionMarkPath), false, errors.New("fail")) + bucketClient.MockExists(path.Join("user-1", "phlaredb/", TenantDeletionMarkPath), false, nil) + bucketClient.MockExists(path.Join("user-2", "phlaredb/", TenantDeletionMarkPath), false, errors.New("fail")) isOwned := func(userID string) (bool, error) { return false, errors.New("failed to check if user is owned") @@ -57,8 +57,8 @@ func TestUsersScanner_ScanUsers_ShouldReturnUsersForWhichOwnerCheckOrTenantDelet func TestUsersScanner_ScanUsers_ShouldNotReturnPrefixedUsedByPyroscopeInternals(t *testing.T) { bucketClient := &objstore.ClientMock{} bucketClient.MockIter("", []string{"user-1", "user-2", PyroscopeInternalsPrefix}, nil) - bucketClient.MockExists(path.Join("user-1", TenantDeletionMarkPath), false, nil) - bucketClient.MockExists(path.Join("user-2", TenantDeletionMarkPath), false, nil) + bucketClient.MockExists(path.Join("user-1", "phlaredb/", TenantDeletionMarkPath), false, nil) + bucketClient.MockExists(path.Join("user-2", "phlaredb/", TenantDeletionMarkPath), false, nil) s := NewTenantsScanner(bucketClient, AllTenants, log.NewNopLogger()) actual, _, err := s.ScanTenants(context.Background()) From d0a13a3ccd962798a21fa3fe101c35df5bfd2065 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Mon, 16 Oct 2023 12:18:36 +0200 Subject: [PATCH 63/74] Makes configurable the split by compaction --- cmd/pyroscope/help-all.txt.tmpl | 2 ++ pkg/compactor/bucket_compactor.go | 21 ++++++++++++++++- pkg/compactor/bucket_compactor_e2e_test.go | 2 ++ pkg/compactor/compactor.go | 14 +++++++---- pkg/compactor/split_merge_compactor.go | 6 +++++ pkg/phlaredb/compact.go | 27 +++++++++++++--------- pkg/phlaredb/compact_test.go | 2 +- 7 files changed, 57 insertions(+), 17 deletions(-) diff --git a/cmd/pyroscope/help-all.txt.tmpl b/cmd/pyroscope/help-all.txt.tmpl index 9d7b42922a..578a26925f 100644 --- a/cmd/pyroscope/help-all.txt.tmpl +++ b/cmd/pyroscope/help-all.txt.tmpl @@ -33,6 +33,8 @@ Usage of ./pyroscope: The sorting to use when deciding which compaction jobs should run first for a given tenant. Supported values are: smallest-range-oldest-blocks-first, newest-blocks-first. (default "smallest-range-oldest-blocks-first") -compactor.compaction-retries int How many times to retry a failed compaction within a single compaction run. (default 3) + -compactor.compaction-split-by string + The strategy to use when splitting blocks during compaction. Supported values are: fingerprint, stacktracePartition. (default "fingerprint") -compactor.compactor-tenant-shard-size int Max number of compactors that can compact blocks for single tenant. 0 to disable the limit and use all compactors. -compactor.data-dir string diff --git a/pkg/compactor/bucket_compactor.go b/pkg/compactor/bucket_compactor.go index 9a1450ac62..5d8408c073 100644 --- a/pkg/compactor/bucket_compactor.go +++ b/pkg/compactor/bucket_compactor.go @@ -225,8 +225,27 @@ type Compactor interface { CompactWithSplitting(ctx context.Context, dst string, dirs []string, shardCount uint64) (result []ulid.ULID, _ error) } +const ( + CompactionSplitByFingerprint = "fingerprint" + CompactionSplitByStacktracePartition = "stacktracePartition" +) + +var CompactionSplitBys = []string{CompactionSplitByFingerprint, CompactionSplitByStacktracePartition} + +func getCompactionSplitBy(name string) phlaredb.SplitByFunc { + switch name { + case CompactionSplitByFingerprint: + return phlaredb.SplitByFingerprint + case CompactionSplitByStacktracePartition: + return phlaredb.SplitByStacktracePartition + default: + return nil + } +} + type BlockCompactor struct { blockOpenConcurrency int + splitBy phlaredb.SplitByFunc logger log.Logger metrics *CompactorMetrics } @@ -361,7 +380,7 @@ func (c *BlockCompactor) CompactWithSplitting(ctx context.Context, dest string, c.metrics.Ran.WithLabelValues(fmt.Sprintf("%d", currentLevel)).Inc() c.metrics.Split.WithLabelValues(fmt.Sprintf("%d", currentLevel)).Observe(float64(shardCount)) - metas, err := phlaredb.CompactWithSplitting(ctx, readers, shardCount, dest) + metas, err := phlaredb.CompactWithSplitting(ctx, readers, shardCount, dest, c.splitBy) if err != nil { return nil, errors.Wrapf(err, "compact blocks %v", dirs) } diff --git a/pkg/compactor/bucket_compactor_e2e_test.go b/pkg/compactor/bucket_compactor_e2e_test.go index e196a4cb9e..298d8833cf 100644 --- a/pkg/compactor/bucket_compactor_e2e_test.go +++ b/pkg/compactor/bucket_compactor_e2e_test.go @@ -29,6 +29,7 @@ import ( phlareobj "github.com/grafana/pyroscope/pkg/objstore" "github.com/grafana/pyroscope/pkg/objstore/providers/filesystem" + "github.com/grafana/pyroscope/pkg/phlaredb" "github.com/grafana/pyroscope/pkg/phlaredb/block" ) @@ -228,6 +229,7 @@ func TestGroupCompactE2E(t *testing.T) { grouper := NewSplitAndMergeGrouper("user-1", []int64{1000, 3000}, 0, 0, logger) metrics := NewBucketCompactorMetrics(blocksMarkedForDeletion, prometheus.NewPedanticRegistry()) bComp, err := NewBucketCompactor(logger, sy, grouper, planner, &BlockCompactor{ + splitBy: phlaredb.SplitByFingerprint, blockOpenConcurrency: 100, logger: logger, metrics: newCompactorMetrics(nil), diff --git a/pkg/compactor/compactor.go b/pkg/compactor/compactor.go index 44571cdf32..329acded9a 100644 --- a/pkg/compactor/compactor.go +++ b/pkg/compactor/compactor.go @@ -29,6 +29,7 @@ import ( "go.uber.org/atomic" "github.com/grafana/pyroscope/pkg/objstore" + "github.com/grafana/pyroscope/pkg/phlaredb" "github.com/grafana/pyroscope/pkg/phlaredb/block" "github.com/grafana/pyroscope/pkg/phlaredb/bucket" "github.com/grafana/pyroscope/pkg/tenant" @@ -52,6 +53,7 @@ const ( var ( errInvalidBlockRanges = "compactor block range periods should be divisible by the previous one, but %s is not divisible by %s" errInvalidCompactionOrder = fmt.Errorf("unsupported compaction order (supported values: %s)", strings.Join(CompactionOrders, ", ")) + errInvalidCompactionSplitBy = fmt.Errorf("unsupported compaction split by (supported values: %s)", strings.Join(CompactionSplitBys, ", ")) errInvalidMaxOpeningBlocksConcurrency = fmt.Errorf("invalid max-opening-blocks-concurrency value, must be positive") RingOp = ring.NewOp([]ring.InstanceState{ring.ACTIVE}, nil) ) @@ -102,6 +104,7 @@ type Config struct { ShardingRing RingConfig `yaml:"sharding_ring"` CompactionJobsOrder string `yaml:"compaction_jobs_order" category:"advanced"` + CompactionSplitBy string `yaml:"compaction_split_by" category:"advanced"` // No need to add options to customize the retry backoff, // given the defaults should be fine, but allow to override @@ -134,6 +137,7 @@ func (cfg *Config) RegisterFlags(f *flag.FlagSet, logger log.Logger) { f.DurationVar(&cfg.CleanupInterval, "compactor.cleanup-interval", 15*time.Minute, "How frequently compactor should run blocks cleanup and maintenance, as well as update the bucket index.") f.IntVar(&cfg.CleanupConcurrency, "compactor.cleanup-concurrency", 20, "Max number of tenants for which blocks cleanup and maintenance should run concurrently.") f.StringVar(&cfg.CompactionJobsOrder, "compactor.compaction-jobs-order", CompactionOrderOldestFirst, fmt.Sprintf("The sorting to use when deciding which compaction jobs should run first for a given tenant. Supported values are: %s.", strings.Join(CompactionOrders, ", "))) + f.StringVar(&cfg.CompactionSplitBy, "compactor.compaction-split-by", CompactionSplitByFingerprint, fmt.Sprintf("The strategy to use when splitting blocks during compaction. Supported values are: %s.", strings.Join(CompactionSplitBys, ", "))) f.DurationVar(&cfg.DeletionDelay, "compactor.deletion-delay", 12*time.Hour, "Time before a block marked for deletion is deleted from bucket. "+ "If not 0, blocks will be marked for deletion and compactor component will permanently delete blocks marked for deletion from the bucket. "+ "If 0, blocks will be deleted straight away. Note that deleting blocks immediately can cause query failures.") @@ -141,7 +145,6 @@ func (cfg *Config) RegisterFlags(f *flag.FlagSet, logger log.Logger) { f.BoolVar(&cfg.NoBlocksFileCleanupEnabled, "compactor.no-blocks-file-cleanup-enabled", false, "If enabled, will delete the bucket-index, markers and debug files in the tenant bucket when there are no blocks left in the index.") // compactor concurrency options f.IntVar(&cfg.MaxOpeningBlocksConcurrency, "compactor.max-opening-blocks-concurrency", 1, "Number of goroutines opening blocks before compaction.") - // f.IntVar(&cfg.MaxClosingBlocksConcurrency, "compactor.max-closing-blocks-concurrency", 1, "Max number of blocks that can be closed concurrently during split compaction. Note that closing of newly compacted block uses a lot of memory for writing index.") f.Var(&cfg.EnabledTenants, "compactor.enabled-tenants", "Comma separated list of tenants that can be compacted. If specified, only these tenants will be compacted by compactor, otherwise all tenants can be compacted. Subject to sharding.") f.Var(&cfg.DisabledTenants, "compactor.disabled-tenants", "Comma separated list of tenants that cannot be compacted by this compactor. If specified, and compactor would normally pick given tenant for compaction (via -compactor.enabled-tenants or sharding), it will be ignored instead.") @@ -158,13 +161,15 @@ func (cfg *Config) Validate() error { if cfg.MaxOpeningBlocksConcurrency < 1 { return errInvalidMaxOpeningBlocksConcurrency } - // if cfg.MaxClosingBlocksConcurrency < 1 { - // return errInvalidMaxClosingBlocksConcurrency - // } + if !util.StringsContain(CompactionOrders, cfg.CompactionJobsOrder) { return errInvalidCompactionOrder } + if !util.StringsContain(CompactionSplitBys, cfg.CompactionSplitBy) { + return errInvalidCompactionSplitBy + } + return nil } @@ -224,6 +229,7 @@ type MultitenantCompactor struct { shardingStrategy shardingStrategy jobsOrder JobsOrderFunc + splitBy phlaredb.SplitByFunc // Metrics. compactionRunsStarted prometheus.Counter diff --git a/pkg/compactor/split_merge_compactor.go b/pkg/compactor/split_merge_compactor.go index ffe58d2d2f..f3d99d72f0 100644 --- a/pkg/compactor/split_merge_compactor.go +++ b/pkg/compactor/split_merge_compactor.go @@ -21,8 +21,14 @@ func splitAndMergeGrouperFactory(_ context.Context, cfg Config, cfgProvider Conf } func splitAndMergeCompactorFactory(ctx context.Context, cfg Config, logger log.Logger, reg prometheus.Registerer) (Compactor, Planner, error) { + splitBy := getCompactionSplitBy(cfg.CompactionSplitBy) + if splitBy == nil { + return nil, nil, errInvalidCompactionSplitBy + } + return &BlockCompactor{ blockOpenConcurrency: cfg.MaxOpeningBlocksConcurrency, + splitBy: splitBy, logger: logger, metrics: newCompactorMetrics(reg), }, NewSplitAndMergePlanner(cfg.BlockRanges.ToMilliseconds()), nil diff --git a/pkg/phlaredb/compact.go b/pkg/phlaredb/compact.go index b6095e8aed..2fea057d7f 100644 --- a/pkg/phlaredb/compact.go +++ b/pkg/phlaredb/compact.go @@ -41,25 +41,24 @@ type BlockReader interface { } func Compact(ctx context.Context, src []BlockReader, dst string) (meta block.Meta, err error) { - metas, err := CompactWithSplitting(ctx, src, 1, dst) + metas, err := CompactWithSplitting(ctx, src, 1, dst, SplitByFingerprint) if err != nil { return block.Meta{}, err } return metas[0], nil } -func CompactWithSplitting(ctx context.Context, src []BlockReader, shardsCount uint64, dst string) ( +func CompactWithSplitting(ctx context.Context, src []BlockReader, splitCount uint64, dst string, splitBy SplitByFunc) ( []block.Meta, error, ) { - if shardsCount == 0 { - shardsCount = 1 + if splitCount == 0 { + splitCount = 1 } - if len(src) <= 1 && shardsCount == 1 { + if len(src) <= 1 && splitCount == 1 { return nil, errors.New("not enough blocks to compact") } var ( - writers = make([]*blockWriter, shardsCount) - shardBy = shardByFingerprint + writers = make([]*blockWriter, splitCount) srcMetas = make([]block.Meta, len(src)) err error ) @@ -74,11 +73,11 @@ func CompactWithSplitting(ctx context.Context, src []BlockReader, shardsCount ui for i := range writers { meta := outMeta.Clone() meta.ULID = ulid.MustNew(outBlocksTime, rand.Reader) - if shardsCount > 1 { + if splitCount > 1 { if meta.Labels == nil { meta.Labels = make(map[string]string) } - meta.Labels[sharding.CompactorShardIDLabel] = sharding.FormatShardIDLabelValue(uint64(i), shardsCount) + meta.Labels[sharding.CompactorShardIDLabel] = sharding.FormatShardIDLabelValue(uint64(i), splitCount) } writers[i], err = newBlockWriter(dst, meta) if err != nil { @@ -95,7 +94,7 @@ func CompactWithSplitting(ctx context.Context, src []BlockReader, shardsCount ui // iterate and splits the rows into series. for rowsIt.Next() { r := rowsIt.At() - shard := int(shardBy(r, shardsCount)) + shard := int(splitBy(r, splitCount)) if err := writers[shard].WriteRow(r); err != nil { return nil, err } @@ -123,10 +122,16 @@ func CompactWithSplitting(ctx context.Context, src []BlockReader, shardsCount ui return out, errs.Err() } -var shardByFingerprint = func(r profileRow, shardsCount uint64) uint64 { +type SplitByFunc func(r profileRow, shardsCount uint64) uint64 + +var SplitByFingerprint = func(r profileRow, shardsCount uint64) uint64 { return uint64(r.fp) % shardsCount } +var SplitByStacktracePartition = func(r profileRow, shardsCount uint64) uint64 { + return r.row.StacktracePartitionID() % shardsCount +} + type blockWriter struct { indexRewriter *indexRewriter symbolsRewriter *symbolsRewriter diff --git a/pkg/phlaredb/compact_test.go b/pkg/phlaredb/compact_test.go index 0b5cce65d9..c44a3558c9 100644 --- a/pkg/phlaredb/compact_test.go +++ b/pkg/phlaredb/compact_test.go @@ -109,7 +109,7 @@ func TestCompactWithSplitting(t *testing.T) { ) }) dst := t.TempDir() - compacted, err := CompactWithSplitting(ctx, []BlockReader{b1, b2, b2, b1}, 16, dst) + compacted, err := CompactWithSplitting(ctx, []BlockReader{b1, b2, b2, b1}, 16, dst, SplitByFingerprint) require.NoError(t, err) // 4 shards one per series. From bf9f143cc9e4ff66a3caf4aba8a46a9d864ba7c4 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Mon, 16 Oct 2023 12:23:52 +0200 Subject: [PATCH 64/74] lint --- pkg/compactor/compactor.go | 2 -- 1 file changed, 2 deletions(-) diff --git a/pkg/compactor/compactor.go b/pkg/compactor/compactor.go index 329acded9a..af280237be 100644 --- a/pkg/compactor/compactor.go +++ b/pkg/compactor/compactor.go @@ -29,7 +29,6 @@ import ( "go.uber.org/atomic" "github.com/grafana/pyroscope/pkg/objstore" - "github.com/grafana/pyroscope/pkg/phlaredb" "github.com/grafana/pyroscope/pkg/phlaredb/block" "github.com/grafana/pyroscope/pkg/phlaredb/bucket" "github.com/grafana/pyroscope/pkg/tenant" @@ -229,7 +228,6 @@ type MultitenantCompactor struct { shardingStrategy shardingStrategy jobsOrder JobsOrderFunc - splitBy phlaredb.SplitByFunc // Metrics. compactionRunsStarted prometheus.Counter From 97e11c73c7c6563ed0cdbe5fc31454cbb5daa304 Mon Sep 17 00:00:00 2001 From: Cyril Tovena Date: Mon, 16 Oct 2023 13:16:37 +0200 Subject: [PATCH 65/74] Correct compaction range for the current default --- cmd/pyroscope/help-all.txt.tmpl | 2 +- .../index.md | 42 ++++++++++++------- .../pyroscope/rendered/micro-services.yaml | 20 ++++----- pkg/compactor/compactor.go | 2 +- 4 files changed, 38 insertions(+), 28 deletions(-) diff --git a/cmd/pyroscope/help-all.txt.tmpl b/cmd/pyroscope/help-all.txt.tmpl index 578a26925f..c14d1a5f81 100644 --- a/cmd/pyroscope/help-all.txt.tmpl +++ b/cmd/pyroscope/help-all.txt.tmpl @@ -16,7 +16,7 @@ Usage of ./pyroscope: -blocks-storage.bucket-store.tenant-sync-concurrency int Maximum number of concurrent tenants synching blocks. (default 10) -compactor.block-ranges value - List of compaction time ranges. (default 4h0m0s,8h0m0s) + List of compaction time ranges. (default 3h0m0s,6h0m0s,12h0m0s) -compactor.block-sync-concurrency int Number of Go routines to use when downloading blocks for compaction and uploading resulting blocks. (default 8) -compactor.blocks-retention-period duration diff --git a/docs/sources/configure-server/reference-configuration-parameters/index.md b/docs/sources/configure-server/reference-configuration-parameters/index.md index cee3eff8f4..b7c4de0731 100644 --- a/docs/sources/configure-server/reference-configuration-parameters/index.md +++ b/docs/sources/configure-server/reference-configuration-parameters/index.md @@ -37,9 +37,9 @@ brackets indicate that a parameter is optional. - ``: a CLI flag prefix based on the context (look at the parent configuration block to see which CLI flags prefix should be used) - ``: a [Prometheus relabeling configuration](https://prometheus.io/docs/prometheus/latest/configuration/configuration/#relabel_config) - `