From 9b8a8883deef672427cab90c037e72f829ebe09a Mon Sep 17 00:00:00 2001 From: yeya24 Date: Sun, 17 Oct 2021 23:10:25 -0700 Subject: [PATCH 01/14] add tombstone cli and mask store series using tombstones Signed-off-by: yeya24 --- cmd/thanos/store.go | 11 +++- cmd/thanos/tools_bucket.go | 59 ++++++++++++++++++++++ docs/components/tools.md | 12 +++++ pkg/store/bucket.go | 49 ++++++++++++++++-- pkg/store/bucket_test.go | 2 +- pkg/tombstone/tombstone.go | 100 +++++++++++++++++++++++++++++++++++++ 6 files changed, 226 insertions(+), 7 deletions(-) create mode 100644 pkg/tombstone/tombstone.go diff --git a/cmd/thanos/store.go b/cmd/thanos/store.go index b257e202b1..c356c9198e 100644 --- a/cmd/thanos/store.go +++ b/cmd/thanos/store.go @@ -356,8 +356,8 @@ func runStore( // bucketStoreReady signals when bucket store is ready. bucketStoreReady := make(chan struct{}) + ctx, cancel := context.WithCancel(context.Background()) { - ctx, cancel := context.WithCancel(context.Background()) g.Add(func() error { defer runutil.CloseWithLogOnErr(logger, bkt, "bucket client") @@ -401,6 +401,15 @@ func runStore( }), ) + { + g.Add(func() error { + return runutil.Repeat(time.Minute*3, ctx.Done(), func() error { + return bs.SyncTombstones(ctx) + }) + }, func(error) { + cancel() + }) + } // Start query (proxy) gRPC StoreAPI. { tlsCfg, err := tls.NewServerConfig(log.With(logger, "protocol", "gRPC"), conf.grpcConfig.tlsSrvCert, conf.grpcConfig.tlsSrvKey, conf.grpcConfig.tlsSrvClientCA) diff --git a/cmd/thanos/tools_bucket.go b/cmd/thanos/tools_bucket.go index 1559ec5050..7c6fe01ce3 100644 --- a/cmd/thanos/tools_bucket.go +++ b/cmd/thanos/tools_bucket.go @@ -33,6 +33,7 @@ import ( "github.com/prometheus/common/route" "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/model/relabel" + "github.com/prometheus/prometheus/promql/parser" "github.com/prometheus/prometheus/tsdb" "github.com/prometheus/prometheus/tsdb/chunkenc" @@ -60,6 +61,7 @@ import ( "github.com/thanos-io/thanos/pkg/runutil" httpserver "github.com/thanos-io/thanos/pkg/server/http" "github.com/thanos-io/thanos/pkg/store" + "github.com/thanos-io/thanos/pkg/tombstone" "github.com/thanos-io/thanos/pkg/ui" "github.com/thanos-io/thanos/pkg/verifier" ) @@ -153,6 +155,13 @@ type bucketMarkBlockConfig struct { blockIDs []string } +type bucketDeleteConfig struct { + timeout time.Duration + matchers string + author string + reason string +} + func (tbc *bucketVerifyConfig) registerBucketVerifyFlag(cmd extkingpin.FlagClause) *bucketVerifyConfig { cmd.Flag("repair", "Attempt to repair blocks for which issues were detected"). Short('r').Default("false").BoolVar(&tbc.repair) @@ -264,6 +273,15 @@ func (tbc *bucketRetentionConfig) registerBucketRetentionFlag(cmd extkingpin.Fla return tbc } +func (tbc *bucketDeleteConfig) registerBucketDeleteFlag(cmd extkingpin.FlagClause) *bucketDeleteConfig { + cmd.Flag("timeout", "Timeout to upload tombstone file to the remote storage").Default("5m").DurationVar(&tbc.timeout) + cmd.Flag("matchers", "The string representing label matchers").Default("").StringVar(&tbc.matchers) + cmd.Flag("author", "Author of the deletion request").Default("not specified").StringVar(&tbc.author) + cmd.Flag("reason", "Reason to perform the deletion request").Default("not specified").StringVar(&tbc.reason) + + return tbc +} + func registerBucket(app extkingpin.AppClause) { cmd := app.Command("bucket", "Bucket utility commands") @@ -278,6 +296,7 @@ func registerBucket(app extkingpin.AppClause) { registerBucketMarkBlock(cmd, objStoreConfig) registerBucketRewrite(cmd, objStoreConfig) registerBucketRetention(cmd, objStoreConfig) + registerBucketDelete(cmd, objStoreConfig) } func registerBucketVerify(app extkingpin.AppClause, objStoreConfig *extflag.PathOrContent) { @@ -1367,3 +1386,43 @@ func registerBucketRetention(app extkingpin.AppClause, objStoreConfig *extflag.P return nil }) } + +func registerBucketDelete(app extkingpin.AppClause, objStoreConfig *extflag.PathOrContent) { + cmd := app.Command("delete", "Delete series command for the object storage. NOTE: Currently it only performs Store API masking in the object storage at chunk level with respect to the tombstones created by the user (Doesn't actually delete the data in objstore).") + + tbc := &bucketDeleteConfig{} + tbc.registerBucketDeleteFlag(cmd) + + minTime := model.TimeOrDuration(cmd.Flag("min-time", "Start of time range limit to delete. Option can be a constant time in RFC3339 format or time duration relative to current time, such as -1d (Calculates the actual timestamp at the tombstone creation time) or 2h45m. Valid duration units are ms, s, m, h, d, w, y."). + Default("0000-01-01T00:00:00Z")) + + maxTime := model.TimeOrDuration(cmd.Flag("max-time", "End of time range limit to delete. Option can be a constant time in RFC3339 format or time duration relative to current time, such as -1d (Calculates the actual timestamp at the tombstone creation time) or 2h45m. Valid duration units are ms, s, m, h, d, w, y."). + Default("9999-12-31T23:59:59Z")) + + cmd.Setup(func(g *run.Group, logger log.Logger, reg *prometheus.Registry, _ opentracing.Tracer, _ <-chan struct{}, _ bool) error { + confContentYaml, err := objStoreConfig.Content() + if err != nil { + return err + } + + bkt, err := client.NewBucket(logger, confContentYaml, reg, "delete") + if err != nil { + return err + } + defer runutil.CloseWithLogOnErr(logger, bkt, "tools delete") + + // Dummy actor to immediately kill the group after the run function returns. + g.Add(func() error { return nil }, func(error) {}) + + m, err := parser.ParseMetricSelector(tbc.matchers) + if err != nil { + return err + } + + ts := tombstone.NewTombstone(m, minTime.PrometheusTimestamp(), maxTime.PrometheusTimestamp(), tbc.author, tbc.reason) + + ctx, cancel := context.WithTimeout(context.Background(), tbc.timeout) + defer cancel() + return tombstone.UploadTombstone(ctx, ts, bkt, logger) + }) +} diff --git a/docs/components/tools.md b/docs/components/tools.md index 776164aeb0..4614b7374f 100644 --- a/docs/components/tools.md +++ b/docs/components/tools.md @@ -78,6 +78,12 @@ Subcommands: Retention applies retention policies on the given bucket. Please make sure no compactor is running on the same bucket at the same time. + tools bucket delete [] + Delete series command for the object storage. NOTE: Currently it only + performs Store API masking in the object storage at chunk level with respect + to the tombstones created by the user (Doesn't actually delete the data in + objstore). + tools rules-check --rules=RULES Check if the rule files are valid or not. @@ -187,6 +193,12 @@ Subcommands: Retention applies retention policies on the given bucket. Please make sure no compactor is running on the same bucket at the same time. + tools bucket delete [] + Delete series command for the object storage. NOTE: Currently it only + performs Store API masking in the object storage at chunk level with respect + to the tombstones created by the user (Doesn't actually delete the data in + objstore). + ``` diff --git a/pkg/store/bucket.go b/pkg/store/bucket.go index 679c245fe6..554a25cf18 100644 --- a/pkg/store/bucket.go +++ b/pkg/store/bucket.go @@ -34,6 +34,7 @@ import ( "github.com/prometheus/prometheus/tsdb/chunks" "github.com/prometheus/prometheus/tsdb/encoding" "github.com/prometheus/prometheus/tsdb/index" + promtombstones "github.com/prometheus/prometheus/tsdb/tombstones" "golang.org/x/sync/errgroup" "google.golang.org/grpc/codes" "google.golang.org/grpc/status" @@ -54,6 +55,7 @@ import ( "github.com/thanos-io/thanos/pkg/store/labelpb" "github.com/thanos-io/thanos/pkg/store/storepb" "github.com/thanos-io/thanos/pkg/strutil" + "github.com/thanos-io/thanos/pkg/tombstone" "github.com/thanos-io/thanos/pkg/tracing" ) @@ -304,6 +306,9 @@ type BucketStore struct { advLabelSets []labelpb.ZLabelSet enableCompatibilityLabel bool + tombstonesMtx sync.RWMutex + tombstones []*tombstone.Tombstone + // Every how many posting offset entry we pool in heap memory. Default in Prometheus is 32. postingOffsetsInMemSampling int @@ -783,6 +788,7 @@ func blockSeries( skipChunks bool, // If true, chunks are not loaded. minTime, maxTime int64, // Series must have data in this time range to be returned. loadAggregates []storepb.Aggr, // List of aggregates to load when loading chunks. + tombstones []*tombstone.Tombstone, ) (storepb.SeriesSet, *queryStats, error) { ps, err := indexr.ExpandedPostings(ctx, matchers) if err != nil { @@ -813,6 +819,7 @@ func blockSeries( lset labels.Labels chks []chunks.Meta ) +PostingsLoop: for _, id := range ps { ok, err := indexr.LoadSeriesForTime(id, &symbolizedLset, &chks, skipChunks, minTime, maxTime) if err != nil { @@ -824,11 +831,35 @@ func blockSeries( } s := seriesEntry{} + if err := indexr.LookupLabelsSymbols(symbolizedLset, &lset); err != nil { + return nil, nil, errors.Wrap(err, "Lookup labels symbols") + } + var tombstoneIntervals promtombstones.Intervals + for _, ts := range tombstones { + for _, matcher := range ts.Matchers { + if val := lset.Get(matcher.Name); val != "" { + if matcher.Matches(val) { + if skipChunks { + continue PostingsLoop + } + tombstoneIntervals.Add(promtombstones.Interval{Mint: ts.MinTime, Maxt: ts.MaxTime}) + } + } + } + } + if !skipChunks { // Schedule loading chunks. s.refs = make([]chunks.ChunkRef, 0, len(chks)) s.chks = make([]storepb.AggrChunk, 0, len(chks)) + ChunkMetasLoop: for j, meta := range chks { + for _, it := range tombstoneIntervals { + if meta.OverlapsClosedInterval(it.Mint, it.Maxt) { + continue ChunkMetasLoop + } + } + // seriesEntry s is appended to res, but not at every outer loop iteration, // therefore len(res) is the index we need here, not outer loop iteration number. if err := chunkr.addLoad(meta.Ref, len(res), j); err != nil { @@ -846,9 +877,6 @@ func blockSeries( return nil, nil, errors.Wrap(err, "exceeded chunks limit") } } - if err := indexr.LookupLabelsSymbols(symbolizedLset, &lset); err != nil { - return nil, nil, errors.Wrap(err, "Lookup labels symbols") - } s.lset = labelpb.ExtendSortedLabels(lset, extLset) res = append(res, s) @@ -1017,6 +1045,9 @@ func (s *BucketStore) Series(req *storepb.SeriesRequest, srv storepb.Store_Serie } s.mtx.RLock() + s.tombstonesMtx.RLock() + tombstones := s.tombstones + s.tombstonesMtx.RUnlock() for _, bs := range s.blockSets { blockMatchers, ok := bs.labelMatchers(matchers...) if !ok { @@ -1069,6 +1100,7 @@ func (s *BucketStore) Series(req *storepb.SeriesRequest, srv storepb.Store_Serie req.SkipChunks, req.MinTime, req.MaxTime, req.Aggregates, + tombstones, ) if err != nil { return errors.Wrapf(err, "fetch series for block %s", b.meta.ULID) @@ -1271,7 +1303,7 @@ func (s *BucketStore) LabelNames(ctx context.Context, req *storepb.LabelNamesReq result = strutil.MergeSlices(res, extRes) } else { - seriesSet, _, err := blockSeries(newCtx, b.extLset, indexr, nil, reqSeriesMatchers, nil, seriesLimiter, true, req.Start, req.End, nil) + seriesSet, _, err := blockSeries(newCtx, b.extLset, indexr, nil, reqSeriesMatchers, nil, seriesLimiter, true, req.Start, req.End, nil, nil) if err != nil { return errors.Wrapf(err, "fetch series for block %s", b.meta.ULID) } @@ -1402,7 +1434,7 @@ func (s *BucketStore) LabelValues(ctx context.Context, req *storepb.LabelValuesR } result = res } else { - seriesSet, _, err := blockSeries(newCtx, b.extLset, indexr, nil, reqSeriesMatchers, nil, seriesLimiter, true, req.Start, req.End, nil) + seriesSet, _, err := blockSeries(newCtx, b.extLset, indexr, nil, reqSeriesMatchers, nil, seriesLimiter, true, req.Start, req.End, nil, nil) if err != nil { return errors.Wrapf(err, "fetch series for block %s", b.meta.ULID) } @@ -1455,6 +1487,13 @@ func (s *BucketStore) LabelValues(ctx context.Context, req *storepb.LabelValuesR }, nil } +func (s *BucketStore) SyncTombstones(ctx context.Context) (err error) { + s.tombstonesMtx.Lock() + s.tombstones, err = tombstone.ReadTombstones(ctx, s.bkt, s.logger) + s.tombstonesMtx.Unlock() + return +} + // bucketBlockSet holds all blocks of an equal label set. It internally splits // them up by downsampling resolution and allows querying. type bucketBlockSet struct { diff --git a/pkg/store/bucket_test.go b/pkg/store/bucket_test.go index 381b0f5053..0768b61deb 100644 --- a/pkg/store/bucket_test.go +++ b/pkg/store/bucket_test.go @@ -2290,7 +2290,7 @@ func benchmarkBlockSeriesWithConcurrency(b *testing.B, concurrency int, blockMet indexReader := blk.indexReader() chunkReader := blk.chunkReader() - seriesSet, _, err := blockSeries(context.Background(), nil, indexReader, chunkReader, matchers, chunksLimiter, seriesLimiter, req.SkipChunks, req.MinTime, req.MaxTime, req.Aggregates) + seriesSet, _, err := blockSeries(context.Background(), nil, indexReader, chunkReader, matchers, chunksLimiter, seriesLimiter, req.SkipChunks, req.MinTime, req.MaxTime, req.Aggregates, nil) testutil.Ok(b, err) // Ensure at least 1 series has been returned (as expected). diff --git a/pkg/tombstone/tombstone.go b/pkg/tombstone/tombstone.go new file mode 100644 index 0000000000..3e876f76e6 --- /dev/null +++ b/pkg/tombstone/tombstone.go @@ -0,0 +1,100 @@ +// Copyright (c) The Thanos Authors. +// Licensed under the Apache License 2.0. + +package tombstone + +import ( + "context" + "encoding/json" + "fmt" + "io/ioutil" + "os" + "path" + "time" + + "github.com/go-kit/kit/log" + "github.com/oklog/ulid" + "github.com/prometheus/prometheus/pkg/timestamp" + + "github.com/thanos-io/thanos/pkg/block/metadata" + "github.com/thanos-io/thanos/pkg/objstore" + "github.com/thanos-io/thanos/pkg/runutil" +) + +const ( + // TombstoneDir is the name of directory to upload tombstones. + TombstoneDir = "thanos/tombstones" +) + +// Tombstone represents a tombstone. +type Tombstone struct { + Matchers metadata.Matchers `json:"matchers"` + MinTime int64 `json:"minTime"` + MaxTime int64 `json:"maxTime"` + CreationTime int64 `json:"creationTime"` + Author string `json:"author"` + Reason string `json:"reason"` +} + +// NewTombstone returns a new instance of Tombstone. +func NewTombstone(matchers metadata.Matchers, minTime, maxTime int64, author string, reason string) *Tombstone { + return &Tombstone{ + Matchers: matchers, + MinTime: minTime, + MaxTime: maxTime, + CreationTime: timestamp.FromTime(time.Now()), + Author: author, + Reason: reason, + } +} + +// GenName generates file name based on Matchers, MinTime and MaxTime of a tombstone. +func GenName() string { + return fmt.Sprintf("tombstones-%s.json", ulid.MustNew(uint64(time.Now().Unix()), nil)) +} + +// UploadTombstone uploads the given tombstone to object storage. +func UploadTombstone(ctx context.Context, tombstone *Tombstone, bkt objstore.Bucket, logger log.Logger) error { + b, err := json.Marshal(tombstone) + if err != nil { + return err + } + + tmpDir := os.TempDir() + + tsPath := tmpDir + "/tombstone.json" + if err := ioutil.WriteFile(tsPath, b, 0644); err != nil { + return err + } + + return objstore.UploadFile(ctx, logger, bkt, tsPath, path.Join(TombstoneDir, GenName())) + +} + +// ReadTombstones returns all the tombstones present in the object storage. +func ReadTombstones(ctx context.Context, bkt objstore.InstrumentedBucketReader, logger log.Logger) ([]*Tombstone, error) { + var ts []*Tombstone + + if err := bkt.Iter(ctx, TombstoneDir, func(name string) error { + tombstoneFilename := path.Join("", name) + tombstoneFile, err := bkt.Get(ctx, tombstoneFilename) + if err != nil { + return nil + } + defer runutil.CloseWithLogOnErr(logger, tombstoneFile, "close bkt tombstone reader") + + var t *Tombstone + tombstone, err := ioutil.ReadAll(tombstoneFile) + if err != nil { + return nil + } + if err := json.Unmarshal(tombstone, t); err != nil { + return nil + } + ts = append(ts, t) + return nil + }); err != nil { + return nil, err + } + return ts, nil +} From bd4c341683ca3206d6f7e09ff7f3321707b94b39 Mon Sep 17 00:00:00 2001 From: Ben Ye Date: Wed, 15 Jun 2022 17:05:31 -0700 Subject: [PATCH 02/14] update imports Signed-off-by: Ben Ye --- cmd/thanos/tools_bucket.go | 2 +- pkg/{ => block}/tombstone/tombstone.go | 2 +- pkg/store/bucket.go | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) rename pkg/{ => block}/tombstone/tombstone.go (98%) diff --git a/cmd/thanos/tools_bucket.go b/cmd/thanos/tools_bucket.go index 7c6fe01ce3..ddc1d69503 100644 --- a/cmd/thanos/tools_bucket.go +++ b/cmd/thanos/tools_bucket.go @@ -9,6 +9,7 @@ import ( "encoding/csv" "encoding/json" "fmt" + "github.com/thanos-io/thanos/pkg/block/tombstone" "io" "io/ioutil" "net/http" @@ -61,7 +62,6 @@ import ( "github.com/thanos-io/thanos/pkg/runutil" httpserver "github.com/thanos-io/thanos/pkg/server/http" "github.com/thanos-io/thanos/pkg/store" - "github.com/thanos-io/thanos/pkg/tombstone" "github.com/thanos-io/thanos/pkg/ui" "github.com/thanos-io/thanos/pkg/verifier" ) diff --git a/pkg/tombstone/tombstone.go b/pkg/block/tombstone/tombstone.go similarity index 98% rename from pkg/tombstone/tombstone.go rename to pkg/block/tombstone/tombstone.go index 3e876f76e6..0073807496 100644 --- a/pkg/tombstone/tombstone.go +++ b/pkg/block/tombstone/tombstone.go @@ -14,7 +14,7 @@ import ( "github.com/go-kit/kit/log" "github.com/oklog/ulid" - "github.com/prometheus/prometheus/pkg/timestamp" + "github.com/prometheus/prometheus/model/timestamp" "github.com/thanos-io/thanos/pkg/block/metadata" "github.com/thanos-io/thanos/pkg/objstore" diff --git a/pkg/store/bucket.go b/pkg/store/bucket.go index 554a25cf18..5c0225ab25 100644 --- a/pkg/store/bucket.go +++ b/pkg/store/bucket.go @@ -42,6 +42,7 @@ import ( "github.com/thanos-io/thanos/pkg/block" "github.com/thanos-io/thanos/pkg/block/indexheader" "github.com/thanos-io/thanos/pkg/block/metadata" + "github.com/thanos-io/thanos/pkg/block/tombstone" "github.com/thanos-io/thanos/pkg/compact/downsample" "github.com/thanos-io/thanos/pkg/component" "github.com/thanos-io/thanos/pkg/extprom" @@ -55,7 +56,6 @@ import ( "github.com/thanos-io/thanos/pkg/store/labelpb" "github.com/thanos-io/thanos/pkg/store/storepb" "github.com/thanos-io/thanos/pkg/strutil" - "github.com/thanos-io/thanos/pkg/tombstone" "github.com/thanos-io/thanos/pkg/tracing" ) From 09c8316df196794c58f8b699dd893bed28d10482 Mon Sep 17 00:00:00 2001 From: Ben Ye Date: Wed, 15 Jun 2022 17:32:48 -0700 Subject: [PATCH 03/14] add sync tombstone duration flag Signed-off-by: Ben Ye --- cmd/thanos/store.go | 4 ++++ docs/components/store.md | 3 +++ go.mod | 2 +- pkg/store/bucket.go | 11 ++++++----- 4 files changed, 14 insertions(+), 6 deletions(-) diff --git a/cmd/thanos/store.go b/cmd/thanos/store.go index c356c9198e..cca15e6089 100644 --- a/cmd/thanos/store.go +++ b/cmd/thanos/store.go @@ -62,6 +62,7 @@ type storeConfig struct { component component.StoreAPI debugLogging bool syncInterval time.Duration + syncTombstonesInterval time.Duration blockSyncConcurrency int blockMetaFetchConcurrency int filterConf *store.FilterConfig @@ -123,6 +124,9 @@ func (sc *storeConfig) registerFlag(cmd extkingpin.FlagClause) { cmd.Flag("block-meta-fetch-concurrency", "Number of goroutines to use when fetching block metadata from object storage."). Default("32").IntVar(&sc.blockMetaFetchConcurrency) + cmd.Flag("sync-tombstone-duration", "Repeat interval for syncing the tombstones between local and remote view."). + Default("3m").DurationVar(&sc.syncInterval) + sc.filterConf = &store.FilterConfig{} cmd.Flag("min-time", "Start of time range limit to serve. Thanos Store will serve only metrics, which happened later than this value. Option can be a constant time in RFC3339 format or time duration relative to current time, such as -1d or 2h45m. Valid duration units are ms, s, m, h, d, w, y."). diff --git a/docs/components/store.md b/docs/components/store.md index 3071e7b928..5aaf12af02 100644 --- a/docs/components/store.md +++ b/docs/components/store.md @@ -180,6 +180,9 @@ Flags: this limit is exceeded. 0 means no limit. --sync-block-duration=3m Repeat interval for syncing the blocks between local and remote view. + --sync-tombstone-duration=3m + Repeat interval for syncing the tombstones + between local and remote view. --tracing.config= Alternative to 'tracing.config-file' flag (mutually exclusive). Content of YAML file with diff --git a/go.mod b/go.mod index 023c7397db..b71e220d4b 100644 --- a/go.mod +++ b/go.mod @@ -31,7 +31,7 @@ require ( github.com/felixge/fgprof v0.9.1 github.com/fortytw2/leaktest v1.3.0 github.com/fsnotify/fsnotify v1.5.1 - github.com/go-kit/kit v0.12.0 // indirect + github.com/go-kit/kit v0.12.0 github.com/go-kit/log v0.2.0 github.com/go-openapi/strfmt v0.21.1 github.com/go-redis/redis/v8 v8.11.4 diff --git a/pkg/store/bucket.go b/pkg/store/bucket.go index 5c0225ab25..5a9d842d98 100644 --- a/pkg/store/bucket.go +++ b/pkg/store/bucket.go @@ -838,12 +838,13 @@ PostingsLoop: for _, ts := range tombstones { for _, matcher := range ts.Matchers { if val := lset.Get(matcher.Name); val != "" { - if matcher.Matches(val) { - if skipChunks { - continue PostingsLoop - } - tombstoneIntervals.Add(promtombstones.Interval{Mint: ts.MinTime, Maxt: ts.MaxTime}) + if !matcher.Matches(val) { + continue } + if skipChunks { + continue PostingsLoop + } + tombstoneIntervals.Add(promtombstones.Interval{Mint: ts.MinTime, Maxt: ts.MaxTime}) } } } From c9387cb345b033f41c4b443735b21a794f91cbe1 Mon Sep 17 00:00:00 2001 From: Ben Ye Date: Wed, 15 Jun 2022 18:14:31 -0700 Subject: [PATCH 04/14] lint Signed-off-by: Ben Ye --- cmd/thanos/store.go | 4 ++-- cmd/thanos/tools_bucket.go | 6 ++++-- pkg/store/bucket.go | 2 +- pkg/{block => }/tombstone/tombstone.go | 10 ++++------ 4 files changed, 11 insertions(+), 11 deletions(-) rename pkg/{block => }/tombstone/tombstone.go (89%) diff --git a/cmd/thanos/store.go b/cmd/thanos/store.go index cca15e6089..f44909892d 100644 --- a/cmd/thanos/store.go +++ b/cmd/thanos/store.go @@ -125,7 +125,7 @@ func (sc *storeConfig) registerFlag(cmd extkingpin.FlagClause) { Default("32").IntVar(&sc.blockMetaFetchConcurrency) cmd.Flag("sync-tombstone-duration", "Repeat interval for syncing the tombstones between local and remote view."). - Default("3m").DurationVar(&sc.syncInterval) + Default("3m").DurationVar(&sc.syncTombstonesInterval) sc.filterConf = &store.FilterConfig{} @@ -407,7 +407,7 @@ func runStore( { g.Add(func() error { - return runutil.Repeat(time.Minute*3, ctx.Done(), func() error { + return runutil.Repeat(conf.syncTombstonesInterval, ctx.Done(), func() error { return bs.SyncTombstones(ctx) }) }, func(error) { diff --git a/cmd/thanos/tools_bucket.go b/cmd/thanos/tools_bucket.go index ddc1d69503..071ff279c3 100644 --- a/cmd/thanos/tools_bucket.go +++ b/cmd/thanos/tools_bucket.go @@ -9,7 +9,8 @@ import ( "encoding/csv" "encoding/json" "fmt" - "github.com/thanos-io/thanos/pkg/block/tombstone" + "github.com/prometheus/prometheus/model/timestamp" + "github.com/thanos-io/thanos/pkg/tombstone" "io" "io/ioutil" "net/http" @@ -1419,7 +1420,8 @@ func registerBucketDelete(app extkingpin.AppClause, objStoreConfig *extflag.Path return err } - ts := tombstone.NewTombstone(m, minTime.PrometheusTimestamp(), maxTime.PrometheusTimestamp(), tbc.author, tbc.reason) + ts := tombstone.NewTombstone(m, minTime.PrometheusTimestamp(), maxTime.PrometheusTimestamp(), + timestamp.FromTime(time.Now()), tbc.author, tbc.reason) ctx, cancel := context.WithTimeout(context.Background(), tbc.timeout) defer cancel() diff --git a/pkg/store/bucket.go b/pkg/store/bucket.go index 5a9d842d98..8eabeeeade 100644 --- a/pkg/store/bucket.go +++ b/pkg/store/bucket.go @@ -42,7 +42,6 @@ import ( "github.com/thanos-io/thanos/pkg/block" "github.com/thanos-io/thanos/pkg/block/indexheader" "github.com/thanos-io/thanos/pkg/block/metadata" - "github.com/thanos-io/thanos/pkg/block/tombstone" "github.com/thanos-io/thanos/pkg/compact/downsample" "github.com/thanos-io/thanos/pkg/component" "github.com/thanos-io/thanos/pkg/extprom" @@ -56,6 +55,7 @@ import ( "github.com/thanos-io/thanos/pkg/store/labelpb" "github.com/thanos-io/thanos/pkg/store/storepb" "github.com/thanos-io/thanos/pkg/strutil" + "github.com/thanos-io/thanos/pkg/tombstone" "github.com/thanos-io/thanos/pkg/tracing" ) diff --git a/pkg/block/tombstone/tombstone.go b/pkg/tombstone/tombstone.go similarity index 89% rename from pkg/block/tombstone/tombstone.go rename to pkg/tombstone/tombstone.go index 0073807496..8006ce3587 100644 --- a/pkg/block/tombstone/tombstone.go +++ b/pkg/tombstone/tombstone.go @@ -12,10 +12,8 @@ import ( "path" "time" - "github.com/go-kit/kit/log" + "github.com/go-kit/log" "github.com/oklog/ulid" - "github.com/prometheus/prometheus/model/timestamp" - "github.com/thanos-io/thanos/pkg/block/metadata" "github.com/thanos-io/thanos/pkg/objstore" "github.com/thanos-io/thanos/pkg/runutil" @@ -23,7 +21,7 @@ import ( const ( // TombstoneDir is the name of directory to upload tombstones. - TombstoneDir = "thanos/tombstones" + TombstoneDir = "tombstones" ) // Tombstone represents a tombstone. @@ -37,12 +35,12 @@ type Tombstone struct { } // NewTombstone returns a new instance of Tombstone. -func NewTombstone(matchers metadata.Matchers, minTime, maxTime int64, author string, reason string) *Tombstone { +func NewTombstone(matchers metadata.Matchers, minTime, maxTime, creationTime int64, author, reason string) *Tombstone { return &Tombstone{ Matchers: matchers, MinTime: minTime, MaxTime: maxTime, - CreationTime: timestamp.FromTime(time.Now()), + CreationTime: creationTime, Author: author, Reason: reason, } From b5551f172c3cd86ed81ab637a2c0698382def380 Mon Sep 17 00:00:00 2001 From: Ben Ye Date: Fri, 17 Jun 2022 22:23:03 -0700 Subject: [PATCH 05/14] improve locking Signed-off-by: Ben Ye --- cmd/thanos/tools_bucket.go | 4 ++-- go.mod | 2 +- pkg/store/bucket.go | 10 +++++++--- 3 files changed, 10 insertions(+), 6 deletions(-) diff --git a/cmd/thanos/tools_bucket.go b/cmd/thanos/tools_bucket.go index 071ff279c3..4261727f63 100644 --- a/cmd/thanos/tools_bucket.go +++ b/cmd/thanos/tools_bucket.go @@ -9,8 +9,6 @@ import ( "encoding/csv" "encoding/json" "fmt" - "github.com/prometheus/prometheus/model/timestamp" - "github.com/thanos-io/thanos/pkg/tombstone" "io" "io/ioutil" "net/http" @@ -35,6 +33,7 @@ import ( "github.com/prometheus/common/route" "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/model/relabel" + "github.com/prometheus/prometheus/model/timestamp" "github.com/prometheus/prometheus/promql/parser" "github.com/prometheus/prometheus/tsdb" "github.com/prometheus/prometheus/tsdb/chunkenc" @@ -63,6 +62,7 @@ import ( "github.com/thanos-io/thanos/pkg/runutil" httpserver "github.com/thanos-io/thanos/pkg/server/http" "github.com/thanos-io/thanos/pkg/store" + "github.com/thanos-io/thanos/pkg/tombstone" "github.com/thanos-io/thanos/pkg/ui" "github.com/thanos-io/thanos/pkg/verifier" ) diff --git a/go.mod b/go.mod index b71e220d4b..29e90200e0 100644 --- a/go.mod +++ b/go.mod @@ -31,7 +31,6 @@ require ( github.com/felixge/fgprof v0.9.1 github.com/fortytw2/leaktest v1.3.0 github.com/fsnotify/fsnotify v1.5.1 - github.com/go-kit/kit v0.12.0 github.com/go-kit/log v0.2.0 github.com/go-openapi/strfmt v0.21.1 github.com/go-redis/redis/v8 v8.11.4 @@ -131,6 +130,7 @@ require ( github.com/elastic/go-sysinfo v1.1.1 // indirect github.com/elastic/go-windows v1.0.1 // indirect github.com/felixge/httpsnoop v1.0.2 // indirect + github.com/go-kit/kit v0.12.0 // indirect github.com/go-logfmt/logfmt v0.5.1 // indirect github.com/go-logr/logr v1.2.2 // indirect github.com/go-logr/stdr v1.2.2 // indirect diff --git a/pkg/store/bucket.go b/pkg/store/bucket.go index 8eabeeeade..2cd2fc60fd 100644 --- a/pkg/store/bucket.go +++ b/pkg/store/bucket.go @@ -1488,11 +1488,15 @@ func (s *BucketStore) LabelValues(ctx context.Context, req *storepb.LabelValuesR }, nil } -func (s *BucketStore) SyncTombstones(ctx context.Context) (err error) { +func (s *BucketStore) SyncTombstones(ctx context.Context) error { + tombstones, err := tombstone.ReadTombstones(ctx, s.bkt, s.logger) + if err != nil { + return err + } s.tombstonesMtx.Lock() - s.tombstones, err = tombstone.ReadTombstones(ctx, s.bkt, s.logger) + s.tombstones = tombstones s.tombstonesMtx.Unlock() - return + return nil } // bucketBlockSet holds all blocks of an equal label set. It internally splits From 2cbb32b4d081bd6eac02fcb330532d4097f54f86 Mon Sep 17 00:00:00 2001 From: Ben Ye Date: Sat, 18 Jun 2022 00:35:12 -0700 Subject: [PATCH 06/14] add separate matchers parsing functions Signed-off-by: Ben Ye --- cmd/thanos/tools_bucket.go | 2 +- pkg/block/metadata/matchers.go | 191 ++++++++++++++++++++ pkg/block/metadata/matchers_test.go | 264 ++++++++++++++++++++++++++++ pkg/block/metadata/meta.go | 12 -- pkg/store/bucket.go | 9 +- pkg/tombstone/tombstone.go | 37 ++-- 6 files changed, 479 insertions(+), 36 deletions(-) create mode 100644 pkg/block/metadata/matchers.go create mode 100644 pkg/block/metadata/matchers_test.go diff --git a/cmd/thanos/tools_bucket.go b/cmd/thanos/tools_bucket.go index 4261727f63..6ab7dc5a71 100644 --- a/cmd/thanos/tools_bucket.go +++ b/cmd/thanos/tools_bucket.go @@ -1425,6 +1425,6 @@ func registerBucketDelete(app extkingpin.AppClause, objStoreConfig *extflag.Path ctx, cancel := context.WithTimeout(context.Background(), tbc.timeout) defer cancel() - return tombstone.UploadTombstone(ctx, ts, bkt, logger) + return tombstone.UploadTombstone(ctx, ts, bkt) }) } diff --git a/pkg/block/metadata/matchers.go b/pkg/block/metadata/matchers.go new file mode 100644 index 0000000000..e747a84e55 --- /dev/null +++ b/pkg/block/metadata/matchers.go @@ -0,0 +1,191 @@ +// Copyright (c) The Thanos Authors. +// Licensed under the Apache License 2.0. + +// Copyright 2018 Prometheus Team +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package metadata + +import ( + "encoding/json" + "regexp" + "strings" + "unicode/utf8" + + "github.com/pkg/errors" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/promql/parser" + "gopkg.in/yaml.v3" +) + +var ( + // '=~' has to come before '=' because otherwise only the '=' + // will be consumed, and the '~' will be part of the 3rd token. + re = regexp.MustCompile(`^\s*([a-zA-Z_:][a-zA-Z0-9_:]*)\s*(=~|=|!=|!~)\s*((?s).*?)\s*$`) + typeMap = map[string]labels.MatchType{ + "=": labels.MatchEqual, + "!=": labels.MatchNotEqual, + "=~": labels.MatchRegexp, + "!~": labels.MatchNotRegexp, + } +) + +type Matchers []*labels.Matcher + +func (m *Matchers) UnmarshalYAML(value *yaml.Node) (err error) { + *m, err = parser.ParseMetricSelector(value.Value) + if err != nil { + return errors.Wrapf(err, "parse metric selector %v", value.Value) + } + return nil +} + +func (m *Matchers) MarshalJSON() ([]byte, error) { + if len(*m) == 0 { + return []byte("[]"), nil + } + result := make([]string, len(*m)) + for i, matcher := range *m { + result[i] = matcher.String() + } + return json.Marshal(result) +} + +func (m *Matchers) UnmarshalJSON(data []byte) (err error) { + var lines []string + if err := json.Unmarshal(data, &lines); err != nil { + return err + } + var pm []*labels.Matcher + for _, line := range lines { + pm, err = ParseMatchers(line) + if err != nil { + return errors.Wrapf(err, "parse matchers %v", line) + } + *m = append(*m, pm...) + } + return nil +} + +// ParseMatchers parses label matchers from string. +// Copied from https://github.com/prometheus/alertmanager/blob/v0.24.0/pkg/labels/parse.go#L55. +func ParseMatchers(s string) ([]*labels.Matcher, error) { + matchers := []*labels.Matcher{} + s = strings.TrimPrefix(s, "{") + s = strings.TrimSuffix(s, "}") + + var ( + insideQuotes bool + escaped bool + token strings.Builder + tokens []string + ) + for _, r := range s { + switch r { + case ',': + if !insideQuotes { + tokens = append(tokens, token.String()) + token.Reset() + continue + } + case '"': + if !escaped { + insideQuotes = !insideQuotes + } else { + escaped = false + } + case '\\': + escaped = !escaped + default: + escaped = false + } + token.WriteRune(r) + } + if s := strings.TrimSpace(token.String()); s != "" { + tokens = append(tokens, s) + } + for _, token := range tokens { + m, err := ParseMatcher(token) + if err != nil { + return nil, err + } + matchers = append(matchers, m) + } + + return matchers, nil +} + +// ParseMatcher parses a label matcher from string. +// Copied from https://github.com/prometheus/alertmanager/blob/v0.24.0/pkg/labels/parse.go#L117. +func ParseMatcher(s string) (*labels.Matcher, error) { + ms := re.FindStringSubmatch(s) + if len(ms) == 0 { + return nil, errors.Errorf("bad matcher format: %s", s) + } + + var ( + rawValue = ms[3] + value strings.Builder + escaped bool + expectTrailingQuote bool + ) + + if rawValue[0] == '"' { + rawValue = strings.TrimPrefix(rawValue, "\"") + expectTrailingQuote = true + } + + if !utf8.ValidString(rawValue) { + return nil, errors.Errorf("matcher value not valid UTF-8: %s", ms[3]) + } + + // Unescape the rawValue: + for i, r := range rawValue { + if escaped { + escaped = false + switch r { + case 'n': + value.WriteByte('\n') + case '"', '\\': + value.WriteRune(r) + default: + // This was a spurious escape, so treat the '\' as literal. + value.WriteByte('\\') + value.WriteRune(r) + } + continue + } + switch r { + case '\\': + if i < len(rawValue)-1 { + escaped = true + continue + } + // '\' encountered as last byte. Treat it as literal. + value.WriteByte('\\') + case '"': + if !expectTrailingQuote || i < len(rawValue)-1 { + return nil, errors.Errorf("matcher value contains unescaped double quote: %s", ms[3]) + } + expectTrailingQuote = false + default: + value.WriteRune(r) + } + } + + if expectTrailingQuote { + return nil, errors.Errorf("matcher value contains unescaped double quote: %s", ms[3]) + } + + return labels.NewMatcher(typeMap[ms[2]], ms[1], value.String()) +} diff --git a/pkg/block/metadata/matchers_test.go b/pkg/block/metadata/matchers_test.go new file mode 100644 index 0000000000..cf7b248417 --- /dev/null +++ b/pkg/block/metadata/matchers_test.go @@ -0,0 +1,264 @@ +// Copyright (c) The Thanos Authors. +// Licensed under the Apache License 2.0. + +// Copyright 2018 Prometheus Team +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package metadata + +import ( + "reflect" + "testing" + + "github.com/prometheus/prometheus/model/labels" +) + +func TestMatchers(t *testing.T) { + for _, tc := range []struct { + input string + want []*labels.Matcher + err string + }{ + { + input: `{foo="bar"}`, + want: func() []*labels.Matcher { + ms := []*labels.Matcher{} + m, _ := labels.NewMatcher(labels.MatchEqual, "foo", "bar") + return append(ms, m) + }(), + }, + { + input: `{foo=~"bar.*"}`, + want: func() []*labels.Matcher { + ms := []*labels.Matcher{} + m, _ := labels.NewMatcher(labels.MatchRegexp, "foo", "bar.*") + return append(ms, m) + }(), + }, + { + input: `{foo!="bar"}`, + want: func() []*labels.Matcher { + ms := []*labels.Matcher{} + m, _ := labels.NewMatcher(labels.MatchNotEqual, "foo", "bar") + return append(ms, m) + }(), + }, + { + input: `{foo!~"bar.*"}`, + want: func() []*labels.Matcher { + ms := []*labels.Matcher{} + m, _ := labels.NewMatcher(labels.MatchNotRegexp, "foo", "bar.*") + return append(ms, m) + }(), + }, + { + input: `{foo="bar", baz!="quux"}`, + want: func() []*labels.Matcher { + ms := []*labels.Matcher{} + m, _ := labels.NewMatcher(labels.MatchEqual, "foo", "bar") + m2, _ := labels.NewMatcher(labels.MatchNotEqual, "baz", "quux") + return append(ms, m, m2) + }(), + }, + { + input: `{foo="bar", baz!~"quux.*"}`, + want: func() []*labels.Matcher { + ms := []*labels.Matcher{} + m, _ := labels.NewMatcher(labels.MatchEqual, "foo", "bar") + m2, _ := labels.NewMatcher(labels.MatchNotRegexp, "baz", "quux.*") + return append(ms, m, m2) + }(), + }, + { + input: `{foo="bar",baz!~".*quux", derp="wat"}`, + want: func() []*labels.Matcher { + ms := []*labels.Matcher{} + m, _ := labels.NewMatcher(labels.MatchEqual, "foo", "bar") + m2, _ := labels.NewMatcher(labels.MatchNotRegexp, "baz", ".*quux") + m3, _ := labels.NewMatcher(labels.MatchEqual, "derp", "wat") + return append(ms, m, m2, m3) + }(), + }, + { + input: `{foo="bar", baz!="quux", derp="wat"}`, + want: func() []*labels.Matcher { + ms := []*labels.Matcher{} + m, _ := labels.NewMatcher(labels.MatchEqual, "foo", "bar") + m2, _ := labels.NewMatcher(labels.MatchNotEqual, "baz", "quux") + m3, _ := labels.NewMatcher(labels.MatchEqual, "derp", "wat") + return append(ms, m, m2, m3) + }(), + }, + { + input: `{foo="bar", baz!~".*quux.*", derp="wat"}`, + want: func() []*labels.Matcher { + ms := []*labels.Matcher{} + m, _ := labels.NewMatcher(labels.MatchEqual, "foo", "bar") + m2, _ := labels.NewMatcher(labels.MatchNotRegexp, "baz", ".*quux.*") + m3, _ := labels.NewMatcher(labels.MatchEqual, "derp", "wat") + return append(ms, m, m2, m3) + }(), + }, + { + input: `{foo="bar", instance=~"some-api.*"}`, + want: func() []*labels.Matcher { + ms := []*labels.Matcher{} + m, _ := labels.NewMatcher(labels.MatchEqual, "foo", "bar") + m2, _ := labels.NewMatcher(labels.MatchRegexp, "instance", "some-api.*") + return append(ms, m, m2) + }(), + }, + { + input: `{foo=""}`, + want: func() []*labels.Matcher { + ms := []*labels.Matcher{} + m, _ := labels.NewMatcher(labels.MatchEqual, "foo", "") + return append(ms, m) + }(), + }, + { + input: `{foo="bar,quux", job="job1"}`, + want: func() []*labels.Matcher { + ms := []*labels.Matcher{} + m, _ := labels.NewMatcher(labels.MatchEqual, "foo", "bar,quux") + m2, _ := labels.NewMatcher(labels.MatchEqual, "job", "job1") + return append(ms, m, m2) + }(), + }, + { + input: `{foo = "bar", dings != "bums", }`, + want: func() []*labels.Matcher { + ms := []*labels.Matcher{} + m, _ := labels.NewMatcher(labels.MatchEqual, "foo", "bar") + m2, _ := labels.NewMatcher(labels.MatchNotEqual, "dings", "bums") + return append(ms, m, m2) + }(), + }, + { + input: `foo=bar,dings!=bums`, + want: func() []*labels.Matcher { + ms := []*labels.Matcher{} + m, _ := labels.NewMatcher(labels.MatchEqual, "foo", "bar") + m2, _ := labels.NewMatcher(labels.MatchNotEqual, "dings", "bums") + return append(ms, m, m2) + }(), + }, + { + input: `{quote="She said: \"Hi, ladies! That's gender-neutral…\""}`, + want: func() []*labels.Matcher { + ms := []*labels.Matcher{} + m, _ := labels.NewMatcher(labels.MatchEqual, "quote", `She said: "Hi, ladies! That's gender-neutral…"`) + return append(ms, m) + }(), + }, + { + input: `statuscode=~"5.."`, + want: func() []*labels.Matcher { + ms := []*labels.Matcher{} + m, _ := labels.NewMatcher(labels.MatchRegexp, "statuscode", "5..") + return append(ms, m) + }(), + }, + { + input: `tricky=~~~`, + want: func() []*labels.Matcher { + ms := []*labels.Matcher{} + m, _ := labels.NewMatcher(labels.MatchRegexp, "tricky", "~~") + return append(ms, m) + }(), + }, + { + input: `trickier==\\=\=\"`, + want: func() []*labels.Matcher { + ms := []*labels.Matcher{} + m, _ := labels.NewMatcher(labels.MatchEqual, "trickier", `=\=\="`) + return append(ms, m) + }(), + }, + { + input: `contains_quote != "\"" , contains_comma !~ "foo,bar" , `, + want: func() []*labels.Matcher { + ms := []*labels.Matcher{} + m, _ := labels.NewMatcher(labels.MatchNotEqual, "contains_quote", `"`) + m2, _ := labels.NewMatcher(labels.MatchNotRegexp, "contains_comma", "foo,bar") + return append(ms, m, m2) + }(), + }, + { + input: `job="value`, + err: `matcher value contains unescaped double quote: "value`, + }, + { + input: `job=value"`, + err: `matcher value contains unescaped double quote: value"`, + }, + { + input: `trickier==\\=\=\""`, + err: `matcher value contains unescaped double quote: =\\=\=\""`, + }, + { + input: `contains_unescaped_quote = foo"bar`, + err: `matcher value contains unescaped double quote: foo"bar`, + }, + { + input: `{invalid-name = "valid label"}`, + err: `bad matcher format: invalid-name = "valid label"`, + }, + { + input: `{foo=~"invalid[regexp"}`, + err: "error parsing regexp: missing closing ]: `[regexp)$`", + }, + // Double escaped strings. + { + input: `"{foo=\"bar"}`, + err: `bad matcher format: "{foo=\"bar"`, + }, + { + input: `"foo=\"bar"`, + err: `bad matcher format: "foo=\"bar"`, + }, + { + input: `"foo=\"bar\""`, + err: `bad matcher format: "foo=\"bar\""`, + }, + { + input: `"foo=\"bar\"`, + err: `bad matcher format: "foo=\"bar\"`, + }, + { + input: `"{foo=\"bar\"}"`, + err: `bad matcher format: "{foo=\"bar\"}"`, + }, + { + input: `"foo="bar""`, + err: `bad matcher format: "foo="bar""`, + }, + } { + t.Run(tc.input, func(t *testing.T) { + got, err := ParseMatchers(tc.input) + if err != nil && tc.err == "" { + t.Fatalf("got error where none expected: %v", err) + } + if err == nil && tc.err != "" { + t.Fatalf("expected error but got none: %v", tc.err) + } + if err != nil && err.Error() != tc.err { + t.Fatalf("error not equal:\ngot %v\nwant %v", err, tc.err) + } + if !reflect.DeepEqual(got, tc.want) { + t.Fatalf("labels not equal:\ngot %v\nwant %v", got, tc.want) + } + }) + } + +} diff --git a/pkg/block/metadata/meta.go b/pkg/block/metadata/meta.go index 787a03c241..9e0082a5c2 100644 --- a/pkg/block/metadata/meta.go +++ b/pkg/block/metadata/meta.go @@ -20,11 +20,9 @@ import ( "github.com/pkg/errors" "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/model/relabel" - "github.com/prometheus/prometheus/promql/parser" "github.com/prometheus/prometheus/tsdb" "github.com/prometheus/prometheus/tsdb/fileutil" "github.com/prometheus/prometheus/tsdb/tombstones" - "gopkg.in/yaml.v3" "github.com/thanos-io/thanos/pkg/runutil" ) @@ -101,16 +99,6 @@ type Rewrite struct { RelabelsApplied []*relabel.Config `json:"relabels_applied,omitempty"` } -type Matchers []*labels.Matcher - -func (m *Matchers) UnmarshalYAML(value *yaml.Node) (err error) { - *m, err = parser.ParseMetricSelector(value.Value) - if err != nil { - return errors.Wrapf(err, "parse metric selector %v", value.Value) - } - return nil -} - type DeletionRequest struct { Matchers Matchers `json:"matchers" yaml:"matchers"` Intervals tombstones.Intervals `json:"intervals,omitempty" yaml:"intervals,omitempty"` diff --git a/pkg/store/bucket.go b/pkg/store/bucket.go index 2cd2fc60fd..0008f36508 100644 --- a/pkg/store/bucket.go +++ b/pkg/store/bucket.go @@ -836,7 +836,7 @@ PostingsLoop: } var tombstoneIntervals promtombstones.Intervals for _, ts := range tombstones { - for _, matcher := range ts.Matchers { + for _, matcher := range *ts.Matchers { if val := lset.Get(matcher.Name); val != "" { if !matcher.Matches(val) { continue @@ -844,7 +844,7 @@ PostingsLoop: if skipChunks { continue PostingsLoop } - tombstoneIntervals.Add(promtombstones.Interval{Mint: ts.MinTime, Maxt: ts.MaxTime}) + tombstoneIntervals = tombstoneIntervals.Add(promtombstones.Interval{Mint: ts.MinTime, Maxt: ts.MaxTime}) } } } @@ -873,6 +873,11 @@ PostingsLoop: s.refs = append(s.refs, meta.Ref) } + // If all chunks are matched by the tombstone then check next posting. + if len(s.chks) == 0 { + continue PostingsLoop + } + // Ensure sample limit through chunksLimiter if we return chunks. if err := chunksLimiter.Reserve(uint64(len(s.chks))); err != nil { return nil, nil, errors.Wrap(err, "exceeded chunks limit") diff --git a/pkg/tombstone/tombstone.go b/pkg/tombstone/tombstone.go index 8006ce3587..e9921b596a 100644 --- a/pkg/tombstone/tombstone.go +++ b/pkg/tombstone/tombstone.go @@ -4,11 +4,12 @@ package tombstone import ( + "bytes" "context" "encoding/json" "fmt" + "github.com/go-kit/log/level" "io/ioutil" - "os" "path" "time" @@ -26,18 +27,18 @@ const ( // Tombstone represents a tombstone. type Tombstone struct { - Matchers metadata.Matchers `json:"matchers"` - MinTime int64 `json:"minTime"` - MaxTime int64 `json:"maxTime"` - CreationTime int64 `json:"creationTime"` - Author string `json:"author"` - Reason string `json:"reason"` + Matchers *metadata.Matchers `json:"matchers"` + MinTime int64 `json:"minTime"` + MaxTime int64 `json:"maxTime"` + CreationTime int64 `json:"creationTime"` + Author string `json:"author"` + Reason string `json:"reason"` } // NewTombstone returns a new instance of Tombstone. func NewTombstone(matchers metadata.Matchers, minTime, maxTime, creationTime int64, author, reason string) *Tombstone { return &Tombstone{ - Matchers: matchers, + Matchers: &matchers, MinTime: minTime, MaxTime: maxTime, CreationTime: creationTime, @@ -52,21 +53,14 @@ func GenName() string { } // UploadTombstone uploads the given tombstone to object storage. -func UploadTombstone(ctx context.Context, tombstone *Tombstone, bkt objstore.Bucket, logger log.Logger) error { +func UploadTombstone(ctx context.Context, tombstone *Tombstone, bkt objstore.Bucket) error { b, err := json.Marshal(tombstone) if err != nil { return err } - tmpDir := os.TempDir() - - tsPath := tmpDir + "/tombstone.json" - if err := ioutil.WriteFile(tsPath, b, 0644); err != nil { - return err - } - - return objstore.UploadFile(ctx, logger, bkt, tsPath, path.Join(TombstoneDir, GenName())) - + tsPath := path.Join(TombstoneDir, GenName()) + return bkt.Upload(ctx, tsPath, bytes.NewBuffer(b)) } // ReadTombstones returns all the tombstones present in the object storage. @@ -81,15 +75,16 @@ func ReadTombstones(ctx context.Context, bkt objstore.InstrumentedBucketReader, } defer runutil.CloseWithLogOnErr(logger, tombstoneFile, "close bkt tombstone reader") - var t *Tombstone + var t Tombstone tombstone, err := ioutil.ReadAll(tombstoneFile) if err != nil { return nil } - if err := json.Unmarshal(tombstone, t); err != nil { + if err := json.Unmarshal(tombstone, &t); err != nil { + level.Error(logger).Log("msg", "failed to unmarshal tombstone", "file", tombstoneFilename, "err", err) return nil } - ts = append(ts, t) + ts = append(ts, &t) return nil }); err != nil { return nil, err From 808ba3344024bb77126268746419e046d99ea251 Mon Sep 17 00:00:00 2001 From: Ben Ye Date: Sat, 18 Jun 2022 02:12:39 -0700 Subject: [PATCH 07/14] add tombstone fetchers implmentation Signed-off-by: Ben Ye --- cmd/thanos/store.go | 7 + cmd/thanos/tools_bucket.go | 14 +- pkg/block/fetcher.go | 2 +- pkg/store/bucket.go | 38 +++- pkg/store/prometheus.go | 4 + pkg/tombstone/fetcher.go | 445 +++++++++++++++++++++++++++++++++++++ pkg/tombstone/tombstone.go | 47 ++-- 7 files changed, 516 insertions(+), 41 deletions(-) create mode 100644 pkg/tombstone/fetcher.go diff --git a/cmd/thanos/store.go b/cmd/thanos/store.go index f44909892d..8fa79f8b5f 100644 --- a/cmd/thanos/store.go +++ b/cmd/thanos/store.go @@ -6,6 +6,7 @@ package main import ( "context" "fmt" + "github.com/thanos-io/thanos/pkg/tombstone" "time" "github.com/alecthomas/units" @@ -314,6 +315,11 @@ func runStore( return errors.Wrap(err, "meta fetcher") } + tombstoneFetcher, err := tombstone.NewTombstoneFetcher(logger, conf.blockMetaFetchConcurrency, bkt, conf.dataDir, extprom.WrapRegistererWithPrefix("thanos_", reg), []tombstone.TombstoneFilter{}) + if err != nil { + return errors.Wrap(err, "tombstone fetcher") + } + // Limit the concurrency on queries against the Thanos store. if conf.maxConcurrency < 0 { return errors.Errorf("max concurrency value cannot be lower than 0 (got %v)", conf.maxConcurrency) @@ -342,6 +348,7 @@ func runStore( bs, err := store.NewBucketStore( bkt, metaFetcher, + tombstoneFetcher, conf.dataDir, store.NewChunksLimiterFactory(conf.maxSampleCount/store.MaxSamplesPerChunk), // The samples limit is an approximation based on the max number of samples per chunk. store.NewSeriesLimiterFactory(conf.maxTouchedSeriesCount), diff --git a/cmd/thanos/tools_bucket.go b/cmd/thanos/tools_bucket.go index 6ab7dc5a71..92ef75fb32 100644 --- a/cmd/thanos/tools_bucket.go +++ b/cmd/thanos/tools_bucket.go @@ -33,7 +33,6 @@ import ( "github.com/prometheus/common/route" "github.com/prometheus/prometheus/model/labels" "github.com/prometheus/prometheus/model/relabel" - "github.com/prometheus/prometheus/model/timestamp" "github.com/prometheus/prometheus/promql/parser" "github.com/prometheus/prometheus/tsdb" "github.com/prometheus/prometheus/tsdb/chunkenc" @@ -161,6 +160,7 @@ type bucketDeleteConfig struct { matchers string author string reason string + labels []string } func (tbc *bucketVerifyConfig) registerBucketVerifyFlag(cmd extkingpin.FlagClause) *bucketVerifyConfig { @@ -279,6 +279,8 @@ func (tbc *bucketDeleteConfig) registerBucketDeleteFlag(cmd extkingpin.FlagClaus cmd.Flag("matchers", "The string representing label matchers").Default("").StringVar(&tbc.matchers) cmd.Flag("author", "Author of the deletion request").Default("not specified").StringVar(&tbc.author) cmd.Flag("reason", "Reason to perform the deletion request").Default("not specified").StringVar(&tbc.reason) + cmd.Flag("label", "Tombstone external labels. Only blocks that contain the matching external labels will be considered."). + PlaceHolder("=\"\"").StringsVar(&tbc.labels) return tbc } @@ -1415,13 +1417,19 @@ func registerBucketDelete(app extkingpin.AppClause, objStoreConfig *extflag.Path // Dummy actor to immediately kill the group after the run function returns. g.Add(func() error { return nil }, func(error) {}) + labels, err := parseFlagLabels(tbc.labels) + if err != nil { + return err + } + m, err := parser.ParseMetricSelector(tbc.matchers) if err != nil { return err } - ts := tombstone.NewTombstone(m, minTime.PrometheusTimestamp(), maxTime.PrometheusTimestamp(), - timestamp.FromTime(time.Now()), tbc.author, tbc.reason) + newID := ulid.MustNew(ulid.Now(), rand.Reader) + ts := tombstone.NewTombstone(newID, m, minTime.PrometheusTimestamp(), maxTime.PrometheusTimestamp(), + tbc.author, tbc.reason, labels) ctx, cancel := context.WithTimeout(context.Background(), tbc.timeout) defer cancel() diff --git a/pkg/block/fetcher.go b/pkg/block/fetcher.go index 24967c05c4..191298be1a 100644 --- a/pkg/block/fetcher.go +++ b/pkg/block/fetcher.go @@ -144,7 +144,7 @@ type MetadataFetcher interface { UpdateOnChange(func([]metadata.Meta, error)) } -// Filter allows filtering or modifying metas from the provided map or returns error. +// MetadataFilter allows filtering or modifying metas from the provided map or returns error. type MetadataFilter interface { Filter(ctx context.Context, metas map[ulid.ULID]*metadata.Meta, synced *extprom.TxGaugeVec, modified *extprom.TxGaugeVec) error } diff --git a/pkg/store/bucket.go b/pkg/store/bucket.go index 0008f36508..d34d28e2dd 100644 --- a/pkg/store/bucket.go +++ b/pkg/store/bucket.go @@ -272,15 +272,16 @@ type FilterConfig struct { // This makes them smaller, but takes extra CPU and memory. // When used with in-memory cache, memory usage should decrease overall, thanks to postings being smaller. type BucketStore struct { - logger log.Logger - reg prometheus.Registerer // TODO(metalmatze) remove and add via BucketStoreOption - metrics *bucketStoreMetrics - bkt objstore.InstrumentedBucketReader - fetcher block.MetadataFetcher - dir string - indexCache storecache.IndexCache - indexReaderPool *indexheader.ReaderPool - chunkPool pool.Bytes + logger log.Logger + reg prometheus.Registerer // TODO(metalmatze) remove and add via BucketStoreOption + metrics *bucketStoreMetrics + bkt objstore.InstrumentedBucketReader + fetcher block.MetadataFetcher + tombstoneFetcher tombstone.TombstoneFetcher + dir string + indexCache storecache.IndexCache + indexReaderPool *indexheader.ReaderPool + chunkPool pool.Bytes // Sets of blocks that have the same labels. They are indexed by a hash over their label set. mtx sync.RWMutex @@ -307,7 +308,7 @@ type BucketStore struct { enableCompatibilityLabel bool tombstonesMtx sync.RWMutex - tombstones []*tombstone.Tombstone + tombstones map[ulid.ULID]*tombstone.Tombstone // Every how many posting offset entry we pool in heap memory. Default in Prometheus is 32. postingOffsetsInMemSampling int @@ -392,6 +393,7 @@ func WithDebugLogging() BucketStoreOption { func NewBucketStore( bkt objstore.InstrumentedBucketReader, fetcher block.MetadataFetcher, + tombstoneFetcher tombstone.TombstoneFetcher, dir string, chunksLimiterFactory ChunksLimiterFactory, seriesLimiterFactory SeriesLimiterFactory, @@ -408,6 +410,7 @@ func NewBucketStore( logger: log.NewNopLogger(), bkt: bkt, fetcher: fetcher, + tombstoneFetcher: tombstoneFetcher, dir: dir, indexCache: noopCache{}, chunkPool: pool.NoopBytes{}, @@ -1059,6 +1062,17 @@ func (s *BucketStore) Series(req *storepb.SeriesRequest, srv storepb.Store_Serie if !ok { continue } + matchedTombstones := make([]*tombstone.Tombstone, 0, len(tombstones)) + for _, ts := range tombstones { + ok, _, err := promMatchesExternalLabels(blockMatchers, ts.Labels) + if err != nil { + return err + } + if !ok { + continue + } + matchedTombstones = append(matchedTombstones, ts) + } blocks := bs.getFor(req.MinTime, req.MaxTime, req.MaxResolutionWindow, reqBlockMatchers) @@ -1106,7 +1120,7 @@ func (s *BucketStore) Series(req *storepb.SeriesRequest, srv storepb.Store_Serie req.SkipChunks, req.MinTime, req.MaxTime, req.Aggregates, - tombstones, + matchedTombstones, ) if err != nil { return errors.Wrapf(err, "fetch series for block %s", b.meta.ULID) @@ -1494,7 +1508,7 @@ func (s *BucketStore) LabelValues(ctx context.Context, req *storepb.LabelValuesR } func (s *BucketStore) SyncTombstones(ctx context.Context) error { - tombstones, err := tombstone.ReadTombstones(ctx, s.bkt, s.logger) + tombstones, _, err := s.tombstoneFetcher.Fetch(ctx) if err != nil { return err } diff --git a/pkg/store/prometheus.go b/pkg/store/prometheus.go index 7a8a46c85e..f248cef746 100644 --- a/pkg/store/prometheus.go +++ b/pkg/store/prometheus.go @@ -552,6 +552,10 @@ func matchesExternalLabels(ms []storepb.LabelMatcher, externalLabels labels.Labe return false, nil, err } + return promMatchesExternalLabels(tms, externalLabels) +} + +func promMatchesExternalLabels(tms []*labels.Matcher, externalLabels labels.Labels) (bool, []*labels.Matcher, error) { if len(externalLabels) == 0 { return true, tms, nil } diff --git a/pkg/tombstone/fetcher.go b/pkg/tombstone/fetcher.go new file mode 100644 index 0000000000..9e32d2fa08 --- /dev/null +++ b/pkg/tombstone/fetcher.go @@ -0,0 +1,445 @@ +// Copyright (c) The Thanos Authors. +// Licensed under the Apache License 2.0. + +package tombstone + +import ( + "context" + "encoding/json" + "github.com/golang/groupcache/singleflight" + "io/ioutil" + "os" + "path" + "path/filepath" + "strings" + "sync" + "time" + + "github.com/go-kit/log" + "github.com/go-kit/log/level" + "github.com/oklog/ulid" + "github.com/pkg/errors" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" + "golang.org/x/sync/errgroup" + + "github.com/thanos-io/thanos/pkg/block" + "github.com/thanos-io/thanos/pkg/errutil" + "github.com/thanos-io/thanos/pkg/extprom" + "github.com/thanos-io/thanos/pkg/objstore" + "github.com/thanos-io/thanos/pkg/runutil" +) + +const ( + tombstoneSubSys = "tombstone" +) + +var ( + ErrorSyncTombstoneNotFound = errors.New("tombstone not found") + ErrorSyncTombstoneCorrupted = errors.New("tombstone corrupted") +) + +type TombstoneFetcher interface { + Fetch(ctx context.Context) (tombstones map[ulid.ULID]*Tombstone, partial map[ulid.ULID]error, err error) +} + +// TombstoneFilter allows filtering or modifying metas from the provided map or returns error. +type TombstoneFilter interface { + Filter(ctx context.Context, tombstones map[ulid.ULID]*Tombstone, synced *extprom.TxGaugeVec) error +} + +func newFetcherMetrics(reg prometheus.Registerer, syncedExtraLabels, modifiedExtraLabels [][]string) *block.FetcherMetrics { + var m block.FetcherMetrics + + m.Syncs = promauto.With(reg).NewCounter(prometheus.CounterOpts{ + Subsystem: tombstoneSubSys, + Name: "syncs_total", + Help: "Total tombstone synchronization attempts", + }) + m.SyncFailures = promauto.With(reg).NewCounter(prometheus.CounterOpts{ + Subsystem: tombstoneSubSys, + Name: "sync_failures_total", + Help: "Total tombstone synchronization failures", + }) + m.SyncDuration = promauto.With(reg).NewHistogram(prometheus.HistogramOpts{ + Subsystem: tombstoneSubSys, + Name: "sync_duration_seconds", + Help: "Duration of the tombstone synchronization in seconds", + Buckets: []float64{0.01, 1, 10, 100, 300, 600, 1000}, + }) + //m.Synced = extprom.NewTxGaugeVec( + // reg, + // prometheus.GaugeOpts{ + // Subsystem: tombstoneSubSys, + // Name: "synced", + // Help: "Number of tombstone synced", + // }, + // []string{"state"}, + // append([][]string{ + // {CorruptedMeta}, + // {NoMeta}, + // {LoadedMeta}, + // {tooFreshMeta}, + // {FailedMeta}, + // {labelExcludedMeta}, + // {timeExcludedMeta}, + // {duplicateMeta}, + // {MarkedForDeletionMeta}, + // {MarkedForNoCompactionMeta}, + // }, syncedExtraLabels...)..., + //) + return &m +} + +type Fetcher struct { + bkt objstore.InstrumentedBucketReader + logger log.Logger + + metrics *block.FetcherMetrics + wrapped *BaseFetcher + filters []TombstoneFilter +} + +// NewTombstoneFetcher returns Fetcher. +func NewTombstoneFetcher(logger log.Logger, concurrency int, bkt objstore.InstrumentedBucketReader, dir string, reg prometheus.Registerer, filters []TombstoneFilter) (*Fetcher, error) { + b, err := NewBaseFetcher(logger, concurrency, bkt, dir, reg) + if err != nil { + return nil, err + } + return &Fetcher{metrics: newFetcherMetrics(reg, nil, nil), wrapped: b, filters: filters, logger: b.logger}, nil +} + +//func (f *Fetcher) Fetch(ctx context.Context) (tombstones map[ulid.ULID]*Tombstone, partial map[ulid.ULID]error, err error) { +// var ( +// ts map[ulid.ULID]*Tombstone +// ) +// +// if err := f.bkt.Iter(ctx, TombstoneDir, func(name string) error { +// tombstoneFilename := path.Join("", name) +// tombstoneFile, err := f.bkt.Get(ctx, tombstoneFilename) +// if err != nil { +// return nil +// } +// defer runutil.CloseWithLogOnErr(f.logger, tombstoneFile, "close bkt tombstone reader") +// +// var t Tombstone +// tombstone, err := ioutil.ReadAll(tombstoneFile) +// if err != nil { +// return nil +// } +// if err := json.Unmarshal(tombstone, &t); err != nil { +// level.Error(f.logger).Log("msg", "failed to unmarshal tombstone", "file", tombstoneFilename, "err", err) +// return nil +// } +// ts = append(ts, &t) +// return nil +// }); err != nil { +// return nil, err +// } +// return ts, nil +//} + +// Fetch returns all block metas as well as partial blocks (blocks without or with corrupted meta file) from the bucket. +// It's caller responsibility to not change the returned metadata files. Maps can be modified. +// +// Returned error indicates a failure in fetching metadata. Returned meta can be assumed as correct, with some blocks missing. +func (f *Fetcher) Fetch(ctx context.Context) (metas map[ulid.ULID]*Tombstone, partial map[ulid.ULID]error, err error) { + return f.wrapped.fetch(ctx, f.metrics, f.filters) +} + +// BaseFetcher is a struct that synchronizes filtered metadata of all block in the object storage with the local state. +// Go-routine safe. +type BaseFetcher struct { + logger log.Logger + concurrency int + bkt objstore.InstrumentedBucketReader + + // Optional local directory to cache meta.json files. + cacheDir string + syncs prometheus.Counter + g singleflight.Group + + mtx sync.Mutex + cached map[ulid.ULID]*Tombstone +} + +// NewBaseFetcher constructs BaseFetcher. +func NewBaseFetcher(logger log.Logger, concurrency int, bkt objstore.InstrumentedBucketReader, dir string, reg prometheus.Registerer) (*BaseFetcher, error) { + if logger == nil { + logger = log.NewNopLogger() + } + + cacheDir := "" + if dir != "" { + cacheDir = filepath.Join(dir, "meta-syncer") + if err := os.MkdirAll(cacheDir, os.ModePerm); err != nil { + return nil, err + } + } + + return &BaseFetcher{ + logger: log.With(logger, "component", "tombstone.BaseFetcher"), + concurrency: concurrency, + bkt: bkt, + cacheDir: cacheDir, + cached: map[ulid.ULID]*Tombstone{}, + //syncs: promauto.With(reg).NewCounter(prometheus.CounterOpts{ + // Subsystem: fetcherSubSys, + // Name: "base_syncs_total", + // Help: "Total tombstone synchronization attempts by base Fetcher", + //}), + }, nil +} + +type response struct { + tombstones map[ulid.ULID]*Tombstone + partial map[ulid.ULID]error + // If tombstoneErrs > 0 it means incomplete view, so some metas, failed to be loaded. + tombstoneErrs errutil.MultiError + + corruptedTombstones float64 +} + +// loadMeta returns metadata from object storage or error. +// It returns `ErrorSyncMetaNotFound` and `ErrorSyncMetaCorrupted` sentinel errors in those cases. +func (f *BaseFetcher) loadTombstone(ctx context.Context, id ulid.ULID) (*Tombstone, error) { + tombstoneFile := path.Join(TombstoneDir, id.String()+".json") + ok, err := f.bkt.Exists(ctx, tombstoneFile) + if err != nil { + return nil, errors.Wrapf(err, "%s file doesn't exist", tombstoneFile) + } + if !ok { + return nil, ErrorSyncTombstoneNotFound + } + + if m, seen := f.cached[id]; seen { + return m, nil + } + + //// Best effort load from local dir. + //if f.cacheDir != "" { + // m, err := metadata.ReadFromDir(cachedBlockDir) + // if err == nil { + // return m, nil + // } + // + // if !errors.Is(err, os.ErrNotExist) { + // level.Warn(f.logger).Log("msg", "best effort read of the local meta.json failed; removing cached block dir", "dir", cachedBlockDir, "err", err) + // if err := os.RemoveAll(cachedBlockDir); err != nil { + // level.Warn(f.logger).Log("msg", "best effort remove of cached dir failed; ignoring", "dir", cachedBlockDir, "err", err) + // } + // } + //} + + r, err := f.bkt.ReaderWithExpectedErrs(f.bkt.IsObjNotFoundErr).Get(ctx, tombstoneFile) + if f.bkt.IsObjNotFoundErr(err) { + // Meta.json was deleted between bkt.Exists and here. + return nil, errors.Wrapf(ErrorSyncTombstoneNotFound, "%v", err) + } + if err != nil { + return nil, errors.Wrapf(err, "get tombstone file: %v", tombstoneFile) + } + + defer runutil.CloseWithLogOnErr(f.logger, r, "close bkt meta get") + + content, err := ioutil.ReadAll(r) + if err != nil { + return nil, errors.Wrapf(err, "read meta file: %v", tombstoneFile) + } + + var t Tombstone + if err := json.Unmarshal(content, &t); err != nil { + return nil, errors.Wrapf(ErrorSyncTombstoneCorrupted, "tombstone %v unmarshal: %v", tombstoneFile, err) + } + + //// Best effort cache in local dir. + //if f.cacheDir != "" { + // if err := os.MkdirAll(cachedBlockDir, os.ModePerm); err != nil { + // level.Warn(f.logger).Log("msg", "best effort mkdir of the meta.json block dir failed; ignoring", "dir", cachedBlockDir, "err", err) + // } + // + // if err := m.WriteToDir(f.logger, cachedBlockDir); err != nil { + // level.Warn(f.logger).Log("msg", "best effort save of the meta.json to local dir failed; ignoring", "dir", cachedBlockDir, "err", err) + // } + //} + return &t, nil +} + +func (f *BaseFetcher) fetchTombstone(ctx context.Context) (interface{}, error) { + f.syncs.Inc() + + var ( + resp = response{ + tombstones: make(map[ulid.ULID]*Tombstone), + partial: make(map[ulid.ULID]error), + } + eg errgroup.Group + ch = make(chan ulid.ULID, f.concurrency) + mtx sync.Mutex + ) + level.Debug(f.logger).Log("msg", "fetching tombstone", "concurrency", f.concurrency) + for i := 0; i < f.concurrency; i++ { + eg.Go(func() error { + for id := range ch { + t, err := f.loadTombstone(ctx, id) + if err == nil { + mtx.Lock() + resp.tombstones[id] = t + mtx.Unlock() + continue + } + + switch errors.Cause(err) { + default: + mtx.Lock() + resp.tombstoneErrs.Add(err) + mtx.Unlock() + continue + case ErrorSyncTombstoneCorrupted: + mtx.Lock() + resp.corruptedTombstones++ + mtx.Unlock() + } + + mtx.Lock() + resp.partial[id] = err + mtx.Unlock() + } + return nil + }) + } + + // Workers scheduled, distribute blocks. + eg.Go(func() error { + defer close(ch) + return f.bkt.Iter(ctx, TombstoneDir, func(name string) error { + id, err := ulid.Parse(strings.TrimSuffix(filepath.Base(name), ".json")) + if err != nil { + return nil + } + + select { + case <-ctx.Done(): + return ctx.Err() + case ch <- id: + } + + return nil + }) + }) + + if err := eg.Wait(); err != nil { + return nil, errors.Wrap(err, "BaseFetcher: iter bucket") + } + + if len(resp.tombstoneErrs) > 0 { + return resp, nil + } + + //// Only for complete view of blocks update the cache. + //cached := make(map[ulid.ULID]*metadata.Meta, len(resp.metas)) + //for id, m := range resp.metas { + // cached[id] = m + //} + + //f.mtx.Lock() + //f.cached = cached + //f.mtx.Unlock() + + //// Best effort cleanup of disk-cached metas. + //if f.cacheDir != "" { + // fis, err := ioutil.ReadDir(f.cacheDir) + // names := make([]string, 0, len(fis)) + // for _, fi := range fis { + // names = append(names, fi.Name()) + // } + // if err != nil { + // level.Warn(f.logger).Log("msg", "best effort remove of not needed cached dirs failed; ignoring", "err", err) + // } else { + // for _, n := range names { + // id, ok := IsBlockDir(n) + // if !ok { + // continue + // } + // + // if _, ok := resp.metas[id]; ok { + // continue + // } + // + // cachedBlockDir := filepath.Join(f.cacheDir, id.String()) + // + // // No such block loaded, remove the local dir. + // if err := os.RemoveAll(cachedBlockDir); err != nil { + // level.Warn(f.logger).Log("msg", "best effort remove of not needed cached dir failed; ignoring", "dir", cachedBlockDir, "err", err) + // } + // } + // } + //} + return resp, nil +} + +func (f *BaseFetcher) fetch(ctx context.Context, metrics *block.FetcherMetrics, filters []TombstoneFilter) (_ map[ulid.ULID]*Tombstone, _ map[ulid.ULID]error, err error) { + start := time.Now() + defer func() { + metrics.SyncDuration.Observe(time.Since(start).Seconds()) + if err != nil { + metrics.SyncFailures.Inc() + } + }() + metrics.Syncs.Inc() + metrics.ResetTx() + + // Run this in thread safe run group. + v, err := f.g.Do("", func() (i interface{}, err error) { + // NOTE: First go routine context will go through. + return f.fetchTombstone(ctx) + }) + if err != nil { + return nil, nil, err + } + resp := v.(response) + + // Copy as same response might be reused by different goroutines. + tombstones := make(map[ulid.ULID]*Tombstone, len(resp.tombstoneErrs)) + for id, m := range resp.tombstones { + tombstones[id] = m + } + + //metrics.Synced.WithLabelValues(FailedMeta).Set(float64(len(resp.metaErrs))) + //metrics.Synced.WithLabelValues(NoMeta).Set(resp.noMetas) + //metrics.Synced.WithLabelValues(CorruptedMeta).Set(resp.corruptedMetas) + + for _, filter := range filters { + // NOTE: filter can update synced metric accordingly to the reason of the exclude. + if err := filter.Filter(ctx, tombstones, metrics.Synced); err != nil { + return nil, nil, errors.Wrap(err, "filter tombstones") + } + } + + //metrics.Synced.WithLabelValues(LoadedMeta).Set(float64(len(metas))) + metrics.Submit() + + if len(resp.tombstoneErrs) > 0 { + return tombstones, resp.partial, errors.Wrap(resp.tombstoneErrs.Err(), "incomplete view") + } + + level.Info(f.logger).Log("msg", "successfully synchronized tombstones", "duration", time.Since(start).String(), "duration_ms", time.Since(start).Milliseconds(), "returned", len(tombstones), "partial", len(resp.partial)) + return tombstones, resp.partial, nil +} + +// TombstoneDelayFilter considers the tombstone file only after it exists longer than the delay period. +type TombstoneDelayFilter struct { + delay time.Duration + logger log.Logger +} + +func (f *TombstoneDelayFilter) Filter(_ context.Context, tombstones map[ulid.ULID]*Tombstone, synced *extprom.TxGaugeVec) error { + for id := range tombstones { + if ulid.Now()-id.Time() < uint64(f.delay/time.Millisecond) { + level.Debug(f.logger).Log("msg", "tombstone is too fresh for now", "tombstone", id) + delete(tombstones, id) + } + } + + return nil +} diff --git a/pkg/tombstone/tombstone.go b/pkg/tombstone/tombstone.go index e9921b596a..5edacc6200 100644 --- a/pkg/tombstone/tombstone.go +++ b/pkg/tombstone/tombstone.go @@ -7,17 +7,15 @@ import ( "bytes" "context" "encoding/json" - "fmt" - "github.com/go-kit/log/level" - "io/ioutil" - "path" - "time" - "github.com/go-kit/log" + "github.com/go-kit/log/level" "github.com/oklog/ulid" + "github.com/prometheus/prometheus/model/labels" "github.com/thanos-io/thanos/pkg/block/metadata" "github.com/thanos-io/thanos/pkg/objstore" "github.com/thanos-io/thanos/pkg/runutil" + "io/ioutil" + "path" ) const ( @@ -27,31 +25,30 @@ const ( // Tombstone represents a tombstone. type Tombstone struct { - Matchers *metadata.Matchers `json:"matchers"` - MinTime int64 `json:"minTime"` - MaxTime int64 `json:"maxTime"` - CreationTime int64 `json:"creationTime"` - Author string `json:"author"` - Reason string `json:"reason"` + ULID ulid.ULID `json:"ulid"` + Matchers *metadata.Matchers `json:"matchers"` + MinTime int64 `json:"minTime"` + MaxTime int64 `json:"maxTime"` + Author string `json:"author"` + Reason string `json:"reason"` + // Labels are the external labels identifying the producer as well as tenant. + // See https://thanos.io/tip/thanos/storage.md#external-labels for details. + Labels labels.Labels `json:"labels"` } // NewTombstone returns a new instance of Tombstone. -func NewTombstone(matchers metadata.Matchers, minTime, maxTime, creationTime int64, author, reason string) *Tombstone { +func NewTombstone(ulid ulid.ULID, matchers metadata.Matchers, minTime, maxTime int64, author, reason string, labels labels.Labels) *Tombstone { return &Tombstone{ - Matchers: &matchers, - MinTime: minTime, - MaxTime: maxTime, - CreationTime: creationTime, - Author: author, - Reason: reason, + ULID: ulid, + Matchers: &matchers, + MinTime: minTime, + MaxTime: maxTime, + Author: author, + Reason: reason, + Labels: labels, } } -// GenName generates file name based on Matchers, MinTime and MaxTime of a tombstone. -func GenName() string { - return fmt.Sprintf("tombstones-%s.json", ulid.MustNew(uint64(time.Now().Unix()), nil)) -} - // UploadTombstone uploads the given tombstone to object storage. func UploadTombstone(ctx context.Context, tombstone *Tombstone, bkt objstore.Bucket) error { b, err := json.Marshal(tombstone) @@ -59,7 +56,7 @@ func UploadTombstone(ctx context.Context, tombstone *Tombstone, bkt objstore.Buc return err } - tsPath := path.Join(TombstoneDir, GenName()) + tsPath := path.Join(TombstoneDir, tombstone.ULID.String()) return bkt.Upload(ctx, tsPath, bytes.NewBuffer(b)) } From ff2f7322042a9f314dd2000f7d75df577bb53b8c Mon Sep 17 00:00:00 2001 From: Ben Ye Date: Sat, 18 Jun 2022 02:18:41 -0700 Subject: [PATCH 08/14] rename filters Signed-off-by: Ben Ye --- cmd/thanos/store.go | 10 ++++------ pkg/store/bucket.go | 4 ++-- pkg/tombstone/fetcher.go | 22 +++++++++++----------- 3 files changed, 17 insertions(+), 19 deletions(-) diff --git a/cmd/thanos/store.go b/cmd/thanos/store.go index 8fa79f8b5f..c2c8dc4459 100644 --- a/cmd/thanos/store.go +++ b/cmd/thanos/store.go @@ -6,10 +6,10 @@ package main import ( "context" "fmt" - "github.com/thanos-io/thanos/pkg/tombstone" "time" "github.com/alecthomas/units" + extflag "github.com/efficientgo/tools/extkingpin" "github.com/go-kit/log" "github.com/go-kit/log/level" grpclogging "github.com/grpc-ecosystem/go-grpc-middleware/v2/interceptors/logging" @@ -18,11 +18,8 @@ import ( "github.com/opentracing/opentracing-go" "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" - "github.com/prometheus/common/route" - commonmodel "github.com/prometheus/common/model" - - extflag "github.com/efficientgo/tools/extkingpin" + "github.com/prometheus/common/route" blocksAPI "github.com/thanos-io/thanos/pkg/api/blocks" "github.com/thanos-io/thanos/pkg/block" @@ -46,6 +43,7 @@ import ( storecache "github.com/thanos-io/thanos/pkg/store/cache" "github.com/thanos-io/thanos/pkg/store/labelpb" "github.com/thanos-io/thanos/pkg/tls" + "github.com/thanos-io/thanos/pkg/tombstone" "github.com/thanos-io/thanos/pkg/ui" ) @@ -315,7 +313,7 @@ func runStore( return errors.Wrap(err, "meta fetcher") } - tombstoneFetcher, err := tombstone.NewTombstoneFetcher(logger, conf.blockMetaFetchConcurrency, bkt, conf.dataDir, extprom.WrapRegistererWithPrefix("thanos_", reg), []tombstone.TombstoneFilter{}) + tombstoneFetcher, err := tombstone.NewFetcher(logger, conf.blockMetaFetchConcurrency, bkt, conf.dataDir, extprom.WrapRegistererWithPrefix("thanos_", reg), []tombstone.Filter{}) if err != nil { return errors.Wrap(err, "tombstone fetcher") } diff --git a/pkg/store/bucket.go b/pkg/store/bucket.go index d34d28e2dd..901734787a 100644 --- a/pkg/store/bucket.go +++ b/pkg/store/bucket.go @@ -277,7 +277,7 @@ type BucketStore struct { metrics *bucketStoreMetrics bkt objstore.InstrumentedBucketReader fetcher block.MetadataFetcher - tombstoneFetcher tombstone.TombstoneFetcher + tombstoneFetcher tombstone.Fetcher dir string indexCache storecache.IndexCache indexReaderPool *indexheader.ReaderPool @@ -393,7 +393,7 @@ func WithDebugLogging() BucketStoreOption { func NewBucketStore( bkt objstore.InstrumentedBucketReader, fetcher block.MetadataFetcher, - tombstoneFetcher tombstone.TombstoneFetcher, + tombstoneFetcher tombstone.Fetcher, dir string, chunksLimiterFactory ChunksLimiterFactory, seriesLimiterFactory SeriesLimiterFactory, diff --git a/pkg/tombstone/fetcher.go b/pkg/tombstone/fetcher.go index 9e32d2fa08..889349cc6f 100644 --- a/pkg/tombstone/fetcher.go +++ b/pkg/tombstone/fetcher.go @@ -6,7 +6,6 @@ package tombstone import ( "context" "encoding/json" - "github.com/golang/groupcache/singleflight" "io/ioutil" "os" "path" @@ -17,6 +16,7 @@ import ( "github.com/go-kit/log" "github.com/go-kit/log/level" + "github.com/golang/groupcache/singleflight" "github.com/oklog/ulid" "github.com/pkg/errors" "github.com/prometheus/client_golang/prometheus" @@ -39,12 +39,12 @@ var ( ErrorSyncTombstoneCorrupted = errors.New("tombstone corrupted") ) -type TombstoneFetcher interface { +type Fetcher interface { Fetch(ctx context.Context) (tombstones map[ulid.ULID]*Tombstone, partial map[ulid.ULID]error, err error) } -// TombstoneFilter allows filtering or modifying metas from the provided map or returns error. -type TombstoneFilter interface { +// Filter allows filtering or modifying metas from the provided map or returns error. +type Filter interface { Filter(ctx context.Context, tombstones map[ulid.ULID]*Tombstone, synced *extprom.TxGaugeVec) error } @@ -91,22 +91,22 @@ func newFetcherMetrics(reg prometheus.Registerer, syncedExtraLabels, modifiedExt return &m } -type Fetcher struct { +type fetcher struct { bkt objstore.InstrumentedBucketReader logger log.Logger metrics *block.FetcherMetrics wrapped *BaseFetcher - filters []TombstoneFilter + filters []Filter } -// NewTombstoneFetcher returns Fetcher. -func NewTombstoneFetcher(logger log.Logger, concurrency int, bkt objstore.InstrumentedBucketReader, dir string, reg prometheus.Registerer, filters []TombstoneFilter) (*Fetcher, error) { +// NewFetcher returns Fetcher. +func NewFetcher(logger log.Logger, concurrency int, bkt objstore.InstrumentedBucketReader, dir string, reg prometheus.Registerer, filters []Filter) (Fetcher, error) { b, err := NewBaseFetcher(logger, concurrency, bkt, dir, reg) if err != nil { return nil, err } - return &Fetcher{metrics: newFetcherMetrics(reg, nil, nil), wrapped: b, filters: filters, logger: b.logger}, nil + return &fetcher{metrics: newFetcherMetrics(reg, nil, nil), wrapped: b, filters: filters, logger: b.logger}, nil } //func (f *Fetcher) Fetch(ctx context.Context) (tombstones map[ulid.ULID]*Tombstone, partial map[ulid.ULID]error, err error) { @@ -143,7 +143,7 @@ func NewTombstoneFetcher(logger log.Logger, concurrency int, bkt objstore.Instru // It's caller responsibility to not change the returned metadata files. Maps can be modified. // // Returned error indicates a failure in fetching metadata. Returned meta can be assumed as correct, with some blocks missing. -func (f *Fetcher) Fetch(ctx context.Context) (metas map[ulid.ULID]*Tombstone, partial map[ulid.ULID]error, err error) { +func (f *fetcher) Fetch(ctx context.Context) (metas map[ulid.ULID]*Tombstone, partial map[ulid.ULID]error, err error) { return f.wrapped.fetch(ctx, f.metrics, f.filters) } @@ -378,7 +378,7 @@ func (f *BaseFetcher) fetchTombstone(ctx context.Context) (interface{}, error) { return resp, nil } -func (f *BaseFetcher) fetch(ctx context.Context, metrics *block.FetcherMetrics, filters []TombstoneFilter) (_ map[ulid.ULID]*Tombstone, _ map[ulid.ULID]error, err error) { +func (f *BaseFetcher) fetch(ctx context.Context, metrics *block.FetcherMetrics, filters []Filter) (_ map[ulid.ULID]*Tombstone, _ map[ulid.ULID]error, err error) { start := time.Now() defer func() { metrics.SyncDuration.Observe(time.Since(start).Seconds()) From 061ac931bb65236125ea49b451a01291ae2f7309 Mon Sep 17 00:00:00 2001 From: Ben Ye Date: Sat, 18 Jun 2022 18:54:40 -0700 Subject: [PATCH 09/14] finish fetcher part and add matching tests Signed-off-by: Ben Ye --- cmd/thanos/store.go | 6 +- cmd/thanos/tools_bucket.go | 10 +- docs/components/store.md | 3 + pkg/block/metadata/meta.go | 4 +- pkg/store/bucket.go | 64 +++++- pkg/store/bucket_test.go | 30 +++ pkg/tombstone/fetcher.go | 249 +++++++++-------------- pkg/tombstone/tombstone.go | 105 +++++++--- pkg/tombstone/tombstone_test.go | 345 ++++++++++++++++++++++++++++++++ 9 files changed, 615 insertions(+), 201 deletions(-) create mode 100644 pkg/tombstone/tombstone_test.go diff --git a/cmd/thanos/store.go b/cmd/thanos/store.go index c2c8dc4459..27c1d7f23f 100644 --- a/cmd/thanos/store.go +++ b/cmd/thanos/store.go @@ -64,6 +64,7 @@ type storeConfig struct { syncTombstonesInterval time.Duration blockSyncConcurrency int blockMetaFetchConcurrency int + tombstoneFetchConcurrency int filterConf *store.FilterConfig selectorRelabelConf extflag.PathOrContent advertiseCompatibilityLabel bool @@ -126,6 +127,9 @@ func (sc *storeConfig) registerFlag(cmd extkingpin.FlagClause) { cmd.Flag("sync-tombstone-duration", "Repeat interval for syncing the tombstones between local and remote view."). Default("3m").DurationVar(&sc.syncTombstonesInterval) + cmd.Flag("tombstone-fetch-concurrency", "Number of goroutines to use when fetching tombstones from object storage."). + Default("4").IntVar(&sc.tombstoneFetchConcurrency) + sc.filterConf = &store.FilterConfig{} cmd.Flag("min-time", "Start of time range limit to serve. Thanos Store will serve only metrics, which happened later than this value. Option can be a constant time in RFC3339 format or time duration relative to current time, such as -1d or 2h45m. Valid duration units are ms, s, m, h, d, w, y."). @@ -313,7 +317,7 @@ func runStore( return errors.Wrap(err, "meta fetcher") } - tombstoneFetcher, err := tombstone.NewFetcher(logger, conf.blockMetaFetchConcurrency, bkt, conf.dataDir, extprom.WrapRegistererWithPrefix("thanos_", reg), []tombstone.Filter{}) + tombstoneFetcher, err := tombstone.NewFetcher(logger, conf.tombstoneFetchConcurrency, bkt, conf.dataDir, extprom.WrapRegistererWithPrefix("thanos_", reg), []tombstone.Filter{}) if err != nil { return errors.Wrap(err, "tombstone fetcher") } diff --git a/cmd/thanos/tools_bucket.go b/cmd/thanos/tools_bucket.go index 92ef75fb32..35bdb18e16 100644 --- a/cmd/thanos/tools_bucket.go +++ b/cmd/thanos/tools_bucket.go @@ -160,7 +160,6 @@ type bucketDeleteConfig struct { matchers string author string reason string - labels []string } func (tbc *bucketVerifyConfig) registerBucketVerifyFlag(cmd extkingpin.FlagClause) *bucketVerifyConfig { @@ -279,8 +278,6 @@ func (tbc *bucketDeleteConfig) registerBucketDeleteFlag(cmd extkingpin.FlagClaus cmd.Flag("matchers", "The string representing label matchers").Default("").StringVar(&tbc.matchers) cmd.Flag("author", "Author of the deletion request").Default("not specified").StringVar(&tbc.author) cmd.Flag("reason", "Reason to perform the deletion request").Default("not specified").StringVar(&tbc.reason) - cmd.Flag("label", "Tombstone external labels. Only blocks that contain the matching external labels will be considered."). - PlaceHolder("=\"\"").StringsVar(&tbc.labels) return tbc } @@ -1417,11 +1414,6 @@ func registerBucketDelete(app extkingpin.AppClause, objStoreConfig *extflag.Path // Dummy actor to immediately kill the group after the run function returns. g.Add(func() error { return nil }, func(error) {}) - labels, err := parseFlagLabels(tbc.labels) - if err != nil { - return err - } - m, err := parser.ParseMetricSelector(tbc.matchers) if err != nil { return err @@ -1429,7 +1421,7 @@ func registerBucketDelete(app extkingpin.AppClause, objStoreConfig *extflag.Path newID := ulid.MustNew(ulid.Now(), rand.Reader) ts := tombstone.NewTombstone(newID, m, minTime.PrometheusTimestamp(), maxTime.PrometheusTimestamp(), - tbc.author, tbc.reason, labels) + tbc.author, tbc.reason) ctx, cancel := context.WithTimeout(context.Background(), tbc.timeout) defer cancel() diff --git a/docs/components/store.md b/docs/components/store.md index 5aaf12af02..b41bcc108e 100644 --- a/docs/components/store.md +++ b/docs/components/store.md @@ -183,6 +183,9 @@ Flags: --sync-tombstone-duration=3m Repeat interval for syncing the tombstones between local and remote view. + --tombstone-fetch-concurrency=4 + Number of goroutines to use when fetching + tombstones from object storage. --tracing.config= Alternative to 'tracing.config-file' flag (mutually exclusive). Content of YAML file with diff --git a/pkg/block/metadata/meta.go b/pkg/block/metadata/meta.go index 9e0082a5c2..d9dbda48ab 100644 --- a/pkg/block/metadata/meta.go +++ b/pkg/block/metadata/meta.go @@ -163,7 +163,7 @@ func (m Meta) WriteToDir(logger log.Logger, dir string) error { if err := f.Close(); err != nil { return err } - return renameFile(logger, tmp, path) + return RenameFile(logger, tmp, path) } // Write writes the given encoded meta to writer. @@ -173,7 +173,7 @@ func (m Meta) Write(w io.Writer) error { return enc.Encode(&m) } -func renameFile(logger log.Logger, from, to string) error { +func RenameFile(logger log.Logger, from, to string) error { if err := os.RemoveAll(to); err != nil { return err } diff --git a/pkg/store/bucket.go b/pkg/store/bucket.go index 901734787a..c831058492 100644 --- a/pkg/store/bucket.go +++ b/pkg/store/bucket.go @@ -1057,21 +1057,33 @@ func (s *BucketStore) Series(req *storepb.SeriesRequest, srv storepb.Store_Serie s.tombstonesMtx.RLock() tombstones := s.tombstones s.tombstonesMtx.RUnlock() + + // Filter tombstones by request time range. + matchedTombstones := filterTombstonesByTimeRange(tombstones, req.MinTime, req.MaxTime) + for _, bs := range s.blockSets { blockMatchers, ok := bs.labelMatchers(matchers...) if !ok { continue } - matchedTombstones := make([]*tombstone.Tombstone, 0, len(tombstones)) - for _, ts := range tombstones { - ok, _, err := promMatchesExternalLabels(blockMatchers, ts.Labels) - if err != nil { - return err - } + // Filter tombstones by block set external labels. + blockSetTombstones := make([]*tombstone.Tombstone, 0, len(matchedTombstones)) + for _, ts := range matchedTombstones { + matchers, ok := ts.MatchLabels(bs.labels) if !ok { continue } - matchedTombstones = append(matchedTombstones, ts) + // If no matchers are left after removing external label matchers, + // we drop the tombstone. + if matchers == nil || len(*matchers) == 0 { + continue + } + blockSetTombstones = append(blockSetTombstones, &tombstone.Tombstone{ + Matchers: matchers, + MinTime: ts.MinTime, + MaxTime: ts.MaxTime, + ULID: ts.ULID, + }) } blocks := bs.getFor(req.MinTime, req.MaxTime, req.MaxResolutionWindow, reqBlockMatchers) @@ -1084,6 +1096,26 @@ func (s *BucketStore) Series(req *storepb.SeriesRequest, srv storepb.Store_Serie b := b gctx := gctx + // Filter tombstones at block level by block metadata. + blockTombstones := make([]*tombstone.Tombstone, 0, len(blockSetTombstones)) + for _, ts := range blockSetTombstones { + matchers, ok := ts.MatchMeta(b.meta) + if !ok { + continue + } + // If no matchers are left after removing external label matchers, + // we drop the tombstone. + if matchers == nil || len(*matchers) == 0 { + continue + } + blockTombstones = append(blockTombstones, &tombstone.Tombstone{ + Matchers: matchers, + MinTime: ts.MinTime, + MaxTime: ts.MaxTime, + ULID: ts.ULID, + }) + } + if s.enableSeriesResponseHints { // Keep track of queried blocks. resHints.AddQueriedBlock(b.meta.ULID) @@ -1120,7 +1152,7 @@ func (s *BucketStore) Series(req *storepb.SeriesRequest, srv storepb.Store_Serie req.SkipChunks, req.MinTime, req.MaxTime, req.Aggregates, - matchedTombstones, + blockTombstones, ) if err != nil { return errors.Wrapf(err, "fetch series for block %s", b.meta.ULID) @@ -2808,3 +2840,19 @@ func (s queryStats) merge(o *queryStats) *queryStats { func NewDefaultChunkBytesPool(maxChunkPoolBytes uint64) (pool.Bytes, error) { return pool.NewBucketedBytes(chunkBytesPoolMinSize, chunkBytesPoolMaxSize, 2, maxChunkPoolBytes) } + +// filterTombstonesByTimeRange filters tombstones map by time range and returns a +// list of tomebstones sorted by min time. +func filterTombstonesByTimeRange(tombstones map[ulid.ULID]*tombstone.Tombstone, mint, maxt int64) []*tombstone.Tombstone { + matchedTombstones := make([]*tombstone.Tombstone, 0, len(tombstones)) + for _, ts := range tombstones { + if !ts.OverlapsClosedInterval(mint, maxt) { + continue + } + matchedTombstones = append(matchedTombstones, ts) + } + sort.Slice(matchedTombstones, func(i, j int) bool { + return matchedTombstones[i].MinTime < matchedTombstones[j].MinTime + }) + return matchedTombstones +} diff --git a/pkg/store/bucket_test.go b/pkg/store/bucket_test.go index 0768b61deb..9bd8d1138f 100644 --- a/pkg/store/bucket_test.go +++ b/pkg/store/bucket_test.go @@ -54,6 +54,7 @@ import ( storetestutil "github.com/thanos-io/thanos/pkg/store/storepb/testutil" "github.com/thanos-io/thanos/pkg/testutil" "github.com/thanos-io/thanos/pkg/testutil/e2eutil" + "github.com/thanos-io/thanos/pkg/tombstone" ) var emptyRelabelConfig = make([]*relabel.Config, 0) @@ -597,6 +598,7 @@ func TestBucketStore_Info(t *testing.T) { testutil.Ok(t, err) bucketStore, err := NewBucketStore( + nil, nil, nil, dir, @@ -844,9 +846,13 @@ func testSharding(t *testing.T, reuseDisk string, bkt objstore.Bucket, all ...ul }) testutil.Ok(t, err) + tombstoneFetcher, err := tombstone.NewFetcher(logger, 20, objstore.WithNoopInstr(bkt), dir, nil, []tombstone.Filter{}) + testutil.Ok(t, err) + bucketStore, err := NewBucketStore( objstore.WithNoopInstr(rec), metaFetcher, + tombstoneFetcher, dir, NewChunksLimiterFactory(0), NewSeriesLimiterFactory(0), @@ -1281,6 +1287,7 @@ func benchBucketSeries(t testutil.TB, skipChunk bool, samplesPerSeries, totalSer st, err := NewBucketStore( ibkt, f, + nil, tmpDir, NewChunksLimiterFactory(0), NewSeriesLimiterFactory(0), @@ -1654,6 +1661,7 @@ func TestSeries_ErrorUnmarshallingRequestHints(t *testing.T) { store, err := NewBucketStore( instrBkt, fetcher, + nil, tmpDir, NewChunksLimiterFactory(10000/MaxSamplesPerChunk), NewSeriesLimiterFactory(0), @@ -1746,6 +1754,7 @@ func TestSeries_BlockWithMultipleChunks(t *testing.T) { store, err := NewBucketStore( instrBkt, fetcher, + nil, tmpDir, NewChunksLimiterFactory(100000/MaxSamplesPerChunk), NewSeriesLimiterFactory(0), @@ -1929,6 +1938,7 @@ func setupStoreForHintsTest(t *testing.T) (testutil.TB, *BucketStore, []*storepb store, err := NewBucketStore( instrBkt, fetcher, + nil, tmpDir, NewChunksLimiterFactory(10000/MaxSamplesPerChunk), NewSeriesLimiterFactory(0), @@ -2317,3 +2327,23 @@ func BenchmarkDownsampledBlockSeries(b *testing.B) { } } } + +// TODO: finish tests. +func TestFilterTombstonesByTimeRange(t *testing.T) { + for _, tcase := range []struct { + name string + tombstones map[ulid.ULID]*tombstone.Tombstone + mint int64 + maxt int64 + expected []*tombstone.Tombstone + }{ + { + name: "", + }, + } { + t.Run(tcase.name, func(t *testing.T) { + got := filterTombstonesByTimeRange(tcase.tombstones, tcase.mint, tcase.maxt) + testutil.Equals(t, tcase.expected, got) + }) + } +} diff --git a/pkg/tombstone/fetcher.go b/pkg/tombstone/fetcher.go index 889349cc6f..4d52eaf5d0 100644 --- a/pkg/tombstone/fetcher.go +++ b/pkg/tombstone/fetcher.go @@ -32,10 +32,16 @@ import ( const ( tombstoneSubSys = "tombstone" + + CorruptedTombstone = "corrupted-tombstone" + LoadedTombstone = "loaded" + FailedTombstone = "failed" + + // Synced label values. + tooFreshTombstone = "too-fresh" ) var ( - ErrorSyncTombstoneNotFound = errors.New("tombstone not found") ErrorSyncTombstoneCorrupted = errors.New("tombstone corrupted") ) @@ -48,7 +54,7 @@ type Filter interface { Filter(ctx context.Context, tombstones map[ulid.ULID]*Tombstone, synced *extprom.TxGaugeVec) error } -func newFetcherMetrics(reg prometheus.Registerer, syncedExtraLabels, modifiedExtraLabels [][]string) *block.FetcherMetrics { +func newFetcherMetrics(reg prometheus.Registerer, syncedExtraLabels [][]string) *block.FetcherMetrics { var m block.FetcherMetrics m.Syncs = promauto.With(reg).NewCounter(prometheus.CounterOpts{ @@ -67,34 +73,26 @@ func newFetcherMetrics(reg prometheus.Registerer, syncedExtraLabels, modifiedExt Help: "Duration of the tombstone synchronization in seconds", Buckets: []float64{0.01, 1, 10, 100, 300, 600, 1000}, }) - //m.Synced = extprom.NewTxGaugeVec( - // reg, - // prometheus.GaugeOpts{ - // Subsystem: tombstoneSubSys, - // Name: "synced", - // Help: "Number of tombstone synced", - // }, - // []string{"state"}, - // append([][]string{ - // {CorruptedMeta}, - // {NoMeta}, - // {LoadedMeta}, - // {tooFreshMeta}, - // {FailedMeta}, - // {labelExcludedMeta}, - // {timeExcludedMeta}, - // {duplicateMeta}, - // {MarkedForDeletionMeta}, - // {MarkedForNoCompactionMeta}, - // }, syncedExtraLabels...)..., - //) + m.Synced = extprom.NewTxGaugeVec( + reg, + prometheus.GaugeOpts{ + Subsystem: tombstoneSubSys, + Name: "synced", + Help: "Number of tombstone synced", + }, + []string{"state"}, + append([][]string{ + {CorruptedTombstone}, + {LoadedTombstone}, + {tooFreshTombstone}, + {FailedTombstone}, + }, syncedExtraLabels...)..., + ) return &m } type fetcher struct { - bkt objstore.InstrumentedBucketReader - logger log.Logger - + logger log.Logger metrics *block.FetcherMetrics wrapped *BaseFetcher filters []Filter @@ -106,39 +104,9 @@ func NewFetcher(logger log.Logger, concurrency int, bkt objstore.InstrumentedBuc if err != nil { return nil, err } - return &fetcher{metrics: newFetcherMetrics(reg, nil, nil), wrapped: b, filters: filters, logger: b.logger}, nil + return &fetcher{metrics: newFetcherMetrics(reg, nil), wrapped: b, filters: filters, logger: b.logger}, nil } -//func (f *Fetcher) Fetch(ctx context.Context) (tombstones map[ulid.ULID]*Tombstone, partial map[ulid.ULID]error, err error) { -// var ( -// ts map[ulid.ULID]*Tombstone -// ) -// -// if err := f.bkt.Iter(ctx, TombstoneDir, func(name string) error { -// tombstoneFilename := path.Join("", name) -// tombstoneFile, err := f.bkt.Get(ctx, tombstoneFilename) -// if err != nil { -// return nil -// } -// defer runutil.CloseWithLogOnErr(f.logger, tombstoneFile, "close bkt tombstone reader") -// -// var t Tombstone -// tombstone, err := ioutil.ReadAll(tombstoneFile) -// if err != nil { -// return nil -// } -// if err := json.Unmarshal(tombstone, &t); err != nil { -// level.Error(f.logger).Log("msg", "failed to unmarshal tombstone", "file", tombstoneFilename, "err", err) -// return nil -// } -// ts = append(ts, &t) -// return nil -// }); err != nil { -// return nil, err -// } -// return ts, nil -//} - // Fetch returns all block metas as well as partial blocks (blocks without or with corrupted meta file) from the bucket. // It's caller responsibility to not change the returned metadata files. Maps can be modified. // @@ -154,7 +122,7 @@ type BaseFetcher struct { concurrency int bkt objstore.InstrumentedBucketReader - // Optional local directory to cache meta.json files. + // Optional local directory to cache tombstone files. cacheDir string syncs prometheus.Counter g singleflight.Group @@ -171,7 +139,7 @@ func NewBaseFetcher(logger log.Logger, concurrency int, bkt objstore.Instrumente cacheDir := "" if dir != "" { - cacheDir = filepath.Join(dir, "meta-syncer") + cacheDir = filepath.Join(dir, "tombstone") if err := os.MkdirAll(cacheDir, os.ModePerm); err != nil { return nil, err } @@ -183,11 +151,11 @@ func NewBaseFetcher(logger log.Logger, concurrency int, bkt objstore.Instrumente bkt: bkt, cacheDir: cacheDir, cached: map[ulid.ULID]*Tombstone{}, - //syncs: promauto.With(reg).NewCounter(prometheus.CounterOpts{ - // Subsystem: fetcherSubSys, - // Name: "base_syncs_total", - // Help: "Total tombstone synchronization attempts by base Fetcher", - //}), + syncs: promauto.With(reg).NewCounter(prometheus.CounterOpts{ + Subsystem: tombstoneSubSys, + Name: "base_syncs_total", + Help: "Total tombstone synchronization attempts by base Fetcher", + }), }, nil } @@ -203,65 +171,63 @@ type response struct { // loadMeta returns metadata from object storage or error. // It returns `ErrorSyncMetaNotFound` and `ErrorSyncMetaCorrupted` sentinel errors in those cases. func (f *BaseFetcher) loadTombstone(ctx context.Context, id ulid.ULID) (*Tombstone, error) { - tombstoneFile := path.Join(TombstoneDir, id.String()+".json") - ok, err := f.bkt.Exists(ctx, tombstoneFile) + fileName := id.String() + ".json" + tombstonePath := path.Join(TombstoneDir, fileName) + ok, err := f.bkt.Exists(ctx, tombstonePath) if err != nil { - return nil, errors.Wrapf(err, "%s file doesn't exist", tombstoneFile) + return nil, errors.Wrapf(err, "%s file exists", tombstonePath) } + // If the tombstone file is missing, we ignored it. if !ok { - return nil, ErrorSyncTombstoneNotFound + return nil, nil } if m, seen := f.cached[id]; seen { return m, nil } - //// Best effort load from local dir. - //if f.cacheDir != "" { - // m, err := metadata.ReadFromDir(cachedBlockDir) - // if err == nil { - // return m, nil - // } - // - // if !errors.Is(err, os.ErrNotExist) { - // level.Warn(f.logger).Log("msg", "best effort read of the local meta.json failed; removing cached block dir", "dir", cachedBlockDir, "err", err) - // if err := os.RemoveAll(cachedBlockDir); err != nil { - // level.Warn(f.logger).Log("msg", "best effort remove of cached dir failed; ignoring", "dir", cachedBlockDir, "err", err) - // } - // } - //} - - r, err := f.bkt.ReaderWithExpectedErrs(f.bkt.IsObjNotFoundErr).Get(ctx, tombstoneFile) + // Best effort load from local dir. + if f.cacheDir != "" { + t, err := ReadFromPath(path.Join(f.cacheDir, fileName)) + if err == nil { + return t, nil + } + + if !errors.Is(err, os.ErrNotExist) { + level.Warn(f.logger).Log("msg", "best effort read of the local tombstone failed; removing cached tombstone", "tombstone", fileName, "err", err) + if err := os.Remove(fileName); err != nil { + level.Warn(f.logger).Log("msg", "best effort remove of cached tombstone failed; ignoring", "tombstone", fileName, "err", err) + } + } + } + + r, err := f.bkt.ReaderWithExpectedErrs(f.bkt.IsObjNotFoundErr).Get(ctx, tombstonePath) if f.bkt.IsObjNotFoundErr(err) { - // Meta.json was deleted between bkt.Exists and here. - return nil, errors.Wrapf(ErrorSyncTombstoneNotFound, "%v", err) + // tombstone was deleted between bkt.Exists and here. + return nil, nil } if err != nil { - return nil, errors.Wrapf(err, "get tombstone file: %v", tombstoneFile) + return nil, errors.Wrapf(err, "get tombstone file: %v", tombstonePath) } defer runutil.CloseWithLogOnErr(f.logger, r, "close bkt meta get") content, err := ioutil.ReadAll(r) if err != nil { - return nil, errors.Wrapf(err, "read meta file: %v", tombstoneFile) + return nil, errors.Wrapf(err, "read meta file: %v", tombstonePath) } var t Tombstone if err := json.Unmarshal(content, &t); err != nil { - return nil, errors.Wrapf(ErrorSyncTombstoneCorrupted, "tombstone %v unmarshal: %v", tombstoneFile, err) + return nil, errors.Wrapf(ErrorSyncTombstoneCorrupted, "tombstone %v unmarshal: %v", tombstonePath, err) } - //// Best effort cache in local dir. - //if f.cacheDir != "" { - // if err := os.MkdirAll(cachedBlockDir, os.ModePerm); err != nil { - // level.Warn(f.logger).Log("msg", "best effort mkdir of the meta.json block dir failed; ignoring", "dir", cachedBlockDir, "err", err) - // } - // - // if err := m.WriteToDir(f.logger, cachedBlockDir); err != nil { - // level.Warn(f.logger).Log("msg", "best effort save of the meta.json to local dir failed; ignoring", "dir", cachedBlockDir, "err", err) - // } - //} + // Best effort cache in local dir. + if f.cacheDir != "" { + if err := t.WriteToDir(f.logger, f.cacheDir); err != nil { + level.Warn(f.logger).Log("msg", "best effort save of the tombstone to local dir failed; ignoring", "tombstone", fileName, "err", err) + } + } return &t, nil } @@ -309,11 +275,15 @@ func (f *BaseFetcher) fetchTombstone(ctx context.Context) (interface{}, error) { }) } - // Workers scheduled, distribute blocks. + // Workers scheduled, distribute tombstones. eg.Go(func() error { defer close(ch) return f.bkt.Iter(ctx, TombstoneDir, func(name string) error { - id, err := ulid.Parse(strings.TrimSuffix(filepath.Base(name), ".json")) + base := filepath.Base(name) + if !strings.HasSuffix(base, ".json") { + return nil + } + id, err := ulid.Parse(strings.TrimSuffix(base, ".json")) if err != nil { return nil } @@ -335,46 +305,6 @@ func (f *BaseFetcher) fetchTombstone(ctx context.Context) (interface{}, error) { if len(resp.tombstoneErrs) > 0 { return resp, nil } - - //// Only for complete view of blocks update the cache. - //cached := make(map[ulid.ULID]*metadata.Meta, len(resp.metas)) - //for id, m := range resp.metas { - // cached[id] = m - //} - - //f.mtx.Lock() - //f.cached = cached - //f.mtx.Unlock() - - //// Best effort cleanup of disk-cached metas. - //if f.cacheDir != "" { - // fis, err := ioutil.ReadDir(f.cacheDir) - // names := make([]string, 0, len(fis)) - // for _, fi := range fis { - // names = append(names, fi.Name()) - // } - // if err != nil { - // level.Warn(f.logger).Log("msg", "best effort remove of not needed cached dirs failed; ignoring", "err", err) - // } else { - // for _, n := range names { - // id, ok := IsBlockDir(n) - // if !ok { - // continue - // } - // - // if _, ok := resp.metas[id]; ok { - // continue - // } - // - // cachedBlockDir := filepath.Join(f.cacheDir, id.String()) - // - // // No such block loaded, remove the local dir. - // if err := os.RemoveAll(cachedBlockDir); err != nil { - // level.Warn(f.logger).Log("msg", "best effort remove of not needed cached dir failed; ignoring", "dir", cachedBlockDir, "err", err) - // } - // } - // } - //} return resp, nil } @@ -387,7 +317,7 @@ func (f *BaseFetcher) fetch(ctx context.Context, metrics *block.FetcherMetrics, } }() metrics.Syncs.Inc() - metrics.ResetTx() + metrics.Synced.ResetTx() // Run this in thread safe run group. v, err := f.g.Do("", func() (i interface{}, err error) { @@ -405,9 +335,8 @@ func (f *BaseFetcher) fetch(ctx context.Context, metrics *block.FetcherMetrics, tombstones[id] = m } - //metrics.Synced.WithLabelValues(FailedMeta).Set(float64(len(resp.metaErrs))) - //metrics.Synced.WithLabelValues(NoMeta).Set(resp.noMetas) - //metrics.Synced.WithLabelValues(CorruptedMeta).Set(resp.corruptedMetas) + metrics.Synced.WithLabelValues(FailedTombstone).Set(float64(len(resp.tombstoneErrs))) + metrics.Synced.WithLabelValues(CorruptedTombstone).Set(resp.corruptedTombstones) for _, filter := range filters { // NOTE: filter can update synced metric accordingly to the reason of the exclude. @@ -416,8 +345,8 @@ func (f *BaseFetcher) fetch(ctx context.Context, metrics *block.FetcherMetrics, } } - //metrics.Synced.WithLabelValues(LoadedMeta).Set(float64(len(metas))) - metrics.Submit() + metrics.Synced.WithLabelValues(LoadedTombstone).Set(float64(len(tombstones))) + metrics.Synced.Submit() if len(resp.tombstoneErrs) > 0 { return tombstones, resp.partial, errors.Wrap(resp.tombstoneErrs.Err(), "incomplete view") @@ -427,16 +356,36 @@ func (f *BaseFetcher) fetch(ctx context.Context, metrics *block.FetcherMetrics, return tombstones, resp.partial, nil } -// TombstoneDelayFilter considers the tombstone file only after it exists longer than the delay period. -type TombstoneDelayFilter struct { +// DelayTombstoneFilter considers the tombstone file only after it exists longer than the delay period. +type DelayTombstoneFilter struct { delay time.Duration logger log.Logger } -func (f *TombstoneDelayFilter) Filter(_ context.Context, tombstones map[ulid.ULID]*Tombstone, synced *extprom.TxGaugeVec) error { +// ConsistencyDelayMetaFilter is a BaseFetcher filter that filters out blocks that are created before a specified consistency delay. +// Not go-routine safe. +type ConsistencyDelayMetaFilter struct { + logger log.Logger + consistencyDelay time.Duration +} + +// NewDelayTombstoneFilter creates DelayTombstoneFilter. +func NewDelayTombstoneFilter(logger log.Logger, delay time.Duration) *DelayTombstoneFilter { + if logger == nil { + logger = log.NewNopLogger() + } + + return &DelayTombstoneFilter{ + logger: logger, + delay: delay, + } +} + +func (f *DelayTombstoneFilter) Filter(_ context.Context, tombstones map[ulid.ULID]*Tombstone, synced *extprom.TxGaugeVec) error { for id := range tombstones { if ulid.Now()-id.Time() < uint64(f.delay/time.Millisecond) { level.Debug(f.logger).Log("msg", "tombstone is too fresh for now", "tombstone", id) + synced.WithLabelValues(tooFreshTombstone).Inc() delete(tombstones, id) } } diff --git a/pkg/tombstone/tombstone.go b/pkg/tombstone/tombstone.go index 5edacc6200..b7df3e0d5d 100644 --- a/pkg/tombstone/tombstone.go +++ b/pkg/tombstone/tombstone.go @@ -7,15 +7,19 @@ import ( "bytes" "context" "encoding/json" + "io" + "os" + "path" + "path/filepath" + "github.com/go-kit/log" - "github.com/go-kit/log/level" "github.com/oklog/ulid" "github.com/prometheus/prometheus/model/labels" + + "github.com/thanos-io/thanos/pkg/block" "github.com/thanos-io/thanos/pkg/block/metadata" "github.com/thanos-io/thanos/pkg/objstore" "github.com/thanos-io/thanos/pkg/runutil" - "io/ioutil" - "path" ) const ( @@ -31,13 +35,10 @@ type Tombstone struct { MaxTime int64 `json:"maxTime"` Author string `json:"author"` Reason string `json:"reason"` - // Labels are the external labels identifying the producer as well as tenant. - // See https://thanos.io/tip/thanos/storage.md#external-labels for details. - Labels labels.Labels `json:"labels"` } // NewTombstone returns a new instance of Tombstone. -func NewTombstone(ulid ulid.ULID, matchers metadata.Matchers, minTime, maxTime int64, author, reason string, labels labels.Labels) *Tombstone { +func NewTombstone(ulid ulid.ULID, matchers metadata.Matchers, minTime, maxTime int64, author, reason string) *Tombstone { return &Tombstone{ ULID: ulid, Matchers: &matchers, @@ -45,10 +46,51 @@ func NewTombstone(ulid ulid.ULID, matchers metadata.Matchers, minTime, maxTime i MaxTime: maxTime, Author: author, Reason: reason, - Labels: labels, } } +// ReadFromPath the tombstone from filesystem path. +func ReadFromPath(path string) (_ *Tombstone, err error) { + f, err := os.Open(path) + if err != nil { + return nil, err + } + defer runutil.ExhaustCloseWithErrCapture(&err, f, "close tombstone") + + var t Tombstone + if err = json.NewDecoder(f).Decode(&t); err != nil { + return nil, err + } + return &t, nil +} + +// WriteToDir writes the encoded meta into /meta.json. +func (t *Tombstone) WriteToDir(logger log.Logger, dir string) error { + // Make any changes to the file appear atomic. + path := filepath.Join(dir, t.ULID.String()+".json") + tmp := path + ".tmp" + + f, err := os.Create(tmp) + if err != nil { + return err + } + + if err := t.Write(f); err != nil { + runutil.CloseWithLogOnErr(logger, f, "close tombstone") + return err + } + if err := f.Close(); err != nil { + return err + } + return metadata.RenameFile(logger, tmp, path) +} + +func (t *Tombstone) Write(w io.Writer) error { + enc := json.NewEncoder(w) + enc.SetIndent("", "\t") + return enc.Encode(&t) +} + // UploadTombstone uploads the given tombstone to object storage. func UploadTombstone(ctx context.Context, tombstone *Tombstone, bkt objstore.Bucket) error { b, err := json.Marshal(tombstone) @@ -56,35 +98,36 @@ func UploadTombstone(ctx context.Context, tombstone *Tombstone, bkt objstore.Buc return err } - tsPath := path.Join(TombstoneDir, tombstone.ULID.String()) + tsPath := path.Join(TombstoneDir, tombstone.ULID.String()+".json") return bkt.Upload(ctx, tsPath, bytes.NewBuffer(b)) } -// ReadTombstones returns all the tombstones present in the object storage. -func ReadTombstones(ctx context.Context, bkt objstore.InstrumentedBucketReader, logger log.Logger) ([]*Tombstone, error) { - var ts []*Tombstone +// OverlapsClosedInterval Returns true if the chunk overlaps [mint, maxt]. +func (t *Tombstone) OverlapsClosedInterval(mint, maxt int64) bool { + return t.MinTime <= maxt && mint <= t.MaxTime +} - if err := bkt.Iter(ctx, TombstoneDir, func(name string) error { - tombstoneFilename := path.Join("", name) - tombstoneFile, err := bkt.Get(ctx, tombstoneFilename) - if err != nil { - return nil - } - defer runutil.CloseWithLogOnErr(logger, tombstoneFile, "close bkt tombstone reader") +func (t *Tombstone) MatchMeta(meta *metadata.Meta) (*metadata.Matchers, bool) { + if !t.OverlapsClosedInterval(meta.MinTime, meta.MaxTime-1) { + return nil, false + } + // We add the special __block_id label to support matching by block ID. + lbls := labels.FromMap(meta.Thanos.Labels) + lbls = append(lbls, labels.Label{Name: block.BlockIDLabel, Value: meta.ULID.String()}) + return t.MatchLabels(lbls) +} - var t Tombstone - tombstone, err := ioutil.ReadAll(tombstoneFile) - if err != nil { - return nil +func (t *Tombstone) MatchLabels(lbls labels.Labels) (*metadata.Matchers, bool) { + matchers := make(metadata.Matchers, 0, len(*t.Matchers)) + for _, m := range *t.Matchers { + v := lbls.Get(m.Name) + if v == "" { + matchers = append(matchers, m) + continue } - if err := json.Unmarshal(tombstone, &t); err != nil { - level.Error(logger).Log("msg", "failed to unmarshal tombstone", "file", tombstoneFilename, "err", err) - return nil + if !m.Matches(v) { + return nil, false } - ts = append(ts, &t) - return nil - }); err != nil { - return nil, err } - return ts, nil + return &matchers, true } diff --git a/pkg/tombstone/tombstone_test.go b/pkg/tombstone/tombstone_test.go new file mode 100644 index 0000000000..e5387a94cc --- /dev/null +++ b/pkg/tombstone/tombstone_test.go @@ -0,0 +1,345 @@ +// Copyright (c) The Thanos Authors. +// Licensed under the Apache License 2.0. + +package tombstone + +import ( + "testing" + + "github.com/oklog/ulid" + "github.com/prometheus/prometheus/model/labels" + "github.com/prometheus/prometheus/tsdb" + + "github.com/thanos-io/thanos/pkg/block" + "github.com/thanos-io/thanos/pkg/block/metadata" + "github.com/thanos-io/thanos/pkg/testutil" +) + +func TestMatchMeta(t *testing.T) { + ulid1 := ulid.MustNew(0, nil) + ulid2 := ulid.MustNew(10, nil) + for _, tcase := range []struct { + name string + tombstone *Tombstone + meta *metadata.Meta + expectedMatchers *metadata.Matchers + expectedMatched bool + }{ + { + name: "empty tombstone matchers, time range overlapped", + tombstone: &Tombstone{ + Matchers: &metadata.Matchers{}, + MinTime: 0, + MaxTime: 20, + }, + meta: &metadata.Meta{ + BlockMeta: tsdb.BlockMeta{ + ULID: ulid1, + MinTime: 0, + MaxTime: 10, + }, + Thanos: metadata.Thanos{ + Labels: map[string]string{"cluster": "one"}, + }, + }, + expectedMatchers: &metadata.Matchers{}, + expectedMatched: true, + }, + { + name: "empty tombstone matchers, time range same", + tombstone: &Tombstone{ + Matchers: &metadata.Matchers{}, + MinTime: 0, + MaxTime: 9, + }, + // TSDB block metadata time is [minTime, maxTime) so it is actually [0, 9] + meta: &metadata.Meta{ + BlockMeta: tsdb.BlockMeta{ + ULID: ulid1, + MinTime: 0, + MaxTime: 10, + }, + Thanos: metadata.Thanos{ + Labels: map[string]string{"cluster": "one"}, + }, + }, + expectedMatchers: &metadata.Matchers{}, + expectedMatched: true, + }, + { + name: "empty tombstone matchers, unmatched time range", + tombstone: &Tombstone{ + Matchers: &metadata.Matchers{}, + MinTime: 0, + MaxTime: 1, + }, + meta: &metadata.Meta{ + BlockMeta: tsdb.BlockMeta{ + ULID: ulid1, + MinTime: 10, + MaxTime: 20, + }, + Thanos: metadata.Thanos{ + Labels: map[string]string{"cluster": "one"}, + }, + }, + expectedMatchers: nil, + expectedMatched: false, + }, + { + name: "empty tombstone matchers, unmatched time range 2", + tombstone: &Tombstone{ + Matchers: &metadata.Matchers{}, + MinTime: 30, + MaxTime: 40, + }, + meta: &metadata.Meta{ + BlockMeta: tsdb.BlockMeta{ + ULID: ulid1, + MinTime: 10, + MaxTime: 20, + }, + Thanos: metadata.Thanos{ + Labels: map[string]string{"cluster": "one"}, + }, + }, + expectedMatchers: nil, + expectedMatched: false, + }, + { + name: "tombstone matchers unrelated to external labels", + tombstone: &Tombstone{ + Matchers: &metadata.Matchers{labels.MustNewMatcher(labels.MatchEqual, "__name__", "up")}, + MinTime: 5, + MaxTime: 25, + }, + meta: &metadata.Meta{ + BlockMeta: tsdb.BlockMeta{ + ULID: ulid1, + MinTime: 10, + MaxTime: 30, + }, + Thanos: metadata.Thanos{ + Labels: map[string]string{"cluster": "one"}, + }, + }, + expectedMatchers: &metadata.Matchers{labels.MustNewMatcher(labels.MatchEqual, "__name__", "up")}, + expectedMatched: true, + }, + { + name: "tombstone matchers matches external labels", + tombstone: &Tombstone{ + Matchers: &metadata.Matchers{labels.MustNewMatcher(labels.MatchEqual, "cluster", "one")}, + MinTime: 5, + MaxTime: 25, + }, + meta: &metadata.Meta{ + BlockMeta: tsdb.BlockMeta{ + ULID: ulid1, + MinTime: 10, + MaxTime: 30, + }, + Thanos: metadata.Thanos{ + Labels: map[string]string{"cluster": "one"}, + }, + }, + expectedMatchers: &metadata.Matchers{}, + expectedMatched: true, + }, + { + name: "tombstone matchers matches external labels", + tombstone: &Tombstone{ + Matchers: &metadata.Matchers{labels.MustNewMatcher(labels.MatchEqual, "cluster", "one"), + labels.MustNewMatcher(labels.MatchEqual, "__test__", "up")}, + MinTime: 5, + MaxTime: 25, + }, + meta: &metadata.Meta{ + BlockMeta: tsdb.BlockMeta{ + ULID: ulid1, + MinTime: 10, + MaxTime: 30, + }, + Thanos: metadata.Thanos{ + Labels: map[string]string{"cluster": "one"}, + }, + }, + expectedMatchers: &metadata.Matchers{labels.MustNewMatcher(labels.MatchEqual, "__test__", "up")}, + expectedMatched: true, + }, + { + name: "tombstone matchers don't match external labels", + tombstone: &Tombstone{ + Matchers: &metadata.Matchers{labels.MustNewMatcher(labels.MatchEqual, "cluster", "two")}, + MinTime: 5, + MaxTime: 25, + }, + meta: &metadata.Meta{ + BlockMeta: tsdb.BlockMeta{ + ULID: ulid1, + MinTime: 10, + MaxTime: 30, + }, + Thanos: metadata.Thanos{ + Labels: map[string]string{"cluster": "one"}, + }, + }, + expectedMatchers: nil, + expectedMatched: false, + }, + { + name: "tombstone matchers match block ID", + tombstone: &Tombstone{ + Matchers: &metadata.Matchers{labels.MustNewMatcher(labels.MatchEqual, block.BlockIDLabel, ulid1.String())}, + MinTime: 5, + MaxTime: 25, + }, + meta: &metadata.Meta{ + BlockMeta: tsdb.BlockMeta{ + ULID: ulid1, + MinTime: 10, + MaxTime: 30, + }, + Thanos: metadata.Thanos{ + Labels: map[string]string{"cluster": "one"}, + }, + }, + expectedMatchers: &metadata.Matchers{}, + expectedMatched: true, + }, + { + name: "tombstone matchers match block ID and external labels", + tombstone: &Tombstone{ + Matchers: &metadata.Matchers{labels.MustNewMatcher(labels.MatchEqual, block.BlockIDLabel, ulid1.String()), + labels.MustNewMatcher(labels.MatchEqual, "cluster", "one")}, + MinTime: 5, + MaxTime: 25, + }, + meta: &metadata.Meta{ + BlockMeta: tsdb.BlockMeta{ + ULID: ulid1, + MinTime: 10, + MaxTime: 30, + }, + Thanos: metadata.Thanos{ + Labels: map[string]string{"cluster": "one"}, + }, + }, + expectedMatchers: &metadata.Matchers{}, + expectedMatched: true, + }, + { + name: "tombstone matchers don't match block ID", + tombstone: &Tombstone{ + Matchers: &metadata.Matchers{labels.MustNewMatcher(labels.MatchEqual, block.BlockIDLabel, ulid2.String()), + labels.MustNewMatcher(labels.MatchEqual, "cluster", "one")}, + MinTime: 5, + MaxTime: 25, + }, + meta: &metadata.Meta{ + BlockMeta: tsdb.BlockMeta{ + ULID: ulid1, + MinTime: 10, + MaxTime: 30, + }, + Thanos: metadata.Thanos{ + Labels: map[string]string{"cluster": "one"}, + }, + }, + expectedMatchers: nil, + expectedMatched: false, + }, + { + name: "tombstone matchers match block ID, external labels and additional matchers", + tombstone: &Tombstone{ + Matchers: &metadata.Matchers{labels.MustNewMatcher(labels.MatchEqual, block.BlockIDLabel, ulid2.String()), + labels.MustNewMatcher(labels.MatchEqual, "cluster", "one"), + labels.MustNewMatcher(labels.MatchEqual, "__name__", "up"), + }, + MinTime: 5, + MaxTime: 25, + }, + meta: &metadata.Meta{ + BlockMeta: tsdb.BlockMeta{ + ULID: ulid1, + MinTime: 10, + MaxTime: 30, + }, + Thanos: metadata.Thanos{ + Labels: map[string]string{"cluster": "one"}, + }, + }, + expectedMatchers: &metadata.Matchers{labels.MustNewMatcher(labels.MatchEqual, "__name__", "up")}, + expectedMatched: true, + }, + } { + t.Run(tcase.name, func(t *testing.T) { + matchers, ok := tcase.tombstone.MatchMeta(tcase.meta) + testutil.Equals(t, tcase.expectedMatched, ok) + testutil.Equals(t, tcase.expectedMatchers, matchers) + }) + } +} + +func TestMatchLabels(t *testing.T) { + for _, tcase := range []struct { + name string + tombstone *Tombstone + labels map[string]string + expectedMatchers *metadata.Matchers + expectedMatched bool + }{ + { + name: "empty tombstone matchers", + tombstone: &Tombstone{ + Matchers: &metadata.Matchers{}, + }, + labels: map[string]string{}, + expectedMatchers: &metadata.Matchers{}, + expectedMatched: true, + }, + { + name: "unrelated matchers", + tombstone: &Tombstone{ + Matchers: &metadata.Matchers{labels.MustNewMatcher(labels.MatchEqual, "tenant", "default")}, + }, + labels: map[string]string{"cluster": "one"}, + expectedMatchers: &metadata.Matchers{labels.MustNewMatcher(labels.MatchEqual, "tenant", "default")}, + expectedMatched: true, + }, + { + name: "matched external label matchers", + tombstone: &Tombstone{ + Matchers: &metadata.Matchers{labels.MustNewMatcher(labels.MatchEqual, "cluster", "one")}, + }, + labels: map[string]string{"cluster": "one"}, + expectedMatchers: &metadata.Matchers{}, + expectedMatched: true, + }, + { + name: "unmatched external label matchers", + tombstone: &Tombstone{ + Matchers: &metadata.Matchers{labels.MustNewMatcher(labels.MatchEqual, "cluster", "one")}, + }, + labels: map[string]string{"cluster": "two"}, + expectedMatchers: nil, + expectedMatched: false, + }, + { + name: "unmatched external label matchers with two matchers", + tombstone: &Tombstone{ + Matchers: &metadata.Matchers{labels.MustNewMatcher(labels.MatchEqual, "cluster", "one"), + labels.MustNewMatcher(labels.MatchEqual, "region", "eu")}, + }, + labels: map[string]string{"cluster": "two", "region": "eu"}, + expectedMatchers: nil, + expectedMatched: false, + }, + } { + t.Run(tcase.name, func(t *testing.T) { + matchers, ok := tcase.tombstone.MatchLabels(labels.FromMap(tcase.labels)) + testutil.Equals(t, tcase.expectedMatched, ok) + testutil.Equals(t, tcase.expectedMatchers, matchers) + }) + } +} From fbf07ec79d20bf31429864dca11c6d47bc72ff95 Mon Sep 17 00:00:00 2001 From: Ben Ye Date: Sun, 19 Jun 2022 14:01:42 -0700 Subject: [PATCH 10/14] fix lint Signed-off-by: Ben Ye --- pkg/store/bucket.go | 3 ++- pkg/tombstone/fetcher.go | 18 +++++++++++------- 2 files changed, 13 insertions(+), 8 deletions(-) diff --git a/pkg/store/bucket.go b/pkg/store/bucket.go index c831058492..49ab24f9d9 100644 --- a/pkg/store/bucket.go +++ b/pkg/store/bucket.go @@ -876,7 +876,8 @@ PostingsLoop: s.refs = append(s.refs, meta.Ref) } - // If all chunks are matched by the tombstone then check next posting. + // If all chunks are matched by the tombstone then + // we don't need to add current series entry, check next posting. if len(s.chks) == 0 { continue PostingsLoop } diff --git a/pkg/tombstone/fetcher.go b/pkg/tombstone/fetcher.go index 4d52eaf5d0..dd282af2b3 100644 --- a/pkg/tombstone/fetcher.go +++ b/pkg/tombstone/fetcher.go @@ -305,6 +305,17 @@ func (f *BaseFetcher) fetchTombstone(ctx context.Context) (interface{}, error) { if len(resp.tombstoneErrs) > 0 { return resp, nil } + + // Only for complete view update the cache. + cached := make(map[ulid.ULID]*Tombstone, len(resp.tombstones)) + for id, m := range resp.tombstones { + cached[id] = m + } + + f.mtx.Lock() + f.cached = cached + f.mtx.Unlock() + return resp, nil } @@ -362,13 +373,6 @@ type DelayTombstoneFilter struct { logger log.Logger } -// ConsistencyDelayMetaFilter is a BaseFetcher filter that filters out blocks that are created before a specified consistency delay. -// Not go-routine safe. -type ConsistencyDelayMetaFilter struct { - logger log.Logger - consistencyDelay time.Duration -} - // NewDelayTombstoneFilter creates DelayTombstoneFilter. func NewDelayTombstoneFilter(logger log.Logger, delay time.Duration) *DelayTombstoneFilter { if logger == nil { From 515e097d075874cd84408206de290cb7ef8782b8 Mon Sep 17 00:00:00 2001 From: Ben Ye Date: Sun, 19 Jun 2022 17:46:26 -0700 Subject: [PATCH 11/14] fix circleci tests Signed-off-by: Ben Ye --- pkg/store/bucket.go | 3 ++ pkg/store/bucket_e2e_test.go | 1 + pkg/store/bucket_test.go | 73 ++++++++++++++++++++++++++++++++- pkg/tombstone/tombstone_test.go | 2 +- 4 files changed, 76 insertions(+), 3 deletions(-) diff --git a/pkg/store/bucket.go b/pkg/store/bucket.go index 49ab24f9d9..a93d19835b 100644 --- a/pkg/store/bucket.go +++ b/pkg/store/bucket.go @@ -2853,6 +2853,9 @@ func filterTombstonesByTimeRange(tombstones map[ulid.ULID]*tombstone.Tombstone, matchedTombstones = append(matchedTombstones, ts) } sort.Slice(matchedTombstones, func(i, j int) bool { + if matchedTombstones[i].MinTime == matchedTombstones[j].MinTime { + return matchedTombstones[i].MaxTime < matchedTombstones[j].MaxTime + } return matchedTombstones[i].MinTime < matchedTombstones[j].MinTime }) return matchedTombstones diff --git a/pkg/store/bucket_e2e_test.go b/pkg/store/bucket_e2e_test.go index 3f0a2ed81a..b41a27db68 100644 --- a/pkg/store/bucket_e2e_test.go +++ b/pkg/store/bucket_e2e_test.go @@ -184,6 +184,7 @@ func prepareStoreWithTestBlocks(t testing.TB, dir string, bkt objstore.Bucket, m store, err := NewBucketStore( objstore.WithNoopInstr(bkt), metaFetcher, + nil, dir, chunksLimiterFactory, seriesLimiterFactory, diff --git a/pkg/store/bucket_test.go b/pkg/store/bucket_test.go index 9bd8d1138f..618174899a 100644 --- a/pkg/store/bucket_test.go +++ b/pkg/store/bucket_test.go @@ -2328,8 +2328,31 @@ func BenchmarkDownsampledBlockSeries(b *testing.B) { } } -// TODO: finish tests. func TestFilterTombstonesByTimeRange(t *testing.T) { + ulid1 := ulid.MustNew(0, nil) + ulid2 := ulid.MustNew(5, nil) + ulid3 := ulid.MustNew(10, nil) + ulid4 := ulid.MustNew(15, nil) + ts1 := &tombstone.Tombstone{ + ULID: ulid1, + MinTime: 0, + MaxTime: 1, + } + ts2 := &tombstone.Tombstone{ + ULID: ulid2, + MinTime: 5, + MaxTime: 10, + } + ts3 := &tombstone.Tombstone{ + ULID: ulid3, + MinTime: 8, + MaxTime: 20, + } + ts4 := &tombstone.Tombstone{ + ULID: ulid4, + MinTime: 11, + MaxTime: 20, + } for _, tcase := range []struct { name string tombstones map[ulid.ULID]*tombstone.Tombstone @@ -2338,7 +2361,53 @@ func TestFilterTombstonesByTimeRange(t *testing.T) { expected []*tombstone.Tombstone }{ { - name: "", + name: "empty tombstones", + tombstones: map[ulid.ULID]*tombstone.Tombstone{}, + mint: 0, + maxt: 10, + expected: []*tombstone.Tombstone{}, + }, + { + name: "one tombstone in timeperiod", + tombstones: map[ulid.ULID]*tombstone.Tombstone{ + ulid1: ts1, + }, + mint: 0, + maxt: 10, + expected: []*tombstone.Tombstone{ts1}, + }, + { + name: "multiple tombstones in timeperiod", + tombstones: map[ulid.ULID]*tombstone.Tombstone{ + ulid1: ts1, + ulid2: ts2, + }, + mint: 0, + maxt: 10, + expected: []*tombstone.Tombstone{ts1, ts2}, + }, + { + name: "multiple tombstones overlapped in timeperiod", + tombstones: map[ulid.ULID]*tombstone.Tombstone{ + ulid1: ts1, + ulid2: ts2, + ulid3: ts3, + }, + mint: 0, + maxt: 10, + expected: []*tombstone.Tombstone{ts1, ts2, ts3}, + }, + { + name: "ulid4 not in period", + tombstones: map[ulid.ULID]*tombstone.Tombstone{ + ulid1: ts1, + ulid2: ts2, + ulid3: ts3, + ulid4: ts4, + }, + mint: 0, + maxt: 10, + expected: []*tombstone.Tombstone{ts1, ts2, ts3}, }, } { t.Run(tcase.name, func(t *testing.T) { diff --git a/pkg/tombstone/tombstone_test.go b/pkg/tombstone/tombstone_test.go index e5387a94cc..307f3fb277 100644 --- a/pkg/tombstone/tombstone_test.go +++ b/pkg/tombstone/tombstone_test.go @@ -252,7 +252,7 @@ func TestMatchMeta(t *testing.T) { { name: "tombstone matchers match block ID, external labels and additional matchers", tombstone: &Tombstone{ - Matchers: &metadata.Matchers{labels.MustNewMatcher(labels.MatchEqual, block.BlockIDLabel, ulid2.String()), + Matchers: &metadata.Matchers{labels.MustNewMatcher(labels.MatchEqual, block.BlockIDLabel, ulid1.String()), labels.MustNewMatcher(labels.MatchEqual, "cluster", "one"), labels.MustNewMatcher(labels.MatchEqual, "__name__", "up"), }, From 00cf8cc38203d1406edf516048f423688007dc55 Mon Sep 17 00:00:00 2001 From: Ben Ye Date: Mon, 20 Jun 2022 01:00:23 -0700 Subject: [PATCH 12/14] add implementation for tombstone cache Signed-off-by: Ben Ye --- pkg/store/bucket.go | 185 +++++++++++++++++-------------------- pkg/store/bucket_test.go | 89 ------------------ pkg/tombstone/cache.go | 72 +++++++++++++++ pkg/tombstone/tombstone.go | 17 +++- 4 files changed, 172 insertions(+), 191 deletions(-) create mode 100644 pkg/tombstone/cache.go diff --git a/pkg/store/bucket.go b/pkg/store/bucket.go index a93d19835b..b59d2d0fd4 100644 --- a/pkg/store/bucket.go +++ b/pkg/store/bucket.go @@ -307,9 +307,6 @@ type BucketStore struct { advLabelSets []labelpb.ZLabelSet enableCompatibilityLabel bool - tombstonesMtx sync.RWMutex - tombstones map[ulid.ULID]*tombstone.Tombstone - // Every how many posting offset entry we pool in heap memory. Default in Prometheus is 32. postingOffsetsInMemSampling int @@ -558,6 +555,10 @@ func (s *BucketStore) InitialSync(ctx context.Context) error { } } + if err := s.SyncTombstones(ctx); err != nil { + return errors.Wrap(err, "sync tombstones") + } + return nil } @@ -791,7 +792,7 @@ func blockSeries( skipChunks bool, // If true, chunks are not loaded. minTime, maxTime int64, // Series must have data in this time range to be returned. loadAggregates []storepb.Aggr, // List of aggregates to load when loading chunks. - tombstones []*tombstone.Tombstone, + tombstoneCache *tombstone.MemTombstonesCache, ) (storepb.SeriesSet, *queryStats, error) { ps, err := indexr.ExpandedPostings(ctx, matchers) if err != nil { @@ -824,6 +825,7 @@ func blockSeries( ) PostingsLoop: for _, id := range ps { + intervals := tombstoneCache.GetIntervalsByRef(id) ok, err := indexr.LoadSeriesForTime(id, &symbolizedLset, &chks, skipChunks, minTime, maxTime) if err != nil { return nil, nil, errors.Wrap(err, "read series") @@ -834,34 +836,13 @@ PostingsLoop: } s := seriesEntry{} - if err := indexr.LookupLabelsSymbols(symbolizedLset, &lset); err != nil { - return nil, nil, errors.Wrap(err, "Lookup labels symbols") - } - var tombstoneIntervals promtombstones.Intervals - for _, ts := range tombstones { - for _, matcher := range *ts.Matchers { - if val := lset.Get(matcher.Name); val != "" { - if !matcher.Matches(val) { - continue - } - if skipChunks { - continue PostingsLoop - } - tombstoneIntervals = tombstoneIntervals.Add(promtombstones.Interval{Mint: ts.MinTime, Maxt: ts.MaxTime}) - } - } - } - if !skipChunks { // Schedule loading chunks. s.refs = make([]chunks.ChunkRef, 0, len(chks)) s.chks = make([]storepb.AggrChunk, 0, len(chks)) - ChunkMetasLoop: for j, meta := range chks { - for _, it := range tombstoneIntervals { - if meta.OverlapsClosedInterval(it.Mint, it.Maxt) { - continue ChunkMetasLoop - } + if (promtombstones.Interval{Mint: meta.MinTime, Maxt: meta.MaxTime}.IsSubrange(intervals)) { + continue } // seriesEntry s is appended to res, but not at every outer loop iteration, @@ -887,6 +868,9 @@ PostingsLoop: return nil, nil, errors.Wrap(err, "exceeded chunks limit") } } + if err := indexr.LookupLabelsSymbols(symbolizedLset, &lset); err != nil { + return nil, nil, errors.Wrap(err, "Lookup labels symbols") + } s.lset = labelpb.ExtendSortedLabels(lset, extLset) res = append(res, s) @@ -1055,37 +1039,11 @@ func (s *BucketStore) Series(req *storepb.SeriesRequest, srv storepb.Store_Serie } s.mtx.RLock() - s.tombstonesMtx.RLock() - tombstones := s.tombstones - s.tombstonesMtx.RUnlock() - - // Filter tombstones by request time range. - matchedTombstones := filterTombstonesByTimeRange(tombstones, req.MinTime, req.MaxTime) - for _, bs := range s.blockSets { blockMatchers, ok := bs.labelMatchers(matchers...) if !ok { continue } - // Filter tombstones by block set external labels. - blockSetTombstones := make([]*tombstone.Tombstone, 0, len(matchedTombstones)) - for _, ts := range matchedTombstones { - matchers, ok := ts.MatchLabels(bs.labels) - if !ok { - continue - } - // If no matchers are left after removing external label matchers, - // we drop the tombstone. - if matchers == nil || len(*matchers) == 0 { - continue - } - blockSetTombstones = append(blockSetTombstones, &tombstone.Tombstone{ - Matchers: matchers, - MinTime: ts.MinTime, - MaxTime: ts.MaxTime, - ULID: ts.ULID, - }) - } blocks := bs.getFor(req.MinTime, req.MaxTime, req.MaxResolutionWindow, reqBlockMatchers) @@ -1097,26 +1055,6 @@ func (s *BucketStore) Series(req *storepb.SeriesRequest, srv storepb.Store_Serie b := b gctx := gctx - // Filter tombstones at block level by block metadata. - blockTombstones := make([]*tombstone.Tombstone, 0, len(blockSetTombstones)) - for _, ts := range blockSetTombstones { - matchers, ok := ts.MatchMeta(b.meta) - if !ok { - continue - } - // If no matchers are left after removing external label matchers, - // we drop the tombstone. - if matchers == nil || len(*matchers) == 0 { - continue - } - blockTombstones = append(blockTombstones, &tombstone.Tombstone{ - Matchers: matchers, - MinTime: ts.MinTime, - MaxTime: ts.MaxTime, - ULID: ts.ULID, - }) - } - if s.enableSeriesResponseHints { // Keep track of queried blocks. resHints.AddQueriedBlock(b.meta.ULID) @@ -1153,7 +1091,7 @@ func (s *BucketStore) Series(req *storepb.SeriesRequest, srv storepb.Store_Serie req.SkipChunks, req.MinTime, req.MaxTime, req.Aggregates, - blockTombstones, + b.tombstoneCache, ) if err != nil { return errors.Wrapf(err, "fetch series for block %s", b.meta.ULID) @@ -1356,7 +1294,7 @@ func (s *BucketStore) LabelNames(ctx context.Context, req *storepb.LabelNamesReq result = strutil.MergeSlices(res, extRes) } else { - seriesSet, _, err := blockSeries(newCtx, b.extLset, indexr, nil, reqSeriesMatchers, nil, seriesLimiter, true, req.Start, req.End, nil, nil) + seriesSet, _, err := blockSeries(newCtx, b.extLset, indexr, nil, reqSeriesMatchers, nil, seriesLimiter, true, req.Start, req.End, nil, b.tombstoneCache) if err != nil { return errors.Wrapf(err, "fetch series for block %s", b.meta.ULID) } @@ -1487,7 +1425,7 @@ func (s *BucketStore) LabelValues(ctx context.Context, req *storepb.LabelValuesR } result = res } else { - seriesSet, _, err := blockSeries(newCtx, b.extLset, indexr, nil, reqSeriesMatchers, nil, seriesLimiter, true, req.Start, req.End, nil, nil) + seriesSet, _, err := blockSeries(newCtx, b.extLset, indexr, nil, reqSeriesMatchers, nil, seriesLimiter, true, req.Start, req.End, nil, b.tombstoneCache) if err != nil { return errors.Wrapf(err, "fetch series for block %s", b.meta.ULID) } @@ -1545,9 +1483,76 @@ func (s *BucketStore) SyncTombstones(ctx context.Context) error { if err != nil { return err } - s.tombstonesMtx.Lock() - s.tombstones = tombstones - s.tombstonesMtx.Unlock() + + s.mtx.RLock() + defer s.mtx.RUnlock() + for _, block := range s.blocks { + for _, id := range block.tombstoneCache.GetTombstoneIDs() { + if _, ok := tombstones[id]; !ok { + block.tombstoneCache.Delete(id) + } + } + } + for _, bs := range s.blockSets { + for tid, t := range tombstones { + if _, ok := t.MatchLabels(bs.labels); !ok { + continue + } + blocks := bs.getFor(t.MinTime, t.MaxTime, downsample.ResLevel2, nil) + for _, block := range blocks { + matchers, ok := t.MatchMeta(block.meta) + // Impossible as we get matches blocks already. + if !ok { + continue + } + if _, ok := block.tombstoneCache.Get(tid); ok { + continue + } + + memTombstone := promtombstones.NewMemTombstones() + indexr := block.indexReader() + ps, err := indexr.ExpandedPostings(ctx, *matchers) + if err != nil { + continue + } + if len(ps) == 0 { + block.tombstoneCache.Set(tid, memTombstone) + continue + } + // Preload all series index data. + if err := indexr.PreloadSeries(ctx, ps); err != nil { + continue + } + + // Transform all series into the response types and mark their relevant chunks + // for preloading. + var ( + symbolizedLset []symbolizedLabel + chks []chunks.Meta + ) + PostingsLoop: + for _, id := range ps { + ok, err := indexr.LoadSeriesForTime(id, &symbolizedLset, &chks, false, t.MinTime, t.MaxTime) + if err != nil { + continue + } + if !ok { + // No matching chunks for this time duration, skip series. + continue + } + for _, chk := range chks { + if chk.OverlapsClosedInterval(t.MinTime, t.MaxTime) { + // Delete only until the current values and not beyond. + tmin, tmax := tombstone.ClampInterval(t.MinTime, t.MaxTime, chks[0].MinTime, chks[len(chks)-1].MaxTime) + memTombstone.AddInterval(id, promtombstones.Interval{Mint: tmin, Maxt: tmax}) + continue PostingsLoop + } + } + } + block.tombstoneCache.Set(tid, memTombstone) + } + } + } return nil } @@ -1709,6 +1714,8 @@ type bucketBlock struct { // Block's labels used by block-level matchers to filter blocks to query. These are used to select blocks using // request hints' BlockMatchers. relabelLabels labels.Labels + + tombstoneCache *tombstone.MemTombstonesCache } func newBucketBlock( @@ -1740,6 +1747,7 @@ func newBucketBlock( Name: block.BlockIDLabel, Value: meta.ULID.String(), }), + tombstoneCache: tombstone.NewMemTombstoneCache(), } sort.Sort(b.extLset) sort.Sort(b.relabelLabels) @@ -2841,22 +2849,3 @@ func (s queryStats) merge(o *queryStats) *queryStats { func NewDefaultChunkBytesPool(maxChunkPoolBytes uint64) (pool.Bytes, error) { return pool.NewBucketedBytes(chunkBytesPoolMinSize, chunkBytesPoolMaxSize, 2, maxChunkPoolBytes) } - -// filterTombstonesByTimeRange filters tombstones map by time range and returns a -// list of tomebstones sorted by min time. -func filterTombstonesByTimeRange(tombstones map[ulid.ULID]*tombstone.Tombstone, mint, maxt int64) []*tombstone.Tombstone { - matchedTombstones := make([]*tombstone.Tombstone, 0, len(tombstones)) - for _, ts := range tombstones { - if !ts.OverlapsClosedInterval(mint, maxt) { - continue - } - matchedTombstones = append(matchedTombstones, ts) - } - sort.Slice(matchedTombstones, func(i, j int) bool { - if matchedTombstones[i].MinTime == matchedTombstones[j].MinTime { - return matchedTombstones[i].MaxTime < matchedTombstones[j].MaxTime - } - return matchedTombstones[i].MinTime < matchedTombstones[j].MinTime - }) - return matchedTombstones -} diff --git a/pkg/store/bucket_test.go b/pkg/store/bucket_test.go index 618174899a..acfdf77b52 100644 --- a/pkg/store/bucket_test.go +++ b/pkg/store/bucket_test.go @@ -2327,92 +2327,3 @@ func BenchmarkDownsampledBlockSeries(b *testing.B) { } } } - -func TestFilterTombstonesByTimeRange(t *testing.T) { - ulid1 := ulid.MustNew(0, nil) - ulid2 := ulid.MustNew(5, nil) - ulid3 := ulid.MustNew(10, nil) - ulid4 := ulid.MustNew(15, nil) - ts1 := &tombstone.Tombstone{ - ULID: ulid1, - MinTime: 0, - MaxTime: 1, - } - ts2 := &tombstone.Tombstone{ - ULID: ulid2, - MinTime: 5, - MaxTime: 10, - } - ts3 := &tombstone.Tombstone{ - ULID: ulid3, - MinTime: 8, - MaxTime: 20, - } - ts4 := &tombstone.Tombstone{ - ULID: ulid4, - MinTime: 11, - MaxTime: 20, - } - for _, tcase := range []struct { - name string - tombstones map[ulid.ULID]*tombstone.Tombstone - mint int64 - maxt int64 - expected []*tombstone.Tombstone - }{ - { - name: "empty tombstones", - tombstones: map[ulid.ULID]*tombstone.Tombstone{}, - mint: 0, - maxt: 10, - expected: []*tombstone.Tombstone{}, - }, - { - name: "one tombstone in timeperiod", - tombstones: map[ulid.ULID]*tombstone.Tombstone{ - ulid1: ts1, - }, - mint: 0, - maxt: 10, - expected: []*tombstone.Tombstone{ts1}, - }, - { - name: "multiple tombstones in timeperiod", - tombstones: map[ulid.ULID]*tombstone.Tombstone{ - ulid1: ts1, - ulid2: ts2, - }, - mint: 0, - maxt: 10, - expected: []*tombstone.Tombstone{ts1, ts2}, - }, - { - name: "multiple tombstones overlapped in timeperiod", - tombstones: map[ulid.ULID]*tombstone.Tombstone{ - ulid1: ts1, - ulid2: ts2, - ulid3: ts3, - }, - mint: 0, - maxt: 10, - expected: []*tombstone.Tombstone{ts1, ts2, ts3}, - }, - { - name: "ulid4 not in period", - tombstones: map[ulid.ULID]*tombstone.Tombstone{ - ulid1: ts1, - ulid2: ts2, - ulid3: ts3, - ulid4: ts4, - }, - mint: 0, - maxt: 10, - expected: []*tombstone.Tombstone{ts1, ts2, ts3}, - }, - } { - t.Run(tcase.name, func(t *testing.T) { - got := filterTombstonesByTimeRange(tcase.tombstones, tcase.mint, tcase.maxt) - testutil.Equals(t, tcase.expected, got) - }) - } -} diff --git a/pkg/tombstone/cache.go b/pkg/tombstone/cache.go new file mode 100644 index 0000000000..d6cbbd4d73 --- /dev/null +++ b/pkg/tombstone/cache.go @@ -0,0 +1,72 @@ +// Copyright (c) The Thanos Authors. +// Licensed under the Apache License 2.0. + +package tombstone + +import ( + "sync" + + "github.com/oklog/ulid" + "github.com/prometheus/prometheus/storage" + "github.com/prometheus/prometheus/tsdb/tombstones" +) + +// MemTombstonesCache is a cache for the MemTombstones associated with each global tombstone file. +type MemTombstonesCache struct { + mtx sync.RWMutex + // tombstone cache: global tombstone ID -> MemTombstones. + tombstones map[ulid.ULID]*tombstones.MemTombstones +} + +// NewMemTombstoneCache initializes the MemTombstonesCache. +func NewMemTombstoneCache() *MemTombstonesCache { + return &MemTombstonesCache{tombstones: make(map[ulid.ULID]*tombstones.MemTombstones)} +} + +// Get gets the MemTombstones from cache. +func (m *MemTombstonesCache) Get(id ulid.ULID) (*tombstones.MemTombstones, bool) { + m.mtx.RLock() + defer m.mtx.RUnlock() + t, exists := m.tombstones[id] + return t, exists +} + +// GetTombstoneIDs returns a list of tombstone IDs. +func (m *MemTombstonesCache) GetTombstoneIDs() []ulid.ULID { + m.mtx.RLock() + defer m.mtx.RUnlock() + res := make([]ulid.ULID, 0, len(m.tombstones)) + for id := range m.tombstones { + res = append(res, id) + } + return res +} + +// Set sets tombstone by ID. +func (m *MemTombstonesCache) Set(id ulid.ULID, tombstone *tombstones.MemTombstones) { + m.mtx.Lock() + defer m.mtx.Unlock() + m.tombstones[id] = tombstone +} + +// Delete deletes the tombstone by ID. +func (m *MemTombstonesCache) Delete(id ulid.ULID) { + m.mtx.Lock() + defer m.mtx.Unlock() + delete(m.tombstones, id) +} + +// GetIntervalsByRef returns a list of merged tombstone intervals by given series refID. +func (m *MemTombstonesCache) GetIntervalsByRef(ref storage.SeriesRef) tombstones.Intervals { + m.mtx.RLock() + defer m.mtx.RUnlock() + var intervals tombstones.Intervals + for _, tombstone := range m.tombstones { + // MemTombstone always return nil error. + ivs, _ := tombstone.Get(ref) + for _, iv := range ivs { + intervals = intervals.Add(iv) + } + } + return intervals +} diff --git a/pkg/tombstone/tombstone.go b/pkg/tombstone/tombstone.go index b7df3e0d5d..3946c4dae1 100644 --- a/pkg/tombstone/tombstone.go +++ b/pkg/tombstone/tombstone.go @@ -7,15 +7,14 @@ import ( "bytes" "context" "encoding/json" + "github.com/go-kit/log" + "github.com/oklog/ulid" + "github.com/prometheus/prometheus/model/labels" "io" "os" "path" "path/filepath" - "github.com/go-kit/log" - "github.com/oklog/ulid" - "github.com/prometheus/prometheus/model/labels" - "github.com/thanos-io/thanos/pkg/block" "github.com/thanos-io/thanos/pkg/block/metadata" "github.com/thanos-io/thanos/pkg/objstore" @@ -131,3 +130,13 @@ func (t *Tombstone) MatchLabels(lbls labels.Labels) (*metadata.Matchers, bool) { } return &matchers, true } + +func ClampInterval(a, b, mint, maxt int64) (int64, int64) { + if a < mint { + a = mint + } + if b > maxt { + b = maxt + } + return a, b +} From 310f43688102b7a5e5abc85a19ee452e46b2865d Mon Sep 17 00:00:00 2001 From: Ben Ye Date: Mon, 20 Jun 2022 10:28:28 -0700 Subject: [PATCH 13/14] update Signed-off-by: Ben Ye --- pkg/store/bucket.go | 88 +++++++++++++++++++++------------------------ 1 file changed, 41 insertions(+), 47 deletions(-) diff --git a/pkg/store/bucket.go b/pkg/store/bucket.go index b59d2d0fd4..0c2c62bb30 100644 --- a/pkg/store/bucket.go +++ b/pkg/store/bucket.go @@ -1492,65 +1492,59 @@ func (s *BucketStore) SyncTombstones(ctx context.Context) error { block.tombstoneCache.Delete(id) } } - } - for _, bs := range s.blockSets { for tid, t := range tombstones { - if _, ok := t.MatchLabels(bs.labels); !ok { + matchers, ok := t.MatchMeta(block.meta) + // Impossible as we get matches blocks already. + if !ok { + continue + } + if _, ok := block.tombstoneCache.Get(tid); ok { + continue + } + memTombstone := promtombstones.NewMemTombstones() + indexr := block.indexReader() + ps, err := indexr.ExpandedPostings(ctx, *matchers) + if err != nil { + level.Error(s.logger).Log("msg", "failed to expand matching posting", "block", block.meta.ULID.String(), "tombstone", tid) + continue + } + if len(ps) == 0 { + block.tombstoneCache.Set(tid, memTombstone) + continue + } + // Preload all series index data. + if err := indexr.PreloadSeries(ctx, ps); err != nil { + level.Error(s.logger).Log("msg", "failed to preload series", "block", block.meta.ULID.String(), "tombstone", tid) continue } - blocks := bs.getFor(t.MinTime, t.MaxTime, downsample.ResLevel2, nil) - for _, block := range blocks { - matchers, ok := t.MatchMeta(block.meta) - // Impossible as we get matches blocks already. - if !ok { - continue - } - if _, ok := block.tombstoneCache.Get(tid); ok { - continue - } - memTombstone := promtombstones.NewMemTombstones() - indexr := block.indexReader() - ps, err := indexr.ExpandedPostings(ctx, *matchers) + // Transform all series into the response types and mark their relevant chunks + // for preloading. + var ( + symbolizedLset []symbolizedLabel + chks []chunks.Meta + ) + PostingsLoop: + for _, id := range ps { + ok, err := indexr.LoadSeriesForTime(id, &symbolizedLset, &chks, false, t.MinTime, t.MaxTime) if err != nil { + level.Error(s.logger).Log("msg", "failed to load series for posting", "block", block.meta.ULID.String(), "tombstone", tid, "posting", id) continue } - if len(ps) == 0 { - block.tombstoneCache.Set(tid, memTombstone) - continue - } - // Preload all series index data. - if err := indexr.PreloadSeries(ctx, ps); err != nil { + if !ok { + // No matching chunks for this time duration, skip series. continue } - - // Transform all series into the response types and mark their relevant chunks - // for preloading. - var ( - symbolizedLset []symbolizedLabel - chks []chunks.Meta - ) - PostingsLoop: - for _, id := range ps { - ok, err := indexr.LoadSeriesForTime(id, &symbolizedLset, &chks, false, t.MinTime, t.MaxTime) - if err != nil { - continue - } - if !ok { - // No matching chunks for this time duration, skip series. - continue - } - for _, chk := range chks { - if chk.OverlapsClosedInterval(t.MinTime, t.MaxTime) { - // Delete only until the current values and not beyond. - tmin, tmax := tombstone.ClampInterval(t.MinTime, t.MaxTime, chks[0].MinTime, chks[len(chks)-1].MaxTime) - memTombstone.AddInterval(id, promtombstones.Interval{Mint: tmin, Maxt: tmax}) - continue PostingsLoop - } + for _, chk := range chks { + if chk.OverlapsClosedInterval(t.MinTime, t.MaxTime) { + // Delete only until the current values and not beyond. + tmin, tmax := tombstone.ClampInterval(t.MinTime, t.MaxTime, chks[0].MinTime, chks[len(chks)-1].MaxTime) + memTombstone.AddInterval(id, promtombstones.Interval{Mint: tmin, Maxt: tmax}) + continue PostingsLoop } } - block.tombstoneCache.Set(tid, memTombstone) } + block.tombstoneCache.Set(tid, memTombstone) } } return nil From ed70ccaaceef25ff80d98600b1fda936a3f80da7 Mon Sep 17 00:00:00 2001 From: Ben Ye Date: Mon, 20 Jun 2022 16:21:40 -0700 Subject: [PATCH 14/14] add more functions Signed-off-by: Ben Ye --- pkg/tombstone/cache.go | 26 +++++++++++++++++++++++++- pkg/tombstone/tombstone.go | 30 ++++++++++++++++++------------ 2 files changed, 43 insertions(+), 13 deletions(-) diff --git a/pkg/tombstone/cache.go b/pkg/tombstone/cache.go index d6cbbd4d73..528e538a3f 100644 --- a/pkg/tombstone/cache.go +++ b/pkg/tombstone/cache.go @@ -56,7 +56,7 @@ func (m *MemTombstonesCache) Delete(id ulid.ULID) { delete(m.tombstones, id) } -// GetIntervalsByRef returns a list of merged tombstone intervals by given series refID. +// GetIntervalsByRef returns a list of merged tombstone intervals by given series ref. func (m *MemTombstonesCache) GetIntervalsByRef(ref storage.SeriesRef) tombstones.Intervals { m.mtx.RLock() defer m.mtx.RUnlock() @@ -70,3 +70,27 @@ func (m *MemTombstonesCache) GetIntervalsByRef(ref storage.SeriesRef) tombstones } return intervals } + +// MergeTombstones merges multiple in-memory tombstones into one in-memory tombstone. +func (m *MemTombstonesCache) MergeTombstones() *tombstones.MemTombstones { + stones := tombstones.NewMemTombstones() + m.mtx.RLock() + defer m.mtx.RUnlock() + if len(m.tombstones) == 0 { + return stones + } + if len(m.tombstones) == 1 { + for _, t := range m.tombstones { + return t + } + } + for _, ts := range m.tombstones { + ts.Iter(func(id storage.SeriesRef, ivs tombstones.Intervals) error { + for _, iv := range ivs { + stones.AddInterval(id, iv) + } + return nil + }) + } + return stones +} diff --git a/pkg/tombstone/tombstone.go b/pkg/tombstone/tombstone.go index 3946c4dae1..5eed5e4f8f 100644 --- a/pkg/tombstone/tombstone.go +++ b/pkg/tombstone/tombstone.go @@ -10,6 +10,7 @@ import ( "github.com/go-kit/log" "github.com/oklog/ulid" "github.com/prometheus/prometheus/model/labels" + "github.com/thanos-io/thanos/pkg/objstore" "io" "os" "path" @@ -17,7 +18,6 @@ import ( "github.com/thanos-io/thanos/pkg/block" "github.com/thanos-io/thanos/pkg/block/metadata" - "github.com/thanos-io/thanos/pkg/objstore" "github.com/thanos-io/thanos/pkg/runutil" ) @@ -90,17 +90,6 @@ func (t *Tombstone) Write(w io.Writer) error { return enc.Encode(&t) } -// UploadTombstone uploads the given tombstone to object storage. -func UploadTombstone(ctx context.Context, tombstone *Tombstone, bkt objstore.Bucket) error { - b, err := json.Marshal(tombstone) - if err != nil { - return err - } - - tsPath := path.Join(TombstoneDir, tombstone.ULID.String()+".json") - return bkt.Upload(ctx, tsPath, bytes.NewBuffer(b)) -} - // OverlapsClosedInterval Returns true if the chunk overlaps [mint, maxt]. func (t *Tombstone) OverlapsClosedInterval(mint, maxt int64) bool { return t.MinTime <= maxt && mint <= t.MaxTime @@ -131,6 +120,23 @@ func (t *Tombstone) MatchLabels(lbls labels.Labels) (*metadata.Matchers, bool) { return &matchers, true } +// UploadTombstone uploads the given tombstone to object storage. +func UploadTombstone(ctx context.Context, tombstone *Tombstone, bkt objstore.Bucket) error { + b, err := json.Marshal(tombstone) + if err != nil { + return err + } + + tsPath := path.Join(TombstoneDir, tombstone.ULID.String()+".json") + return bkt.Upload(ctx, tsPath, bytes.NewBuffer(b)) +} + +// RemoveTombstone removes the tombstone from object storage +func RemoveTombstone(ctx context.Context, ulid ulid.ULID, bkt objstore.Bucket) error { + tsPath := path.Join(TombstoneDir, ulid.String()+".json") + return bkt.Delete(ctx, tsPath) +} + func ClampInterval(a, b, mint, maxt int64) (int64, int64) { if a < mint { a = mint