From c61ac666244faaa2d86d5b946a51ff75d08f9c91 Mon Sep 17 00:00:00 2001 From: Alex Le Date: Tue, 13 Dec 2022 21:27:39 -0800 Subject: [PATCH 01/12] POC on sharded compaction Signed-off-by: Alex Le --- pkg/block/block.go | 10 +++ pkg/block/fetcher.go | 4 + pkg/block/partition_info.go | 79 ++++++++++++++++++ pkg/compact/compact.go | 142 ++++++++++++++++++++++++++------ pkg/compact/planner.go | 8 ++ pkg/compact/sharded_postings.go | 69 ++++++++++++++++ 6 files changed, 286 insertions(+), 26 deletions(-) create mode 100644 pkg/block/partition_info.go create mode 100644 pkg/compact/sharded_postings.go diff --git a/pkg/block/block.go b/pkg/block/block.go index 7886e5d869..6ccb7518e8 100644 --- a/pkg/block/block.go +++ b/pkg/block/block.go @@ -37,6 +37,8 @@ const ( IndexHeaderFilename = "index-header" // ChunksDirname is the known dir name for chunks with compressed samples. ChunksDirname = "chunks" + // PartitionInfoFilename is JSON filename for partition information. + PartitionInfoFilename = "partition-info.json" // DebugMetas is a directory for debug meta files that happen in the past. Useful for debugging. DebugMetas = "debug/metas" @@ -153,6 +155,14 @@ func upload(ctx context.Context, logger log.Logger, bkt objstore.Bucket, bdir st return cleanUp(logger, bkt, id, errors.Wrap(err, "upload index")) } + // level 1 blocks should not have partition info file + if meta.Compaction.Level > 1 { + if err := objstore.UploadFile(ctx, logger, bkt, filepath.Join(bdir, PartitionInfoFilename), path.Join(id.String(), PartitionInfoFilename)); err != nil { + // Don't call cleanUp here. Partition info file acts in a similar way as meta file. + return errors.Wrap(err, "upload partition info") + } + } + // Meta.json always need to be uploaded as a last item. This will allow to assume block directories without meta file to be pending uploads. if err := bkt.Upload(ctx, path.Join(id.String(), MetaFilename), strings.NewReader(metaEncoded.String())); err != nil { // Don't call cleanUp here. Despite getting error, meta.json may have been uploaded in certain cases, diff --git a/pkg/block/fetcher.go b/pkg/block/fetcher.go index 3a228f994f..1a9e213847 100644 --- a/pkg/block/fetcher.go +++ b/pkg/block/fetcher.go @@ -576,6 +576,10 @@ func (f *LabelShardedMetaFilter) Filter(_ context.Context, metas map[ulid.ULID]* var _ MetadataFilter = &DeduplicateFilter{} +type IDeduplicateFilter interface { + DuplicateIDs() []ulid.ULID +} + // DeduplicateFilter is a BaseFetcher filter that filters out older blocks that have exactly the same data. // Not go-routine safe. type DeduplicateFilter struct { diff --git a/pkg/block/partition_info.go b/pkg/block/partition_info.go new file mode 100644 index 0000000000..24602cdbd9 --- /dev/null +++ b/pkg/block/partition_info.go @@ -0,0 +1,79 @@ +package block + +import ( + "encoding/json" + "io" + "os" + "path/filepath" + + "github.com/go-kit/log" + "github.com/prometheus/prometheus/tsdb/fileutil" + + "github.com/thanos-io/thanos/pkg/runutil" +) + +type PartitionInfo struct { + PartitionedGroupID uint32 `json:"partitionedGroupID"` + PartitionCount int `json:"partitionCount"` + PartitionID int `json:"partitionID"` +} + +// WriteToDir writes the encoded partition info into /partition-info.json. +func (p PartitionInfo) WriteToDir(logger log.Logger, dir string) error { + // Make any changes to the file appear atomic. + path := filepath.Join(dir, PartitionInfoFilename) + tmp := path + ".tmp" + + f, err := os.Create(tmp) + if err != nil { + return err + } + + if err := p.Write(f); err != nil { + runutil.CloseWithLogOnErr(logger, f, "close partition info") + return err + } + if err := f.Close(); err != nil { + return err + } + return renameFile(logger, tmp, path) +} + +// Write writes the given encoded partition info to writer. +func (p PartitionInfo) Write(w io.Writer) error { + enc := json.NewEncoder(w) + enc.SetIndent("", "\t") + return enc.Encode(&p) +} + +func renameFile(logger log.Logger, from, to string) error { + if err := os.RemoveAll(to); err != nil { + return err + } + if err := os.Rename(from, to); err != nil { + return err + } + + // Directory was renamed; sync parent dir to persist rename. + pdir, err := fileutil.OpenDir(filepath.Dir(to)) + if err != nil { + return err + } + + if err = fileutil.Fdatasync(pdir); err != nil { + runutil.CloseWithLogOnErr(logger, pdir, "close dir") + return err + } + return pdir.Close() +} + +// Read the block partition info from the given reader. +func ReadPartitionInfo(rc io.ReadCloser) (_ *PartitionInfo, err error) { + defer runutil.ExhaustCloseWithErrCapture(&err, rc, "close partition info JSON") + + var p PartitionInfo + if err = json.NewDecoder(rc).Decode(&p); err != nil { + return nil, err + } + return &p, nil +} diff --git a/pkg/compact/compact.go b/pkg/compact/compact.go index c6c36bf3cc..54309819e5 100644 --- a/pkg/compact/compact.go +++ b/pkg/compact/compact.go @@ -22,6 +22,7 @@ import ( "github.com/prometheus/client_golang/prometheus/promauto" "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/tsdb" + "github.com/prometheus/prometheus/tsdb/index" "github.com/thanos-io/objstore" "golang.org/x/sync/errgroup" @@ -58,7 +59,7 @@ type Syncer struct { blocks map[ulid.ULID]*metadata.Meta partial map[ulid.ULID]error metrics *syncerMetrics - duplicateBlocksFilter *block.DeduplicateFilter + duplicateBlocksFilter block.IDeduplicateFilter ignoreDeletionMarkFilter *block.IgnoreDeletionMarkFilter } @@ -95,7 +96,7 @@ func newSyncerMetrics(reg prometheus.Registerer, blocksMarkedForDeletion, garbag // 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.MetadataFetcher, duplicateBlocksFilter *block.DeduplicateFilter, ignoreDeletionMarkFilter *block.IgnoreDeletionMarkFilter, blocksMarkedForDeletion, garbageCollectedBlocks prometheus.Counter) (*Syncer, error) { +func NewMetaSyncer(logger log.Logger, reg prometheus.Registerer, bkt objstore.Bucket, fetcher block.MetadataFetcher, duplicateBlocksFilter block.IDeduplicateFilter, ignoreDeletionMarkFilter *block.IgnoreDeletionMarkFilter, blocksMarkedForDeletion, garbageCollectedBlocks prometheus.Counter) (*Syncer, error) { if logger == nil { logger = log.NewNopLogger() } @@ -177,7 +178,7 @@ func (s *Syncer) GarbageCollect(ctx context.Context) error { if _, exists := deletionMarkMap[id]; exists { continue } - garbageIDs = append(garbageIDs, id) + //garbageIDs = append(garbageIDs, id) } for _, id := range garbageIDs { @@ -350,6 +351,9 @@ type Group struct { hashFunc metadata.HashFunc blockFilesConcurrency int compactBlocksFetchConcurrency int + partitionedGroupID uint32 + partitionCount int + partitionID int } // NewGroup returns a new compaction group. @@ -490,6 +494,24 @@ func (cg *Group) Resolution() int64 { return cg.resolution } +func (cg *Group) PartitionedGroupID() uint32 { + return cg.partitionedGroupID +} + +func (cg *Group) PartitionCount() int { + return cg.partitionCount +} + +func (cg *Group) PartitionID() int { + return cg.partitionID +} + +func (cg *Group) SetPartitionInfo(partitionedGroupID uint32, partitionCount int, partitionID int) { + cg.partitionedGroupID = partitionedGroupID + cg.partitionCount = partitionCount + cg.partitionID = partitionID +} + // CompactProgressMetrics contains Prometheus metrics related to compaction progress. type CompactProgressMetrics struct { NumberOfCompactionRuns *prometheus.GaugeVec @@ -728,6 +750,19 @@ type Planner interface { // Plan returns a list of blocks that should be compacted into single one. // The blocks can be overlapping. The provided metadata has to be ordered by minTime. Plan(ctx context.Context, metasByMinTime []*metadata.Meta) ([]*metadata.Meta, error) + + PlanWithPartition(ctx context.Context, metasByMinTime []*metadata.Meta, partitionID int, errChan chan error) ([]*metadata.Meta, error) +} + +type ExtraCompactionCompleteChecker interface { + IsComplete(group *Group, blockID ulid.ULID) bool +} + +type DefaultCompactionCompleteChecker struct { +} + +func (c DefaultCompactionCompleteChecker) IsComplete(_ *Group, _ ulid.ULID) bool { + return true } // Compactor provides compaction against an underlying storage of time series data. @@ -747,11 +782,12 @@ type Compactor interface { // * The source dirs are marked Deletable. // * Returns empty ulid.ULID{}. Compact(dest string, dirs []string, open []*tsdb.Block) (ulid.ULID, error) + CompactWithAdditionalPostings(dest string, dirs []string, open []*tsdb.Block, additionalPostingsProvider index.AdditionalPostingsProvider) (ulid.ULID, error) } // Compact plans and runs a single compaction against the group. The compacted result // is uploaded into the bucket the blocks were retrieved from. -func (cg *Group) Compact(ctx context.Context, dir string, planner Planner, comp Compactor) (shouldRerun bool, compID ulid.ULID, rerr error) { +func (cg *Group) Compact(ctx context.Context, dir string, planner Planner, comp Compactor, completeChecker ExtraCompactionCompleteChecker) (shouldRerun bool, compID ulid.ULID, rerr error) { cg.compactionRunsStarted.Inc() subDir := filepath.Join(dir, cg.Key()) @@ -771,10 +807,13 @@ func (cg *Group) Compact(ctx context.Context, dir string, planner Planner, comp return false, ulid.ULID{}, errors.Wrap(err, "create compaction group dir") } + errChan := make(chan error, 1) err := tracing.DoInSpanWithErr(ctx, "compaction_group", func(ctx context.Context) (err error) { - shouldRerun, compID, err = cg.compact(ctx, subDir, planner, comp) + shouldRerun, compID, err = cg.compact(ctx, subDir, planner, comp, completeChecker, errChan) return err }, opentracing.Tags{"group.key": cg.Key()}) + errChan <- err + close(errChan) if err != nil { cg.compactionFailures.Inc() return false, ulid.ULID{}, err @@ -974,7 +1013,7 @@ func RepairIssue347(ctx context.Context, logger log.Logger, bkt objstore.Bucket, return nil } -func (cg *Group) compact(ctx context.Context, dir string, planner Planner, comp Compactor) (shouldRerun bool, compID ulid.ULID, _ error) { +func (cg *Group) compact(ctx context.Context, dir string, planner Planner, comp Compactor, completeChecker ExtraCompactionCompleteChecker, errChan chan error) (shouldRerun bool, compID ulid.ULID, _ error) { cg.mtx.Lock() defer cg.mtx.Unlock() @@ -992,8 +1031,13 @@ func (cg *Group) compact(ctx context.Context, dir string, planner Planner, comp var toCompact []*metadata.Meta if err := tracing.DoInSpanWithErr(ctx, "compaction_planning", func(ctx context.Context) (e error) { - toCompact, e = planner.Plan(ctx, cg.metasByMinTime) - return e + if cg.partitionCount > 0 { + toCompact, e = planner.PlanWithPartition(ctx, cg.metasByMinTime, cg.partitionID, errChan) + return e + } else { + toCompact, e = planner.Plan(ctx, cg.metasByMinTime) + return e + } }); err != nil { return false, ulid.ULID{}, errors.Wrap(err, "plan compaction") } @@ -1006,7 +1050,7 @@ func (cg *Group) compact(ctx context.Context, dir string, planner Planner, comp // Due to #183 we verify that none of the blocks in the plan have overlapping sources. // This is one potential source of how we could end up with duplicated chunks. - uniqueSources := map[ulid.ULID]struct{}{} + //uniqueSources := map[ulid.ULID]struct{}{} // Once we have a plan we need to download the actual data. groupCompactionBegin := time.Now() @@ -1017,12 +1061,12 @@ func (cg *Group) compact(ctx context.Context, dir string, planner Planner, comp toCompactDirs := make([]string, 0, len(toCompact)) for _, m := range toCompact { bdir := filepath.Join(dir, m.ULID.String()) - for _, s := range m.Compaction.Sources { - if _, ok := uniqueSources[s]; ok { - return false, ulid.ULID{}, halt(errors.Errorf("overlapping sources detected for plan %v", toCompact)) - } - uniqueSources[s] = struct{}{} - } + //for _, s := range m.Compaction.Sources { + // if _, ok := uniqueSources[s]; ok { + // return false, ulid.ULID{}, halt(errors.Errorf("overlapping sources detected for plan %v", toCompact)) + // } + // uniqueSources[s] = struct{}{} + //} func(ctx context.Context, meta *metadata.Meta) { g.Go(func() error { if err := tracing.DoInSpanWithErr(ctx, "compaction_block_download", func(ctx context.Context) error { @@ -1072,7 +1116,11 @@ func (cg *Group) compact(ctx context.Context, dir string, planner Planner, comp begin = time.Now() if err := tracing.DoInSpanWithErr(ctx, "compaction", func(ctx context.Context) (e error) { - compID, e = comp.Compact(dir, toCompactDirs, nil) + additionalPostingsProvider := &ShardedPostingsProvider{ + partitionID: uint64(cg.partitionID), + partitionCount: uint64(cg.partitionCount), + } + compID, e = comp.CompactWithAdditionalPostings(dir, toCompactDirs, nil, additionalPostingsProvider) return e }); err != nil { return false, ulid.ULID{}, halt(errors.Wrapf(err, "compact blocks %v", toCompactDirs)) @@ -1082,7 +1130,7 @@ func (cg *Group) compact(ctx context.Context, dir string, planner Planner, comp level.Info(cg.logger).Log("msg", "compacted block would have no samples, deleting source blocks", "blocks", sourceBlockStr) for _, meta := range toCompact { if meta.Stats.NumSamples == 0 { - if err := cg.deleteBlock(meta.ULID, filepath.Join(dir, meta.ULID.String())); err != nil { + if err := cg.deleteBlock(meta.ULID, filepath.Join(dir, meta.ULID.String()), completeChecker); err != nil { level.Warn(cg.logger).Log("msg", "failed to mark for deletion an empty block found during compaction", "block", meta.ULID) } } @@ -1110,6 +1158,15 @@ func (cg *Group) compact(ctx context.Context, dir string, planner Planner, comp return false, ulid.ULID{}, errors.Wrapf(err, "failed to finalize the block %s", bdir) } + partitionInfo := block.PartitionInfo{ + PartitionedGroupID: cg.partitionedGroupID, + PartitionCount: cg.partitionCount, + PartitionID: cg.partitionID, + } + if err := partitionInfo.WriteToDir(cg.logger, bdir); err != nil { + return false, ulid.ULID{}, errors.Wrapf(err, "failed to put partition info for the block %s", bdir) + } + if err = os.Remove(filepath.Join(bdir, "tombstones")); err != nil { return false, ulid.ULID{}, errors.Wrap(err, "remove tombstones") } @@ -1145,7 +1202,7 @@ func (cg *Group) compact(ctx context.Context, dir string, planner Planner, comp // Eventually the block we just uploaded should get synced into the group again (including sync-delay). for _, meta := range toCompact { err = tracing.DoInSpanWithErr(ctx, "compaction_block_delete", func(ctx context.Context) error { - return cg.deleteBlock(meta.ULID, filepath.Join(dir, meta.ULID.String())) + return cg.deleteBlock(meta.ULID, filepath.Join(dir, meta.ULID.String()), completeChecker) }, opentracing.Tags{"block.id": meta.ULID}) if err != nil { return false, ulid.ULID{}, retry(errors.Wrapf(err, "mark old block for deletion from bucket")) @@ -1158,17 +1215,19 @@ func (cg *Group) compact(ctx context.Context, dir string, planner Planner, comp return true, compID, nil } -func (cg *Group) deleteBlock(id ulid.ULID, bdir string) error { +func (cg *Group) deleteBlock(id ulid.ULID, bdir string, completeChecker ExtraCompactionCompleteChecker) 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(cg.logger).Log("msg", "marking compacted block for deletion", "old_block", id) - if err := block.MarkForDeletion(delCtx, cg.logger, cg.bkt, id, "source of compacted block", cg.blocksMarkedForDeletion); err != nil { - return errors.Wrapf(err, "mark block %s for deletion from bucket", id) + if completeChecker.IsComplete(cg, 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(cg.logger).Log("msg", "marking compacted block for deletion", "old_block", id) + if err := block.MarkForDeletion(delCtx, cg.logger, cg.bkt, id, "source of compacted block", cg.blocksMarkedForDeletion); err != nil { + return errors.Wrapf(err, "mark block %s for deletion from bucket", id) + } } return nil } @@ -1180,6 +1239,7 @@ type BucketCompactor struct { grouper Grouper comp Compactor planner Planner + completeChecker ExtraCompactionCompleteChecker compactDir string bkt objstore.Bucket concurrency int @@ -1197,6 +1257,35 @@ func NewBucketCompactor( bkt objstore.Bucket, concurrency int, skipBlocksWithOutOfOrderChunks bool, +) (*BucketCompactor, error) { + if concurrency <= 0 { + return nil, errors.Errorf("invalid concurrency level (%d), concurrency level must be > 0", concurrency) + } + return NewBucketCompactorWithCompleteChecker( + logger, + sy, + grouper, + planner, + comp, + DefaultCompactionCompleteChecker{}, + compactDir, + bkt, + concurrency, + skipBlocksWithOutOfOrderChunks, + ) +} + +func NewBucketCompactorWithCompleteChecker( + logger log.Logger, + sy *Syncer, + grouper Grouper, + planner Planner, + comp Compactor, + completeChecker ExtraCompactionCompleteChecker, + compactDir string, + bkt objstore.Bucket, + concurrency int, + skipBlocksWithOutOfOrderChunks bool, ) (*BucketCompactor, error) { if concurrency <= 0 { return nil, errors.Errorf("invalid concurrency level (%d), concurrency level must be > 0", concurrency) @@ -1207,6 +1296,7 @@ func NewBucketCompactor( grouper: grouper, planner: planner, comp: comp, + completeChecker: completeChecker, compactDir: compactDir, bkt: bkt, concurrency: concurrency, @@ -1247,7 +1337,7 @@ func (c *BucketCompactor) Compact(ctx context.Context) (rerr error) { go func() { defer wg.Done() for g := range groupChan { - shouldRerunGroup, _, err := g.Compact(workCtx, c.compactDir, c.planner, c.comp) + shouldRerunGroup, _, err := g.Compact(workCtx, c.compactDir, c.planner, c.comp, c.completeChecker) if err == nil { if shouldRerunGroup { mtx.Lock() diff --git a/pkg/compact/planner.go b/pkg/compact/planner.go index 5c2a93df8d..18c4e62ccd 100644 --- a/pkg/compact/planner.go +++ b/pkg/compact/planner.go @@ -93,6 +93,10 @@ func (p *tsdbBasedPlanner) plan(noCompactMarked map[ulid.ULID]*metadata.NoCompac return nil, nil } +func (p *tsdbBasedPlanner) PlanWithPartition(ctx context.Context, metasByMinTime []*metadata.Meta, partitionID int, errChan chan error) ([]*metadata.Meta, error) { + return nil, errors.New("not support with partitioning") +} + // selectMetas returns the dir metas that should be compacted into a single new block. // If only a single block range is configured, the result is always nil. // Copied and adjusted from https://github.com/prometheus/prometheus/blob/3d8826a3d42566684283a9b7f7e812e412c24407/tsdb/compact.go#L229. @@ -303,3 +307,7 @@ PlanLoop: return plan, nil } } + +func (t *largeTotalIndexSizeFilter) PlanWithPartition(ctx context.Context, metasByMinTime []*metadata.Meta, partitionID int, errChan chan error) ([]*metadata.Meta, error) { + return nil, errors.New("not support with partitioning") +} diff --git a/pkg/compact/sharded_postings.go b/pkg/compact/sharded_postings.go new file mode 100644 index 0000000000..00318d9569 --- /dev/null +++ b/pkg/compact/sharded_postings.go @@ -0,0 +1,69 @@ +package compact + +import ( + "fmt" + + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/tsdb" + "github.com/prometheus/prometheus/tsdb/chunks" + "github.com/prometheus/prometheus/tsdb/index" +) + +type ShardedPostingsProvider struct { + partitionCount uint64 + partitionID uint64 +} + +func (p *ShardedPostingsProvider) GetPostings(originalPostings index.Postings, indexReader tsdb.IndexReader) index.Postings { + return NewShardedPosting(originalPostings, p.partitionCount, p.partitionID, indexReader.Series) +} + +type ShardedPostings struct { + //postings Postings + //labelsFn func(ref storage.SeriesRef, lset *labels.Labels, chks *[]chunks.Meta) error + series []storage.SeriesRef + cur storage.SeriesRef + initialized bool + + //bufChks []chunks.Meta + //bufLbls labels.Labels +} + +func NewShardedPosting(postings index.Postings, partitionCount uint64, partitionId uint64, labelsFn func(ref storage.SeriesRef, lset *labels.Labels, chks *[]chunks.Meta) error) *ShardedPostings { + bufChks := make([]chunks.Meta, 0) + bufLbls := make(labels.Labels, 0) + series := make([]storage.SeriesRef, 0) + for postings.Next() { + err := labelsFn(postings.At(), &bufLbls, &bufChks) + if err != nil { + fmt.Printf("err: %v", err) + } + if bufLbls.Hash()%partitionCount == partitionId { + posting := postings.At() + series = append(series, posting) + } + } + return &ShardedPostings{series: series, initialized: false} +} + +func (p *ShardedPostings) Next() bool { + if len(p.series) > 0 { + p.cur, p.series = p.series[0], p.series[1:] + return true + } + return false +} + +func (p *ShardedPostings) At() storage.SeriesRef { + return p.cur +} + +func (p *ShardedPostings) Seek(v storage.SeriesRef) bool { + fmt.Println("ran seek") + return false +} + +func (p *ShardedPostings) Err() error { + return nil +} From 45a4cfa8c417ac2240c9858c2d81a3aa6e50bfb6 Mon Sep 17 00:00:00 2001 From: Alex Le Date: Tue, 3 Jan 2023 11:47:01 -0800 Subject: [PATCH 02/12] Refactored partition info to be part of meta Signed-off-by: Alex Le --- pkg/block/block.go | 10 ----- pkg/block/metadata/meta.go | 9 ++++ pkg/block/partition_info.go | 79 --------------------------------- pkg/compact/compact.go | 65 +++++++++++++++++++-------- pkg/compact/sharded_postings.go | 4 +- 5 files changed, 57 insertions(+), 110 deletions(-) delete mode 100644 pkg/block/partition_info.go diff --git a/pkg/block/block.go b/pkg/block/block.go index 6ccb7518e8..7886e5d869 100644 --- a/pkg/block/block.go +++ b/pkg/block/block.go @@ -37,8 +37,6 @@ const ( IndexHeaderFilename = "index-header" // ChunksDirname is the known dir name for chunks with compressed samples. ChunksDirname = "chunks" - // PartitionInfoFilename is JSON filename for partition information. - PartitionInfoFilename = "partition-info.json" // DebugMetas is a directory for debug meta files that happen in the past. Useful for debugging. DebugMetas = "debug/metas" @@ -155,14 +153,6 @@ func upload(ctx context.Context, logger log.Logger, bkt objstore.Bucket, bdir st return cleanUp(logger, bkt, id, errors.Wrap(err, "upload index")) } - // level 1 blocks should not have partition info file - if meta.Compaction.Level > 1 { - if err := objstore.UploadFile(ctx, logger, bkt, filepath.Join(bdir, PartitionInfoFilename), path.Join(id.String(), PartitionInfoFilename)); err != nil { - // Don't call cleanUp here. Partition info file acts in a similar way as meta file. - return errors.Wrap(err, "upload partition info") - } - } - // Meta.json always need to be uploaded as a last item. This will allow to assume block directories without meta file to be pending uploads. if err := bkt.Upload(ctx, path.Join(id.String(), MetaFilename), strings.NewReader(metaEncoded.String())); err != nil { // Don't call cleanUp here. Despite getting error, meta.json may have been uploaded in certain cases, diff --git a/pkg/block/metadata/meta.go b/pkg/block/metadata/meta.go index 787a03c241..797ccd7860 100644 --- a/pkg/block/metadata/meta.go +++ b/pkg/block/metadata/meta.go @@ -90,6 +90,9 @@ type Thanos struct { // Rewrites is present when any rewrite (deletion, relabel etc) were applied to this block. Optional. Rewrites []Rewrite `json:"rewrites,omitempty"` + + // PartitionInfo is used for partitioning compaction to keep track of partition information of result block. Optional. + PartitionInfo *PartitionInfo `json:"partition_info,omitempty"` } type Rewrite struct { @@ -101,6 +104,12 @@ type Rewrite struct { RelabelsApplied []*relabel.Config `json:"relabels_applied,omitempty"` } +type PartitionInfo struct { + PartitionedGroupID uint32 `json:"partitionedGroupID"` + PartitionCount int `json:"partitionCount"` + PartitionID int `json:"partitionID"` +} + type Matchers []*labels.Matcher func (m *Matchers) UnmarshalYAML(value *yaml.Node) (err error) { diff --git a/pkg/block/partition_info.go b/pkg/block/partition_info.go deleted file mode 100644 index 24602cdbd9..0000000000 --- a/pkg/block/partition_info.go +++ /dev/null @@ -1,79 +0,0 @@ -package block - -import ( - "encoding/json" - "io" - "os" - "path/filepath" - - "github.com/go-kit/log" - "github.com/prometheus/prometheus/tsdb/fileutil" - - "github.com/thanos-io/thanos/pkg/runutil" -) - -type PartitionInfo struct { - PartitionedGroupID uint32 `json:"partitionedGroupID"` - PartitionCount int `json:"partitionCount"` - PartitionID int `json:"partitionID"` -} - -// WriteToDir writes the encoded partition info into /partition-info.json. -func (p PartitionInfo) WriteToDir(logger log.Logger, dir string) error { - // Make any changes to the file appear atomic. - path := filepath.Join(dir, PartitionInfoFilename) - tmp := path + ".tmp" - - f, err := os.Create(tmp) - if err != nil { - return err - } - - if err := p.Write(f); err != nil { - runutil.CloseWithLogOnErr(logger, f, "close partition info") - return err - } - if err := f.Close(); err != nil { - return err - } - return renameFile(logger, tmp, path) -} - -// Write writes the given encoded partition info to writer. -func (p PartitionInfo) Write(w io.Writer) error { - enc := json.NewEncoder(w) - enc.SetIndent("", "\t") - return enc.Encode(&p) -} - -func renameFile(logger log.Logger, from, to string) error { - if err := os.RemoveAll(to); err != nil { - return err - } - if err := os.Rename(from, to); err != nil { - return err - } - - // Directory was renamed; sync parent dir to persist rename. - pdir, err := fileutil.OpenDir(filepath.Dir(to)) - if err != nil { - return err - } - - if err = fileutil.Fdatasync(pdir); err != nil { - runutil.CloseWithLogOnErr(logger, pdir, "close dir") - return err - } - return pdir.Close() -} - -// Read the block partition info from the given reader. -func ReadPartitionInfo(rc io.ReadCloser) (_ *PartitionInfo, err error) { - defer runutil.ExhaustCloseWithErrCapture(&err, rc, "close partition info JSON") - - var p PartitionInfo - if err = json.NewDecoder(rc).Decode(&p); err != nil { - return nil, err - } - return &p, nil -} diff --git a/pkg/compact/compact.go b/pkg/compact/compact.go index 54309819e5..16b8187077 100644 --- a/pkg/compact/compact.go +++ b/pkg/compact/compact.go @@ -22,7 +22,6 @@ import ( "github.com/prometheus/client_golang/prometheus/promauto" "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/tsdb" - "github.com/prometheus/prometheus/tsdb/index" "github.com/thanos-io/objstore" "golang.org/x/sync/errgroup" @@ -765,6 +764,22 @@ func (c DefaultCompactionCompleteChecker) IsComplete(_ *Group, _ ulid.ULID) bool return true } +type CompactionLifecycleCallback interface { + PreCompactionCallback(group *Group, toCompactBlocks []*metadata.Meta) error + PostCompactionCallback(group *Group, blockID ulid.ULID) error +} + +type NoopCompactionLifecycleCallback struct { +} + +func (c NoopCompactionLifecycleCallback) PreCompactionCallback(_ *Group, _ []*metadata.Meta) error { + return nil +} + +func (c NoopCompactionLifecycleCallback) PostCompactionCallback(_ *Group, _ ulid.ULID) error { + return nil +} + // 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. @@ -782,12 +797,12 @@ type Compactor interface { // * The source dirs are marked Deletable. // * Returns empty ulid.ULID{}. Compact(dest string, dirs []string, open []*tsdb.Block) (ulid.ULID, error) - CompactWithAdditionalPostings(dest string, dirs []string, open []*tsdb.Block, additionalPostingsProvider index.AdditionalPostingsProvider) (ulid.ULID, error) + CompactWithAdditionalPostings(dest string, dirs []string, open []*tsdb.Block, additionalPostingsProvider tsdb.AdditionalPostingsFunc) (ulid.ULID, error) } // Compact plans and runs a single compaction against the group. The compacted result // is uploaded into the bucket the blocks were retrieved from. -func (cg *Group) Compact(ctx context.Context, dir string, planner Planner, comp Compactor, completeChecker ExtraCompactionCompleteChecker) (shouldRerun bool, compID ulid.ULID, rerr error) { +func (cg *Group) Compact(ctx context.Context, dir string, planner Planner, comp Compactor, completeChecker ExtraCompactionCompleteChecker, compactionLifecycleCallback CompactionLifecycleCallback) (shouldRerun bool, compID ulid.ULID, rerr error) { cg.compactionRunsStarted.Inc() subDir := filepath.Join(dir, cg.Key()) @@ -809,7 +824,7 @@ func (cg *Group) Compact(ctx context.Context, dir string, planner Planner, comp errChan := make(chan error, 1) err := tracing.DoInSpanWithErr(ctx, "compaction_group", func(ctx context.Context) (err error) { - shouldRerun, compID, err = cg.compact(ctx, subDir, planner, comp, completeChecker, errChan) + shouldRerun, compID, err = cg.compact(ctx, subDir, planner, comp, completeChecker, compactionLifecycleCallback, errChan) return err }, opentracing.Tags{"group.key": cg.Key()}) errChan <- err @@ -1013,7 +1028,7 @@ func RepairIssue347(ctx context.Context, logger log.Logger, bkt objstore.Bucket, return nil } -func (cg *Group) compact(ctx context.Context, dir string, planner Planner, comp Compactor, completeChecker ExtraCompactionCompleteChecker, errChan chan error) (shouldRerun bool, compID ulid.ULID, _ error) { +func (cg *Group) compact(ctx context.Context, dir string, planner Planner, comp Compactor, completeChecker ExtraCompactionCompleteChecker, compactionLifecycleCallback CompactionLifecycleCallback, errChan chan error) (shouldRerun bool, compID ulid.ULID, _ error) { cg.mtx.Lock() defer cg.mtx.Unlock() @@ -1046,6 +1061,12 @@ func (cg *Group) compact(ctx context.Context, dir string, planner Planner, comp return false, ulid.ULID{}, nil } + level.Info(cg.logger).Log("msg", "running pre compaction callback", "plan", fmt.Sprintf("%v", toCompact)) + if err := compactionLifecycleCallback.PreCompactionCallback(cg, toCompact); err != nil { + return false, ulid.ULID{}, errors.Wrapf(err, "failed to run pre compaction callback for plan: %s", fmt.Sprintf("%v", toCompact)) + } + level.Info(cg.logger).Log("msg", "finished running pre compaction callback", "plan", fmt.Sprintf("%v", toCompact)) + level.Info(cg.logger).Log("msg", "compaction available and planned; downloading blocks", "plan", fmt.Sprintf("%v", toCompact)) // Due to #183 we verify that none of the blocks in the plan have overlapping sources. @@ -1116,11 +1137,11 @@ func (cg *Group) compact(ctx context.Context, dir string, planner Planner, comp begin = time.Now() if err := tracing.DoInSpanWithErr(ctx, "compaction", func(ctx context.Context) (e error) { - additionalPostingsProvider := &ShardedPostingsProvider{ + additionalPostingsFunc := &ShardedPostingsFunc{ partitionID: uint64(cg.partitionID), partitionCount: uint64(cg.partitionCount), } - compID, e = comp.CompactWithAdditionalPostings(dir, toCompactDirs, nil, additionalPostingsProvider) + compID, e = comp.CompactWithAdditionalPostings(dir, toCompactDirs, nil, additionalPostingsFunc) return e }); err != nil { return false, ulid.ULID{}, halt(errors.Wrapf(err, "compact blocks %v", toCompactDirs)) @@ -1153,20 +1174,16 @@ func (cg *Group) compact(ctx context.Context, dir string, planner Planner, comp Downsample: metadata.ThanosDownsample{Resolution: cg.resolution}, Source: metadata.CompactorSource, SegmentFiles: block.GetSegmentFiles(bdir), + PartitionInfo: &metadata.PartitionInfo{ + PartitionedGroupID: cg.partitionedGroupID, + PartitionCount: cg.partitionCount, + PartitionID: cg.partitionID, + }, }, nil) if err != nil { return false, ulid.ULID{}, errors.Wrapf(err, "failed to finalize the block %s", bdir) } - partitionInfo := block.PartitionInfo{ - PartitionedGroupID: cg.partitionedGroupID, - PartitionCount: cg.partitionCount, - PartitionID: cg.partitionID, - } - if err := partitionInfo.WriteToDir(cg.logger, bdir); err != nil { - return false, ulid.ULID{}, errors.Wrapf(err, "failed to put partition info for the block %s", bdir) - } - if err = os.Remove(filepath.Join(bdir, "tombstones")); err != nil { return false, ulid.ULID{}, errors.Wrap(err, "remove tombstones") } @@ -1197,6 +1214,12 @@ func (cg *Group) compact(ctx context.Context, dir string, planner Planner, comp } level.Info(cg.logger).Log("msg", "uploaded block", "result_block", compID, "duration", time.Since(begin), "duration_ms", time.Since(begin).Milliseconds()) + level.Info(cg.logger).Log("msg", "running post compaction callback", "result_block", compID) + if err := compactionLifecycleCallback.PostCompactionCallback(cg, compID); err != nil { + return false, ulid.ULID{}, retry(errors.Wrapf(err, "failed to run post compaction callback for result block %s", compID)) + } + level.Info(cg.logger).Log("msg", "finished running post compaction callback", "result_block", compID) + // Mark for deletion the blocks we just compacted from the group and bucket so they do not get included // into the next planning cycle. // Eventually the block we just uploaded should get synced into the group again (including sync-delay). @@ -1240,6 +1263,7 @@ type BucketCompactor struct { comp Compactor planner Planner completeChecker ExtraCompactionCompleteChecker + compactionLifecycleCallback CompactionLifecycleCallback compactDir string bkt objstore.Bucket concurrency int @@ -1261,13 +1285,14 @@ func NewBucketCompactor( if concurrency <= 0 { return nil, errors.Errorf("invalid concurrency level (%d), concurrency level must be > 0", concurrency) } - return NewBucketCompactorWithCompleteChecker( + return NewBucketCompactorWithCheckerAndCallback( logger, sy, grouper, planner, comp, DefaultCompactionCompleteChecker{}, + NoopCompactionLifecycleCallback{}, compactDir, bkt, concurrency, @@ -1275,13 +1300,14 @@ func NewBucketCompactor( ) } -func NewBucketCompactorWithCompleteChecker( +func NewBucketCompactorWithCheckerAndCallback( logger log.Logger, sy *Syncer, grouper Grouper, planner Planner, comp Compactor, completeChecker ExtraCompactionCompleteChecker, + compactionLifecycleCallback CompactionLifecycleCallback, compactDir string, bkt objstore.Bucket, concurrency int, @@ -1297,6 +1323,7 @@ func NewBucketCompactorWithCompleteChecker( planner: planner, comp: comp, completeChecker: completeChecker, + compactionLifecycleCallback: compactionLifecycleCallback, compactDir: compactDir, bkt: bkt, concurrency: concurrency, @@ -1337,7 +1364,7 @@ func (c *BucketCompactor) Compact(ctx context.Context) (rerr error) { go func() { defer wg.Done() for g := range groupChan { - shouldRerunGroup, _, err := g.Compact(workCtx, c.compactDir, c.planner, c.comp, c.completeChecker) + shouldRerunGroup, _, err := g.Compact(workCtx, c.compactDir, c.planner, c.comp, c.completeChecker, c.compactionLifecycleCallback) if err == nil { if shouldRerunGroup { mtx.Lock() diff --git a/pkg/compact/sharded_postings.go b/pkg/compact/sharded_postings.go index 00318d9569..5ae560e4d8 100644 --- a/pkg/compact/sharded_postings.go +++ b/pkg/compact/sharded_postings.go @@ -10,12 +10,12 @@ import ( "github.com/prometheus/prometheus/tsdb/index" ) -type ShardedPostingsProvider struct { +type ShardedPostingsFunc struct { partitionCount uint64 partitionID uint64 } -func (p *ShardedPostingsProvider) GetPostings(originalPostings index.Postings, indexReader tsdb.IndexReader) index.Postings { +func (p *ShardedPostingsFunc) GetPostings(originalPostings index.Postings, indexReader tsdb.IndexReader) index.Postings { return NewShardedPosting(originalPostings, p.partitionCount, p.partitionID, indexReader.Series) } From 9dbe33e64780a52fda03d392b48305f69571ce78 Mon Sep 17 00:00:00 2001 From: Alex Le Date: Tue, 7 Mar 2023 10:07:22 -0800 Subject: [PATCH 03/12] Override prometheus populate block func Signed-off-by: Alex Le --- pkg/compact/background_chunks_series_set.go | 60 ++++++ pkg/compact/compact.go | 10 +- pkg/compact/sharded_populate_block_func.go | 217 ++++++++++++++++++++ 3 files changed, 282 insertions(+), 5 deletions(-) create mode 100644 pkg/compact/background_chunks_series_set.go create mode 100644 pkg/compact/sharded_populate_block_func.go diff --git a/pkg/compact/background_chunks_series_set.go b/pkg/compact/background_chunks_series_set.go new file mode 100644 index 0000000000..900b87f8ab --- /dev/null +++ b/pkg/compact/background_chunks_series_set.go @@ -0,0 +1,60 @@ +package compact + +import ( + "context" + "github.com/prometheus/prometheus/storage" +) + +type backgrounChunkSeriesSet struct { + nextSet chan storage.ChunkSeries + actual storage.ChunkSeries + cs storage.ChunkSeriesSet +} + +func (b *backgrounChunkSeriesSet) Next() bool { + select { + case s, ok := <-b.nextSet: + b.actual = s + return ok + } +} + +func (b *backgrounChunkSeriesSet) At() storage.ChunkSeries { + return b.actual +} + +func (b *backgrounChunkSeriesSet) Err() error { + return b.cs.Err() +} + +func (b *backgrounChunkSeriesSet) Warnings() storage.Warnings { + return b.cs.Warnings() +} + +func (b *backgrounChunkSeriesSet) run(ctx context.Context) { + for { + if !b.cs.Next() { + close(b.nextSet) + return + } + + select { + case b.nextSet <- b.cs.At(): + case <-ctx.Done(): + return + } + } +} + +func NewBackgroundChunkSeriesSet(ctx context.Context, cs storage.ChunkSeriesSet) storage.ChunkSeriesSet { + r := &backgrounChunkSeriesSet{ + cs: cs, + nextSet: make(chan storage.ChunkSeries, 1000), + } + + go func() { + r.run(ctx) + }() + + return r +} diff --git a/pkg/compact/compact.go b/pkg/compact/compact.go index 16b8187077..73549b4f19 100644 --- a/pkg/compact/compact.go +++ b/pkg/compact/compact.go @@ -797,7 +797,7 @@ type Compactor interface { // * The source dirs are marked Deletable. // * Returns empty ulid.ULID{}. Compact(dest string, dirs []string, open []*tsdb.Block) (ulid.ULID, error) - CompactWithAdditionalPostings(dest string, dirs []string, open []*tsdb.Block, additionalPostingsProvider tsdb.AdditionalPostingsFunc) (ulid.ULID, error) + CompactWithPopulateBlockFunc(dest string, dirs []string, open []*tsdb.Block, populateBlockFunc tsdb.PopulateBlockFunc) (ulid.ULID, error) } // Compact plans and runs a single compaction against the group. The compacted result @@ -1137,11 +1137,11 @@ func (cg *Group) compact(ctx context.Context, dir string, planner Planner, comp begin = time.Now() if err := tracing.DoInSpanWithErr(ctx, "compaction", func(ctx context.Context) (e error) { - additionalPostingsFunc := &ShardedPostingsFunc{ - partitionID: uint64(cg.partitionID), - partitionCount: uint64(cg.partitionCount), + populateBlockFunc := ShardedPopulateBlockFunc{ + partitionCount: cg.partitionCount, + partitionId: cg.partitionID, } - compID, e = comp.CompactWithAdditionalPostings(dir, toCompactDirs, nil, additionalPostingsFunc) + compID, e = comp.CompactWithPopulateBlockFunc(dir, toCompactDirs, nil, populateBlockFunc) return e }); err != nil { return false, ulid.ULID{}, halt(errors.Wrapf(err, "compact blocks %v", toCompactDirs)) diff --git a/pkg/compact/sharded_populate_block_func.go b/pkg/compact/sharded_populate_block_func.go new file mode 100644 index 0000000000..e6d920e741 --- /dev/null +++ b/pkg/compact/sharded_populate_block_func.go @@ -0,0 +1,217 @@ +package compact + +import ( + "context" + "fmt" + "golang.org/x/sync/errgroup" + "io" + "sync" + "time" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/pkg/errors" + + "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/tsdb" + "github.com/prometheus/prometheus/tsdb/chunkenc" + "github.com/prometheus/prometheus/tsdb/chunks" + tsdb_errors "github.com/prometheus/prometheus/tsdb/errors" + "github.com/prometheus/prometheus/tsdb/index" +) + +type ShardedPopulateBlockFunc struct { + partitionCount int + partitionId int +} + +// PopulateBlock fills the index and chunk writers with new data gathered as the union +// of the provided blocks. It returns meta information for the new block. +// It expects sorted blocks input by mint. +func (c ShardedPopulateBlockFunc) PopulateBlock(metrics *tsdb.CompactorMetrics, logger log.Logger, chunkPool chunkenc.Pool, ctx context.Context, mergeFunc storage.VerticalChunkSeriesMergeFunc, blocks []tsdb.BlockReader, meta *tsdb.BlockMeta, indexw tsdb.IndexWriter, chunkw tsdb.ChunkWriter) (err error) { + if len(blocks) == 0 { + return errors.New("cannot populate block from no readers") + } + + var ( + sets []storage.ChunkSeriesSet + setsMtx sync.Mutex + symbols index.StringIter + closers []io.Closer + overlapping bool + ) + defer func() { + errs := tsdb_errors.NewMulti(err) + if cerr := tsdb_errors.CloseAll(closers); cerr != nil { + errs.Add(errors.Wrap(cerr, "close")) + } + err = errs.Err() + metrics.PopulatingBlocks.Set(0) + }() + metrics.PopulatingBlocks.Set(1) + + globalMaxt := blocks[0].Meta().MaxTime + g, _ := errgroup.WithContext(ctx) + g.SetLimit(8) + for i, b := range blocks { + select { + case <-ctx.Done(): + return ctx.Err() + default: + } + + if !overlapping { + if i > 0 && b.Meta().MinTime < globalMaxt { + metrics.OverlappingBlocks.Inc() + overlapping = true + level.Info(logger).Log("msg", "Found overlapping blocks during compaction", "ulid", meta.ULID) + } + if b.Meta().MaxTime > globalMaxt { + globalMaxt = b.Meta().MaxTime + } + } + + indexr, err := b.Index() + if err != nil { + return errors.Wrapf(err, "open index reader for block %+v", b.Meta()) + } + closers = append(closers, indexr) + + chunkr, err := b.Chunks() + if err != nil { + return errors.Wrapf(err, "open chunk reader for block %+v", b.Meta()) + } + closers = append(closers, chunkr) + + tombsr, err := b.Tombstones() + if err != nil { + return errors.Wrapf(err, "open tombstone reader for block %+v", b.Meta()) + } + closers = append(closers, tombsr) + + k, v := index.AllPostingsKey() + all, err := indexr.Postings(k, v) + if err != nil { + return err + } + all = indexr.SortedPostings(all) + g.Go(func() error { + shardStart := time.Now() + shardedPosting := index.NewShardedPosting(all, uint64(c.partitionCount), uint64(c.partitionId), indexr.Series) + fmt.Printf("finished sharding, duration: %v\n", time.Since(shardStart)) + // Blocks meta is half open: [min, max), so subtract 1 to ensure we don't hold samples with exact meta.MaxTime timestamp. + setsMtx.Lock() + sets = append(sets, tsdb.NewBlockChunkSeriesSet(meta.ULID, indexr, chunkr, tombsr, shardedPosting, meta.MinTime, meta.MaxTime-1, false)) + setsMtx.Unlock() + return nil + }) + syms := indexr.Symbols() + if i == 0 { + symbols = syms + continue + } + symbols = tsdb.NewMergedStringIter(symbols, syms) + } + if err := g.Wait(); err != nil { + return err + } + + for symbols.Next() { + if err := indexw.AddSymbol(symbols.At()); err != nil { + return errors.Wrap(err, "add symbol") + } + } + if symbols.Err() != nil { + return errors.Wrap(symbols.Err(), "next symbol") + } + + var ( + ref = storage.SeriesRef(0) + ch = make(chan func() error, 1000) + ) + + set := sets[0] + if len(sets) > 1 { + iCtx, cancel := context.WithCancel(ctx) + // Merge series using specified chunk series merger. + // The default one is the compacting series merger. + set = NewBackgroundChunkSeriesSet(iCtx, storage.NewMergeChunkSeriesSet(sets, mergeFunc)) + defer cancel() + } + + go func() { + // Iterate over all sorted chunk series. + for set.Next() { + select { + case <-ctx.Done(): + ch <- func() error { return ctx.Err() } + default: + } + s := set.At() + chksIter := s.Iterator() + + var chks []chunks.Meta + var wg sync.WaitGroup + r := ref + wg.Add(1) + go func() { + for chksIter.Next() { + // We are not iterating in streaming way over chunk as + // it's more efficient to do bulk write for index and + // chunk file purposes. + chks = append(chks, chksIter.At()) + } + wg.Done() + }() + + ch <- func() error { + wg.Wait() + if chksIter.Err() != nil { + return errors.Wrap(chksIter.Err(), "chunk iter") + } + + // Skip the series with all deleted chunks. + if len(chks) == 0 { + return nil + } + + if err := chunkw.WriteChunks(chks...); err != nil { + return errors.Wrap(err, "write chunks") + } + if err := indexw.AddSeries(r, s.Labels(), chks...); err != nil { + return errors.Wrap(err, "add series") + } + + meta.Stats.NumChunks += uint64(len(chks)) + meta.Stats.NumSeries++ + for _, chk := range chks { + meta.Stats.NumSamples += uint64(chk.Chunk.NumSamples()) + } + + for _, chk := range chks { + if err := chunkPool.Put(chk.Chunk); err != nil { + return errors.Wrap(err, "put chunk") + } + } + + return nil + } + + ref++ + } + close(ch) + }() + + for callback := range ch { + err := callback() + if err != nil { + return err + } + } + + if set.Err() != nil { + return errors.Wrap(set.Err(), "iterate compaction set") + } + + return nil +} From c71ed9d51b50446e00230726f912582826c2a9f5 Mon Sep 17 00:00:00 2001 From: Alex Le Date: Wed, 12 Apr 2023 18:56:56 -0700 Subject: [PATCH 04/12] Make block populator plugable through CompactionLifecycleCallback Signed-off-by: Alex Le --- go.mod | 96 ++++--- go.sum | 263 ++++++++++---------- pkg/block/metadata/meta.go | 6 +- pkg/block/metadata/meta_test.go | 100 ++++++++ pkg/compact/background_chunks_series_set.go | 60 ----- pkg/compact/compact.go | 59 +++-- pkg/compact/sharded_populate_block_func.go | 217 ---------------- pkg/compact/sharded_postings.go | 69 ----- 8 files changed, 313 insertions(+), 557 deletions(-) delete mode 100644 pkg/compact/background_chunks_series_set.go delete mode 100644 pkg/compact/sharded_populate_block_func.go delete mode 100644 pkg/compact/sharded_postings.go diff --git a/go.mod b/go.mod index 96312eea77..ffc7a810a7 100644 --- a/go.mod +++ b/go.mod @@ -3,13 +3,12 @@ module github.com/thanos-io/thanos go 1.18 require ( - cloud.google.com/go/storage v1.27.0 // indirect - cloud.google.com/go/trace v1.4.0 + cloud.google.com/go/storage v1.28.1 // indirect + cloud.google.com/go/trace v1.8.0 github.com/GoogleCloudPlatform/opentelemetry-operations-go/exporter/trace v1.8.3 github.com/NYTimes/gziphandler v1.1.1 github.com/alecthomas/units v0.0.0-20211218093645-b94a6e3cc137 github.com/alicebob/miniredis/v2 v2.22.0 - github.com/armon/go-metrics v0.4.0 // indirect github.com/blang/semver/v4 v4.0.0 github.com/bradfitz/gomemcache v0.0.0-20190913173617-a41fca850d0b github.com/cespare/xxhash v1.1.0 @@ -31,7 +30,7 @@ require ( 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.2 + github.com/golang/protobuf v1.5.3 github.com/golang/snappy v0.0.4 github.com/googleapis/gax-go v2.0.2+incompatible github.com/gorilla/mux v1.8.0 // indirect @@ -47,7 +46,7 @@ require ( github.com/leanovate/gopter v0.2.9 github.com/lightstep/lightstep-tracer-go v0.25.0 github.com/lovoo/gcloud-opentracing v0.3.0 - github.com/miekg/dns v1.1.50 + github.com/miekg/dns v1.1.53 github.com/minio/minio-go/v7 v7.0.45 // indirect github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f github.com/oklog/run v1.1.0 @@ -62,12 +61,12 @@ require ( github.com/prometheus/alertmanager v0.25.0 github.com/prometheus/client_golang v1.14.0 github.com/prometheus/client_model v0.3.0 - github.com/prometheus/common v0.39.1-0.20230202092144-f9c1994be032 - github.com/prometheus/exporter-toolkit v0.8.2 + github.com/prometheus/common v0.42.0 + github.com/prometheus/exporter-toolkit v0.9.1 // Prometheus maps version 2.x.y to tags v0.x.y. - github.com/prometheus/prometheus v0.42.0 + github.com/prometheus/prometheus v0.43.1-0.20230404063149-1936868e9d13 github.com/sony/gobreaker v0.5.0 - github.com/stretchr/testify v1.8.1 + github.com/stretchr/testify v1.8.2 github.com/thanos-community/promql-engine v0.0.0-20230224075812-ae04bbea7613 github.com/thanos-io/objstore v0.0.0-20230201072718-11ffbc490204 github.com/uber/jaeger-client-go v2.30.0+incompatible @@ -77,21 +76,21 @@ require ( go.elastic.co/apm v1.11.0 go.elastic.co/apm/module/apmot v1.11.0 go.opentelemetry.io/contrib/propagators/ot v1.13.0 // indirect - go.opentelemetry.io/otel v1.13.0 + go.opentelemetry.io/otel v1.14.0 go.opentelemetry.io/otel/bridge/opentracing v1.12.0 - go.opentelemetry.io/otel/sdk v1.12.0 - go.opentelemetry.io/otel/trace v1.13.0 + go.opentelemetry.io/otel/sdk v1.14.0 + go.opentelemetry.io/otel/trace v1.14.0 go.uber.org/atomic v1.10.0 go.uber.org/automaxprocs v1.5.1 - go.uber.org/goleak v1.2.0 - golang.org/x/crypto v0.3.0 - golang.org/x/net v0.7.0 + go.uber.org/goleak v1.2.1 + golang.org/x/crypto v0.7.0 + golang.org/x/net v0.8.0 golang.org/x/sync v0.1.0 - golang.org/x/text v0.7.0 + golang.org/x/text v0.8.0 golang.org/x/time v0.3.0 - google.golang.org/api v0.108.0 // indirect - google.golang.org/genproto v0.0.0-20230131230820-1c016267d619 // indirect - google.golang.org/grpc v1.52.1 + google.golang.org/api v0.114.0 // indirect + google.golang.org/genproto v0.0.0-20230306155012-7f2fa6fef1f4 // indirect + google.golang.org/grpc v1.53.0 google.golang.org/grpc/examples v0.0.0-20211119005141-f45e61797429 gopkg.in/alecthomas/kingpin.v2 v2.2.6 gopkg.in/fsnotify.v1 v1.4.7 @@ -105,9 +104,9 @@ require ( ) require ( - cloud.google.com/go v0.105.0 // indirect - cloud.google.com/go/compute v1.14.0 // indirect - cloud.google.com/go/iam v0.8.0 // indirect + cloud.google.com/go v0.110.0 // indirect + cloud.google.com/go/compute v1.18.0 // indirect + cloud.google.com/go/iam v0.12.0 // indirect github.com/Azure/azure-sdk-for-go/sdk/azcore v1.2.0 // indirect github.com/Azure/azure-sdk-for-go/sdk/azidentity v1.2.0 // indirect github.com/Azure/azure-sdk-for-go/sdk/internal v1.1.1 // indirect @@ -115,15 +114,15 @@ require ( github.com/AzureAD/microsoft-authentication-library-for-go v0.7.0 // indirect go.opentelemetry.io/contrib/samplers/jaegerremote v0.7.0 go.opentelemetry.io/otel/exporters/jaeger v1.12.0 - go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.12.0 - go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracegrpc v1.12.0 - go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracehttp v1.12.0 + go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.14.0 + go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracegrpc v1.14.0 + go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracehttp v1.14.0 ) require ( go.opentelemetry.io/contrib/propagators/autoprop v0.38.0 go4.org/intern v0.0.0-20220617035311-6925f38cc365 - golang.org/x/exp v0.0.0-20230124195608-d38c7dcee874 + golang.org/x/exp v0.0.0-20230307190834-24139beb5833 ) require go4.org/unsafe/assume-no-moving-gc v0.0.0-20230209150437-ee73d164e760 // indirect @@ -136,8 +135,8 @@ require ( github.com/alicebob/gopher-json v0.0.0-20200520072559-a9ecdc9d1d3a // indirect github.com/aliyun/aliyun-oss-go-sdk v2.2.2+incompatible // indirect github.com/armon/go-radix v1.0.0 // indirect - github.com/asaskevich/govalidator v0.0.0-20210307081110-f21760c49a8d // indirect - github.com/aws/aws-sdk-go v1.44.187 // indirect + github.com/asaskevich/govalidator v0.0.0-20230301143203-a9d515a09cc2 // indirect + github.com/aws/aws-sdk-go v1.44.217 // indirect github.com/aws/aws-sdk-go-v2 v1.16.0 // indirect github.com/aws/aws-sdk-go-v2/config v1.15.1 // indirect github.com/aws/aws-sdk-go-v2/credentials v1.11.0 // indirect @@ -154,7 +153,7 @@ require ( github.com/cenkalti/backoff/v4 v4.2.0 // indirect github.com/chromedp/sysutil v1.0.0 // indirect github.com/clbanning/mxj v1.8.4 // indirect - github.com/coreos/go-systemd/v22 v22.4.0 // indirect + github.com/coreos/go-systemd/v22 v22.5.0 // indirect github.com/dennwc/varint v1.0.0 // indirect github.com/dgryski/go-rendezvous v0.0.0-20200823014737-9f7001d12a5f // indirect github.com/dnaeon/go-vcr v1.2.0 // indirect @@ -169,31 +168,30 @@ require ( github.com/go-ole/go-ole v1.2.6 // indirect github.com/go-openapi/analysis v0.21.4 // indirect github.com/go-openapi/errors v0.20.3 // indirect - github.com/go-openapi/jsonpointer v0.19.5 // indirect - github.com/go-openapi/jsonreference v0.20.0 // indirect + github.com/go-openapi/jsonpointer v0.19.6 // indirect + github.com/go-openapi/jsonreference v0.20.2 // indirect github.com/go-openapi/loads v0.21.2 // indirect - github.com/go-openapi/spec v0.20.7 // indirect + github.com/go-openapi/spec v0.20.8 // indirect github.com/go-openapi/swag v0.22.3 // indirect - github.com/go-openapi/validate v0.22.0 // indirect + github.com/go-openapi/validate v0.22.1 // indirect github.com/gobwas/httphead v0.1.0 // indirect github.com/gobwas/pool v0.2.1 // indirect github.com/gobwas/ws v1.1.0 // indirect github.com/gofrs/flock v0.8.1 // indirect github.com/gogo/googleapis v1.4.0 // indirect - github.com/golang-jwt/jwt/v4 v4.4.3 // indirect + github.com/golang-jwt/jwt/v4 v4.5.0 // indirect github.com/google/go-cmp v0.5.9 github.com/google/go-querystring v1.1.0 // indirect - github.com/google/pprof v0.0.0-20230111200839-76d1ae5aea2b // indirect + github.com/google/pprof v0.0.0-20230228050547-1710fef4ab10 // indirect github.com/google/uuid v1.3.0 // indirect - github.com/googleapis/enterprise-certificate-proxy v0.2.1 // indirect - github.com/googleapis/gax-go/v2 v2.7.0 // indirect - github.com/grpc-ecosystem/grpc-gateway/v2 v2.11.1 // indirect + github.com/googleapis/enterprise-certificate-proxy v0.2.3 // indirect + github.com/googleapis/gax-go/v2 v2.7.1 // indirect + github.com/grpc-ecosystem/grpc-gateway/v2 v2.15.2 // indirect github.com/jmespath/go-jmespath v0.4.0 // indirect github.com/joeshaw/multierror v0.0.0-20140124173710-69b34d4ec901 // indirect github.com/josharian/intern v1.0.0 // indirect github.com/julienschmidt/httprouter v1.3.0 // indirect github.com/klauspost/cpuid/v2 v2.1.0 // indirect - github.com/kr/pretty v0.3.0 // indirect github.com/kylelemons/godebug v1.1.0 // indirect github.com/lightstep/lightstep-tracer-common/golang/gogo v0.0.0-20210210170715-a8dfcb80d3a7 // indirect github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0 // indirect @@ -229,25 +227,25 @@ require ( github.com/yusufpapurcu/wmi v1.2.2 // indirect go.elastic.co/apm/module/apmhttp v1.11.0 // indirect go.elastic.co/fastjson v1.1.0 // indirect - go.mongodb.org/mongo-driver v1.11.0 // indirect + go.mongodb.org/mongo-driver v1.11.2 // indirect go.opencensus.io v0.24.0 // indirect - go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.39.0 // indirect + go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.40.0 // indirect go.opentelemetry.io/contrib/propagators/aws v1.13.0 // indirect go.opentelemetry.io/contrib/propagators/b3 v1.13.0 // indirect go.opentelemetry.io/contrib/propagators/jaeger v1.13.0 // indirect - go.opentelemetry.io/otel/exporters/otlp/internal/retry v1.12.0 // indirect - go.opentelemetry.io/otel/metric v0.36.0 // indirect + go.opentelemetry.io/otel/exporters/otlp/internal/retry v1.14.0 // indirect + go.opentelemetry.io/otel/metric v0.37.0 // indirect go.opentelemetry.io/proto/otlp v0.19.0 // indirect go.uber.org/multierr v1.9.0 // indirect - golang.org/x/mod v0.7.0 // indirect - golang.org/x/oauth2 v0.4.0 // indirect - golang.org/x/sys v0.5.0 // indirect - golang.org/x/tools v0.5.0 // indirect + golang.org/x/mod v0.9.0 // indirect + golang.org/x/oauth2 v0.6.0 // indirect + golang.org/x/sys v0.6.0 // indirect + golang.org/x/tools v0.7.0 // indirect golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2 // indirect gonum.org/v1/gonum v0.12.0 // indirect google.golang.org/appengine v1.6.7 // indirect - google.golang.org/protobuf v1.28.1 // indirect - gopkg.in/ini.v1 v1.66.6 // indirect + google.golang.org/protobuf v1.29.1 // indirect + gopkg.in/ini.v1 v1.67.0 // indirect howett.net/plist v0.0.0-20181124034731-591f970eefbb // indirect ) diff --git a/go.sum b/go.sum index 8677491c68..4fead335a6 100644 --- a/go.sum +++ b/go.sum @@ -27,8 +27,8 @@ cloud.google.com/go v0.97.0/go.mod h1:GF7l59pYBVlXQIBLx3a761cZ41F9bBH3JUlihCt2Ud cloud.google.com/go v0.99.0/go.mod h1:w0Xx2nLzqWJPuozYQX+hFfCSI8WioryfRDzkoI/Y2ZA= cloud.google.com/go v0.100.2/go.mod h1:4Xra9TjzAeYHrl5+oeLlzbM2k3mjVhZh4UqTZ//w99A= cloud.google.com/go v0.102.0/go.mod h1:oWcCzKlqJ5zgHQt9YsaeTY9KzIvjyy0ArmiBUgpQ+nc= -cloud.google.com/go v0.105.0 h1:DNtEKRBAAzeS4KyIory52wWHuClNaXJ5x1F7xa4q+5Y= -cloud.google.com/go v0.105.0/go.mod h1:PrLgOJNe5nfE9UMxKxgXj4mD3voiP+YQ6gdt6KMFOKM= +cloud.google.com/go v0.110.0 h1:Zc8gqp3+a9/Eyph2KDmcGaPtbKRIoqq4YTlL4NMD0Ys= +cloud.google.com/go v0.110.0/go.mod h1:SJnCLqQ0FCFGSZMUNUf84MV3Aia54kn7pi8st7tMzaY= cloud.google.com/go/bigquery v1.0.1/go.mod h1:i/xbL2UlR5RvWAURpBYZTtm/cXjCha9lbfbpx4poX+o= cloud.google.com/go/bigquery v1.3.0/go.mod h1:PjpwJnslEMmckchkHFfq+HTD2DmtT67aNFKH1/VBDHE= cloud.google.com/go/bigquery v1.4.0/go.mod h1:S8dzgnTigyfTmLBfrtrhyYhwRxG72rYxvftPBK2Dvzc= @@ -41,17 +41,17 @@ cloud.google.com/go/compute v1.5.0/go.mod h1:9SMHyhJlzhlkJqrPAc839t2BZFTSk6Jdj6m cloud.google.com/go/compute v1.6.0/go.mod h1:T29tfhtVbq1wvAPo0E3+7vhgmkOYeXjhFvz/FMzPu0s= cloud.google.com/go/compute v1.6.1/go.mod h1:g85FgpzFvNULZ+S8AYq87axRKuf2Kh7deLqV/jJ3thU= cloud.google.com/go/compute v1.7.0/go.mod h1:435lt8av5oL9P3fv1OEzSbSUe+ybHXGMPQHHZWZxy9U= -cloud.google.com/go/compute v1.14.0 h1:hfm2+FfxVmnRlh6LpB7cg1ZNU+5edAHmW679JePztk0= -cloud.google.com/go/compute v1.14.0/go.mod h1:YfLtxrj9sU4Yxv+sXzZkyPjEyPBZfXHUvjxega5vAdo= +cloud.google.com/go/compute v1.18.0 h1:FEigFqoDbys2cvFkZ9Fjq4gnHBP55anJ0yQyau2f9oY= +cloud.google.com/go/compute v1.18.0/go.mod h1:1X7yHxec2Ga+Ss6jPyjxRxpu2uu7PLgsOVXvgU0yacs= cloud.google.com/go/compute/metadata v0.2.3 h1:mg4jlk7mCAj6xXp9UJ4fjI9VUI5rubuGBW5aJ7UnBMY= cloud.google.com/go/compute/metadata v0.2.3/go.mod h1:VAV5nSsACxMJvgaAuX6Pk2AawlZn8kiOGuCv6gTkwuA= cloud.google.com/go/datastore v1.0.0/go.mod h1:LXYbyblFSglQ5pkeyhO+Qmw7ukd3C+pD7TKLgZqpHYE= cloud.google.com/go/datastore v1.1.0/go.mod h1:umbIZjpQpHh4hmRpGhH4tLFup+FVzqBi1b3c64qFpCk= cloud.google.com/go/iam v0.3.0/go.mod h1:XzJPvDayI+9zsASAFO68Hk07u3z+f+JrT2xXNdp4bnY= -cloud.google.com/go/iam v0.8.0 h1:E2osAkZzxI/+8pZcxVLcDtAQx/u+hZXVryUaYQ5O0Kk= -cloud.google.com/go/iam v0.8.0/go.mod h1:lga0/y3iH6CX7sYqypWJ33hf7kkfXJag67naqGESjkE= -cloud.google.com/go/longrunning v0.3.0 h1:NjljC+FYPV3uh5/OwWT6pVU+doBqMg2x/rZlE+CamDs= -cloud.google.com/go/monitoring v1.8.0 h1:c9riaGSPQ4dUKWB+M1Fl0N+iLxstMbCktdEwYSPGDvA= +cloud.google.com/go/iam v0.12.0 h1:DRtTY29b75ciH6Ov1PHb4/iat2CLCvrOm40Q0a6DFpE= +cloud.google.com/go/iam v0.12.0/go.mod h1:knyHGviacl11zrtZUoDuYpDgLjvr28sLQaG0YB2GYAY= +cloud.google.com/go/longrunning v0.4.1 h1:v+yFJOfKC3yZdY6ZUI933pIYdhyhV8S3NpWrXWmg7jM= +cloud.google.com/go/monitoring v1.12.0 h1:+X79DyOP/Ny23XIqSIb37AvFWSxDN15w/ktklVvPLso= cloud.google.com/go/pubsub v1.0.1/go.mod h1:R0Gpsv3s54REJCy4fxDixWD93lHJMoZTyQ2kNxGRt3I= cloud.google.com/go/pubsub v1.1.0/go.mod h1:EwwdRX2sKPjnvnqCa270oGRyludottCI76h+R3AArQw= cloud.google.com/go/pubsub v1.2.0/go.mod h1:jhfEVHT8odbXTkndysNHCcx0awwzvfOlguIAii9o8iA= @@ -62,10 +62,10 @@ cloud.google.com/go/storage v1.6.0/go.mod h1:N7U0C8pVQ/+NIKOBQyamJIeKQKkZ+mxpohl cloud.google.com/go/storage v1.8.0/go.mod h1:Wv1Oy7z6Yz3DshWRJFhqM/UCfaWIRTdp0RXyy7KQOVs= cloud.google.com/go/storage v1.10.0/go.mod h1:FLPqc6j+Ki4BU591ie1oL6qBQGu2Bl/tZ9ullr3+Kg0= cloud.google.com/go/storage v1.22.1/go.mod h1:S8N1cAStu7BOeFfE8KAQzmyyLkK8p/vmRq6kuBTW58Y= -cloud.google.com/go/storage v1.27.0 h1:YOO045NZI9RKfCj1c5A/ZtuuENUc8OAW+gHdGnDgyMQ= -cloud.google.com/go/storage v1.27.0/go.mod h1:x9DOL8TK/ygDUMieqwfhdpQryTeEkhGKMi80i/iqR2s= -cloud.google.com/go/trace v1.4.0 h1:qO9eLn2esajC9sxpqp1YKX37nXC3L4BfGnPS0Cx9dYo= -cloud.google.com/go/trace v1.4.0/go.mod h1:UG0v8UBqzusp+z63o7FK74SdFE+AXpCLdFb1rshXG+Y= +cloud.google.com/go/storage v1.28.1 h1:F5QDG5ChchaAVQhINh24U99OWHURqrW8OmQcGKXcbgI= +cloud.google.com/go/storage v1.28.1/go.mod h1:Qnisd4CqDdo6BGs2AD5LLnEsmSQ80wQ5ogcBBKhU86Y= +cloud.google.com/go/trace v1.8.0 h1:GFPLxbp5/FzdgTzor3nlNYNxMd6hLmzkE7sA9F0qQcA= +cloud.google.com/go/trace v1.8.0/go.mod h1:zH7vcsbAhklH8hWFig58HvxcxyQbaIqMarMg9hn5ECA= dmitri.shuralyov.com/gpu/mtl v0.0.0-20190408044501-666a987793e9/go.mod h1:H6x//7gZCb22OMCxBHrMx7a5I7Hp++hsVxbQ4BYO7hU= github.com/Azure/azure-sdk-for-go v65.0.0+incompatible h1:HzKLt3kIwMm4KeJYTdx9EbjRYTySD/t8i1Ee/W5EGXw= github.com/Azure/azure-sdk-for-go/sdk/azcore v1.2.0 h1:sVW/AFBTGyJxDaMYlq0ct3jUXTtj12tQ6zE2GZUgVQw= @@ -88,14 +88,13 @@ github.com/AzureAD/microsoft-authentication-library-for-go v0.7.0 h1:VgSJlZH5u0k github.com/AzureAD/microsoft-authentication-library-for-go v0.7.0/go.mod h1:BDJ5qMFKx9DugEg3+uQSDCdbYPr5s9vBTrL9P8TpqOU= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802/go.mod h1:IVnqGOEym/WlBOVXweHU+Q+/VP0lqqI8lqeDx9IjBqo= -github.com/DataDog/datadog-go v3.2.0+incompatible/go.mod h1:LButxg5PwREeZtORoXG3tL4fMGNddJ+vMq1mwgfaqoQ= github.com/GoogleCloudPlatform/opentelemetry-operations-go/exporter/trace v1.8.3 h1:i84ZOPT35YCJROyuf97VP/VEdYhQce/8NTLOWq5tqJw= github.com/GoogleCloudPlatform/opentelemetry-operations-go/exporter/trace v1.8.3/go.mod h1:3+qm+VCJbVmQ9uscVz+8h1rRkJEy9ZNFGgpT1XB9mPg= github.com/GoogleCloudPlatform/opentelemetry-operations-go/internal/resourcemapping v0.32.3 h1:FhsH8qgWFkkPlPXBZ68uuT/FH/R+DLTtVPxjLEBs1v4= github.com/GoogleCloudPlatform/opentelemetry-operations-go/internal/resourcemapping v0.32.3/go.mod h1:9a+Opaevo9fybhUvQkEG7fR6Zk7pYrW/s9NC4fODFIQ= github.com/HdrHistogram/hdrhistogram-go v1.1.2 h1:5IcZpTvzydCQeHzK4Ef/D5rrSqwxob0t8PQPMybUNFM= github.com/Knetic/govaluate v3.0.1-0.20171022003610-9aa49832a739+incompatible/go.mod h1:r7JcOSlj0wfOMncg0iLm8Leh48TZaKVeNIfJntJ2wa0= -github.com/Microsoft/go-winio v0.5.1 h1:aPJp2QD7OOrhO5tQXqQoGSJc+DjDtWTGLOmNyAm6FgY= +github.com/Microsoft/go-winio v0.6.0 h1:slsWYD/zyx7lCXoZVlvQrj0hPTM1HI4+v1sIda2yDvg= github.com/NYTimes/gziphandler v1.1.1 h1:ZUDjpQae29j0ryrS0u/B8HZfJBtBQHjqw2rQ2cqUQ3I= github.com/NYTimes/gziphandler v1.1.1/go.mod h1:n/CVRwUEOgIxrgPvAQhUUr9oeUtvrhMomdKFjzJNB0c= github.com/OneOfOne/xxhash v1.2.2/go.mod h1:HSdplMjZKSmBqAxg5vPj2TmRDmfkzw+cTzAElWljhcU= @@ -131,21 +130,20 @@ github.com/apache/thrift v0.12.0/go.mod h1:cp2SuWMxlEZw2r+iP2GNCdIi4C1qmUzdZFSVb github.com/apache/thrift v0.13.0/go.mod h1:cp2SuWMxlEZw2r+iP2GNCdIi4C1qmUzdZFSVb+bacwQ= github.com/armon/circbuf v0.0.0-20150827004946-bbbad097214e/go.mod h1:3U/XgcO3hCbHZ8TKRvWD2dDTCfh9M9ya+I9JpbB7O8o= github.com/armon/go-metrics v0.0.0-20180917152333-f0300d1749da/go.mod h1:Q73ZrmVTwzkszR9V5SSuryQ31EELlFMUz1kKyl939pY= -github.com/armon/go-metrics v0.4.0 h1:yCQqn7dwca4ITXb+CbubHmedzaQYHhNhrEXLYUeEe8Q= -github.com/armon/go-metrics v0.4.0/go.mod h1:E6amYzXo6aW1tqzoZGT755KkbgrJsSdpwZ+3JqfkOG4= +github.com/armon/go-metrics v0.4.1 h1:hR91U9KYmb6bLBYLQjyM+3j+rcd/UhE+G78SFnF8gJA= github.com/armon/go-radix v0.0.0-20180808171621-7fddfc383310/go.mod h1:ufUuZ+zHj4x4TnLV4JWEpy2hxWSpsRywHrMgIH9cCH8= github.com/armon/go-radix v1.0.0 h1:F4z6KzEeeQIMeLFa97iZU6vupzoecKdU5TX24SNppXI= github.com/armon/go-radix v1.0.0/go.mod h1:ufUuZ+zHj4x4TnLV4JWEpy2hxWSpsRywHrMgIH9cCH8= github.com/armon/go-socks5 v0.0.0-20160902184237-e75332964ef5/go.mod h1:wHh0iHkYZB8zMSxRWpUBQtwG5a7fFgvEO+odwuTv2gs= github.com/aryann/difflib v0.0.0-20170710044230-e206f873d14a/go.mod h1:DAHtR1m6lCRdSC2Tm3DSWRPvIPr6xNKyeHdqDQSQT+A= github.com/asaskevich/govalidator v0.0.0-20200907205600-7a23bdc65eef/go.mod h1:WaHUgvxTVq04UNunO+XhnAqY/wQc+bxr74GqbsZ/Jqw= -github.com/asaskevich/govalidator v0.0.0-20210307081110-f21760c49a8d h1:Byv0BzEl3/e6D5CLfI0j/7hiIEtvGVFPCZ7Ei2oq8iQ= -github.com/asaskevich/govalidator v0.0.0-20210307081110-f21760c49a8d/go.mod h1:WaHUgvxTVq04UNunO+XhnAqY/wQc+bxr74GqbsZ/Jqw= +github.com/asaskevich/govalidator v0.0.0-20230301143203-a9d515a09cc2 h1:DklsrG3dyBCFEj5IhUbnKptjxatkF07cF2ak3yi77so= +github.com/asaskevich/govalidator v0.0.0-20230301143203-a9d515a09cc2/go.mod h1:WaHUgvxTVq04UNunO+XhnAqY/wQc+bxr74GqbsZ/Jqw= github.com/aws/aws-lambda-go v1.13.3/go.mod h1:4UKl9IzQMoD+QF79YdCuzCwp8VbmG4VAQwij/eHl5CU= github.com/aws/aws-sdk-go v1.27.0/go.mod h1:KmX6BPdI08NWTb3/sm4ZGu5ShLoqVDhKgpiN924inxo= github.com/aws/aws-sdk-go v1.38.35/go.mod h1:hcU610XS61/+aQV88ixoOzUoG7v3b31pl2zKMmprdro= -github.com/aws/aws-sdk-go v1.44.187 h1:D5CsRomPnlwDHJCanL2mtaLIcbhjiWxNh5j8zvaWdJA= -github.com/aws/aws-sdk-go v1.44.187/go.mod h1:aVsgQcEevwlmQ7qHE9I3h+dtQgpqhFB+i8Phjh7fkwI= +github.com/aws/aws-sdk-go v1.44.217 h1:FcWC56MRl+k756aH3qeMQTylSdeJ58WN0iFz3fkyRz0= +github.com/aws/aws-sdk-go v1.44.217/go.mod h1:aVsgQcEevwlmQ7qHE9I3h+dtQgpqhFB+i8Phjh7fkwI= github.com/aws/aws-sdk-go-v2 v0.18.0/go.mod h1:JWVYvqSMppoMJC0x5wdwiImzgXTI9FuZwxzkQq9wy+g= github.com/aws/aws-sdk-go-v2 v1.16.0 h1:cBAYjiiexRAg9v2z9vb6IdxAa7ef4KCtjW7w7e3GxGo= github.com/aws/aws-sdk-go-v2 v1.16.0/go.mod h1:lJYcuZZEHWNIb6ugJjbQY1fykdoobWbOS7kJYb4APoI= @@ -199,8 +197,6 @@ github.com/chromedp/sysutil v1.0.0/go.mod h1:kgWmDdq8fTzXYcKIBqIYvRRTnYb9aNS9moA github.com/chzyer/logex v1.1.10/go.mod h1:+Ywpsq7O8HXn0nuIou7OrIPyXbp3wmkHB+jjWRnGsAI= github.com/chzyer/readline v0.0.0-20180603132655-2972be24d48e/go.mod h1:nSuG5e5PlCu98SY8svDHJxuZscDgtXS6KTTbou5AhLI= github.com/chzyer/test v0.0.0-20180213035817-a1ea475d72b1/go.mod h1:Q3SI9o4m/ZMnBNeIyt5eFwwo7qiLfzFZmjNmxjkiQlU= -github.com/circonus-labs/circonus-gometrics v2.3.1+incompatible/go.mod h1:nmEj6Dob7S7YxXgwXpfOuvO54S+tGdZdw9fuRZt25Ag= -github.com/circonus-labs/circonusllhist v0.1.3/go.mod h1:kMXHVDlOchFAehlya5ePtbp5jckzBHf4XRpQvBOLI+I= github.com/clbanning/mxj v1.8.4 h1:HuhwZtbyvyOw+3Z1AowPkU87JkJUSv751ELWaiTpj8I= github.com/clbanning/mxj v1.8.4/go.mod h1:BVjHeAH+rl9rs6f+QIpeRl0tfu10SXn1pUSa5PVGJng= github.com/clbanning/x2j v0.0.0-20191024224557-825249438eec/go.mod h1:jMjuTZXRI4dUb/I5gc9Hdhagfvm9+RyrPryS/auMzxE= @@ -208,13 +204,14 @@ github.com/cncf/udpa/go v0.0.0-20210930031921-04548b0d99d4/go.mod h1:6pvJx4me5XP github.com/cncf/xds/go v0.0.0-20210805033703-aa0b78936158/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= github.com/cncf/xds/go v0.0.0-20210922020428-25de7278fc84/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= github.com/cncf/xds/go v0.0.0-20211011173535-cb28da3451f1/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= -github.com/cncf/xds/go v0.0.0-20220314180256-7f1daf1720fc h1:PYXxkRUBGUMa5xgMVMDl62vEklZvKpVaxQeN9ie7Hfk= +github.com/cncf/xds/go v0.0.0-20230112175826-46e39c7b9b43 h1:XP+uhjN0yBCN/tPkr8Z0BNDc5rZam9RG6UWyf2FrSQ0= github.com/cockroachdb/datadriven v0.0.0-20190809214429-80d97fb3cbaa/go.mod h1:zn76sxSg3SzpJ0PPJaLDCu+Bu0Lg3sKTORVIj19EIF8= github.com/codahale/hdrhistogram v0.0.0-20161010025455-3a0bb77429bd/go.mod h1:sE/e/2PUdi/liOCUjSTXgM1o87ZssimdTWN964YiIeI= github.com/coreos/go-semver v0.2.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk= github.com/coreos/go-systemd v0.0.0-20180511133405-39ca1b05acc7/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4= -github.com/coreos/go-systemd/v22 v22.4.0 h1:y9YHcjnjynCd/DVbg5j9L/33jQM3MxJlbj/zWskzfGU= github.com/coreos/go-systemd/v22 v22.4.0/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= +github.com/coreos/go-systemd/v22 v22.5.0 h1:RrqgGjYQKalulkV8NGVIfkXQf6YYmOyiJKk8iXXhfZs= +github.com/coreos/go-systemd/v22 v22.5.0/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= github.com/coreos/pkg v0.0.0-20160727233714-3ac0863d7acf/go.mod h1:E3G3o1h8I7cfcXa63jLwjI0eiQQMgzzUDFVpN/nH/eA= github.com/cpuguy83/go-md2man/v2 v2.0.0-20190314233015-f79a8a8ca69d/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU= github.com/creack/pty v1.1.7/go.mod h1:lj5s0c3V2DBrqTV7llrYr5NG6My20zk30Fl46Y7DoTY= @@ -229,11 +226,11 @@ github.com/dennwc/varint v1.0.0/go.mod h1:hnItb35rvZvJrbTALZtY/iQfDs48JKRG1RPpgz github.com/dgrijalva/jwt-go v3.2.0+incompatible/go.mod h1:E3ru+11k8xSBh+hMPgOLZmtrrCbhqsmaPHjLKYnJCaQ= github.com/dgryski/go-rendezvous v0.0.0-20200823014737-9f7001d12a5f h1:lO4WD4F/rVNCu3HqELle0jiPLLBs70cWOduZpkS1E78= github.com/dgryski/go-rendezvous v0.0.0-20200823014737-9f7001d12a5f/go.mod h1:cuUVRXasLTGF7a8hSLbxyZXjz+1KgoB3wDUb6vlszIc= -github.com/digitalocean/godo v1.95.0 h1:S48/byPKui7RHZc1wYEPfRvkcEvToADNb5I3guu95xg= +github.com/digitalocean/godo v1.98.0 h1:potyC1eD0N9n5/P4/WmJuKgg+OGYZOBWEW+/aKTX6QQ= github.com/dnaeon/go-vcr v1.2.0 h1:zHCHvJYTMh1N7xnV7zf1m1GPBF9Ad0Jk/whtQ1663qI= github.com/dnaeon/go-vcr v1.2.0/go.mod h1:R4UdLID7HZT3taECzJs4YgbbH6PIGXB6W/sc5OLb6RQ= github.com/docker/distribution v2.8.1+incompatible h1:Q50tZOPR6T/hjNsyc9g8/syEs6bk8XXApsHjKukMl68= -github.com/docker/docker v20.10.23+incompatible h1:1ZQUUYAdh+oylOT85aA2ZcfRp22jmLhoaEcVEfK8dyA= +github.com/docker/docker v23.0.1+incompatible h1:vjgvJZxprTTE1A37nm+CLNAdwu6xZekyoiVlUZEINcY= github.com/docker/go-connections v0.4.0 h1:El9xVISelRB7BuFusrZozjnkIM5YnzCViNKohAFqRJQ= github.com/docker/go-units v0.5.0 h1:69rxXcBk27SvSaaxTtLh/8llcHD8vYHT7WSdRZ/jvr4= github.com/dustin/go-humanize v0.0.0-20171111073723-bb3d318650d4/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk= @@ -257,15 +254,15 @@ github.com/elastic/go-sysinfo v1.8.1/go.mod h1:JfllUnzoQV/JRYymbH3dO1yggI3mV2oTK github.com/elastic/go-windows v1.0.0/go.mod h1:TsU0Nrp7/y3+VwE82FoZF8gC/XFg/Elz6CcloAxnPgU= github.com/elastic/go-windows v1.0.1 h1:AlYZOldA+UJ0/2nBuqWdo90GFCgG9xuyw9SYzGUtJm0= github.com/elastic/go-windows v1.0.1/go.mod h1:FoVvqWSun28vaDQPbj2Elfc0JahhPB7WQEGa3c814Ss= -github.com/emicklei/go-restful/v3 v3.9.0 h1:XwGDlfxEnQZzuopoqxwSEllNcCOM9DhhFyhFIIGKwxE= +github.com/emicklei/go-restful/v3 v3.10.1 h1:rc42Y5YTp7Am7CS630D7JmhRjq4UlEUuEKfrDac4bSQ= github.com/envoyproxy/go-control-plane v0.9.10-0.20210907150352-cf90f659a021/go.mod h1:AFq3mo9L8Lqqiid3OhADV3RfLJnjiw63cSpi+fDTRC0= -github.com/envoyproxy/go-control-plane v0.10.3 h1:xdCVXxEe0Y3FQith+0cj2irwZudqGYvecuLB1HtdexY= +github.com/envoyproxy/go-control-plane v0.11.0 h1:jtLewhRR2vMRNnq2ZZUoCjUlgut+Y0+sDDWPOfwOi1o= github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= github.com/envoyproxy/protoc-gen-validate v0.9.1 h1:PS7VIOgmSVhWUEeZwTe7z7zouA22Cr590PzXKbZHOVY= github.com/facette/natsort v0.0.0-20181210072756-2cd4dd1e2dcb h1:IT4JYU7k4ikYg1SCxNI1/Tieq/NFvh6dzLdgi7eu0tM= github.com/facette/natsort v0.0.0-20181210072756-2cd4dd1e2dcb/go.mod h1:bH6Xx7IW64qjjJq8M2u4dxNaBiDfKK+z/3eGDpXEQhc= github.com/fatih/color v1.7.0/go.mod h1:Zm6kSWBoL9eyXnKyktHP6abPY2pDugNf5KwzbycvMj4= -github.com/fatih/color v1.13.0 h1:8LOYc1KYPPmyKMuN8QV2DNRWNbLo6LZ0iLs8+mlH53w= +github.com/fatih/color v1.14.1 h1:qfhVLaG5s+nCROl1zJsZRxFeYrHLqWroPOQ8BWiNb4w= github.com/fatih/structtag v1.2.0 h1:/OdNE99OxoI/PqaW/SuSK9uxxT3f/tcSZgon/ssNSx4= github.com/fatih/structtag v1.2.0/go.mod h1:mBJUNpUnHmRKrKlQQlmCrh5PuhftFbNv8Ys4/aAZl94= github.com/felixge/fgprof v0.9.2 h1:tAMHtWMyl6E0BimjVbFt7fieU6FpjttsZN7j0wT5blc= @@ -316,18 +313,20 @@ github.com/go-openapi/errors v0.20.2/go.mod h1:cM//ZKUKyO06HSwqAelJ5NsEMMcpa6VpX github.com/go-openapi/errors v0.20.3 h1:rz6kiC84sqNQoqrtulzaL/VERgkoCyB6WdEkc2ujzUc= github.com/go-openapi/errors v0.20.3/go.mod h1:Z3FlZ4I8jEGxjUK+bugx3on2mIAk4txuAOhlsB1FSgk= github.com/go-openapi/jsonpointer v0.19.3/go.mod h1:Pl9vOtqEWErmShwVjC8pYs9cog34VGT37dQOVbmoatg= -github.com/go-openapi/jsonpointer v0.19.5 h1:gZr+CIYByUqjcgeLXnQu2gHYQC9o73G2XUeOFYEICuY= github.com/go-openapi/jsonpointer v0.19.5/go.mod h1:Pl9vOtqEWErmShwVjC8pYs9cog34VGT37dQOVbmoatg= +github.com/go-openapi/jsonpointer v0.19.6 h1:eCs3fxoIi3Wh6vtgmLTOjdhSpiqphQ+DaPn38N2ZdrE= +github.com/go-openapi/jsonpointer v0.19.6/go.mod h1:osyAmYz/mB/C3I+WsTTSgw1ONzaLJoLCyoi6/zppojs= github.com/go-openapi/jsonreference v0.19.6/go.mod h1:diGHMEHg2IqXZGKxqyvWdfWU/aim5Dprw5bqpKkTvns= -github.com/go-openapi/jsonreference v0.20.0 h1:MYlu0sBgChmCfJxxUKZ8g1cPWFOB37YSZqewK7OKeyA= github.com/go-openapi/jsonreference v0.20.0/go.mod h1:Ag74Ico3lPc+zR+qjn4XBUmXymS4zJbYVCZmcgkasdo= +github.com/go-openapi/jsonreference v0.20.2 h1:3sVjiK66+uXK/6oQ8xgcRKcFgQ5KXa2KvnJRumpMGbE= +github.com/go-openapi/jsonreference v0.20.2/go.mod h1:Bl1zwGIM8/wsvqjsOQLJ/SH+En5Ap4rVB5KVcIDZG2k= github.com/go-openapi/loads v0.21.1/go.mod h1:/DtAMXXneXFjbQMGEtbamCZb+4x7eGwkvZCvBmwUG+g= github.com/go-openapi/loads v0.21.2 h1:r2a/xFIYeZ4Qd2TnGpWDIQNcP80dIaZgf704za8enro= github.com/go-openapi/loads v0.21.2/go.mod h1:Jq58Os6SSGz0rzh62ptiu8Z31I+OTHqmULx5e/gJbNw= github.com/go-openapi/spec v0.20.4/go.mod h1:faYFR1CvsJZ0mNsmsphTMSoRrNV3TEDoAM7FOEWeq8I= github.com/go-openapi/spec v0.20.6/go.mod h1:2OpW+JddWPrpXSCIX8eOx7lZ5iyuWj3RYR6VaaBKcWA= -github.com/go-openapi/spec v0.20.7 h1:1Rlu/ZrOCCob0n+JKKJAWhNWMPW8bOZRg8FJaY+0SKI= -github.com/go-openapi/spec v0.20.7/go.mod h1:2OpW+JddWPrpXSCIX8eOx7lZ5iyuWj3RYR6VaaBKcWA= +github.com/go-openapi/spec v0.20.8 h1:ubHmXNY3FCIOinT8RNrrPfGc9t7I1qhPtdOGoG2AxRU= +github.com/go-openapi/spec v0.20.8/go.mod h1:2OpW+JddWPrpXSCIX8eOx7lZ5iyuWj3RYR6VaaBKcWA= github.com/go-openapi/strfmt v0.21.0/go.mod h1:ZRQ409bWMj+SOgXofQAGTIo2Ebu72Gs+WaRADcS5iNg= github.com/go-openapi/strfmt v0.21.1/go.mod h1:I/XVKeLc5+MM5oPNN7P6urMOpuLXEcNrCX/rPGuWb0k= github.com/go-openapi/strfmt v0.21.3 h1:xwhj5X6CjXEZZHMWy1zKJxvW9AfHC9pkyUjLvHtKG7o= @@ -337,11 +336,11 @@ github.com/go-openapi/swag v0.19.15/go.mod h1:QYRuS/SOXUCsnplDa677K7+DxSOj6IPNl/ github.com/go-openapi/swag v0.21.1/go.mod h1:QYRuS/SOXUCsnplDa677K7+DxSOj6IPNl/eQntq43wQ= github.com/go-openapi/swag v0.22.3 h1:yMBqmnQ0gyZvEb/+KzuWZOXgllrXT4SADYbvDaXHv/g= github.com/go-openapi/swag v0.22.3/go.mod h1:UzaqsxGiab7freDnrUUra0MwWfN/q7tE4j+VcZ0yl14= -github.com/go-openapi/validate v0.22.0 h1:b0QecH6VslW/TxtpKgzpO1SNG7GU2FsaqKdP1E2T50Y= -github.com/go-openapi/validate v0.22.0/go.mod h1:rjnrwK57VJ7A8xqfpAOEKRH8yQSGUriMu5/zuPSQ1hg= +github.com/go-openapi/validate v0.22.1 h1:G+c2ub6q47kfX1sOBLwIQwzBVt8qmOAARyo/9Fqs9NU= +github.com/go-openapi/validate v0.22.1/go.mod h1:rjnrwK57VJ7A8xqfpAOEKRH8yQSGUriMu5/zuPSQ1hg= github.com/go-redis/redis/v8 v8.11.5 h1:AcZZR7igkdvfVmQTPnu9WE37LRrO/YrBH5zWyjDC0oI= github.com/go-redis/redis/v8 v8.11.5/go.mod h1:gREzHqY1hg6oD9ngVRbLStwAWKhA0FEgq8Jd4h5lpwo= -github.com/go-resty/resty/v2 v2.1.1-0.20191201195748-d7b97669fe48 h1:JVrqSeQfdhYRFk24TvhTZWU0q8lfCojxZQFi3Ou7+uY= +github.com/go-resty/resty/v2 v2.7.0 h1:me+K9p3uhSmXtrBZ4k9jcEAfJmuC8IivWHwaLZwPrFY= github.com/go-sql-driver/mysql v1.4.0/go.mod h1:zAC/RDZ24gD3HViQzih4MyKcchzm+sOG5ZlKdlhCg5w= github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= github.com/go-zookeeper/zk v1.0.3 h1:7M2kwOsc//9VeeFiPtf+uSJlVpU66x9Ba5+8XK7/TDg= @@ -392,8 +391,8 @@ github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69 github.com/gogo/status v1.0.3/go.mod h1:SavQ51ycCLnc7dGyJxp8YAmudx8xqiVrRf+6IXRsugc= github.com/gogo/status v1.1.1 h1:DuHXlSFHNKqTQ+/ACf5Vs6r4X/dH2EgIzR9Vr+H65kg= github.com/gogo/status v1.1.1/go.mod h1:jpG3dM5QPcqu19Hg8lkUhBFBa3TcLs1DG7+2Jqci7oU= -github.com/golang-jwt/jwt/v4 v4.4.3 h1:Hxl6lhQFj4AnOX6MLrsCb/+7tCj7DxP7VA+2rDIq5AU= -github.com/golang-jwt/jwt/v4 v4.4.3/go.mod h1:m21LjoU+eqJr34lmDMbreY2eSTRJ1cv77w39/MY0Ch0= +github.com/golang-jwt/jwt/v4 v4.5.0 h1:7cYmW1XlMY7h7ii7UhUyChSgS5wUJEnm9uZVTGqOWzg= +github.com/golang-jwt/jwt/v4 v4.5.0/go.mod h1:m21LjoU+eqJr34lmDMbreY2eSTRJ1cv77w39/MY0Ch0= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= github.com/golang/glog v1.0.0 h1:nfP3RFugxnNRyKgeWd4oI1nYvXpxrx8ck8ZrcizshdQ= github.com/golang/glog v1.0.0/go.mod h1:EWib/APOK0SL3dFbYqvxE3UYd8E6s1ouQ7iEp/0LWV4= @@ -427,8 +426,9 @@ github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw github.com/golang/protobuf v1.4.3/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= github.com/golang/protobuf v1.5.1/go.mod h1:DopwsBzvsk0Fs44TXzsVbJyPhcCPeIwnvohx4u74HPM= -github.com/golang/protobuf v1.5.2 h1:ROPKBNFfQgOUMifHyP+KYbvpjbdoFNs+aK7DXlji0Tw= github.com/golang/protobuf v1.5.2/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= +github.com/golang/protobuf v1.5.3 h1:KhyjKVUg7Usr/dYsdSqoFveMYd5ko72D+zANwlG1mmg= +github.com/golang/protobuf v1.5.3/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= github.com/golang/snappy v0.0.0-20180518054509-2e65f85255db/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/golang/snappy v0.0.1/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/golang/snappy v0.0.3/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= @@ -436,7 +436,7 @@ github.com/golang/snappy v0.0.4 h1:yAGX7huGHXlcLOEtBnF4w7FQwA26wojNCwOYAEhLjQM= github.com/golang/snappy v0.0.4/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= github.com/google/btree v1.0.0/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= -github.com/google/gnostic v0.5.7-v3refs h1:FhTMOKj2VhjpouxvWJAV1TL304uMlb9zcDqkl6cEI54= +github.com/google/gnostic v0.6.9 h1:ZK/5VhkoX835RikCHpSUJV9a+S3e1zLh59YnyWeBW+0= github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= @@ -462,8 +462,8 @@ github.com/google/martian v2.1.0+incompatible h1:/CP5g8u/VJHijgedC/Legn3BAbAaWPg github.com/google/martian v2.1.0+incompatible/go.mod h1:9I4somxYTbIHy5NJKHRl3wXiIaQGbYVAs8BPL6v8lEs= github.com/google/martian/v3 v3.0.0/go.mod h1:y5Zk1BBys9G+gd6Jrk0W3cC1+ELVxBWuIGO+w/tUAp0= github.com/google/martian/v3 v3.1.0/go.mod h1:y5Zk1BBys9G+gd6Jrk0W3cC1+ELVxBWuIGO+w/tUAp0= -github.com/google/martian/v3 v3.2.1 h1:d8MncMlErDFTwQGBK1xhv026j9kqhvw1Qv9IbWT1VLQ= github.com/google/martian/v3 v3.2.1/go.mod h1:oBOf6HBosgwRXnUGWUB05QECsc6uvmMiJ3+6W4l/CUk= +github.com/google/martian/v3 v3.3.2 h1:IqNFLAmvJOgVlpdEBiQbDc2EwKW77amAycfTuWKdfvw= github.com/google/pprof v0.0.0-20181206194817-3ea8567a2e57/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= github.com/google/pprof v0.0.0-20190515194954-54271f7e092f/go.mod h1:zfwlbNMJ+OItoe0UupaVj+oy1omPYYDuagoSzA8v9mc= github.com/google/pprof v0.0.0-20191218002539-d4f498aebedc/go.mod h1:ZgVRPoUq/hfqzAqh7sHMqb3I9Rq5C59dIz2SbBwJ4eM= @@ -479,8 +479,8 @@ github.com/google/pprof v0.0.0-20210601050228-01bbb1931b22/go.mod h1:kpwsk12EmLe github.com/google/pprof v0.0.0-20210609004039-a478d1d731e9/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= github.com/google/pprof v0.0.0-20210720184732-4bb14d4b1be1/go.mod h1:kpwsk12EmLew5upagYY7GY0pfYCcupk39gWOCRROcvE= github.com/google/pprof v0.0.0-20211214055906-6f57359322fd/go.mod h1:KgnwoLYCZ8IQu3XUZ8Nc/bM9CCZFOyjUNOSygVozoDg= -github.com/google/pprof v0.0.0-20230111200839-76d1ae5aea2b h1:8htHrh2bw9c7Idkb7YNac+ZpTqLMjRpI+FWu51ltaQc= -github.com/google/pprof v0.0.0-20230111200839-76d1ae5aea2b/go.mod h1:dDKJzRmX4S37WGHujM7tX//fmj1uioxKzKxz3lo4HJo= +github.com/google/pprof v0.0.0-20230228050547-1710fef4ab10 h1:CqYfpuYIjnlNxM3msdyPRKabhXZWbKjf3Q8BWROFBso= +github.com/google/pprof v0.0.0-20230228050547-1710fef4ab10/go.mod h1:79YE0hCXdHag9sBkw2o+N/YnZtTkXi0UT9Nnixa5eYk= github.com/google/renameio v0.1.0/go.mod h1:KWCgfxg9yswjAJkECMjeO8J8rahYeXnNhOm40UhjYkI= github.com/google/uuid v1.0.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/google/uuid v1.1.1/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= @@ -488,8 +488,8 @@ github.com/google/uuid v1.1.2/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+ github.com/google/uuid v1.3.0 h1:t6JiXgmwXMjEs8VusXIJk2BXHsn+wx8BZdTaoZ5fu7I= github.com/google/uuid v1.3.0/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/googleapis/enterprise-certificate-proxy v0.0.0-20220520183353-fd19c99a87aa/go.mod h1:17drOmN3MwGY7t0e+Ei9b45FFGA3fBs3x36SsCg1hq8= -github.com/googleapis/enterprise-certificate-proxy v0.2.1 h1:RY7tHKZcRlk788d5WSo/e83gOyyy742E8GSs771ySpg= -github.com/googleapis/enterprise-certificate-proxy v0.2.1/go.mod h1:AwSRAtLfXpU5Nm3pW+v7rGDHp09LsPtGY9MduiEsR9k= +github.com/googleapis/enterprise-certificate-proxy v0.2.3 h1:yk9/cqRKtT9wXZSsRH9aurXEpJX+U6FLtpYTdC3R06k= +github.com/googleapis/enterprise-certificate-proxy v0.2.3/go.mod h1:AwSRAtLfXpU5Nm3pW+v7rGDHp09LsPtGY9MduiEsR9k= github.com/googleapis/gax-go v2.0.2+incompatible h1:silFMLAnr330+NRuag/VjIGF7TLp/LBrV2CJKFLWEww= github.com/googleapis/gax-go v2.0.2+incompatible/go.mod h1:SFVmujtThgffbyetf+mdk2eWhX2bMyUtNHzFKcPA9HY= github.com/googleapis/gax-go/v2 v2.0.4/go.mod h1:0Wqv26UfaUD9n4G6kQubkQ+KchISgw+vpHVxEJEs9eg= @@ -499,11 +499,11 @@ github.com/googleapis/gax-go/v2 v2.1.1/go.mod h1:hddJymUZASv3XPyGkUpKj8pPO47Rmb0 github.com/googleapis/gax-go/v2 v2.2.0/go.mod h1:as02EH8zWkzwUoLbBaFeQ+arQaj/OthfcblKl4IGNaM= github.com/googleapis/gax-go/v2 v2.3.0/go.mod h1:b8LNqSzNabLiUpXKkY7HAR5jr6bIT99EXz9pXxye9YM= github.com/googleapis/gax-go/v2 v2.4.0/go.mod h1:XOTVJ59hdnfJLIP/dh8n5CGryZR2LxK9wbMD5+iXC6c= -github.com/googleapis/gax-go/v2 v2.7.0 h1:IcsPKeInNvYi7eqSaDjiZqDDKu5rsmunY0Y1YupQSSQ= -github.com/googleapis/gax-go/v2 v2.7.0/go.mod h1:TEop28CZZQ2y+c0VxMUmu1lV+fQx57QpBWsYpwqHJx8= +github.com/googleapis/gax-go/v2 v2.7.1 h1:gF4c0zjUP2H/s/hEGyLA3I0fA2ZWjzYiONAD6cvPr8A= +github.com/googleapis/gax-go/v2 v2.7.1/go.mod h1:4orTrqY6hXxxaUL4LHIPl6lGo8vAE38/qKbhSAKP6QI= github.com/googleapis/go-type-adapters v1.0.0/go.mod h1:zHW75FOG2aur7gAO2B+MLby+cLsWGBF62rFAi7WjWO4= github.com/googleinterns/cloud-operations-api-mock v0.0.0-20200709193332-a1e58c29bdd3 h1:eHv/jVY/JNop1xg2J9cBb4EzyMpWZoNCP1BslSAIkOI= -github.com/gophercloud/gophercloud v1.1.1 h1:MuGyqbSxiuVBqkPZ3+Nhbytk1xZxhmfCB2Rg1cJWFWM= +github.com/gophercloud/gophercloud v1.2.0 h1:1oXyj4g54KBg/kFtCdMM6jtxSzeIyg8wv4z1HoGPp1E= github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= github.com/gorilla/context v1.1.1/go.mod h1:kBGZzfjB9CEq2AlWe17Uuf7NDRt0dE0s8S51q0aT7Yg= github.com/gorilla/mux v1.6.2/go.mod h1:1lud6UwP+6orDFRuTfBEV8e9/aOM/c4fVVCaMa2zaAs= @@ -528,26 +528,24 @@ github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0/go.mod h1:8NvIoxWQoOIhqOTXgf github.com/grpc-ecosystem/grpc-gateway v1.9.5/go.mod h1:vNeuVxBJEsws4ogUvrchl83t/GYV9WGTSLVdBhOQFDY= github.com/grpc-ecosystem/grpc-gateway v1.16.0/go.mod h1:BDjrQk3hbvj6Nolgz8mAMFbcEtjT1g+wF4CSlocrBnw= github.com/grpc-ecosystem/grpc-gateway/v2 v2.7.0/go.mod h1:hgWBS7lorOAVIJEQMi4ZsPv9hVvWI6+ch50m39Pf2Ks= -github.com/grpc-ecosystem/grpc-gateway/v2 v2.11.1 h1:/sDbPb60SusIXjiJGYLUoS/rAQurQmvGWmwn2bBPM9c= -github.com/grpc-ecosystem/grpc-gateway/v2 v2.11.1/go.mod h1:G+WkljZi4mflcqVxYSgvt8MNctRQHjEH8ubKtt1Ka3w= +github.com/grpc-ecosystem/grpc-gateway/v2 v2.15.2 h1:gDLXvp5S9izjldquuoAhDzccbskOL6tDC5jMSyx3zxE= +github.com/grpc-ecosystem/grpc-gateway/v2 v2.15.2/go.mod h1:7pdNwVWBBHGiCxa9lAszqCJMbfTISJ7oMftp8+UGV08= github.com/grpc-ecosystem/grpc-opentracing v0.0.0-20180507213350-8e809c8a8645/go.mod h1:6iZfnjpejD4L/4DwD7NryNaJyCQdzwWwH2MWhCA90Kw= github.com/hashicorp/consul/api v1.3.0/go.mod h1:MmDNSzIMUjNpY/mQ398R4bk2FnqQLoPndWW5VkKPlCE= -github.com/hashicorp/consul/api v1.18.0 h1:R7PPNzTCeN6VuQNDwwhZWJvzCtGSrNpJqfb22h3yH9g= +github.com/hashicorp/consul/api v1.20.0 h1:9IHTjNVSZ7MIwjlW3N3a7iGiykCMDpxZu8jsxFJh0yc= github.com/hashicorp/consul/sdk v0.3.0/go.mod h1:VKf9jXwCTEY1QZP2MOLRhb5i/I/ssyNV1vwHyQBF0x8= github.com/hashicorp/cronexpr v1.1.1 h1:NJZDd87hGXjoZBdvyCF9mX4DCq5Wy7+A/w+A7q0wn6c= -github.com/hashicorp/errwrap v1.0.0 h1:hLrqtEDnRye3+sgx6z4qVLNuviH3MR5aQ0ykNJa/UYA= github.com/hashicorp/errwrap v1.0.0/go.mod h1:YH+1FKiLXxHSkmPseP+kNlulaMuP3n2brvKWEqk/Jc4= -github.com/hashicorp/go-cleanhttp v0.5.0/go.mod h1:JpRdi6/HCYpAwUzNwuwqhbovhLtngrth3wmdIIUrZ80= +github.com/hashicorp/errwrap v1.1.0 h1:OxrOeh75EUXMY8TBjag2fzXGZ40LB6IKw45YeGUDY2I= github.com/hashicorp/go-cleanhttp v0.5.1/go.mod h1:JpRdi6/HCYpAwUzNwuwqhbovhLtngrth3wmdIIUrZ80= github.com/hashicorp/go-cleanhttp v0.5.2 h1:035FKYIWjmULyFRBKPs8TBQoi0x6d9G4xc9neXJWAZQ= -github.com/hashicorp/go-hclog v0.16.2 h1:K4ev2ib4LdQETX5cSZBG0DVLk1jwGqSPXBjdah3veNs= +github.com/hashicorp/go-hclog v1.4.0 h1:ctuWFGrhFha8BnnzxqeRGidlEcQkDyL5u8J8t5eA11I= github.com/hashicorp/go-immutable-radix v1.0.0/go.mod h1:0y9vanUI8NX6FsYoO3zeMjhV/C5i9g4Q3DwcSNZ4P60= github.com/hashicorp/go-immutable-radix v1.3.1 h1:DKHmCUm2hRBK510BaiZlwvpD40f8bJFeZnpfm2KLowc= github.com/hashicorp/go-msgpack v0.5.3/go.mod h1:ahLV/dePpqEmjfWmKiqvPkv/twdG7iPBM1vqhUKIvfM= github.com/hashicorp/go-multierror v1.0.0/go.mod h1:dHtQlpGsu+cZNNAkkCN/P3hoUDHhCYQXV3UM06sGGrk= github.com/hashicorp/go-multierror v1.1.1 h1:H5DkEtf6CXdFp0N0Em5UCwQpXMWke8IA0+lD48awMYo= -github.com/hashicorp/go-retryablehttp v0.5.3/go.mod h1:9B5zBasrRhHXnJnui7y6sL7es7NDiJgTc6Er0maI1Xs= -github.com/hashicorp/go-retryablehttp v0.7.1 h1:sUiuQAnLlbvmExtFQs72iFW/HXeUn8Z1aJLQ4LJJbTQ= +github.com/hashicorp/go-retryablehttp v0.7.2 h1:AcYqCvkpalPnPF2pn0KamgwamS42TqUDDYFRKq/RAd0= github.com/hashicorp/go-rootcerts v1.0.0/go.mod h1:K6zTfqpRlCUIjkwsN4Z+hiSfzSTQa6eBIzfwKfwNnHU= github.com/hashicorp/go-rootcerts v1.0.2 h1:jzhAVGtqPKbwpyCPELlgNWhE1znq+qwJtW5Oi2viEzc= github.com/hashicorp/go-sockaddr v1.0.0/go.mod h1:7Xibr9yA9JjQq1JpNB2Vw7kxv8xerXegt+ozgdvDeDU= @@ -563,19 +561,19 @@ github.com/hashicorp/golang-lru v0.6.0/go.mod h1:iADmTwqILo4mZ8BN3D2Q6+9jd8WM5uG github.com/hashicorp/logutils v1.0.0/go.mod h1:QIAnNjmIWmVIIkWDTG1z5v++HQmx9WQRO+LraFDTW64= github.com/hashicorp/mdns v1.0.0/go.mod h1:tL+uN++7HEJ6SQLQ2/p+z2pH24WQKWjBPkE0mNTz8vQ= github.com/hashicorp/memberlist v0.1.3/go.mod h1:ajVTdAv/9Im8oMAAj5G31PhhMCZJV2pPBoIllUwCN7I= -github.com/hashicorp/nomad/api v0.0.0-20230124213148-69fd1a0e4bf7 h1:XOdd3JHyeQnBRxotBo9ibxBFiYGuYhQU25s/YeV2cTU= +github.com/hashicorp/nomad/api v0.0.0-20230308192510-48e7d70fcd4b h1:EkuSTU8c/63q4LMayj8ilgg/4I5PXDFVcnqKfs9qcwI= github.com/hashicorp/serf v0.8.2/go.mod h1:6hOLApaqBFA1NXqRQAsxw9QxuDEvNxSQRwA/JwenrHc= github.com/hashicorp/serf v0.10.1 h1:Z1H2J60yRKvfDYAOZLd2MU0ND4AH/WDz7xYHDWQsIPY= -github.com/hetznercloud/hcloud-go v1.39.0 h1:RUlzI458nGnPR6dlcZlrsGXYC1hQlFbKdm8tVtEQQB0= +github.com/hetznercloud/hcloud-go v1.41.0 h1:KJGFRRc68QiVu4PrEP5BmCQVveCP2CM26UGQUKGpIUs= github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU= github.com/hudl/fargo v1.3.0/go.mod h1:y3CKSmjA+wD2gak7sUSXTAoopbhU08POFhmITJgmKTg= github.com/ianlancetaylor/demangle v0.0.0-20181102032728-5e5cf60278f6/go.mod h1:aSSvb/t6k1mPoxDqO4vJh6VOCGPwU4O0C2/Eqndh1Sc= github.com/ianlancetaylor/demangle v0.0.0-20200824232613-28f6c0f3b639/go.mod h1:aSSvb/t6k1mPoxDqO4vJh6VOCGPwU4O0C2/Eqndh1Sc= github.com/ianlancetaylor/demangle v0.0.0-20210905161508-09a460cdf81d/go.mod h1:aYm2/VgdVmcIU8iMfdMvDMsRAQjcfZSKFby6HOFvi/w= -github.com/imdario/mergo v0.3.12 h1:b6R2BslTbIEToALKP7LxUvijTsNI9TAe80pLWN2g/HU= +github.com/imdario/mergo v0.3.13 h1:lFzP57bqS/wsqKssCGmtLAb8A0wKjLGrve2q3PPVcBk= github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8= github.com/influxdata/influxdb1-client v0.0.0-20191209144304-8bf82d3c094d/go.mod h1:qj24IKcXYK6Iy9ceXlo3Tc+vtHo9lIhSX5JddghvEPo= -github.com/ionos-cloud/sdk-go/v6 v6.1.3 h1:vb6yqdpiqaytvreM0bsn2pXw+1YDvEk2RKSmBAQvgDQ= +github.com/ionos-cloud/sdk-go/v6 v6.1.5 h1:BFqThLOgrGJWeo7w6UDyYuNxyi/GqEmNPl7C/YcQ8Fw= github.com/jessevdk/go-flags v1.4.0/go.mod h1:4FA24M0QyGHXBuZZK/XkWh8h0e1EYbRYJSGM75WSRxI= github.com/jmespath/go-jmespath v0.0.0-20180206201540-c2b33e8439af/go.mod h1:Nht3zPeWKUH0NzdCt2Blrr5ys8VGpn0CEB0cQHVjt7k= github.com/jmespath/go-jmespath v0.4.0 h1:BEgLn5cpjn8UN1mAw4NjwDrS35OdebyEtFe+9YPoQUg= @@ -593,7 +591,6 @@ github.com/jpillora/backoff v1.0.0/go.mod h1:J/6gKK9jxlEcS3zixgDgUAsiuZ7yrSoa/FX github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= github.com/json-iterator/go v1.1.7/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= github.com/json-iterator/go v1.1.8/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= -github.com/json-iterator/go v1.1.9/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= github.com/json-iterator/go v1.1.10/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= github.com/json-iterator/go v1.1.11/go.mod h1:KdQUCv79m/52Kvf8AW2vK1V8akMuk1QjK/uOdHXbAo4= github.com/json-iterator/go v1.1.12 h1:PV8peI4a0ysnczrg+LtxykD8LfKY9ML6u2jnxaEnrnM= @@ -623,8 +620,8 @@ github.com/konsorten/go-windows-terminal-sequences v1.0.2/go.mod h1:T0+1ngSBFLxv github.com/konsorten/go-windows-terminal-sequences v1.0.3/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= -github.com/kr/pretty v0.3.0 h1:WgNl7dwNpEZ6jJ9k1snq4pZsg7DOEN8hP9Xw0Tsjwk0= -github.com/kr/pretty v0.3.0/go.mod h1:640gp4NfQd8pI5XOwp5fnNeVWj67G7CFk/SaSQn7NBk= +github.com/kr/pretty v0.2.1/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= +github.com/kr/pretty v0.3.1 h1:flRD4NNwYAUpkphVc1HcthR4KEIFJ65n8Mw5qdRn3LE= github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= @@ -639,7 +636,7 @@ github.com/lightstep/lightstep-tracer-common/golang/gogo v0.0.0-20210210170715-a github.com/lightstep/lightstep-tracer-go v0.18.1/go.mod h1:jlF1pusYV4pidLvZ+XD0UBX0ZE6WURAspgAczcDHrL4= github.com/lightstep/lightstep-tracer-go v0.25.0 h1:sGVnz8h3jTQuHKMbUe2949nXm3Sg09N1UcR3VoQNN5E= github.com/lightstep/lightstep-tracer-go v0.25.0/go.mod h1:G1ZAEaqTHFPWpWunnbUn1ADEY/Jvzz7jIOaXwAfD6A8= -github.com/linode/linodego v1.12.0 h1:33mOIrZ+gVva14gyJMKPZ85mQGovAvZCEP1ftgmFBjA= +github.com/linode/linodego v1.14.1 h1:uGxQyy0BidoEpLGdvfi4cPgEW+0YUFsEGrLEhcTfjNc= github.com/lovoo/gcloud-opentracing v0.3.0 h1:nAeKG70rIsog0TelcEtt6KU0Y1s5qXtsDLnHp0urPLU= github.com/lovoo/gcloud-opentracing v0.3.0/go.mod h1:ZFqk2y38kMDDikZPAK7ynTTGuyt17nSPdS3K5e+ZTBY= github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0 h1:6E+4a0GO5zZEnZ81pIr0yLvtUWk2if982qA3F3QD6H4= @@ -652,10 +649,10 @@ github.com/mailru/easyjson v0.7.7/go.mod h1:xzfreul335JAWq5oZzymOObrkdz5UnU4kGfJ github.com/markbates/oncer v0.0.0-20181203154359-bf2de49a0be2/go.mod h1:Ld9puTsIW75CHf65OeIOkyKbteujpZVXDpWK6YGZbxE= github.com/markbates/safe v1.0.1/go.mod h1:nAqgmRi7cY2nqMc92/bSEeQA+R4OheNU2T1kNSCBdG0= github.com/mattn/go-colorable v0.0.9/go.mod h1:9vuHe8Xs5qXnSaW/c/ABM9alt+Vo+STaOChaDxuIBZU= -github.com/mattn/go-colorable v0.1.12 h1:jF+Du6AlPIjs2BiUiQlKOX0rt3SujHxPnksPKZbaA40= +github.com/mattn/go-colorable v0.1.13 h1:fFA4WZxdEF4tXPZVKMLwD8oUnCTTo08duU7wxecdEvA= github.com/mattn/go-isatty v0.0.3/go.mod h1:M+lRXTBqGeGNdLjl/ufCoiOlB5xdOkqRJdNxMWT7Zi4= github.com/mattn/go-isatty v0.0.4/go.mod h1:M+lRXTBqGeGNdLjl/ufCoiOlB5xdOkqRJdNxMWT7Zi4= -github.com/mattn/go-isatty v0.0.14 h1:yVuAays6BHfxijgZPzw+3Zlu5yQgKGP2/hcQbHb7S9Y= +github.com/mattn/go-isatty v0.0.17 h1:BTarxUcIeDqL27Mc+vyvdWYSL28zpIhv3RoTdsLMPng= github.com/mattn/go-runewidth v0.0.2/go.mod h1:LwmH8dsx7+W8Uxz3IHJYH5QSwggIsqBzpuz5H//U1FU= github.com/mattn/go-runewidth v0.0.9/go.mod h1:H031xJmbD/WCDINGzjvQ9THkh0rPKHF+m2gUSrubnMI= github.com/mattn/go-runewidth v0.0.13 h1:lTGmDsbAYt5DmK6OnoV7EuIF1wEIFAcxld6ypU4OSgU= @@ -665,8 +662,8 @@ github.com/matttproud/golang_protobuf_extensions v1.0.4 h1:mmDVorXM7PCGKw94cs5zk github.com/matttproud/golang_protobuf_extensions v1.0.4/go.mod h1:BSXmuO+STAnVfrANrmjBb36TMTDstsz7MSK+HVaYKv4= github.com/mgutz/ansi v0.0.0-20170206155736-9520e82c474b/go.mod h1:01TrycV0kFyexm33Z7vhZRXopbI8J3TDReVlkTgMUxE= github.com/miekg/dns v1.0.14/go.mod h1:W1PPwlIAgtquWBMBEV9nkV9Cazfe8ScdGz/Lj7v3Nrg= -github.com/miekg/dns v1.1.50 h1:DQUfb9uc6smULcREF09Uc+/Gd46YWqJd5DbpPE9xkcA= -github.com/miekg/dns v1.1.50/go.mod h1:e3IlAVfNqAllflbibAZEWOXOQ+Ynzk/dDozDxY7XnME= +github.com/miekg/dns v1.1.53 h1:ZBkuHr5dxHtB1caEOlZTLPo7D3L3TWckgUUs/RHfDxw= +github.com/miekg/dns v1.1.53/go.mod h1:uInx36IzPl7FYnDcMeVWxj9byh7DutNykX4G9Sj60FY= github.com/minio/md5-simd v1.1.2 h1:Gdi1DZK69+ZVMoNHRXJyNcxrMA4dSxoYHZSQbirFg34= github.com/minio/md5-simd v1.1.2/go.mod h1:MzdKDxYpY2BT9XQFocsiZf/NKVtR7nkE4RoEpN+20RM= github.com/minio/minio-go/v7 v7.0.45 h1:g4IeM9M9pW/Lo8AGGNOjBZYlvmtlE1N5TQEYWXRWzIs= @@ -752,7 +749,6 @@ github.com/orisano/pixelmatch v0.0.0-20210112091706-4fa4c7ba91d5 h1:1SoBaSPudixR github.com/ovh/go-ovh v1.3.0 h1:mvZaddk4E4kLcXhzb+cxBsMPYp2pHqiQpWYkInsuZPQ= github.com/pact-foundation/pact-go v1.0.4/go.mod h1:uExwJY4kCzNPcHRj+hCR/HBbOOIwwtUjcrb0b5/5kLM= github.com/pascaldekloe/goe v0.0.0-20180627143212-57f6aae5913c/go.mod h1:lzWF7FIEvWOWxwDKqyGYQf6ZUaNfKdP144TG7ZOy1lc= -github.com/pascaldekloe/goe v0.1.0/go.mod h1:lzWF7FIEvWOWxwDKqyGYQf6ZUaNfKdP144TG7ZOy1lc= github.com/pborman/uuid v1.2.0/go.mod h1:X/NO0urCmaxf9VXbdlT7C2Yzkj2IKimNn4k+gtPdI/k= github.com/pelletier/go-toml v1.7.0/go.mod h1:vwGMzjaWMwyfHwgIBhI2YUM4fB6nL6lVAvS1LBMMhTE= github.com/performancecopilot/speed v3.0.0+incompatible/go.mod h1:/CLtqpZ5gBg1M9iaPbIdPPGyKcA8hKdoy6hAWba7Yac= @@ -778,7 +774,6 @@ github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXP github.com/prometheus/client_golang v0.9.3-0.20190127221311-3c4408c8b829/go.mod h1:p2iRAGwDERtqlqzRXnrOVns+ignqQo//hLXqYxZYVNs= github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5FsnadC4Ky3P0J6CfImo= github.com/prometheus/client_golang v1.3.0/go.mod h1:hJaj2vgQTGQmVCsAACORcieXFeDPbaTKGT+JTgUa3og= -github.com/prometheus/client_golang v1.4.0/go.mod h1:e9GMxYsXl05ICDXkRhurwBS4Q3OK1iX/F2sw+iXX5zU= github.com/prometheus/client_golang v1.7.1/go.mod h1:PY5Wy2awLA44sXw4AOSfFBetzPP4j5+D6mVACh+pe2M= github.com/prometheus/client_golang v1.11.0/go.mod h1:Z6t4BnS23TR94PD6BsDNk8yVqroYurpAkEiz0P2BEV0= github.com/prometheus/client_golang v1.12.1/go.mod h1:3Z9XVyYiZYEO+YQWt3RD2R3jrbd179Rt297l4aS6nDY= @@ -796,18 +791,18 @@ github.com/prometheus/client_model v0.3.0/go.mod h1:LDGWKZIo7rky3hgvBe+caln+Dr3d github.com/prometheus/common v0.2.0/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= github.com/prometheus/common v0.4.1/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= github.com/prometheus/common v0.7.0/go.mod h1:DjGbpBbp5NYNiECxcL/VnbXCCaQpKd3tt26CguLLsqA= -github.com/prometheus/common v0.9.1/go.mod h1:yhUN8i9wzaXS3w1O07YhxHEBxD+W35wd8bs7vj7HSQ4= github.com/prometheus/common v0.10.0/go.mod h1:Tlit/dnDKsSWFlCLTWaA1cyBgKHSMdTB80sz/V91rCo= github.com/prometheus/common v0.26.0/go.mod h1:M7rCNAaPfAosfx8veZJCuw84e35h3Cfd9VFqTh1DIvc= github.com/prometheus/common v0.29.0/go.mod h1:vu+V0TpY+O6vW9J44gczi3Ap/oXXR10b+M/gUGO4Hls= github.com/prometheus/common v0.32.1/go.mod h1:vu+V0TpY+O6vW9J44gczi3Ap/oXXR10b+M/gUGO4Hls= github.com/prometheus/common v0.37.0/go.mod h1:phzohg0JFMnBEFGxTDbfu3QyL5GI8gTQJFhYO5B3mfA= -github.com/prometheus/common v0.39.1-0.20230202092144-f9c1994be032 h1:OKL/rHv39LgZmRQZZcUOCzl0UuFu3OC6IMofPoseG0Y= -github.com/prometheus/common v0.39.1-0.20230202092144-f9c1994be032/go.mod h1:L65ZJPSmfn/UBWLQIHV7dBrKFidB/wPlF1y5TlSt9OE= +github.com/prometheus/common v0.42.0 h1:EKsfXEYo4JpWMHH5cg+KOUWeuJSov1Id8zGR8eeI1YM= +github.com/prometheus/common v0.42.0/go.mod h1:xBwqVerjNdUDjgODMpudtOMwlOwf2SaTr1yjz4b7Zbc= github.com/prometheus/common/sigv4 v0.1.0 h1:qoVebwtwwEhS85Czm2dSROY5fTo2PAPEVdDeppTwGX4= github.com/prometheus/common/sigv4 v0.1.0/go.mod h1:2Jkxxk9yYvCkE5G1sQT7GuEXm57JrvHu9k5YwTjsNtI= -github.com/prometheus/exporter-toolkit v0.8.2 h1:sbJAfBXQFkG6sUkbwBun8MNdzW9+wd5YfPYofbmj0YM= github.com/prometheus/exporter-toolkit v0.8.2/go.mod h1:00shzmJL7KxcsabLWcONwpyNEuWhREOnFqZW7vadFS0= +github.com/prometheus/exporter-toolkit v0.9.1 h1:cNkC01riqiOS+kh3zdnNwRsbe/Blh0WwK3ij5rPJ9Sw= +github.com/prometheus/exporter-toolkit v0.9.1/go.mod h1:iFlTmFISCix0vyuyBmm0UqOUCTao9+RsAsKJP3YM9ec= github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= github.com/prometheus/procfs v0.0.0-20190117184657-bf6a532e95b1/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= github.com/prometheus/procfs v0.0.0-20190425082905-87a4384529e0/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= @@ -820,8 +815,8 @@ github.com/prometheus/procfs v0.7.3/go.mod h1:cz+aTbrPOrUb4q7XlbU9ygM+/jj0fzG6c1 github.com/prometheus/procfs v0.8.0/go.mod h1:z7EfXMXOkbkqb9IINtpCn86r/to3BnA0uaxHdg830/4= github.com/prometheus/procfs v0.9.0 h1:wzCHvIvM5SxWqYvwgVL7yJY8Lz3PKn49KQtpgMYJfhI= github.com/prometheus/procfs v0.9.0/go.mod h1:+pB4zwohETzFnmlpe6yd2lSc+0/46IYZRB/chUwxUZY= -github.com/prometheus/prometheus v0.42.0 h1:G769v8covTkOiNckXFIwLx01XE04OE6Fr0JPA0oR2nI= -github.com/prometheus/prometheus v0.42.0/go.mod h1:Pfqb/MLnnR2KK+0vchiaH39jXxvLMBk+3lnIGP4N7Vk= +github.com/prometheus/prometheus v0.43.1-0.20230404063149-1936868e9d13 h1:2hRA1Yq60kAiEkip3saeYDub70kG8/xNQGfFP5cJQkY= +github.com/prometheus/prometheus v0.43.1-0.20230404063149-1936868e9d13/go.mod h1:iZFykJUalD/dDig0wkc358XB6lvO70aj8b1CY18Xwjo= github.com/rcrowley/go-metrics v0.0.0-20181016184325-3113b8401b8a/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4= github.com/rivo/uniseg v0.2.0 h1:S1pD9weZBuJdFmowNwbpi7BJ8TNftyUImj/0WQi72jY= github.com/rivo/uniseg v0.2.0/go.mod h1:J6wj4VEh+S6ZtnVlnTBMWIodfgj8LQOQFoIToxlJtxc= @@ -830,8 +825,7 @@ github.com/rogpeppe/fastuuid v1.2.0/go.mod h1:jVj6XXZzXRy/MSR5jhDC/2q6DgLz+nrA6L github.com/rogpeppe/go-internal v1.1.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= github.com/rogpeppe/go-internal v1.2.2/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= -github.com/rogpeppe/go-internal v1.6.1 h1:/FiVV8dS/e+YqF2JvO3yXRFbBLTIuSDkuC7aBOAvL+k= -github.com/rogpeppe/go-internal v1.6.1/go.mod h1:xXDCJY+GAPziupqXw64V24skbSoqbTEfhy4qGm1nDQc= +github.com/rogpeppe/go-internal v1.9.0 h1:73kH8U+JUqXU8lRuOHeVHaa/SZPifC7BkcraZVejAe8= github.com/rs/xid v1.4.0 h1:qd7wPTDkN6KQx2VmMBLrpHkiyQwgFXRnkOLacUiaSNY= github.com/rs/xid v1.4.0/go.mod h1:trrq9SKmegXys3aeAKXMUTdJsYXVwGY3RLcfgqegfbg= github.com/rueian/rueidis v0.0.93 h1:cG905akj2+QyHx0x9y4mN0K8vLi6M94QiyoLulXS3l0= @@ -842,7 +836,7 @@ github.com/samuel/go-zookeeper v0.0.0-20190923202752-2cc03de413da/go.mod h1:gi+0 github.com/santhosh-tekuri/jsonschema v1.2.4 h1:hNhW8e7t+H1vgY+1QeEQpveR6D4+OwKPXCfD2aieJis= github.com/santhosh-tekuri/jsonschema v1.2.4/go.mod h1:TEAUOeZSmIxTTuHatJzrvARHiuO9LYd+cIxzgEHCQI4= github.com/satori/go.uuid v1.2.1-0.20181028125025-b2ce2384e17b h1:gQZ0qzfKHQIybLANtM3mBXNUtOfsCFXeTsnBqCsx1KM= -github.com/scaleway/scaleway-sdk-go v1.0.0-beta.12 h1:Aaz4T7dZp7cB2cv7D/tGtRdSMh48sRaDYr7Jh0HV4qQ= +github.com/scaleway/scaleway-sdk-go v1.0.0-beta.14 h1:yFl3jyaSVLNYXlnNYM5z2pagEk1dYQhfr1p20T1NyKY= github.com/sean-/seed v0.0.0-20170313163322-e2103e2c3529/go.mod h1:DxrIzT+xaE7yg65j358z/aeFdxmN0P9QXhEzd20vsDc= github.com/sercand/kuberesolver v2.4.0+incompatible h1:WE2OlRf6wjLxHwNkkFLQGaZcVLEXjMjBPjjEU5vksH8= github.com/sercand/kuberesolver v2.4.0+incompatible/go.mod h1:lWF3GL0xptCB/vCiJPl/ZshwPsX/n4Y7u0CW9E7aQIQ= @@ -887,8 +881,9 @@ github.com/stretchr/testify v1.6.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/ github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.7.1/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO+kdMU+MU= -github.com/stretchr/testify v1.8.1 h1:w7B6lhMri9wdJUVmEZPGGhZzrYTPvgJArz7wNPgYKsk= github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= +github.com/stretchr/testify v1.8.2 h1:+h33VjcLVPDHtOdpUCuF+7gSuG3yGIftsP1YvFihtJ8= +github.com/stretchr/testify v1.8.2/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= github.com/tencentcloud/tencentcloud-sdk-go/tencentcloud/common v1.0.194/go.mod h1:7sCQWVkxcsR38nffDW057DRGk8mUjK1Ing/EFOK8s8Y= github.com/tencentcloud/tencentcloud-sdk-go/tencentcloud/kms v1.0.194/go.mod h1:yrBKWhChnDqNz1xuXdSbWXG56XawEq0G5j1lg4VwBD4= github.com/tencentyun/cos-go-sdk-v5 v0.7.40 h1:W6vDGKCHe4wBACI1d2UgE6+50sJFhRWU4O8IB2ozzxM= @@ -911,7 +906,6 @@ github.com/tklauser/numcpus v0.2.1/go.mod h1:9aU+wOc6WjUIZEwWMP62PL/41d65P+iks1g github.com/tklauser/numcpus v0.4.0 h1:E53Dm1HjH1/R2/aoCtXtPgzmElmn51aOkhCFSuZq//o= github.com/tklauser/numcpus v0.4.0/go.mod h1:1+UI3pD8NW14VMwdgJNJ1ESk2UnwhAnz5hMwiKKqXCQ= github.com/tmc/grpc-websocket-proxy v0.0.0-20170815181823-89b8d40f7ca8/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= -github.com/tv42/httpunix v0.0.0-20150427012821-b75d8614f926/go.mod h1:9ESjWnEqriFuLhtthL60Sar/7RFoluCcXsuvEwTV5KM= github.com/uber/jaeger-client-go v2.28.0+incompatible/go.mod h1:WVhlPFC8FDjOFMMWRy2pZqQJSXxYSwNYOkTr/Z6d3Kk= github.com/uber/jaeger-client-go v2.30.0+incompatible h1:D6wyKGCecFaSRUpo8lCVbaOOb6ThwMmTEbhRwtKR97o= github.com/uber/jaeger-client-go v2.30.0+incompatible/go.mod h1:WVhlPFC8FDjOFMMWRy2pZqQJSXxYSwNYOkTr/Z6d3Kk= @@ -955,8 +949,8 @@ go.etcd.io/etcd v0.0.0-20191023171146-3cf2f69b5738/go.mod h1:dnLIgRNXwCJa5e+c6mI go.mongodb.org/mongo-driver v1.7.3/go.mod h1:NqaYOwnXWr5Pm7AOpO5QFxKJ503nbMse/R79oO62zWg= go.mongodb.org/mongo-driver v1.7.5/go.mod h1:VXEWRZ6URJIkUq2SCAyapmhH0ZLRBP+FT4xhp5Zvxng= go.mongodb.org/mongo-driver v1.10.0/go.mod h1:wsihk0Kdgv8Kqu1Anit4sfK+22vSFbUrAVEYRhCXrA8= -go.mongodb.org/mongo-driver v1.11.0 h1:FZKhBSTydeuffHj9CBjXlR8vQLee1cQyTWYPA6/tqiE= -go.mongodb.org/mongo-driver v1.11.0/go.mod h1:s7p5vEtfbeR1gYi6pnj3c3/urpbLv2T5Sfd6Rp2HBB8= +go.mongodb.org/mongo-driver v1.11.2 h1:+1v2rDQUWNcGW7/7E0Jvdz51V38XXxJfhzbV17aNHCw= +go.mongodb.org/mongo-driver v1.11.2/go.mod h1:s7p5vEtfbeR1gYi6pnj3c3/urpbLv2T5Sfd6Rp2HBB8= go.opencensus.io v0.20.1/go.mod h1:6WKK9ahsWS3RSO+PY9ZHZUfv2irvY6gN279GOPZjmmk= go.opencensus.io v0.20.2/go.mod h1:6WKK9ahsWS3RSO+PY9ZHZUfv2irvY6gN279GOPZjmmk= go.opencensus.io v0.21.0/go.mod h1:mSImk1erAIZhrmZN+AvHh14ztQfjbGwt4TtuofqLduU= @@ -968,8 +962,8 @@ go.opencensus.io v0.22.5/go.mod h1:5pWMHQbX5EPX2/62yrJeAkowc+lfs/XD7Uxpq3pI6kk= go.opencensus.io v0.23.0/go.mod h1:XItmlyltB5F7CS4xOC1DcqMoFqwtC6OG2xF7mCv7P7E= go.opencensus.io v0.24.0 h1:y73uSU6J157QMP2kn2r30vwW1A2W2WFwSCGnAVxeaD0= go.opencensus.io v0.24.0/go.mod h1:vNK8G9p7aAivkbmorf4v+7Hgx+Zs0yY+0fOtgBfjQKo= -go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.39.0 h1:vFEBG7SieZJzvnRWQ81jxpuEqe6J8Ex+hgc9CqOTzHc= -go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.39.0/go.mod h1:9rgTcOKdIhDOC0IcAu8a+R+FChqSUBihKpM1lVNi6T0= +go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.40.0 h1:lE9EJyw3/JhrjWH/hEy9FptnalDQgj7vpbgC2KCCCxE= +go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.40.0/go.mod h1:pcQ3MM3SWvrA71U4GDqv9UFDJ3HQsW7y5ZO3tDTlUdI= go.opentelemetry.io/contrib/propagators/autoprop v0.38.0 h1:WZwiLCwOL0XW/6TVT7LTtdRDveoHZ6q3wL+0iYsBcdE= go.opentelemetry.io/contrib/propagators/autoprop v0.38.0/go.mod h1:JBebP2d0HiffbfelbIEoBOCl4790g7Z8lD1scUd3Vd8= go.opentelemetry.io/contrib/propagators/aws v1.13.0 h1:9qOAQhTeJGiaYNfCCnRmL12XZGIaxclqS5yfkSXpn8o= @@ -982,26 +976,26 @@ go.opentelemetry.io/contrib/propagators/ot v1.13.0 h1:tHWNd0WRS6w9keZoZg9aF3zYoh go.opentelemetry.io/contrib/propagators/ot v1.13.0/go.mod h1:R6Op9T6LxNaMRVlGD0wVwz40LSsAq296CXiEydKLQBU= go.opentelemetry.io/contrib/samplers/jaegerremote v0.7.0 h1:E+RlfFhGZ5Tk0wO1oOJYC0Il4Q7SjE8ZMl8x/VTK9Pk= go.opentelemetry.io/contrib/samplers/jaegerremote v0.7.0/go.mod h1:cuBMmL+iGJ4UpZi6dykQlIUxqKSMkp5eu1C1UjUJYFI= -go.opentelemetry.io/otel v1.13.0 h1:1ZAKnNQKwBBxFtww/GwxNUyTf0AxkZzrukO8MeXqe4Y= -go.opentelemetry.io/otel v1.13.0/go.mod h1:FH3RtdZCzRkJYFTCsAKDy9l/XYjMdNv6QrkFFB8DvVg= +go.opentelemetry.io/otel v1.14.0 h1:/79Huy8wbf5DnIPhemGB+zEPVwnN6fuQybr/SRXa6hM= +go.opentelemetry.io/otel v1.14.0/go.mod h1:o4buv+dJzx8rohcUeRmWUZhqupFvzWis188WlggnNeU= go.opentelemetry.io/otel/bridge/opentracing v1.12.0 h1:tU684zGp/ft9QpXRixnoeKbz0vNjrcd3tEDsYy+uJUI= go.opentelemetry.io/otel/bridge/opentracing v1.12.0/go.mod h1:qjLYKFXmUQhZHVa0EbQOY29U061UO/14B+NGWUOnOnk= go.opentelemetry.io/otel/exporters/jaeger v1.12.0 h1:1Vy11S0iAD70EPfcP3N2f2IhLq/cIuTW+Zt010MswR8= go.opentelemetry.io/otel/exporters/jaeger v1.12.0/go.mod h1:SCLbaspEoU9mGJZB6ksc2iSGU6CLWY5yefchDqOM0IM= -go.opentelemetry.io/otel/exporters/otlp/internal/retry v1.12.0 h1:UfDENi+LTcLjQ/JhaXimjlIgn7wWjwbEMmdREm2Gyng= -go.opentelemetry.io/otel/exporters/otlp/internal/retry v1.12.0/go.mod h1:rqbht/LlhVBgn5+k3M5QK96K5Xb0DvXpMJ5SFQpY6uw= -go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.12.0 h1:ZVqtSAxrR4+ofzayuww0/EKamCjjnwnXTMRZzMudJoU= -go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.12.0/go.mod h1:IlaGLENJkAl9+Xoo3J0unkdOwtL+rmqZ3ryMjUtYA94= -go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracegrpc v1.12.0 h1:+tsVdWosoqDfX6cdHAeacZozjQS94ySBd+aUXFwnNKA= -go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracegrpc v1.12.0/go.mod h1:jSqjV+Knu1Jyvh+l3fx7V210Ev3HHgNQAi8YqpXaQP8= -go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracehttp v1.12.0 h1:L23MzcHDznr05xOM1Ng1F98L0nVd7hm/S7y2jW9IRB4= -go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracehttp v1.12.0/go.mod h1:C+onYX2j5QH653b3wGJwowYr8jLMjBJw35QcaCQQK0U= -go.opentelemetry.io/otel/metric v0.36.0 h1:t0lgGI+L68QWt3QtOIlqM9gXoxqxWLhZ3R/e5oOAY0Q= -go.opentelemetry.io/otel/metric v0.36.0/go.mod h1:wKVw57sd2HdSZAzyfOM9gTqqE8v7CbqWsYL6AyrH9qk= -go.opentelemetry.io/otel/sdk v1.12.0 h1:8npliVYV7qc0t1FKdpU08eMnOjgPFMnriPhn0HH4q3o= -go.opentelemetry.io/otel/sdk v1.12.0/go.mod h1:WYcvtgquYvgODEvxOry5owO2y9MyciW7JqMz6cpXShE= -go.opentelemetry.io/otel/trace v1.13.0 h1:CBgRZ6ntv+Amuj1jDsMhZtlAPT6gbyIRdaIzFhfBSdY= -go.opentelemetry.io/otel/trace v1.13.0/go.mod h1:muCvmmO9KKpvuXSf3KKAXXB2ygNYHQ+ZfI5X08d3tds= +go.opentelemetry.io/otel/exporters/otlp/internal/retry v1.14.0 h1:/fXHZHGvro6MVqV34fJzDhi7sHGpX3Ej/Qjmfn003ho= +go.opentelemetry.io/otel/exporters/otlp/internal/retry v1.14.0/go.mod h1:UFG7EBMRdXyFstOwH028U0sVf+AvukSGhF0g8+dmNG8= +go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.14.0 h1:TKf2uAs2ueguzLaxOCBXNpHxfO/aC7PAdDsSH0IbeRQ= +go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.14.0/go.mod h1:HrbCVv40OOLTABmOn1ZWty6CHXkU8DK/Urc43tHug70= +go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracegrpc v1.14.0 h1:ap+y8RXX3Mu9apKVtOkM6WSFESLM8K3wNQyOU8sWHcc= +go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracegrpc v1.14.0/go.mod h1:5w41DY6S9gZrbjuq6Y+753e96WfPha5IcsOSZTtullM= +go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracehttp v1.14.0 h1:3jAYbRHQAqzLjd9I4tzxwJ8Pk/N6AqBcF6m1ZHrxG94= +go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracehttp v1.14.0/go.mod h1:+N7zNjIJv4K+DeX67XXET0P+eIciESgaFDBqh+ZJFS4= +go.opentelemetry.io/otel/metric v0.37.0 h1:pHDQuLQOZwYD+Km0eb657A25NaRzy0a+eLyKfDXedEs= +go.opentelemetry.io/otel/metric v0.37.0/go.mod h1:DmdaHfGt54iV6UKxsV9slj2bBRJcKC1B1uvDLIioc1s= +go.opentelemetry.io/otel/sdk v1.14.0 h1:PDCppFRDq8A1jL9v6KMI6dYesaq+DFcDZvjsoGvxGzY= +go.opentelemetry.io/otel/sdk v1.14.0/go.mod h1:bwIC5TjrNG6QDCHNWvW4HLHtUQ4I+VQDsnjhvyZCALM= +go.opentelemetry.io/otel/trace v1.14.0 h1:wp2Mmvj41tDsyAJXiWDWpfNsOiIyd38fy85pyKcFq/M= +go.opentelemetry.io/otel/trace v1.14.0/go.mod h1:8avnQLK+CG77yNLUae4ea2JDQ6iT+gozhnZjy/rw9G8= go.opentelemetry.io/proto/otlp v0.7.0/go.mod h1:PqfVotwruBrMGOCsRd/89rSnXhoiJIqeYNgFYFoEGnI= go.opentelemetry.io/proto/otlp v0.19.0 h1:IVN6GR+mhC4s5yfcTbmzHYODqvWAp3ZedA2SJPI1Nnw= go.opentelemetry.io/proto/otlp v0.19.0/go.mod h1:H7XAot3MsfNsj7EXtrA2q5xSNQ10UqI405h3+duxN4U= @@ -1013,8 +1007,8 @@ go.uber.org/atomic v1.10.0 h1:9qC72Qh0+3MqyJbAn8YU5xVq1frD8bn3JtD2oXtafVQ= go.uber.org/atomic v1.10.0/go.mod h1:LUxbIzbOniOlMKjJjyPfpl4v+PKK2cNJn91OQbhoJI0= go.uber.org/automaxprocs v1.5.1 h1:e1YG66Lrk73dn4qhg8WFSvhF0JuFQF0ERIp4rpuV8Qk= go.uber.org/automaxprocs v1.5.1/go.mod h1:BF4eumQw0P9GtnuxxovUd06vwm1o18oMzFtK66vU6XU= -go.uber.org/goleak v1.2.0 h1:xqgm/S+aQvhWFTtR0XK3Jvg7z8kGV8P4X14IzwN3Eqk= -go.uber.org/goleak v1.2.0/go.mod h1:XJYK+MuIchqpmGmUSAzotztawfKvYLUIgg7guXrwVUo= +go.uber.org/goleak v1.2.1 h1:NBol2c7O1ZokfZ0LEU9K6Whx/KnwvepVetCUhtKja4A= +go.uber.org/goleak v1.2.1/go.mod h1:qlT2yGI9QafXHhZZLxlSuNsMw3FFLxBr+tBRlmO1xH4= go.uber.org/multierr v1.1.0/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/0= go.uber.org/multierr v1.3.0/go.mod h1:VgVr7evmIr6uPjLBxg28wmKNXyqE9akIJ5XnfpiKl+4= go.uber.org/multierr v1.9.0 h1:7fIwc/ZtS0q++VgcfqFDxSBZVv/Xo49/SYnDFupUwlI= @@ -1040,8 +1034,8 @@ golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPh golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.0.0-20220622213112-05595931fe9d/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/crypto v0.0.0-20221012134737-56aed061732a/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= -golang.org/x/crypto v0.3.0 h1:a06MkbcxBrEFc0w0QIZWXrH/9cCX6KJyWbBOIwAn+7A= -golang.org/x/crypto v0.3.0/go.mod h1:hebNnKkNXi2UzZN1eVRvBB7co0a+JxK6XbPiWVs/3J4= +golang.org/x/crypto v0.7.0 h1:AvwMYaRytfdeVt3u6mLaxYtErKYjxA2OXjJ1HHq6t3A= +golang.org/x/crypto v0.7.0/go.mod h1:pYwdfH91IfpZVANVyUOhSIPZaFoJGxTFbZhFTx+dXZU= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190306152737-a1d7652674e8/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190510132918-efd6b22b2522/go.mod h1:ZjyILWgesfNpC6sMxTJOJm9Kp84zZh5NQWvqDGG3Qr8= @@ -1052,8 +1046,8 @@ golang.org/x/exp v0.0.0-20191227195350-da58074b4299/go.mod h1:2RIsYlXP63K8oxa1u0 golang.org/x/exp v0.0.0-20200119233911-0405dc783f0a/go.mod h1:2RIsYlXP63K8oxa1u096TMicItID8zy7Y6sNkU49FU4= golang.org/x/exp v0.0.0-20200207192155-f17229e696bd/go.mod h1:J/WKrq2StrnmMY6+EHIKF9dgMWnmCNThgcyBT1FY9mM= golang.org/x/exp v0.0.0-20200224162631-6cc2880d07d6/go.mod h1:3jZMyOhIsHpP37uCMkUooju7aAi5cS1Q23tOzKc+0MU= -golang.org/x/exp v0.0.0-20230124195608-d38c7dcee874 h1:kWC3b7j6Fu09SnEBr7P4PuQyM0R6sqyH9R+EjIvT1nQ= -golang.org/x/exp v0.0.0-20230124195608-d38c7dcee874/go.mod h1:CxIveKay+FTh1D0yPZemJVgC/95VzuuOLq5Qi4xnoYc= +golang.org/x/exp v0.0.0-20230307190834-24139beb5833 h1:SChBja7BCQewoTAU7IgvucQKMIXrEpFxNMs0spT3/5s= +golang.org/x/exp v0.0.0-20230307190834-24139beb5833/go.mod h1:CxIveKay+FTh1D0yPZemJVgC/95VzuuOLq5Qi4xnoYc= golang.org/x/image v0.0.0-20190227222117-0694c2d4d067/go.mod h1:kZ7UVZpmo3dzQBMxlp+ypCbDeSB+sBbTgSJuh5dn5js= golang.org/x/image v0.0.0-20190802002840-cff245a6509b/go.mod h1:FeLwcggjj3mMvU+oOTbSwawSJRM1uh48EjtB4UJZlP0= golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= @@ -1066,7 +1060,6 @@ golang.org/x/lint v0.0.0-20191125180803-fdd1cda4f05f/go.mod h1:5qLYkcX4OjUUV8bRu golang.org/x/lint v0.0.0-20200130185559-910be7a94367/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= golang.org/x/lint v0.0.0-20200302205851-738671d3881b/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= golang.org/x/lint v0.0.0-20201208152925-83fdc39ff7b5/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= -golang.org/x/lint v0.0.0-20210508222113-6edffad5e616 h1:VLliZ0d+/avPrXXH+OakdXhpJuEoBZuwh1m2j7U6Iug= golang.org/x/lint v0.0.0-20210508222113-6edffad5e616/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= golang.org/x/mobile v0.0.0-20190312151609-d3739f865fa6/go.mod h1:z+o9i4GpDbdi3rU15maQ/Ox0txvL9dWGYEHz965HBQE= golang.org/x/mobile v0.0.0-20190719004257-d2bd2a29d028/go.mod h1:E/iHnbuqvinMTCcRqshq8CkpyQDoeVncDDYHnLhea+o= @@ -1080,8 +1073,8 @@ golang.org/x/mod v0.4.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.4.1/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91VN4djpZkiMVwK6gcyfeH4XE8wZrZaV4= -golang.org/x/mod v0.7.0 h1:LapD9S96VoQRhi/GrNTqeBJFrUjs5UHCAtTlgwA5oZA= -golang.org/x/mod v0.7.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= +golang.org/x/mod v0.9.0 h1:KENHtAZL2y3NLMYZeHY9DW8HW8V+kQyJsY/V9JlKvCs= +golang.org/x/mod v0.9.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180906233101-161cd47e91fd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181023162649-9b4f9f5ad519/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= @@ -1127,7 +1120,6 @@ golang.org/x/net v0.0.0-20210405180319-a5a99cb37ef4/go.mod h1:p54w0d4576C0XHj96b golang.org/x/net v0.0.0-20210421230115-4e50805a0758/go.mod h1:72T/g9IO56b78aLF+1Kcs5dz7/ng1VjMUvfKvpfy+jM= golang.org/x/net v0.0.0-20210503060351-7fd8e65b6420/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20210525063256-abc453219eb5/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= -golang.org/x/net v0.0.0-20210726213435-c6fcb2dbf985/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20211112202133-69e39bad7dc2/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20220127200216-cd36cc0744dd/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk= golang.org/x/net v0.0.0-20220225172249-27dd8689420f/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk= @@ -1139,8 +1131,8 @@ golang.org/x/net v0.0.0-20220624214902-1bab6f366d9e/go.mod h1:XRhObCWvk6IyKnWLug golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= golang.org/x/net v0.0.0-20220909164309-bea034e7d591/go.mod h1:YDH+HFinaLZZlnHAfSS6ZXJJ9M9t4Dl22yv3iI2vPwk= golang.org/x/net v0.1.0/go.mod h1:Cx3nUiGt4eDBEyega/BKRp+/AlGL8hYe7U9odMt2Cco= -golang.org/x/net v0.7.0 h1:rJrUqqhjsgNp7KqAIc25s9pZnjU7TUcSY7HcVZjdn1g= -golang.org/x/net v0.7.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= +golang.org/x/net v0.8.0 h1:Zrh2ngAOFYneWTAIAPethzeaQLuHwhuBkuV6ZiRnUaQ= +golang.org/x/net v0.8.0/go.mod h1:QVkue5JL9kW//ek3r6jTKnTFis1tRmNAW2P1shuFdJc= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= @@ -1162,8 +1154,8 @@ golang.org/x/oauth2 v0.0.0-20220309155454-6242fa91716a/go.mod h1:DAh4E804XQdzx2j golang.org/x/oauth2 v0.0.0-20220411215720-9780585627b5/go.mod h1:DAh4E804XQdzx2j+YRIaUnCqCV2RuMz24cGBJ5QYIrc= golang.org/x/oauth2 v0.0.0-20220608161450-d0670ef3b1eb/go.mod h1:jaDAt6Dkxork7LmZnYtzbRWj0W47D86a3TGe0YHBvmE= golang.org/x/oauth2 v0.0.0-20220909003341-f21342109be1/go.mod h1:h4gKUeWbJ4rQPri7E0u6Gs4e9Ri2zaLxzw5DI5XGrYg= -golang.org/x/oauth2 v0.4.0 h1:NF0gk8LVPg1Ml7SSbGyySuoxdsXitj7TvgvuRxIMc/M= -golang.org/x/oauth2 v0.4.0/go.mod h1:RznEsdpjGAINPTOF0UH/t+xJ75L18YO3Ho6Pyn+uRec= +golang.org/x/oauth2 v0.6.0 h1:Lh8GPgSKBfWSwFvtuWOfeI3aAAnbXTSutYxJiOJFgIw= +golang.org/x/oauth2 v0.6.0/go.mod h1:ycmewcwgD4Rpr3eZJLSB4Kyyljb3qDh40vJ8STE5HKw= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -1272,12 +1264,12 @@ golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20220728004956-3c1f35247d10/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220908164124-27713097b956/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.1.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.5.0 h1:MUK/U/4lj1t1oPg0HfuXDN/Z1wv31ZJ/YcPiGccS4DU= -golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.6.0 h1:MVltZSvRTcU2ljQOhs94SXPftV6DCNnZViHeQps87pQ= +golang.org/x/sys v0.6.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/term v0.1.0/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= -golang.org/x/term v0.5.0 h1:n2a8QNdAb0sZNpU9R1ALUXBbY+w51fCQDN+7EdxNBsY= +golang.org/x/term v0.6.0 h1:clScbb1cHjoCkyRbWwBEUZ5H/tIFu5TAXIqaZD0Gcjw= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= @@ -1288,8 +1280,8 @@ golang.org/x/text v0.3.5/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= golang.org/x/text v0.4.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= -golang.org/x/text v0.7.0 h1:4BRB4x83lYWy72KwLD/qYDuTu7q9PjSagHvijDw7cLo= -golang.org/x/text v0.7.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= +golang.org/x/text v0.8.0 h1:57P1ETyNKtuIjB4SRd15iJxuhj8Gc416Y78H3qgMh68= +golang.org/x/text v0.8.0/go.mod h1:e1OnstbJyHTd6l/uOt8jFFHp6TRDWZR/bV3emEE/zU8= golang.org/x/time v0.0.0-20180412165947-fbb02b2291d2/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= @@ -1359,10 +1351,9 @@ golang.org/x/tools v0.1.2/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/tools v0.1.3/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/tools v0.1.4/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/tools v0.1.5/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= -golang.org/x/tools v0.1.6-0.20210726203631-07bc1bf47fb2/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/tools v0.1.12/go.mod h1:hNGJHUnrk76NpqgfD5Aqm5Crs+Hm0VOH/i9J2+nxYbc= -golang.org/x/tools v0.5.0 h1:+bSpV5HIeWkuvgaMfI3UmKRThoTA5ODJTUd8T17NO+4= -golang.org/x/tools v0.5.0/go.mod h1:N+Kgy78s5I24c24dU8OfWNEotWjutIs8SnJvn5IDq+k= +golang.org/x/tools v0.7.0 h1:W4OVu8VVOaIO0yzWMNdepAulS7YfoS3Zabrm8DOXXU4= +golang.org/x/tools v0.7.0/go.mod h1:4pg6aUX35JBAogB10C9AtvVL+qowtN4pT3CGSQex14s= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= @@ -1414,8 +1405,8 @@ google.golang.org/api v0.75.0/go.mod h1:pU9QmyHLnzlpar1Mjt4IbapUCy8J+6HD6GeELN69 google.golang.org/api v0.78.0/go.mod h1:1Sg78yoMLOhlQTeF+ARBoytAcH1NNyyl390YMy6rKmw= google.golang.org/api v0.80.0/go.mod h1:xY3nI94gbvBrE0J6NHXhxOmW97HG7Khjkku6AFB3Hyg= google.golang.org/api v0.84.0/go.mod h1:NTsGnUFJMYROtiquksZHBWtHfeMC7iYthki7Eq3pa8o= -google.golang.org/api v0.108.0 h1:WVBc/faN0DkKtR43Q/7+tPny9ZoLZdIiAyG5Q9vFClg= -google.golang.org/api v0.108.0/go.mod h1:2Ts0XTHNVWxypznxWOYUeI4g3WdP9Pk2Qk58+a/O9MY= +google.golang.org/api v0.114.0 h1:1xQPji6cO2E2vLiI+C/XiFAnsn1WV3mjaEwGLhi3grE= +google.golang.org/api v0.114.0/go.mod h1:ifYI2ZsFK6/uGddGfAD5BMxlnkBqCmqHSDUVi45N5Yg= google.golang.org/appengine v1.2.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= @@ -1506,8 +1497,8 @@ google.golang.org/genproto v0.0.0-20220518221133-4f43b3371335/go.mod h1:RAyBrSAP google.golang.org/genproto v0.0.0-20220523171625-347a074981d8/go.mod h1:RAyBrSAP7Fh3Nc84ghnVLDPuV51xc9agzmm4Ph6i0Q4= google.golang.org/genproto v0.0.0-20220608133413-ed9918b62aac/go.mod h1:KEWEmljWE5zPzLBa/oHl6DaEt9LmfH6WtH1OHIvleBA= google.golang.org/genproto v0.0.0-20220616135557-88e70c0c3a90/go.mod h1:KEWEmljWE5zPzLBa/oHl6DaEt9LmfH6WtH1OHIvleBA= -google.golang.org/genproto v0.0.0-20230131230820-1c016267d619 h1:p0kMzw6AG0JEzd7Z+kXqOiLhC6gjUQTbtS2zR0Q3DbI= -google.golang.org/genproto v0.0.0-20230131230820-1c016267d619/go.mod h1:RGgjbofJ8xD9Sq1VVhDM1Vok1vRONV+rg+CjzG4SZKM= +google.golang.org/genproto v0.0.0-20230306155012-7f2fa6fef1f4 h1:DdoeryqhaXp1LtT/emMP1BRJPHHKFi5akj/nbx/zNTA= +google.golang.org/genproto v0.0.0-20230306155012-7f2fa6fef1f4/go.mod h1:NWraEVixdDnqcqQ30jipen1STv2r/n24Wb7twVTGR4s= google.golang.org/grpc v1.45.0 h1:NEpgUqV3Z+ZjkqMsxMg11IaDrXY4RY6CQukSGK0uI1M= google.golang.org/grpc v1.45.0/go.mod h1:lN7owxKUQEqMfSyQikvvk5tf/6zMPsrK+ONuO11+0rQ= google.golang.org/grpc/cmd/protoc-gen-go-grpc v1.1.0/go.mod h1:6Kw0yEErY5E/yWrBtf03jp27GLLJujG4z/JK95pnjjw= @@ -1527,21 +1518,23 @@ google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp0 google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= google.golang.org/protobuf v1.27.1/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= google.golang.org/protobuf v1.28.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= -google.golang.org/protobuf v1.28.1 h1:d0NfwRgPtno5B1Wa6L2DAG+KivqkdutMf1UhdNx175w= google.golang.org/protobuf v1.28.1/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= +google.golang.org/protobuf v1.29.1 h1:7QBf+IK2gx70Ap/hDsOmam3GE0v9HicjfEdAxE62UoM= +google.golang.org/protobuf v1.29.1/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20200227125254-8fa46927fb4f/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= +gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= gopkg.in/cheggaaa/pb.v1 v1.0.25/go.mod h1:V/YB90LKu/1FcN3WVnfiiE5oMCibMjukxqG/qStrOgw= gopkg.in/errgo.v2 v2.1.0/go.mod h1:hNsd1EY+bozCKY1Ytp96fpM3vjJbqLJn88ws8XvfDNI= gopkg.in/fsnotify.v1 v1.4.7 h1:xOHLXZwVvI9hhs+cLKq5+I5onOuwQLhQwiu63xxlHs4= gopkg.in/fsnotify.v1 v1.4.7/go.mod h1:Tz8NjZHkW78fSQdbUxIjBTcgA1z1m8ZHf0WmKUhAMys= gopkg.in/gcfg.v1 v1.2.3/go.mod h1:yesOnuUOFQAhST5vPY4nbZsb/huCgGGXlipJsBn0b3o= gopkg.in/inf.v0 v0.9.1 h1:73M5CoZyi3ZLMOyDlQh031Cx6N9NDJ2Vvfl76EDAgDc= -gopkg.in/ini.v1 v1.66.6 h1:LATuAqN/shcYAOkv3wl2L4rkaKqkcgTBQjOyYDvcPKI= -gopkg.in/ini.v1 v1.66.6/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= +gopkg.in/ini.v1 v1.67.0 h1:Dgnx+6+nfE+IfzjUEISNeydPJh9AXNNsWbGP9KzCsOA= +gopkg.in/ini.v1 v1.67.0/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= gopkg.in/resty.v1 v1.12.0/go.mod h1:mDo4pnntr5jdWRML875a/NmxYqAlA73dVijT2AXvQQo= gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7 h1:uRGJdciOHaEIrze2W8Q3AKkepLTh2hOroT7a+7czfdQ= gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7/go.mod h1:dt/ZhP58zS4L8KSrWDmTeBkI65Dw0HsyUHuEVlX15mw= @@ -1571,11 +1564,11 @@ honnef.co/go/tools v0.0.1-2020.1.3/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9 honnef.co/go/tools v0.0.1-2020.1.4/go.mod h1:X/FiERA/W4tHapMX5mGpAtMSVEeEUOyHaw9vFzvIQ3k= howett.net/plist v0.0.0-20181124034731-591f970eefbb h1:jhnBjNi9UFpfpl8YZhA9CrOqpnJdvzuiHsl/dnxl11M= howett.net/plist v0.0.0-20181124034731-591f970eefbb/go.mod h1:vMygbs4qMhSZSc4lCUl2OEE+rDiIIJAIdR4m7MiMcm0= -k8s.io/api v0.26.1 h1:f+SWYiPd/GsiWwVRz+NbFyCgvv75Pk9NK6dlkZgpCRQ= -k8s.io/apimachinery v0.26.1 h1:8EZ/eGJL+hY/MYCNwhmDzVqq2lPl3N3Bo8rvweJwXUQ= -k8s.io/client-go v0.26.1 h1:87CXzYJnAMGaa/IDDfRdhTzxk/wzGZ+/HUQpqgVSZXU= -k8s.io/kube-openapi v0.0.0-20221207184640-f3cff1453715 h1:tBEbstoM+K0FiBV5KGAKQ0kuvf54v/hwpldiJt69w1s= -k8s.io/utils v0.0.0-20221128185143-99ec85e7a448 h1:KTgPnR10d5zhztWptI952TNtt/4u5h3IzDXkdIMuo2Y= +k8s.io/api v0.26.2 h1:dM3cinp3PGB6asOySalOZxEG4CZ0IAdJsrYZXE/ovGQ= +k8s.io/apimachinery v0.26.2 h1:da1u3D5wfR5u2RpLhE/ZtZS2P7QvDgLZTi9wrNZl/tQ= +k8s.io/client-go v0.26.2 h1:s1WkVujHX3kTp4Zn4yGNFK+dlDXy1bAAkIl+cFAiuYI= +k8s.io/kube-openapi v0.0.0-20230303024457-afdc3dddf62d h1:VcFq5n7wCJB2FQMCIHfC+f+jNcGgNMar1uKd6rVlifU= +k8s.io/utils v0.0.0-20230308161112-d77c459e9343 h1:m7tbIjXGcGIAtpmQr7/NAi7RsWoW3E7Zcm4jI1HicTc= rsc.io/binaryregexp v0.2.0/go.mod h1:qTv7/COck+e2FymRvadv62gMdZztPaShugOCi3I+8D8= rsc.io/quote/v3 v3.1.0/go.mod h1:yEA65RcK8LyAZtP9Kv3t0HmxON59tX3rD+tICJqUlj0= rsc.io/sampler v1.3.0/go.mod h1:T1hPZKmBbMNahiBKFy5HrXp6adAjACjK9JXDnKaTXpA= diff --git a/pkg/block/metadata/meta.go b/pkg/block/metadata/meta.go index 797ccd7860..db9381ba51 100644 --- a/pkg/block/metadata/meta.go +++ b/pkg/block/metadata/meta.go @@ -105,9 +105,9 @@ type Rewrite struct { } type PartitionInfo struct { - PartitionedGroupID uint32 `json:"partitionedGroupID"` - PartitionCount int `json:"partitionCount"` - PartitionID int `json:"partitionID"` + PartitionedGroupID uint32 `json:"partitioned_group_id"` + PartitionCount int `json:"partition_count"` + PartitionID int `json:"partition_id"` } type Matchers []*labels.Matcher diff --git a/pkg/block/metadata/meta_test.go b/pkg/block/metadata/meta_test.go index 0be53197e8..e80265d1c2 100644 --- a/pkg/block/metadata/meta_test.go +++ b/pkg/block/metadata/meta_test.go @@ -211,4 +211,104 @@ func TestMeta_ReadWrite(t *testing.T) { m1.Thanos.Labels = map[string]string{} testutil.Equals(t, m1, *retMeta) }) + + t.Run("partition info write/read/write", func(t *testing.T) { + b := bytes.Buffer{} + m1 := Meta{ + BlockMeta: tsdb.BlockMeta{ + ULID: ulid.MustNew(5, nil), + MinTime: 2424, + MaxTime: 134, + Version: 1, + Compaction: tsdb.BlockMetaCompaction{ + Sources: []ulid.ULID{ulid.MustNew(1, nil), ulid.MustNew(2, nil)}, + Parents: []tsdb.BlockDesc{ + { + ULID: ulid.MustNew(3, nil), + MinTime: 442, + MaxTime: 24225, + }, + }, + Level: 123, + }, + Stats: tsdb.BlockStats{NumChunks: 14, NumSamples: 245, NumSeries: 4}, + }, + Thanos: Thanos{ + Version: 1, + Labels: map[string]string{"ext": "lset1"}, + Source: ReceiveSource, + Files: []File{ + {RelPath: "chunks/000001", SizeBytes: 3751}, + {RelPath: "index", SizeBytes: 401}, + {RelPath: "meta.json"}, + }, + Downsample: ThanosDownsample{ + Resolution: 123144, + }, + PartitionInfo: &PartitionInfo{ + PartitionedGroupID: 111, + PartitionCount: 8, + PartitionID: 5, + }, + }, + } + testutil.Ok(t, m1.Write(&b)) + testutil.Equals(t, `{ + "ulid": "00000000050000000000000000", + "minTime": 2424, + "maxTime": 134, + "stats": { + "numSamples": 245, + "numSeries": 4, + "numChunks": 14 + }, + "compaction": { + "level": 123, + "sources": [ + "00000000010000000000000000", + "00000000020000000000000000" + ], + "parents": [ + { + "ulid": "00000000030000000000000000", + "minTime": 442, + "maxTime": 24225 + } + ] + }, + "version": 1, + "thanos": { + "version": 1, + "labels": { + "ext": "lset1" + }, + "downsample": { + "resolution": 123144 + }, + "source": "receive", + "files": [ + { + "rel_path": "chunks/000001", + "size_bytes": 3751 + }, + { + "rel_path": "index", + "size_bytes": 401 + }, + { + "rel_path": "meta.json" + } + ], + "partition_info": { + "partitioned_group_id": 111, + "partition_count": 8, + "partition_id": 5 + } + } +} +`, b.String()) + retMeta, err := Read(io.NopCloser(&b)) + testutil.Ok(t, err) + testutil.Equals(t, m1, *retMeta) + }) } diff --git a/pkg/compact/background_chunks_series_set.go b/pkg/compact/background_chunks_series_set.go deleted file mode 100644 index 900b87f8ab..0000000000 --- a/pkg/compact/background_chunks_series_set.go +++ /dev/null @@ -1,60 +0,0 @@ -package compact - -import ( - "context" - "github.com/prometheus/prometheus/storage" -) - -type backgrounChunkSeriesSet struct { - nextSet chan storage.ChunkSeries - actual storage.ChunkSeries - cs storage.ChunkSeriesSet -} - -func (b *backgrounChunkSeriesSet) Next() bool { - select { - case s, ok := <-b.nextSet: - b.actual = s - return ok - } -} - -func (b *backgrounChunkSeriesSet) At() storage.ChunkSeries { - return b.actual -} - -func (b *backgrounChunkSeriesSet) Err() error { - return b.cs.Err() -} - -func (b *backgrounChunkSeriesSet) Warnings() storage.Warnings { - return b.cs.Warnings() -} - -func (b *backgrounChunkSeriesSet) run(ctx context.Context) { - for { - if !b.cs.Next() { - close(b.nextSet) - return - } - - select { - case b.nextSet <- b.cs.At(): - case <-ctx.Done(): - return - } - } -} - -func NewBackgroundChunkSeriesSet(ctx context.Context, cs storage.ChunkSeriesSet) storage.ChunkSeriesSet { - r := &backgrounChunkSeriesSet{ - cs: cs, - nextSet: make(chan storage.ChunkSeries, 1000), - } - - go func() { - r.run(ctx) - }() - - return r -} diff --git a/pkg/compact/compact.go b/pkg/compact/compact.go index 73549b4f19..e47b53d77d 100644 --- a/pkg/compact/compact.go +++ b/pkg/compact/compact.go @@ -767,19 +767,35 @@ func (c DefaultCompactionCompleteChecker) IsComplete(_ *Group, _ ulid.ULID) bool type CompactionLifecycleCallback interface { PreCompactionCallback(group *Group, toCompactBlocks []*metadata.Meta) error PostCompactionCallback(group *Group, blockID ulid.ULID) error + GetBlockPopulator(group *Group, logger log.Logger) (tsdb.PopulateBlockFunc, error) } -type NoopCompactionLifecycleCallback struct { +type DefaultCompactionLifecycleCallback struct { } -func (c NoopCompactionLifecycleCallback) PreCompactionCallback(_ *Group, _ []*metadata.Meta) error { +func (c DefaultCompactionLifecycleCallback) PreCompactionCallback(_ *Group, toCompactBlocks []*metadata.Meta) error { + // Due to #183 we verify that none of the blocks in the plan have overlapping sources. + // This is one potential source of how we could end up with duplicated chunks. + uniqueSources := map[ulid.ULID]struct{}{} + for _, m := range toCompactBlocks { + for _, s := range m.Compaction.Sources { + if _, ok := uniqueSources[s]; ok { + return halt(errors.Errorf("overlapping sources detected for plan %v", toCompactBlocks)) + } + uniqueSources[s] = struct{}{} + } + } return nil } -func (c NoopCompactionLifecycleCallback) PostCompactionCallback(_ *Group, _ ulid.ULID) error { +func (c DefaultCompactionLifecycleCallback) PostCompactionCallback(_ *Group, _ ulid.ULID) error { return nil } +func (c DefaultCompactionLifecycleCallback) GetBlockPopulator(_ *Group, _ log.Logger) (tsdb.PopulateBlockFunc, error) { + return tsdb.DefaultPopulateBlockFunc{}, nil +} + // 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. @@ -1061,41 +1077,35 @@ func (cg *Group) compact(ctx context.Context, dir string, planner Planner, comp return false, ulid.ULID{}, nil } - level.Info(cg.logger).Log("msg", "running pre compaction callback", "plan", fmt.Sprintf("%v", toCompact)) - if err := compactionLifecycleCallback.PreCompactionCallback(cg, toCompact); err != nil { - return false, ulid.ULID{}, errors.Wrapf(err, "failed to run pre compaction callback for plan: %s", fmt.Sprintf("%v", toCompact)) - } - level.Info(cg.logger).Log("msg", "finished running pre compaction callback", "plan", fmt.Sprintf("%v", toCompact)) - - level.Info(cg.logger).Log("msg", "compaction available and planned; downloading blocks", "plan", fmt.Sprintf("%v", toCompact)) - - // Due to #183 we verify that none of the blocks in the plan have overlapping sources. - // This is one potential source of how we could end up with duplicated chunks. - //uniqueSources := map[ulid.ULID]struct{}{} + level.Info(cg.logger).Log("msg", "compaction available and planned", "plan", fmt.Sprintf("%v", toCompact)) // Once we have a plan we need to download the actual data. groupCompactionBegin := time.Now() begin := groupCompactionBegin + + if err := compactionLifecycleCallback.PreCompactionCallback(cg, toCompact); err != nil { + return false, ulid.ULID{}, errors.Wrapf(err, "failed to run pre compaction callback for plan: %s", fmt.Sprintf("%v", toCompact)) + } + level.Info(cg.logger).Log("msg", "finished running pre compaction callback; downloading blocks", "plan", fmt.Sprintf("%v", toCompact), "duration", time.Since(begin), "duration_ms", time.Since(begin).Milliseconds()) + + begin = time.Now() g, errCtx := errgroup.WithContext(ctx) g.SetLimit(cg.compactBlocksFetchConcurrency) toCompactDirs := make([]string, 0, len(toCompact)) for _, m := range toCompact { bdir := filepath.Join(dir, m.ULID.String()) - //for _, s := range m.Compaction.Sources { - // if _, ok := uniqueSources[s]; ok { - // return false, ulid.ULID{}, halt(errors.Errorf("overlapping sources detected for plan %v", toCompact)) - // } - // uniqueSources[s] = struct{}{} - //} func(ctx context.Context, meta *metadata.Meta) { g.Go(func() error { + start := time.Now() if err := tracing.DoInSpanWithErr(ctx, "compaction_block_download", func(ctx context.Context) error { return block.Download(ctx, cg.logger, cg.bkt, meta.ULID, bdir, objstore.WithFetchConcurrency(cg.blockFilesConcurrency)) }, opentracing.Tags{"block.id": meta.ULID}); err != nil { return retry(errors.Wrapf(err, "download block %s", meta.ULID)) } + level.Debug(cg.logger).Log("msg", "downloaded block", "block", meta.ULID.String(), "duration", time.Since(start), "duration_ms", time.Since(start).Milliseconds()) + start = time.Now() // Ensure all input blocks are valid. var stats block.HealthStats if err := tracing.DoInSpanWithErr(ctx, "compaction_block_health_stats", func(ctx context.Context) (e error) { @@ -1121,6 +1131,7 @@ func (cg *Group) compact(ctx context.Context, dir string, planner Planner, comp return errors.Wrapf(err, "block id %s, try running with --debug.accept-malformed-index", meta.ULID) } + level.Debug(cg.logger).Log("msg", "verified block", "block", meta.ULID.String(), "duration", time.Since(start), "duration_ms", time.Since(start).Milliseconds()) return nil }) }(errCtx, m) @@ -1137,9 +1148,9 @@ func (cg *Group) compact(ctx context.Context, dir string, planner Planner, comp begin = time.Now() if err := tracing.DoInSpanWithErr(ctx, "compaction", func(ctx context.Context) (e error) { - populateBlockFunc := ShardedPopulateBlockFunc{ - partitionCount: cg.partitionCount, - partitionId: cg.partitionID, + populateBlockFunc, e := compactionLifecycleCallback.GetBlockPopulator(cg, cg.logger) + if e != nil { + return e } compID, e = comp.CompactWithPopulateBlockFunc(dir, toCompactDirs, nil, populateBlockFunc) return e @@ -1292,7 +1303,7 @@ func NewBucketCompactor( planner, comp, DefaultCompactionCompleteChecker{}, - NoopCompactionLifecycleCallback{}, + DefaultCompactionLifecycleCallback{}, compactDir, bkt, concurrency, diff --git a/pkg/compact/sharded_populate_block_func.go b/pkg/compact/sharded_populate_block_func.go deleted file mode 100644 index e6d920e741..0000000000 --- a/pkg/compact/sharded_populate_block_func.go +++ /dev/null @@ -1,217 +0,0 @@ -package compact - -import ( - "context" - "fmt" - "golang.org/x/sync/errgroup" - "io" - "sync" - "time" - - "github.com/go-kit/log" - "github.com/go-kit/log/level" - "github.com/pkg/errors" - - "github.com/prometheus/prometheus/storage" - "github.com/prometheus/prometheus/tsdb" - "github.com/prometheus/prometheus/tsdb/chunkenc" - "github.com/prometheus/prometheus/tsdb/chunks" - tsdb_errors "github.com/prometheus/prometheus/tsdb/errors" - "github.com/prometheus/prometheus/tsdb/index" -) - -type ShardedPopulateBlockFunc struct { - partitionCount int - partitionId int -} - -// PopulateBlock fills the index and chunk writers with new data gathered as the union -// of the provided blocks. It returns meta information for the new block. -// It expects sorted blocks input by mint. -func (c ShardedPopulateBlockFunc) PopulateBlock(metrics *tsdb.CompactorMetrics, logger log.Logger, chunkPool chunkenc.Pool, ctx context.Context, mergeFunc storage.VerticalChunkSeriesMergeFunc, blocks []tsdb.BlockReader, meta *tsdb.BlockMeta, indexw tsdb.IndexWriter, chunkw tsdb.ChunkWriter) (err error) { - if len(blocks) == 0 { - return errors.New("cannot populate block from no readers") - } - - var ( - sets []storage.ChunkSeriesSet - setsMtx sync.Mutex - symbols index.StringIter - closers []io.Closer - overlapping bool - ) - defer func() { - errs := tsdb_errors.NewMulti(err) - if cerr := tsdb_errors.CloseAll(closers); cerr != nil { - errs.Add(errors.Wrap(cerr, "close")) - } - err = errs.Err() - metrics.PopulatingBlocks.Set(0) - }() - metrics.PopulatingBlocks.Set(1) - - globalMaxt := blocks[0].Meta().MaxTime - g, _ := errgroup.WithContext(ctx) - g.SetLimit(8) - for i, b := range blocks { - select { - case <-ctx.Done(): - return ctx.Err() - default: - } - - if !overlapping { - if i > 0 && b.Meta().MinTime < globalMaxt { - metrics.OverlappingBlocks.Inc() - overlapping = true - level.Info(logger).Log("msg", "Found overlapping blocks during compaction", "ulid", meta.ULID) - } - if b.Meta().MaxTime > globalMaxt { - globalMaxt = b.Meta().MaxTime - } - } - - indexr, err := b.Index() - if err != nil { - return errors.Wrapf(err, "open index reader for block %+v", b.Meta()) - } - closers = append(closers, indexr) - - chunkr, err := b.Chunks() - if err != nil { - return errors.Wrapf(err, "open chunk reader for block %+v", b.Meta()) - } - closers = append(closers, chunkr) - - tombsr, err := b.Tombstones() - if err != nil { - return errors.Wrapf(err, "open tombstone reader for block %+v", b.Meta()) - } - closers = append(closers, tombsr) - - k, v := index.AllPostingsKey() - all, err := indexr.Postings(k, v) - if err != nil { - return err - } - all = indexr.SortedPostings(all) - g.Go(func() error { - shardStart := time.Now() - shardedPosting := index.NewShardedPosting(all, uint64(c.partitionCount), uint64(c.partitionId), indexr.Series) - fmt.Printf("finished sharding, duration: %v\n", time.Since(shardStart)) - // Blocks meta is half open: [min, max), so subtract 1 to ensure we don't hold samples with exact meta.MaxTime timestamp. - setsMtx.Lock() - sets = append(sets, tsdb.NewBlockChunkSeriesSet(meta.ULID, indexr, chunkr, tombsr, shardedPosting, meta.MinTime, meta.MaxTime-1, false)) - setsMtx.Unlock() - return nil - }) - syms := indexr.Symbols() - if i == 0 { - symbols = syms - continue - } - symbols = tsdb.NewMergedStringIter(symbols, syms) - } - if err := g.Wait(); err != nil { - return err - } - - for symbols.Next() { - if err := indexw.AddSymbol(symbols.At()); err != nil { - return errors.Wrap(err, "add symbol") - } - } - if symbols.Err() != nil { - return errors.Wrap(symbols.Err(), "next symbol") - } - - var ( - ref = storage.SeriesRef(0) - ch = make(chan func() error, 1000) - ) - - set := sets[0] - if len(sets) > 1 { - iCtx, cancel := context.WithCancel(ctx) - // Merge series using specified chunk series merger. - // The default one is the compacting series merger. - set = NewBackgroundChunkSeriesSet(iCtx, storage.NewMergeChunkSeriesSet(sets, mergeFunc)) - defer cancel() - } - - go func() { - // Iterate over all sorted chunk series. - for set.Next() { - select { - case <-ctx.Done(): - ch <- func() error { return ctx.Err() } - default: - } - s := set.At() - chksIter := s.Iterator() - - var chks []chunks.Meta - var wg sync.WaitGroup - r := ref - wg.Add(1) - go func() { - for chksIter.Next() { - // We are not iterating in streaming way over chunk as - // it's more efficient to do bulk write for index and - // chunk file purposes. - chks = append(chks, chksIter.At()) - } - wg.Done() - }() - - ch <- func() error { - wg.Wait() - if chksIter.Err() != nil { - return errors.Wrap(chksIter.Err(), "chunk iter") - } - - // Skip the series with all deleted chunks. - if len(chks) == 0 { - return nil - } - - if err := chunkw.WriteChunks(chks...); err != nil { - return errors.Wrap(err, "write chunks") - } - if err := indexw.AddSeries(r, s.Labels(), chks...); err != nil { - return errors.Wrap(err, "add series") - } - - meta.Stats.NumChunks += uint64(len(chks)) - meta.Stats.NumSeries++ - for _, chk := range chks { - meta.Stats.NumSamples += uint64(chk.Chunk.NumSamples()) - } - - for _, chk := range chks { - if err := chunkPool.Put(chk.Chunk); err != nil { - return errors.Wrap(err, "put chunk") - } - } - - return nil - } - - ref++ - } - close(ch) - }() - - for callback := range ch { - err := callback() - if err != nil { - return err - } - } - - if set.Err() != nil { - return errors.Wrap(set.Err(), "iterate compaction set") - } - - return nil -} diff --git a/pkg/compact/sharded_postings.go b/pkg/compact/sharded_postings.go deleted file mode 100644 index 5ae560e4d8..0000000000 --- a/pkg/compact/sharded_postings.go +++ /dev/null @@ -1,69 +0,0 @@ -package compact - -import ( - "fmt" - - "github.com/prometheus/prometheus/model/labels" - "github.com/prometheus/prometheus/storage" - "github.com/prometheus/prometheus/tsdb" - "github.com/prometheus/prometheus/tsdb/chunks" - "github.com/prometheus/prometheus/tsdb/index" -) - -type ShardedPostingsFunc struct { - partitionCount uint64 - partitionID uint64 -} - -func (p *ShardedPostingsFunc) GetPostings(originalPostings index.Postings, indexReader tsdb.IndexReader) index.Postings { - return NewShardedPosting(originalPostings, p.partitionCount, p.partitionID, indexReader.Series) -} - -type ShardedPostings struct { - //postings Postings - //labelsFn func(ref storage.SeriesRef, lset *labels.Labels, chks *[]chunks.Meta) error - series []storage.SeriesRef - cur storage.SeriesRef - initialized bool - - //bufChks []chunks.Meta - //bufLbls labels.Labels -} - -func NewShardedPosting(postings index.Postings, partitionCount uint64, partitionId uint64, labelsFn func(ref storage.SeriesRef, lset *labels.Labels, chks *[]chunks.Meta) error) *ShardedPostings { - bufChks := make([]chunks.Meta, 0) - bufLbls := make(labels.Labels, 0) - series := make([]storage.SeriesRef, 0) - for postings.Next() { - err := labelsFn(postings.At(), &bufLbls, &bufChks) - if err != nil { - fmt.Printf("err: %v", err) - } - if bufLbls.Hash()%partitionCount == partitionId { - posting := postings.At() - series = append(series, posting) - } - } - return &ShardedPostings{series: series, initialized: false} -} - -func (p *ShardedPostings) Next() bool { - if len(p.series) > 0 { - p.cur, p.series = p.series[0], p.series[1:] - return true - } - return false -} - -func (p *ShardedPostings) At() storage.SeriesRef { - return p.cur -} - -func (p *ShardedPostings) Seek(v storage.SeriesRef) bool { - fmt.Println("ran seek") - return false -} - -func (p *ShardedPostings) Err() error { - return nil -} From 554b19dcbded725fcca8e93fe2b70576d35c544a Mon Sep 17 00:00:00 2001 From: Alex Le Date: Wed, 12 Apr 2023 19:38:27 -0700 Subject: [PATCH 05/12] Uncommented code Signed-off-by: Alex Le --- pkg/compact/compact.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/compact/compact.go b/pkg/compact/compact.go index e47b53d77d..49d5e0f4cc 100644 --- a/pkg/compact/compact.go +++ b/pkg/compact/compact.go @@ -177,7 +177,7 @@ func (s *Syncer) GarbageCollect(ctx context.Context) error { if _, exists := deletionMarkMap[id]; exists { continue } - //garbageIDs = append(garbageIDs, id) + garbageIDs = append(garbageIDs, id) } for _, id := range garbageIDs { From 66f60d85dfd24509eb56f7083e537636aa8c37c0 Mon Sep 17 00:00:00 2001 From: Alex Le Date: Thu, 13 Apr 2023 11:33:07 -0700 Subject: [PATCH 06/12] refactor Signed-off-by: Alex Le --- go.mod | 8 ++--- go.sum | 21 +++++------- pkg/compact/compact.go | 74 +++++++++++++++++------------------------- 3 files changed, 41 insertions(+), 62 deletions(-) diff --git a/go.mod b/go.mod index 54a608c806..b69f10da8d 100644 --- a/go.mod +++ b/go.mod @@ -46,7 +46,7 @@ require ( github.com/leanovate/gopter v0.2.9 github.com/lightstep/lightstep-tracer-go v0.25.0 github.com/lovoo/gcloud-opentracing v0.3.0 - github.com/miekg/dns v1.1.51 + github.com/miekg/dns v1.1.53 github.com/minio/minio-go/v7 v7.0.45 // indirect github.com/mwitkow/go-conntrack v0.0.0-20190716064945-2f068394615f github.com/oklog/run v1.1.0 @@ -88,7 +88,7 @@ require ( golang.org/x/sync v0.1.0 golang.org/x/text v0.8.0 golang.org/x/time v0.3.0 - google.golang.org/api v0.111.0 // indirect + google.golang.org/api v0.114.0 // indirect google.golang.org/genproto v0.0.0-20230306155012-7f2fa6fef1f4 // indirect google.golang.org/grpc v1.53.0 google.golang.org/grpc/examples v0.0.0-20211119005141-f45e61797429 @@ -185,7 +185,7 @@ require ( github.com/google/pprof v0.0.0-20230228050547-1710fef4ab10 // indirect github.com/google/uuid v1.3.0 // indirect github.com/googleapis/enterprise-certificate-proxy v0.2.3 // indirect - github.com/googleapis/gax-go/v2 v2.7.0 // indirect + github.com/googleapis/gax-go/v2 v2.7.1 // indirect github.com/grpc-ecosystem/grpc-gateway/v2 v2.15.2 // indirect github.com/jmespath/go-jmespath v0.4.0 // indirect github.com/joeshaw/multierror v0.0.0-20140124173710-69b34d4ec901 // indirect @@ -244,7 +244,7 @@ require ( golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2 // indirect gonum.org/v1/gonum v0.12.0 // indirect google.golang.org/appengine v1.6.7 // indirect - google.golang.org/protobuf v1.29.0 // indirect + google.golang.org/protobuf v1.29.1 // indirect gopkg.in/ini.v1 v1.67.0 // indirect howett.net/plist v0.0.0-20181124034731-591f970eefbb // indirect ) diff --git a/go.sum b/go.sum index 7ee982e27b..751763c0be 100644 --- a/go.sum +++ b/go.sum @@ -227,7 +227,7 @@ github.com/dennwc/varint v1.0.0/go.mod h1:hnItb35rvZvJrbTALZtY/iQfDs48JKRG1RPpgz github.com/dgrijalva/jwt-go v3.2.0+incompatible/go.mod h1:E3ru+11k8xSBh+hMPgOLZmtrrCbhqsmaPHjLKYnJCaQ= github.com/dgryski/go-rendezvous v0.0.0-20200823014737-9f7001d12a5f h1:lO4WD4F/rVNCu3HqELle0jiPLLBs70cWOduZpkS1E78= github.com/dgryski/go-rendezvous v0.0.0-20200823014737-9f7001d12a5f/go.mod h1:cuUVRXasLTGF7a8hSLbxyZXjz+1KgoB3wDUb6vlszIc= -github.com/digitalocean/godo v1.97.0 h1:p9w1yCcWMZcxFSLPToNGXA96WfUVLXqoHti6GzVomL4= +github.com/digitalocean/godo v1.98.0 h1:potyC1eD0N9n5/P4/WmJuKgg+OGYZOBWEW+/aKTX6QQ= github.com/dnaeon/go-vcr v1.2.0 h1:zHCHvJYTMh1N7xnV7zf1m1GPBF9Ad0Jk/whtQ1663qI= github.com/dnaeon/go-vcr v1.2.0/go.mod h1:R4UdLID7HZT3taECzJs4YgbbH6PIGXB6W/sc5OLb6RQ= github.com/docker/distribution v2.8.1+incompatible h1:Q50tZOPR6T/hjNsyc9g8/syEs6bk8XXApsHjKukMl68= @@ -574,7 +574,7 @@ github.com/ianlancetaylor/demangle v0.0.0-20210905161508-09a460cdf81d/go.mod h1: github.com/imdario/mergo v0.3.13 h1:lFzP57bqS/wsqKssCGmtLAb8A0wKjLGrve2q3PPVcBk= github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8= github.com/influxdata/influxdb1-client v0.0.0-20191209144304-8bf82d3c094d/go.mod h1:qj24IKcXYK6Iy9ceXlo3Tc+vtHo9lIhSX5JddghvEPo= -github.com/ionos-cloud/sdk-go/v6 v6.1.4 h1:BJHhFA8Q1SZC7VOXqKKr2BV2ysQ2/4hlk1e4hZte7GY= +github.com/ionos-cloud/sdk-go/v6 v6.1.5 h1:BFqThLOgrGJWeo7w6UDyYuNxyi/GqEmNPl7C/YcQ8Fw= github.com/jessevdk/go-flags v1.4.0/go.mod h1:4FA24M0QyGHXBuZZK/XkWh8h0e1EYbRYJSGM75WSRxI= github.com/jmespath/go-jmespath v0.0.0-20180206201540-c2b33e8439af/go.mod h1:Nht3zPeWKUH0NzdCt2Blrr5ys8VGpn0CEB0cQHVjt7k= github.com/jmespath/go-jmespath v0.4.0 h1:BEgLn5cpjn8UN1mAw4NjwDrS35OdebyEtFe+9YPoQUg= @@ -663,8 +663,8 @@ github.com/matttproud/golang_protobuf_extensions v1.0.4 h1:mmDVorXM7PCGKw94cs5zk github.com/matttproud/golang_protobuf_extensions v1.0.4/go.mod h1:BSXmuO+STAnVfrANrmjBb36TMTDstsz7MSK+HVaYKv4= github.com/mgutz/ansi v0.0.0-20170206155736-9520e82c474b/go.mod h1:01TrycV0kFyexm33Z7vhZRXopbI8J3TDReVlkTgMUxE= github.com/miekg/dns v1.0.14/go.mod h1:W1PPwlIAgtquWBMBEV9nkV9Cazfe8ScdGz/Lj7v3Nrg= -github.com/miekg/dns v1.1.51 h1:0+Xg7vObnhrz/4ZCZcZh7zPXlmU0aveS2HDBd0m0qSo= -github.com/miekg/dns v1.1.51/go.mod h1:2Z9d3CP1LQWihRZUf29mQ19yDThaI4DAYzte2CaQW5c= +github.com/miekg/dns v1.1.53 h1:ZBkuHr5dxHtB1caEOlZTLPo7D3L3TWckgUUs/RHfDxw= +github.com/miekg/dns v1.1.53/go.mod h1:uInx36IzPl7FYnDcMeVWxj9byh7DutNykX4G9Sj60FY= github.com/minio/md5-simd v1.1.2 h1:Gdi1DZK69+ZVMoNHRXJyNcxrMA4dSxoYHZSQbirFg34= github.com/minio/md5-simd v1.1.2/go.mod h1:MzdKDxYpY2BT9XQFocsiZf/NKVtR7nkE4RoEpN+20RM= github.com/minio/minio-go/v7 v7.0.45 h1:g4IeM9M9pW/Lo8AGGNOjBZYlvmtlE1N5TQEYWXRWzIs= @@ -1074,7 +1074,6 @@ golang.org/x/mod v0.4.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.4.1/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.4.2/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91VN4djpZkiMVwK6gcyfeH4XE8wZrZaV4= -golang.org/x/mod v0.7.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= golang.org/x/mod v0.9.0 h1:KENHtAZL2y3NLMYZeHY9DW8HW8V+kQyJsY/V9JlKvCs= golang.org/x/mod v0.9.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= @@ -1133,7 +1132,6 @@ golang.org/x/net v0.0.0-20220624214902-1bab6f366d9e/go.mod h1:XRhObCWvk6IyKnWLug golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= golang.org/x/net v0.0.0-20220909164309-bea034e7d591/go.mod h1:YDH+HFinaLZZlnHAfSS6ZXJJ9M9t4Dl22yv3iI2vPwk= golang.org/x/net v0.1.0/go.mod h1:Cx3nUiGt4eDBEyega/BKRp+/AlGL8hYe7U9odMt2Cco= -golang.org/x/net v0.2.0/go.mod h1:KqCZLdyyvdV855qA2rE3GC2aiw5xGR5TEjj8smXukLY= golang.org/x/net v0.8.0 h1:Zrh2ngAOFYneWTAIAPethzeaQLuHwhuBkuV6ZiRnUaQ= golang.org/x/net v0.8.0/go.mod h1:QVkue5JL9kW//ek3r6jTKnTFis1tRmNAW2P1shuFdJc= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= @@ -1267,13 +1265,11 @@ golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.0.0-20220728004956-3c1f35247d10/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220908164124-27713097b956/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.1.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.2.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.6.0 h1:MVltZSvRTcU2ljQOhs94SXPftV6DCNnZViHeQps87pQ= golang.org/x/sys v0.6.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/term v0.1.0/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= -golang.org/x/term v0.2.0/go.mod h1:TVmDHMZPmdnySmBfhjOoOdhjzdE1h4u1VwSiw2l1Nuc= golang.org/x/term v0.6.0 h1:clScbb1cHjoCkyRbWwBEUZ5H/tIFu5TAXIqaZD0Gcjw= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= @@ -1357,7 +1353,6 @@ golang.org/x/tools v0.1.3/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/tools v0.1.4/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/tools v0.1.5/go.mod h1:o0xws9oXOQQZyjljx8fwUC0k7L1pTE6eaCbjGeHmOkk= golang.org/x/tools v0.1.12/go.mod h1:hNGJHUnrk76NpqgfD5Aqm5Crs+Hm0VOH/i9J2+nxYbc= -golang.org/x/tools v0.3.0/go.mod h1:/rWhSS2+zyEVwoJf8YAX6L2f0ntZ7Kn/mGgAWcipA5k= golang.org/x/tools v0.7.0 h1:W4OVu8VVOaIO0yzWMNdepAulS7YfoS3Zabrm8DOXXU4= golang.org/x/tools v0.7.0/go.mod h1:4pg6aUX35JBAogB10C9AtvVL+qowtN4pT3CGSQex14s= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= @@ -1411,8 +1406,8 @@ google.golang.org/api v0.75.0/go.mod h1:pU9QmyHLnzlpar1Mjt4IbapUCy8J+6HD6GeELN69 google.golang.org/api v0.78.0/go.mod h1:1Sg78yoMLOhlQTeF+ARBoytAcH1NNyyl390YMy6rKmw= google.golang.org/api v0.80.0/go.mod h1:xY3nI94gbvBrE0J6NHXhxOmW97HG7Khjkku6AFB3Hyg= google.golang.org/api v0.84.0/go.mod h1:NTsGnUFJMYROtiquksZHBWtHfeMC7iYthki7Eq3pa8o= -google.golang.org/api v0.111.0 h1:bwKi+z2BsdwYFRKrqwutM+axAlYLz83gt5pDSXCJT+0= -google.golang.org/api v0.111.0/go.mod h1:qtFHvU9mhgTJegR31csQ+rwxyUTHOKFqCKWp1J0fdw0= +google.golang.org/api v0.114.0 h1:1xQPji6cO2E2vLiI+C/XiFAnsn1WV3mjaEwGLhi3grE= +google.golang.org/api v0.114.0/go.mod h1:ifYI2ZsFK6/uGddGfAD5BMxlnkBqCmqHSDUVi45N5Yg= google.golang.org/appengine v1.2.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= @@ -1525,8 +1520,8 @@ google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQ google.golang.org/protobuf v1.27.1/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= google.golang.org/protobuf v1.28.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= google.golang.org/protobuf v1.28.1/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= -google.golang.org/protobuf v1.29.0 h1:44S3JjaKmLEE4YIkjzexaP+NzZsudE3Zin5Njn/pYX0= -google.golang.org/protobuf v1.29.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= +google.golang.org/protobuf v1.29.1 h1:7QBf+IK2gx70Ap/hDsOmam3GE0v9HicjfEdAxE62UoM= +google.golang.org/protobuf v1.29.1/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= diff --git a/pkg/compact/compact.go b/pkg/compact/compact.go index 49d5e0f4cc..1c8204cd82 100644 --- a/pkg/compact/compact.go +++ b/pkg/compact/compact.go @@ -350,9 +350,7 @@ type Group struct { hashFunc metadata.HashFunc blockFilesConcurrency int compactBlocksFetchConcurrency int - partitionedGroupID uint32 - partitionCount int - partitionID int + partitionInfo metadata.PartitionInfo } // NewGroup returns a new compaction group. @@ -493,22 +491,12 @@ func (cg *Group) Resolution() int64 { return cg.resolution } -func (cg *Group) PartitionedGroupID() uint32 { - return cg.partitionedGroupID +func (cg *Group) PartitionedInfo() metadata.PartitionInfo { + return cg.partitionInfo } -func (cg *Group) PartitionCount() int { - return cg.partitionCount -} - -func (cg *Group) PartitionID() int { - return cg.partitionID -} - -func (cg *Group) SetPartitionInfo(partitionedGroupID uint32, partitionCount int, partitionID int) { - cg.partitionedGroupID = partitionedGroupID - cg.partitionCount = partitionCount - cg.partitionID = partitionID +func (cg *Group) SetPartitionInfo(partitionInfo metadata.PartitionInfo) { + cg.partitionInfo = partitionInfo } // CompactProgressMetrics contains Prometheus metrics related to compaction progress. @@ -753,14 +741,14 @@ type Planner interface { PlanWithPartition(ctx context.Context, metasByMinTime []*metadata.Meta, partitionID int, errChan chan error) ([]*metadata.Meta, error) } -type ExtraCompactionCompleteChecker interface { - IsComplete(group *Group, blockID ulid.ULID) bool +type BlockDeletableChecker interface { + CanDelete(group *Group, blockID ulid.ULID) bool } -type DefaultCompactionCompleteChecker struct { +type DefaultBlockDeletableChecker struct { } -func (c DefaultCompactionCompleteChecker) IsComplete(_ *Group, _ ulid.ULID) bool { +func (c DefaultBlockDeletableChecker) CanDelete(_ *Group, _ ulid.ULID) bool { return true } @@ -813,12 +801,12 @@ type Compactor interface { // * The source dirs are marked Deletable. // * Returns empty ulid.ULID{}. Compact(dest string, dirs []string, open []*tsdb.Block) (ulid.ULID, error) - CompactWithPopulateBlockFunc(dest string, dirs []string, open []*tsdb.Block, populateBlockFunc tsdb.PopulateBlockFunc) (ulid.ULID, error) + CompactWithPopulateBlockFunc(dest string, dirs []string, open []*tsdb.Block, blockPopulator tsdb.PopulateBlockFunc) (ulid.ULID, error) } // Compact plans and runs a single compaction against the group. The compacted result // is uploaded into the bucket the blocks were retrieved from. -func (cg *Group) Compact(ctx context.Context, dir string, planner Planner, comp Compactor, completeChecker ExtraCompactionCompleteChecker, compactionLifecycleCallback CompactionLifecycleCallback) (shouldRerun bool, compID ulid.ULID, rerr error) { +func (cg *Group) Compact(ctx context.Context, dir string, planner Planner, comp Compactor, blockDeletableChecker BlockDeletableChecker, compactionLifecycleCallback CompactionLifecycleCallback) (shouldRerun bool, compID ulid.ULID, rerr error) { cg.compactionRunsStarted.Inc() subDir := filepath.Join(dir, cg.Key()) @@ -840,7 +828,7 @@ func (cg *Group) Compact(ctx context.Context, dir string, planner Planner, comp errChan := make(chan error, 1) err := tracing.DoInSpanWithErr(ctx, "compaction_group", func(ctx context.Context) (err error) { - shouldRerun, compID, err = cg.compact(ctx, subDir, planner, comp, completeChecker, compactionLifecycleCallback, errChan) + shouldRerun, compID, err = cg.compact(ctx, subDir, planner, comp, blockDeletableChecker, compactionLifecycleCallback, errChan) return err }, opentracing.Tags{"group.key": cg.Key()}) errChan <- err @@ -1044,7 +1032,7 @@ func RepairIssue347(ctx context.Context, logger log.Logger, bkt objstore.Bucket, return nil } -func (cg *Group) compact(ctx context.Context, dir string, planner Planner, comp Compactor, completeChecker ExtraCompactionCompleteChecker, compactionLifecycleCallback CompactionLifecycleCallback, errChan chan error) (shouldRerun bool, compID ulid.ULID, _ error) { +func (cg *Group) compact(ctx context.Context, dir string, planner Planner, comp Compactor, blockDeletableChecker BlockDeletableChecker, compactionLifecycleCallback CompactionLifecycleCallback, errChan chan error) (shouldRerun bool, compID ulid.ULID, _ error) { cg.mtx.Lock() defer cg.mtx.Unlock() @@ -1062,8 +1050,8 @@ func (cg *Group) compact(ctx context.Context, dir string, planner Planner, comp var toCompact []*metadata.Meta if err := tracing.DoInSpanWithErr(ctx, "compaction_planning", func(ctx context.Context) (e error) { - if cg.partitionCount > 0 { - toCompact, e = planner.PlanWithPartition(ctx, cg.metasByMinTime, cg.partitionID, errChan) + if cg.partitionInfo.PartitionCount > 0 { + toCompact, e = planner.PlanWithPartition(ctx, cg.metasByMinTime, cg.partitionInfo.PartitionID, errChan) return e } else { toCompact, e = planner.Plan(ctx, cg.metasByMinTime) @@ -1162,7 +1150,7 @@ func (cg *Group) compact(ctx context.Context, dir string, planner Planner, comp level.Info(cg.logger).Log("msg", "compacted block would have no samples, deleting source blocks", "blocks", sourceBlockStr) for _, meta := range toCompact { if meta.Stats.NumSamples == 0 { - if err := cg.deleteBlock(meta.ULID, filepath.Join(dir, meta.ULID.String()), completeChecker); err != nil { + if err := cg.deleteBlock(meta.ULID, filepath.Join(dir, meta.ULID.String()), blockDeletableChecker); err != nil { level.Warn(cg.logger).Log("msg", "failed to mark for deletion an empty block found during compaction", "block", meta.ULID) } } @@ -1181,15 +1169,11 @@ func (cg *Group) compact(ctx context.Context, dir string, planner Planner, comp index := filepath.Join(bdir, block.IndexFilename) newMeta, err := metadata.InjectThanos(cg.logger, bdir, metadata.Thanos{ - Labels: cg.labels.Map(), - Downsample: metadata.ThanosDownsample{Resolution: cg.resolution}, - Source: metadata.CompactorSource, - SegmentFiles: block.GetSegmentFiles(bdir), - PartitionInfo: &metadata.PartitionInfo{ - PartitionedGroupID: cg.partitionedGroupID, - PartitionCount: cg.partitionCount, - PartitionID: cg.partitionID, - }, + Labels: cg.labels.Map(), + Downsample: metadata.ThanosDownsample{Resolution: cg.resolution}, + Source: metadata.CompactorSource, + SegmentFiles: block.GetSegmentFiles(bdir), + PartitionInfo: &cg.partitionInfo, }, nil) if err != nil { return false, ulid.ULID{}, errors.Wrapf(err, "failed to finalize the block %s", bdir) @@ -1236,7 +1220,7 @@ func (cg *Group) compact(ctx context.Context, dir string, planner Planner, comp // Eventually the block we just uploaded should get synced into the group again (including sync-delay). for _, meta := range toCompact { err = tracing.DoInSpanWithErr(ctx, "compaction_block_delete", func(ctx context.Context) error { - return cg.deleteBlock(meta.ULID, filepath.Join(dir, meta.ULID.String()), completeChecker) + return cg.deleteBlock(meta.ULID, filepath.Join(dir, meta.ULID.String()), blockDeletableChecker) }, opentracing.Tags{"block.id": meta.ULID}) if err != nil { return false, ulid.ULID{}, retry(errors.Wrapf(err, "mark old block for deletion from bucket")) @@ -1249,12 +1233,12 @@ func (cg *Group) compact(ctx context.Context, dir string, planner Planner, comp return true, compID, nil } -func (cg *Group) deleteBlock(id ulid.ULID, bdir string, completeChecker ExtraCompactionCompleteChecker) error { +func (cg *Group) deleteBlock(id ulid.ULID, bdir string, blockDeletableChecker BlockDeletableChecker) error { if err := os.RemoveAll(bdir); err != nil { return errors.Wrapf(err, "remove old block dir %s", id) } - if completeChecker.IsComplete(cg, id) { + if blockDeletableChecker.CanDelete(cg, 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() @@ -1273,7 +1257,7 @@ type BucketCompactor struct { grouper Grouper comp Compactor planner Planner - completeChecker ExtraCompactionCompleteChecker + blockDeletableChecker BlockDeletableChecker compactionLifecycleCallback CompactionLifecycleCallback compactDir string bkt objstore.Bucket @@ -1302,7 +1286,7 @@ func NewBucketCompactor( grouper, planner, comp, - DefaultCompactionCompleteChecker{}, + DefaultBlockDeletableChecker{}, DefaultCompactionLifecycleCallback{}, compactDir, bkt, @@ -1317,7 +1301,7 @@ func NewBucketCompactorWithCheckerAndCallback( grouper Grouper, planner Planner, comp Compactor, - completeChecker ExtraCompactionCompleteChecker, + blockDeletableChecker BlockDeletableChecker, compactionLifecycleCallback CompactionLifecycleCallback, compactDir string, bkt objstore.Bucket, @@ -1333,7 +1317,7 @@ func NewBucketCompactorWithCheckerAndCallback( grouper: grouper, planner: planner, comp: comp, - completeChecker: completeChecker, + blockDeletableChecker: blockDeletableChecker, compactionLifecycleCallback: compactionLifecycleCallback, compactDir: compactDir, bkt: bkt, @@ -1375,7 +1359,7 @@ func (c *BucketCompactor) Compact(ctx context.Context) (rerr error) { go func() { defer wg.Done() for g := range groupChan { - shouldRerunGroup, _, err := g.Compact(workCtx, c.compactDir, c.planner, c.comp, c.completeChecker, c.compactionLifecycleCallback) + shouldRerunGroup, _, err := g.Compact(workCtx, c.compactDir, c.planner, c.comp, c.blockDeletableChecker, c.compactionLifecycleCallback) if err == nil { if shouldRerunGroup { mtx.Lock() From 42dbe8ad75535e44e615b65d3e91172c3de2117f Mon Sep 17 00:00:00 2001 From: Alex Le Date: Thu, 13 Apr 2023 11:41:09 -0700 Subject: [PATCH 07/12] rename Signed-off-by: Alex Le --- pkg/block/fetcher.go | 22 +++++++++++----------- pkg/compact/compact.go | 4 ++-- 2 files changed, 13 insertions(+), 13 deletions(-) diff --git a/pkg/block/fetcher.go b/pkg/block/fetcher.go index 9bda24c24b..a8582343d7 100644 --- a/pkg/block/fetcher.go +++ b/pkg/block/fetcher.go @@ -577,28 +577,28 @@ func (f *LabelShardedMetaFilter) Filter(_ context.Context, metas map[ulid.ULID]* return nil } -var _ MetadataFilter = &DeduplicateFilter{} +var _ MetadataFilter = &DefaultDeduplicateFilter{} -type IDeduplicateFilter interface { +type DeduplicateFilter interface { DuplicateIDs() []ulid.ULID } -// DeduplicateFilter is a BaseFetcher filter that filters out older blocks that have exactly the same data. +// DefaultDeduplicateFilter is a BaseFetcher filter that filters out older blocks that have exactly the same data. // Not go-routine safe. -type DeduplicateFilter struct { +type DefaultDeduplicateFilter struct { duplicateIDs []ulid.ULID concurrency int mu sync.Mutex } -// NewDeduplicateFilter creates DeduplicateFilter. -func NewDeduplicateFilter(concurrency int) *DeduplicateFilter { - return &DeduplicateFilter{concurrency: concurrency} +// NewDeduplicateFilter creates DefaultDeduplicateFilter. +func NewDeduplicateFilter(concurrency int) *DefaultDeduplicateFilter { + return &DefaultDeduplicateFilter{concurrency: concurrency} } // Filter filters out duplicate blocks that can be formed // from two or more overlapping blocks that fully submatches the source blocks of the older blocks. -func (f *DeduplicateFilter) Filter(_ context.Context, metas map[ulid.ULID]*metadata.Meta, synced GaugeVec, modified GaugeVec) error { +func (f *DefaultDeduplicateFilter) Filter(_ context.Context, metas map[ulid.ULID]*metadata.Meta, synced GaugeVec, modified GaugeVec) error { f.duplicateIDs = f.duplicateIDs[:0] var wg sync.WaitGroup @@ -630,7 +630,7 @@ func (f *DeduplicateFilter) Filter(_ context.Context, metas map[ulid.ULID]*metad return nil } -func (f *DeduplicateFilter) filterGroup(metaSlice []*metadata.Meta, metas map[ulid.ULID]*metadata.Meta, synced GaugeVec) { +func (f *DefaultDeduplicateFilter) filterGroup(metaSlice []*metadata.Meta, metas map[ulid.ULID]*metadata.Meta, synced GaugeVec) { sort.Slice(metaSlice, func(i, j int) bool { ilen := len(metaSlice[i].Compaction.Sources) jlen := len(metaSlice[j].Compaction.Sources) @@ -672,8 +672,8 @@ childLoop: f.mu.Unlock() } -// DuplicateIDs returns slice of block ids that are filtered out by DeduplicateFilter. -func (f *DeduplicateFilter) DuplicateIDs() []ulid.ULID { +// DuplicateIDs returns slice of block ids that are filtered out by DefaultDeduplicateFilter. +func (f *DefaultDeduplicateFilter) DuplicateIDs() []ulid.ULID { return f.duplicateIDs } diff --git a/pkg/compact/compact.go b/pkg/compact/compact.go index 1c8204cd82..5912d24380 100644 --- a/pkg/compact/compact.go +++ b/pkg/compact/compact.go @@ -58,7 +58,7 @@ type Syncer struct { blocks map[ulid.ULID]*metadata.Meta partial map[ulid.ULID]error metrics *syncerMetrics - duplicateBlocksFilter block.IDeduplicateFilter + duplicateBlocksFilter block.DeduplicateFilter ignoreDeletionMarkFilter *block.IgnoreDeletionMarkFilter } @@ -95,7 +95,7 @@ func newSyncerMetrics(reg prometheus.Registerer, blocksMarkedForDeletion, garbag // 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.MetadataFetcher, duplicateBlocksFilter block.IDeduplicateFilter, ignoreDeletionMarkFilter *block.IgnoreDeletionMarkFilter, blocksMarkedForDeletion, garbageCollectedBlocks prometheus.Counter) (*Syncer, error) { +func NewMetaSyncer(logger log.Logger, reg prometheus.Registerer, bkt objstore.Bucket, fetcher block.MetadataFetcher, duplicateBlocksFilter block.DeduplicateFilter, ignoreDeletionMarkFilter *block.IgnoreDeletionMarkFilter, blocksMarkedForDeletion, garbageCollectedBlocks prometheus.Counter) (*Syncer, error) { if logger == nil { logger = log.NewNopLogger() } From ed90f792389fa705f59d12c1599d7795b7c86ceb Mon Sep 17 00:00:00 2001 From: Alex Le Date: Thu, 20 Apr 2023 13:31:21 -0700 Subject: [PATCH 08/12] make Group.partitionInfo nil-able Signed-off-by: Alex Le --- pkg/compact/compact.go | 20 ++++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/pkg/compact/compact.go b/pkg/compact/compact.go index 5912d24380..171eea28df 100644 --- a/pkg/compact/compact.go +++ b/pkg/compact/compact.go @@ -350,7 +350,7 @@ type Group struct { hashFunc metadata.HashFunc blockFilesConcurrency int compactBlocksFetchConcurrency int - partitionInfo metadata.PartitionInfo + partitionInfo *metadata.PartitionInfo } // NewGroup returns a new compaction group. @@ -491,11 +491,11 @@ func (cg *Group) Resolution() int64 { return cg.resolution } -func (cg *Group) PartitionedInfo() metadata.PartitionInfo { +func (cg *Group) PartitionedInfo() *metadata.PartitionInfo { return cg.partitionInfo } -func (cg *Group) SetPartitionInfo(partitionInfo metadata.PartitionInfo) { +func (cg *Group) SetPartitionInfo(partitionInfo *metadata.PartitionInfo) { cg.partitionInfo = partitionInfo } @@ -755,7 +755,7 @@ func (c DefaultBlockDeletableChecker) CanDelete(_ *Group, _ ulid.ULID) bool { type CompactionLifecycleCallback interface { PreCompactionCallback(group *Group, toCompactBlocks []*metadata.Meta) error PostCompactionCallback(group *Group, blockID ulid.ULID) error - GetBlockPopulator(group *Group, logger log.Logger) (tsdb.PopulateBlockFunc, error) + GetBlockPopulator(group *Group, logger log.Logger) (tsdb.BlockPopulator, error) } type DefaultCompactionLifecycleCallback struct { @@ -780,8 +780,8 @@ func (c DefaultCompactionLifecycleCallback) PostCompactionCallback(_ *Group, _ u return nil } -func (c DefaultCompactionLifecycleCallback) GetBlockPopulator(_ *Group, _ log.Logger) (tsdb.PopulateBlockFunc, error) { - return tsdb.DefaultPopulateBlockFunc{}, nil +func (c DefaultCompactionLifecycleCallback) GetBlockPopulator(_ *Group, _ log.Logger) (tsdb.BlockPopulator, error) { + return tsdb.DefaultBlockPopulator{}, nil } // Compactor provides compaction against an underlying storage of time series data. @@ -801,7 +801,7 @@ type Compactor interface { // * The source dirs are marked Deletable. // * Returns empty ulid.ULID{}. Compact(dest string, dirs []string, open []*tsdb.Block) (ulid.ULID, error) - CompactWithPopulateBlockFunc(dest string, dirs []string, open []*tsdb.Block, blockPopulator tsdb.PopulateBlockFunc) (ulid.ULID, error) + CompactWithBlockPopulator(dest string, dirs []string, open []*tsdb.Block, blockPopulator tsdb.BlockPopulator) (ulid.ULID, error) } // Compact plans and runs a single compaction against the group. The compacted result @@ -1050,7 +1050,7 @@ func (cg *Group) compact(ctx context.Context, dir string, planner Planner, comp var toCompact []*metadata.Meta if err := tracing.DoInSpanWithErr(ctx, "compaction_planning", func(ctx context.Context) (e error) { - if cg.partitionInfo.PartitionCount > 0 { + if cg.partitionInfo != nil && cg.partitionInfo.PartitionCount > 0 { toCompact, e = planner.PlanWithPartition(ctx, cg.metasByMinTime, cg.partitionInfo.PartitionID, errChan) return e } else { @@ -1140,7 +1140,7 @@ func (cg *Group) compact(ctx context.Context, dir string, planner Planner, comp if e != nil { return e } - compID, e = comp.CompactWithPopulateBlockFunc(dir, toCompactDirs, nil, populateBlockFunc) + compID, e = comp.CompactWithBlockPopulator(dir, toCompactDirs, nil, populateBlockFunc) return e }); err != nil { return false, ulid.ULID{}, halt(errors.Wrapf(err, "compact blocks %v", toCompactDirs)) @@ -1173,7 +1173,7 @@ func (cg *Group) compact(ctx context.Context, dir string, planner Planner, comp Downsample: metadata.ThanosDownsample{Resolution: cg.resolution}, Source: metadata.CompactorSource, SegmentFiles: block.GetSegmentFiles(bdir), - PartitionInfo: &cg.partitionInfo, + PartitionInfo: cg.partitionInfo, }, nil) if err != nil { return false, ulid.ULID{}, errors.Wrapf(err, "failed to finalize the block %s", bdir) From bc2be701f8619a055dc47362b0457743b486511f Mon Sep 17 00:00:00 2001 From: Alex Le Date: Wed, 7 Jun 2023 20:06:37 -0700 Subject: [PATCH 09/12] add extension field to thanos meta Signed-off-by: Alex Le --- pkg/block/metadata/meta.go | 28 +++++--- pkg/block/metadata/meta_test.go | 121 +++++++++++++++++++------------- pkg/compact/compact.go | 75 +++++++++++--------- pkg/compact/planner.go | 4 +- pkg/compact/planner_test.go | 16 ++--- 5 files changed, 141 insertions(+), 103 deletions(-) diff --git a/pkg/block/metadata/meta.go b/pkg/block/metadata/meta.go index db9381ba51..e6e1958839 100644 --- a/pkg/block/metadata/meta.go +++ b/pkg/block/metadata/meta.go @@ -91,8 +91,26 @@ type Thanos struct { // Rewrites is present when any rewrite (deletion, relabel etc) were applied to this block. Optional. Rewrites []Rewrite `json:"rewrites,omitempty"` - // PartitionInfo is used for partitioning compaction to keep track of partition information of result block. Optional. - PartitionInfo *PartitionInfo `json:"partition_info,omitempty"` + // Extensions are used for plugin any arbitrary additional information for block. Optional. + Extensions any `json:"extensions,omitempty"` +} + +func (m *Thanos) ParseExtensions(v any) (any, error) { + return ConvertExtensions(m.Extensions, v) +} + +func ConvertExtensions(extensions any, v any) (any, error) { + if extensions == nil { + return nil, nil + } + extensionsContent, err := json.Marshal(extensions) + if err != nil { + return nil, err + } + if err = json.Unmarshal(extensionsContent, v); err != nil { + return nil, err + } + return v, nil } type Rewrite struct { @@ -104,12 +122,6 @@ type Rewrite struct { RelabelsApplied []*relabel.Config `json:"relabels_applied,omitempty"` } -type PartitionInfo struct { - PartitionedGroupID uint32 `json:"partitioned_group_id"` - PartitionCount int `json:"partition_count"` - PartitionID int `json:"partition_id"` -} - type Matchers []*labels.Matcher func (m *Matchers) UnmarshalYAML(value *yaml.Node) (err error) { diff --git a/pkg/block/metadata/meta_test.go b/pkg/block/metadata/meta_test.go index e80265d1c2..e7a19f2ab3 100644 --- a/pkg/block/metadata/meta_test.go +++ b/pkg/block/metadata/meta_test.go @@ -212,7 +212,7 @@ func TestMeta_ReadWrite(t *testing.T) { testutil.Equals(t, m1, *retMeta) }) - t.Run("partition info write/read/write", func(t *testing.T) { + t.Run("extensions write/read/write", func(t *testing.T) { b := bytes.Buffer{} m1 := Meta{ BlockMeta: tsdb.BlockMeta{ @@ -221,34 +221,19 @@ func TestMeta_ReadWrite(t *testing.T) { MaxTime: 134, Version: 1, Compaction: tsdb.BlockMetaCompaction{ - Sources: []ulid.ULID{ulid.MustNew(1, nil), ulid.MustNew(2, nil)}, - Parents: []tsdb.BlockDesc{ - { - ULID: ulid.MustNew(3, nil), - MinTime: 442, - MaxTime: 24225, - }, - }, Level: 123, }, Stats: tsdb.BlockStats{NumChunks: 14, NumSamples: 245, NumSeries: 4}, }, Thanos: Thanos{ - Version: 1, - Labels: map[string]string{"ext": "lset1"}, - Source: ReceiveSource, - Files: []File{ - {RelPath: "chunks/000001", SizeBytes: 3751}, - {RelPath: "index", SizeBytes: 401}, - {RelPath: "meta.json"}, - }, + Labels: map[string]string{"ext": "lset1"}, + Source: ReceiveSource, Downsample: ThanosDownsample{ Resolution: 123144, }, - PartitionInfo: &PartitionInfo{ - PartitionedGroupID: 111, - PartitionCount: 8, - PartitionID: 5, + Extensions: &TestExtensions{ + Field1: 1, + Field2: "test_string", }, }, } @@ -263,22 +248,10 @@ func TestMeta_ReadWrite(t *testing.T) { "numChunks": 14 }, "compaction": { - "level": 123, - "sources": [ - "00000000010000000000000000", - "00000000020000000000000000" - ], - "parents": [ - { - "ulid": "00000000030000000000000000", - "minTime": 442, - "maxTime": 24225 - } - ] + "level": 123 }, "version": 1, "thanos": { - "version": 1, "labels": { "ext": "lset1" }, @@ -286,29 +259,77 @@ func TestMeta_ReadWrite(t *testing.T) { "resolution": 123144 }, "source": "receive", - "files": [ - { - "rel_path": "chunks/000001", - "size_bytes": 3751 + "extensions": { + "field1": 1, + "field2": "test_string" + } + } +} +`, b.String()) + retMeta, err := Read(io.NopCloser(&b)) + testutil.Ok(t, err) + retExtensions, err := retMeta.Thanos.ParseExtensions(&TestExtensions{}) + _, ok := retExtensions.(*TestExtensions) + testutil.Equals(t, true, ok) + testutil.Ok(t, err) + testutil.Equals(t, m1.Thanos.Extensions, retExtensions) + }) + + t.Run("empty extensions write/read/write", func(t *testing.T) { + b := bytes.Buffer{} + m1 := Meta{ + BlockMeta: tsdb.BlockMeta{ + ULID: ulid.MustNew(5, nil), + MinTime: 2424, + MaxTime: 134, + Version: 1, + Compaction: tsdb.BlockMetaCompaction{ + Level: 123, + }, + Stats: tsdb.BlockStats{NumChunks: 14, NumSamples: 245, NumSeries: 4}, }, - { - "rel_path": "index", - "size_bytes": 401 + Thanos: Thanos{ + Labels: map[string]string{"ext": "lset1"}, + Source: ReceiveSource, + Downsample: ThanosDownsample{ + Resolution: 123144, + }, }, - { - "rel_path": "meta.json" - } - ], - "partition_info": { - "partitioned_group_id": 111, - "partition_count": 8, - "partition_id": 5 } + testutil.Ok(t, m1.Write(&b)) + testutil.Equals(t, `{ + "ulid": "00000000050000000000000000", + "minTime": 2424, + "maxTime": 134, + "stats": { + "numSamples": 245, + "numSeries": 4, + "numChunks": 14 + }, + "compaction": { + "level": 123 + }, + "version": 1, + "thanos": { + "labels": { + "ext": "lset1" + }, + "downsample": { + "resolution": 123144 + }, + "source": "receive" } } `, b.String()) retMeta, err := Read(io.NopCloser(&b)) testutil.Ok(t, err) - testutil.Equals(t, m1, *retMeta) + retExtensions, err := retMeta.Thanos.ParseExtensions(&TestExtensions{}) + testutil.Ok(t, err) + testutil.Equals(t, m1.Thanos.Extensions, retExtensions) }) } + +type TestExtensions struct { + Field1 int `json:"field1"` + Field2 string `json:"field2"` +} diff --git a/pkg/compact/compact.go b/pkg/compact/compact.go index 8de6f2344d..ab0f91b39d 100644 --- a/pkg/compact/compact.go +++ b/pkg/compact/compact.go @@ -5,6 +5,7 @@ package compact import ( "context" + "encoding/json" "fmt" "math" "os" @@ -350,7 +351,7 @@ type Group struct { hashFunc metadata.HashFunc blockFilesConcurrency int compactBlocksFetchConcurrency int - partitionInfo *metadata.PartitionInfo + extensions any } // NewGroup returns a new compaction group. @@ -491,12 +492,23 @@ func (cg *Group) Resolution() int64 { return cg.resolution } -func (cg *Group) PartitionedInfo() *metadata.PartitionInfo { - return cg.partitionInfo +func (cg *Group) Extensions() any { + return cg.extensions } -func (cg *Group) SetPartitionInfo(partitionInfo *metadata.PartitionInfo) { - cg.partitionInfo = partitionInfo +func (cg *Group) ParseExtensions(v any) error { + extensionsContent, err := json.Marshal(cg.extensions) + if err != nil { + return err + } + if err = json.Unmarshal(extensionsContent, v); err != nil { + return err + } + return nil +} + +func (cg *Group) SetExtensions(extensions any) { + cg.extensions = extensions } // CompactProgressMetrics contains Prometheus metrics related to compaction progress. @@ -544,7 +556,7 @@ func (ps *CompactionProgressCalculator) ProgressCalculate(ctx context.Context, g if len(g.IDs()) == 1 { continue } - plan, err := ps.planner.Plan(ctx, g.metasByMinTime) + plan, err := ps.planner.Plan(ctx, g.metasByMinTime, nil, g.extensions) if err != nil { return errors.Wrapf(err, "could not plan") } @@ -736,9 +748,7 @@ func (rs *RetentionProgressCalculator) ProgressCalculate(ctx context.Context, gr type Planner interface { // Plan returns a list of blocks that should be compacted into single one. // The blocks can be overlapping. The provided metadata has to be ordered by minTime. - Plan(ctx context.Context, metasByMinTime []*metadata.Meta) ([]*metadata.Meta, error) - - PlanWithPartition(ctx context.Context, metasByMinTime []*metadata.Meta, partitionID int, errChan chan error) ([]*metadata.Meta, error) + Plan(ctx context.Context, metasByMinTime []*metadata.Meta, errChan chan error, extensions any) ([]*metadata.Meta, error) } type BlockDeletableChecker interface { @@ -753,15 +763,15 @@ func (c DefaultBlockDeletableChecker) CanDelete(_ *Group, _ ulid.ULID) bool { } type CompactionLifecycleCallback interface { - PreCompactionCallback(group *Group, toCompactBlocks []*metadata.Meta) error - PostCompactionCallback(group *Group, blockID ulid.ULID) error - GetBlockPopulator(group *Group, logger log.Logger) (tsdb.BlockPopulator, error) + PreCompactionCallback(ctx context.Context, logger log.Logger, group *Group, toCompactBlocks []*metadata.Meta) error + PostCompactionCallback(ctx context.Context, logger log.Logger, group *Group, blockID ulid.ULID) error + GetBlockPopulator(ctx context.Context, logger log.Logger, group *Group) (tsdb.BlockPopulator, error) } type DefaultCompactionLifecycleCallback struct { } -func (c DefaultCompactionLifecycleCallback) PreCompactionCallback(_ *Group, toCompactBlocks []*metadata.Meta) error { +func (c DefaultCompactionLifecycleCallback) PreCompactionCallback(_ context.Context, _ log.Logger, _ *Group, toCompactBlocks []*metadata.Meta) error { // Due to #183 we verify that none of the blocks in the plan have overlapping sources. // This is one potential source of how we could end up with duplicated chunks. uniqueSources := map[ulid.ULID]struct{}{} @@ -776,11 +786,11 @@ func (c DefaultCompactionLifecycleCallback) PreCompactionCallback(_ *Group, toCo return nil } -func (c DefaultCompactionLifecycleCallback) PostCompactionCallback(_ *Group, _ ulid.ULID) error { +func (c DefaultCompactionLifecycleCallback) PostCompactionCallback(_ context.Context, _ log.Logger, _ *Group, _ ulid.ULID) error { return nil } -func (c DefaultCompactionLifecycleCallback) GetBlockPopulator(_ *Group, _ log.Logger) (tsdb.BlockPopulator, error) { +func (c DefaultCompactionLifecycleCallback) GetBlockPopulator(_ context.Context, _ log.Logger, _ *Group) (tsdb.BlockPopulator, error) { return tsdb.DefaultBlockPopulator{}, nil } @@ -1050,13 +1060,8 @@ func (cg *Group) compact(ctx context.Context, dir string, planner Planner, comp var toCompact []*metadata.Meta if err := tracing.DoInSpanWithErr(ctx, "compaction_planning", func(ctx context.Context) (e error) { - if cg.partitionInfo != nil && cg.partitionInfo.PartitionCount > 0 { - toCompact, e = planner.PlanWithPartition(ctx, cg.metasByMinTime, cg.partitionInfo.PartitionID, errChan) - return e - } else { - toCompact, e = planner.Plan(ctx, cg.metasByMinTime) - return e - } + toCompact, e = planner.Plan(ctx, cg.metasByMinTime, errChan, cg.extensions) + return e }); err != nil { return false, ulid.ULID{}, errors.Wrap(err, "plan compaction") } @@ -1071,7 +1076,7 @@ func (cg *Group) compact(ctx context.Context, dir string, planner Planner, comp groupCompactionBegin := time.Now() begin := groupCompactionBegin - if err := compactionLifecycleCallback.PreCompactionCallback(cg, toCompact); err != nil { + if err := compactionLifecycleCallback.PreCompactionCallback(ctx, cg.logger, cg, toCompact); err != nil { return false, ulid.ULID{}, errors.Wrapf(err, "failed to run pre compaction callback for plan: %s", fmt.Sprintf("%v", toCompact)) } level.Info(cg.logger).Log("msg", "finished running pre compaction callback; downloading blocks", "plan", fmt.Sprintf("%v", toCompact), "duration", time.Since(begin), "duration_ms", time.Since(begin).Milliseconds()) @@ -1136,7 +1141,7 @@ func (cg *Group) compact(ctx context.Context, dir string, planner Planner, comp begin = time.Now() if err := tracing.DoInSpanWithErr(ctx, "compaction", func(ctx context.Context) (e error) { - populateBlockFunc, e := compactionLifecycleCallback.GetBlockPopulator(cg, cg.logger) + populateBlockFunc, e := compactionLifecycleCallback.GetBlockPopulator(ctx, cg.logger, cg) if e != nil { return e } @@ -1169,11 +1174,11 @@ func (cg *Group) compact(ctx context.Context, dir string, planner Planner, comp index := filepath.Join(bdir, block.IndexFilename) newMeta, err := metadata.InjectThanos(cg.logger, bdir, metadata.Thanos{ - Labels: cg.labels.Map(), - Downsample: metadata.ThanosDownsample{Resolution: cg.resolution}, - Source: metadata.CompactorSource, - SegmentFiles: block.GetSegmentFiles(bdir), - PartitionInfo: cg.partitionInfo, + Labels: cg.labels.Map(), + Downsample: metadata.ThanosDownsample{Resolution: cg.resolution}, + Source: metadata.CompactorSource, + SegmentFiles: block.GetSegmentFiles(bdir), + Extensions: cg.extensions, }, nil) if err != nil { return false, ulid.ULID{}, errors.Wrapf(err, "failed to finalize the block %s", bdir) @@ -1209,12 +1214,6 @@ func (cg *Group) compact(ctx context.Context, dir string, planner Planner, comp } level.Info(cg.logger).Log("msg", "uploaded block", "result_block", compID, "duration", time.Since(begin), "duration_ms", time.Since(begin).Milliseconds()) - level.Info(cg.logger).Log("msg", "running post compaction callback", "result_block", compID) - if err := compactionLifecycleCallback.PostCompactionCallback(cg, compID); err != nil { - return false, ulid.ULID{}, retry(errors.Wrapf(err, "failed to run post compaction callback for result block %s", compID)) - } - level.Info(cg.logger).Log("msg", "finished running post compaction callback", "result_block", compID) - // Mark for deletion the blocks we just compacted from the group and bucket so they do not get included // into the next planning cycle. // Eventually the block we just uploaded should get synced into the group again (including sync-delay). @@ -1228,6 +1227,12 @@ func (cg *Group) compact(ctx context.Context, dir string, planner Planner, comp cg.groupGarbageCollectedBlocks.Inc() } + level.Info(cg.logger).Log("msg", "running post compaction callback", "result_block", compID) + if err := compactionLifecycleCallback.PostCompactionCallback(ctx, cg.logger, cg, compID); err != nil { + return false, ulid.ULID{}, retry(errors.Wrapf(err, "failed to run post compaction callback for result block %s", compID)) + } + level.Info(cg.logger).Log("msg", "finished running post compaction callback", "result_block", compID) + level.Info(cg.logger).Log("msg", "finished compacting blocks", "result_block", compID, "source_blocks", sourceBlockStr, "duration", time.Since(groupCompactionBegin), "duration_ms", time.Since(groupCompactionBegin).Milliseconds()) return true, compID, nil diff --git a/pkg/compact/planner.go b/pkg/compact/planner.go index 18c4e62ccd..808679df11 100644 --- a/pkg/compact/planner.go +++ b/pkg/compact/planner.go @@ -49,7 +49,7 @@ func NewPlanner(logger log.Logger, ranges []int64, noCompBlocks *GatherNoCompact } // TODO(bwplotka): Consider smarter algorithm, this prefers smaller iterative compactions vs big single one: https://github.com/thanos-io/thanos/issues/3405 -func (p *tsdbBasedPlanner) Plan(_ context.Context, metasByMinTime []*metadata.Meta) ([]*metadata.Meta, error) { +func (p *tsdbBasedPlanner) Plan(_ context.Context, metasByMinTime []*metadata.Meta, _ chan error, _ any) ([]*metadata.Meta, error) { return p.plan(p.noCompBlocksFunc(), metasByMinTime) } @@ -247,7 +247,7 @@ func WithLargeTotalIndexSizeFilter(with *tsdbBasedPlanner, bkt objstore.Bucket, return &largeTotalIndexSizeFilter{tsdbBasedPlanner: with, bkt: bkt, totalMaxIndexSizeBytes: totalMaxIndexSizeBytes, markedForNoCompact: markedForNoCompact} } -func (t *largeTotalIndexSizeFilter) Plan(ctx context.Context, metasByMinTime []*metadata.Meta) ([]*metadata.Meta, error) { +func (t *largeTotalIndexSizeFilter) Plan(ctx context.Context, metasByMinTime []*metadata.Meta, _ chan error, _ any) ([]*metadata.Meta, error) { noCompactMarked := t.noCompBlocksFunc() copiedNoCompactMarked := make(map[ulid.ULID]*metadata.NoCompactMark, len(noCompactMarked)) for k, v := range noCompactMarked { diff --git a/pkg/compact/planner_test.go b/pkg/compact/planner_test.go index 9c36dbd54b..256a8be7bb 100644 --- a/pkg/compact/planner_test.go +++ b/pkg/compact/planner_test.go @@ -30,7 +30,7 @@ type tsdbPlannerAdapter struct { comp tsdb.Compactor } -func (p *tsdbPlannerAdapter) Plan(_ context.Context, metasByMinTime []*metadata.Meta) ([]*metadata.Meta, error) { +func (p *tsdbPlannerAdapter) Plan(_ context.Context, metasByMinTime []*metadata.Meta, errChan chan error, _ any) ([]*metadata.Meta, error) { // TSDB planning works based on the meta.json files in the given dir. Mock it up. for _, meta := range metasByMinTime { bdir := filepath.Join(p.dir, meta.ULID.String()) @@ -364,7 +364,7 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { }) tsdbPlanner.dir = dir - plan, err := tsdbPlanner.Plan(context.Background(), metasByMinTime) + plan, err := tsdbPlanner.Plan(context.Background(), metasByMinTime, nil, nil) testutil.Ok(t, err) testutil.Equals(t, c.expected, plan) }) @@ -377,7 +377,7 @@ func TestPlanners_Plan_Compatibility(t *testing.T) { return metasByMinTime[i].MinTime < metasByMinTime[j].MinTime }) - plan, err := tsdbBasedPlanner.Plan(context.Background(), metasByMinTime) + plan, err := tsdbBasedPlanner.Plan(context.Background(), metasByMinTime, nil, nil) testutil.Ok(t, err) testutil.Equals(t, c.expected, plan) }) @@ -440,12 +440,12 @@ func TestRangeWithFailedCompactionWontGetSelected(t *testing.T) { defer func() { testutil.Ok(t, os.RemoveAll(dir)) }() tsdbPlanner.dir = dir - plan, err := tsdbPlanner.Plan(context.Background(), c.metas) + plan, err := tsdbPlanner.Plan(context.Background(), c.metas, nil, nil) testutil.Ok(t, err) testutil.Equals(t, []*metadata.Meta(nil), plan) }) t.Run("tsdbBasedPlanner", func(t *testing.T) { - plan, err := tsdbBasedPlanner.Plan(context.Background(), c.metas) + plan, err := tsdbBasedPlanner.Plan(context.Background(), c.metas, nil, nil) testutil.Ok(t, err) testutil.Equals(t, []*metadata.Meta(nil), plan) }) @@ -638,7 +638,7 @@ func TestTSDBBasedPlanner_PlanWithNoCompactMarks(t *testing.T) { return metasByMinTime[i].MinTime < metasByMinTime[j].MinTime }) g.noCompactMarkedMap = c.noCompactMarks - plan, err := tsdbBasedPlanner.Plan(context.Background(), metasByMinTime) + plan, err := tsdbBasedPlanner.Plan(context.Background(), metasByMinTime, nil, nil) testutil.Ok(t, err) testutil.Equals(t, c.expected, plan) }) @@ -814,7 +814,7 @@ func TestLargeTotalIndexSizeFilter_Plan(t *testing.T) { return metasByMinTime[i].MinTime < metasByMinTime[j].MinTime }) - plan, err := planner.Plan(context.Background(), metasByMinTime) + plan, err := planner.Plan(context.Background(), metasByMinTime, nil, nil) testutil.Ok(t, err) for _, m := range plan { @@ -847,7 +847,7 @@ func TestLargeTotalIndexSizeFilter_Plan(t *testing.T) { m.Thanos = metadata.Thanos{} } - plan, err := planner.Plan(context.Background(), metasByMinTime) + plan, err := planner.Plan(context.Background(), metasByMinTime, nil, nil) testutil.Ok(t, err) testutil.Equals(t, c.expected, plan) testutil.Equals(t, c.expectedMarks, promtest.ToFloat64(marked)-lastMarkValue) From ff552b159d69487311b33797a19c7c51df2efd07 Mon Sep 17 00:00:00 2001 From: Alex Le Date: Tue, 11 Jul 2023 15:47:15 -0700 Subject: [PATCH 10/12] fixed merge issue Signed-off-by: Alex Le --- pkg/block/metadata/meta_test.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/pkg/block/metadata/meta_test.go b/pkg/block/metadata/meta_test.go index 8f1ca560e0..ecfa075228 100644 --- a/pkg/block/metadata/meta_test.go +++ b/pkg/block/metadata/meta_test.go @@ -269,6 +269,7 @@ func TestMeta_ReadWrite(t *testing.T) { "resolution": 123144 }, "source": "receive", + "index_stats": {}, "extensions": { "field1": 1, "field2": "test_string" @@ -327,7 +328,8 @@ func TestMeta_ReadWrite(t *testing.T) { "downsample": { "resolution": 123144 }, - "source": "receive" + "source": "receive", + "index_stats": {} } } `, b.String()) From 8b751472d01f2e96260ccf44860cb0045c759fd6 Mon Sep 17 00:00:00 2001 From: Alex Le Date: Wed, 12 Jul 2023 14:41:37 -0700 Subject: [PATCH 11/12] Clean up Signed-off-by: Alex Le --- pkg/compact/compact.go | 12 ------------ pkg/compact/planner.go | 8 -------- 2 files changed, 20 deletions(-) diff --git a/pkg/compact/compact.go b/pkg/compact/compact.go index 8e680fc43d..2e20ad73db 100644 --- a/pkg/compact/compact.go +++ b/pkg/compact/compact.go @@ -5,7 +5,6 @@ package compact import ( "context" - "encoding/json" "fmt" "math" "os" @@ -497,17 +496,6 @@ func (cg *Group) Extensions() any { return cg.extensions } -func (cg *Group) ParseExtensions(v any) error { - extensionsContent, err := json.Marshal(cg.extensions) - if err != nil { - return err - } - if err = json.Unmarshal(extensionsContent, v); err != nil { - return err - } - return nil -} - func (cg *Group) SetExtensions(extensions any) { cg.extensions = extensions } diff --git a/pkg/compact/planner.go b/pkg/compact/planner.go index 808679df11..783191cacf 100644 --- a/pkg/compact/planner.go +++ b/pkg/compact/planner.go @@ -93,10 +93,6 @@ func (p *tsdbBasedPlanner) plan(noCompactMarked map[ulid.ULID]*metadata.NoCompac return nil, nil } -func (p *tsdbBasedPlanner) PlanWithPartition(ctx context.Context, metasByMinTime []*metadata.Meta, partitionID int, errChan chan error) ([]*metadata.Meta, error) { - return nil, errors.New("not support with partitioning") -} - // selectMetas returns the dir metas that should be compacted into a single new block. // If only a single block range is configured, the result is always nil. // Copied and adjusted from https://github.com/prometheus/prometheus/blob/3d8826a3d42566684283a9b7f7e812e412c24407/tsdb/compact.go#L229. @@ -307,7 +303,3 @@ PlanLoop: return plan, nil } } - -func (t *largeTotalIndexSizeFilter) PlanWithPartition(ctx context.Context, metasByMinTime []*metadata.Meta, partitionID int, errChan chan error) ([]*metadata.Meta, error) { - return nil, errors.New("not support with partitioning") -} From 169f38ef2795d12f14bc00f40440d51e55ad5cc1 Mon Sep 17 00:00:00 2001 From: Alex Le Date: Fri, 14 Jul 2023 09:54:24 -0700 Subject: [PATCH 12/12] Added comment to ConvertExtensions func Signed-off-by: Alex Le --- pkg/block/metadata/meta.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/pkg/block/metadata/meta.go b/pkg/block/metadata/meta.go index 84728451ac..a390ff7ae3 100644 --- a/pkg/block/metadata/meta.go +++ b/pkg/block/metadata/meta.go @@ -107,6 +107,8 @@ func (m *Thanos) ParseExtensions(v any) (any, error) { return ConvertExtensions(m.Extensions, v) } +// ConvertExtensions converts extensions with `any` type into specific type `v` +// that the caller expects. func ConvertExtensions(extensions any, v any) (any, error) { if extensions == nil { return nil, nil